From bf5f311b710a46d17aadd4a5aaeade17a4ef3b04 Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Thu, 13 Mar 2025 17:48:44 +0200 Subject: [PATCH 01/22] Extract Cursor class and update javadoc style --- .../db/tries/CollectionMergeTrie.java | 217 +++--- .../org/apache/cassandra/db/tries/Cursor.java | 257 +++++++ .../cassandra/db/tries/CursorWalkable.java | 27 + .../cassandra/db/tries/InMemoryReadTrie.java | 214 +++--- .../cassandra/db/tries/InMemoryTrie.java | 637 +++++++--------- .../apache/cassandra/db/tries/MergeTrie.java | 50 +- .../cassandra/db/tries/PrefixedTrie.java | 26 +- .../cassandra/db/tries/SingletonCursor.java | 134 ++++ .../cassandra/db/tries/SingletonTrie.java | 147 ---- .../apache/cassandra/db/tries/SlicedTrie.java | 60 +- .../org/apache/cassandra/db/tries/Trie.java | 691 +++++------------- .../apache/cassandra/db/tries/TrieDumper.java | 6 +- .../db/tries/TrieEntriesIterator.java | 14 +- .../cassandra/db/tries/TrieEntriesWalker.java | 12 +- .../db/tries/TriePathReconstructor.java | 2 +- .../cassandra/db/tries/TrieTailsIterator.java | 22 +- .../db/tries/TrieValuesIterator.java | 12 +- .../db/tries/InMemoryTrieTestBase.java | 8 +- .../cassandra/db/tries/SlicedTrieTest.java | 4 +- 19 files changed, 1125 insertions(+), 1415 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/tries/Cursor.java create mode 100644 src/java/org/apache/cassandra/db/tries/CursorWalkable.java create mode 100644 src/java/org/apache/cassandra/db/tries/SingletonCursor.java delete mode 100644 src/java/org/apache/cassandra/db/tries/SingletonTrie.java diff --git a/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java b/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java index 04f732627f4a..0bc20f1cb809 100644 --- a/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java +++ b/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java @@ -17,28 +17,26 @@ */ package org.apache.cassandra.db.tries; +import com.google.common.collect.Iterables; + import java.util.ArrayList; import java.util.Collection; import java.util.List; - -import com.google.common.collect.Iterables; - +import java.util.function.BiFunction; import org.apache.cassandra.utils.bytecomparable.ByteComparable; -/** - * A merged view of multiple tries. - * - * This is accomplished by walking the cursors in parallel; the merged cursor takes the position and features of the - * smallest and advances with it; when multiple cursors are equal, all of them are advanced. The ordered view of the - * cursors is maintained using a custom binary min-heap, built for efficiently reforming the heap when the top elements - * are advanced (see {@link CollectionMergeCursor}). - * - * Crucial for the efficiency of this is the fact that when they are advanced like this, we can compare cursors' - * positions by their depth descending and then incomingTransition ascending. - * - * See Trie.md for further details. - */ -class CollectionMergeTrie extends Trie +/// A merged view of multiple tries. +/// +/// This is accomplished by walking the cursors in parallel; the merged cursor takes the position and features of the +/// smallest and advances with it; when multiple cursors are equal, all of them are advanced. The ordered view of the +/// cursors is maintained using a custom binary min-heap, built for efficiently reforming the heap when the top elements +/// are advanced (see [CollectionMergeCursor]). +/// +/// Crucial for the efficiency of this is the fact that when they are advanced like this, we can compare cursors' +/// positions by their depth descending and then incomingTransition ascending. +/// +/// See [Trie.md](./Trie.md) for further details. +class CollectionMergeTrie implements Trie { private final CollectionMergeResolver resolver; // only called on more than one input protected final Collection> inputs; @@ -50,7 +48,7 @@ class CollectionMergeTrie extends Trie } @Override - protected Cursor cursor(Direction direction) + public Cursor cursor(Direction direction) { return new CollectionMergeCursor<>(resolver, direction, inputs); } @@ -74,69 +72,61 @@ static boolean equalCursor(Cursor c1, Cursor c2) return c1.depth() == c2.depth() && c1.incomingTransition() == c2.incomingTransition(); } - /* - * The merge cursor is a variation of the idea of a merge iterator with one key observation: because we advance - * the source iterators together, we can compare them just by depth and incoming transition. - * - * The most straightforward way to implement merging of iterators is to use a {@code PriorityQueue}, - * {@code poll} it to find the next item to consume, then {@code add} the iterator back after advancing. - * This is not very efficient as {@code poll} and {@code add} in all cases require at least - * {@code log(size)} comparisons and swaps (usually more than {@code 2*log(size)}) per consumed item, even - * if the input is suitable for fast iteration. - * - * The implementation below makes use of the fact that replacing the top element in a binary heap can be - * done much more efficiently than separately removing it and placing it back, especially in the cases where - * the top iterator is to be used again very soon (e.g. when there are large sections of the output where - * only a limited number of input iterators overlap, which is normally the case in many practically useful - * situations, e.g. levelled compaction). - * - * The implementation builds and maintains a binary heap of sources (stored in an array), where we do not - * add items after the initial construction. Instead we advance the smallest element (which is at the top - * of the heap) and push it down to find its place for its new position. Should this source be exhausted, - * we swap it with the last source in the heap and proceed by pushing that down in the heap. - * - * In the case where we have multiple sources with matching positions, the merging algorithm - * must be able to merge all equal values. To achieve this {@code content} walks the heap to - * find all equal cursors without advancing them, and separately {@code advance} advances - * all equal sources and restores the heap structure. - * - * The latter is done equivalently to the process of initial construction of a min-heap using back-to-front - * heapification as done in the classic heapsort algorithm. It only needs to heapify subheaps whose top item - * is advanced (i.e. one whose position matches the current), and we can do that recursively from - * bottom to top. Should a source be exhausted when advancing, it can be thrown away by swapping in the last - * source in the heap (note: we must be careful to advance that source too if required). - * - * To make it easier to advance efficienty in single-sourced branches of tries, we extract the current smallest - * cursor (the head) separately, and start any advance with comparing that to the heap's first. When the smallest - * cursor remains the same (e.g. in branches coming from a single source) this makes it possible to advance with - * just one comparison instead of two at the expense of increasing the number by one in the general case. - * - * Note: This is a simplification of the MergeIterator code from CASSANDRA-8915, without the leading ordered - * section and equalParent flag since comparisons of cursor positions are cheap. - */ + /// The merge cursor is a variation of the idea of a merge iterator with one key observation: because we advance + /// the source iterators together, we can compare them just by depth and incoming transition. + /// + /// The most straightforward way to implement merging of iterators is to use a `PriorityQueue`, + /// `poll` it to find the next item to consume, then `add` the iterator back after advancing. + /// This is not very efficient as `poll` and `add` in all cases require at least + /// `log(size)` comparisons and swaps (usually more than `2*log(size)`) per consumed item, even + /// if the input is suitable for fast iteration. + /// + /// The implementation below makes use of the fact that replacing the top element in a binary heap can be + /// done much more efficiently than separately removing it and placing it back, especially in the cases where + /// the top iterator is to be used again very soon (e.g. when there are large sections of the output where + /// only a limited number of input iterators overlap, which is normally the case in many practically useful + /// situations, e.g. levelled compaction). + /// + /// The implementation builds and maintains a binary heap of sources (stored in an array), where we do not + /// add items after the initial construction. Instead we advance the smallest element (which is at the top + /// of the heap) and push it down to find its place for its new position. Should this source be exhausted, + /// we swap it with the last source in the heap and proceed by pushing that down in the heap. + /// + /// In the case where we have multiple sources with matching positions, the merging algorithm + /// must be able to merge all equal values. To achieve this `content` walks the heap to + /// find all equal cursors without advancing them, and separately `advance` advances + /// all equal sources and restores the heap structure. + /// + /// The latter is done equivalently to the process of initial construction of a min-heap using back-to-front + /// heapification as done in the classic heapsort algorithm. It only needs to heapify subheaps whose top item + /// is advanced (i.e. one whose position matches the current), and we can do that recursively from + /// bottom to top. Should a source be exhausted when advancing, it can be thrown away by swapping in the last + /// source in the heap (note: we must be careful to advance that source too if required). + /// + /// To make it easier to advance efficienty in single-sourced branches of tries, we extract the current smallest + /// cursor (the head) separately, and start any advance with comparing that to the heap's first. When the smallest + /// cursor remains the same (e.g. in branches coming from a single source) this makes it possible to advance with + /// just one comparison instead of two at the expense of increasing the number by one in the general case. + /// + /// Note: This is a simplification of the MergeIterator code from CASSANDRA-8915, without the leading ordered + /// section and equalParent flag since comparisons of cursor positions are cheap. static class CollectionMergeCursor implements Cursor { private final CollectionMergeResolver resolver; private final Direction direction; - /** - * The smallest cursor, tracked separately to improve performance in single-source sections of the trie. - */ + /// The smallest cursor, tracked separately to improve performance in single-source sections of the trie. private Cursor head; - /** - * Binary heap of the remaining cursors. The smallest element is at position 0. - * Every element i is smaller than or equal to its two children, i.e. - * heap[i] <= heap[i*2 + 1] && heap[i] <= heap[i*2 + 2] - */ + /// Binary heap of the remaining cursors. The smallest element is at position 0. + /// Every element `i` is smaller than or equal to its two children, i.e. + /// ```heap[i] <= heap[i*2 + 1] && heap[i] <= heap[i*2 + 2]``` private final Cursor[] heap; - /** - * A list used to collect contents during content() calls. - */ + /// A list used to collect contents during [#content()] calls. private final List contents; - public CollectionMergeCursor(CollectionMergeResolver resolver, Direction direction, Collection> inputs) + private CollectionMergeCursor(CollectionMergeResolver resolver, Direction direction, Collection inputs, BiFunction> extractor) { this.resolver = resolver; this.direction = direction; @@ -145,9 +135,9 @@ public CollectionMergeCursor(CollectionMergeResolver resolver, Direction dire heap = new Cursor[count - 1]; contents = new ArrayList<>(count); int i = -1; - for (Trie trie : inputs) + for (I src : inputs) { - Cursor cursor = trie.cursor(direction); + Cursor cursor = extractor.apply(src, direction); assert cursor.depth() == 0; if (i >= 0) heap[i] = cursor; @@ -158,9 +148,12 @@ public CollectionMergeCursor(CollectionMergeResolver resolver, Direction dire // The cursors are all currently positioned on the root and thus in valid heap order. } - /** - * Interface for internal operations that can be applied to selected top elements of the heap. - */ + public CollectionMergeCursor(CollectionMergeResolver resolver, Direction direction, Collection> inputs) + { + this(resolver, direction, inputs, Trie::cursor); + } + + /// Interface for internal operations that can be applied to selected top elements of the heap. interface HeapOp { void apply(CollectionMergeCursor self, Cursor cursor, int index); @@ -171,20 +164,16 @@ default boolean shouldContinueWithChild(Cursor child, Cursor head) } } - /** - * Apply a non-interfering operation, i.e. one that does not change the cursor state, to all inputs in the heap - * that satisfy the {@link HeapOp#shouldContinueWithChild} condition (by default, being equal to the head). - * For interfering operations like advancing the cursors, use {@link #advanceSelectedAndRestoreHeap(AdvancingHeapOp)}. - */ + /// Apply a non-interfering operation, i.e. one that does not change the cursor state, to all inputs in the heap + /// that satisfy the [HeapOp#shouldContinueWithChild] condition (by default, being equal to the head). + /// For interfering operations like advancing the cursors, use [#advanceSelectedAndRestoreHeap(AdvancingHeapOp)]. private void applyToSelectedInHeap(HeapOp action) { applyToSelectedElementsInHeap(action, 0); } - /** - * Interface for internal advancing operations that can be applied to the heap cursors. This interface provides - * the code to restore the heap structure after advancing the cursors. - */ + /// Interface for internal advancing operations that can be applied to the heap cursors. This interface provides + /// the code to restore the heap structure after advancing the cursors. interface AdvancingHeapOp extends HeapOp { void apply(Cursor cursor); @@ -203,25 +192,21 @@ default void apply(CollectionMergeCursor self, Cursor cursor, int index) } - /** - * Advance the state of all inputs in the heap that satisfy the {@link HeapOp#shouldContinueWithChild} condition - * (by default, being equal to the head) and restore the heap invariant. - */ + /// Advance the state of all inputs in the heap that satisfy the [#shouldContinueWithChild] condition + /// (by default, being equal to the head) and restore the heap invariant. private void advanceSelectedAndRestoreHeap(AdvancingHeapOp action) { applyToSelectedElementsInHeap(action, 0); } - /** - * Apply an operation to all elements on the heap that satisfy, recursively through the heap hierarchy, the - * {@code shouldContinueWithChild} condition (being equal to the head by default). Descends recursively in the - * heap structure to all selected children and applies the operation on the way back. - *

- * This operation can be something that does not change the cursor state (see {@link #content}) or an operation - * that advances the cursor to a new state, wrapped in a {@link AdvancingHeapOp} ({@link #advance} or - * {@link #skipTo}). The latter interface takes care of pushing elements down in the heap after advancing - * and restores the subheap state on return from each level of the recursion. - */ + /// Apply an operation to all elements on the heap that satisfy, recursively through the heap hierarchy, the + /// `shouldContinueWithChild` condition (being equal to the head by default). Descends recursively in the + /// heap structure to all selected children and applies the operation on the way back. + /// + /// This operation can be something that does not change the cursor state (see [#content]) or an operation + /// that advances the cursor to a new state, wrapped in a [AdvancingHeapOp] ([#advance] or + /// [#skipTo]). The latter interface takes care of pushing elements down in the heap after advancing + /// and restores the subheap state on return from each level of the recursion. private void applyToSelectedElementsInHeap(HeapOp action, int index) { if (index >= heap.length) @@ -240,10 +225,8 @@ private void applyToSelectedElementsInHeap(HeapOp action, int index) action.apply(this, item, index); } - /** - * Push the given state down in the heap from the given index until it finds its proper place among - * the subheap rooted at that position. - */ + /// Push the given state down in the heap from the given index until it finds its proper place among + /// the subheap rooted at that position. private void heapifyDown(Cursor item, int index) { while (true) @@ -263,12 +246,10 @@ private void heapifyDown(Cursor item, int index) heap[index] = item; } - /** - * Check if the head is greater than the top element in the heap, and if so, swap them and push down the new - * top until its proper place. - * @param headDepth the depth of the head cursor (as returned by e.g. advance). - * @return the new head element's depth - */ + /// Check if the head is greater than the top element in the heap, and if so, swap them and push down the new + /// top until its proper place. + /// @param headDepth the depth of the head cursor (as returned by e.g. advance). + /// @return the new head element's depth private int maybeSwapHead(int headDepth) { int heap0Depth = heap[0].depth(); @@ -399,28 +380,26 @@ private void collectContent(Cursor item, int index) } @Override - public Trie tailTrie() + public Cursor tailCursor(Direction dir) { if (!branchHasMultipleSources()) - return head.tailTrie(); + return head.tailCursor(dir); - List> inputs = new ArrayList<>(heap.length); - inputs.add(head.tailTrie()); - applyToSelectedInHeap((self, cursor, index) -> inputs.add(cursor.tailTrie())); + List> inputs = new ArrayList<>(heap.length + 1); + inputs.add(head); + applyToSelectedInHeap((self, cursor, index) -> inputs.add(cursor)); - return new CollectionMergeTrie<>(inputs, resolver); + return new CollectionMergeCursor<>(resolver, dir, inputs, Cursor::tailCursor); } } - /** - * Special instance for sources that are guaranteed distinct. The main difference is that we can form unordered - * value list by concatenating sources. - */ + /// Special instance for sources that are guaranteed distinct. The main difference is that we can form unordered + /// value list by concatenating sources. static class Distinct extends CollectionMergeTrie { Distinct(Collection> inputs) { - super(inputs, throwingResolver()); + super(inputs, Trie.throwingResolver()); } @Override diff --git a/src/java/org/apache/cassandra/db/tries/Cursor.java b/src/java/org/apache/cassandra/db/tries/Cursor.java new file mode 100644 index 000000000000..24e28d76effb --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/Cursor.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.agrona.DirectBuffer; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +/// A trie cursor. +/// +/// This is the internal representation of a trie, which enables efficient walks and basic operations (merge, +/// slice) on tries. +/// +/// The cursor represents the state of a walk over the nodes of trie. It provides three main features: +/// - the current [#depth] or descend-depth in the trie; +/// - the [#incomingTransition], i.e. the byte that was used to reach the current point; +/// - the [#content] associated with the current node, +/// +/// and provides methods for advancing to the next position. This is enough information to extract all paths, and +/// also to easily compare cursors over different tries that are advanced together. Advancing is always done in +/// order; if one imagines the set of nodes in the trie with their associated paths, a cursor may only advance from a +/// node with a lexicographically smaller path to one with bigger. The [#advance] operation moves to the immediate +/// next, it is also possible to skip over some items to a specific position ahead ([#skipTo]). +/// +/// Moving to the immediate next position in the lexicographic order is accomplished by: +/// - if the current node has children, moving to its first child; +/// - otherwise, ascend the parent chain and return the next child of the closest parent that still has any. +/// +/// As long as the trie is not exhausted, advancing always takes one step down, from the current node, or from a node +/// on the parent chain. By comparing the new depth (which `advance` also returns) with the one before the advance, +/// one can tell if the former was the case (if `newDepth == oldDepth + 1`) and how many steps up we had to take +/// (`oldDepth + 1 - newDepth`). When following a path down, the cursor will stop on all prefixes. +/// +/// When it is created the cursor is placed on the root node with `depth() = 0`, `incomingTransition() = -1`. +/// Since tries can have mappings for empty, content() can possibly be non-null. The cursor is exhausted when it +/// returns a depth of -1 (the operations that advance a cursor return the depth, and `depth()` will also +/// return -1 if queried afterwards). It is not allowed for a cursor to start in exhausted state; once a cursor is +/// exhausted, calling any of the advance methods or `tailTrie` is an error. +/// +/// For example, the following trie: +///

+///  t
+///   r
+///    e
+///     e *
+///    i
+///     e *
+///     p *
+///  w
+///   i
+///    n  *
+/// 
+/// has nodes reachable with the paths +/// `"", t, tr, tre, tree*, tri, trie*, trip*, w, wi, win*` +/// and the cursor will list them with the following `(depth, incomingTransition)` pairs: +/// `(0, -1), (1, t), (2, r), (3, e), (4, e)*, (3, i), (4, e)*, (4, p)*, (1, w), (2, i), (3, n)*` +/// +/// Because we exhaust transitions on bigger depths before we go the next transition on the smaller ones, when +/// cursors are advanced together their positions can be easily compared using only the [#depth] and +/// [#incomingTransition]: +/// - one that is higher in depth is before one that is lower; +/// - for equal depths, the one with smaller incomingTransition is first. +/// +/// If we consider walking the trie above in parallel with this: +///
+///  t
+///   r
+///    i
+///     c
+///      k *
+///  u
+///   p *
+/// 
+/// the combined iteration will proceed as follows:
+///  (0, -1)+  (0, -1)+          cursors equal, advance both
+///  (1, t)+   (1, t)+   t       cursors equal, advance both
+///  (2, r)+   (2, r)+   tr      cursors equal, advance both
+///  (3, e)+ < (3, i)    tre     cursors not equal, advance smaller (3 = 3, e < i)
+///  (4, e)+ < (3, i)    tree*   cursors not equal, advance smaller (4 > 3)
+///  (3, i)+   (3, i)+   tri     cursors equal, advance both
+///  (4, e)  > (4, c)+   tric    cursors not equal, advance smaller (4 = 4, e > c)
+///  (4, e)  > (5, k)+   trick*  cursors not equal, advance smaller (4 < 5)
+///  (4, e)+ < (1, u)    trie*   cursors not equal, advance smaller (4 > 1)
+///  (4, p)+ < (1, u)    trip*   cursors not equal, advance smaller (4 > 1)
+///  (1, w)  > (1, u)+   u       cursors not equal, advance smaller (1 = 1, w > u)
+///  (1, w)  > (2, p)+   up*     cursors not equal, advance smaller (1 < 2)
+///  (1, w)+ < (-1, -1)  w       cursors not equal, advance smaller (1 > -1)
+///  (2, i)+ < (-1, -1)  wi      cursors not equal, advance smaller (2 > -1)
+///  (3, n)+ < (-1, -1)  win*    cursors not equal, advance smaller (3 > -1)
+///  (-1, -1)  (-1, -1)          both exhasted
+///  
+/// +/// Cursors are created with a direction (forward or reverse), which specifies the order in which a node's children +/// are iterated (smaller first or larger first). Note that entries returned in reverse direction are in +/// lexicographic order for the inverted alphabet, which is not the same as being presented in reverse. For example, +/// a cursor for a trie containing "ab", "abc" and "cba", will visit the nodes in order "cba", "ab", "abc", i.e. +/// prefixes will still be reported before their descendants. +/// +/// Also see [Trie.md](./Trie.md) for further documentation. +public interface Cursor +{ + /// @return the current descend-depth; 0, if the cursor has just been created and is positioned on the root, + /// and -1, if the trie has been exhausted. + int depth(); + + /// @return the last transition taken; if positioned on the root, return -1 + int incomingTransition(); + + /// @return the content associated with the current node. This may be non-null for any presented node, including + /// the root. + T content(); + + /// Returns the direction in which this cursor is progressing. + Direction direction(); + + /// Returns the byte-comparable version that this trie uses. + ByteComparable.Version byteComparableVersion(); + + /// Advance one position to the node whose associated path is next lexicographically. + /// This can be either: + /// - descending one level to the first child of the current node, + /// - ascending to the closest parent that has remaining children, and then descending one level to its next + /// child. + /// + /// It is an error to call this after the trie has already been exhausted (i.e. when `depth() == -1`); + /// for performance reasons we won't always check this. + /// + /// @return depth (can be `prev+1` or `<=prev`), -1 means that the trie is exhausted + int advance(); + + /// Advance, descending multiple levels if the cursor can do this for the current position without extra work + /// (e.g. when positioned on a chain node in a memtable trie). If the current node does not have children this + /// is exactly the same as advance(), otherwise it may take multiple steps down (but will not necessarily, even + /// if they exist). + /// + /// Note that if any positions are skipped, their content must be null. + /// + /// This is an optional optimization; the default implementation falls back to calling advance. + /// + /// It is an error to call this after the trie has already been exhausted (i.e. when `depth() == -1`); + /// for performance reasons we won't always check this. + /// + /// @param receiver object that will receive all transitions taken except the last; + /// on ascend, or if only one step down was taken, it will not receive any + /// @return the new depth, -1 if the trie is exhausted + default int advanceMultiple(TransitionsReceiver receiver) + { + return advance(); + } + + /// Advance all the way to the next node with non-null content. + /// + /// It is an error to call this after the trie has already been exhausted (i.e. when `depth() == -1`); + /// for performance reasons we won't always check this. + /// + /// @param receiver object that will receive all taken transitions + /// @return the content, null if the trie is exhausted + default T advanceToContent(ResettingTransitionsReceiver receiver) + { + int prevDepth = depth(); + while (true) + { + int currDepth = advanceMultiple(receiver); + if (currDepth <= 0) + return null; + if (receiver != null) + { + if (currDepth <= prevDepth) + receiver.resetPathLength(currDepth - 1); + receiver.addPathByte(incomingTransition()); + } + T content = content(); + if (content != null) + return content; + prevDepth = currDepth; + } + } + + /// Advance to the specified depth and incoming transition or the first valid position that is after the specified + /// position. The inputs must be something that could be returned by a single call to [#advance] (i.e. + /// `depth` must be <= current depth + 1, and `incomingTransition` must be higher than what the + /// current state saw at the requested depth). + /// + /// @return the new depth, always <= previous depth + 1; -1 if the trie is exhausted + int skipTo(int skipDepth, int skipTransition); + + /// Descend into the cursor with the given path. + /// + /// @return True if the descent is positioned at the end of the given path, false if the trie did not have a path + /// for it. In the latter case the cursor is positioned at the first node that follows the given key in iteration + /// order. + default boolean descendAlong(ByteSource bytes) + { + int next = bytes.next(); + int depth = depth(); + while (next != ByteSource.END_OF_STREAM) + { + if (skipTo(++depth, next) != depth || incomingTransition() != next) + return false; + next = bytes.next(); + } + return true; + } + + /// Returns a tail trie, i.e. a trie whose root is the current position. Walking a tail trie will list all + /// descendants of the current position with depth adjusted by the current depth. + /// + /// It is an error to call `tailTrie` on an exhausted cursor. + /// + /// Descendants that override this class should return their specific cursor type. + Cursor tailCursor(Direction direction); + + /// Used by [#advanceMultiple] to feed the transitions taken. + interface TransitionsReceiver + { + /// Add a single byte to the path. + void addPathByte(int nextByte); + /// Add the count bytes from position pos in the given buffer. + void addPathBytes(DirectBuffer buffer, int pos, int count); + } + + /// Used by [#advanceToContent] to track the transitions and backtracking taken. + interface ResettingTransitionsReceiver extends TransitionsReceiver + { + /// Delete all bytes beyond the given length. + void resetPathLength(int newLength); + } + + /// A push interface for walking over a trie. Builds upon [TransitionsReceiver] to be given the bytes of the + /// path, and adds methods called on encountering content and completion. + /// See [TrieDumper] for an example of how this can be used, and [TrieEntriesWalker] as a base class + /// for other common usages. + interface Walker extends Cursor.ResettingTransitionsReceiver + { + /// Called when content is found. + void content(T content); + + /// Called at the completion of the walk. + R complete(); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/CursorWalkable.java b/src/java/org/apache/cassandra/db/tries/CursorWalkable.java new file mode 100644 index 000000000000..70cbfe246147 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/CursorWalkable.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.tries; + +/// Package-private interface for trie implementations, defining a method of extracting the internal cursor +/// representation of the trie. +/// +/// @param The specific type of cursor a descendant uses. +public interface CursorWalkable +{ + C cursor(Direction direction); +} diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java index ecddfd9544ef..6b6410927fc3 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java @@ -25,12 +25,10 @@ import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.apache.cassandra.utils.bytecomparable.ByteComparable; -/** - * In-memory trie built for fast modification and reads executing concurrently with writes from a single mutator thread. - * - * This class provides the read-only functionality, expanded in {@link InMemoryTrie} to writes. - */ -public class InMemoryReadTrie extends Trie +/// In-memory trie built for fast modification and reads executing concurrently with writes from a single mutator thread. +/// +/// This class provides the read-only functionality, expanded in [InMemoryTrie] to writes. +public class InMemoryReadTrie implements Trie { /* TRIE FORMAT AND NODE TYPES @@ -174,11 +172,9 @@ Cell offsets used to identify node types (by comparing them to the node 'pointer // Offset of the next pointer in a non-shared prefix node static final int PREFIX_POINTER_OFFSET = LAST_POINTER_OFFSET - PREFIX_OFFSET; - /** - * Value used as null for node pointers. - * No node can use this address (we enforce this by not allowing chain nodes to grow to position 0). - * Do not change this as the code relies there being a NONE placed in all bytes of the cell that are not set. - */ + /// Value used as null for node pointers. + /// No node can use this address (we enforce this by not allowing chain nodes to grow to position 0). + /// Do not change this as the code relies on there being a `NONE` placed in all bytes of the cell that are not set. static final int NONE = 0; volatile int root; @@ -252,9 +248,7 @@ int inBufferOffset(int pos) } - /** - * Pointer offset for a node pointer. - */ + /// Pointer offset for a node pointer. int offset(int pos) { return pos & (CELL_SIZE - 1); @@ -270,22 +264,18 @@ final int getUnsignedShortVolatile(int pos) return getBuffer(pos).getShortVolatile(inBufferOffset(pos)) & 0xFFFF; } - /** - * Following a pointer must be done using a volatile read to enforce happens-before between reading the node we - * advance to and the preparation of that node that finishes in a volatile write of the pointer that makes it - * visible. - */ + /// Following a pointer must be done using a volatile read to enforce happens-before between reading the node we + /// advance to and the preparation of that node that finishes in a volatile write of the pointer that makes it + /// visible. final int getIntVolatile(int pos) { return getBuffer(pos).getIntVolatile(inBufferOffset(pos)); } - /** - * Get the content for the given content pointer. - * - * @param id content pointer, encoded as ~index where index is the position in the content array. - * @return the current content value. - */ + /// Get the content for the given content pointer. + /// + /// @param id content pointer, encoded as ~index where index is the position in the content array. + /// @return the current content value. T getContent(int id) { int leadBit = getBufferIdx(~id, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); @@ -313,17 +303,13 @@ boolean isNullOrLeaf(int node) return node <= NONE; } - /** - * Returns the number of transitions in a chain cell entered with the given pointer. - */ + /// Returns the number of transitions in a chain cell entered with the given pointer. private int chainCellLength(int node) { return LAST_POINTER_OFFSET - offset(node); } - /** - * Get a node's child for the given transition character - */ + /// Get a node's child for the given transition character int getChild(int node, int trans) { if (isNullOrLeaf(node)) @@ -366,12 +352,10 @@ protected int followContentTransition(int node) return node; } - /** - * Advance as long as the cell pointed to by the given pointer will let you. - *

- * This is the same as getChild(node, first), except for chain nodes where it would walk the fill chain as long as - * the input source matches. - */ + /// Advance as long as the cell pointed to by the given pointer will let you. + /// + /// This is the same as `getChild(node, first)`, except for chain nodes where it would walk the fill chain as long + /// as the input source matches. int advance(int node, int first, ByteSource rest) { if (isNullOrLeaf(node)) @@ -401,9 +385,7 @@ int advance(int node, int first, ByteSource rest) } } - /** - * Get the child for the given transition character, knowing that the node is sparse - */ + /// Get the child for the given transition character, knowing that the node is sparse int getSparseChild(int node, int trans) { for (int i = 0; i < SPARSE_CHILD_COUNT; ++i) @@ -423,39 +405,31 @@ int getSparseChild(int node, int trans) return NONE; } - /** - * Given a transition, returns the corresponding index (within the node cell) of the pointer to the mid cell of - * a split node. - */ + /// Given a transition, returns the corresponding index (within the node cell) of the pointer to the mid cell of + /// a split node. int splitNodeMidIndex(int trans) { // first 2 bits of the 2-3-3 split return (trans >> 6) & 0x3; } - /** - * Given a transition, returns the corresponding index (within the mid cell) of the pointer to the tail cell of - * a split node. - */ + /// Given a transition, returns the corresponding index (within the mid cell) of the pointer to the tail cell of + /// a split node. int splitNodeTailIndex(int trans) { // second 3 bits of the 2-3-3 split return (trans >> 3) & 0x7; } - /** - * Given a transition, returns the corresponding index (within the tail cell) of the pointer to the child of - * a split node. - */ + /// Given a transition, returns the corresponding index (within the tail cell) of the pointer to the child of + /// a split node. int splitNodeChildIndex(int trans) { // third 3 bits of the 2-3-3 split return trans & 0x7; } - /** - * Get the child for the given transition character, knowing that the node is split - */ + /// Get the child for the given transition character, knowing that the node is split int getSplitChild(int node, int trans) { int mid = getSplitCellPointer(node, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT); @@ -468,9 +442,7 @@ int getSplitChild(int node, int trans) return getSplitCellPointer(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); } - /** - * Get the content for a given node - */ + /// Get the content for a given node T getNodeContent(int node) { if (isLeaf(node)) @@ -495,12 +467,10 @@ int getSplitCellPointer(int node, int childIndex, int subLevelLimit) return getIntVolatile(splitCellPointerAddress(node, childIndex, subLevelLimit)); } - /** - * Backtracking state for a cursor. - * - * To avoid allocations and pointer-chasing, the backtracking data is stored in a simple int array with - * BACKTRACK_INTS_PER_ENTRY ints for each level. - */ + /// Backtracking state for a cursor. + /// + /// To avoid allocations and pointer-chasing, the backtracking data is stored in a simple int array with + /// `BACKTRACK_INTS_PER_ENTRY` ints for each level. private static class CursorBacktrackingState { static final int BACKTRACK_INTS_PER_ENTRY = 3; @@ -534,21 +504,19 @@ int depth(int backtrackDepth) } } - /* - * Cursor implementation. - * - * InMemoryTrie cursors maintain their backtracking state in CursorBacktrackingState where they store - * information about the node to backtrack to and the transitions still left to take or attempt. - * - * This information is different for the different types of node: - * - for leaf and chain no backtracking is saved (because we know there are no further transitions) - * - for sparse we store the remainder of the order word - * - for split we store one entry per sub-level of the 2-3-3 split - * - * When the cursor is asked to advance it first checks the current node for children, and if there aren't any - * (i.e. it is positioned on a leaf node), it goes one level up the backtracking chain, where we are guaranteed to - * have a remaining child to advance to. When there's nothing to backtrack to, the trie is exhausted. - */ + /// Cursor implementation. + /// + /// `InMemoryTrie` cursors maintain their backtracking state in [CursorBacktrackingState] where they store + /// information about the node to backtrack to and the transitions still left to take or attempt. + /// + /// This information is different for the different types of node: + /// - for leaf and chain no backtracking is saved (because we know there are no further transitions) + /// - for sparse we store the remainder of the order word + /// - for split we store one entry per sub-level of the 2-3-3 split + /// + /// When the cursor is asked to advance it first checks the current node for children, and if there aren't any + /// (i.e. it is positioned on a leaf node), it goes one level up the backtracking chain, where we are guaranteed to + /// have a remaining child to advance to. When there's nothing to backtrack to, the trie is exhausted. class InMemoryCursor extends CursorBacktrackingState implements Cursor { private int currentNode; @@ -556,12 +524,18 @@ class InMemoryCursor extends CursorBacktrackingState implements Cursor private int incomingTransition; private T content; private final Direction direction; - int depth = -1; + int depth; InMemoryCursor(Direction direction) { + this(direction, root, 0, -1); + } + + InMemoryCursor(Direction direction, int root, int depth, int incomingTransition) + { + this.depth = depth - 1; this.direction = direction; - descendInto(root, -1); + descendInto(root, incomingTransition); } @Override @@ -662,10 +636,10 @@ public ByteComparable.Version byteComparableVersion() } @Override - public Trie tailTrie() + public Cursor tailCursor(Direction dir) { - assert depth >= 0 : "tailTrie called on exhausted cursor"; - return new InMemoryReadTrie<>(byteComparableVersion, buffers, contentArrays, currentFullNode); + assert depth >= 0 : "tailCursor called on exhausted cursor"; + return new InMemoryCursor(dir, currentFullNode, 0, -1); } private int exhausted() @@ -748,18 +722,16 @@ private int advanceToNextChildWithTarget(int node, int data, int transition) } } - /** - * Descend into the sub-levels of a split node. Advances to the first child and creates backtracking entries - * for the following ones. We use the bits of trans (lowest non-zero ones) to identify which sub-level an - * entry refers to. - * - * @param node The node or cell id, must have offset SPLIT_OFFSET. - * @param limit The transition limit for the current sub-level (4 for the start, 8 for the others). - * @param collected The transition bits collected from the parent chain (e.g. 0x40 after following 1 on the top - * sub-level). - * @param shift This level's bit shift (6 for start, 3 for mid and 0 for tail). - * @return the depth reached after descending. - */ + /// Descend into the sub-levels of a split node. Advances to the first child and creates backtracking entries + /// for the following ones. We use the bits of trans (lowest non-zero ones) to identify which sub-level an + /// entry refers to. + /// + /// @param node The node or cell id, must have offset `SPLIT_OFFSET`. + /// @param limit The transition limit for the current sub-level (4 for the start, 8 for the others). + /// @param collected The transition bits collected from the parent chain (e.g. 0x40 after following 1 on the top + /// sub-level). + /// @param shift This level's bit shift (6 for start, 3 for mid and 0 for tail). + /// @return the depth reached after descending. int descendInSplitSublevel(int node, int limit, int collected, int shift) { while (true) @@ -795,10 +767,8 @@ int descendInSplitSublevel(int node, int limit, int collected, int shift) } } - /** - * As above, but also makes sure that the descend selects a value at least as big as the given - * {@code minTransition}. - */ + /// As above, but also makes sure that the descent selects a value at least as big as the given + /// `minTransition`. private int descendInSplitSublevelWithTarget(int node, int limit, int collected, int shift, int minTransition) { minTransition -= collected; @@ -846,9 +816,7 @@ private int descendInSplitSublevelWithTarget(int node, int limit, int collected, } } - /** - * Backtrack to a split sub-level. The level is identified by the lowest non-0 bits in data. - */ + /// Backtrack to a split sub-level. The level is identified by the lowest non-0 bits in data. int nextValidSplitTransition(int node, int data) { // Note: This is equivalent to return advanceToSplitTransition(node, data, data) but quicker. @@ -892,10 +860,8 @@ int nextValidSplitTransition(int node, int data) SPLIT_LEVEL_SHIFT * 1); } - /** - * Backtrack to a split sub-level and advance to given transition if it fits within the sublevel. - * The level is identified by the lowest non-0 bits in data as above. - */ + /// Backtrack to a split sub-level and advance to given transition if it fits within the sublevel. + /// The level is identified by the lowest non-0 bits in data as above. private int advanceToSplitTransition(int node, int data, int skipTransition) { assert data >= 0 && data <= 0xFF; @@ -924,9 +890,7 @@ private int advanceToSplitTransition(int node, int data, int skipTransition) return descendInSplitSublevelWithTarget(node, sublevelLimit, data & sublevelMask, sublevelShift, skipTransition); } - /** - * Look for any further non-null transitions on this sub-level and, if found, add a backtracking entry. - */ + /// Look for any further non-null transitions on this sub-level and, if found, add a backtracking entry. private void maybeAddSplitBacktrack(int node, int startAfter, int limit, int collected, int shift) { int nextChildIndex; @@ -969,10 +933,8 @@ private int nextValidSparseTransition(int node, int data) return descendInto(child, transition); } - /** - * Prepare the sparse node order word for iteration. For forward iteration, this means just reading it. - * For reverse, we also invert the data so that the peeling code above still works. - */ + /// Prepare the sparse node order word for iteration. For forward iteration, this means just reading it. + /// For reverse, we also invert the data so that the peeling code above still works. int prepareOrderWord(int node) { int fwdState = getUnsignedShortVolatile(node + SPARSE_ORDER_OFFSET); @@ -1008,9 +970,7 @@ int prepareOrderWord(int node) } } - /** - * Returns the state which marks the exhaustion of the order word. - */ + /// Returns the state which marks the exhaustion of the order word. int exhaustedOrderWord() { return direction.select(0, 1); @@ -1106,10 +1066,8 @@ public InMemoryCursor cursor(Direction direction) Direct read methods */ - /** - * Get the content mapped by the specified key. - * Fast implementation using integer node addresses. - */ + /// Get the content mapped by the specified key. + /// Fast implementation using integer node addresses. @Override public T get(ByteComparable path) { @@ -1137,10 +1095,8 @@ public ByteComparable.Version byteComparableVersion() return byteComparableVersion; } - /** - * Override of dump to provide more detailed printout that includes the type of each node in the trie. - * We do this via a wrapping cursor that returns a content string for the type of node for every node we return. - */ + /// Override of dump to provide more detailed printout that includes the type of each node in the trie. + /// We do this via a wrapping cursor that returns a content string for the type of node for every node we return. @Override public String dump(Function contentToString) { @@ -1191,7 +1147,7 @@ public ByteComparable.Version byteComparableVersion() } @Override - public Trie tailTrie() + public Cursor tailCursor(Direction direction) { throw new AssertionError(); } @@ -1230,12 +1186,10 @@ public String content() return type; } } - return process(new TrieDumper<>(Function.identity()), new TypedNodesCursor()); + return Trie.process(new TrieDumper<>(Function.identity()), new TypedNodesCursor()); } - /** - * For use in debugging, dump info about the given node. - */ + /// For use in debugging, dump info about the given node. @SuppressWarnings("unused") String dumpNode(int node) { diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java index 0baff9db76a7..8157dc6863a3 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java @@ -38,49 +38,43 @@ import org.apache.cassandra.utils.concurrent.OpOrder; import org.github.jamm.MemoryLayoutSpecification; -/** - * In-memory trie built for fast modification and reads executing concurrently with writes from a single mutator thread. - *

- * The main method for performing writes is {@link #apply(Trie, UpsertTransformer, Predicate)} which takes a trie as - * an argument and merges it into the current trie using the methods supplied by the given {@link UpsertTransformer}, - * force copying anything below the points where the third argument returns true. - *

- * The predicate can be used to implement several forms of atomicity and consistency guarantees: - * - *

  • if the predicate is {@code nf -> false}, neither atomicity nor sequential consistency is guaranteed - readers - * can see any mixture of old and modified content - *
  • if the predicate is {@code nf -> true}, full sequential consistency will be provided, i.e. if a reader sees any - * part of a modification, it will see all of it, and all the results of all previous modifications - *
  • if the predicate is {@code nf -> nf.isBranching()} the write will be atomic, i.e. either none or all of the - * content of the merged trie will be visible by concurrent readers, but not sequentially consistent, i.e. there - * may be writes that are not visible to a reader even when they precede writes that are visible. - *
  • if the predicate is {@code nf -> (nf.content())} the write will be consistent below the identified - * point (used e.g. by Memtable to ensure partition-level consistency) - * - *

    - * Additionally, the class provides several simpler write methods for efficiency and convenience: - * - *

  • {@link #putRecursive(ByteComparable, Object, UpsertTransformer)} inserts a single value using a recursive walk. - * It cannot provide consistency (single-path writes are always atomic). This is more efficient as it stores the - * walk state in the stack rather than on the heap but can cause a {@code StackOverflowException}. - *
  • {@link #putSingleton(ByteComparable, Object, UpsertTransformer)} is a non-recursive version of the above, using - * the {@code apply} machinery. - *
  • {@link #putSingleton(ByteComparable, Object, UpsertTransformer, boolean)} uses the fourth argument to choose - * between the two methods above, where some external property can be used to decide if the keys are short enough - * to permit recursive execution. - * - *

    - * Because it uses 32-bit pointers in byte buffers, this trie has a fixed size limit of 2GB. - */ +/// In-memory trie built for fast modification and reads executing concurrently with writes from a single mutator thread. +/// +/// The main method for performing writes is [#apply(Trie,UpsertTransformer,Predicate)] which takes a trie as +/// an argument and merges it into the current trie using the methods supplied by the given [UpsertTransformer], +/// force copying anything below the points where the third argument returns true. +/// +/// +/// The predicate can be used to implement several forms of atomicity and consistency guarantees: +/// - if the predicate is `nf -> false`, neither atomicity nor sequential consistency is guaranteed - readers +/// can see any mixture of old and modified content +/// - if the predicate is `nf -> true`, full sequential consistency will be provided, i.e. if a reader sees any +/// part of a modification, it will see all of it, and all the results of all previous modifications +/// - if the predicate is `nf -> nf.isBranching()` the write will be atomic, i.e. either none or all of the +/// content of the merged trie will be visible by concurrent readers, but not sequentially consistent, i.e. there +/// may be writes that are not visible to a reader even when they precede writes that are visible. +/// - if the predicate is `nf -> (nf.content())` the write will be consistent below the identified +/// point (used e.g. by Memtable to ensure partition-level consistency) +/// +/// +/// Additionally, the class provides several simpler write methods for efficiency and convenience: +/// - [#putRecursive(ByteComparable,Object,UpsertTransformer)] inserts a single value using a recursive walk. +/// It cannot provide consistency (single-path writes are always atomic). This is more efficient as it stores the +/// walk state in the stack rather than on the heap but can cause a `StackOverflowException`. +/// - [#putSingleton(ByteComparable,Object,UpsertTransformer)] is a non-recursive version of the above, using +/// the `apply` machinery. +/// - [#putSingleton(ByteComparable,Object,UpsertTransformer,boolean)] uses the fourth argument to choose +/// between the two methods above, where some external property can be used to decide if the keys are short enough +/// to permit recursive execution. +/// +/// Because it uses 32-bit pointers in byte buffers, this trie has a fixed size limit of 2GB. public class InMemoryTrie extends InMemoryReadTrie { // See the trie format description in InMemoryReadTrie. - /** - * Trie size limit. This is not enforced, but users must check from time to time that it is not exceeded (using - * {@link #reachedAllocatedSizeThreshold()}) and start switching to a new trie if it is. - * This must be done to avoid tries growing beyond their hard 2GB size limit (due to the 32-bit pointers). - */ + /// Trie size limit. This is not enforced, but users must check from time to time that it is not exceeded (using + /// [#reachedAllocatedSizeThreshold()]) and start switching to a new trie if it is. + /// This must be done to avoid tries growing beyond their hard 2GB size limit (due to the 32-bit pointers). @VisibleForTesting static final int ALLOCATED_SIZE_THRESHOLD; static @@ -220,10 +214,8 @@ private void putByte(int pos, byte value) getBuffer(pos).putByte(inBufferOffset(pos), value); } - /** - * Allocate a new cell in the data buffers. This is called by the memory allocation strategy when it runs out of - * free cells to reuse. - */ + /// Allocate a new cell in the data buffers. This is called by the memory allocation strategy when it runs out of + /// free cells to reuse. private int allocateNewCell() throws TrieSpaceExhaustedException { // Note: If this method is modified, please run InMemoryTrieTest.testOver1GSize to verify it acts correctly @@ -246,11 +238,9 @@ private int allocateNewCell() throws TrieSpaceExhaustedException return v; } - /** - * Allocate a cell to use for storing data. This uses the memory allocation strategy to reuse cells if any are - * available, or to allocate new cells using {@link #allocateNewCell}. Because some node types rely on cells being - * filled with 0 as initial state, any cell we get through the allocator must also be cleaned. - */ + /// Allocate a cell to use for storing data. This uses the memory allocation strategy to reuse cells if any are + /// available, or to allocate new cells using [#allocateNewCell]. Because some node types rely on cells being + /// filled with 0 as initial state, any cell we get through the allocator must also be cleaned. private int allocateCell() throws TrieSpaceExhaustedException { int cell = cellAllocator.allocate(); @@ -263,10 +253,8 @@ private void recycleCell(int cell) cellAllocator.recycle(cell & -CELL_SIZE); } - /** - * Creates a copy of a given cell and marks the original for recycling. Used when a mutation needs to force-copy - * paths to ensure earlier states are still available for concurrent readers. - */ + /// Creates a copy of a given cell and marks the original for recycling. Used when a mutation needs to force-copy + /// paths to ensure earlier states are still available for concurrent readers. private int copyCell(int cell) throws TrieSpaceExhaustedException { int copy = cellAllocator.allocate(); @@ -275,10 +263,8 @@ private int copyCell(int cell) throws TrieSpaceExhaustedException return copy | (cell & (CELL_SIZE - 1)); } - /** - * Allocate a new position in the object array. Used by the memory allocation strategy to allocate a content spot - * when it runs out of recycled positions. - */ + /// Allocate a new position in the object array. Used by the memory allocation strategy to allocate a content spot + /// when it runs out of recycled positions. private int allocateNewObject() { int index = contentCount++; @@ -293,12 +279,10 @@ private int allocateNewObject() } - /** - * Add a new content value. - * - * @return A content id that can be used to reference the content, encoded as ~index where index is the - * position of the value in the content array. - */ + /// Add a new content value. + /// + /// @return A content id that can be used to reference the content, encoded as `~index` where index is the + /// position of the value in the content array. private int addContent(@Nonnull T value) throws TrieSpaceExhaustedException { Preconditions.checkNotNull(value, "Content value cannot be null"); @@ -312,12 +296,10 @@ private int addContent(@Nonnull T value) throws TrieSpaceExhaustedException return ~index; } - /** - * Change the content associated with a given content id. - * - * @param id content id, encoded as ~index where index is the position in the content array - * @param value new content value to store - */ + /// Change the content associated with a given content id. + /// + /// @param id content id, encoded as `~index` where index is the position in the content array + /// @param value new content value to store private void setContent(int id, T value) { int leadBit = getBufferIdx(~id, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); @@ -331,9 +313,7 @@ private void releaseContent(int id) objectAllocator.recycle(~id); } - /** - * Called to clean up all buffers when the trie is known to no longer be needed. - */ + /// Called to clean up all buffers when the trie is known to no longer be needed. public void discardBuffers() { if (bufferType == BufferType.ON_HEAP) @@ -392,24 +372,22 @@ private int getCopyOrAllocate(int pointerAddress, int originalChild, int offsetW // first time, and such readers must pass through reading that pointer, which forces a happens-before relationship // that extends to all values written by this thread before it. - /** - * Attach a child to the given non-content node. This may be an update for an existing branch, or a new child for - * the node. An update _is_ required (i.e. this is only called when the newChild pointer is not the same as the - * existing value). - * This method is called when the original node content must be preserved for concurrent readers (i.e. any cell to - * be modified needs to be copied first.) - * - * @param node pointer to the node to update or copy - * @param originalNode pointer to the node as it was before any updates in the current modification (i.e. apply - * call) were started. In other words, the node that is currently reachable by readers if they - * follow the same key, and which will become unreachable for new readers after this update - * completes. Used to avoid copying again if already done -- if node is already != originalNode - * (which is the case when a second or further child of a node is changed by an update), - * then node is currently not reachable and can be safely modified or completely overwritten. - * @param trans transition to modify/add - * @param newChild new child pointer - * @return pointer to the updated node - */ + /// Attach a child to the given non-content node. This may be an update for an existing branch, or a new child for + /// the node. An update _is_ required (i.e. this is only called when the `newChild` pointer is not the same as the + /// existing value). + /// This method is called when the original node content must be preserved for concurrent readers (i.e. any cell to + /// be modified needs to be copied first.) + /// + /// @param node pointer to the node to update or copy + /// @param originalNode pointer to the node as it was before any updates in the current modification (i.e. apply + /// call) were started. In other words, the node that is currently reachable by readers if they + /// follow the same key, and which will become unreachable for new readers after this update + /// completes. Used to avoid copying again if already done -- if `node` is already != `originalNode` + /// (which is the case when a second or further child of a node is changed by an update), + /// then node is currently not reachable and can be safely modified or completely overwritten. + /// @param trans transition to modify/add + /// @param newChild new child pointer + /// @return pointer to the updated node private int attachChildCopying(int node, int originalNode, int trans, int newChild) throws TrieSpaceExhaustedException { assert !isLeaf(node) : "attachChild cannot be used on content nodes."; @@ -432,15 +410,13 @@ private int attachChildCopying(int node, int originalNode, int trans, int newChi } } - /** - * Attach a child to the given node. This may be an update for an existing branch, or a new child for the node. - * An update _is_ required (i.e. this is only called when the newChild pointer is not the same as the existing value). - * - * @param node pointer to the node to update or copy - * @param trans transition to modify/add - * @param newChild new child pointer - * @return pointer to the updated node; same as node if update was in-place - */ + /// Attach a child to the given node. This may be an update for an existing branch, or a new child for the node. + /// An update _is_ required (i.e. this is only called when the newChild pointer is not the same as the existing value). + /// + /// @param node pointer to the node to update or copy + /// @param trans transition to modify/add + /// @param newChild new child pointer + /// @return pointer to the updated node; same as node if update was in-place private int attachChild(int node, int trans, int newChild) throws TrieSpaceExhaustedException { assert !isLeaf(node) : "attachChild cannot be used on content nodes."; @@ -458,9 +434,7 @@ private int attachChild(int node, int trans, int newChild) throws TrieSpaceExhau } } - /** - * Attach a child to the given split node. This may be an update for an existing branch, or a new child for the node. - */ + /// Attach a child to the given split node. This may be an update for an existing branch, or a new child for the node. private int attachChildToSplit(int node, int trans, int newChild) throws TrieSpaceExhaustedException { int midPos = splitCellPointerAddress(node, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT); @@ -493,10 +467,8 @@ private int attachChildToSplit(int node, int trans, int newChild) throws TrieSpa return node; } - /** - * Non-volatile version of attachChildToSplit. Used when the split node is not reachable yet (during the conversion - * from sparse). - */ + /// Non-volatile version of `attachChildToSplit`. Used when the split node is not reachable yet (during the conversion + /// from sparse). private int attachChildToSplitNonVolatile(int node, int trans, int newChild) throws TrieSpaceExhaustedException { assert offset(node) == SPLIT_OFFSET : "Invalid split node in trie"; @@ -511,11 +483,9 @@ private int attachChildToSplitNonVolatile(int node, int trans, int newChild) thr return node; } - /** - * Attach a child to the given split node, copying all modified content to enable atomic visibility - * of modification. - * This may be an update for an existing branch, or a new child for the node. - */ + /// Attach a child to the given split node, copying all modified content to enable atomic visibility + /// of modification. + /// This may be an update for an existing branch, or a new child for the node. private int attachChildToSplitCopying(int node, int originalNode, int trans, int newChild) throws TrieSpaceExhaustedException { if (offset(originalNode) != SPLIT_OFFSET) // includes originalNode == NONE @@ -539,9 +509,7 @@ private int attachChildToSplitCopying(int node, int originalNode, int trans, int return node; } - /** - * Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. - */ + /// Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. private int attachChildToSparse(int node, int trans, int newChild) throws TrieSpaceExhaustedException { int index; @@ -592,10 +560,8 @@ else if (existing < trans) return node; } - /** - * Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. - * Resulting node is not reachable, no volatile set needed. - */ + /// Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. + /// Resulting node is not reachable, no volatile set needed. private int attachChildToSparseCopying(int node, int originalNode, int trans, int newChild) throws TrieSpaceExhaustedException { int index; @@ -654,15 +620,13 @@ private int upgradeSparseToSplit(int node, int trans, int newChild) throws TrieS return split; } - /** - * Insert the given newIndex in the base-6 encoded order word in the correct position with respect to the ordering. - *

    - * E.g. - * - insertOrderWord(120, 3, 0) must return 1203 (decimal 48*6 + 3) - * - insertOrderWord(120, 3, 1, ptr) must return 1230 (decimal 8*36 + 3*6 + 0) - * - insertOrderWord(120, 3, 2, ptr) must return 1320 (decimal 1*216 + 3*36 + 12) - * - insertOrderWord(120, 3, 3, ptr) must return 3120 (decimal 3*216 + 48) - */ + /// Insert the given newIndex in the base-6 encoded order word in the correct position with respect to the ordering. + /// + /// E.g. + /// - `insertOrderWord(120, 3, 0)` must return 1203 (decimal 48*6 + 3) + /// - `insertOrderWord(120, 3, 1, ptr)` must return 1230 (decimal 8*36 + 3*6 + 0) + /// - `insertOrderWord(120, 3, 2, ptr)` must return 1320 (decimal 1*216 + 3*36 + 12) + /// - `insertOrderWord(120, 3, 3, ptr)` must return 3120 (decimal 3*216 + 48) private static int insertInOrderWord(int order, int newIndex, int smallerCount) { int r = 1; @@ -674,15 +638,14 @@ private static int insertInOrderWord(int order, int newIndex, int smallerCount) return tail + (head * 6 + newIndex) * r; } - /** - * Attach a child to the given chain node. This may be an update for an existing branch with different target - * address, or a second child for the node. - * This method always copies the node -- with the exception of updates that change the child of the last node in a - * chain cell with matching transition byte (which this method is not used for, see attachChild), modifications to - * chain nodes cannot be done in place, either because we introduce a new transition byte and have to convert from - * the single-transition chain type to sparse, or because we have to remap the child from the implicit node + 1 to - * something else. - */ + /// Attach a child to the given chain node. This may be an update for an existing branch with different target + /// address, or a second child for the node. + /// + /// This method always copies the node -- with the exception of updates that change the child of the last node in a + /// chain cell with matching transition byte (which this method is not used for, see [#attachChild]), modifications to + /// chain nodes cannot be done in place, either because we introduce a new transition byte and have to convert from + /// the single-transition chain type to sparse, or because we have to remap the child from the implicit node + 1 to + /// something else. private int attachChildToChain(int node, int transitionByte, int newChild) throws TrieSpaceExhaustedException { int existingByte = getUnsignedByte(node); @@ -710,9 +673,7 @@ private int attachChildToChain(int node, int transitionByte, int newChild) throw return convertChainToSparse(node, existingByte, newChild, transitionByte); } - /** - * Attach a child to the given chain node, when we are force-copying. - */ + /// Attach a child to the given chain node, when we are force-copying. private int attachChildToChainCopying(int node, int originalNode, int transitionByte, int newChild) throws TrieSpaceExhaustedException { @@ -763,9 +724,7 @@ private boolean isExpandableChain(int newChild) return newChild > 0 && newChild - 1 > NONE && newOffset > CHAIN_MIN_OFFSET && newOffset <= CHAIN_MAX_OFFSET; } - /** - * Create a sparse node with two children. - */ + /// Create a sparse node with two children. private int createSparseNode(int byte1, int child1, int byte2, int child2) throws TrieSpaceExhaustedException { assert byte1 != byte2 : "Attempted to create a sparse node with two of the same transition"; @@ -788,11 +747,9 @@ private int createSparseNode(int byte1, int child1, int byte2, int child2) throw return node; } - /** - * Creates a chain node with the single provided transition (pointing to the provided child). - * Note that to avoid creating inefficient tries with under-utilized chain nodes, this should only be called from - * {@link #expandOrCreateChainNode} and other call-sites should call {@link #expandOrCreateChainNode}. - */ + /// Creates a chain node with the single provided transition (pointing to the provided child). + /// Note that to avoid creating inefficient tries with under-utilized chain nodes, this should only be called from + /// [#expandOrCreateChainNode] and other call-sites should call [#expandOrCreateChainNode]. private int createNewChainNode(int transitionByte, int newChild) throws TrieSpaceExhaustedException { int newNode = allocateCell() + LAST_POINTER_OFFSET - 1; @@ -803,8 +760,8 @@ private int createNewChainNode(int transitionByte, int newChild) throws TrieSpac return newNode; } - /** Like {@link #createNewChainNode}, but if the new child is already a chain node and has room, expand - * it instead of creating a brand new node. */ + /// Like [#createNewChainNode], but if the new child is already a chain node and has room, expand + /// it instead of creating a brand new node. private int expandOrCreateChainNode(int transitionByte, int newChild) throws TrieSpaceExhaustedException { if (isExpandableChain(newChild)) @@ -884,22 +841,20 @@ private boolean isEmbeddedPrefixNode(int node) return getUnsignedByte(node + PREFIX_FLAGS_OFFSET) < CELL_SIZE; } - /** - * Copy the content from an existing node, if it has any, to a newly-prepared update for its child. - * - * @param existingPreContentNode pointer to the existing node before skipping over content nodes, i.e. this is - * either the same as existingPostContentNode or a pointer to a prefix or leaf node - * whose child is existingPostContentNode - * @param existingPostContentNode pointer to the existing node being updated, after any content nodes have been - * skipped and before any modification have been applied; always a non-content node - * @param updatedPostContentNode is the updated node, i.e. the node to which all relevant modifications have been - * applied; if the modifications were applied in-place, this will be the same as - * existingPostContentNode, otherwise a completely different pointer; always a non- - * content node - * @param forcedCopy whether or not we need to preserve all pre-existing data for concurrent readers - * @return a node which has the children of updatedPostContentNode combined with the content of - * existingPreContentNode - */ + /// Copy the content from an existing node, if it has any, to a newly-prepared update for its child. + /// + /// @param existingPreContentNode pointer to the existing node before skipping over content nodes, i.e. this is + /// either the same as existingPostContentNode or a pointer to a prefix or leaf node + /// whose child is `existingPostContentNode` + /// @param existingPostContentNode pointer to the existing node being updated, after any content nodes have been + /// skipped and before any modification have been applied; always a non-content node + /// @param updatedPostContentNode is the updated node, i.e. the node to which all relevant modifications have been + /// applied; if the modifications were applied in-place, this will be the same as + /// `existingPostContentNode`, otherwise a completely different pointer; always a non- + /// content node + /// @param forcedCopy whether or not we need to preserve all pre-existing data for concurrent readers + /// @return a node which has the children of updatedPostContentNode combined with the content of + /// `existingPreContentNode` private int preserveContent(int existingPreContentNode, int existingPostContentNode, int updatedPostContentNode, @@ -927,23 +882,19 @@ private int preserveContent(int existingPreContentNode, private final ApplyState applyState = new ApplyState(); - /** - * Represents the state for an {@link #apply} operation. Contains a stack of all nodes we descended through - * and used to update the nodes with any new data during ascent. - *

    - * To make this as efficient and GC-friendly as possible, we use an integer array (instead of is an object stack) - * and we reuse the same object. The latter is safe because memtable tries cannot be mutated in parallel by multiple - * writers. - */ + /// Represents the state for an [#apply] operation. Contains a stack of all nodes we descended through + /// and used to update the nodes with any new data during ascent. + /// + /// To make this as efficient and GC-friendly as possible, we use an integer array (instead of is an object stack) + /// and we reuse the same object. The latter is safe because memtable tries cannot be mutated in parallel by multiple + /// writers. private class ApplyState implements KeyProducer { int[] data = new int[16 * 5]; int currentDepth = -1; - /** - * Pointer to the existing node before skipping over content nodes, i.e. this is either the same as - * existingPostContentNode or a pointer to a prefix or leaf node whose child is existingPostContentNode. - */ + /// Pointer to the existing node before skipping over content nodes, i.e. this is either the same as + /// existingPostContentNode or a pointer to a prefix or leaf node whose child is `existingPostContentNode`. int existingPreContentNode() { return data[currentDepth * 5 + 0]; @@ -953,10 +904,8 @@ void setExistingPreContentNode(int value) data[currentDepth * 5 + 0] = value; } - /** - * Pointer to the existing node being updated, after any content nodes have been skipped and before any - * modification have been applied. Always a non-content node. - */ + /// Pointer to the existing node being updated, after any content nodes have been skipped and before any + /// modification have been applied. Always a non-content node. int existingPostContentNode() { return data[currentDepth * 5 + 1]; @@ -966,14 +915,12 @@ void setExistingPostContentNode(int value) data[currentDepth * 5 + 1] = value; } - /** - * The updated node, i.e. the node to which the relevant modifications are being applied. This will change as - * children are processed and attached to the node. After all children have been processed, this will contain - * the fully updated node (i.e. the union of existingPostContentNode and mutationNode) without any content, - * which will be processed separately and, if necessary, attached ahead of this. If the modifications were - * applied in-place, this will be the same as existingPostContentNode, otherwise a completely different - * pointer. Always a non-content node. - */ + /// The updated node, i.e. the node to which the relevant modifications are being applied. This will change as + /// children are processed and attached to the node. After all children have been processed, this will contain + /// the fully updated node (i.e. the union of `existingPostContentNode` and `mutationNode`) without any content, + /// which will be processed separately and, if necessary, attached ahead of this. If the modifications were + /// applied in-place, this will be the same as `existingPostContentNode`, otherwise a completely different + /// pointer. Always a non-content node. int updatedPostContentNode() { return data[currentDepth * 5 + 2]; @@ -983,9 +930,7 @@ void setUpdatedPostContentNode(int value) data[currentDepth * 5 + 2] = value; } - /** - * The transition we took on the way down. - */ + /// The transition we took on the way down. int transition() { return data[currentDepth * 5 + 3]; @@ -999,10 +944,8 @@ int transitionAtDepth(int stackDepth) return data[stackDepth * 5 + 3]; } - /** - * The compiled content id. Needed because we can only access a cursor's content on the way down but we can't - * attach it until we ascend from the node. - */ + /// The compiled content id. Needed because we can only access a cursor's content on the way down but we can't + /// attach it until we ascend from the node. int contentId() { return data[currentDepth * 5 + 4]; @@ -1025,9 +968,7 @@ ApplyState start() return this; } - /** - * Returns true if the depth signals mutation cursor is exhausted. - */ + /// Returns true if the depth signals mutation cursor is exhausted. boolean advanceTo(int depth, int transition, int forcedCopyDepth) throws TrieSpaceExhaustedException { while (currentDepth > Math.max(0, depth - 1)) @@ -1043,9 +984,7 @@ boolean advanceTo(int depth, int transition, int forcedCopyDepth) throws TrieSpa return false; } - /** - * Descend to a child node. Prepares a new entry in the stack for the node. - */ + /// Descend to a child node. Prepares a new entry in the stack for the node. void descend(int transition) { setTransition(transition); @@ -1117,9 +1056,7 @@ else if (forcedCopy) } } - /** - * Attach a child to the current node. - */ + /// Attach a child to the current node. private void attachChild(int transition, int child, boolean forcedCopy) throws TrieSpaceExhaustedException { int updatedPostContentNode = updatedPostContentNode(); @@ -1136,10 +1073,8 @@ else if (forcedCopy) child)); } - /** - * Apply the collected content to a node. Converts NONE to a leaf node, and adds or updates a prefix for all - * others. - */ + /// Apply the collected content to a node. Converts `NONE` to a leaf node, and adds or updates a prefix for all + /// others. private int applyContent(boolean forcedCopy) throws TrieSpaceExhaustedException { // Note: the old content id itself is already released by setContent. Here we must release any standalone @@ -1223,11 +1158,9 @@ else if (prefixWasPresent && !prefixWasEmbedded) return existingPrePrefixNode; } - /** - * After a node's children are processed, this is called to ascend from it. This means applying the collected - * content to the compiled updatedPostContentNode and creating a mapping in the parent to it (or updating if - * one already exists). - */ + /// After a node's children are processed, this is called to ascend from it. This means applying the collected + /// content to the compiled `updatedPostContentNode` and creating a mapping in the parent to it (or updating if + /// one already exists). void attachAndMoveToParentState(int forcedCopyDepth) throws TrieSpaceExhaustedException { int updatedFullNode = applyContent(currentDepth >= forcedCopyDepth); @@ -1238,9 +1171,7 @@ void attachAndMoveToParentState(int forcedCopyDepth) throws TrieSpaceExhaustedEx attachChild(transition(), updatedFullNode, currentDepth >= forcedCopyDepth); } - /** - * Ascend and update the root at the end of processing. - */ + /// Ascend and update the root at the end of processing. void attachRoot(int forcedCopyDepth) throws TrieSpaceExhaustedException { int updatedPreContentNode = applyContent(0 >= forcedCopyDepth); @@ -1301,102 +1232,82 @@ public ByteComparable.Version byteComparableVersion() public interface KeyProducer { - /** - * Get the bytes of the path leading to this node. - */ + /// Get the bytes of the path leading to this node. byte[] getBytes(); - /** - * Get the bytes of the path leading to this node from the closest ancestor whose content, after any new inserts - * have been applied, satisfies the given predicate. - * Note that the predicate is not called for the current position, because its content is not yet prepared when - * the method is being called. - */ + /// Get the bytes of the path leading to this node from the closest ancestor whose content, after any new inserts + /// have been applied, satisfies the given predicate. + /// Note that the predicate is not called for the current position, because its content is not yet prepared when + /// the method is being called. byte[] getBytes(Predicate shouldStop); ByteComparable.Version byteComparableVersion(); } - /** - * Somewhat similar to {@link Trie.MergeResolver}, this encapsulates logic to be applied whenever new content is - * being upserted into a {@link InMemoryTrie}. Unlike {@link Trie.MergeResolver}, {@link UpsertTransformer} will be - * applied no matter if there's pre-existing content for that trie key/path or not. - * - * @param The content type for this {@link InMemoryTrie}. - * @param The type of the new content being applied to this {@link InMemoryTrie}. - */ + /// Somewhat similar to [Trie.MergeResolver], this encapsulates logic to be applied whenever new content is + /// being upserted into a [InMemoryTrie]. Unlike [Trie.MergeResolver], [UpsertTransformer] will be + /// applied no matter if there's pre-existing content for that trie key/path or not. + /// + /// @param The content type for this [InMemoryTrie]. + /// @param The type of the new content being applied to this [InMemoryTrie]. public interface UpsertTransformerWithKeyProducer { - /** - * Called when there's content in the updating trie. - * - * @param existing Existing content for this key, or null if there isn't any. - * @param update The update, always non-null. - * @param keyState An interface that can be used to retrieve the path of the value being updated. - * @return The combined value to use. Cannot be null. - */ + /// Called when there's content in the updating trie. + /// + /// @param existing Existing content for this key, or null if there isn't any. + /// @param update The update, always non-null. + /// @param keyState An interface that can be used to retrieve the path of the value being updated. + /// @return The combined value to use. @Nonnull T apply(T existing, @Nonnull U update, @Nonnull KeyProducer keyState); } - /** - * Somewhat similar to {@link Trie.MergeResolver}, this encapsulates logic to be applied whenever new content is - * being upserted into a {@link InMemoryTrie}. Unlike {@link Trie.MergeResolver}, {@link UpsertTransformer} will be - * applied no matter if there's pre-existing content for that trie key/path or not. - *

    - * A version of the above that does not use a {@link KeyProducer}. - * - * @param The content type for this {@link InMemoryTrie}. - * @param The type of the new content being applied to this {@link InMemoryTrie}. - */ + /// Somewhat similar to [Trie.MergeResolver], this encapsulates logic to be applied whenever new content is + /// being upserted into a [InMemoryTrie]. Unlike [Trie.MergeResolver], [UpsertTransformer] will be + /// applied no matter if there's pre-existing content for that trie key/path or not. + /// + /// A version of the above that does not use a [KeyProducer]. + /// + /// @param The content type for this [InMemoryTrie]. + /// @param The type of the new content being applied to this [InMemoryTrie]. public interface UpsertTransformer extends UpsertTransformerWithKeyProducer { - /** - * Called when there's content in the updating trie. - * - * @param existing Existing content for this key, or null if there isn't any. - * @param update The update, always non-null. - * @return The combined value to use. Cannot be null. - */ + /// Called when there's content in the updating trie. + /// + /// @param existing Existing content for this key, or null if there isn't any. + /// @param update The update, always non-null. + /// @return The combined value to use. Cannot be null. @Nonnull T apply(T existing, @Nonnull U update); - /** - * Version of the above that also provides the path of a value being updated. - * - * @param existing Existing content for this key, or null if there isn't any. - * @param update The update, always non-null. - * @param keyState An interface that can be used to retrieve the path of the value being updated. - * @return The combined value to use. Cannot be null. - */ + /// Version of the above that also provides the path of a value being updated. + /// + /// @param existing Existing content for this key, or null if there isn't any. + /// @param update The update, always non-null. + /// @param keyState An interface that can be used to retrieve the path of the value being updated. + /// @return The combined value to use. Cannot be null. default @Nonnull T apply(T existing, @Nonnull U update, @Nonnull KeyProducer keyState) { return apply(existing, update); } } - /** - * Interface providing features of the mutating node during mutation done using {@link #apply}. - * Effectively a subset of the {@link Trie.Cursor} interface which only permits operations that are safe to - * perform before iterating the children of the mutation node to apply the branch mutation. - * - * This is mainly used as an argument to predicates that decide when to copy substructure when modifying tries, - * which enables different kinds of atomicity and consistency guarantees. - * - * See the InMemoryTrie javadoc or InMemoryTrieThreadedTest for demonstration of the typical usages and what they - * achieve. - */ + /// Interface providing features of the mutating node during mutation done using [#apply]. + /// Effectively a subset of the [Cursor] interface which only permits operations that are safe to + /// perform before iterating the children of the mutation node to apply the branch mutation. + /// + /// This is mainly used as an argument to predicates that decide when to copy substructure when modifying tries, + /// which enables different kinds of atomicity and consistency guarantees. + /// + /// See the InMemoryTrie javadoc or InMemoryTrieThreadedTest for demonstration of the typical usages and what they + /// achieve. public interface NodeFeatures { - /** - * Whether or not the node has more than one descendant. If a checker needs mutations to be atomic, they can - * return true when this becomes true. - */ + /// Whether or not the node has more than one descendant. If a checker needs mutations to be atomic, they can + /// return true when this becomes true. boolean isBranching(); - /** - * The metadata associated with the node. If readers need to see a consistent view (i.e. where older updates - * cannot be missed if a new one is presented) below some specified point (e.g. within a partition), the checker - * should return true when it identifies that point. - */ + /// The metadata associated with the node. If readers need to see a consistent view (i.e. where older updates + /// cannot be missed if a new one is presented) below some specified point (e.g. within a partition), the checker + /// should return true when it identifies that point. T content(); } @@ -1465,7 +1376,7 @@ public boolean isBranching() { // This is not very efficient, but we only currently use this option in tests. // If it's needed for production use, isBranching should be implemented in the cursor interface. - Cursor dupe = mutationCursor.tailTrie().cursor(Direction.FORWARD); + Cursor dupe = mutationCursor.tailCursor(Direction.FORWARD); int childDepth = dupe.advance(); return childDepth > 0 && dupe.skipTo(childDepth, dupe.incomingTransition() + 1) == childDepth; @@ -1478,16 +1389,14 @@ public U content() } } - /** - * Modify this trie to apply the mutation given in the form of a trie. Any content in the mutation will be resolved - * with the given function before being placed in this trie (even if there's no pre-existing content in this trie). - * @param mutation the mutation to be applied, given in the form of a trie. Note that its content can be of type - * different than the element type for this memtable trie. - * @param transformer a function applied to the potentially pre-existing value for the given key, and the new - * value. Applied even if there's no pre-existing value in the memtable trie. - * @param needsForcedCopy a predicate which decides when to fully copy a branch to provide atomicity guarantees to - * concurrent readers. See NodeFeatures for details. - */ + /// Modify this trie to apply the mutation given in the form of a trie. Any content in the mutation will be resolved + /// with the given function before being placed in this trie (even if there's no pre-existing content in this trie). + /// @param mutation the mutation to be applied, given in the form of a trie. Note that its content can be of type + /// different than the element type for this memtable trie. + /// @param transformer a function applied to the potentially pre-existing value for the given key, and the new + /// value. Applied even if there's no pre-existing value in the memtable trie. + /// @param needsForcedCopy a predicate which decides when to fully copy a branch to provide atomicity guarantees to + /// concurrent readers. See NodeFeatures for details. public void apply(Trie mutation, final UpsertTransformerWithKeyProducer transformer, final Predicate> needsForcedCopy) @@ -1510,16 +1419,14 @@ public void apply(Trie mutation, } } - /** - * Modify this trie to apply the mutation given in the form of a trie. Any content in the mutation will be resolved - * with the given function before being placed in this trie (even if there's no pre-existing content in this trie). - * @param mutation the mutation to be applied, given in the form of a trie. Note that its content can be of type - * different than the element type for this memtable trie. - * @param transformer a function applied to the potentially pre-existing value for the given key, and the new - * value. Applied even if there's no pre-existing value in the memtable trie. - * @param needsForcedCopy a predicate which decides when to fully copy a branch to provide atomicity guarantees to - * concurrent readers. See NodeFeatures for details. - */ + /// Modify this trie to apply the mutation given in the form of a trie. Any content in the mutation will be resolved + /// with the given function before being placed in this trie (even if there's no pre-existing content in this trie). + /// @param mutation the mutation to be applied, given in the form of a trie. Note that its content can be of type + /// different than the element type for this memtable trie. + /// @param transformer a function applied to the potentially pre-existing value for the given key, and the new + /// value. Applied even if there's no pre-existing value in the memtable trie. + /// @param needsForcedCopy a predicate which decides when to fully copy a branch to provide atomicity guarantees to + /// concurrent readers. See NodeFeatures for details. public void apply(Trie mutation, final UpsertTransformer transformer, final Predicate> needsForcedCopy) @@ -1528,18 +1435,16 @@ public void apply(Trie mutation, apply(mutation, (UpsertTransformerWithKeyProducer) transformer, needsForcedCopy); } - /** - * Map-like put method, using the apply machinery above which cannot run into stack overflow. When the correct - * position in the trie has been reached, the value will be resolved with the given function before being placed in - * the trie (even if there's no pre-existing content in this trie). - * @param key the trie path/key for the given value. - * @param value the value being put in the memtable trie. Note that it can be of type different than the element - * type for this memtable trie. It's up to the {@code transformer} to return the final value that will stay in - * the memtable trie. - * @param transformer a function applied to the potentially pre-existing value for the given key, and the new - * value (of a potentially different type), returning the final value that will stay in the memtable trie. Applied - * even if there's no pre-existing value in the memtable trie. - */ + /// Map-like put method, using the apply machinery above which cannot run into stack overflow. When the correct + /// position in the trie has been reached, the value will be resolved with the given function before being placed in + /// the trie (even if there's no pre-existing content in this trie). + /// @param key the trie path/key for the given value. + /// @param value the value being put in the memtable trie. Note that it can be of type different than the element + /// type for this memtable trie. It's up to the `transformer` to return the final value that will stay in + /// the memtable trie. + /// @param transformer a function applied to the potentially pre-existing value for the given key, and the new + /// value (of a potentially different type), returning the final value that will stay in the memtable trie. Applied + /// even if there's no pre-existing value in the memtable trie. public void putSingleton(ByteComparable key, R value, UpsertTransformer transformer) throws TrieSpaceExhaustedException @@ -1561,18 +1466,16 @@ public void putSingleton(ByteComparable key, putSingleton(key, value, transformer); } - /** - * Map-like put method, using a fast recursive implementation through the key bytes. May run into stack overflow if - * the trie becomes too deep. When the correct position in the trie has been reached, the value will be resolved - * with the given function before being placed in the trie (even if there's no pre-existing content in this trie). - * @param key the trie path/key for the given value. - * @param value the value being put in the memtable trie. Note that it can be of type different than the element - * type for this memtable trie. It's up to the {@code transformer} to return the final value that will stay in - * the memtable trie. - * @param transformer a function applied to the potentially pre-existing value for the given key, and the new - * value (of a potentially different type), returning the final value that will stay in the memtable trie. Applied - * even if there's no pre-existing value in the memtable trie. - */ + /// Map-like put method, using a fast recursive implementation through the key bytes. May run into stack overflow if + /// the trie becomes too deep. When the correct position in the trie has been reached, the value will be resolved + /// with the given function before being placed in the trie (even if there's no pre-existing content in this trie). + /// @param key the trie path/key for the given value. + /// @param value the value being put in the memtable trie. Note that it can be of type different than the element + /// type for this memtable trie. It's up to the `transformer` to return the final value that will stay in + /// the memtable trie. + /// @param transformer a function applied to the potentially pre-existing value for the given key, and the new + /// value (of a potentially different type), returning the final value that will stay in the memtable trie. Applied + /// even if there's no pre-existing value in the memtable trie. public void putRecursive(ByteComparable key, R value, final UpsertTransformer transformer) throws TrieSpaceExhaustedException { try @@ -1643,22 +1546,18 @@ private void abortMutation() objectAllocator.abortMutation(); } - /** - * Returns true if the allocation threshold has been reached. To be called by the the writing thread (ideally, just - * after the write completes). When this returns true, the user should switch to a new trie as soon as feasible. - *

    - * The trie expects up to 10% growth above this threshold. Any growth beyond that may be done inefficiently, and - * the trie will fail altogether when the size grows beyond 2G - 256 bytes. - */ + /// Returns true if the allocation threshold has been reached. To be called by the the writing thread (ideally, just + /// after the write completes). When this returns true, the user should switch to a new trie as soon as feasible. + /// + /// The trie expects up to 10% growth above this threshold. Any growth beyond that may be done inefficiently, and + /// the trie will fail altogether when the size grows beyond 2G - 256 bytes. public boolean reachedAllocatedSizeThreshold() { return allocatedPos >= ALLOCATED_SIZE_THRESHOLD; } - /** - * For tests only! Advance the allocation pointer (and allocate space) by this much to test behaviour close to - * full. - */ + /// For tests only! Advance the allocation pointer (and allocate space) by this much to test behaviour close to + /// full. @VisibleForTesting int advanceAllocatedPos(int wantedPos) throws TrieSpaceExhaustedException { @@ -1667,36 +1566,30 @@ int advanceAllocatedPos(int wantedPos) throws TrieSpaceExhaustedException return allocatedPos; } - /** - * For tests only! Returns the current allocation position. - */ + /// For tests only! Returns the current allocation position. @VisibleForTesting int getAllocatedPos() { return allocatedPos; } - /** - * Returns the off heap size of the memtable trie itself, not counting any space taken by referenced content, or - * any space that has been allocated but is not currently in use (e.g. recycled cells or preallocated buffer). - * The latter means we are undercounting the actual usage, but the purpose of this reporting is to decide when - * to flush out e.g. a memtable and if we include the unused space we would almost always end up flushing out - * immediately after allocating a large buffer and not having a chance to use it. Counting only used space makes it - * possible to flush out before making these large allocations. - */ + /// Returns the off heap size of the memtable trie itself, not counting any space taken by referenced content, or + /// any space that has been allocated but is not currently in use (e.g. recycled cells or preallocated buffer). + /// The latter means we are undercounting the actual usage, but the purpose of this reporting is to decide when + /// to flush out e.g. a memtable and if we include the unused space we would almost always end up flushing out + /// immediately after allocating a large buffer and not having a chance to use it. Counting only used space makes it + /// possible to flush out before making these large allocations. public long usedSizeOffHeap() { return bufferType == BufferType.ON_HEAP ? 0 : usedBufferSpace(); } - /** - * Returns the on heap size of the memtable trie itself, not counting any space taken by referenced content, or - * any space that has been allocated but is not currently in use (e.g. recycled cells or preallocated buffer). - * The latter means we are undercounting the actual usage, but the purpose of this reporting is to decide when - * to flush out e.g. a memtable and if we include the unused space we would almost always end up flushing out - * immediately after allocating a large buffer and not having a chance to use it. Counting only used space makes it - * possible to flush out before making these large allocations. - */ + /// Returns the on heap size of the memtable trie itself, not counting any space taken by referenced content, or + /// any space that has been allocated but is not currently in use (e.g. recycled cells or preallocated buffer). + /// The latter means we are undercounting the actual usage, but the purpose of this reporting is to decide when + /// to flush out e.g. a memtable and if we include the unused space we would almost always end up flushing out + /// immediately after allocating a large buffer and not having a chance to use it. Counting only used space makes it + /// possible to flush out before making these large allocations. public long usedSizeOnHeap() { return usedObjectSpace() + @@ -1715,10 +1608,8 @@ private long usedObjectSpace() return (contentCount - objectAllocator.indexCountInPipeline()) * MemoryLayoutSpecification.SPEC.getReferenceSize(); } - /** - * Returns the amount of memory that has been allocated for various buffers but isn't currently in use. - * The total on-heap space used by the trie is {@code usedSizeOnHeap() + unusedReservedOnHeapMemory()}. - */ + /// Returns the amount of memory that has been allocated for various buffers but isn't currently in use. + /// The total on-heap space used by the trie is `usedSizeOnHeap() + unusedReservedOnHeapMemory()`. @VisibleForTesting public long unusedReservedOnHeapMemory() { @@ -1743,13 +1634,11 @@ public long unusedReservedOnHeapMemory() return bufferOverhead + contentOverhead; } - /** - * Release all recycled content references, including the ones waiting in still incomplete recycling lists. - * This is a test method and can cause null pointer exceptions if used on a live trie. - *

    - * If similar functionality is required for non-test purposes, a version of this should be developed that only - * releases references on barrier-complete lists. - */ + /// Release all recycled content references, including the ones waiting in still incomplete recycling lists. + /// This is a test method and can cause null pointer exceptions if used on a live trie. + /// + /// If similar functionality is required for non-test purposes, a version of this should be developed that only + /// releases references on barrier-complete lists. @VisibleForTesting public void releaseReferencesUnsafe() { @@ -1757,9 +1646,7 @@ public void releaseReferencesUnsafe() setContent(~idx, null); } - /** - * Returns the number of values in the trie - */ + /// Returns the number of values in the trie public int valuesCount() { return contentCount; diff --git a/src/java/org/apache/cassandra/db/tries/MergeTrie.java b/src/java/org/apache/cassandra/db/tries/MergeTrie.java index ffdfee4267e8..96b250900f21 100644 --- a/src/java/org/apache/cassandra/db/tries/MergeTrie.java +++ b/src/java/org/apache/cassandra/db/tries/MergeTrie.java @@ -21,18 +21,16 @@ import org.apache.cassandra.utils.bytecomparable.ByteComparable; -/** - * A merged view of two tries. - * - * This is accomplished by walking the two cursors in parallel; the merged cursor takes the position and features of the - * smaller and advances with it; when the two cursors are equal, both are advanced. - * - * Crucial for the efficiency of this is the fact that when they are advanced like this, we can compare cursors' - * positions by their depth descending and then incomingTransition ascending. - * - * See Trie.md for further details. - */ -class MergeTrie extends Trie +/// A merged view of two tries. +/// +/// This is accomplished by walking the two cursors in parallel; the merged cursor takes the position and features of the +/// smaller and advances with it; when the two cursors are equal, both are advanced. +/// +/// Crucial for the efficiency of this is the fact that when they are advanced like this, we can compare cursors' +/// positions by their depth descending and then incomingTransition ascending. +/// +/// See [Trie.md](./Trie.md) for further details. +class MergeTrie implements Trie { private final MergeResolver resolver; protected final Trie t1; @@ -46,9 +44,9 @@ class MergeTrie extends Trie } @Override - protected Cursor cursor(Direction direction) + public Cursor cursor(Direction direction) { - return new MergeCursor<>(resolver, direction, t1, t2); + return new MergeCursor<>(resolver, t1.cursor(direction), t2.cursor(direction)); } static class MergeCursor implements Cursor @@ -61,12 +59,12 @@ static class MergeCursor implements Cursor boolean atC1; boolean atC2; - MergeCursor(MergeResolver resolver, Direction direction, Trie t1, Trie t2) + MergeCursor(MergeResolver resolver, Cursor c1, Cursor c2) { this.resolver = resolver; - this.direction = direction; - this.c1 = t1.cursor(direction); - this.c2 = t2.cursor(direction); + this.direction = c1.direction(); + this.c1 = c1; + this.c2 = c2; assert c1.depth() == 0; assert c2.depth() == 0; atC1 = atC2 = true; @@ -173,28 +171,26 @@ else if (nc == null) } @Override - public Trie tailTrie() + public Cursor tailCursor(Direction dir) { if (atC1 && atC2) - return new MergeTrie<>(resolver, c1.tailTrie(), c2.tailTrie()); + return new MergeCursor<>(resolver, c1.tailCursor(dir), c2.tailCursor(dir)); else if (atC1) - return c1.tailTrie(); + return c1.tailCursor(dir); else if (atC2) - return c2.tailTrie(); + return c2.tailCursor(dir); else throw new AssertionError(); } } - /** - * Special instance for sources that are guaranteed (by the caller) distinct. The main difference is that we can - * form unordered value list by concatenating sources. - */ + /// Special instance for sources that are guaranteed (by the caller) distinct. The main difference is that we can + /// form unordered value list by concatenating sources. static class Distinct extends MergeTrie { Distinct(Trie input1, Trie input2) { - super(throwingResolver(), input1, input2); + super(Trie.throwingResolver(), input1, input2); } @Override diff --git a/src/java/org/apache/cassandra/db/tries/PrefixedTrie.java b/src/java/org/apache/cassandra/db/tries/PrefixedTrie.java index cf5f9dd63513..0445b2fbe00f 100644 --- a/src/java/org/apache/cassandra/db/tries/PrefixedTrie.java +++ b/src/java/org/apache/cassandra/db/tries/PrefixedTrie.java @@ -21,10 +21,8 @@ import org.apache.cassandra.utils.bytecomparable.ByteComparable; import org.apache.cassandra.utils.bytecomparable.ByteSource; -/** - * Prefixed trie. Represents the content of the given trie with the prefix prepended to all keys. - */ -public class PrefixedTrie extends Trie +/// Prefixed trie. Represents the content of the given trie with the prefix prepended to all keys. +public class PrefixedTrie implements Trie { final ByteComparable prefix; final Trie trie; @@ -36,21 +34,21 @@ public PrefixedTrie(ByteComparable prefix, Trie trie) } @Override - protected Trie.Cursor cursor(Direction direction) + public Cursor cursor(Direction direction) { - Trie.Cursor sourceCursor = trie.cursor(direction); - return new Cursor<>(prefix.asComparableBytes(sourceCursor.byteComparableVersion()), sourceCursor); + Cursor sourceCursor = trie.cursor(direction); + return new PrefixedCursor<>(prefix.asComparableBytes(sourceCursor.byteComparableVersion()), sourceCursor); } - private static class Cursor implements Trie.Cursor + private static class PrefixedCursor implements Cursor { - final Trie.Cursor tail; + final Cursor tail; ByteSource prefixBytes; int nextPrefixByte; int incomingTransition; int depthOfPrefix; - Cursor(ByteSource prefix, Trie.Cursor tail) + PrefixedCursor(ByteSource prefix, Cursor tail) { this.tail = tail; prefixBytes = prefix; @@ -101,7 +99,7 @@ public int advance() } @Override - public int advanceMultiple(Trie.TransitionsReceiver receiver) + public int advanceMultiple(TransitionsReceiver receiver) { if (prefixDone()) return completeAdvanceInTail(tail.advanceMultiple(receiver)); @@ -155,10 +153,10 @@ public T content() } @Override - public Trie tailTrie() + public Cursor tailCursor(Direction direction) { if (prefixDone()) - return tail.tailTrie(); + return tail.tailCursor(direction); else { assert depthOfPrefix >= 0 : "tailTrie called on exhausted cursor"; @@ -166,7 +164,7 @@ public Trie tailTrie() prefixBytes = ByteSource.duplicatable(prefixBytes); ByteSource.Duplicatable duplicatableSource = (ByteSource.Duplicatable) prefixBytes; - return new PrefixedTrie<>(v -> duplicatableSource.duplicate(), tail.tailTrie()); + return new PrefixedCursor<>(duplicatableSource.duplicate(), tail.tailCursor(direction)); } } } diff --git a/src/java/org/apache/cassandra/db/tries/SingletonCursor.java b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java new file mode 100644 index 000000000000..10f7b57145ca --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +/// Trie cursor for a singleton trie, mapping a given key to a value. +class SingletonCursor implements Cursor +{ + private final Direction direction; + private ByteSource src; + private final ByteComparable.Version byteComparableVersion; + private final T value; + private int currentDepth = 0; + private int currentTransition = -1; + private int nextTransition; + + + public SingletonCursor(Direction direction, ByteSource src, ByteComparable.Version byteComparableVersion, T value) + { + this.src = src; + this.direction = direction; + this.byteComparableVersion = byteComparableVersion; + this.value = value; + this.nextTransition = src.next(); + } + + @Override + public int advance() + { + currentTransition = nextTransition; + if (currentTransition != ByteSource.END_OF_STREAM) + { + nextTransition = src.next(); + return ++currentDepth; + } + else + { + return currentDepth = -1; + } + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + if (nextTransition == ByteSource.END_OF_STREAM) + return currentDepth = -1; + int current = nextTransition; + int depth = currentDepth; + int next = src.next(); + while (next != ByteSource.END_OF_STREAM) + { + if (receiver != null) + receiver.addPathByte(current); + current = next; + next = src.next(); + ++depth; + } + currentTransition = current; + nextTransition = next; + return currentDepth = ++depth; + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + if (skipDepth <= currentDepth) + { + assert skipDepth < currentDepth || direction.gt(skipTransition, currentTransition); + return currentDepth = -1; // no alternatives + } + if (direction.gt(skipTransition, nextTransition)) + return currentDepth = -1; // request is skipping over our path + + return advance(); + } + + @Override + public int depth() + { + return currentDepth; + } + + @Override + public T content() + { + return nextTransition == ByteSource.END_OF_STREAM ? value : null; + } + + @Override + public int incomingTransition() + { + return currentTransition; + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return byteComparableVersion; + } + + @Override + public SingletonCursor tailCursor(Direction dir) + { + if (!(src instanceof ByteSource.Duplicatable)) + src = ByteSource.duplicatable(src); + ByteSource.Duplicatable duplicatableSource = (ByteSource.Duplicatable) src; + + return new SingletonCursor(dir, duplicatableSource.duplicate(), byteComparableVersion, value); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/SingletonTrie.java b/src/java/org/apache/cassandra/db/tries/SingletonTrie.java deleted file mode 100644 index e3eb62783ea1..000000000000 --- a/src/java/org/apache/cassandra/db/tries/SingletonTrie.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.db.tries; - -import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import org.apache.cassandra.utils.bytecomparable.ByteSource; - -/** - * Singleton trie, mapping the given key to value. - */ -class SingletonTrie extends Trie -{ - private final ByteComparable key; - private final ByteComparable.Version byteComparableVersion; - private final T value; - - SingletonTrie(ByteComparable key, ByteComparable.Version byteComparableVersion, T value) - { - this.byteComparableVersion = byteComparableVersion; - this.key = key; - this.value = value; - } - - public Cursor cursor(Direction direction) - { - return new Cursor(direction); - } - - class Cursor implements Trie.Cursor - { - private final Direction direction; - private ByteSource src = key.asComparableBytes(byteComparableVersion); - private int currentDepth = 0; - private int currentTransition = -1; - private int nextTransition = src.next(); - - public Cursor(Direction direction) - { - this.direction = direction; - } - - @Override - public int advance() - { - currentTransition = nextTransition; - if (currentTransition != ByteSource.END_OF_STREAM) - { - nextTransition = src.next(); - return ++currentDepth; - } - else - { - return currentDepth = -1; - } - } - - @Override - public int advanceMultiple(TransitionsReceiver receiver) - { - if (nextTransition == ByteSource.END_OF_STREAM) - return currentDepth = -1; - int current = nextTransition; - int depth = currentDepth; - int next = src.next(); - while (next != ByteSource.END_OF_STREAM) - { - if (receiver != null) - receiver.addPathByte(current); - current = next; - next = src.next(); - ++depth; - } - currentTransition = current; - nextTransition = next; - return currentDepth = ++depth; - } - - @Override - public int skipTo(int skipDepth, int skipTransition) - { - if (skipDepth <= currentDepth) - { - assert skipDepth < currentDepth || direction.gt(skipTransition, currentTransition); - return currentDepth = -1; // no alternatives - } - if (direction.gt(skipTransition, nextTransition)) - return currentDepth = -1; // request is skipping over our path - - return advance(); - } - - @Override - public int depth() - { - return currentDepth; - } - - @Override - public T content() - { - return nextTransition == ByteSource.END_OF_STREAM ? value : null; - } - - @Override - public int incomingTransition() - { - return currentTransition; - } - - @Override - public Direction direction() - { - return direction; - } - - @Override - public ByteComparable.Version byteComparableVersion() - { - return byteComparableVersion; - } - - @Override - public Trie tailTrie() - { - if (!(src instanceof ByteSource.Duplicatable)) - src = ByteSource.duplicatable(src); - ByteSource.Duplicatable duplicatableSource = (ByteSource.Duplicatable) src; - - return new SingletonTrie(v -> duplicatableSource.duplicate(), byteComparableVersion, value); - } - } -} diff --git a/src/java/org/apache/cassandra/db/tries/SlicedTrie.java b/src/java/org/apache/cassandra/db/tries/SlicedTrie.java index c14f0adde620..461581a755e6 100644 --- a/src/java/org/apache/cassandra/db/tries/SlicedTrie.java +++ b/src/java/org/apache/cassandra/db/tries/SlicedTrie.java @@ -39,7 +39,7 @@ * * See Trie.md for further details. */ -public class SlicedTrie extends Trie +public class SlicedTrie implements Trie { private final Trie source; @@ -73,7 +73,7 @@ static ByteSource openAndMaybeAdd0(ByteComparable key, ByteComparable.Version by } @Override - protected Cursor cursor(Direction direction) + public Cursor cursor(Direction direction) { Cursor sourceCursor = source.cursor(direction); // The cursor is left-inclusive and right-exclusive by default. If we need to change the inclusiveness, adjust @@ -106,8 +106,7 @@ protected Cursor cursor(Direction direction) leftSource, leftNext, rightSource, - rightNext, - direction); + rightNext); } String sliceString() @@ -165,11 +164,10 @@ public SlicedCursor(Cursor source, ByteSource leftSource, int leftNext, ByteSource rightSource, - int rightNext, - Direction direction) + int rightNext) { this.source = source; - this.direction = direction; + this.direction = source.direction(); start = direction.select(leftSource, rightSource); end = direction.select(rightSource, leftSource); startNext = direction.select(leftNext, rightNext); @@ -392,19 +390,19 @@ public T content() } @Override - public Trie tailTrie() + public Cursor tailCursor(Direction dir) { - final Trie sourceTail = source.tailTrie(); + final Cursor sourceTail = source.tailCursor(dir); switch (state) { case INSIDE: return sourceTail; case COMMON_PREFIX: - return makeTrie(sourceTail, duplicatableStart(), startNext, duplicatableEnd(), endNext, direction); + return makeCursor(sourceTail, duplicatableStart(), startNext, duplicatableEnd(), endNext, direction); case START_PREFIX: - return makeTrie(sourceTail, duplicatableStart(), startNext, null, -1, direction); + return makeCursor(sourceTail, duplicatableStart(), startNext, null, -1, direction); case END_PREFIX: - return makeTrie(sourceTail, null, -1, duplicatableEnd(), endNext, direction); + return makeCursor(sourceTail, null, -1, duplicatableEnd(), endNext, direction); default: throw new UnsupportedOperationException("tailTrie on a slice boundary"); } @@ -429,30 +427,22 @@ private ByteSource.Duplicatable duplicatableEnd() } - private static Trie makeTrie(Trie source, - ByteSource.Duplicatable startSource, - int startNext, - ByteSource.Duplicatable endSource, - int endNext, - Direction direction) + private static Cursor makeCursor(Cursor source, + ByteSource.Duplicatable startSource, + int startNext, + ByteSource.Duplicatable endSource, + int endNext, + Direction startAndEndDirection) { - ByteSource.Duplicatable leftSource = direction.select(startSource, endSource); - ByteSource.Duplicatable rightSource = direction.select(endSource, startSource); - int leftNext = direction.select(startNext, endNext); - int rightNext = direction.select(endNext, startNext); - return new Trie() - { - @Override - protected Cursor cursor(Direction direction) - { - return new SlicedCursor<>(source.cursor(direction), - leftSource != null ? leftSource.duplicate() : null, - leftNext, - rightSource != null ? rightSource.duplicate() : null, - rightNext, - direction); - } - }; + ByteSource.Duplicatable leftSource = startAndEndDirection.select(startSource, endSource); + ByteSource.Duplicatable rightSource = startAndEndDirection.select(endSource, startSource); + int leftNext = startAndEndDirection.select(startNext, endNext); + int rightNext = startAndEndDirection.select(endNext, startNext); + return new SlicedCursor<>(source, + leftSource != null ? leftSource.duplicate() : null, + leftNext, + rightSource != null ? rightSource.duplicate() : null, + rightNext); } } } diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java index dbe7e26699c8..19fe9828b909 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.java +++ b/src/java/org/apache/cassandra/db/tries/Trie.java @@ -17,321 +17,52 @@ */ package org.apache.cassandra.db.tries; +import com.google.common.collect.ImmutableList; + import java.util.Collection; import java.util.Iterator; import java.util.Map; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; - -import com.google.common.collect.ImmutableList; - import org.agrona.DirectBuffer; import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import org.apache.cassandra.utils.bytecomparable.ByteSource; - -/** - * Base class for tries. - *

    - * Normal users of tries will only use the public methods, which provide various transformations of the trie, conversion - * of its content to other formats (e.g. iterable of values), and several forms of processing. - *

    - * For any unimplemented data extraction operations one can build on the {@link TrieEntriesWalker} (for-each processing) - * and {@link TrieEntriesIterator} (to iterator) base classes, which provide the necessary mechanisms to handle walking - * the trie. - *

    - * The internal representation of tries using this interface is defined in the {@link Cursor} interface. - *

    - * Cursors are a method of presenting the internal structure of a trie without representing nodes as objects, which is - * still useful for performing the basic operations on tries (iteration, slicing/intersection and merging). A cursor - * will list the nodes of a trie in order, together with information about the path that was taken to reach them. - *

    - * To begin traversal over a trie, one must retrieve a cursor by calling {@link #cursor}. Because cursors are - * stateful, the traversal must always proceed from one thread. Should concurrent reads be required, separate calls to - * {@link #cursor} must be made. Any modification that has completed before the construction of a cursor must be - * visible, but any later concurrent modifications may be presented fully, partially or not at all; this also means that - * if multiple are made, the cursor may see any part of any subset of them. - *

    - * Note: This model only supports depth-first traversals. We do not currently have a need for breadth-first walks. - *

    - * See Trie.md for further description of the trie representation model. - * - * @param The content type of the trie. - */ -public abstract class Trie -{ - /** - * A trie cursor. - *

    - * This is the internal representation of the trie, which enables efficient walks and basic operations (merge, - * slice) on tries. - *

    - * The cursor represents the state of a walk over the nodes of trie. It provides three main features:

      - *
    • the current {@code depth} or descend-depth in the trie;
    • - *
    • the {@code incomingTransition}, i.e. the byte that was used to reach the current point;
    • - *
    • the {@code content} associated with the current node,
    • - *
    - * and provides methods for advancing to the next position. This is enough information to extract all paths, and - * also to easily compare cursors over different tries that are advanced together. Advancing is always done in - * order; if one imagines the set of nodes in the trie with their associated paths, a cursor may only advance from a - * node with a lexicographically smaller path to one with bigger. The {@code advance} operation moves to the immediate - * next, it is also possible to skip over some items e.g. all children of the current node ({@code skipChildren}). - *

    - * Moving to the immediate next position in the lexicographic order is accomplished by:

      - *
    • if the current node has children, moving to its first child;
    • - *
    • otherwise, ascend the parent chain and return the next child of the closest parent that still has any.
    • - *
    - * As long as the trie is not exhausted, advancing always takes one step down, from the current node, or from a node - * on the parent chain. By comparing the new depth (which {@code advance} also returns) with the one before the advance, - * one can tell if the former was the case (if {@code newDepth == oldDepth + 1}) and how many steps up we had to take - * ({@code oldDepth + 1 - newDepth}). When following a path down, the cursor will stop on all prefixes. - *

    - * When it is created the cursor is placed on the root node with {@code depth() = 0}, {@code incomingTransition() = -1}. - * Since tries can have mappings for empty, content() can possibly be non-null. The cursor is exhausted when it - * returns a depth of -1 (the operations that advance a cursor return the depth, and {@code depth()} will also - * return -1 if queried afterwards). It is not allowed for a cursor to start in exhausted state; once a cursor is - * exhausted, calling any of the advance methods or {@code tailTrie} is an error. - *

    - * For example, the following trie:
    - *

    -     *  t
    -     *   r
    -     *    e
    -     *     e *
    -     *    i
    -     *     e *
    -     *     p *
    -     *  w
    -     *   i
    -     *    n  *
    -     * 
    - * has nodes reachable with the paths
    - *   "", t, tr, tre, tree*, tri, trie*, trip*, w, wi, win*
    - * and the cursor will list them with the following {@code (depth, incomingTransition)} pairs:
    - *   (0, -1), (1, t), (2, r), (3, e), (4, e)*, (3, i), (4, e)*, (4, p)*, (1, w), (2, i), (3, n)* - *

    - * Because we exhaust transitions on bigger depths before we go the next transition on the smaller ones, when - * cursors are advanced together their positions can be easily compared using only the {@code depth} and - * {@code incomingTransition}:

      - *
    • one that is higher in depth is before one that is lower;
    • - *
    • for equal depths, the one with smaller incomingTransition is first.
    • - *
    - * If we consider walking the trie above in parallel with this:
    - *
    -     *  t
    -     *   r
    -     *    i
    -     *     c
    -     *      k *
    -     *  u
    -     *   p *
    -     * 
    - * the combined iteration will proceed as follows:
    -     *  (0, -1)+  (0, -1)+          cursors equal, advance both
    -     *  (1, t)+   (1, t)+   t       cursors equal, advance both
    -     *  (2, r)+   (2, r)+   tr      cursors equal, advance both
    -     *  (3, e)+ < (3, i)    tre     cursors not equal, advance smaller (3 = 3, e < i)
    -     *  (4, e)+ < (3, i)    tree*   cursors not equal, advance smaller (4 > 3)
    -     *  (3, i)+   (3, i)+   tri     cursors equal, advance both
    -     *  (4, e)  > (4, c)+   tric    cursors not equal, advance smaller (4 = 4, e > c)
    -     *  (4, e)  > (5, k)+   trick*  cursors not equal, advance smaller (4 < 5)
    -     *  (4, e)+ < (1, u)    trie*   cursors not equal, advance smaller (4 > 1)
    -     *  (4, p)+ < (1, u)    trip*   cursors not equal, advance smaller (4 > 1)
    -     *  (1, w)  > (1, u)+   u       cursors not equal, advance smaller (1 = 1, w > u)
    -     *  (1, w)  > (2, p)+   up*     cursors not equal, advance smaller (1 < 2)
    -     *  (1, w)+ < (-1, -1)  w       cursors not equal, advance smaller (1 > -1)
    -     *  (2, i)+ < (-1, -1)  wi      cursors not equal, advance smaller (2 > -1)
    -     *  (3, n)+ < (-1, -1)  win*    cursors not equal, advance smaller (3 > -1)
    -     *  (-1, -1)  (-1, -1)          both exhasted
    -     *  
    - *

    - * Cursors are created with a direction (forward or reverse), which specifies the order in which a node's children - * are iterated (smaller first or larger first). Note that entries returned in reverse direction are in - * lexicographic order for the inverted alphabet, which is not the same as being presented in reverse. For example, - * a cursor for a trie containing "ab", "abc" and "cba", will visit the nodes in order "cba", "ab", "abc", i.e. - * prefixes will still be reported before their descendants. - */ - protected interface Cursor - { - - /** - * @return the current descend-depth; 0, if the cursor has just been created and is positioned on the root, - * and -1, if the trie has been exhausted. - */ - int depth(); - - /** - * @return the last transition taken; if positioned on the root, return -1 - */ - int incomingTransition(); - - /** - * @return the content associated with the current node. This may be non-null for any presented node, including - * the root. - */ - T content(); - - /** - * Returns the direction in which this cursor is progressing. - */ - Direction direction(); - - /** - * Returns the byte-comparable version that this trie uses. - */ - ByteComparable.Version byteComparableVersion(); - - /** - * Advance one position to the node whose associated path is next lexicographically. - * This can be either:

      - *
    • descending one level to the first child of the current node, - *
    • ascending to the closest parent that has remaining children, and then descending one level to its next - * child. - *
    - * It is an error to call this after the trie has already been exhausted (i.e. when depth() == -1); - * for performance reasons we won't always check this. - * - * @return depth (can be prev+1 or <=prev), -1 means that the trie is exhausted - */ - int advance(); - - /** - * Advance, descending multiple levels if the cursor can do this for the current position without extra work - * (e.g. when positioned on a chain node in a memtable trie). If the current node does not have children this - * is exactly the same as advance(), otherwise it may take multiple steps down (but will not necessarily, even - * if they exist). - *

    - * Note that if any positions are skipped, their content must be null. - *

    - * This is an optional optimization; the default implementation falls back to calling advance. - *

    - * It is an error to call this after the trie has already been exhausted (i.e. when depth() == -1); - * for performance reasons we won't always check this. - * - * @param receiver object that will receive all transitions taken except the last; - * on ascend, or if only one step down was taken, it will not receive any - * @return the new depth, -1 if the trie is exhausted - */ - default int advanceMultiple(TransitionsReceiver receiver) - { - return advance(); - } - - /** - * Advance all the way to the next node with non-null content. - *

    - * It is an error to call this after the trie has already been exhausted (i.e. when depth() == -1); - * for performance reasons we won't always check this. - * - * @param receiver object that will receive all taken transitions - * @return the content, null if the trie is exhausted - */ - default T advanceToContent(ResettingTransitionsReceiver receiver) - { - int prevDepth = depth(); - while (true) - { - int currDepth = advanceMultiple(receiver); - if (currDepth <= 0) - return null; - if (receiver != null) - { - if (currDepth <= prevDepth) - receiver.resetPathLength(currDepth - 1); - receiver.addPathByte(incomingTransition()); - } - T content = content(); - if (content != null) - return content; - prevDepth = currDepth; - } - } - - /** - * Advance to the specified depth and incoming transition or the first valid position that is after the specified - * position. The inputs must be something that could be returned by a single call to {@link #advance} (i.e. - * {@code depth} must be <= current depth + 1, and {@code incomingTransition} must be higher than what the - * current state saw at the requested depth. - * - * @return the new depth, always <= previous depth; -1 if the trie is exhausted - */ - int skipTo(int skipDepth, int skipTransition); - - /** - * Descend into the cursor with the given path. - * - * @return True if the descent is positioned at the end of the given path, false if the trie did not have a path - * for it. In the latter case the cursor is positioned at the first node that follows the given key in iteration - * order. - */ - default boolean descendAlong(ByteSource bytes) - { - int next = bytes.next(); - int depth = depth(); - while (next != ByteSource.END_OF_STREAM) - { - if (skipTo(++depth, next) != depth || incomingTransition() != next) - return false; - next = bytes.next(); - } - return true; - } - /** - * Returns a tail trie, i.e. a trie whose root is the current position. Walking a tail trie will list all - * descendants of the current position with depth adjusted by the current depth. - *

    - * It is an error to call tailTrie on an exhausted cursor. - */ - Trie tailTrie(); - } - - protected abstract Cursor cursor(Direction direction); - - /** - * Used by {@link Cursor#advanceMultiple} to feed the transitions taken. - */ - protected interface TransitionsReceiver - { - /** Add a single byte to the path. */ - void addPathByte(int nextByte); - /** Add the count bytes from position pos in the given buffer. */ - void addPathBytes(DirectBuffer buffer, int pos, int count); - } - - /** - * Used by {@link Cursor#advanceToContent} to track the transitions and backtracking taken. - */ - protected interface ResettingTransitionsReceiver extends TransitionsReceiver - { - /** Delete all bytes beyond the given length. */ - void resetPathLength(int newLength); - } - - /** - * A push interface for walking over the trie. Builds upon TransitionsReceiver to be given the bytes of the - * path, and adds methods called on encountering content and completion. - * See {@link TrieDumper} for an example of how this can be used, and {@link TrieEntriesWalker} as a base class - * for other common usages. - */ - protected interface Walker extends ResettingTransitionsReceiver - { - /** Called when content is found. */ - void content(T content); - - /** Called at the completion of the walk. */ - R complete(); - } - - /** - * Adapter interface providing the methods a {@link Walker} to a {@link Consumer}, so that the latter can be used - * with {@link #process}. - * - * This enables calls like - * trie.forEachEntry(x -> System.out.println(x)); - * to be mapped directly to a single call to {@link #process} without extra allocations. - */ - public interface ValueConsumer extends Consumer, Walker +/// Base trie interface. +/// +/// Normal users of tries will only use the public methods, which provide various transformations of the trie, conversion +/// of its content to other formats (e.g. iterable of values), and several forms of processing. +/// +/// For any unimplemented data extraction operations one can build on the [TrieEntriesWalker] (for-each processing) +/// and [TrieEntriesIterator] (to iterator) base classes, which provide the necessary mechanisms to handle walking +/// the trie. +/// +/// The internal representation of tries using this interface is defined in the [Cursor] interface, accessed via the +/// [CursorWalkable] interface's [#cursor] method. +/// +/// Cursors are a method of presenting the internal structure of a trie without representing nodes as objects, which is +/// still useful for performing the basic operations on tries (iteration, slicing/intersection and merging). A cursor +/// will list the nodes of a trie in order, together with information about the path that was taken to reach them. +/// +/// To begin traversal over a trie, one must retrieve a cursor by calling [#cursor]. Because cursors are +/// stateful, the traversal must always proceed from one thread. Should concurrent reads be required, separate calls to +/// [#cursor] must be made. Any modification that has completed before the construction of a cursor must be +/// visible, but any later concurrent modifications may be presented fully, partially or not at all; this also means that +/// if multiple are made, the cursor may see any part of any subset of them. +/// +/// Note: This model only supports depth-first traversals. We do not currently have a need for breadth-first walks. +/// +/// See [Trie.md](./Trie.md) for further description of the trie representation model. +/// +/// @param The content type of the trie. +public interface Trie extends CursorWalkable> +{ + /// Adapter interface providing the methods a [Cursor.Walker] to a [Consumer], so that the latter can be used + /// with [#process]. + /// This enables calls like + /// `trie.forEachEntry(x -> System.out.println(x));` + /// to be mapped directly to a single call to [#process] without extra allocations. + public interface ValueConsumer extends Consumer, Cursor.Walker { @Override default void content(T content) @@ -364,26 +95,20 @@ default void addPathBytes(DirectBuffer buffer, int pos, int count) } } - /** - * Call the given consumer on all content values in the trie in order. - */ - public void forEachValue(ValueConsumer consumer) + /// Call the given consumer on all content values in the trie in order. + default void forEachValue(ValueConsumer consumer) { process(consumer, Direction.FORWARD); } - /** - * Call the given consumer on all (path, content) pairs with non-null content in the trie in order. - */ - public void forEachEntry(BiConsumer consumer) + /// Call the given consumer on all (path, content) pairs with non-null content in the trie in order. + default void forEachEntry(BiConsumer consumer) { forEachEntry(Direction.FORWARD, consumer); } - /** - * Call the given consumer on all (path, content) pairs with non-null content in the trie in order. - */ - public void forEachEntry(Direction direction, BiConsumer consumer) + /// Call the given consumer on all (path, content) pairs with non-null content in the trie in order. + default void forEachEntry(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); process(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion()), cursor); @@ -391,15 +116,13 @@ public void forEachEntry(Direction direction, BiConsumer R process(Walker walker, Direction direction) + /// Process the trie using the given [Cursor.Walker]. + default R process(Cursor.Walker walker, Direction direction) { return process(walker, cursor(direction)); } - static R process(Walker walker, Cursor cursor) + static R process(Cursor.Walker walker, Cursor cursor) { assert cursor.depth() == 0 : "The provided cursor has already been advanced."; T content = cursor.content(); // handle content on the root node @@ -415,19 +138,15 @@ static R process(Walker walker, Cursor cursor) } - /** - * Process the trie using the given ValueConsumer, skipping all branches below the top content-bearing node. - */ - public Void forEachValueSkippingBranches(Direction direction, ValueConsumer consumer) + /// Process the trie using the given [ValueConsumer], skipping all branches below the top content-bearing node. + default void forEachValueSkippingBranches(Direction direction, ValueConsumer consumer) { - return processSkippingBranches(consumer, cursor(direction)); + processSkippingBranches(consumer, cursor(direction)); } - /** - * Call the given consumer on all (path, content) pairs with non-null content in the trie in order, skipping all - * branches below the top content-bearing node. - */ - public void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer) + /// Call the given consumer on all `(path, content)` pairs with non-null content in the trie in order, skipping all + /// branches below the top content-bearing node. + default void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); processSkippingBranches(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion()), cursor); @@ -435,15 +154,13 @@ public void forEachEntrySkippingBranches(Direction direction, BiConsumer R processSkippingBranches(Walker walker, Direction direction) + /// Process the trie using the given [Cursor.Walker], skipping all branches below the top content-bearing node. + default R processSkippingBranches(Cursor.Walker walker, Direction direction) { return processSkippingBranches(walker, cursor(direction)); } - static R processSkippingBranches(Walker walker, Cursor cursor) + static R processSkippingBranches(Cursor.Walker walker, Cursor cursor) { assert cursor.depth() == 0 : "The provided cursor has already been advanced."; T content = cursor.content(); // handle content on the root node @@ -468,10 +185,8 @@ static R processSkippingBranches(Walker walker, Cursor cursor) return walker.complete(); } - /** - * Map-like get by key. - */ - public T get(ByteComparable key) + /// Map-like get by key. + default T get(ByteComparable key) { Cursor cursor = cursor(Direction.FORWARD); if (cursor.descendAlong(key.asComparableBytes(cursor.byteComparableVersion()))) @@ -480,228 +195,179 @@ public T get(ByteComparable key) return null; } - /** - * Constuct a textual representation of the trie. - */ - public String dump() + /// Constuct a textual representation of the trie. + default String dump() { return dump(Object::toString); } - /** - * Constuct a textual representation of the trie using the given content-to-string mapper. - */ - public String dump(Function contentToString) + /// Constuct a textual representation of the trie using the given content-to-string mapper. + default String dump(Function contentToString) { return process(new TrieDumper<>(contentToString), Direction.FORWARD); } - /** - * Returns a singleton trie mapping the given byte path to content. - */ - public static Trie singleton(ByteComparable b, ByteComparable.Version byteComparableVersion, T v) + /// Returns a singleton trie mapping the given byte path to content. + static Trie singleton(ByteComparable b, ByteComparable.Version byteComparableVersion, T v) { - return new SingletonTrie<>(b, byteComparableVersion, v); + return dir -> new SingletonCursor<>(dir, b.asComparableBytes(byteComparableVersion), byteComparableVersion, v); } - /** - * Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries. - * The view is live, i.e. any write to the source will be reflected in the subtrie. - * - * This method will throw an assertion error if the bounds provided are not correctly ordered, including with - * respect to the `includeLeft` and `includeRight` constraints (i.e. subtrie(x, false, x, false) is an invalid call - * but subtrie(x, true, x, false) is inefficient but fine for an empty subtrie). - * - * @param left the left bound for the returned subtrie. If {@code null}, the resulting subtrie is not left-bounded. - * @param includeLeft whether {@code left} is an inclusive bound of not. - * @param right the right bound for the returned subtrie. If {@code null}, the resulting subtrie is not right-bounded. - * @param includeRight whether {@code right} is an inclusive bound of not. - * @return a view of the subtrie containing all the keys of this trie falling between {@code left} (inclusively if - * {@code includeLeft}) and {@code right} (inclusively if {@code includeRight}). - */ - public Trie subtrie(ByteComparable left, boolean includeLeft, ByteComparable right, boolean includeRight) + /// Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries. + /// The view is live, i.e. any write to the source will be reflected in the subtrie. + /// This method will throw an assertion error if the bounds provided are not correctly ordered, including with + /// respect to the `includeLeft` and `includeRight` constraints (i.e. `subtrie(x, false, x, false)` is an invalid call + /// but `subtrie(x, true, x, false)` is inefficient but fine for an empty subtrie). + /// + /// @param left the left bound for the returned subtrie. If `null`, the resulting subtrie is not left-bounded. + /// @param includeLeft whether `left` is an inclusive bound of not. + /// @param right the right bound for the returned subtrie. If `null`, the resulting subtrie is not right-bounded. + /// @param includeRight whether `right` is an inclusive bound of not. + /// @return a view of the subtrie containing all the keys of this trie falling between `left` (inclusively if + /// `includeLeft`) and `right` (inclusively if `includeRight`). + default Trie subtrie(ByteComparable left, boolean includeLeft, ByteComparable right, boolean includeRight) { if (left == null && right == null) return this; return new SlicedTrie<>(this, left, includeLeft, right, includeRight); } - /** - * Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries. - * The view is live, i.e. any write to the source will be reflected in the subtrie. - * - * @param left the left bound for the returned subtrie, inclusive. If {@code null}, the resulting subtrie is not - * left-bounded. - * @param right the right bound for the returned subtrie, exclusive. If {@code null}, the resulting subtrie is not - * right-bounded. - * @return a view of the subtrie containing all the keys of this trie falling between {@code left} inclusively and - * {@code right} exclusively. - */ - public Trie subtrie(ByteComparable left, ByteComparable right) + /// Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries. + /// The view is live, i.e. any write to the source will be reflected in the subtrie. + /// + /// @param left the left bound for the returned subtrie, inclusive. If `null`, the resulting subtrie is not + /// left-bounded. + /// @param right the right bound for the returned subtrie, exclusive. If `null`, the resulting subtrie is not + /// right-bounded. + /// @return a view of the subtrie containing all the keys of this trie falling between `left` inclusively and + /// `right` exclusively. + default Trie subtrie(ByteComparable left, ByteComparable right) { return subtrie(left, true, right, false); } - /** - * Returns the ordered entry set of this trie's content as an iterable. - */ - public Iterable> entrySet() + /// Returns the ordered entry set of this trie's content as an iterable. + default Iterable> entrySet() { return this::entryIterator; } - /** - * Returns the ordered entry set of this trie's content as an iterable. - */ - public Iterable> entrySet(Direction direction) + /// Returns the ordered entry set of this trie's content as an iterable. + default Iterable> entrySet(Direction direction) { return () -> entryIterator(direction); } - /** - * Returns the ordered entry set of this trie's content in an iterator. - */ - public Iterator> entryIterator() + /// Returns the ordered entry set of this trie's content in an iterator. + default Iterator> entryIterator() { return entryIterator(Direction.FORWARD); } - /** - * Returns the ordered entry set of this trie's content in an iterator. - */ - public Iterator> entryIterator(Direction direction) + /// Returns the ordered entry set of this trie's content in an iterator. + default Iterator> entryIterator(Direction direction) { return new TrieEntriesIterator.AsEntries<>(cursor(direction)); } - /** - * Returns the ordered entry set of this trie's content in an iterable, filtered by the given type. - */ - public Iterable> filteredEntrySet(Class clazz) + /// Returns the ordered entry set of this trie's content in an iterable, filtered by the given type. + default Iterable> filteredEntrySet(Class clazz) { return filteredEntrySet(Direction.FORWARD, clazz); } - /** - * Returns the ordered entry set of this trie's content in an iterable, filtered by the given type. - */ - public Iterable> filteredEntrySet(Direction direction, Class clazz) + /// Returns the ordered entry set of this trie's content in an iterable, filtered by the given type. + default Iterable> filteredEntrySet(Direction direction, Class clazz) { return () -> filteredEntryIterator(direction, clazz); } - /** - * Returns the ordered entry set of this trie's content in an iterator, filtered by the given type. - */ - public Iterator> filteredEntryIterator(Direction direction, Class clazz) + /// Returns the ordered entry set of this trie's content in an iterator, filtered by the given type. + default Iterator> filteredEntryIterator(Direction direction, Class clazz) { return new TrieEntriesIterator.AsEntriesFilteredByType<>(cursor(direction), clazz); } - /** - * Returns the ordered set of values of this trie as an iterable. - */ - public Iterable values() + /// Returns the ordered set of values of this trie as an iterable. + default Iterable values() { return this::valueIterator; } - /** - * Returns the ordered set of values of this trie as an iterable. - */ - public Iterable values(Direction direction) + /// Returns the ordered set of values of this trie as an iterable. + default Iterable values(Direction direction) { return direction.isForward() ? this::valueIterator : this::reverseValueIterator; } - /** - * Returns the ordered set of values of this trie in an iterator. - */ - public Iterator valueIterator() + /// Returns the ordered set of values of this trie in an iterator. + default Iterator valueIterator() { return valueIterator(Direction.FORWARD); } - /** - * Returns the inversely ordered set of values of this trie in an iterator. - */ - public Iterator reverseValueIterator() + /// Returns the inversely ordered set of values of this trie in an iterator. + default Iterator reverseValueIterator() { return valueIterator(Direction.REVERSE); } - /** - * Returns the ordered set of values of this trie in an iterator. - */ - public Iterator valueIterator(Direction direction) + /// Returns the ordered set of values of this trie in an iterator. + default Iterator valueIterator(Direction direction) { return new TrieValuesIterator<>(cursor(direction)); } - /** - * Returns the ordered set of values of this trie in an iterable, filtered by the given type. - */ - public Iterable filteredValues(Class clazz) + /// Returns the ordered set of values of this trie in an iterable, filtered by the given type. + default Iterable filteredValues(Class clazz) { return filteredValues(Direction.FORWARD, clazz); } - /** - * Returns the ordered set of values of this trie in an iterable, filtered by the given type. - */ - public Iterable filteredValues(Direction direction, Class clazz) + /// Returns the ordered set of values of this trie in an iterable, filtered by the given type. + default Iterable filteredValues(Direction direction, Class clazz) { return () -> filteredValuesIterator(direction, clazz); } - /** - * Returns the ordered set of values of this trie in an iterator, filtered by the given type. - */ - public Iterator filteredValuesIterator(Direction direction, Class clazz) + /// Returns the ordered set of values of this trie in an iterator, filtered by the given type. + default Iterator filteredValuesIterator(Direction direction, Class clazz) { return new TrieValuesIterator.FilteredByType<>(cursor(direction), clazz); } - /** - * Returns the values in any order. For some tries this is much faster than the ordered iterable. - */ - public Iterable valuesUnordered() + /// Returns the values in any order. For some tries this is much faster than the ordered iterable. + default Iterable valuesUnordered() { return values(); } - /** - * Resolver of content of merged nodes, used for two-source merges (i.e. mergeWith). - */ - public interface MergeResolver + /// Resolver of content of merged nodes, used for two-source merges (i.e. mergeWith). + interface MergeResolver { // Note: No guarantees about argument order. // E.g. during t1.mergeWith(t2, resolver), resolver may be called with t1 or t2's items as first argument. T resolve(T b1, T b2); } - /** - * Constructs a view of the merge of this trie with the given one. The view is live, i.e. any write to any of the - * sources will be reflected in the merged view. - * - * If there is content for a given key in both sources, the resolver will be called to obtain the combination. - * (The resolver will not be called if there's content from only one source.) - */ - public Trie mergeWith(Trie other, MergeResolver resolver) + /// Constructs a view of the merge of this trie with the given one. The view is live, i.e. any write to any of the + /// sources will be reflected in the merged view. + /// + /// If there is content for a given key in both sources, the resolver will be called to obtain the combination. + /// (The resolver will not be called if there's content from only one source.) + default Trie mergeWith(Trie other, MergeResolver resolver) { return new MergeTrie<>(resolver, this, other); } - /** - * Resolver of content of merged nodes. - * - * The resolver's methods are only called if more than one of the merged nodes contain content, and the - * order in which the arguments are given is not defined. Only present non-null values will be included in the - * collection passed to the resolving methods. - * - * Can also be used as a two-source resolver. - */ - public interface CollectionMergeResolver extends MergeResolver + /// Resolver of content of merged nodes. + /// + /// The resolver's methods are only called if more than one of the merged nodes contain content, and the + /// order in which the arguments are given is not defined. Only present non-null values will be included in the + /// collection passed to the resolving methods. + /// + /// Can also be used as a two-source resolver. + interface CollectionMergeResolver extends MergeResolver { T resolve(Collection contents); @@ -712,7 +378,8 @@ default T resolve(T c1, T c2) } } - private static final CollectionMergeResolver THROWING_RESOLVER = new CollectionMergeResolver() + /// Not to be used directly, call [#throwingResolver()] instead. + static CollectionMergeResolver THROWING_RESOLVER = new CollectionMergeResolver() { @Override public Object resolve(Collection contents) @@ -726,24 +393,20 @@ private AssertionError error() } }; - /** - * Returns a resolver that throws whenever more than one of the merged nodes contains content. - * Can be used to merge tries that are known to have distinct content paths. - */ + /// Returns a resolver that throws whenever more than one of the merged nodes contains content. + /// Can be used to merge tries that are known to have distinct content paths. @SuppressWarnings("unchecked") - public static CollectionMergeResolver throwingResolver() + static CollectionMergeResolver throwingResolver() { return (CollectionMergeResolver) THROWING_RESOLVER; } - /** - * Constructs a view of the merge of multiple tries. The view is live, i.e. any write to any of the - * sources will be reflected in the merged view. - * - * If there is content for a given key in more than one sources, the resolver will be called to obtain the - * combination. (The resolver will not be called if there's content from only one source.) - */ - public static Trie merge(Collection> sources, CollectionMergeResolver resolver) + /// Constructs a view of the merge of multiple tries. The view is live, i.e. any write to any of the + /// sources will be reflected in the merged view. + /// + /// If there is content for a given key in more than one sources, the resolver will be called to obtain the + /// combination. (The resolver will not be called if there's content from only one source.) + static Trie merge(Collection> sources, CollectionMergeResolver resolver) { switch (sources.size()) { @@ -763,13 +426,11 @@ public static Trie merge(Collection> sources, Collectio } } - /** - * Constructs a view of the merge of multiple tries, where each source must have distinct keys. The view is live, - * i.e. any write to any of the sources will be reflected in the merged view. - * - * If there is content for a given key in more than one sources, the merge will throw an assertion error. - */ - public static Trie mergeDistinct(Collection> sources) + /// Constructs a view of the merge of multiple tries, where each source must have distinct keys. The view is live, + /// i.e. any write to any of the sources will be reflected in the merged view. + /// + /// If there is content for a given key in more than one sources, the merge will throw an assertion error. + static Trie mergeDistinct(Collection> sources) { switch (sources.size()) { @@ -789,58 +450,46 @@ public static Trie mergeDistinct(Collection> sources) } } - /** - * Returns a Trie that is a view of this one, where the given prefix is prepended before the root. - */ - public Trie prefixedBy(ByteComparable prefix) + /// Returns a Trie that is a view of this one, where the given prefix is prepended before the root. + default Trie prefixedBy(ByteComparable prefix) { return new PrefixedTrie(prefix, this); } - /** - * Returns an entry set containing all tail tree constructed at the points that contain content of - * the given type. - */ - public Iterable>> tailTries(Direction direction, Class clazz) + /// Returns an entry set containing all tail tree constructed at the points that contain content of + /// the given type. + default Iterable>> tailTries(Direction direction, Class clazz) { return () -> new TrieTailsIterator.AsEntries<>(cursor(direction), clazz); } - /** - * Returns a trie that corresponds to the branch of this trie rooted at the given prefix. - *

    - * The result will include the same values as {@code subtrie(prefix, nextBranch(prefix))}, but the keys in the - * resulting trie will not include the prefix. In other words, - * {@code tailTrie(prefix).prefixedBy(prefix) = subtrie(prefix, nextBranch(prefix))} - * where nextBranch stands for the key adjusted by adding one at the last position. - */ - public Trie tailTrie(ByteComparable prefix) + /// Returns a trie that corresponds to the branch of this trie rooted at the given prefix. + /// + /// The result will include the same values as `subtrie(prefix, nextBranch(prefix))`, but the keys in the + /// resulting trie will not include the prefix. In other words, + /// ```tailTrie(prefix).prefixedBy(prefix) = subtrie(prefix, nextBranch(prefix))``` + /// where `nextBranch` stands for the key adjusted by adding one at the last position. + default Trie tailTrie(ByteComparable prefix) { Cursor c = cursor(Direction.FORWARD); if (c.descendAlong(prefix.asComparableBytes(c.byteComparableVersion()))) - return c.tailTrie(); + return dir -> c.tailCursor(dir); else return null; } - public static Trie empty(ByteComparable.Version byteComparableVersion) + static Trie empty(ByteComparable.Version byteComparableVersion) { - return new Trie() - { - public Cursor cursor(Direction dir) - { - return new EmptyCursor<>(dir, byteComparableVersion); - } - }; + return dir -> new EmptyCursor<>(dir, byteComparableVersion); } - static class EmptyCursor implements Cursor + class EmptyCursor implements Cursor { private final Direction direction; private final ByteComparable.Version byteComparableVersion; int depth; - public EmptyCursor(Direction direction, ByteComparable.Version byteComparableVersion) + EmptyCursor(Direction direction, ByteComparable.Version byteComparableVersion) { this.direction = direction; this.byteComparableVersion = byteComparableVersion; @@ -865,10 +514,10 @@ public ByteComparable.Version byteComparableVersion() } @Override - public Trie tailTrie() + public Cursor tailCursor(Direction direction) { assert depth == 0 : "tailTrie called on exhausted cursor"; - return empty(byteComparableVersion); + return new EmptyCursor<>(direction, byteComparableVersion); } public int depth() diff --git a/src/java/org/apache/cassandra/db/tries/TrieDumper.java b/src/java/org/apache/cassandra/db/tries/TrieDumper.java index 9dfb2c190d6d..e09739d9f100 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieDumper.java +++ b/src/java/org/apache/cassandra/db/tries/TrieDumper.java @@ -21,10 +21,8 @@ import org.agrona.DirectBuffer; -/** - * Simple utility class for dumping the structure of a trie to string. - */ -class TrieDumper implements Trie.Walker +/// Simple utility class for dumping the structure of a trie to string. +class TrieDumper implements Cursor.Walker { private final StringBuilder b; private final Function contentToString; diff --git a/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java b/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java index 99e3f764244d..0909579ad982 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java +++ b/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java @@ -26,13 +26,11 @@ import org.apache.cassandra.utils.bytecomparable.ByteComparable; -/** - * Convertor of trie entries to iterator where each entry is passed through {@link #mapContent} (to be implemented by - * descendants). - */ +/// Convertor of trie entries to iterator where each entry is passed through [#mapContent] (to be implemented by +/// descendants). public abstract class TrieEntriesIterator extends TriePathReconstructor implements Iterator { - private final Trie.Cursor cursor; + private final Cursor cursor; private final Predicate predicate; T next; boolean gotNext; @@ -42,7 +40,7 @@ protected TrieEntriesIterator(Trie trie, Direction direction, Predicate pr this(trie.cursor(direction), predicate); } - TrieEntriesIterator(Trie.Cursor cursor, Predicate predicate) + TrieEntriesIterator(Cursor cursor, Predicate predicate) { this.cursor = cursor; this.predicate = predicate; @@ -88,7 +86,7 @@ ByteComparable.Version byteComparableVersion() */ static class AsEntries extends TrieEntriesIterator> { - public AsEntries(Trie.Cursor cursor) + public AsEntries(Cursor cursor) { super(cursor, Predicates.alwaysTrue()); } @@ -105,7 +103,7 @@ protected Map.Entry mapContent(T content, byte[] b */ static class AsEntriesFilteredByType extends TrieEntriesIterator> { - public AsEntriesFilteredByType(Trie.Cursor cursor, Class clazz) + public AsEntriesFilteredByType(Cursor cursor, Class clazz) { super(cursor, clazz::isInstance); } diff --git a/src/java/org/apache/cassandra/db/tries/TrieEntriesWalker.java b/src/java/org/apache/cassandra/db/tries/TrieEntriesWalker.java index 362fe8f112b7..fc13751a99f8 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieEntriesWalker.java +++ b/src/java/org/apache/cassandra/db/tries/TrieEntriesWalker.java @@ -21,11 +21,9 @@ import org.apache.cassandra.utils.bytecomparable.ByteComparable; -/** - * Walker of trie entries, used with Trie.process to walk all content in order and provide the path through which values - * are reached. - */ -public abstract class TrieEntriesWalker extends TriePathReconstructor implements Trie.Walker +/// Walker of trie entries, used with [Trie#process] to walk all content in order and provide the path through which values +/// are reached. +public abstract class TrieEntriesWalker extends TriePathReconstructor implements Cursor.Walker { @Override public void content(T content) @@ -35,9 +33,7 @@ public void content(T content) protected abstract void content(T content, byte[] bytes, int byteLength); - /** - * Iterator representing the content of the trie a sequence of (path, content) pairs. - */ + /// Iterator representing the content of the trie a sequence of (path, content) pairs. static class WithConsumer extends TrieEntriesWalker { private final BiConsumer consumer; diff --git a/src/java/org/apache/cassandra/db/tries/TriePathReconstructor.java b/src/java/org/apache/cassandra/db/tries/TriePathReconstructor.java index c59d126fe272..be1dc975d327 100644 --- a/src/java/org/apache/cassandra/db/tries/TriePathReconstructor.java +++ b/src/java/org/apache/cassandra/db/tries/TriePathReconstructor.java @@ -23,7 +23,7 @@ import org.agrona.DirectBuffer; import org.apache.cassandra.utils.bytecomparable.ByteComparable; -public class TriePathReconstructor implements Trie.ResettingTransitionsReceiver +public class TriePathReconstructor implements Cursor.ResettingTransitionsReceiver { protected byte[] keyBytes = new byte[32]; protected int keyPos = 0; diff --git a/src/java/org/apache/cassandra/db/tries/TrieTailsIterator.java b/src/java/org/apache/cassandra/db/tries/TrieTailsIterator.java index e15ce6548206..dff6ca87c2bb 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieTailsIterator.java +++ b/src/java/org/apache/cassandra/db/tries/TrieTailsIterator.java @@ -24,13 +24,11 @@ import org.apache.cassandra.utils.bytecomparable.ByteComparable; -/** - * Iterator of trie entries that constructs tail tries for the content-bearing branches that satisfy the given predicate - * and skips over the returned branches. - */ +/// Iterator of trie entries that constructs tail tries for the content-bearing branches that satisfy the given predicate +/// and skips over the returned branches. public abstract class TrieTailsIterator extends TriePathReconstructor implements Iterator { - final Trie.Cursor cursor; + final Cursor cursor; private final Predicate predicate; private T next; private boolean gotNext; @@ -42,7 +40,7 @@ protected TrieTailsIterator(Trie trie, Direction direction, Predicate pred assert cursor.depth() == 0; } - TrieTailsIterator(Trie.Cursor cursor, Predicate predicate) + TrieTailsIterator(Cursor cursor, Predicate predicate) { this.cursor = cursor; this.predicate = predicate; @@ -86,7 +84,7 @@ public V next() gotNext = false; T v = next; next = null; - return mapContent(v, cursor.tailTrie(), keyBytes, keyPos); + return mapContent(v, dir -> cursor.tailCursor(dir), keyBytes, keyPos); } ByteComparable.Version byteComparableVersion() @@ -96,14 +94,12 @@ ByteComparable.Version byteComparableVersion() protected abstract V mapContent(T value, Trie tailTrie, byte[] bytes, int byteLength); - /** - * Iterator representing the selected content of the trie a sequence of {@code (path, tail)} pairs, where - * {@code tail} is the branch of the trie rooted at the selected content node (reachable by following - * {@code path}). The tail trie will have the selected content at its root. - */ + /// Iterator representing the selected content of the trie a sequence of `(path, tail)` pairs, where + /// `tail` is the branch of the trie rooted at the selected content node (reachable by following + /// `path`). The tail trie will have the selected content at its root. static class AsEntries extends TrieTailsIterator>> { - public AsEntries(Trie.Cursor cursor, Class clazz) + public AsEntries(Cursor cursor, Class clazz) { super(cursor, clazz::isInstance); } diff --git a/src/java/org/apache/cassandra/db/tries/TrieValuesIterator.java b/src/java/org/apache/cassandra/db/tries/TrieValuesIterator.java index 0a99c3ff0b99..d972c401317e 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieValuesIterator.java +++ b/src/java/org/apache/cassandra/db/tries/TrieValuesIterator.java @@ -19,16 +19,14 @@ import java.util.Iterator; -/** - * Ordered iterator of trie content. - */ +/// Ordered iterator of trie content. class TrieValuesIterator implements Iterator { - private final Trie.Cursor cursor; + private final Cursor cursor; T next; boolean gotNext; - protected TrieValuesIterator(Trie.Cursor cursor) + protected TrieValuesIterator(Cursor cursor) { this.cursor = cursor; assert cursor.depth() == 0; @@ -60,12 +58,12 @@ public T next() static class FilteredByType implements Iterator { - private final Trie.Cursor cursor; + private final Cursor cursor; T next; boolean gotNext; Class clazz; - FilteredByType(Trie.Cursor cursor, Class clazz) + FilteredByType(Cursor cursor, Class clazz) { this.cursor = cursor; this.clazz = clazz; diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java index 102c3cbc76a3..caebe467bb47 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java @@ -219,7 +219,7 @@ public SpecStackEntry(Object[] spec, Object content, SpecStackEntry parent, Dire } } - public static class CursorFromSpec implements Trie.Cursor + public static class CursorFromSpec implements Cursor { SpecStackEntry stack; int depth; @@ -305,9 +305,9 @@ public ByteComparable.Version byteComparableVersion() } @Override - public Trie tailTrie() + public Cursor tailCursor(Direction dir) { - throw new UnsupportedOperationException("tailTrie on test cursor"); + throw new UnsupportedOperationException("tailCursor on test cursor"); } } @@ -316,7 +316,7 @@ static Trie specifiedTrie(Object[] nodeDef) return new Trie() { @Override - protected Cursor cursor(Direction direction) + public Cursor cursor(Direction direction) { return new CursorFromSpec(nodeDef, direction); } diff --git a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java index c9ab05122e92..e7fa358339eb 100644 --- a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java @@ -296,7 +296,7 @@ private static Trie singleLevelIntTrie(int childs) return new Trie() { @Override - protected Cursor cursor(Direction direction) + public Cursor cursor(Direction direction) { return new singleLevelCursor(direction); } @@ -370,7 +370,7 @@ public ByteComparable.Version byteComparableVersion() } @Override - public Trie tailTrie() + public Cursor tailCursor(Direction d) { throw new UnsupportedOperationException("tailTrie on test cursor"); } From e7f4f00fa47d2920dd9a54e9903d005e5bea2b88 Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Fri, 14 Mar 2025 12:32:55 +0200 Subject: [PATCH 02/22] Change transformations to implement Cursor --- .../db/tries/CollectionMergeCursor.java | 372 +++++++++++++++ .../db/tries/CollectionMergeTrie.java | 411 ---------------- .../cassandra/db/tries/MergeCursor.java | 164 +++++++ .../apache/cassandra/db/tries/MergeTrie.java | 202 -------- .../cassandra/db/tries/PrefixedCursor.java | 157 ++++++ .../cassandra/db/tries/PrefixedTrie.java | 171 ------- .../cassandra/db/tries/SlicedCursor.java | 409 ++++++++++++++++ .../apache/cassandra/db/tries/SlicedTrie.java | 448 ------------------ .../org/apache/cassandra/db/tries/Trie.java | 91 +++- .../db/tries/CollectionMergeTrieTest.java | 32 +- .../cassandra/db/tries/MergeTrieTest.java | 3 +- .../cassandra/db/tries/SlicedTrieTest.java | 2 +- 12 files changed, 1200 insertions(+), 1262 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java delete mode 100644 src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java create mode 100644 src/java/org/apache/cassandra/db/tries/MergeCursor.java delete mode 100644 src/java/org/apache/cassandra/db/tries/MergeTrie.java create mode 100644 src/java/org/apache/cassandra/db/tries/PrefixedCursor.java delete mode 100644 src/java/org/apache/cassandra/db/tries/PrefixedTrie.java create mode 100644 src/java/org/apache/cassandra/db/tries/SlicedCursor.java delete mode 100644 src/java/org/apache/cassandra/db/tries/SlicedTrie.java diff --git a/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java b/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java new file mode 100644 index 000000000000..53144a106e38 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java @@ -0,0 +1,372 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import com.google.common.collect.Iterables; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.function.BiFunction; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// A merged view of multiple tries. +/// +/// This is accomplished by walking the cursors in parallel; the merged cursor takes the position and features of the +/// smallest and advances with it; when multiple cursors are equal, all of them are advanced. The ordered view of the +/// cursors is maintained using a custom binary min-heap, built for efficiently reforming the heap when the top elements +/// are advanced. +/// +/// Crucial for the efficiency of this is the fact that when they are advanced like this, we can compare cursors' +/// positions by their `depth` descending and then `incomingTransition` ascending. +/// See [Trie.md](./Trie.md) for further details. +/// +/// The merge cursor is a variation of the idea of a merge iterator with one key observation: because we advance +/// the source iterators together, we can compare them just by depth and incoming transition. +/// +/// The most straightforward way to implement merging of iterators is to use a `PriorityQueue`, +/// `poll` it to find the next item to consume, then `add` the iterator back after advancing. +/// This is not very efficient as `poll` and `add` in all cases require at least +/// `log(size)` comparisons and swaps (usually more than `2*log(size)`) per consumed item, even +/// if the input is suitable for fast iteration. +/// +/// The implementation below makes use of the fact that replacing the top element in a binary heap can be +/// done much more efficiently than separately removing it and placing it back, especially in the cases where +/// the top iterator is to be used again very soon (e.g. when there are large sections of the output where +/// only a limited number of input iterators overlap, which is normally the case in many practically useful +/// situations, e.g. levelled compaction). +/// +/// The implementation builds and maintains a binary heap of sources (stored in an array), where we do not +/// add items after the initial construction. Instead we advance the smallest element (which is at the top +/// of the heap) and push it down to find its place for its new position. Should this source be exhausted, +/// we swap it with the last source in the heap and proceed by pushing that down in the heap. +/// +/// In the case where we have multiple sources with matching positions, the merging algorithm +/// must be able to merge all equal values. To achieve this `content` walks the heap to +/// find all equal cursors without advancing them, and separately `advance` advances +/// all equal sources and restores the heap structure. +/// +/// The latter is done equivalently to the process of initial construction of a min-heap using back-to-front +/// heapification as done in the classic heapsort algorithm. It only needs to heapify subheaps whose top item +/// is advanced (i.e. one whose position matches the current), and we can do that recursively from +/// bottom to top. Should a source be exhausted when advancing, it can be thrown away by swapping in the last +/// source in the heap (note: we must be careful to advance that source too if required). +/// +/// To make it easier to advance efficienty in single-sourced branches of tries, we extract the current smallest +/// cursor (the head) separately, and start any advance with comparing that to the heap's first. When the smallest +/// cursor remains the same (e.g. in branches coming from a single source) this makes it possible to advance with +/// just one comparison instead of two at the expense of increasing the number by one in the general case. +/// +/// Note: This is a simplification of the MergeIterator code from CASSANDRA-8915, without the leading ordered +/// section and equalParent flag since comparisons of cursor positions are cheap. +class CollectionMergeCursor implements Cursor +{ + private final Trie.CollectionMergeResolver resolver; + private final Direction direction; + + /// The smallest cursor, tracked separately to improve performance in single-source sections of the trie. + private Cursor head; + + /// Binary heap of the remaining cursors. The smallest element is at position 0. + /// Every element `i` is smaller than or equal to its two children, i.e. + /// ```heap[i] <= heap[i*2 + 1] && heap[i] <= heap[i*2 + 2]``` + private final Cursor[] heap; + + /// A list used to collect contents during [#content()] calls. + private final List contents; + + public CollectionMergeCursor(Trie.CollectionMergeResolver resolver, Direction direction, Collection inputs, BiFunction> extractor) + { + this.resolver = resolver; + this.direction = direction; + int count = inputs.size(); + // Get cursors for all inputs. Put one of them in head and the rest in the heap. + heap = new Cursor[count - 1]; + contents = new ArrayList<>(count); + int i = -1; + for (I src : inputs) + { + Cursor cursor = extractor.apply(src, direction); + assert cursor.depth() == 0; + if (i >= 0) + heap[i] = cursor; + else + head = cursor; + ++i; + } + // The cursors are all currently positioned on the root and thus in valid heap order. + } + + /// Interface for internal operations that can be applied to selected top elements of the heap. + interface HeapOp + { + void apply(CollectionMergeCursor self, Cursor cursor, int index); + + default boolean shouldContinueWithChild(Cursor child, Cursor head) + { + return equalCursor(child, head); + } + } + + /// Apply a non-interfering operation, i.e. one that does not change the cursor state, to all inputs in the heap + /// that satisfy the [HeapOp#shouldContinueWithChild] condition (by default, being equal to the head). + /// For interfering operations like advancing the cursors, use [#advanceSelectedAndRestoreHeap(AdvancingHeapOp)]. + private void applyToSelectedInHeap(HeapOp action) + { + applyToSelectedElementsInHeap(action, 0); + } + + /// Interface for internal advancing operations that can be applied to the heap cursors. This interface provides + /// the code to restore the heap structure after advancing the cursors. + interface AdvancingHeapOp extends HeapOp + { + void apply(Cursor cursor); + + default void apply(CollectionMergeCursor self, Cursor cursor, int index) + { + // Apply the operation, which should advance the position of the element. + apply(cursor); + + // This method is called on the back path of the recursion. At this point the heaps at both children are + // advanced and well-formed. + // Place current node in its proper position. + self.heapifyDown(cursor, index); + // The heap rooted at index is now advanced and well-formed. + } + } + + + /// Advance the state of all inputs in the heap that satisfy the [#shouldContinueWithChild] condition + /// (by default, being equal to the head) and restore the heap invariant. + private void advanceSelectedAndRestoreHeap(AdvancingHeapOp action) + { + applyToSelectedElementsInHeap(action, 0); + } + + /// Apply an operation to all elements on the heap that satisfy, recursively through the heap hierarchy, the + /// `shouldContinueWithChild` condition (being equal to the head by default). Descends recursively in the + /// heap structure to all selected children and applies the operation on the way back. + /// + /// This operation can be something that does not change the cursor state (see [#content]) or an operation + /// that advances the cursor to a new state, wrapped in a [AdvancingHeapOp] ([#advance] or + /// [#skipTo]). The latter interface takes care of pushing elements down in the heap after advancing + /// and restores the subheap state on return from each level of the recursion. + private void applyToSelectedElementsInHeap(HeapOp action, int index) + { + if (index >= heap.length) + return; + Cursor item = heap[index]; + if (!action.shouldContinueWithChild(item, head)) + return; + + // If the children are at the same position, they also need advancing and their subheap + // invariant to be restored. + applyToSelectedElementsInHeap(action, index * 2 + 1); + applyToSelectedElementsInHeap(action, index * 2 + 2); + + // Apply the action. This is done on the reverse direction to give the action a chance to form proper + // subheaps and combine them on processing the parent. + action.apply(this, item, index); + } + + /// Push the given state down in the heap from the given index until it finds its proper place among + /// the subheap rooted at that position. + private void heapifyDown(Cursor item, int index) + { + while (true) + { + int next = index * 2 + 1; + if (next >= heap.length) + break; + // Select the smaller of the two children to push down to. + if (next + 1 < heap.length && greaterCursor(direction, heap[next], heap[next + 1])) + ++next; + // If the child is greater or equal, the invariant has been restored. + if (!greaterCursor(direction, item, heap[next])) + break; + heap[index] = heap[next]; + index = next; + } + heap[index] = item; + } + + /// Check if the head is greater than the top element in the heap, and if so, swap them and push down the new + /// top until its proper place. + /// + /// @param headDepth the depth of the head cursor (as returned by e.g. advance). + /// @return the new head element's depth + private int maybeSwapHead(int headDepth) + { + int heap0Depth = heap[0].depth(); + if (headDepth > heap0Depth || + (headDepth == heap0Depth && direction.le(head.incomingTransition(), heap[0].incomingTransition()))) + return headDepth; // head is still smallest + + // otherwise we need to swap heap and heap[0] + Cursor newHeap0 = head; + head = heap[0]; + heapifyDown(newHeap0, 0); + return heap0Depth; + } + + boolean branchHasMultipleSources() + { + return equalCursor(heap[0], head); + } + + @Override + public int advance() + { + advanceSelectedAndRestoreHeap(Cursor::advance); + return maybeSwapHead(head.advance()); + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + // If the current position is present in just one cursor, we can safely descend multiple levels within + // its branch as no one of the other tries has content for it. + if (branchHasMultipleSources()) + return advance(); // More than one source at current position, do single-step advance. + + // If there are no children, i.e. the cursor ascends, we have to check if it's become larger than some + // other candidate. + return maybeSwapHead(head.advanceMultiple(receiver)); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + // We need to advance all cursors that stand before the requested position. + // If a child cursor does not need to advance as it is greater than the skip position, neither of the ones + // below it in the heap hierarchy do as they can't have an earlier position. + class SkipTo implements AdvancingHeapOp + { + @Override + public boolean shouldContinueWithChild(Cursor child, Cursor head) + { + // When the requested position descends, the inplicit prefix bytes are those of the head cursor, + // and thus we need to check against that if it is a match. + if (equalCursor(child, head)) + return true; + // Otherwise we can compare the child's position against a cursor advanced as requested, and need + // to skip only if it would be before it. + int childDepth = child.depth(); + return childDepth > skipDepth || + childDepth == skipDepth && direction.lt(child.incomingTransition(), skipTransition); + } + + @Override + public void apply(Cursor cursor) + { + cursor.skipTo(skipDepth, skipTransition); + } + } + + applyToSelectedElementsInHeap(new SkipTo(), 0); + return maybeSwapHead(head.skipTo(skipDepth, skipTransition)); + } + + @Override + public int depth() + { + return head.depth(); + } + + @Override + public int incomingTransition() + { + return head.incomingTransition(); + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return head.byteComparableVersion(); + } + + @Override + public T content() + { + if (!branchHasMultipleSources()) + return head.content(); + + applyToSelectedInHeap(CollectionMergeCursor::collectContent); + collectContent(head, -1); + + T toReturn; + switch (contents.size()) + { + case 0: + toReturn = null; + break; + case 1: + toReturn = contents.get(0); + break; + default: + toReturn = resolver.resolve(contents); + break; + } + contents.clear(); + return toReturn; + } + + private void collectContent(Cursor item, int index) + { + T itemContent = item.content(); + if (itemContent != null) + contents.add(itemContent); + } + + @Override + public Cursor tailCursor(Direction dir) + { + if (!branchHasMultipleSources()) + return head.tailCursor(dir); + + List> inputs = new ArrayList<>(heap.length + 1); + inputs.add(head); + applyToSelectedInHeap((self, cursor, index) -> inputs.add(cursor)); + + return new CollectionMergeCursor<>(resolver, dir, inputs, Cursor::tailCursor); + } + + /// Compare the positions of two cursors. One is before the other when + /// - its depth is greater, or + /// - its depth is equal, and the incoming transition is smaller. + static boolean greaterCursor(Direction direction, Cursor c1, Cursor c2) + { + int c1depth = c1.depth(); + int c2depth = c2.depth(); + if (c1depth != c2depth) + return c1depth < c2depth; + return direction.lt(c2.incomingTransition(), c1.incomingTransition()); + } + + static boolean equalCursor(Cursor c1, Cursor c2) + { + return c1.depth() == c2.depth() && c1.incomingTransition() == c2.incomingTransition(); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java b/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java deleted file mode 100644 index 0bc20f1cb809..000000000000 --- a/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java +++ /dev/null @@ -1,411 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.db.tries; - -import com.google.common.collect.Iterables; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.function.BiFunction; -import org.apache.cassandra.utils.bytecomparable.ByteComparable; - -/// A merged view of multiple tries. -/// -/// This is accomplished by walking the cursors in parallel; the merged cursor takes the position and features of the -/// smallest and advances with it; when multiple cursors are equal, all of them are advanced. The ordered view of the -/// cursors is maintained using a custom binary min-heap, built for efficiently reforming the heap when the top elements -/// are advanced (see [CollectionMergeCursor]). -/// -/// Crucial for the efficiency of this is the fact that when they are advanced like this, we can compare cursors' -/// positions by their depth descending and then incomingTransition ascending. -/// -/// See [Trie.md](./Trie.md) for further details. -class CollectionMergeTrie implements Trie -{ - private final CollectionMergeResolver resolver; // only called on more than one input - protected final Collection> inputs; - - CollectionMergeTrie(Collection> inputs, CollectionMergeResolver resolver) - { - this.resolver = resolver; - this.inputs = inputs; - } - - @Override - public Cursor cursor(Direction direction) - { - return new CollectionMergeCursor<>(resolver, direction, inputs); - } - - /** - * Compare the positions of two cursors. One is before the other when - * - its depth is greater, or - * - its depth is equal, and the incoming transition is smaller. - */ - static boolean greaterCursor(Direction direction, Cursor c1, Cursor c2) - { - int c1depth = c1.depth(); - int c2depth = c2.depth(); - if (c1depth != c2depth) - return c1depth < c2depth; - return direction.lt(c2.incomingTransition(), c1.incomingTransition()); - } - - static boolean equalCursor(Cursor c1, Cursor c2) - { - return c1.depth() == c2.depth() && c1.incomingTransition() == c2.incomingTransition(); - } - - /// The merge cursor is a variation of the idea of a merge iterator with one key observation: because we advance - /// the source iterators together, we can compare them just by depth and incoming transition. - /// - /// The most straightforward way to implement merging of iterators is to use a `PriorityQueue`, - /// `poll` it to find the next item to consume, then `add` the iterator back after advancing. - /// This is not very efficient as `poll` and `add` in all cases require at least - /// `log(size)` comparisons and swaps (usually more than `2*log(size)`) per consumed item, even - /// if the input is suitable for fast iteration. - /// - /// The implementation below makes use of the fact that replacing the top element in a binary heap can be - /// done much more efficiently than separately removing it and placing it back, especially in the cases where - /// the top iterator is to be used again very soon (e.g. when there are large sections of the output where - /// only a limited number of input iterators overlap, which is normally the case in many practically useful - /// situations, e.g. levelled compaction). - /// - /// The implementation builds and maintains a binary heap of sources (stored in an array), where we do not - /// add items after the initial construction. Instead we advance the smallest element (which is at the top - /// of the heap) and push it down to find its place for its new position. Should this source be exhausted, - /// we swap it with the last source in the heap and proceed by pushing that down in the heap. - /// - /// In the case where we have multiple sources with matching positions, the merging algorithm - /// must be able to merge all equal values. To achieve this `content` walks the heap to - /// find all equal cursors without advancing them, and separately `advance` advances - /// all equal sources and restores the heap structure. - /// - /// The latter is done equivalently to the process of initial construction of a min-heap using back-to-front - /// heapification as done in the classic heapsort algorithm. It only needs to heapify subheaps whose top item - /// is advanced (i.e. one whose position matches the current), and we can do that recursively from - /// bottom to top. Should a source be exhausted when advancing, it can be thrown away by swapping in the last - /// source in the heap (note: we must be careful to advance that source too if required). - /// - /// To make it easier to advance efficienty in single-sourced branches of tries, we extract the current smallest - /// cursor (the head) separately, and start any advance with comparing that to the heap's first. When the smallest - /// cursor remains the same (e.g. in branches coming from a single source) this makes it possible to advance with - /// just one comparison instead of two at the expense of increasing the number by one in the general case. - /// - /// Note: This is a simplification of the MergeIterator code from CASSANDRA-8915, without the leading ordered - /// section and equalParent flag since comparisons of cursor positions are cheap. - static class CollectionMergeCursor implements Cursor - { - private final CollectionMergeResolver resolver; - private final Direction direction; - - /// The smallest cursor, tracked separately to improve performance in single-source sections of the trie. - private Cursor head; - - /// Binary heap of the remaining cursors. The smallest element is at position 0. - /// Every element `i` is smaller than or equal to its two children, i.e. - /// ```heap[i] <= heap[i*2 + 1] && heap[i] <= heap[i*2 + 2]``` - private final Cursor[] heap; - - /// A list used to collect contents during [#content()] calls. - private final List contents; - - private CollectionMergeCursor(CollectionMergeResolver resolver, Direction direction, Collection inputs, BiFunction> extractor) - { - this.resolver = resolver; - this.direction = direction; - int count = inputs.size(); - // Get cursors for all inputs. Put one of them in head and the rest in the heap. - heap = new Cursor[count - 1]; - contents = new ArrayList<>(count); - int i = -1; - for (I src : inputs) - { - Cursor cursor = extractor.apply(src, direction); - assert cursor.depth() == 0; - if (i >= 0) - heap[i] = cursor; - else - head = cursor; - ++i; - } - // The cursors are all currently positioned on the root and thus in valid heap order. - } - - public CollectionMergeCursor(CollectionMergeResolver resolver, Direction direction, Collection> inputs) - { - this(resolver, direction, inputs, Trie::cursor); - } - - /// Interface for internal operations that can be applied to selected top elements of the heap. - interface HeapOp - { - void apply(CollectionMergeCursor self, Cursor cursor, int index); - - default boolean shouldContinueWithChild(Cursor child, Cursor head) - { - return equalCursor(child, head); - } - } - - /// Apply a non-interfering operation, i.e. one that does not change the cursor state, to all inputs in the heap - /// that satisfy the [HeapOp#shouldContinueWithChild] condition (by default, being equal to the head). - /// For interfering operations like advancing the cursors, use [#advanceSelectedAndRestoreHeap(AdvancingHeapOp)]. - private void applyToSelectedInHeap(HeapOp action) - { - applyToSelectedElementsInHeap(action, 0); - } - - /// Interface for internal advancing operations that can be applied to the heap cursors. This interface provides - /// the code to restore the heap structure after advancing the cursors. - interface AdvancingHeapOp extends HeapOp - { - void apply(Cursor cursor); - - default void apply(CollectionMergeCursor self, Cursor cursor, int index) - { - // Apply the operation, which should advance the position of the element. - apply(cursor); - - // This method is called on the back path of the recursion. At this point the heaps at both children are - // advanced and well-formed. - // Place current node in its proper position. - self.heapifyDown(cursor, index); - // The heap rooted at index is now advanced and well-formed. - } - } - - - /// Advance the state of all inputs in the heap that satisfy the [#shouldContinueWithChild] condition - /// (by default, being equal to the head) and restore the heap invariant. - private void advanceSelectedAndRestoreHeap(AdvancingHeapOp action) - { - applyToSelectedElementsInHeap(action, 0); - } - - /// Apply an operation to all elements on the heap that satisfy, recursively through the heap hierarchy, the - /// `shouldContinueWithChild` condition (being equal to the head by default). Descends recursively in the - /// heap structure to all selected children and applies the operation on the way back. - /// - /// This operation can be something that does not change the cursor state (see [#content]) or an operation - /// that advances the cursor to a new state, wrapped in a [AdvancingHeapOp] ([#advance] or - /// [#skipTo]). The latter interface takes care of pushing elements down in the heap after advancing - /// and restores the subheap state on return from each level of the recursion. - private void applyToSelectedElementsInHeap(HeapOp action, int index) - { - if (index >= heap.length) - return; - Cursor item = heap[index]; - if (!action.shouldContinueWithChild(item, head)) - return; - - // If the children are at the same position, they also need advancing and their subheap - // invariant to be restored. - applyToSelectedElementsInHeap(action, index * 2 + 1); - applyToSelectedElementsInHeap(action, index * 2 + 2); - - // Apply the action. This is done on the reverse direction to give the action a chance to form proper - // subheaps and combine them on processing the parent. - action.apply(this, item, index); - } - - /// Push the given state down in the heap from the given index until it finds its proper place among - /// the subheap rooted at that position. - private void heapifyDown(Cursor item, int index) - { - while (true) - { - int next = index * 2 + 1; - if (next >= heap.length) - break; - // Select the smaller of the two children to push down to. - if (next + 1 < heap.length && greaterCursor(direction, heap[next], heap[next + 1])) - ++next; - // If the child is greater or equal, the invariant has been restored. - if (!greaterCursor(direction, item, heap[next])) - break; - heap[index] = heap[next]; - index = next; - } - heap[index] = item; - } - - /// Check if the head is greater than the top element in the heap, and if so, swap them and push down the new - /// top until its proper place. - /// @param headDepth the depth of the head cursor (as returned by e.g. advance). - /// @return the new head element's depth - private int maybeSwapHead(int headDepth) - { - int heap0Depth = heap[0].depth(); - if (headDepth > heap0Depth || - (headDepth == heap0Depth && direction.le(head.incomingTransition(), heap[0].incomingTransition()))) - return headDepth; // head is still smallest - - // otherwise we need to swap heap and heap[0] - Cursor newHeap0 = head; - head = heap[0]; - heapifyDown(newHeap0, 0); - return heap0Depth; - } - - boolean branchHasMultipleSources() - { - return equalCursor(heap[0], head); - } - - @Override - public int advance() - { - advanceSelectedAndRestoreHeap(Cursor::advance); - return maybeSwapHead(head.advance()); - } - - @Override - public int advanceMultiple(TransitionsReceiver receiver) - { - // If the current position is present in just one cursor, we can safely descend multiple levels within - // its branch as no one of the other tries has content for it. - if (branchHasMultipleSources()) - return advance(); // More than one source at current position, do single-step advance. - - // If there are no children, i.e. the cursor ascends, we have to check if it's become larger than some - // other candidate. - return maybeSwapHead(head.advanceMultiple(receiver)); - } - - @Override - public int skipTo(int skipDepth, int skipTransition) - { - // We need to advance all cursors that stand before the requested position. - // If a child cursor does not need to advance as it is greater than the skip position, neither of the ones - // below it in the heap hierarchy do as they can't have an earlier position. - class SkipTo implements AdvancingHeapOp - { - @Override - public boolean shouldContinueWithChild(Cursor child, Cursor head) - { - // When the requested position descends, the inplicit prefix bytes are those of the head cursor, - // and thus we need to check against that if it is a match. - if (equalCursor(child, head)) - return true; - // Otherwise we can compare the child's position against a cursor advanced as requested, and need - // to skip only if it would be before it. - int childDepth = child.depth(); - return childDepth > skipDepth || - childDepth == skipDepth && direction.lt(child.incomingTransition(), skipTransition); - } - - @Override - public void apply(Cursor cursor) - { - cursor.skipTo(skipDepth, skipTransition); - } - } - - applyToSelectedElementsInHeap(new SkipTo(), 0); - return maybeSwapHead(head.skipTo(skipDepth, skipTransition)); - } - - @Override - public int depth() - { - return head.depth(); - } - - @Override - public int incomingTransition() - { - return head.incomingTransition(); - } - - @Override - public Direction direction() - { - return direction; - } - - @Override - public ByteComparable.Version byteComparableVersion() - { - return head.byteComparableVersion(); - } - - @Override - public T content() - { - if (!branchHasMultipleSources()) - return head.content(); - - applyToSelectedInHeap(CollectionMergeCursor::collectContent); - collectContent(head, -1); - - T toReturn; - switch (contents.size()) - { - case 0: - toReturn = null; - break; - case 1: - toReturn = contents.get(0); - break; - default: - toReturn = resolver.resolve(contents); - break; - } - contents.clear(); - return toReturn; - } - - private void collectContent(Cursor item, int index) - { - T itemContent = item.content(); - if (itemContent != null) - contents.add(itemContent); - } - - @Override - public Cursor tailCursor(Direction dir) - { - if (!branchHasMultipleSources()) - return head.tailCursor(dir); - - List> inputs = new ArrayList<>(heap.length + 1); - inputs.add(head); - applyToSelectedInHeap((self, cursor, index) -> inputs.add(cursor)); - - return new CollectionMergeCursor<>(resolver, dir, inputs, Cursor::tailCursor); - } - } - - /// Special instance for sources that are guaranteed distinct. The main difference is that we can form unordered - /// value list by concatenating sources. - static class Distinct extends CollectionMergeTrie - { - Distinct(Collection> inputs) - { - super(inputs, Trie.throwingResolver()); - } - - @Override - public Iterable valuesUnordered() - { - return Iterables.concat(Iterables.transform(inputs, Trie::valuesUnordered)); - } - } -} diff --git a/src/java/org/apache/cassandra/db/tries/MergeCursor.java b/src/java/org/apache/cassandra/db/tries/MergeCursor.java new file mode 100644 index 000000000000..d39270872274 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MergeCursor.java @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// A merged view of two trie cursors. +/// +/// This is accomplished by walking the two cursors in parallel; the merged cursor takes the position and features of the +/// smaller and advances with it; when the two cursors are equal, both are advanced. +/// +/// Crucial for the efficiency of this is the fact that when they are advanced like this, we can compare cursors' +/// positions by their `depth` descending and then `incomingTransition` ascending. +/// See [Trie.md](./Trie.md) for further details. +class MergeCursor implements Cursor +{ + private final Trie.MergeResolver resolver; + private final Direction direction; + private final Cursor c1; + private final Cursor c2; + + boolean atC1; + boolean atC2; + + MergeCursor(Trie.MergeResolver resolver, Cursor c1, Cursor c2) + { + this.resolver = resolver; + this.direction = c1.direction(); + this.c1 = c1; + this.c2 = c2; + assert c1.depth() == 0; + assert c2.depth() == 0; + atC1 = atC2 = true; + } + + @Override + public int advance() + { + return checkOrder(atC1 ? c1.advance() : c1.depth(), + atC2 ? c2.advance() : c2.depth()); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + int c1depth = c1.depth(); + int c2depth = c2.depth(); + assert skipDepth <= c1depth + 1 || skipDepth <= c2depth + 1; + if (atC1 || skipDepth < c1depth || skipDepth == c1depth && direction.gt(skipTransition, c1.incomingTransition())) + c1depth = c1.skipTo(skipDepth, skipTransition); + if (atC2 || skipDepth < c2depth || skipDepth == c2depth && direction.gt(skipTransition, c2.incomingTransition())) + c2depth = c2.skipTo(skipDepth, skipTransition); + + return checkOrder(c1depth, c2depth); + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + // While we are on a shared position, we must descend one byte at a time to maintain the cursor ordering. + if (atC1 && atC2) + return checkOrder(c1.advance(), c2.advance()); + + // If we are in a branch that's only covered by one of the sources, we can use its advanceMultiple as it is + // only different from advance if it takes multiple steps down, which does not change the order of the + // cursors. + // Since it might ascend, we still have to check the order after the call. + if (atC1) + return checkOrder(c1.advanceMultiple(receiver), c2.depth()); + else // atC2 + return checkOrder(c1.depth(), c2.advanceMultiple(receiver)); + } + + private int checkOrder(int c1depth, int c2depth) + { + if (c1depth > c2depth) + { + atC1 = true; + atC2 = false; + return c1depth; + } + if (c1depth < c2depth) + { + atC1 = false; + atC2 = true; + return c2depth; + } + // c1depth == c2depth + int c1trans = c1.incomingTransition(); + int c2trans = c2.incomingTransition(); + atC1 = direction.le(c1trans, c2trans); + atC2 = direction.le(c2trans, c1trans); + assert atC1 | atC2; + return c1depth; + } + + @Override + public int depth() + { + return atC1 ? c1.depth() : c2.depth(); + } + + @Override + public int incomingTransition() + { + return atC1 ? c1.incomingTransition() : c2.incomingTransition(); + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + assert c1.byteComparableVersion() == c2.byteComparableVersion() : + "Merging cursors with different byteComparableVersions: " + + c1.byteComparableVersion() + " vs " + c2.byteComparableVersion(); + return c1.byteComparableVersion(); + } + + public T content() + { + T mc = atC2 ? c2.content() : null; + T nc = atC1 ? c1.content() : null; + if (mc == null) + return nc; + else if (nc == null) + return mc; + else + return resolver.resolve(nc, mc); + } + + @Override + public Cursor tailCursor(Direction dir) + { + if (atC1 && atC2) + return new MergeCursor<>(resolver, c1.tailCursor(dir), c2.tailCursor(dir)); + else if (atC1) + return c1.tailCursor(dir); + else if (atC2) + return c2.tailCursor(dir); + else + throw new AssertionError(); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/MergeTrie.java b/src/java/org/apache/cassandra/db/tries/MergeTrie.java deleted file mode 100644 index 96b250900f21..000000000000 --- a/src/java/org/apache/cassandra/db/tries/MergeTrie.java +++ /dev/null @@ -1,202 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.db.tries; - -import com.google.common.collect.Iterables; - -import org.apache.cassandra.utils.bytecomparable.ByteComparable; - -/// A merged view of two tries. -/// -/// This is accomplished by walking the two cursors in parallel; the merged cursor takes the position and features of the -/// smaller and advances with it; when the two cursors are equal, both are advanced. -/// -/// Crucial for the efficiency of this is the fact that when they are advanced like this, we can compare cursors' -/// positions by their depth descending and then incomingTransition ascending. -/// -/// See [Trie.md](./Trie.md) for further details. -class MergeTrie implements Trie -{ - private final MergeResolver resolver; - protected final Trie t1; - protected final Trie t2; - - MergeTrie(MergeResolver resolver, Trie t1, Trie t2) - { - this.resolver = resolver; - this.t1 = t1; - this.t2 = t2; - } - - @Override - public Cursor cursor(Direction direction) - { - return new MergeCursor<>(resolver, t1.cursor(direction), t2.cursor(direction)); - } - - static class MergeCursor implements Cursor - { - private final MergeResolver resolver; - private final Direction direction; - private final Cursor c1; - private final Cursor c2; - - boolean atC1; - boolean atC2; - - MergeCursor(MergeResolver resolver, Cursor c1, Cursor c2) - { - this.resolver = resolver; - this.direction = c1.direction(); - this.c1 = c1; - this.c2 = c2; - assert c1.depth() == 0; - assert c2.depth() == 0; - atC1 = atC2 = true; - } - - @Override - public int advance() - { - return checkOrder(atC1 ? c1.advance() : c1.depth(), - atC2 ? c2.advance() : c2.depth()); - } - - @Override - public int skipTo(int skipDepth, int skipTransition) - { - int c1depth = c1.depth(); - int c2depth = c2.depth(); - assert skipDepth <= c1depth + 1 || skipDepth <= c2depth + 1; - if (atC1 || skipDepth < c1depth || skipDepth == c1depth && direction.gt(skipTransition, c1.incomingTransition())) - c1depth = c1.skipTo(skipDepth, skipTransition); - if (atC2 || skipDepth < c2depth || skipDepth == c2depth && direction.gt(skipTransition, c2.incomingTransition())) - c2depth = c2.skipTo(skipDepth, skipTransition); - - return checkOrder(c1depth, c2depth); - } - - @Override - public int advanceMultiple(TransitionsReceiver receiver) - { - // While we are on a shared position, we must descend one byte at a time to maintain the cursor ordering. - if (atC1 && atC2) - return checkOrder(c1.advance(), c2.advance()); - - // If we are in a branch that's only covered by one of the sources, we can use its advanceMultiple as it is - // only different from advance if it takes multiple steps down, which does not change the order of the - // cursors. - // Since it might ascend, we still have to check the order after the call. - if (atC1) - return checkOrder(c1.advanceMultiple(receiver), c2.depth()); - else // atC2 - return checkOrder(c1.depth(), c2.advanceMultiple(receiver)); - } - - private int checkOrder(int c1depth, int c2depth) - { - if (c1depth > c2depth) - { - atC1 = true; - atC2 = false; - return c1depth; - } - if (c1depth < c2depth) - { - atC1 = false; - atC2 = true; - return c2depth; - } - // c1depth == c2depth - int c1trans = c1.incomingTransition(); - int c2trans = c2.incomingTransition(); - atC1 = direction.le(c1trans, c2trans); - atC2 = direction.le(c2trans, c1trans); - assert atC1 | atC2; - return c1depth; - } - - @Override - public int depth() - { - return atC1 ? c1.depth() : c2.depth(); - } - - @Override - public int incomingTransition() - { - return atC1 ? c1.incomingTransition() : c2.incomingTransition(); - } - - @Override - public Direction direction() - { - return direction; - } - - @Override - public ByteComparable.Version byteComparableVersion() - { - assert c1.byteComparableVersion() == c2.byteComparableVersion() : - "Merging cursors with different byteComparableVersions: " + - c1.byteComparableVersion() + " vs " + c2.byteComparableVersion(); - return c1.byteComparableVersion(); - } - - public T content() - { - T mc = atC2 ? c2.content() : null; - T nc = atC1 ? c1.content() : null; - if (mc == null) - return nc; - else if (nc == null) - return mc; - else - return resolver.resolve(nc, mc); - } - - @Override - public Cursor tailCursor(Direction dir) - { - if (atC1 && atC2) - return new MergeCursor<>(resolver, c1.tailCursor(dir), c2.tailCursor(dir)); - else if (atC1) - return c1.tailCursor(dir); - else if (atC2) - return c2.tailCursor(dir); - else - throw new AssertionError(); - } - } - - /// Special instance for sources that are guaranteed (by the caller) distinct. The main difference is that we can - /// form unordered value list by concatenating sources. - static class Distinct extends MergeTrie - { - Distinct(Trie input1, Trie input2) - { - super(Trie.throwingResolver(), input1, input2); - } - - @Override - public Iterable valuesUnordered() - { - return Iterables.concat(t1.valuesUnordered(), t2.valuesUnordered()); - } - } -} diff --git a/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java b/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java new file mode 100644 index 000000000000..4481ee900698 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +/// Prefixed cursor. Prepends the given prefix to all keys of the supplied cursor. +class PrefixedCursor implements Cursor +{ + final Cursor tail; + ByteSource prefixBytes; + int nextPrefixByte; + int incomingTransition; + int depthOfPrefix; + + PrefixedCursor(ByteComparable prefix, Cursor tail) + { + this(prefix.asComparableBytes(tail.byteComparableVersion()), tail); + } + + PrefixedCursor(ByteSource prefix, Cursor tail) + { + this.tail = tail; + prefixBytes = prefix; + incomingTransition = -1; + nextPrefixByte = prefixBytes.next(); + depthOfPrefix = 0; + } + + int completeAdvanceInTail(int depthInTail) + { + if (depthInTail < 0) + return exhausted(); + + incomingTransition = tail.incomingTransition(); + return depthInTail + depthOfPrefix; + } + + boolean prefixDone() + { + return nextPrefixByte == ByteSource.END_OF_STREAM; + } + + @Override + public int depth() + { + if (prefixDone()) + return tail.depth() + depthOfPrefix; + else + return depthOfPrefix; + } + + @Override + public int incomingTransition() + { + return incomingTransition; + } + + @Override + public int advance() + { + if (prefixDone()) + return completeAdvanceInTail(tail.advance()); + + ++depthOfPrefix; + incomingTransition = nextPrefixByte; + nextPrefixByte = prefixBytes.next(); + return depthOfPrefix; + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + if (prefixDone()) + return completeAdvanceInTail(tail.advanceMultiple(receiver)); + + while (!prefixDone()) + { + receiver.addPathByte(incomingTransition); + ++depthOfPrefix; + incomingTransition = nextPrefixByte; + nextPrefixByte = prefixBytes.next(); + } + return depthOfPrefix; + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + // regardless if we exhausted prefix, if caller asks for depth <= prefix depth, we're done. + if (skipDepth <= depthOfPrefix) + return exhausted(); + if (prefixDone()) + return completeAdvanceInTail(tail.skipTo(skipDepth - depthOfPrefix, skipTransition)); + assert skipDepth == depthOfPrefix + 1 : "Invalid advance request to depth " + skipDepth + " to cursor at depth " + depthOfPrefix; + if (tail.direction().gt(skipTransition, nextPrefixByte)) + return exhausted(); + return advance(); + } + + private int exhausted() + { + incomingTransition = -1; + depthOfPrefix = -1; + nextPrefixByte = 0; // to make prefixDone() false so incomingTransition/depth/content are -1/-1/null + return depthOfPrefix; + } + + public Direction direction() + { + return tail.direction(); + } + + public ByteComparable.Version byteComparableVersion() + { + return tail.byteComparableVersion(); + } + + @Override + public T content() + { + return prefixDone() ? tail.content() : null; + } + + @Override + public Cursor tailCursor(Direction direction) + { + if (prefixDone()) + return tail.tailCursor(direction); + else + { + assert depthOfPrefix >= 0 : "tailTrie called on exhausted cursor"; + if (!(prefixBytes instanceof ByteSource.Duplicatable)) + prefixBytes = ByteSource.duplicatable(prefixBytes); + ByteSource.Duplicatable duplicatableSource = (ByteSource.Duplicatable) prefixBytes; + + return new PrefixedCursor<>(duplicatableSource.duplicate(), tail.tailCursor(direction)); + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/PrefixedTrie.java b/src/java/org/apache/cassandra/db/tries/PrefixedTrie.java deleted file mode 100644 index 0445b2fbe00f..000000000000 --- a/src/java/org/apache/cassandra/db/tries/PrefixedTrie.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.db.tries; - -import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import org.apache.cassandra.utils.bytecomparable.ByteSource; - -/// Prefixed trie. Represents the content of the given trie with the prefix prepended to all keys. -public class PrefixedTrie implements Trie -{ - final ByteComparable prefix; - final Trie trie; - - public PrefixedTrie(ByteComparable prefix, Trie trie) - { - this.prefix = prefix; - this.trie = trie; - } - - @Override - public Cursor cursor(Direction direction) - { - Cursor sourceCursor = trie.cursor(direction); - return new PrefixedCursor<>(prefix.asComparableBytes(sourceCursor.byteComparableVersion()), sourceCursor); - } - - private static class PrefixedCursor implements Cursor - { - final Cursor tail; - ByteSource prefixBytes; - int nextPrefixByte; - int incomingTransition; - int depthOfPrefix; - - PrefixedCursor(ByteSource prefix, Cursor tail) - { - this.tail = tail; - prefixBytes = prefix; - incomingTransition = -1; - nextPrefixByte = prefixBytes.next(); - depthOfPrefix = 0; - } - - int completeAdvanceInTail(int depthInTail) - { - if (depthInTail < 0) - return exhausted(); - - incomingTransition = tail.incomingTransition(); - return depthInTail + depthOfPrefix; - } - - boolean prefixDone() - { - return nextPrefixByte == ByteSource.END_OF_STREAM; - } - - @Override - public int depth() - { - if (prefixDone()) - return tail.depth() + depthOfPrefix; - else - return depthOfPrefix; - } - - @Override - public int incomingTransition() - { - return incomingTransition; - } - - @Override - public int advance() - { - if (prefixDone()) - return completeAdvanceInTail(tail.advance()); - - ++depthOfPrefix; - incomingTransition = nextPrefixByte; - nextPrefixByte = prefixBytes.next(); - return depthOfPrefix; - } - - @Override - public int advanceMultiple(TransitionsReceiver receiver) - { - if (prefixDone()) - return completeAdvanceInTail(tail.advanceMultiple(receiver)); - - while (!prefixDone()) - { - receiver.addPathByte(incomingTransition); - ++depthOfPrefix; - incomingTransition = nextPrefixByte; - nextPrefixByte = prefixBytes.next(); - } - return depthOfPrefix; - } - - @Override - public int skipTo(int skipDepth, int skipTransition) - { - // regardless if we exhausted prefix, if caller asks for depth <= prefix depth, we're done. - if (skipDepth <= depthOfPrefix) - return exhausted(); - if (prefixDone()) - return completeAdvanceInTail(tail.skipTo(skipDepth - depthOfPrefix, skipTransition)); - assert skipDepth == depthOfPrefix + 1 : "Invalid advance request to depth " + skipDepth + " to cursor at depth " + depthOfPrefix; - if (tail.direction().gt(skipTransition, nextPrefixByte)) - return exhausted(); - return advance(); - } - - private int exhausted() - { - incomingTransition = -1; - depthOfPrefix = -1; - nextPrefixByte = 0; // to make prefixDone() false so incomingTransition/depth/content are -1/-1/null - return depthOfPrefix; - } - - public Direction direction() - { - return tail.direction(); - } - - public ByteComparable.Version byteComparableVersion() - { - return tail.byteComparableVersion(); - } - - @Override - public T content() - { - return prefixDone() ? tail.content() : null; - } - - @Override - public Cursor tailCursor(Direction direction) - { - if (prefixDone()) - return tail.tailCursor(direction); - else - { - assert depthOfPrefix >= 0 : "tailTrie called on exhausted cursor"; - if (!(prefixBytes instanceof ByteSource.Duplicatable)) - prefixBytes = ByteSource.duplicatable(prefixBytes); - ByteSource.Duplicatable duplicatableSource = (ByteSource.Duplicatable) prefixBytes; - - return new PrefixedCursor<>(duplicatableSource.duplicate(), tail.tailCursor(direction)); - } - } - } -} diff --git a/src/java/org/apache/cassandra/db/tries/SlicedCursor.java b/src/java/org/apache/cassandra/db/tries/SlicedCursor.java new file mode 100644 index 000000000000..5b0401fa09f1 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/SlicedCursor.java @@ -0,0 +1,409 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +class SlicedCursor implements Cursor +{ + private ByteSource start; + private ByteSource end; + private final Cursor source; + private final Direction direction; + + State state; + int startNext; + int startNextDepth; + int endNext; + int endNextDepth; + + private enum State + { + /** + * The cursor is at the initial phase while it is walking prefixes of both bounds. + * Content is not to be reported. + */ + COMMON_PREFIX, + /** + * The cursor is positioned on some prefix of the start bound, strictly before any prefix of the end bound in + * iteration order. + * Content should only be reported in the reverse direction (as these prefixes are prefixes of the right bound + * and included in the slice). + */ + START_PREFIX, + /** + * The cursor is positioned inside the range, i.e. strictly between any prefixes of the start and end bounds. + * All content should be reported. + */ + INSIDE, + /** + * The cursor is positioned on some prefix of the end bound, strictly after any prefix of the start bound. + * Content should only be reported in the forward direction. + */ + END_PREFIX, + /** + * The cursor is positioned beyond the end bound. Exhaustion (depth -1) has been reported. + */ + EXHAUSTED; + } + + public SlicedCursor(Cursor source, + ByteSource leftSource, + int leftNext, + ByteSource rightSource, + int rightNext) + { + this.source = source; + this.direction = source.direction(); + start = direction.select(leftSource, rightSource); + end = direction.select(rightSource, leftSource); + startNext = direction.select(leftNext, rightNext); + endNext = direction.select(rightNext, leftNext); + startNextDepth = start != null ? 1 : 0; + endNextDepth = end != null ? 1 : 0; + state = start != null + ? end != null + ? State.COMMON_PREFIX + : State.START_PREFIX + : end != null + ? State.END_PREFIX + : State.INSIDE; + } + + @Override + public int advance() + { + int newDepth; + int transition; + + switch (state) + { + case COMMON_PREFIX: + case START_PREFIX: + // Skip any transitions before the start bound + newDepth = source.skipTo(startNextDepth, startNext); + transition = source.incomingTransition(); + return checkBothBounds(newDepth, transition); + case INSIDE: + case END_PREFIX: + newDepth = source.advance(); + transition = source.incomingTransition(); + return checkEndBound(newDepth, transition); + default: + throw new AssertionError(); + } + } + + private int markDone() + { + state = State.EXHAUSTED; + return -1; + } + + int checkBothBounds(int newDepth, int transition) + { + // Check if we are still following the start bound + if (newDepth == startNextDepth && transition == startNext) + { + assert startNext != ByteSource.END_OF_STREAM; + startNext = start.next(); + ++startNextDepth; + State currState = state; + // In the forward direction the exact match for the left bound and all descendant states are + // included in the set. + // In the reverse direction we will instead use the -1 as target transition and thus ascend on + // the next advance (skipping the exact right bound and all its descendants). + if (startNext == ByteSource.END_OF_STREAM && direction.isForward()) + state = State.INSIDE; // checkEndBound may adjust this to END_PREFIX + if (currState == State.START_PREFIX) + return newDepth; // there is no need to check the end bound as we descended along a + // strictly earlier path + } + else // otherwise we are beyond the start bound + state = State.INSIDE; // checkEndBound may adjust this to END_PREFIX + + return checkEndBound(newDepth, transition); + } + + private int checkEndBound(int newDepth, int transition) + { + // Cursor positions compare by depth descending and transition ascending. + if (newDepth > endNextDepth) + return newDepth; // happy and quick path in the interior of the slice + // (state == State.INSIDE can be asserted here (we skip it for efficiency)) + if (newDepth < endNextDepth) + return markDone(); + // newDepth == endDepth + if (direction.lt(transition, endNext)) + { + adjustStateStrictlyBeforeEnd(); + return newDepth; + } + if (direction.lt(endNext, transition)) + return markDone(); + + // Following end bound + endNext = end.next(); + ++endNextDepth; + if (endNext == ByteSource.END_OF_STREAM) + { + // At the exact end bound. + if (direction.isForward()) + { + // In forward direction the right bound is not included in the slice. + return markDone(); + } + else + { + // In reverse, the left bound and all its descendants are included, thus we use the -1 as limiting + // transition. We can also see the bound as strictly ahead of our current position as the current + // branch should be fully included. + adjustStateStrictlyBeforeEnd(); + } + } + else + adjustStateAtEndPrefix(); + return newDepth; + } + + private void adjustStateAtEndPrefix() + { + switch (state) + { + case INSIDE: + state = State.END_PREFIX; + break; + } + } + + private void adjustStateStrictlyBeforeEnd() + { + switch (state) + { + case COMMON_PREFIX: + state = State.START_PREFIX; + break; + case END_PREFIX: + state = State.INSIDE; + break; + } + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + switch (state) + { + case COMMON_PREFIX: + case START_PREFIX: + case END_PREFIX: + return advance(); // descend only one level to be able to compare cursors correctly + case INSIDE: + int depth = source.depth(); + int newDepth = source.advanceMultiple(receiver); + if (newDepth > depth) + return newDepth; // successfully descended + // we ascended, check if we are still within boundaries + return checkEndBound(newDepth, source.incomingTransition()); + default: + throw new AssertionError(); + } + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + // if skipping beyond end, we are done + if (skipDepth < endNextDepth || skipDepth == endNextDepth && direction.gt(skipTransition, endNext)) + return markDone(); + // if skipping before start, adjust request to skip to start + if (skipDepth == startNextDepth && direction.lt(skipTransition, startNext)) + skipTransition = startNext; + + switch (state) + { + case START_PREFIX: + case COMMON_PREFIX: + return checkBothBounds(source.skipTo(skipDepth, skipTransition), source.incomingTransition()); + case INSIDE: + case END_PREFIX: + return checkEndBound(source.skipTo(skipDepth, skipTransition), source.incomingTransition()); + default: + throw new AssertionError("Cursor already exhaused."); + } + } + + @Override + public int depth() + { + return state == State.EXHAUSTED ? -1 : source.depth(); + } + + @Override + public int incomingTransition() + { + return source.incomingTransition(); + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return source.byteComparableVersion(); + } + + @Override + public T content() + { + switch (state) + { + case INSIDE: + return source.content(); + // Additionally, prefixes of the right bound (which are not prefixes of the left) need to be reported: + case START_PREFIX: + // start prefixes in reverse direction (but making sure we don't report the exact match); + return !direction.isForward() && startNext != ByteSource.END_OF_STREAM ? source.content() : null; + case END_PREFIX: + // end prefixes in forward direction. + return direction.isForward() ? source.content() : null; + default: + return null; + } + } + + @Override + public Cursor tailCursor(Direction dir) + { + final Cursor sourceTail = source.tailCursor(dir); + switch (state) + { + case INSIDE: + return sourceTail; + case COMMON_PREFIX: + return maybeSwapBoundsAndMakeCursor(sourceTail, duplicatableStart(), startNext, duplicatableEnd(), endNext, direction); + case START_PREFIX: + return maybeSwapBoundsAndMakeCursor(sourceTail, duplicatableStart(), startNext, null, -1, direction); + case END_PREFIX: + return maybeSwapBoundsAndMakeCursor(sourceTail, null, -1, duplicatableEnd(), endNext, direction); + default: + throw new UnsupportedOperationException("tailCursor on a slice boundary"); + } + } + + private ByteSource.Duplicatable duplicatableStart() + { + if (start == null || start instanceof ByteSource.Duplicatable) + return (ByteSource.Duplicatable) start; + ByteSource.Duplicatable duplicatable = ByteSource.duplicatable(start); + start = duplicatable; + return duplicatable; + } + + private ByteSource.Duplicatable duplicatableEnd() + { + if (end == null || end instanceof ByteSource.Duplicatable) + return (ByteSource.Duplicatable) end; + ByteSource.Duplicatable duplicatable = ByteSource.duplicatable(end); + end = duplicatable; + return duplicatable; + } + + private static Cursor maybeSwapBoundsAndMakeCursor(Cursor source, + ByteSource.Duplicatable startSource, + int startNext, + ByteSource.Duplicatable endSource, + int endNext, + Direction startAndEndDirection) + { + ByteSource.Duplicatable leftSource = startAndEndDirection.select(startSource, endSource); + ByteSource.Duplicatable rightSource = startAndEndDirection.select(endSource, startSource); + int leftNext = startAndEndDirection.select(startNext, endNext); + int rightNext = startAndEndDirection.select(endNext, startNext); + return new SlicedCursor<>(source, + leftSource != null ? leftSource.duplicate() : null, + leftNext, + rightSource != null ? rightSource.duplicate() : null, + rightNext); + } + + + public static Cursor create(Cursor sourceCursor, + ByteComparable left, boolean includeLeft, + ByteComparable right, boolean includeRight) + { + Direction direction = sourceCursor.direction(); + // The cursor is left-inclusive and right-exclusive by default. If we need to change the inclusiveness, adjust + // the bound to the next possible value by adding a 00 byte at the end. + ByteSource leftSource = openAndMaybeAdd0(left, sourceCursor.byteComparableVersion(), !includeLeft); + ByteSource rightSource = openAndMaybeAdd0(right, sourceCursor.byteComparableVersion(), includeRight); + + // Empty left bound is the same as having no left bound, adjust for that. + int leftNext = -1; + if (leftSource != null) + { + leftNext = leftSource.next(); + if (leftNext == ByteSource.END_OF_STREAM) + leftSource = null; + } + + // Empty right bound means the result can only be empty. Make things easier for the cursor by handling this. + int rightNext = -1; + if (rightSource != null) + { + rightNext = rightSource.next(); + if (rightNext == ByteSource.END_OF_STREAM) + { + assert leftSource == null : "Invalid range " + sliceString(sourceCursor.byteComparableVersion(), left, includeLeft, right, includeRight); + return new Trie.EmptyCursor<>(direction, sourceCursor.byteComparableVersion()); + } + } + + return new SlicedCursor<>(sourceCursor, + leftSource, + leftNext, + rightSource, + rightNext); + } + + static String sliceString(ByteComparable.Version version, ByteComparable left, boolean includeLeft, ByteComparable right, boolean includeRight) + { + return String.format("%s%s;%s%s", + includeLeft ? "[" : "(", + left.byteComparableAsString(version), + right.byteComparableAsString(version), + includeRight ? "]" : ")"); + } + + static ByteSource openAndMaybeAdd0(ByteComparable key, ByteComparable.Version byteComparableVersion, boolean shouldAdd0) + { + if (key == null) + return null; + ByteSource src = key.asComparableBytes(byteComparableVersion); + if (shouldAdd0) + return ByteSource.append(src, 0); + else + return src; + } +} diff --git a/src/java/org/apache/cassandra/db/tries/SlicedTrie.java b/src/java/org/apache/cassandra/db/tries/SlicedTrie.java deleted file mode 100644 index 461581a755e6..000000000000 --- a/src/java/org/apache/cassandra/db/tries/SlicedTrie.java +++ /dev/null @@ -1,448 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.db.tries; - -import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import org.apache.cassandra.utils.bytecomparable.ByteSource; - -/** - * Represents a sliced view of a trie, i.e. the content within the given pair of bounds. - * - * Applied by advancing three tries in parallel: the left bound, the source and the right bound. While the source - * bound is smallest, we don't issue any content and skip over any children. As soon as the left bound becomes strictly - * smaller, we stop processing it (as it's a singleton trie it will remain smaller until it's exhausted) and start - * issuing the nodes and content from the source. As soon as the right bound becomes strictly smaller, we finish the - * walk. - * - * We don't explicitly construct tries for the two bounds; tracking the current depth (= prefix length) and transition - * as characters are requested from the key is sufficient as it is a trie with just a single descent path. Because we - * need the next character to tell if it's been exhausted, we keep these one position ahead. The source is always - * advanced, thus this gives us the thing to compare it against after the advance. - * - * We also track the current state to make some decisions a little simpler. - * - * See Trie.md for further details. - */ -public class SlicedTrie implements Trie -{ - private final Trie source; - - /** Left-side boundary. The characters of this are requested as we descend along the left-side boundary. */ - private final ByteComparable left; - - /** Right-side boundary. The characters of this are requested as we descend along the right-side boundary. */ - private final ByteComparable right; - - private final boolean includeLeft; - private final boolean includeRight; - - public SlicedTrie(Trie source, ByteComparable left, boolean includeLeft, ByteComparable right, boolean includeRight) - { - this.source = source; - this.left = left; - this.right = right; - this.includeLeft = includeLeft; - this.includeRight = includeRight; - } - - static ByteSource openAndMaybeAdd0(ByteComparable key, ByteComparable.Version byteComparableVersion, boolean shouldAdd0) - { - if (key == null) - return null; - ByteSource src = key.asComparableBytes(byteComparableVersion); - if (shouldAdd0) - return ByteSource.append(src, 0); - else - return src; - } - - @Override - public Cursor cursor(Direction direction) - { - Cursor sourceCursor = source.cursor(direction); - // The cursor is left-inclusive and right-exclusive by default. If we need to change the inclusiveness, adjust - // the bound to the next possible value by adding a 00 byte at the end. - ByteSource leftSource = openAndMaybeAdd0(left, sourceCursor.byteComparableVersion(), !includeLeft); - ByteSource rightSource = openAndMaybeAdd0(right, sourceCursor.byteComparableVersion(), includeRight); - - // Empty left bound is the same as having no left bound, adjust for that. - int leftNext = -1; - if (leftSource != null) - { - leftNext = leftSource.next(); - if (leftNext == ByteSource.END_OF_STREAM) - leftSource = null; - } - - // Empty right bound means the result can only be empty. Make things easier for the cursor by handling this. - int rightNext = -1; - if (rightSource != null) - { - rightNext = rightSource.next(); - if (rightNext == ByteSource.END_OF_STREAM) - { - assert leftSource == null : "Invalid range " + sliceString(); - return new Trie.EmptyCursor<>(direction, sourceCursor.byteComparableVersion()); - } - } - - return new SlicedCursor<>(sourceCursor, - leftSource, - leftNext, - rightSource, - rightNext); - } - - String sliceString() - { - ByteComparable.Version version = source.cursor(Direction.FORWARD).byteComparableVersion(); - return String.format("%s%s;%s%s", - includeLeft ? "[" : "(", - left.byteComparableAsString(version), - right.byteComparableAsString(version), - includeRight ? "]" : ")"); - } - - private enum State - { - /** - * The cursor is at the initial phase while it is walking prefixes of both bounds. - * Content is not to be reported. - */ - COMMON_PREFIX, - /** - * The cursor is positioned on some prefix of the start bound, strictly before any prefix of the end bound in - * iteration order. - * Content should only be reported in the reverse direction (as these prefixes are prefixes of the right bound - * and included in the slice). - */ - START_PREFIX, - /** - * The cursor is positioned inside the range, i.e. strictly between any prefixes of the start and end bounds. - * All content should be reported. - */ - INSIDE, - /** - * The cursor is positioned on some prefix of the end bound, strictly after any prefix of the start bound. - * Content should only be reported in the forward direction. - */ - END_PREFIX, - /** The cursor is positioned beyond the end bound. Exhaustion (depth -1) has been reported. */ - EXHAUSTED; - } - - private static class SlicedCursor implements Cursor - { - private ByteSource start; - private ByteSource end; - private final Cursor source; - private final Direction direction; - - State state; - int startNext; - int startNextDepth; - int endNext; - int endNextDepth; - - public SlicedCursor(Cursor source, - ByteSource leftSource, - int leftNext, - ByteSource rightSource, - int rightNext) - { - this.source = source; - this.direction = source.direction(); - start = direction.select(leftSource, rightSource); - end = direction.select(rightSource, leftSource); - startNext = direction.select(leftNext, rightNext); - endNext = direction.select(rightNext, leftNext); - startNextDepth = start != null ? 1 : 0; - endNextDepth = end != null ? 1 : 0; - state = start != null - ? end != null - ? State.COMMON_PREFIX - : State.START_PREFIX - : end != null - ? State.END_PREFIX - : State.INSIDE; - } - - @Override - public int advance() - { - int newDepth; - int transition; - - switch (state) - { - case COMMON_PREFIX: - case START_PREFIX: - // Skip any transitions before the start bound - newDepth = source.skipTo(startNextDepth, startNext); - transition = source.incomingTransition(); - return checkBothBounds(newDepth, transition); - case INSIDE: - case END_PREFIX: - newDepth = source.advance(); - transition = source.incomingTransition(); - return checkEndBound(newDepth, transition); - default: - throw new AssertionError(); - } - } - - private int markDone() - { - state = State.EXHAUSTED; - return -1; - } - - int checkBothBounds(int newDepth, int transition) - { - // Check if we are still following the start bound - if (newDepth == startNextDepth && transition == startNext) - { - assert startNext != ByteSource.END_OF_STREAM; - startNext = start.next(); - ++startNextDepth; - State currState = state; - // In the forward direction the exact match for the left bound and all descendant states are - // included in the set. - // In the reverse direction we will instead use the -1 as target transition and thus ascend on - // the next advance (skipping the exact right bound and all its descendants). - if (startNext == ByteSource.END_OF_STREAM && direction.isForward()) - state = State.INSIDE; // checkEndBound may adjust this to END_PREFIX - if (currState == State.START_PREFIX) - return newDepth; // there is no need to check the end bound as we descended along a - // strictly earlier path - } - else // otherwise we are beyond the start bound - state = State.INSIDE; // checkEndBound may adjust this to END_PREFIX - - return checkEndBound(newDepth, transition); - } - - private int checkEndBound(int newDepth, int transition) - { - // Cursor positions compare by depth descending and transition ascending. - if (newDepth > endNextDepth) - return newDepth; // happy and quick path in the interior of the slice - // (state == State.INSIDE can be asserted here (we skip it for efficiency)) - if (newDepth < endNextDepth) - return markDone(); - // newDepth == endDepth - if (direction.lt(transition, endNext)) - { - adjustStateStrictlyBeforeEnd(); - return newDepth; - } - if (direction.lt(endNext, transition)) - return markDone(); - - // Following end bound - endNext = end.next(); - ++endNextDepth; - if (endNext == ByteSource.END_OF_STREAM) - { - // At the exact end bound. - if (direction.isForward()) - { - // In forward direction the right bound is not included in the slice. - return markDone(); - } - else - { - // In reverse, the left bound and all its descendants are included, thus we use the -1 as limiting - // transition. We can also see the bound as strictly ahead of our current position as the current - // branch should be fully included. - adjustStateStrictlyBeforeEnd(); - } - } - else - adjustStateAtEndPrefix(); - return newDepth; - } - - private void adjustStateAtEndPrefix() - { - switch (state) - { - case INSIDE: - state = State.END_PREFIX; - break; - } - } - - private void adjustStateStrictlyBeforeEnd() - { - switch (state) - { - case COMMON_PREFIX: - state = State.START_PREFIX; - break; - case END_PREFIX: - state = State.INSIDE; - break; - } - } - - @Override - public int advanceMultiple(TransitionsReceiver receiver) - { - switch (state) - { - case COMMON_PREFIX: - case START_PREFIX: - case END_PREFIX: - return advance(); // descend only one level to be able to compare cursors correctly - case INSIDE: - int depth = source.depth(); - int newDepth = source.advanceMultiple(receiver); - if (newDepth > depth) - return newDepth; // successfully descended - // we ascended, check if we are still within boundaries - return checkEndBound(newDepth, source.incomingTransition()); - default: - throw new AssertionError(); - } - } - - @Override - public int skipTo(int skipDepth, int skipTransition) - { - // if skipping beyond end, we are done - if (skipDepth < endNextDepth || skipDepth == endNextDepth && direction.gt(skipTransition, endNext)) - return markDone(); - // if skipping before start, adjust request to skip to start - if (skipDepth == startNextDepth && direction.lt(skipTransition, startNext)) - skipTransition = startNext; - - switch (state) - { - case START_PREFIX: - case COMMON_PREFIX: - return checkBothBounds(source.skipTo(skipDepth, skipTransition), source.incomingTransition()); - case INSIDE: - case END_PREFIX: - return checkEndBound(source.skipTo(skipDepth, skipTransition), source.incomingTransition()); - default: - throw new AssertionError("Cursor already exhaused."); - } - } - - @Override - public int depth() - { - return state == State.EXHAUSTED ? -1 : source.depth(); - } - - @Override - public int incomingTransition() - { - return source.incomingTransition(); - } - - @Override - public Direction direction() - { - return direction; - } - - @Override - public ByteComparable.Version byteComparableVersion() - { - return source.byteComparableVersion(); - } - - @Override - public T content() - { - switch (state) - { - case INSIDE: - return source.content(); - // Additionally, prefixes of the right bound (which are not prefixes of the left) need to be reported: - case START_PREFIX: - // start prefixes in reverse direction (but making sure we don't report the exact match); - return !direction.isForward() && startNext != ByteSource.END_OF_STREAM ? source.content() : null; - case END_PREFIX: - // end prefixes in forward direction. - return direction.isForward() ? source.content() : null; - default: - return null; - } - } - - @Override - public Cursor tailCursor(Direction dir) - { - final Cursor sourceTail = source.tailCursor(dir); - switch (state) - { - case INSIDE: - return sourceTail; - case COMMON_PREFIX: - return makeCursor(sourceTail, duplicatableStart(), startNext, duplicatableEnd(), endNext, direction); - case START_PREFIX: - return makeCursor(sourceTail, duplicatableStart(), startNext, null, -1, direction); - case END_PREFIX: - return makeCursor(sourceTail, null, -1, duplicatableEnd(), endNext, direction); - default: - throw new UnsupportedOperationException("tailTrie on a slice boundary"); - } - } - - private ByteSource.Duplicatable duplicatableStart() - { - if (start == null || start instanceof ByteSource.Duplicatable) - return (ByteSource.Duplicatable) start; - ByteSource.Duplicatable duplicatable = ByteSource.duplicatable(start); - start = duplicatable; - return duplicatable; - } - - private ByteSource.Duplicatable duplicatableEnd() - { - if (end == null || end instanceof ByteSource.Duplicatable) - return (ByteSource.Duplicatable) end; - ByteSource.Duplicatable duplicatable = ByteSource.duplicatable(end); - end = duplicatable; - return duplicatable; - } - - - private static Cursor makeCursor(Cursor source, - ByteSource.Duplicatable startSource, - int startNext, - ByteSource.Duplicatable endSource, - int endNext, - Direction startAndEndDirection) - { - ByteSource.Duplicatable leftSource = startAndEndDirection.select(startSource, endSource); - ByteSource.Duplicatable rightSource = startAndEndDirection.select(endSource, startSource); - int leftNext = startAndEndDirection.select(startNext, endNext); - int rightNext = startAndEndDirection.select(endNext, startNext); - return new SlicedCursor<>(source, - leftSource != null ? leftSource.duplicate() : null, - leftNext, - rightSource != null ? rightSource.duplicate() : null, - rightNext); - } - } -} diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java index 19fe9828b909..61c6cc8d6efd 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.java +++ b/src/java/org/apache/cassandra/db/tries/Trie.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db.tries; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import java.util.Collection; import java.util.Iterator; @@ -229,7 +230,7 @@ default Trie subtrie(ByteComparable left, boolean includeLeft, ByteComparable { if (left == null && right == null) return this; - return new SlicedTrie<>(this, left, includeLeft, right, includeRight); + return dir -> SlicedCursor.create(cursor(dir), left, includeLeft, right, includeRight); } /// Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries. @@ -357,7 +358,7 @@ interface MergeResolver /// (The resolver will not be called if there's content from only one source.) default Trie mergeWith(Trie other, MergeResolver resolver) { - return new MergeTrie<>(resolver, this, other); + return dir -> new MergeCursor<>(resolver, this.cursor(dir), other.cursor(dir)); } /// Resolver of content of merged nodes. @@ -378,6 +379,31 @@ default T resolve(T c1, T c2) } } + /// Constructs a view of the merge of multiple tries. The view is live, i.e. any write to any of the + /// sources will be reflected in the merged view. + /// + /// If there is content for a given key in more than one sources, the resolver will be called to obtain the + /// combination. (The resolver will not be called if there's content from only one source.) + static Trie merge(Collection> sources, CollectionMergeResolver resolver) + { + switch (sources.size()) + { + case 0: + throw new AssertionError(); + case 1: + return sources.iterator().next(); + case 2: + { + Iterator> it = sources.iterator(); + Trie t1 = it.next(); + Trie t2 = it.next(); + return t1.mergeWith(t2, resolver); + } + default: + return dir -> new CollectionMergeCursor<>(resolver, dir, sources, Trie::cursor); + } + } + /// Not to be used directly, call [#throwingResolver()] instead. static CollectionMergeResolver THROWING_RESOLVER = new CollectionMergeResolver() { @@ -401,29 +427,26 @@ static CollectionMergeResolver throwingResolver() return (CollectionMergeResolver) THROWING_RESOLVER; } - /// Constructs a view of the merge of multiple tries. The view is live, i.e. any write to any of the - /// sources will be reflected in the merged view. + /// Constructs a view of the merge of two tries, where each source must have distinct keys. The view is live, i.e. + /// any write to any of the sources will be reflected in the merged view. /// - /// If there is content for a given key in more than one sources, the resolver will be called to obtain the - /// combination. (The resolver will not be called if there's content from only one source.) - static Trie merge(Collection> sources, CollectionMergeResolver resolver) + /// If there is content for a given key in more than one sources, the merge will throw an assertion error. + static Trie mergeDistinct(Trie t1, Trie t2) { - switch (sources.size()) + return new Trie() { - case 0: - throw new AssertionError(); - case 1: - return sources.iterator().next(); - case 2: - { - Iterator> it = sources.iterator(); - Trie t1 = it.next(); - Trie t2 = it.next(); - return t1.mergeWith(t2, resolver); - } - default: - return new CollectionMergeTrie<>(sources, resolver); - } + @Override + public Cursor cursor(Direction direction) + { + return new MergeCursor<>(throwingResolver(), t1.cursor(direction), t2.cursor(direction)); + } + + @Override + public Iterable valuesUnordered() + { + return Iterables.concat(t1.valuesUnordered(), t2.valuesUnordered()); + } + }; } /// Constructs a view of the merge of multiple tries, where each source must have distinct keys. The view is live, @@ -443,17 +466,35 @@ static Trie mergeDistinct(Collection> sources) Iterator> it = sources.iterator(); Trie t1 = it.next(); Trie t2 = it.next(); - return new MergeTrie.Distinct<>(t1, t2); + return mergeDistinct(t1, t2); } default: - return new CollectionMergeTrie.Distinct<>(sources); + return mergeDistinctTrie(sources); } } + private static Trie mergeDistinctTrie(Collection> sources) + { + return new Trie() + { + @Override + public Cursor cursor(Direction direction) + { + return new CollectionMergeCursor<>(Trie.throwingResolver(), direction, sources, Trie::cursor); + } + + @Override + public Iterable valuesUnordered() + { + return Iterables.concat(Iterables.transform(sources, Trie::valuesUnordered)); + } + }; + } + /// Returns a Trie that is a view of this one, where the given prefix is prepended before the root. default Trie prefixedBy(ByteComparable prefix) { - return new PrefixedTrie(prefix, this); + return dir -> new PrefixedCursor(prefix, cursor(dir)); } /// Returns an entry set containing all tail tree constructed at the points that contain content of diff --git a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java index 259ffbd07be9..82badafa0657 100644 --- a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java @@ -20,12 +20,14 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Random; import java.util.SortedMap; import java.util.TreeMap; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import org.junit.Test; import org.apache.cassandra.utils.bytecomparable.ByteComparable; @@ -51,7 +53,7 @@ public void testDirect() content1.putAll(content2); // construct directly, trie.merge() will defer to mergeWith on two sources - Trie union = new CollectionMergeTrie<>(ImmutableList.of(trie1, trie2), x -> x.iterator().next()); + Trie union = makeCollectionMergeTrie(trie1, trie2); assertSameContent(union, content1); } @@ -71,11 +73,16 @@ public void testWithDuplicates() addToInMemoryTrie(generateKeys(new Random(5), COUNT), content2, trie2, true); content1.putAll(content2); - Trie union = new CollectionMergeTrie<>(ImmutableList.of(trie1, trie2), x -> x.iterator().next()); + Trie union = makeCollectionMergeTrie(trie1, trie2); assertSameContent(union, content1); } + private static Trie makeCollectionMergeTrie(InMemoryTrie... tries) + { + return dir -> new CollectionMergeCursor<>(x -> x.iterator().next(), dir, List.of(tries), Trie::cursor); + } + @Test public void testDistinct() { @@ -89,11 +96,30 @@ public void testDistinct() InMemoryTrie trie2 = makeInMemoryTrie(src2, content2, true); content1.putAll(content2); - Trie union = new CollectionMergeTrie.Distinct<>(ImmutableList.of(trie1, trie2)); + Trie union = mergeDistinctTrie(ImmutableList.of(trie1, trie2)); assertSameContent(union, content1); } + private static Trie mergeDistinctTrie(Collection> sources) + { + // This duplicates the code in the private Trie.mergeDistinctTrie + return new Trie() + { + @Override + public Cursor cursor(Direction direction) + { + return new CollectionMergeCursor<>(Trie.throwingResolver(), direction, sources, Trie::cursor); + } + + @Override + public Iterable valuesUnordered() + { + return Iterables.concat(Iterables.transform(sources, Trie::valuesUnordered)); + } + }; + } + @Test public void testMultiple() { diff --git a/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java b/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java index 8b9dcfc74f97..d4959d187553 100644 --- a/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.List; import java.util.Random; import java.util.SortedMap; import java.util.TreeMap; @@ -85,7 +86,7 @@ public void testDistinct() InMemoryTrie trie2 = makeInMemoryTrie(src2, content2, true); content1.putAll(content2); - Trie union = new MergeTrie.Distinct<>(trie1, trie2); + Trie union = Trie.mergeDistinct(trie1, trie2); assertSameContent(union, content1); } diff --git a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java index e7fa358339eb..6cb22fab566a 100644 --- a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java @@ -151,7 +151,7 @@ public void testSingletonSubtrie() { int cmp1 = l != null ? ByteComparable.compare(key, l, byteComparableVersion) : 1; int cmp2 = r != null ? ByteComparable.compare(r, key, byteComparableVersion) : 1; - Trie ix = new SlicedTrie<>(Trie.singleton(key, byteComparableVersion, true), l, includeLeft, r, includeRight); + Trie ix = Trie.singleton(key, byteComparableVersion, true).subtrie(l, includeLeft, r, includeRight); boolean expected = true; if (cmp1 < 0 || cmp1 == 0 && !includeLeft) expected = false; From 09a8b568c4ad7517219ad09f74d47bb40b3510e1 Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Fri, 14 Mar 2025 15:21:13 +0200 Subject: [PATCH 03/22] Adds the ability to verify cursors' behaviour for debugging --- .../config/CassandraRelevantProperties.java | 3 + .../db/tries/CollectionMergeCursor.java | 2 +- .../cassandra/db/tries/CursorWalkable.java | 2 +- .../cassandra/db/tries/InMemoryReadTrie.java | 4 +- .../cassandra/db/tries/SingletonCursor.java | 14 +- .../cassandra/db/tries/SlicedCursor.java | 2 +- .../org/apache/cassandra/db/tries/Trie.java | 18 +- .../apache/cassandra/db/tries/TrieDumper.java | 2 +- .../db/tries/VerificationCursor.java | 232 ++++++++++++++++++ .../db/tries/CollectionMergeTrieTest.java | 2 +- .../db/tries/InMemoryTrieTestBase.java | 14 +- .../cassandra/db/tries/SlicedTrieTest.java | 2 +- 12 files changed, 272 insertions(+), 25 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/tries/VerificationCursor.java diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index 436f8448332c..549f63217893 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -284,7 +284,10 @@ public enum CassandraRelevantProperties /** what class to use for mbean registeration */ MBEAN_REGISTRATION_CLASS("org.apache.cassandra.mbean_registration_class"), + /** Size limit for in-memory tries. This determines when the trie will report that it is full to trigger a flush. */ MEMTABLE_TRIE_SIZE_LIMIT("cassandra.trie_size_limit_mb"), + /** To be used for tests: whether trie cursors should be verified for correctness. */ + TRIE_DEBUG("cassandra.debug_tries"), /** This property indicates if the code is running under the in-jvm dtest framework */ DTEST_IS_IN_JVM_DTEST("org.apache.cassandra.dtest.is_in_jvm_dtest"), diff --git a/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java b/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java index 53144a106e38..aa61fd9559d5 100644 --- a/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java +++ b/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java @@ -91,7 +91,7 @@ class CollectionMergeCursor implements Cursor /// A list used to collect contents during [#content()] calls. private final List contents; - public CollectionMergeCursor(Trie.CollectionMergeResolver resolver, Direction direction, Collection inputs, BiFunction> extractor) + CollectionMergeCursor(Trie.CollectionMergeResolver resolver, Direction direction, Collection inputs, BiFunction> extractor) { this.resolver = resolver; this.direction = direction; diff --git a/src/java/org/apache/cassandra/db/tries/CursorWalkable.java b/src/java/org/apache/cassandra/db/tries/CursorWalkable.java index 70cbfe246147..c3317d8930e2 100644 --- a/src/java/org/apache/cassandra/db/tries/CursorWalkable.java +++ b/src/java/org/apache/cassandra/db/tries/CursorWalkable.java @@ -21,7 +21,7 @@ /// representation of the trie. /// /// @param The specific type of cursor a descendant uses. -public interface CursorWalkable +interface CursorWalkable { C cursor(Direction direction); } diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java index 6b6410927fc3..319eb31d4800 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java @@ -1057,7 +1057,7 @@ private boolean isChainNode(int node) return !isNullOrLeaf(node) && offset(node) <= CHAIN_MAX_OFFSET; } - public InMemoryCursor cursor(Direction direction) + public InMemoryCursor makeCursor(Direction direction) { return new InMemoryCursor(direction); } @@ -1100,7 +1100,7 @@ public ByteComparable.Version byteComparableVersion() @Override public String dump(Function contentToString) { - InMemoryCursor source = cursor(Direction.FORWARD); + InMemoryCursor source = makeCursor(Direction.FORWARD); class TypedNodesCursor implements Cursor { @Override diff --git a/src/java/org/apache/cassandra/db/tries/SingletonCursor.java b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java index 10f7b57145ca..21e4a20ea15a 100644 --- a/src/java/org/apache/cassandra/db/tries/SingletonCursor.java +++ b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java @@ -53,7 +53,7 @@ public int advance() } else { - return currentDepth = -1; + return done(); } } @@ -61,7 +61,7 @@ public int advance() public int advanceMultiple(TransitionsReceiver receiver) { if (nextTransition == ByteSource.END_OF_STREAM) - return currentDepth = -1; + return done(); int current = nextTransition; int depth = currentDepth; int next = src.next(); @@ -84,14 +84,20 @@ public int skipTo(int skipDepth, int skipTransition) if (skipDepth <= currentDepth) { assert skipDepth < currentDepth || direction.gt(skipTransition, currentTransition); - return currentDepth = -1; // no alternatives + return done(); // no alternatives } if (direction.gt(skipTransition, nextTransition)) - return currentDepth = -1; // request is skipping over our path + return done(); // request is skipping over our path return advance(); } + private int done() + { + currentTransition = -1; + return currentDepth = -1; + } + @Override public int depth() { diff --git a/src/java/org/apache/cassandra/db/tries/SlicedCursor.java b/src/java/org/apache/cassandra/db/tries/SlicedCursor.java index 5b0401fa09f1..32001805fffd 100644 --- a/src/java/org/apache/cassandra/db/tries/SlicedCursor.java +++ b/src/java/org/apache/cassandra/db/tries/SlicedCursor.java @@ -259,7 +259,7 @@ public int depth() @Override public int incomingTransition() { - return source.incomingTransition(); + return state == State.EXHAUSTED ? -1 : source.incomingTransition(); } @Override diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java index 61c6cc8d6efd..2ac784b21d3f 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.java +++ b/src/java/org/apache/cassandra/db/tries/Trie.java @@ -27,6 +27,7 @@ import java.util.function.Consumer; import java.util.function.Function; import org.agrona.DirectBuffer; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.utils.bytecomparable.ByteComparable; /// Base trie interface. @@ -58,12 +59,14 @@ /// @param The content type of the trie. public interface Trie extends CursorWalkable> { + boolean DEBUG = CassandraRelevantProperties.TRIE_DEBUG.getBoolean(); + /// Adapter interface providing the methods a [Cursor.Walker] to a [Consumer], so that the latter can be used /// with [#process]. /// This enables calls like /// `trie.forEachEntry(x -> System.out.println(x));` /// to be mapped directly to a single call to [#process] without extra allocations. - public interface ValueConsumer extends Consumer, Cursor.Walker + interface ValueConsumer extends Consumer, Cursor.Walker { @Override default void content(T content) @@ -436,7 +439,7 @@ static Trie mergeDistinct(Trie t1, Trie t2) return new Trie() { @Override - public Cursor cursor(Direction direction) + public Cursor makeCursor(Direction direction) { return new MergeCursor<>(throwingResolver(), t1.cursor(direction), t2.cursor(direction)); } @@ -478,7 +481,7 @@ private static Trie mergeDistinctTrie(Collection> sourc return new Trie() { @Override - public Cursor cursor(Direction direction) + public Cursor makeCursor(Direction direction) { return new CollectionMergeCursor<>(Trie.throwingResolver(), direction, sources, Trie::cursor); } @@ -582,4 +585,13 @@ public int incomingTransition() return -1; } } + + Cursor makeCursor(Direction direction); + + @Override + default Cursor cursor(Direction direction) + { + return DEBUG ? new VerificationCursor.Plain<>(makeCursor(direction), 0, 0, -1) + : makeCursor(direction); + } } diff --git a/src/java/org/apache/cassandra/db/tries/TrieDumper.java b/src/java/org/apache/cassandra/db/tries/TrieDumper.java index e09739d9f100..5839e34b0b96 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieDumper.java +++ b/src/java/org/apache/cassandra/db/tries/TrieDumper.java @@ -29,7 +29,7 @@ class TrieDumper implements Cursor.Walker int needsIndent = -1; int currentLength = 0; - public TrieDumper(Function contentToString) + TrieDumper(Function contentToString) { this.contentToString = contentToString; this.b = new StringBuilder(); diff --git a/src/java/org/apache/cassandra/db/tries/VerificationCursor.java b/src/java/org/apache/cassandra/db/tries/VerificationCursor.java new file mode 100644 index 000000000000..697d53d19010 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/VerificationCursor.java @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Arrays; + +import com.google.common.base.Preconditions; + +import org.agrona.DirectBuffer; +import org.apache.cassandra.utils.Hex; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +public interface VerificationCursor +{ + int EXHAUSTED_DEPTH = -1; + int EXHAUSTED_TRANSITION = -1; + int INITIAL_TRANSITION = -1; + + /** + * Verifies: + * - advance does advance, depth <= prevDepth + 1 and transition is higher than previous at the same depth + * (this requires path tracking) + * - skipTo is not called with earlier or equal position (including lower levels) + * - maybeSkipTo is not called with earlier position that can't be identified with depth/incomingTransition only + * (i.e. seeks to lower depth with an incoming transition that lower than the previous at that depth) + * - exhausted state is depth = -1, incomingTransition = -1 (maybe change to 0?) + * - start state is depth = 0, incomingTransition = -1 (maybe change to 0?) + */ + class Plain> implements Cursor, Cursor.TransitionsReceiver + { + final Direction direction; + final C source; + final int minDepth; + int returnedDepth; + int returnedTransition; + byte[] path; + + Cursor.TransitionsReceiver chainedReceiver = null; + boolean advanceMultipleCalledReceiver; + + Plain(C cursor, int minDepth, int expectedDepth, int expectedTransition) + { + this.direction = cursor.direction(); + this.source = cursor; + this.minDepth = minDepth; + this.returnedDepth = expectedDepth; + this.returnedTransition = expectedTransition; + this.path = new byte[16]; + Preconditions.checkState(source.depth() == expectedDepth && source.incomingTransition() == expectedTransition, + "Invalid initial depth %s with incoming transition %s (must be %s, %s)", + source.depth(), source.incomingTransition(), + expectedDepth, expectedTransition); + } + + @Override + public int depth() + { + Preconditions.checkState(returnedDepth == source.depth(), + "Depth changed without advance: %s -> %s", returnedDepth, source.depth()); + return returnedDepth; + } + + @Override + public int incomingTransition() + { + Preconditions.checkState(returnedTransition == source.incomingTransition(), + "Transition changed without advance: %s -> %s", returnedTransition, source.incomingTransition()); + return source.incomingTransition(); + } + + @Override + public T content() + { + return source.content(); + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return source.byteComparableVersion(); + } + + @Override + public int advance() + { + return verify(source.advance()); + } + + @Override + public int advanceMultiple(Cursor.TransitionsReceiver receiver) + { + advanceMultipleCalledReceiver = false; + chainedReceiver = receiver; + int depth = source.advanceMultiple(this); + chainedReceiver = null; + Preconditions.checkState(!advanceMultipleCalledReceiver || depth == returnedDepth + 1, + "advanceMultiple returned depth %s did not match depth %s after added characters", + depth, returnedDepth + 1); + return verify(depth); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + verifySkipRequest(skipDepth, skipTransition); + return verify(source.skipTo(skipDepth, skipTransition)); + } + + private void verifySkipRequest(int skipDepth, int skipTransition) + { + Preconditions.checkState(skipDepth <= returnedDepth + 1, + "Skip descends more than one level: %s -> %s", + returnedDepth, + skipDepth); + if (skipDepth <= returnedDepth && skipDepth > minDepth) + Preconditions.checkState(direction.lt(getByte(skipDepth), skipTransition), + "Skip goes backwards to %s at depth %s where it already visited %s", + skipTransition, skipDepth, getByte(skipDepth)); + + } + + private int verify(int depth) + { + Preconditions.checkState(depth <= returnedDepth + 1, + "Cursor advanced more than one level: %s -> %s", + returnedDepth, + depth); + Preconditions.checkState(depth < 0 || depth > minDepth, + "Cursor ascended to depth %s beyond its minimum depth %s", + depth, minDepth); + final int transition = source.incomingTransition(); + if (depth < 0) + { + Preconditions.checkState(depth == EXHAUSTED_DEPTH && transition == EXHAUSTED_TRANSITION, + "Cursor exhausted state should be %s, %s but was %s, %s", + EXHAUSTED_DEPTH, EXHAUSTED_TRANSITION, + depth, transition); + } + else if (depth <= returnedDepth) + { + Preconditions.checkState(direction.lt(getByte(depth), transition), + "Cursor went backwards to %s at depth %s where it already visited %s", + transition, depth, getByte(depth)); + } + returnedDepth = depth; + returnedTransition = transition; + if (depth >= 0) + addByte(returnedTransition, depth); + return depth; + } + + @Override + public Plain tailCursor(Direction direction) + { + return new Plain<>((C) source.tailCursor(direction), 0, 0, INITIAL_TRANSITION); + } + + + @Override + public void addPathByte(int nextByte) + { + addByte(nextByte, ++returnedDepth); + returnedTransition = nextByte; + if (chainedReceiver != null) + chainedReceiver.addPathByte(nextByte); + } + + private void addByte(int nextByte, int depth) + { + int index = depth - minDepth - 1; + if (index >= path.length) + path = Arrays.copyOf(path, path.length * 2); + path[index] = (byte) nextByte; + } + + private int getByte(int depth) + { + return path[depth - minDepth - 1] & 0xFF; + } + + @Override + public void addPathBytes(DirectBuffer buffer, int pos, int count) + { + for (int i = 0; i < count; ++i) + addByte(buffer.getByte(pos + i), returnedDepth + 1 + i); + returnedDepth += count; + returnedTransition = buffer.getByte(pos + count - 1) & 0xFF; + if (chainedReceiver != null) + chainedReceiver.addPathBytes(buffer, pos, count); + } + + @Override + public String toString() + { + StringBuilder builder = new StringBuilder(); + builder.append(source.getClass().getTypeName() + .replace(source.getClass().getPackageName() + '.', "")); + if (returnedDepth < 0) + { + builder.append(" exhausted"); + } + else + { + builder.append(" at "); + builder.append(Hex.bytesToHex(path, 0, returnedDepth - minDepth)); + } + return builder.toString(); + } + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java index 82badafa0657..43bbd2b6b196 100644 --- a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java @@ -107,7 +107,7 @@ private static Trie mergeDistinctTrie(Collection> sourc return new Trie() { @Override - public Cursor cursor(Direction direction) + public Cursor makeCursor(Direction direction) { return new CollectionMergeCursor<>(Trie.throwingResolver(), direction, sources, Trie::cursor); } diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java index caebe467bb47..2d9a979d0162 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java @@ -242,6 +242,7 @@ public int advance() } if (current == null) { + stack = current; assert depth == -1; return depth; } @@ -283,12 +284,12 @@ public int depth() public ByteBuffer content() { - return (ByteBuffer) stack.content; + return stack != null ? (ByteBuffer) stack.content : null; } public int incomingTransition() { - SpecStackEntry parent = stack.parent; + SpecStackEntry parent = stack != null ? stack.parent : null; return parent != null ? parent.curChild + 0x30 : -1; } @@ -313,14 +314,7 @@ public Cursor tailCursor(Direction dir) static Trie specifiedTrie(Object[] nodeDef) { - return new Trie() - { - @Override - public Cursor cursor(Direction direction) - { - return new CursorFromSpec(nodeDef, direction); - } - }; + return direction -> new CursorFromSpec(nodeDef, direction); } @Test diff --git a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java index 6cb22fab566a..ddd90ebfa141 100644 --- a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java @@ -296,7 +296,7 @@ private static Trie singleLevelIntTrie(int childs) return new Trie() { @Override - public Cursor cursor(Direction direction) + public Cursor makeCursor(Direction direction) { return new singleLevelCursor(direction); } From 635feaad3835beff11725b03c6aeda8ca216e9ea Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Tue, 18 Mar 2025 14:23:46 +0200 Subject: [PATCH 04/22] Put direction argument first in forEach/process --- .../cassandra/db/memtable/TrieMemtable.java | 2 +- .../org/apache/cassandra/db/tries/Cursor.java | 61 ++++++++++++- .../cassandra/db/tries/InMemoryReadTrie.java | 2 +- .../cassandra/db/tries/SlicedCursor.java | 2 +- .../org/apache/cassandra/db/tries/Trie.java | 89 ++++--------------- .../tries/InMemoryTrieReadBench.java | 2 +- .../db/tries/InMemoryTrieTestBase.java | 48 +++++----- 7 files changed, 106 insertions(+), 100 deletions(-) diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java index aac0fe35846e..1fa030bbe1f1 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java @@ -539,7 +539,7 @@ public FlushCollection getFlushSet(PartitionPosition from, Trie toFlush = mergedTrie.subtrie(from, true, to, false); var counter = new KeySizeAndCountCollector(); // need to jump over tails keys - toFlush.processSkippingBranches(counter, Direction.FORWARD); + toFlush.processSkippingBranches(Direction.FORWARD, counter); int partitionCount = counter.keyCount; long partitionKeySize = counter.keySize; diff --git a/src/java/org/apache/cassandra/db/tries/Cursor.java b/src/java/org/apache/cassandra/db/tries/Cursor.java index 24e28d76effb..0e9c752a1e68 100644 --- a/src/java/org/apache/cassandra/db/tries/Cursor.java +++ b/src/java/org/apache/cassandra/db/tries/Cursor.java @@ -113,7 +113,7 @@ /// prefixes will still be reported before their descendants. /// /// Also see [Trie.md](./Trie.md) for further documentation. -public interface Cursor +interface Cursor { /// @return the current descend-depth; 0, if the cursor has just been created and is positioned on the root, /// and -1, if the trie has been exhausted. @@ -254,4 +254,63 @@ interface Walker extends Cursor.ResettingTransitionsReceiver /// Called at the completion of the walk. R complete(); } + + class Empty implements Cursor + { + private final Direction direction; + private final ByteComparable.Version byteComparableVersion; + int depth; + + Empty(Direction direction, ByteComparable.Version byteComparableVersion) + { + this.direction = direction; + this.byteComparableVersion = byteComparableVersion; + depth = 0; + } + + public int advance() + { + return depth = -1; + } + + public int skipTo(int skipDepth, int skipTransition) + { + return depth = -1; + } + + public ByteComparable.Version byteComparableVersion() + { + if (byteComparableVersion != null) + return byteComparableVersion; + throw new AssertionError(); + } + + @Override + public Cursor tailCursor(Direction direction) + { + assert depth == 0 : "tailTrie called on exhausted cursor"; + return new Empty<>(direction, byteComparableVersion); + } + + public int depth() + { + return depth; + } + + public T content() + { + return null; + } + + @Override + public Direction direction() + { + return direction; + } + + public int incomingTransition() + { + return -1; + } + } } diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java index 319eb31d4800..4e2361fc3b1e 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java @@ -1186,7 +1186,7 @@ public String content() return type; } } - return Trie.process(new TrieDumper<>(Function.identity()), new TypedNodesCursor()); + return Trie.process(new TypedNodesCursor(), new TrieDumper<>(Function.identity())); } /// For use in debugging, dump info about the given node. diff --git a/src/java/org/apache/cassandra/db/tries/SlicedCursor.java b/src/java/org/apache/cassandra/db/tries/SlicedCursor.java index 32001805fffd..05823ca746a1 100644 --- a/src/java/org/apache/cassandra/db/tries/SlicedCursor.java +++ b/src/java/org/apache/cassandra/db/tries/SlicedCursor.java @@ -376,7 +376,7 @@ public static Cursor create(Cursor sourceCursor, if (rightNext == ByteSource.END_OF_STREAM) { assert leftSource == null : "Invalid range " + sliceString(sourceCursor.byteComparableVersion(), left, includeLeft, right, includeRight); - return new Trie.EmptyCursor<>(direction, sourceCursor.byteComparableVersion()); + return new Empty<>(direction, sourceCursor.byteComparableVersion()); } } diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java index 2ac784b21d3f..1d74bc56d0ee 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.java +++ b/src/java/org/apache/cassandra/db/tries/Trie.java @@ -102,7 +102,13 @@ default void addPathBytes(DirectBuffer buffer, int pos, int count) /// Call the given consumer on all content values in the trie in order. default void forEachValue(ValueConsumer consumer) { - process(consumer, Direction.FORWARD); + process(Direction.FORWARD, consumer); + } + + /// Call the given consumer on all content values in the trie in order. + default void forEachValue(Direction direction, ValueConsumer consumer) + { + process(direction, consumer); } /// Call the given consumer on all (path, content) pairs with non-null content in the trie in order. @@ -115,18 +121,18 @@ default void forEachEntry(BiConsumer consumer) default void forEachEntry(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); - process(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion()), cursor); + process(cursor, new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); // Note: we can't do the ValueConsumer trick here, because the implementation requires state and cannot be // implemented with default methods alone. } /// Process the trie using the given [Cursor.Walker]. - default R process(Cursor.Walker walker, Direction direction) + default R process(Direction direction, Cursor.Walker walker) { - return process(walker, cursor(direction)); + return process(cursor(direction), walker); } - static R process(Cursor.Walker walker, Cursor cursor) + static R process(Cursor cursor, Cursor.Walker walker) { assert cursor.depth() == 0 : "The provided cursor has already been advanced."; T content = cursor.content(); // handle content on the root node @@ -145,7 +151,7 @@ static R process(Cursor.Walker walker, Cursor cursor) /// Process the trie using the given [ValueConsumer], skipping all branches below the top content-bearing node. default void forEachValueSkippingBranches(Direction direction, ValueConsumer consumer) { - processSkippingBranches(consumer, cursor(direction)); + processSkippingBranches(cursor(direction), consumer); } /// Call the given consumer on all `(path, content)` pairs with non-null content in the trie in order, skipping all @@ -153,18 +159,18 @@ default void forEachValueSkippingBranches(Direction direction, ValueConsumer default void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); - processSkippingBranches(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion()), cursor); + processSkippingBranches(cursor, new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); // Note: we can't do the ValueConsumer trick here, because the implementation requires state and cannot be // implemented with default methods alone. } /// Process the trie using the given [Cursor.Walker], skipping all branches below the top content-bearing node. - default R processSkippingBranches(Cursor.Walker walker, Direction direction) + default R processSkippingBranches(Direction direction, Cursor.Walker walker) { - return processSkippingBranches(walker, cursor(direction)); + return processSkippingBranches(cursor(direction), walker); } - static R processSkippingBranches(Cursor.Walker walker, Cursor cursor) + static R processSkippingBranches(Cursor cursor, Cursor.Walker walker) { assert cursor.depth() == 0 : "The provided cursor has already been advanced."; T content = cursor.content(); // handle content on the root node @@ -208,7 +214,7 @@ default String dump() /// Constuct a textual representation of the trie using the given content-to-string mapper. default String dump(Function contentToString) { - return process(new TrieDumper<>(contentToString), Direction.FORWARD); + return process(Direction.FORWARD, new TrieDumper<>(contentToString)); } /// Returns a singleton trie mapping the given byte path to content. @@ -524,66 +530,7 @@ default Trie tailTrie(ByteComparable prefix) static Trie empty(ByteComparable.Version byteComparableVersion) { - return dir -> new EmptyCursor<>(dir, byteComparableVersion); - } - - class EmptyCursor implements Cursor - { - private final Direction direction; - private final ByteComparable.Version byteComparableVersion; - int depth; - - EmptyCursor(Direction direction, ByteComparable.Version byteComparableVersion) - { - this.direction = direction; - this.byteComparableVersion = byteComparableVersion; - depth = 0; - } - - public int advance() - { - return depth = -1; - } - - public int skipTo(int skipDepth, int skipTransition) - { - return depth = -1; - } - - public ByteComparable.Version byteComparableVersion() - { - if (byteComparableVersion != null) - return byteComparableVersion; - throw new AssertionError(); - } - - @Override - public Cursor tailCursor(Direction direction) - { - assert depth == 0 : "tailTrie called on exhausted cursor"; - return new EmptyCursor<>(direction, byteComparableVersion); - } - - public int depth() - { - return depth; - } - - public T content() - { - return null; - } - - @Override - public Direction direction() - { - return direction; - } - - public int incomingTransition() - { - return -1; - } + return dir -> new Cursor.Empty<>(dir, byteComparableVersion); } Cursor makeCursor(Direction direction); diff --git a/test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieReadBench.java b/test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieReadBench.java index 8004d00554f4..849b8f688ced 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieReadBench.java +++ b/test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieReadBench.java @@ -152,7 +152,7 @@ public Void complete() } } Counter counter = new Counter(); - trie.process(counter, direction); + trie.process(direction, counter); return counter.sum; } diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java index 2d9a979d0162..89b0ecdead56 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java @@ -321,30 +321,30 @@ static Trie specifiedTrie(Object[] nodeDef) public void testEntriesNullChildBug() { Object[] trieDef = new Object[] - { - new Object[] { // 0 - ByteBufferUtil.bytes(1), // 01 - ByteBufferUtil.bytes(2) // 02 - }, - // If requestChild returns null, bad things can happen (DB-2982) - null, // 1 - ByteBufferUtil.bytes(3), // 2 - new Object[] { // 3 - ByteBufferUtil.bytes(4), // 30 - // Also try null on the Remaining.ONE path - null // 31 - }, - ByteBufferUtil.bytes(5), // 4 - // Also test requestUniqueDescendant returning null - new Object[] { // 5 - new Object[] { // 50 - new Object[] { // 500 - null // 5000 - } - } - }, - ByteBufferUtil.bytes(6) // 6 - }; + { + new Object[] { // 0 + ByteBufferUtil.bytes(1), // 01 + ByteBufferUtil.bytes(2) // 02 + }, + // If requestChild returns null, bad things can happen (DB-2982) + null, // 1 + ByteBufferUtil.bytes(3), // 2 + new Object[] { // 3 + ByteBufferUtil.bytes(4), // 30 + // Also try null on the Remaining.ONE path + null // 31 + }, + ByteBufferUtil.bytes(5), // 4 + // Also test requestUniqueDescendant returning null + new Object[] { // 5 + new Object[] { // 50 + new Object[] { // 500 + null // 5000 + } + } + }, + ByteBufferUtil.bytes(6) // 6 + }; SortedMap expected = new TreeMap<>(forwardComparator); expected.put(comparable("00"), ByteBufferUtil.bytes(1)); From 7e57c129f3cba71fbd94f495676ea7a9c253a697 Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Tue, 18 Mar 2025 15:29:18 +0200 Subject: [PATCH 05/22] Extract BaseTrie --- .../apache/cassandra/db/tries/BaseTrie.java | 230 ++++++++++++++++++ .../org/apache/cassandra/db/tries/Trie.java | 204 ++-------------- 2 files changed, 254 insertions(+), 180 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/tries/BaseTrie.java diff --git a/src/java/org/apache/cassandra/db/tries/BaseTrie.java b/src/java/org/apache/cassandra/db/tries/BaseTrie.java new file mode 100644 index 000000000000..27c67d86e268 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/BaseTrie.java @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Iterator; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import org.agrona.DirectBuffer; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// Base trie interface, providing various transformations of the trie, conversion +/// of its content to other formats (e.g. iterable of values), and several forms of processing. +/// +/// For any unimplemented data extraction operations one can build on the [TrieEntriesWalker] (for-each processing) +/// and [TrieEntriesIterator] (to iterator) base classes, which provide the necessary mechanisms to handle walking +/// the trie. +/// +/// See [Trie.md](./Trie.md) for further description of the trie representation model. +/// +/// @param The content type of the trie. +public interface BaseTrie +{ + /// Adapter interface providing the methods a [Cursor.Walker] to a [Consumer], so that the latter can be used + /// with [#process]. + /// This enables calls like + /// `trie.forEachEntry(x -> System.out.println(x));` + /// to be mapped directly to a single call to [#process] without extra allocations. + interface ValueConsumer extends Consumer, Cursor.Walker + { + @Override + default void content(T2 content) + { + accept(content); + } + + @Override + default Void complete() + { + return null; + } + + @Override + default void resetPathLength(int newDepth) + { + // not tracking path + } + + @Override + default void addPathByte(int nextByte) + { + // not tracking path + } + + @Override + default void addPathBytes(DirectBuffer buffer, int pos, int count) + { + // not tracking path + } + } + + /// Call the given consumer on all content values in the trie in order. + default void forEachValue(ValueConsumer consumer) + { + process(Direction.FORWARD, consumer); + } + + /// Call the given consumer on all content values in the trie in order. + default void forEachValue(Direction direction, ValueConsumer consumer) + { + process(direction, consumer); + } + + /// Call the given consumer on all (path, content) pairs with non-null content in the trie in order. + default void forEachEntry(BiConsumer consumer) + { + forEachEntry(Direction.FORWARD, consumer); + } + + /// Call the given consumer on all (path, content) pairs with non-null content in the trie in order. + void forEachEntry(Direction direction, BiConsumer consumer); + + /// Process the trie using the given [Cursor.Walker]. + R process(Direction direction, Cursor.Walker walker); + + /// Process the trie using the given [ValueConsumer], skipping all branches below the top content-bearing node. + default void forEachValueSkippingBranches(Direction direction, ValueConsumer consumer) + { + processSkippingBranches(direction, consumer); + } + + /// Call the given consumer on all `(path, content)` pairs with non-null content in the trie in order, skipping all + /// branches below the top content-bearing node. + void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer); + + /// Process the trie using the given [Cursor.Walker], skipping all branches below the top content-bearing node. + R processSkippingBranches(Direction direction, Cursor.Walker walker); + + /// Map-like get by key. + T get(ByteComparable key); + + /// Constuct a textual representation of the trie. + default String dump() + { + return dump(Object::toString); + } + + /// Constuct a textual representation of the trie using the given content-to-string mapper. + default String dump(Function contentToString) + { + return process(Direction.FORWARD, new TrieDumper<>(contentToString)); + } + + /// Returns the ordered entry set of this trie's content as an iterable. + default Iterable> entrySet() + { + return this::entryIterator; + } + + /// Returns the ordered entry set of this trie's content as an iterable. + default Iterable> entrySet(Direction direction) + { + return () -> entryIterator(direction); + } + + /// Returns the ordered entry set of this trie's content in an iterator. + default Iterator> entryIterator() + { + return entryIterator(Direction.FORWARD); + } + + /// Returns the ordered entry set of this trie's content in an iterator. + Iterator> entryIterator(Direction direction); + + /// Returns the ordered entry set of this trie's content in an iterable, filtered by the given type. + default Iterable> filteredEntrySet(Class clazz) + { + return filteredEntrySet(Direction.FORWARD, clazz); + } + + /// Returns the ordered entry set of this trie's content in an iterable, filtered by the given type. + default Iterable> filteredEntrySet(Direction direction, Class clazz) + { + return () -> filteredEntryIterator(direction, clazz); + } + + /// Returns the ordered entry set of this trie's content in an iterator, filtered by the given type. + Iterator> filteredEntryIterator(Direction direction, Class clazz); + + /// Returns the ordered set of values of this trie as an iterable. + default Iterable values() + { + return this::valueIterator; + } + + /// Returns the ordered set of values of this trie as an iterable. + default Iterable values(Direction direction) + { + return direction.isForward() ? this::valueIterator : this::reverseValueIterator; + } + + /// Returns the ordered set of values of this trie in an iterator. + default Iterator valueIterator() + { + return valueIterator(Direction.FORWARD); + } + + /// Returns the inversely ordered set of values of this trie in an iterator. + default Iterator reverseValueIterator() + { + return valueIterator(Direction.REVERSE); + } + + /// Returns the ordered set of values of this trie in an iterator. + Iterator valueIterator(Direction direction); + + /// Returns the ordered set of values of this trie in an iterable, filtered by the given type. + default Iterable filteredValues(Class clazz) + { + return filteredValues(Direction.FORWARD, clazz); + } + + /// Returns the ordered set of values of this trie in an iterable, filtered by the given type. + default Iterable filteredValues(Direction direction, Class clazz) + { + return () -> filteredValuesIterator(direction, clazz); + } + + /// Returns the ordered set of values of this trie in an iterator, filtered by the given type. + Iterator filteredValuesIterator(Direction direction, Class clazz); + + /// Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries. + /// The view is live, i.e. any write to the source will be reflected in the subtrie. + /// + /// @param left the left bound for the returned subtrie, inclusive. If `null`, the resulting subtrie is not + /// left-bounded. + /// @param right the right bound for the returned subtrie, exclusive. If `null`, the resulting subtrie is not + /// right-bounded. + /// @return a view of the subtrie containing all the keys of this trie falling between `left` inclusively and + /// `right` exclusively. + BaseTrie subtrie(ByteComparable left, ByteComparable right); + + /// Returns a Trie that is a view of this one, where the given prefix is prepended before the root. + BaseTrie prefixedBy(ByteComparable prefix); + + /// Returns a trie that corresponds to the branch of this trie rooted at the given prefix. + /// + /// The result will include the same values as `subtrie(prefix, nextBranch(prefix))`, but the keys in the + /// resulting trie will not include the prefix. In other words, + /// ```tailTrie(prefix).prefixedBy(prefix) = subtrie(prefix, nextBranch(prefix))``` + /// where `nextBranch` stands for the key adjusted by adding one at the last position. + BaseTrie tailTrie(ByteComparable prefix); +} diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java index 1d74bc56d0ee..592144b0320d 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.java +++ b/src/java/org/apache/cassandra/db/tries/Trie.java @@ -24,16 +24,14 @@ import java.util.Iterator; import java.util.Map; import java.util.function.BiConsumer; -import java.util.function.Consumer; -import java.util.function.Function; -import org.agrona.DirectBuffer; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.utils.bytecomparable.ByteComparable; -/// Base trie interface. +/// Basic deterministic trie interface. /// -/// Normal users of tries will only use the public methods, which provide various transformations of the trie, conversion -/// of its content to other formats (e.g. iterable of values), and several forms of processing. +/// Normal users of tries will only use the public methods of [BaseTrie] and this class, which provide various +/// transformations of the trie, conversion of its content to other formats (e.g. iterable of values), and several +/// forms of processing. /// /// For any unimplemented data extraction operations one can build on the [TrieEntriesWalker] (for-each processing) /// and [TrieEntriesIterator] (to iterator) base classes, which provide the necessary mechanisms to handle walking @@ -57,67 +55,11 @@ /// See [Trie.md](./Trie.md) for further description of the trie representation model. /// /// @param The content type of the trie. -public interface Trie extends CursorWalkable> +public interface Trie extends CursorWalkable>, BaseTrie { boolean DEBUG = CassandraRelevantProperties.TRIE_DEBUG.getBoolean(); - /// Adapter interface providing the methods a [Cursor.Walker] to a [Consumer], so that the latter can be used - /// with [#process]. - /// This enables calls like - /// `trie.forEachEntry(x -> System.out.println(x));` - /// to be mapped directly to a single call to [#process] without extra allocations. - interface ValueConsumer extends Consumer, Cursor.Walker - { - @Override - default void content(T content) - { - accept(content); - } - - @Override - default Void complete() - { - return null; - } - - @Override - default void resetPathLength(int newDepth) - { - // not tracking path - } - - @Override - default void addPathByte(int nextByte) - { - // not tracking path - } - - @Override - default void addPathBytes(DirectBuffer buffer, int pos, int count) - { - // not tracking path - } - } - - /// Call the given consumer on all content values in the trie in order. - default void forEachValue(ValueConsumer consumer) - { - process(Direction.FORWARD, consumer); - } - - /// Call the given consumer on all content values in the trie in order. - default void forEachValue(Direction direction, ValueConsumer consumer) - { - process(direction, consumer); - } - - /// Call the given consumer on all (path, content) pairs with non-null content in the trie in order. - default void forEachEntry(BiConsumer consumer) - { - forEachEntry(Direction.FORWARD, consumer); - } - - /// Call the given consumer on all (path, content) pairs with non-null content in the trie in order. + @Override default void forEachEntry(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); @@ -126,7 +68,7 @@ default void forEachEntry(Direction direction, BiConsumer R process(Direction direction, Cursor.Walker walker) { return process(cursor(direction), walker); @@ -148,14 +90,7 @@ static R process(Cursor cursor, Cursor.Walker walker) } - /// Process the trie using the given [ValueConsumer], skipping all branches below the top content-bearing node. - default void forEachValueSkippingBranches(Direction direction, ValueConsumer consumer) - { - processSkippingBranches(cursor(direction), consumer); - } - - /// Call the given consumer on all `(path, content)` pairs with non-null content in the trie in order, skipping all - /// branches below the top content-bearing node. + @Override default void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); @@ -164,7 +99,7 @@ default void forEachEntrySkippingBranches(Direction direction, BiConsumer R processSkippingBranches(Direction direction, Cursor.Walker walker) { return processSkippingBranches(cursor(direction), walker); @@ -195,7 +130,7 @@ static R processSkippingBranches(Cursor cursor, Cursor.Walker wa return walker.complete(); } - /// Map-like get by key. + @Override default T get(ByteComparable key) { Cursor cursor = cursor(Direction.FORWARD); @@ -205,18 +140,6 @@ default T get(ByteComparable key) return null; } - /// Constuct a textual representation of the trie. - default String dump() - { - return dump(Object::toString); - } - - /// Constuct a textual representation of the trie using the given content-to-string mapper. - default String dump(Function contentToString) - { - return process(Direction.FORWARD, new TrieDumper<>(contentToString)); - } - /// Returns a singleton trie mapping the given byte path to content. static Trie singleton(ByteComparable b, ByteComparable.Version byteComparableVersion, T v) { @@ -242,105 +165,31 @@ default Trie subtrie(ByteComparable left, boolean includeLeft, ByteComparable return dir -> SlicedCursor.create(cursor(dir), left, includeLeft, right, includeRight); } - /// Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries. - /// The view is live, i.e. any write to the source will be reflected in the subtrie. - /// - /// @param left the left bound for the returned subtrie, inclusive. If `null`, the resulting subtrie is not - /// left-bounded. - /// @param right the right bound for the returned subtrie, exclusive. If `null`, the resulting subtrie is not - /// right-bounded. - /// @return a view of the subtrie containing all the keys of this trie falling between `left` inclusively and - /// `right` exclusively. + @Override default Trie subtrie(ByteComparable left, ByteComparable right) { return subtrie(left, true, right, false); } - /// Returns the ordered entry set of this trie's content as an iterable. - default Iterable> entrySet() - { - return this::entryIterator; - } - - /// Returns the ordered entry set of this trie's content as an iterable. - default Iterable> entrySet(Direction direction) - { - return () -> entryIterator(direction); - } - - /// Returns the ordered entry set of this trie's content in an iterator. - default Iterator> entryIterator() - { - return entryIterator(Direction.FORWARD); - } - - /// Returns the ordered entry set of this trie's content in an iterator. + @Override default Iterator> entryIterator(Direction direction) { return new TrieEntriesIterator.AsEntries<>(cursor(direction)); } - /// Returns the ordered entry set of this trie's content in an iterable, filtered by the given type. - default Iterable> filteredEntrySet(Class clazz) - { - return filteredEntrySet(Direction.FORWARD, clazz); - } - - /// Returns the ordered entry set of this trie's content in an iterable, filtered by the given type. - default Iterable> filteredEntrySet(Direction direction, Class clazz) - { - return () -> filteredEntryIterator(direction, clazz); - } - - /// Returns the ordered entry set of this trie's content in an iterator, filtered by the given type. + @Override default Iterator> filteredEntryIterator(Direction direction, Class clazz) { return new TrieEntriesIterator.AsEntriesFilteredByType<>(cursor(direction), clazz); } - /// Returns the ordered set of values of this trie as an iterable. - default Iterable values() - { - return this::valueIterator; - } - - /// Returns the ordered set of values of this trie as an iterable. - default Iterable values(Direction direction) - { - return direction.isForward() ? this::valueIterator : this::reverseValueIterator; - } - - /// Returns the ordered set of values of this trie in an iterator. - default Iterator valueIterator() - { - return valueIterator(Direction.FORWARD); - } - - /// Returns the inversely ordered set of values of this trie in an iterator. - default Iterator reverseValueIterator() - { - return valueIterator(Direction.REVERSE); - } - - /// Returns the ordered set of values of this trie in an iterator. + @Override default Iterator valueIterator(Direction direction) { return new TrieValuesIterator<>(cursor(direction)); } - /// Returns the ordered set of values of this trie in an iterable, filtered by the given type. - default Iterable filteredValues(Class clazz) - { - return filteredValues(Direction.FORWARD, clazz); - } - - /// Returns the ordered set of values of this trie in an iterable, filtered by the given type. - default Iterable filteredValues(Direction direction, Class clazz) - { - return () -> filteredValuesIterator(direction, clazz); - } - - /// Returns the ordered set of values of this trie in an iterator, filtered by the given type. + @Override default Iterator filteredValuesIterator(Direction direction, Class clazz) { return new TrieValuesIterator.FilteredByType<>(cursor(direction), clazz); @@ -500,25 +349,13 @@ public Iterable valuesUnordered() }; } - /// Returns a Trie that is a view of this one, where the given prefix is prepended before the root. + @Override default Trie prefixedBy(ByteComparable prefix) { return dir -> new PrefixedCursor(prefix, cursor(dir)); } - /// Returns an entry set containing all tail tree constructed at the points that contain content of - /// the given type. - default Iterable>> tailTries(Direction direction, Class clazz) - { - return () -> new TrieTailsIterator.AsEntries<>(cursor(direction), clazz); - } - - /// Returns a trie that corresponds to the branch of this trie rooted at the given prefix. - /// - /// The result will include the same values as `subtrie(prefix, nextBranch(prefix))`, but the keys in the - /// resulting trie will not include the prefix. In other words, - /// ```tailTrie(prefix).prefixedBy(prefix) = subtrie(prefix, nextBranch(prefix))``` - /// where `nextBranch` stands for the key adjusted by adding one at the last position. + @Override default Trie tailTrie(ByteComparable prefix) { Cursor c = cursor(Direction.FORWARD); @@ -528,6 +365,13 @@ default Trie tailTrie(ByteComparable prefix) return null; } + /// Returns an entry set containing all tail tree constructed at the points that contain content of + /// the given type. + default Iterable>> tailTries(Direction direction, Class clazz) + { + return () -> new TrieTailsIterator.AsEntries<>(cursor(direction), clazz); + } + static Trie empty(ByteComparable.Version byteComparableVersion) { return dir -> new Cursor.Empty<>(dir, byteComparableVersion); From 31ea5db3da6e02110564a77b131c6f084383d94b Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Tue, 18 Mar 2025 16:48:02 +0200 Subject: [PATCH 06/22] Add concrete type to BaseTrie --- .../apache/cassandra/db/tries/BaseTrie.java | 9 ++-- .../org/apache/cassandra/db/tries/Cursor.java | 44 ++++++++++++++++ .../org/apache/cassandra/db/tries/Trie.java | 51 ++----------------- 3 files changed, 54 insertions(+), 50 deletions(-) diff --git a/src/java/org/apache/cassandra/db/tries/BaseTrie.java b/src/java/org/apache/cassandra/db/tries/BaseTrie.java index 27c67d86e268..e74833b941ee 100644 --- a/src/java/org/apache/cassandra/db/tries/BaseTrie.java +++ b/src/java/org/apache/cassandra/db/tries/BaseTrie.java @@ -36,7 +36,8 @@ /// See [Trie.md](./Trie.md) for further description of the trie representation model. /// /// @param The content type of the trie. -public interface BaseTrie +/// @param The concrete subtype of the trie. +public interface BaseTrie> { /// Adapter interface providing the methods a [Cursor.Walker] to a [Consumer], so that the latter can be used /// with [#process]. @@ -215,10 +216,10 @@ default Iterable filteredValues(Direction direction, Class c /// right-bounded. /// @return a view of the subtrie containing all the keys of this trie falling between `left` inclusively and /// `right` exclusively. - BaseTrie subtrie(ByteComparable left, ByteComparable right); + C subtrie(ByteComparable left, ByteComparable right); /// Returns a Trie that is a view of this one, where the given prefix is prepended before the root. - BaseTrie prefixedBy(ByteComparable prefix); + C prefixedBy(ByteComparable prefix); /// Returns a trie that corresponds to the branch of this trie rooted at the given prefix. /// @@ -226,5 +227,5 @@ default Iterable filteredValues(Direction direction, Class c /// resulting trie will not include the prefix. In other words, /// ```tailTrie(prefix).prefixedBy(prefix) = subtrie(prefix, nextBranch(prefix))``` /// where `nextBranch` stands for the key adjusted by adding one at the last position. - BaseTrie tailTrie(ByteComparable prefix); + C tailTrie(ByteComparable prefix); } diff --git a/src/java/org/apache/cassandra/db/tries/Cursor.java b/src/java/org/apache/cassandra/db/tries/Cursor.java index 0e9c752a1e68..bcec07d51f5b 100644 --- a/src/java/org/apache/cassandra/db/tries/Cursor.java +++ b/src/java/org/apache/cassandra/db/tries/Cursor.java @@ -255,6 +255,50 @@ interface Walker extends Cursor.ResettingTransitionsReceiver R complete(); } + /// Process the trie using the given [Walker]. + /// This method should only be called on a freshly constructed cursor. + default R process(Cursor.Walker walker) + { + assert depth() == 0 : "The provided cursor has already been advanced."; + T content = content(); // handle content on the root node + if (content == null) + content = advanceToContent(walker); + + while (content != null) + { + walker.content(content); + content = advanceToContent(walker); + } + return walker.complete(); + } + + /// Process the trie using the given [Walker], skipping over branches where content was found. + /// This method should only be called on a freshly constructed cursor. + default R processSkippingBranches(Cursor.Walker walker) + { + assert depth() == 0 : "The provided cursor has already been advanced."; + T content = content(); // handle content on the root node + if (content != null) + { + walker.content(content); + return walker.complete(); + } + content = advanceToContent(walker); + + while (content != null) + { + walker.content(content); + if (skipTo(depth(), incomingTransition() + direction().increase) < 0) + break; + walker.resetPathLength(depth() - 1); + walker.addPathByte(incomingTransition()); + content = content(); + if (content == null) + content = advanceToContent(walker); + } + return walker.complete(); + } + class Empty implements Cursor { private final Direction direction; diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java index 592144b0320d..7f64655affe7 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.java +++ b/src/java/org/apache/cassandra/db/tries/Trie.java @@ -55,7 +55,7 @@ /// See [Trie.md](./Trie.md) for further description of the trie representation model. /// /// @param The content type of the trie. -public interface Trie extends CursorWalkable>, BaseTrie +public interface Trie extends CursorWalkable>, BaseTrie> { boolean DEBUG = CassandraRelevantProperties.TRIE_DEBUG.getBoolean(); @@ -63,7 +63,7 @@ public interface Trie extends CursorWalkable>, BaseTrie default void forEachEntry(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); - process(cursor, new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); + cursor.process(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); // Note: we can't do the ValueConsumer trick here, because the implementation requires state and cannot be // implemented with default methods alone. } @@ -71,30 +71,14 @@ default void forEachEntry(Direction direction, BiConsumer R process(Direction direction, Cursor.Walker walker) { - return process(cursor(direction), walker); + return cursor(direction).process(walker); } - static R process(Cursor cursor, Cursor.Walker walker) - { - assert cursor.depth() == 0 : "The provided cursor has already been advanced."; - T content = cursor.content(); // handle content on the root node - if (content == null) - content = cursor.advanceToContent(walker); - - while (content != null) - { - walker.content(content); - content = cursor.advanceToContent(walker); - } - return walker.complete(); - } - - @Override default void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); - processSkippingBranches(cursor, new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); + cursor.processSkippingBranches(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); // Note: we can't do the ValueConsumer trick here, because the implementation requires state and cannot be // implemented with default methods alone. } @@ -102,32 +86,7 @@ default void forEachEntrySkippingBranches(Direction direction, BiConsumer R processSkippingBranches(Direction direction, Cursor.Walker walker) { - return processSkippingBranches(cursor(direction), walker); - } - - static R processSkippingBranches(Cursor cursor, Cursor.Walker walker) - { - assert cursor.depth() == 0 : "The provided cursor has already been advanced."; - T content = cursor.content(); // handle content on the root node - if (content != null) - { - walker.content(content); - return walker.complete(); - } - content = cursor.advanceToContent(walker); - - while (content != null) - { - walker.content(content); - if (cursor.skipTo(cursor.depth(), cursor.incomingTransition() + cursor.direction().increase) < 0) - break; - walker.resetPathLength(cursor.depth() - 1); - walker.addPathByte(cursor.incomingTransition()); - content = cursor.content(); - if (content == null) - content = cursor.advanceToContent(walker); - } - return walker.complete(); + return cursor(direction).processSkippingBranches(walker); } @Override From a70a2d2125ad5e5b4d0f7721e107c6984ba1d42c Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Tue, 18 Mar 2025 17:32:43 +0200 Subject: [PATCH 07/22] Add CursorWalkable interface to BaseTrie and move implementations there --- .../apache/cassandra/db/tries/BaseTrie.java | 65 ++++++++++++++---- .../cassandra/db/tries/InMemoryReadTrie.java | 2 +- .../org/apache/cassandra/db/tries/Trie.java | 66 +------------------ 3 files changed, 53 insertions(+), 80 deletions(-) diff --git a/src/java/org/apache/cassandra/db/tries/BaseTrie.java b/src/java/org/apache/cassandra/db/tries/BaseTrie.java index e74833b941ee..e2f03c719f4f 100644 --- a/src/java/org/apache/cassandra/db/tries/BaseTrie.java +++ b/src/java/org/apache/cassandra/db/tries/BaseTrie.java @@ -36,8 +36,8 @@ /// See [Trie.md](./Trie.md) for further description of the trie representation model. /// /// @param The content type of the trie. -/// @param The concrete subtype of the trie. -public interface BaseTrie> +/// @param The concrete subtype of the trie. +public interface BaseTrie, Q extends BaseTrie> extends CursorWalkable { /// Adapter interface providing the methods a [Cursor.Walker] to a [Consumer], so that the latter can be used /// with [#process]. @@ -96,10 +96,19 @@ default void forEachEntry(BiConsumer consumer) } /// Call the given consumer on all (path, content) pairs with non-null content in the trie in order. - void forEachEntry(Direction direction, BiConsumer consumer); + default void forEachEntry(Direction direction, BiConsumer consumer) + { + Cursor cursor = cursor(direction); + cursor.process(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); + // Note: we can't do the ValueConsumer trick here, because the implementation requires state and cannot be + // implemented with default methods alone. + } /// Process the trie using the given [Cursor.Walker]. - R process(Direction direction, Cursor.Walker walker); + default R process(Direction direction, Cursor.Walker walker) + { + return cursor(direction).process(walker); + } /// Process the trie using the given [ValueConsumer], skipping all branches below the top content-bearing node. default void forEachValueSkippingBranches(Direction direction, ValueConsumer consumer) @@ -109,13 +118,29 @@ default void forEachValueSkippingBranches(Direction direction, ValueConsumer /// Call the given consumer on all `(path, content)` pairs with non-null content in the trie in order, skipping all /// branches below the top content-bearing node. - void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer); + default void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer) + { + Cursor cursor = cursor(direction); + cursor.processSkippingBranches(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); + // Note: we can't do the ValueConsumer trick here, because the implementation requires state and cannot be + // implemented with default methods alone. + } /// Process the trie using the given [Cursor.Walker], skipping all branches below the top content-bearing node. - R processSkippingBranches(Direction direction, Cursor.Walker walker); + default R processSkippingBranches(Direction direction, Cursor.Walker walker) + { + return cursor(direction).processSkippingBranches(walker); + } /// Map-like get by key. - T get(ByteComparable key); + default T get(ByteComparable key) + { + Cursor cursor = cursor(Direction.FORWARD); + if (cursor.descendAlong(key.asComparableBytes(cursor.byteComparableVersion()))) + return cursor.content(); + else + return null; + } /// Constuct a textual representation of the trie. default String dump() @@ -148,7 +173,10 @@ default Iterator> entryIterator() } /// Returns the ordered entry set of this trie's content in an iterator. - Iterator> entryIterator(Direction direction); + default Iterator> entryIterator(Direction direction) + { + return new TrieEntriesIterator.AsEntries<>(cursor(direction)); + } /// Returns the ordered entry set of this trie's content in an iterable, filtered by the given type. default Iterable> filteredEntrySet(Class clazz) @@ -163,7 +191,10 @@ default Iterable> filtered } /// Returns the ordered entry set of this trie's content in an iterator, filtered by the given type. - Iterator> filteredEntryIterator(Direction direction, Class clazz); + default Iterator> filteredEntryIterator(Direction direction, Class clazz) + { + return new TrieEntriesIterator.AsEntriesFilteredByType<>(cursor(direction), clazz); + } /// Returns the ordered set of values of this trie as an iterable. default Iterable values() @@ -190,7 +221,10 @@ default Iterator reverseValueIterator() } /// Returns the ordered set of values of this trie in an iterator. - Iterator valueIterator(Direction direction); + default Iterator valueIterator(Direction direction) + { + return new TrieValuesIterator<>(cursor(direction)); + } /// Returns the ordered set of values of this trie in an iterable, filtered by the given type. default Iterable filteredValues(Class clazz) @@ -205,7 +239,10 @@ default Iterable filteredValues(Direction direction, Class c } /// Returns the ordered set of values of this trie in an iterator, filtered by the given type. - Iterator filteredValuesIterator(Direction direction, Class clazz); + default Iterator filteredValuesIterator(Direction direction, Class clazz) + { + return new TrieValuesIterator.FilteredByType<>(cursor(direction), clazz); + } /// Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries. /// The view is live, i.e. any write to the source will be reflected in the subtrie. @@ -216,10 +253,10 @@ default Iterable filteredValues(Direction direction, Class c /// right-bounded. /// @return a view of the subtrie containing all the keys of this trie falling between `left` inclusively and /// `right` exclusively. - C subtrie(ByteComparable left, ByteComparable right); + Q subtrie(ByteComparable left, ByteComparable right); /// Returns a Trie that is a view of this one, where the given prefix is prepended before the root. - C prefixedBy(ByteComparable prefix); + Q prefixedBy(ByteComparable prefix); /// Returns a trie that corresponds to the branch of this trie rooted at the given prefix. /// @@ -227,5 +264,5 @@ default Iterable filteredValues(Direction direction, Class c /// resulting trie will not include the prefix. In other words, /// ```tailTrie(prefix).prefixedBy(prefix) = subtrie(prefix, nextBranch(prefix))``` /// where `nextBranch` stands for the key adjusted by adding one at the last position. - C tailTrie(ByteComparable prefix); + Q tailTrie(ByteComparable prefix); } diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java index 4e2361fc3b1e..f9a1893f68cc 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java @@ -1186,7 +1186,7 @@ public String content() return type; } } - return Trie.process(new TypedNodesCursor(), new TrieDumper<>(Function.identity())); + return new TypedNodesCursor().process(new TrieDumper<>(Function.identity())); } /// For use in debugging, dump info about the given node. diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java index 7f64655affe7..7eaf5048a60b 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.java +++ b/src/java/org/apache/cassandra/db/tries/Trie.java @@ -55,50 +55,10 @@ /// See [Trie.md](./Trie.md) for further description of the trie representation model. /// /// @param The content type of the trie. -public interface Trie extends CursorWalkable>, BaseTrie> +public interface Trie extends BaseTrie, Trie> { boolean DEBUG = CassandraRelevantProperties.TRIE_DEBUG.getBoolean(); - @Override - default void forEachEntry(Direction direction, BiConsumer consumer) - { - Cursor cursor = cursor(direction); - cursor.process(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); - // Note: we can't do the ValueConsumer trick here, because the implementation requires state and cannot be - // implemented with default methods alone. - } - - @Override - default R process(Direction direction, Cursor.Walker walker) - { - return cursor(direction).process(walker); - } - - @Override - default void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer) - { - Cursor cursor = cursor(direction); - cursor.processSkippingBranches(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); - // Note: we can't do the ValueConsumer trick here, because the implementation requires state and cannot be - // implemented with default methods alone. - } - - @Override - default R processSkippingBranches(Direction direction, Cursor.Walker walker) - { - return cursor(direction).processSkippingBranches(walker); - } - - @Override - default T get(ByteComparable key) - { - Cursor cursor = cursor(Direction.FORWARD); - if (cursor.descendAlong(key.asComparableBytes(cursor.byteComparableVersion()))) - return cursor.content(); - else - return null; - } - /// Returns a singleton trie mapping the given byte path to content. static Trie singleton(ByteComparable b, ByteComparable.Version byteComparableVersion, T v) { @@ -130,30 +90,6 @@ default Trie subtrie(ByteComparable left, ByteComparable right) return subtrie(left, true, right, false); } - @Override - default Iterator> entryIterator(Direction direction) - { - return new TrieEntriesIterator.AsEntries<>(cursor(direction)); - } - - @Override - default Iterator> filteredEntryIterator(Direction direction, Class clazz) - { - return new TrieEntriesIterator.AsEntriesFilteredByType<>(cursor(direction), clazz); - } - - @Override - default Iterator valueIterator(Direction direction) - { - return new TrieValuesIterator<>(cursor(direction)); - } - - @Override - default Iterator filteredValuesIterator(Direction direction, Class clazz) - { - return new TrieValuesIterator.FilteredByType<>(cursor(direction), clazz); - } - /// Returns the values in any order. For some tries this is much faster than the ordered iterable. default Iterable valuesUnordered() { From 125b325124c6990d3766b996c994653e63bf9f57 Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Fri, 4 Apr 2025 18:26:28 +0300 Subject: [PATCH 08/22] Run trie tests with verification by default --- .../cassandra/db/tries/CollectionMergeTrieTest.java | 8 ++++++++ .../apache/cassandra/db/tries/InMemoryTrieTestBase.java | 8 ++++++++ .../cassandra/db/tries/InMemoryTrieThreadedTest.java | 2 ++ .../unit/org/apache/cassandra/db/tries/MergeTrieTest.java | 8 ++++++++ .../org/apache/cassandra/db/tries/PrefixTailTrieTest.java | 8 ++++++++ .../org/apache/cassandra/db/tries/SlicedTrieTest.java | 8 ++++++++ 6 files changed, 42 insertions(+) diff --git a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java index 43bbd2b6b196..f7196a67397b 100644 --- a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java @@ -28,8 +28,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import org.junit.BeforeClass; import org.junit.Test; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.utils.bytecomparable.ByteComparable; import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.*; @@ -37,6 +39,12 @@ public class CollectionMergeTrieTest { + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + private static final int COUNT = 15000; Random rand = new Random(); diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java index 89b0ecdead56..974b5347e004 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java @@ -30,10 +30,12 @@ import com.google.common.collect.Lists; import com.google.common.collect.Multiset; import org.junit.Assert; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.io.compress.BufferType; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.bytecomparable.ByteComparable; @@ -45,6 +47,12 @@ @RunWith(Parameterized.class) public abstract class InMemoryTrieTestBase { + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + // Set this to true (in combination with smaller count) to dump the tries while debugging a problem. // Do not commit the code with VERBOSE = true. private static final boolean VERBOSE = false; diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java index 8435772bd067..cd7a1fee21d8 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java @@ -41,6 +41,8 @@ public class InMemoryTrieThreadedTest { + // Note: This should not be run by default with verification to have the higher concurrency of faster writes and reads. + private static final int COUNT = 30000; private static final int OTHERS = COUNT / 10; private static final int PROGRESS_UPDATE = COUNT / 15; diff --git a/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java b/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java index d4959d187553..f6711f331884 100644 --- a/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java @@ -25,14 +25,22 @@ import java.util.SortedMap; import java.util.TreeMap; +import org.junit.BeforeClass; import org.junit.Test; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.utils.bytecomparable.ByteComparable; import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.*; public class MergeTrieTest { + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + private static final int COUNT = 15000; Random rand = new Random(); diff --git a/test/unit/org/apache/cassandra/db/tries/PrefixTailTrieTest.java b/test/unit/org/apache/cassandra/db/tries/PrefixTailTrieTest.java index 1424a8d1f530..ffa50b1d0273 100644 --- a/test/unit/org/apache/cassandra/db/tries/PrefixTailTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/PrefixTailTrieTest.java @@ -32,8 +32,10 @@ import com.google.common.base.Predicates; import com.google.common.collect.Iterables; import com.google.common.primitives.Bytes; +import org.junit.BeforeClass; import org.junit.Test; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.Hex; import org.apache.cassandra.utils.bytecomparable.ByteComparable; @@ -52,6 +54,12 @@ public class PrefixTailTrieTest { + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + private static final int COUNT_TAIL = 5000; private static final int COUNT_HEAD = 25; public static final Comparator BYTE_COMPARABLE_COMPARATOR = (a, b) -> ByteComparable.compare(a, b, byteComparableVersion); diff --git a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java index ddd90ebfa141..3c6768f77449 100644 --- a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java @@ -30,9 +30,11 @@ import java.util.TreeMap; import org.junit.Assert; +import org.junit.BeforeClass; import org.junit.Test; import com.googlecode.concurrenttrees.common.Iterables; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.utils.bytecomparable.ByteComparable; import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.asString; @@ -45,6 +47,12 @@ public class SlicedTrieTest { + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + public static final ByteComparable[] BOUNDARIES = toByteComparable(new String[]{ "test1", "test11", From 3275662182ca5cd385f7db5c875f4451a40867ff Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Mon, 2 Jun 2025 18:24:59 +0300 Subject: [PATCH 09/22] Fix prefixed and singleton tailCursor --- .../cassandra/db/tries/PrefixedCursor.java | 21 +++++++++++++------ .../cassandra/db/tries/SingletonCursor.java | 19 ++++++++++++----- 2 files changed, 29 insertions(+), 11 deletions(-) diff --git a/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java b/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java index 4481ee900698..a46f07988778 100644 --- a/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java +++ b/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java @@ -36,11 +36,16 @@ class PrefixedCursor implements Cursor } PrefixedCursor(ByteSource prefix, Cursor tail) + { + this(prefix.next(), prefix, tail); + } + + PrefixedCursor(int firstPrefixByte, ByteSource prefix, Cursor tail) { this.tail = tail; prefixBytes = prefix; incomingTransition = -1; - nextPrefixByte = prefixBytes.next(); + nextPrefixByte = firstPrefixByte; depthOfPrefix = 0; } @@ -139,6 +144,14 @@ public T content() return prefixDone() ? tail.content() : null; } + ByteSource.Duplicatable duplicateSource() + { + if (!(prefixBytes instanceof ByteSource.Duplicatable)) + prefixBytes = ByteSource.duplicatable(prefixBytes); + ByteSource.Duplicatable duplicatableSource = (ByteSource.Duplicatable) prefixBytes; + return duplicatableSource.duplicate(); + } + @Override public Cursor tailCursor(Direction direction) { @@ -147,11 +160,7 @@ public Cursor tailCursor(Direction direction) else { assert depthOfPrefix >= 0 : "tailTrie called on exhausted cursor"; - if (!(prefixBytes instanceof ByteSource.Duplicatable)) - prefixBytes = ByteSource.duplicatable(prefixBytes); - ByteSource.Duplicatable duplicatableSource = (ByteSource.Duplicatable) prefixBytes; - - return new PrefixedCursor<>(duplicatableSource.duplicate(), tail.tailCursor(direction)); + return new PrefixedCursor<>(nextPrefixByte, duplicateSource(), tail.tailCursor(direction)); } } } diff --git a/src/java/org/apache/cassandra/db/tries/SingletonCursor.java b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java index 21e4a20ea15a..ef6390cbe541 100644 --- a/src/java/org/apache/cassandra/db/tries/SingletonCursor.java +++ b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java @@ -30,16 +30,21 @@ class SingletonCursor implements Cursor private final T value; private int currentDepth = 0; private int currentTransition = -1; - private int nextTransition; + protected int nextTransition; public SingletonCursor(Direction direction, ByteSource src, ByteComparable.Version byteComparableVersion, T value) + { + this(direction, src.next(), src, byteComparableVersion, value); + } + + public SingletonCursor(Direction direction, int firstByte, ByteSource src, ByteComparable.Version byteComparableVersion, T value) { this.src = src; this.direction = direction; this.byteComparableVersion = byteComparableVersion; this.value = value; - this.nextTransition = src.next(); + this.nextTransition = firstByte; } @Override @@ -129,12 +134,16 @@ public ByteComparable.Version byteComparableVersion() } @Override - public SingletonCursor tailCursor(Direction dir) + public SingletonCursor tailCursor(Direction dir) + { + return new SingletonCursor<>(dir, nextTransition, duplicateSource(), byteComparableVersion, value); + } + + ByteSource.Duplicatable duplicateSource() { if (!(src instanceof ByteSource.Duplicatable)) src = ByteSource.duplicatable(src); ByteSource.Duplicatable duplicatableSource = (ByteSource.Duplicatable) src; - - return new SingletonCursor(dir, duplicatableSource.duplicate(), byteComparableVersion, value); + return duplicatableSource.duplicate(); } } From 0579577493f4a3315299888783581fd9b7aeb2be Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Wed, 19 Mar 2025 13:09:59 +0200 Subject: [PATCH 10/22] Implement TrieSet and change slices to use intersection This also changes the behaviour of subtries to always include boundaries, their prefixes and their descendant branches. This is necessary for well-defined reverse walks and helps present metadata on the path of queried ranges, and is not a real limitation for the prefix-free keys that we use. --- .../cassandra/db/memtable/TrieMemtable.java | 2 +- .../db/memtable/TrieMemtableStage1.java | 10 +- .../apache/cassandra/db/tries/BaseTrie.java | 33 +- .../db/tries/CollectionMergeCursor.java | 2 - .../db/tries/IntersectionCursor.java | 219 +++++++ .../cassandra/db/tries/RangesCursor.java | 323 +++++++++++ .../cassandra/db/tries/SlicedCursor.java | 409 -------------- .../org/apache/cassandra/db/tries/Trie.java | 26 +- .../org/apache/cassandra/db/tries/Trie.md | 216 ++++++- .../apache/cassandra/db/tries/TrieSet.java | 84 +++ .../cassandra/db/tries/TrieSetCursor.java | 144 +++++ .../db/tries/TrieSetIntersectionCursor.java | 294 ++++++++++ .../db/tries/TrieSetNegatedCursor.java | 84 +++ .../db/tries/VerificationCursor.java | 121 +++- .../index/sai/memory/TrieMemoryIndex.java | 11 +- .../utils/bytecomparable/ByteComparable.java | 10 - .../cassandra/db/tries/CellReuseTest.java | 23 +- .../db/tries/CollectionMergeTrieTest.java | 64 ++- .../db/tries/InMemoryTriePutTest.java | 41 +- .../db/tries/InMemoryTrieTestBase.java | 300 +++------- .../db/tries/InMemoryTrieThreadedTest.java | 33 +- .../db/tries/IntersectionTrieTest.java | 518 +++++++++++++++++ .../cassandra/db/tries/MergeTrieTest.java | 39 +- .../db/tries/PrefixTailTrieTest.java | 114 ++-- .../cassandra/db/tries/RangesTrieSetTest.java | 420 ++++++++++++++ .../cassandra/db/tries/SlicedTrieTest.java | 315 ++++++----- .../apache/cassandra/db/tries/TrieToDot.java | 123 ++++ .../cassandra/db/tries/TrieToDotTest.java | 40 ++ .../cassandra/db/tries/TrieToMermaid.java | 122 ++++ .../cassandra/db/tries/TrieToMermaidTest.java | 40 ++ .../apache/cassandra/db/tries/TrieUtil.java | 533 ++++++++++++++++++ .../disk/v1/trie/TrieTermsDictionaryTest.java | 2 +- 32 files changed, 3686 insertions(+), 1029 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/tries/IntersectionCursor.java create mode 100644 src/java/org/apache/cassandra/db/tries/RangesCursor.java delete mode 100644 src/java/org/apache/cassandra/db/tries/SlicedCursor.java create mode 100644 src/java/org/apache/cassandra/db/tries/TrieSet.java create mode 100644 src/java/org/apache/cassandra/db/tries/TrieSetCursor.java create mode 100644 src/java/org/apache/cassandra/db/tries/TrieSetIntersectionCursor.java create mode 100644 src/java/org/apache/cassandra/db/tries/TrieSetNegatedCursor.java create mode 100644 test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/RangesTrieSetTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/TrieToDot.java create mode 100644 test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/TrieToMermaid.java create mode 100644 test/unit/org/apache/cassandra/db/tries/TrieToMermaidTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/TrieUtil.java diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java index 1fa030bbe1f1..f290b526eaff 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java @@ -536,7 +536,7 @@ protected void content(Object content, byte[] bytes, int byteLength) public FlushCollection getFlushSet(PartitionPosition from, PartitionPosition to) { - Trie toFlush = mergedTrie.subtrie(from, true, to, false); + Trie toFlush = mergedTrie.subtrie(toComparableBound(from, true), toComparableBound(to, true)); var counter = new KeySizeAndCountCollector(); // need to jump over tails keys toFlush.processSkippingBranches(Direction.FORWARD, counter); diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java index 7f45ff50af7c..6323f0d54e1c 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java @@ -387,7 +387,8 @@ public MemtableUnfilteredPartitionIterator makePartitionIterator(final ColumnFil boolean includeStart = isBound || keyRange instanceof IncludingExcludingBounds; boolean includeStop = isBound || keyRange instanceof Range; - Trie subMap = mergedTrie.subtrie(left, includeStart, right, includeStop); + Trie subMap = mergedTrie.subtrie(toComparableBound(left, includeStart), + toComparableBound(right, !includeStop)); return new MemtableUnfilteredPartitionIterator(metadata(), allocator.ensureOnHeap(), @@ -396,6 +397,11 @@ public MemtableUnfilteredPartitionIterator makePartitionIterator(final ColumnFil dataRange); } + private static ByteComparable toComparableBound(PartitionPosition position, boolean before) + { + return position.isMinimum() ? null : position.asComparableBound(before); + } + public Partition getPartition(DecoratedKey key) { int shardIndex = boundaries.getShardForKey(key); @@ -426,7 +432,7 @@ private static DecoratedKey getPartitionKeyFromPath(TableMetadata metadata, Byte public FlushCollection getFlushSet(PartitionPosition from, PartitionPosition to) { - Trie toFlush = mergedTrie.subtrie(from, true, to, false); + Trie toFlush = mergedTrie.subtrie(toComparableBound(from, true), toComparableBound(to, true)); long keySize = 0; int keyCount = 0; diff --git a/src/java/org/apache/cassandra/db/tries/BaseTrie.java b/src/java/org/apache/cassandra/db/tries/BaseTrie.java index e2f03c719f4f..5984808c564a 100644 --- a/src/java/org/apache/cassandra/db/tries/BaseTrie.java +++ b/src/java/org/apache/cassandra/db/tries/BaseTrie.java @@ -244,16 +244,33 @@ default Iterator filteredValuesIterator(Direction direction, Cl return new TrieValuesIterator.FilteredByType<>(cursor(direction), clazz); } - /// Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries. + /// Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries, + /// inclusive of both bounds and any prefix of the bounds. + /// /// The view is live, i.e. any write to the source will be reflected in the subtrie. /// - /// @param left the left bound for the returned subtrie, inclusive. If `null`, the resulting subtrie is not - /// left-bounded. - /// @param right the right bound for the returned subtrie, exclusive. If `null`, the resulting subtrie is not - /// right-bounded. - /// @return a view of the subtrie containing all the keys of this trie falling between `left` inclusively and - /// `right` exclusively. - Q subtrie(ByteComparable left, ByteComparable right); + /// This method will not check its arguments for correctness. The resulting trie may throw an exception if the right + /// bound is smaller than the left. + /// + /// This package is designed to walk tries efficiently using cursors that necessarily present prefix nodes before + /// children. Lexicographically correct slices (where e.g. the left bound and prefixes of the right are included in + /// the set but prefixes of the left are not) are not contiguous in this representation in both iteration directions + /// (because a prefix of the left bound must necessarily be presented before the left bound itself in reverse order) + /// and are thus not supported. However, if the encoded keys are prefix-free, this limitation is immaterial. + /// + /// @param left the left bound for the returned subtrie. If `null`, the resulting subtrie is not left-bounded. + /// @param right the right bound for the returned subtrie. If `null`, the resulting subtrie is not right-bounded. + /// @return a view of the subtrie containing all the keys of this trie falling between `left` and `right`, + /// including both bounds and any prefix of the bounds. + default Q subtrie(ByteComparable left, ByteComparable right) + { + return intersect(TrieSet.range(cursor(Direction.FORWARD).byteComparableVersion(), left, right)); + } + + /// Returns a view of this trie that is an intersection of its content with the given set. + /// + /// The view is live, i.e. any write to the source will be reflected in the intersection. + Q intersect(TrieSet set); /// Returns a Trie that is a view of this one, where the given prefix is prepended before the root. Q prefixedBy(ByteComparable prefix); diff --git a/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java b/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java index aa61fd9559d5..ad9ea6d2c195 100644 --- a/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java +++ b/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java @@ -18,8 +18,6 @@ package org.apache.cassandra.db.tries; -import com.google.common.collect.Iterables; - import java.util.ArrayList; import java.util.Collection; import java.util.List; diff --git a/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java b/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java new file mode 100644 index 000000000000..3e118a4ded6c --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// The implementation of the intersection of a trie with a set +abstract class IntersectionCursor> implements Cursor +{ + enum State + { + /// The exact position is inside the set, source and set cursors are at the same position. + MATCHING, + /// The set cursor is ahead; the current position, as well as any before the set cursor's are inside the set. + SET_AHEAD + } + + final C source; + final TrieSetCursor set; + final Direction direction; + State state; + + IntersectionCursor(C source, TrieSetCursor set) + { + this.direction = source.direction(); + this.source = source; + this.set = set; + matchingPosition(depth()); + } + + @Override + public int depth() + { + return source.depth(); + } + + @Override + public int incomingTransition() + { + return source.incomingTransition(); + } + + @Override + public int advance() + { + if (state == State.SET_AHEAD) + return advanceInCoveredBranch(set.depth(), source.advance()); + + // The set is assumed sparser, so we advance that first. + int setDepth = set.advance(); + if (set.precedingIncluded()) + return advanceInCoveredBranch(setDepth, source.advance()); + else + return advanceSourceToIntersection(setDepth, set.incomingTransition()); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + if (state == State.SET_AHEAD) + return advanceInCoveredBranch(set.depth(), source.skipTo(skipDepth, skipTransition)); + + int setDepth = set.skipTo(skipDepth, skipTransition); + if (set.precedingIncluded()) + return advanceInCoveredBranch(setDepth, source.skipTo(skipDepth, skipTransition)); + else + return advanceSourceToIntersection(setDepth, set.incomingTransition()); + } + + @Override + public int advanceMultiple(Cursor.TransitionsReceiver receiver) + { + // We can only apply advanceMultiple if we are fully inside a covered branch. + if (state == State.SET_AHEAD) + return advanceInCoveredBranch(set.depth(), source.advanceMultiple(receiver)); + + int setDepth = set.advance(); + if (set.precedingIncluded()) + return advanceInCoveredBranch(setDepth, source.advance()); + else + return advanceSourceToIntersection(setDepth, set.incomingTransition()); + } + + private int advanceInCoveredBranch(int setDepth, int sourceDepth) + { + // Check if the advanced source is still in the covered area. + if (sourceDepth > setDepth) // most common fast path + return coveredAreaWithSetAhead(sourceDepth); + if (sourceDepth < 0) + return exhausted(); + + int sourceTransition = source.incomingTransition(); + if (sourceDepth == setDepth) + { + int setTransition = set.incomingTransition(); + if (direction.lt(sourceTransition, setTransition)) + return coveredAreaWithSetAhead(sourceDepth); + if (sourceTransition == setTransition) + return matchingPosition(sourceDepth); + } + + // Source moved beyond the set position. Advance the set too. + setDepth = set.skipTo(sourceDepth, sourceTransition); + int setTransition = set.incomingTransition(); + if (setDepth == sourceDepth && setTransition == sourceTransition) + return matchingPosition(sourceDepth); + + // At this point set is ahead. Check content to see if we are in a covered branch. + // If not, we need to skip the source as well and repeat the process. + if (set.precedingIncluded()) + return coveredAreaWithSetAhead(sourceDepth); + else + return advanceSourceToIntersection(setDepth, setTransition); + } + + private int advanceSourceToIntersection(int setDepth, int setTransition) + { + while (true) + { + // Set is ahead of source, but outside the covered area. Skip source to the set's position. + int sourceDepth = source.skipTo(setDepth, setTransition); + int sourceTransition = source.incomingTransition(); + if (sourceDepth < 0) + return exhausted(); + if (sourceDepth == setDepth && sourceTransition == setTransition) + return matchingPosition(setDepth); + + // Source is now ahead of the set. + setDepth = set.skipTo(sourceDepth, sourceTransition); + setTransition = set.incomingTransition(); + if (setDepth == sourceDepth && setTransition == sourceTransition) + return matchingPosition(setDepth); + + // At this point set is ahead. Check content to see if we are in a covered branch. + if (set.precedingIncluded()) + return coveredAreaWithSetAhead(sourceDepth); + } + } + + private int coveredAreaWithSetAhead(int depth) + { + state = State.SET_AHEAD; + return depth; + } + + private int matchingPosition(int depth) + { + // If we are matching a boundary of the set, include all its children by using a set-ahead state, ensuring that + // the set will only be advanced once the source ascends to its depth again. + if (set.branchIncluded()) + state = State.SET_AHEAD; + else + state = State.MATCHING; + return depth; + } + + private int exhausted() + { + state = State.MATCHING; + return -1; + } + + @Override + public T content() + { + return source.content(); + } + + @Override + public Direction direction() + { + return source.direction(); + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return source.byteComparableVersion(); + } + + /// Intersection cursor for [Trie]. + static class Plain extends IntersectionCursor> + { + public Plain(Cursor source, TrieSetCursor set) + { + super(source, set); + } + + @Override + public Cursor tailCursor(Direction direction) + { + switch (state) + { + case MATCHING: + return new Plain<>(source.tailCursor(direction), set.tailCursor(direction)); + case SET_AHEAD: + return source.tailCursor(direction); + default: + throw new AssertionError(); + } + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/RangesCursor.java b/src/java/org/apache/cassandra/db/tries/RangesCursor.java new file mode 100644 index 000000000000..8c59b2b6ff84 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/RangesCursor.java @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Arrays; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +/// A cursor for a [TrieSet] that represents a set of ranges. +/// +/// Ranges in this trie design always include all prefixes and all descendants of the start and end points. That is, +/// the range of ("abc", "afg") includes "a", "ab", "abc", "af, "afg", as well as any string staring with "abc", "ac", +/// "ad", "ae", "afg". The range of ("abc", "abc") includes "a", "ab", "abc" and any string starting with "abc". +/// +/// Ranges that contain prefixes (e.g. ("ab", "abc")) are invalid as they cannot be specified without violating order in +/// one of the directions (i.e. "ab" is before "abc" in forward order, but not after it in reverse). +/// The reason for these restrictions is to ensure that all individual ranges are contiguous spans when iterated in +/// forward as well as backward order. This in turn makes it possible to have simple range trie and intersection +/// implementations where the representations of range trie sets do not differ when iterated in the two directions. +/// +/// Thes types of ranges are actually preferable to us, because we use prefix-free keys with terminators that leave +/// room for greater- and less-than positions, and at prefix nodes we store metadata applicable to the whole branch. +/// +/// The ranges are specified by passing a sequence of boundaries, where each even boundary is the opening boundary, and +/// the odd ones are the closing boundary for a range. The boundaries must be in order and cannot overlap, but it is +/// possible to repeat a boundary, e.g. +/// - `[a, a] == point a`: an even number of repeats starting at an even position specifies a point +/// (i.e. the set of all prefixes and all descendants of that key) +/// - `[a, b, b, c] == [a, c]`: an even number of repeats which starts at an odd position is ignored +/// - `[a, a, a, b] == [a, b]`: an odd number of repeats is the same as a single copy +/// +/// If the first boundary is null, the range is open on the left, and if the last non-null boundary is on an even +/// position (i.e. if the array has an even length and its last boundary is null, or if that last boundary is cut off), +/// the range is open on the right: +/// - `[null, a]` covers all keys smaller than `a`, plus the prefixes and descendants of `a` +/// - `[a, null]` or `[a]` covers all keys greater than `a`, plus the prefixes and descendants of `a` +class RangesCursor implements TrieSetCursor +{ + private final ByteComparable.Version byteComparableVersion; + private final Direction direction; + + /// The current index in the boundaries array. This is the input that will be advanced on the next [#advance] call. + /// When the current boundary is exhausted, it will advance (in the direction of the cursor). When it surpasses + /// [#completedIdx], the cursor is exhausted. + private int currentIdx; + /// The index at which the ranges end. Depending on the direction of iteration, the end or start of the array, + /// adjusted to remove null boundaries. + private final int completedIdx; + /// The next byte for all boundaries. + int[] nexts; + /// The depth (processed number of bytes) for all boundaries. + int[] depths; + /// Byte sources producing the rest of the bytes of the boundaries. + ByteSource[] sources; + /// The currently reached depth (reported to the user). This is usually `depths[currentIdx] - 1`. + int currentDepth; + /// The current incoming transition. + int currentTransition; + /// Current range state, returned by [#state]. + RangeState currentState; + + public RangesCursor(Direction direction, ByteComparable.Version byteComparableVersion, ByteComparable... boundaries) + { + this.byteComparableVersion = byteComparableVersion; + this.direction = direction; + // handle empty array (== full range) and nulls at the end (same as not there, odd length == open end range) + int length = boundaries.length; + if (length == 0) + { + boundaries = new ByteComparable[]{ null }; + length = 1; + } + if (length > 1 && boundaries[length - 1] == null) + --length; + + nexts = new int[length]; + depths = new int[length]; + sources = new ByteSource[length]; + int first = 0; + if (boundaries[0] == null) + { + first = 1; + sources[0] = null; + nexts[0] = ByteSource.END_OF_STREAM; + } + currentIdx = direction.select(first, length - 1); + for (int i = first; i < length; ++i) + { + depths[i] = 1; + if (boundaries[i] != null) + { + sources[i] = boundaries[i].asComparableBytes(byteComparableVersion); + nexts[i] = sources[i].next(); + } + else + throw new AssertionError("Null can only be used as the first or last boundary."); + } + currentDepth = 0; + currentTransition = -1; + completedIdx = direction.select(length - 1, first); + // If this cursor is already exhausted (i.e. it is a [null, null] range), use 0 as next character to not report + // a boundary at the root. + skipCompletedAndSelectContained(direction.le(currentIdx, completedIdx) ? nexts[currentIdx] : 0, + completedIdx); + } + + private RangesCursor(Direction direction, + ByteComparable.Version byteComparableVersion, + int[] nexts, + int[] depths, + ByteSource[] sources, + int firstIdxInclusive, + int lastIdxExclusive, + int currentDepth, + int currentTransition, + RangeState currentState) + { + this.byteComparableVersion = byteComparableVersion; + this.direction = direction; + this.nexts = nexts; + this.depths = depths; + this.sources = sources; + this.currentIdx = direction.select(firstIdxInclusive, lastIdxExclusive - 1); + this.completedIdx = direction.select(lastIdxExclusive - 1, firstIdxInclusive); + this.currentDepth = currentDepth; + this.currentTransition = currentTransition; + this.currentState = currentState; + } + + @Override + public int depth() + { + return currentDepth; + } + + @Override + public int incomingTransition() + { + return currentTransition; + } + + @Override + public RangeState state() + { + return currentState; + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return byteComparableVersion; + } + + @Override + public int advance() + { + if (direction.gt(currentIdx, completedIdx)) + return exhausted(); + + // Advance the current idx + currentTransition = nexts[currentIdx]; + currentDepth = depths[currentIdx]++; + int next = ByteSource.END_OF_STREAM; + if (currentTransition != ByteSource.END_OF_STREAM) + next = nexts[currentIdx] = sources[currentIdx].next(); + + // Also advance all others that are at the same position and have the same next byte. + int endIdx = currentIdx + direction.increase; + while (direction.le(endIdx, completedIdx) + && depths[endIdx] == currentDepth && nexts[endIdx] == currentTransition) + { + depths[endIdx]++; + nexts[endIdx] = sources[endIdx].next(); + endIdx += direction.increase; + } + + return skipCompletedAndSelectContained(next, endIdx - direction.increase); + } + + private int skipCompletedAndSelectContained(int next, int endIdx) + { + int containedSelection = 0; + // in reverse direction the roles of current and end idx are swapped + containedSelection |= (direction.select(currentIdx, endIdx) & 1); // even left index means not valid before + containedSelection |= ((direction.select(endIdx, currentIdx) & 1) ^ 1) << 1; // even end index means not valid after + if (next == ByteSource.END_OF_STREAM) + { + containedSelection |= 4; // exact match, point and children included; reportable node + while (direction.le(currentIdx, endIdx)) + { + assert nexts[currentIdx] == ByteSource.END_OF_STREAM : "Prefixes are not allowed in trie ranges."; + currentIdx += direction.increase; + } + } + currentState = RangeState.values()[containedSelection]; + return currentDepth; + } + + // Note: Sets don't need `advanceMultiple` because they are meant to apply as a restriction on other tries, + // and the combined walks necessary to implement such restrictions can only proceed one step at a time. + // Once the restriction identifies that a branch in covered by the set, it can use the trie's `advanceMultiple` + // method. + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + while (direction.le(currentIdx, completedIdx) + && (depths[currentIdx] > skipDepth || + depths[currentIdx] == skipDepth && direction.lt(nexts[currentIdx], skipTransition))) + currentIdx += direction.increase; + return advance(); + } + + private int exhausted() + { + currentDepth = -1; + currentTransition = -1; + return skipCompletedAndSelectContained(0, completedIdx); + } + + @Override + public TrieSetCursor tailCursor(Direction direction) + { + return tailCopyOf(this, direction); + } + + private static RangesCursor tailCopyOf(RangesCursor copyFrom, Direction newDirection) + { + Direction copyDirection = copyFrom.direction; + int startInclusive = copyFrom.currentIdx; + int endExclusive = findEndOfMatchingValues(copyDirection, startInclusive, copyFrom.completedIdx, copyFrom.depths, copyFrom.currentDepth); + + if (startInclusive == endExclusive) + return boundaryMatchingCursor(copyFrom, newDirection); + + int endInclusive = endExclusive - copyDirection.increase; + int firstInclusive = copyDirection.select(startInclusive, endInclusive); + int lastExclusive = copyDirection.select(endInclusive, startInclusive) + 1; + + // Duplicate all boundaries that are positioned at the current point (if they are not, they cannot affect the + // tail trie). + // From the left we can only drop an even number of boundaries. + int firstAdjusted = firstInclusive & -2; + ByteSource[] sources = new ByteSource[lastExclusive - firstAdjusted]; + final int[] depths = Arrays.copyOfRange(copyFrom.depths, firstAdjusted, lastExclusive); + final int startDepth = copyFrom.currentDepth; + for (int i = firstInclusive; i < lastExclusive; ++i) + { + if (copyFrom.sources[i] != null) + { + ByteSource.Duplicatable dupe = ByteSource.duplicatable(copyFrom.sources[i]); + copyFrom.sources[i] = dupe; + sources[i - firstAdjusted] = dupe.duplicate(); + } + depths[i - firstAdjusted] -= startDepth; + } + return new RangesCursor(newDirection, + copyFrom.byteComparableVersion, + Arrays.copyOfRange(copyFrom.nexts, firstAdjusted, lastExclusive), + depths, + sources, + firstInclusive - firstAdjusted, + lastExclusive - firstAdjusted, + 0, + -1, + copyFrom.currentState); + } + + private static int findEndOfMatchingValues(Direction direction, int startInclusive, int completedInclusive, int[] depths, int currentDepth) + { + int endExclusive; + for (endExclusive = startInclusive; + direction.le(endExclusive, completedInclusive) && depths[endExclusive] > currentDepth; + endExclusive += direction.increase) + { + } + return endExclusive; + } + + private static RangesCursor boundaryMatchingCursor(RangesCursor copyFrom, Direction newDirection) + { + // There are no further ranges to follow. The current state is the only thing we need to present, but + // we need to make sure we present the right final state after advancing over the current state. + // Prepare a combination of current and completed index that produces true or false precedingIncluded() result + // on the exhausted() call. + final RangeState state = copyFrom.currentState; + // This gives us the included/excluded state after the current position. + int currentIdx = state.precedingIncluded(newDirection.opposite()) ? 1 : 0; + + return new RangesCursor(newDirection, + copyFrom.byteComparableVersion, + new int[0], + new int[0], + new ByteSource[0], + currentIdx, + currentIdx, + 0, + -1, + state); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/SlicedCursor.java b/src/java/org/apache/cassandra/db/tries/SlicedCursor.java deleted file mode 100644 index 05823ca746a1..000000000000 --- a/src/java/org/apache/cassandra/db/tries/SlicedCursor.java +++ /dev/null @@ -1,409 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.db.tries; - -import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import org.apache.cassandra.utils.bytecomparable.ByteSource; - -class SlicedCursor implements Cursor -{ - private ByteSource start; - private ByteSource end; - private final Cursor source; - private final Direction direction; - - State state; - int startNext; - int startNextDepth; - int endNext; - int endNextDepth; - - private enum State - { - /** - * The cursor is at the initial phase while it is walking prefixes of both bounds. - * Content is not to be reported. - */ - COMMON_PREFIX, - /** - * The cursor is positioned on some prefix of the start bound, strictly before any prefix of the end bound in - * iteration order. - * Content should only be reported in the reverse direction (as these prefixes are prefixes of the right bound - * and included in the slice). - */ - START_PREFIX, - /** - * The cursor is positioned inside the range, i.e. strictly between any prefixes of the start and end bounds. - * All content should be reported. - */ - INSIDE, - /** - * The cursor is positioned on some prefix of the end bound, strictly after any prefix of the start bound. - * Content should only be reported in the forward direction. - */ - END_PREFIX, - /** - * The cursor is positioned beyond the end bound. Exhaustion (depth -1) has been reported. - */ - EXHAUSTED; - } - - public SlicedCursor(Cursor source, - ByteSource leftSource, - int leftNext, - ByteSource rightSource, - int rightNext) - { - this.source = source; - this.direction = source.direction(); - start = direction.select(leftSource, rightSource); - end = direction.select(rightSource, leftSource); - startNext = direction.select(leftNext, rightNext); - endNext = direction.select(rightNext, leftNext); - startNextDepth = start != null ? 1 : 0; - endNextDepth = end != null ? 1 : 0; - state = start != null - ? end != null - ? State.COMMON_PREFIX - : State.START_PREFIX - : end != null - ? State.END_PREFIX - : State.INSIDE; - } - - @Override - public int advance() - { - int newDepth; - int transition; - - switch (state) - { - case COMMON_PREFIX: - case START_PREFIX: - // Skip any transitions before the start bound - newDepth = source.skipTo(startNextDepth, startNext); - transition = source.incomingTransition(); - return checkBothBounds(newDepth, transition); - case INSIDE: - case END_PREFIX: - newDepth = source.advance(); - transition = source.incomingTransition(); - return checkEndBound(newDepth, transition); - default: - throw new AssertionError(); - } - } - - private int markDone() - { - state = State.EXHAUSTED; - return -1; - } - - int checkBothBounds(int newDepth, int transition) - { - // Check if we are still following the start bound - if (newDepth == startNextDepth && transition == startNext) - { - assert startNext != ByteSource.END_OF_STREAM; - startNext = start.next(); - ++startNextDepth; - State currState = state; - // In the forward direction the exact match for the left bound and all descendant states are - // included in the set. - // In the reverse direction we will instead use the -1 as target transition and thus ascend on - // the next advance (skipping the exact right bound and all its descendants). - if (startNext == ByteSource.END_OF_STREAM && direction.isForward()) - state = State.INSIDE; // checkEndBound may adjust this to END_PREFIX - if (currState == State.START_PREFIX) - return newDepth; // there is no need to check the end bound as we descended along a - // strictly earlier path - } - else // otherwise we are beyond the start bound - state = State.INSIDE; // checkEndBound may adjust this to END_PREFIX - - return checkEndBound(newDepth, transition); - } - - private int checkEndBound(int newDepth, int transition) - { - // Cursor positions compare by depth descending and transition ascending. - if (newDepth > endNextDepth) - return newDepth; // happy and quick path in the interior of the slice - // (state == State.INSIDE can be asserted here (we skip it for efficiency)) - if (newDepth < endNextDepth) - return markDone(); - // newDepth == endDepth - if (direction.lt(transition, endNext)) - { - adjustStateStrictlyBeforeEnd(); - return newDepth; - } - if (direction.lt(endNext, transition)) - return markDone(); - - // Following end bound - endNext = end.next(); - ++endNextDepth; - if (endNext == ByteSource.END_OF_STREAM) - { - // At the exact end bound. - if (direction.isForward()) - { - // In forward direction the right bound is not included in the slice. - return markDone(); - } - else - { - // In reverse, the left bound and all its descendants are included, thus we use the -1 as limiting - // transition. We can also see the bound as strictly ahead of our current position as the current - // branch should be fully included. - adjustStateStrictlyBeforeEnd(); - } - } - else - adjustStateAtEndPrefix(); - return newDepth; - } - - private void adjustStateAtEndPrefix() - { - switch (state) - { - case INSIDE: - state = State.END_PREFIX; - break; - } - } - - private void adjustStateStrictlyBeforeEnd() - { - switch (state) - { - case COMMON_PREFIX: - state = State.START_PREFIX; - break; - case END_PREFIX: - state = State.INSIDE; - break; - } - } - - @Override - public int advanceMultiple(TransitionsReceiver receiver) - { - switch (state) - { - case COMMON_PREFIX: - case START_PREFIX: - case END_PREFIX: - return advance(); // descend only one level to be able to compare cursors correctly - case INSIDE: - int depth = source.depth(); - int newDepth = source.advanceMultiple(receiver); - if (newDepth > depth) - return newDepth; // successfully descended - // we ascended, check if we are still within boundaries - return checkEndBound(newDepth, source.incomingTransition()); - default: - throw new AssertionError(); - } - } - - @Override - public int skipTo(int skipDepth, int skipTransition) - { - // if skipping beyond end, we are done - if (skipDepth < endNextDepth || skipDepth == endNextDepth && direction.gt(skipTransition, endNext)) - return markDone(); - // if skipping before start, adjust request to skip to start - if (skipDepth == startNextDepth && direction.lt(skipTransition, startNext)) - skipTransition = startNext; - - switch (state) - { - case START_PREFIX: - case COMMON_PREFIX: - return checkBothBounds(source.skipTo(skipDepth, skipTransition), source.incomingTransition()); - case INSIDE: - case END_PREFIX: - return checkEndBound(source.skipTo(skipDepth, skipTransition), source.incomingTransition()); - default: - throw new AssertionError("Cursor already exhaused."); - } - } - - @Override - public int depth() - { - return state == State.EXHAUSTED ? -1 : source.depth(); - } - - @Override - public int incomingTransition() - { - return state == State.EXHAUSTED ? -1 : source.incomingTransition(); - } - - @Override - public Direction direction() - { - return direction; - } - - @Override - public ByteComparable.Version byteComparableVersion() - { - return source.byteComparableVersion(); - } - - @Override - public T content() - { - switch (state) - { - case INSIDE: - return source.content(); - // Additionally, prefixes of the right bound (which are not prefixes of the left) need to be reported: - case START_PREFIX: - // start prefixes in reverse direction (but making sure we don't report the exact match); - return !direction.isForward() && startNext != ByteSource.END_OF_STREAM ? source.content() : null; - case END_PREFIX: - // end prefixes in forward direction. - return direction.isForward() ? source.content() : null; - default: - return null; - } - } - - @Override - public Cursor tailCursor(Direction dir) - { - final Cursor sourceTail = source.tailCursor(dir); - switch (state) - { - case INSIDE: - return sourceTail; - case COMMON_PREFIX: - return maybeSwapBoundsAndMakeCursor(sourceTail, duplicatableStart(), startNext, duplicatableEnd(), endNext, direction); - case START_PREFIX: - return maybeSwapBoundsAndMakeCursor(sourceTail, duplicatableStart(), startNext, null, -1, direction); - case END_PREFIX: - return maybeSwapBoundsAndMakeCursor(sourceTail, null, -1, duplicatableEnd(), endNext, direction); - default: - throw new UnsupportedOperationException("tailCursor on a slice boundary"); - } - } - - private ByteSource.Duplicatable duplicatableStart() - { - if (start == null || start instanceof ByteSource.Duplicatable) - return (ByteSource.Duplicatable) start; - ByteSource.Duplicatable duplicatable = ByteSource.duplicatable(start); - start = duplicatable; - return duplicatable; - } - - private ByteSource.Duplicatable duplicatableEnd() - { - if (end == null || end instanceof ByteSource.Duplicatable) - return (ByteSource.Duplicatable) end; - ByteSource.Duplicatable duplicatable = ByteSource.duplicatable(end); - end = duplicatable; - return duplicatable; - } - - private static Cursor maybeSwapBoundsAndMakeCursor(Cursor source, - ByteSource.Duplicatable startSource, - int startNext, - ByteSource.Duplicatable endSource, - int endNext, - Direction startAndEndDirection) - { - ByteSource.Duplicatable leftSource = startAndEndDirection.select(startSource, endSource); - ByteSource.Duplicatable rightSource = startAndEndDirection.select(endSource, startSource); - int leftNext = startAndEndDirection.select(startNext, endNext); - int rightNext = startAndEndDirection.select(endNext, startNext); - return new SlicedCursor<>(source, - leftSource != null ? leftSource.duplicate() : null, - leftNext, - rightSource != null ? rightSource.duplicate() : null, - rightNext); - } - - - public static Cursor create(Cursor sourceCursor, - ByteComparable left, boolean includeLeft, - ByteComparable right, boolean includeRight) - { - Direction direction = sourceCursor.direction(); - // The cursor is left-inclusive and right-exclusive by default. If we need to change the inclusiveness, adjust - // the bound to the next possible value by adding a 00 byte at the end. - ByteSource leftSource = openAndMaybeAdd0(left, sourceCursor.byteComparableVersion(), !includeLeft); - ByteSource rightSource = openAndMaybeAdd0(right, sourceCursor.byteComparableVersion(), includeRight); - - // Empty left bound is the same as having no left bound, adjust for that. - int leftNext = -1; - if (leftSource != null) - { - leftNext = leftSource.next(); - if (leftNext == ByteSource.END_OF_STREAM) - leftSource = null; - } - - // Empty right bound means the result can only be empty. Make things easier for the cursor by handling this. - int rightNext = -1; - if (rightSource != null) - { - rightNext = rightSource.next(); - if (rightNext == ByteSource.END_OF_STREAM) - { - assert leftSource == null : "Invalid range " + sliceString(sourceCursor.byteComparableVersion(), left, includeLeft, right, includeRight); - return new Empty<>(direction, sourceCursor.byteComparableVersion()); - } - } - - return new SlicedCursor<>(sourceCursor, - leftSource, - leftNext, - rightSource, - rightNext); - } - - static String sliceString(ByteComparable.Version version, ByteComparable left, boolean includeLeft, ByteComparable right, boolean includeRight) - { - return String.format("%s%s;%s%s", - includeLeft ? "[" : "(", - left.byteComparableAsString(version), - right.byteComparableAsString(version), - includeRight ? "]" : ")"); - } - - static ByteSource openAndMaybeAdd0(ByteComparable key, ByteComparable.Version byteComparableVersion, boolean shouldAdd0) - { - if (key == null) - return null; - ByteSource src = key.asComparableBytes(byteComparableVersion); - if (shouldAdd0) - return ByteSource.append(src, 0); - else - return src; - } -} diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java index 7eaf5048a60b..af1ca0b336fa 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.java +++ b/src/java/org/apache/cassandra/db/tries/Trie.java @@ -23,7 +23,6 @@ import java.util.Collection; import java.util.Iterator; import java.util.Map; -import java.util.function.BiConsumer; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.utils.bytecomparable.ByteComparable; @@ -65,29 +64,10 @@ static Trie singleton(ByteComparable b, ByteComparable.Version byteCompar return dir -> new SingletonCursor<>(dir, b.asComparableBytes(byteComparableVersion), byteComparableVersion, v); } - /// Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries. - /// The view is live, i.e. any write to the source will be reflected in the subtrie. - /// This method will throw an assertion error if the bounds provided are not correctly ordered, including with - /// respect to the `includeLeft` and `includeRight` constraints (i.e. `subtrie(x, false, x, false)` is an invalid call - /// but `subtrie(x, true, x, false)` is inefficient but fine for an empty subtrie). - /// - /// @param left the left bound for the returned subtrie. If `null`, the resulting subtrie is not left-bounded. - /// @param includeLeft whether `left` is an inclusive bound of not. - /// @param right the right bound for the returned subtrie. If `null`, the resulting subtrie is not right-bounded. - /// @param includeRight whether `right` is an inclusive bound of not. - /// @return a view of the subtrie containing all the keys of this trie falling between `left` (inclusively if - /// `includeLeft`) and `right` (inclusively if `includeRight`). - default Trie subtrie(ByteComparable left, boolean includeLeft, ByteComparable right, boolean includeRight) - { - if (left == null && right == null) - return this; - return dir -> SlicedCursor.create(cursor(dir), left, includeLeft, right, includeRight); - } - @Override - default Trie subtrie(ByteComparable left, ByteComparable right) + default Trie intersect(TrieSet set) { - return subtrie(left, true, right, false); + return dir -> new IntersectionCursor.Plain<>(cursor(dir), set.cursor(dir)); } /// Returns the values in any order. For some tries this is much faster than the ordered iterable. @@ -255,7 +235,7 @@ default Trie tailTrie(ByteComparable prefix) { Cursor c = cursor(Direction.FORWARD); if (c.descendAlong(prefix.asComparableBytes(c.byteComparableVersion()))) - return dir -> c.tailCursor(dir); + return c::tailCursor; else return null; } diff --git a/src/java/org/apache/cassandra/db/tries/Trie.md b/src/java/org/apache/cassandra/db/tries/Trie.md index 72da9cb3d30d..9093f5cb8a12 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.md +++ b/src/java/org/apache/cassandra/db/tries/Trie.md @@ -205,7 +205,7 @@ The same argument holds when `b` is the smaller cursor to be advanced. ## Merging two tries -Two tries can be merged using `Trie.mergeWith`, which is implemented using the class `MergeTrie`. The implementation +Two tries can be merged using `Trie.mergeWith`, which is implemented using the class `MergeCursor`. The implementation is a straightforward application of the parallel walking scheme above, where the merged cursor presents the depth and incoming transition of the currently smaller cursor, and advances by advancing the smaller cursor, or both if they are equal. @@ -218,7 +218,7 @@ the other through "bb" — condition 2. is violated, the latter will have hi ## Merging an arbitrary number of tries -Merging is extended to an arbitrary number of sources in `CollectionMergeTrie`, used through the static `Trie.merge`. +Merging is extended to an arbitrary number of sources in `CollectionMergeCursor`, used through the static `Trie.merge`. The process is a generalization of the above, implemented using the min-heap solution from `MergeIterator` applied to cursors. @@ -229,28 +229,6 @@ descendants) at the expense of possibly adding one additional comparison in the As above, when we know that the head element is not equal to the heap top (i.e. it's necessarily smaller) we can use its `advanceMultiple` safely. -## Slicing tries - -Slicing, implemented in `SlicedTrie` and used via `Trie.subtrie`, can also be seen as a variation of the parallel -walk. In this case we walk the source as well as singletons of the two bounds. - -More precisely, while the source cursor is smaller than the left bound, we don't produce any output. That is, we -keep advancing in a loop, but to avoid walking subtries unnecessarily, we use `skipChildren` instead of `advance`. -As we saw above, a smaller cursor that descends remains smaller, thus there is no point to do so when we are -ahead of the left bound. When the source matches a node from the left bound, we descend both and pass the -state to the consumer. As soon as the source becomes known greater than the left bound, we can stop processing -the latter and pass any state we see to the consumer. - -Throughout this we also process the right bound cursor and we stop the iteration (by returning `depth = -1`) -as soon as the source becomes larger than the right bound. - -`SlicedTrie` does not use singleton tries and cursors over them but opts to implement them directly, using an -implicit representation using a pair of `depth` and `incomingTransition` for each bound. - -In slices we can also use `advanceMultiple` when we are certain to be strictly inside the slice, i.e. beyond the -left bound and before the right bound. As above, descending to any depth in this case is safe as the -result will remain smaller than the right bound. - ## Reverse iteration Tries and trie cursors support reverse iteration. Reverse trie iteration presents data in lexicographic order @@ -259,4 +237,192 @@ direction; the latter is only guaranteed if the entries in the trie can contain is prefix-free like the byte-ordered type translations). This difference is imposed by the cursor interfaces which necessarily have to present parent nodes before their -children and do not preserve or present any state on ascent. \ No newline at end of file +children and do not preserve or present any state on ascent. + + +# Trie sets + +The simplest way to implement a set in the trie paradigm is to define +an infinite trie that returns `true` for all positions that are covered by the set. Such a set is very easy to define +and apply, but unfortunately is not at all efficient because an intersection must necessarily walk the set cursor for +every covered position, which introduces a lot of overhead and makes it impossible to apply efficiency improvements such +as `advanceMultiple`. + +Instead, our trie sets (defined in `TrieSet/TrieSetCursor`) implement sets of ranges of keys by listing the boundaries of +each range and their prefixes. This makes it possible to identify fully contained regions of the set and proceed inside +such regions without touching the set cursor. + +Trie set cursors specify a "state" at any position they list. This state includes information about the inclusion of trie +branches before, after and below the listed position. When we are applying a set to a trie (i.e. intersecting the trie +with it), we would walk the two cursors in parallel. If the set moves ahead, we use the state to determine whether the +position of the trie cursor is covered by the set. Similarly, when a `skipTo` is performed on the set, the same state +flags can tell us if the set covers the position we attempted to skip to, when the set cursor does not have an exact +match and skips over the requested position. + +## Trie set content + +Trie sets list the boundary points for the represented ranges. For example, the range `[abc, ade]` will be represented +by the trie +``` +a -> + b -> + c -> START + d -> + e -> END +``` +where `START` is a state marking a left boundary, and `END` marks a right boundary. To be able to easily say that e.g. +`aa` is not covered by the set, but `ac` is, nodes on the prefix path also keep track of a richer state that also +provide information on the coverage on both sides of the position. + +The full state trie for the above example is +``` +a -> START_END_PREFIX + b -> START_PREFIX + c -> START + d -> END_PREFIX + e -> END +``` +The "prefix" states are not reported by `content()`, but they are used to determine the inclusion of preceding positions +in the set. `START_PREFIX` denotes a prefix of a left boundary, and thus positions before it are not covered by the set, +but positions after it are. Similarly, `END_PREFIX` is a prefix of a right boundary, which has the opposite coverage on +the two sides. `START_END_PREFIX` is a prefix of both a left and a right boundary (or more generally a boundary of some +number of pairs of left and right boundaries), and thus neither side of that prefix belongs to the covered set. + +There are several additional states that the sets can list: +- `POINT` is a position that is both the start and end boundary of a range. This is a singleton branch covered by the + set, e.g. `[abc, abc]` is represented by the trie + ``` + a -> START_END_PREFIX + b -> START_END_PREFIX + c -> POINT + ``` +- `END_START_PREFIX` is the prefix of both a right and left boundary (or more generally any number of pairs of right + and left boundary). This is a position that is covered by the set on both sides but whose branch is not entirely in + the set. For example, the ranges `[abc, adc] + [ade, afg]` are represented by the trie + ``` + a -> START_END_PREFIX + b -> START_PREFIX + c -> START + d -> END_START_PREFIX + c -> END + e -> START + f -> END_PREFIX + g -> END + ``` +- `COVERED` is a position which is both an end of one boundary and the start of another. Such boundaries have no effect + and the position at which they are reported is fully covered on both sides, as well as on the whole descendant branch. + For example, the ranges `[abc, ade] + [ade, afg]` are represented by the trie + ``` + a -> START_END_PREFIX + b -> START_PREFIX + c -> START + d -> END_START_PREFIX + e -> COVERED + f -> END_PREFIX + g -> END + ``` + +## Inclusivity and prefixes + +One important feature of the trie sets is that they are always inclusive of all boundaries, all their prefixes and all +their descendants. Additionally, set boundaries cannot contain prefixes of other boundaries. This is imposed by the +necessity to perform cursor walks on tries in both forward and reverse direction. + +For example, consider the range `[a, aaa]`. In both directions of a cursor walk `a` precedes `aaa`, which means that +if we are to accept such a range, it must be presented as `a: START, aaa: END` in a forward walk, and as +`a: END, aaa: START` in a reverse walk. Allowing different trie content depending on the direction of the walk is not +an acceptable solution because of the complexity and confusion that can bring. + +Additionally, if we are to exclude some prefixes or descendants, so that e.g. `[aa, bb)` includes `aaa` and `b` but not +`a` or `bbb`, the reverse iteration of this range (which would report `a` before `aa` and `bbb` after `b`) would not be +a contiguous range, which would also introduce unacceptable complexity. + +The above is only a material limitation when the keys allow prefixes. If the keys we work with are prefix-free and can +present positions before and after any valid key (both provided by our byte-comparable translation), we can still +correctly define ranges between any two keys, with the posibility of inclusive or exclusive boundaries as needed. + +As we also would also like to retrieve metadata on the paths leading to queried keys (e.g. a partition marker and stats +when we query information for a subset of rows in that partition), the fact that these sets always produce prefixes can +be seen as an advantage. + +## Converting ranges to trie sets + +The main usage of a trie set is to return subtries bounded by one or more key ranges. We achieve this as the +intersection of a trie with a trie set that represents the ranges. The ranges are constructed by taking an array of +ordered boundaries, walking them in parallel and presenting states as follows: +- if the keys that we are currently descending on start on an odd position in the array, this is a prefix of a left + boundary, so the state we need should cover the positions to the left of it (i.e. be `START_PREFIX`, + `START_END_PREFIX`, `START` or `COVERED`). +- if the keys that we are currently descending on end on an even position in the array, this is a prefix of a right + boundary, so the state we need should cover the positions to the right of it (i.e. be `END_PREFIX`, + `START_END_PREFIX`, `END` or `COVERED`). +- if the keys that we are currently descending on are exhausted (i.e. return `END_OF_STREAM`), we need to report a + boundary state (i.e. `START`, `END`, `POINT` or `COVERED`) and we need to advance and ascend to the next keys in the + array. + +For the `[abc, adc] + [ade, afg]` example above, the ranges construction will accept the array `[abc, adc, ade, afg]` +and proceed as follows: +- We start at depth 0, with all four array positions assigned depth 0. +- On the first `advance` call, we advance all sources at depth 0, and all of them return `a` and depth 1. Since the left + index is 0 (left excluded), the right index is 3 (right excluded), and the key is not exhausted, the state returned + should be `START_END_PREFIX`. +- On the next `advance` call, we advance all sources at the current depth 1 (this is again all 4). This time they return + different characters, thus we restrict our advancing set to just index 0, with character `b` and depth 2. + Since the left index is 0 (left excluded), the right index is 0 (right included), and the key is not exhausted, the + state returned should be `START_PREFIX`. +- On the next `advance` call, we advance all sources at the current depth 2. This is only the source at index 0, which + returns `c` and depth 3. Since the left index is 0 (left excluded), the right index is 0 (right included), and the key + is exhausted, the state returned is `START`. +- On the next `advance` call we recognize that the currently followed key is exhausted, so we advance to the next set, + which has depth 2 and character `d`. This includes two keys, with indexes 1 and 2. Since the left index is 1 (left + included), the right index is 2 (right included), and the key is not exhausted, the state returned should be + `END_START_PREFIX`. +- On the next `advance` call we advance the sources at the current depth 2, which are the sources at indexes 1 and 2. + They return different characters, so we restrict our advancing set to just index 1, with character `c` and depth 3. + Since the left index is 1 (left included), the right index is 1 (right excluded), and the key is exhausted, the + state returned should be `END`. +- On the next `advance` call we recognize that the currently followed key is exhausted, so we advance to the next set, + which has depth 3 and character `e`, and includes only the key at array index 2. Since the index for both sides is 2, + left is excluded and right is included, which with the exhausted key gives us the state `START`. +- On the next `advance` call we recognize that the currently followed key is exhausted, so we advance to the next set, + which has depth 2 and character `f`, and includes only the key at array index 3. For index 3 on both sides (left + included, right excluded) and a key that is not exhausted, the state returned is `END_PREFIX`. +- On the next `advance` call we descend along key 3, which is `g` and exhausted, so the state returned is `END` + (left included, right excluded, key exhausted). +- The next `advance` call moves outside the span of the array, and we return a depth of -1, denoting the end of the + walk. + +## Intersecting a trie with a trie set + +Set intersection is performed by walking the source and set with a parallel walk. If the set advances beyond the +position of the trie, we check the state of the set to see if the position is covered by the set (done by +`TrieSetCursor.precedingIncluded`). If it is, we can present all content in the trie until it catches up with the set +position, and we can also apply `advanceMultiple` as a direct call on the trie. If the position is not covered by the +set, we perform a `skipTo` call to the current position of the set. This may move beyond the current position of the +set, so we must skip the set to the new position, and then repeat the above steps. + +If at any point both trie and set are at the same position, we can report that position and advance both trie and set +on the next `advance` or `skipTo` call. In this case `advanceMultiple` cannot be used and must act as `advance`. + +## Set algebra + +A variation of the above can also be applied to sets, giving us set intersections. + +We can also perform "weak" negation of a set by simply inverting the returned states. This has the effect of changing +the covered branches, but not any boundary or their descendant branches. For example, the weak inverse of the range +`[abc, ade]` is the set `[null, abc] + [ade, null]`, which is represented as +``` +END_START_PREFIX +a -> END_START_PREFIX + b -> END_PREFIX + c -> END + d -> START_PREFIX + e -> START +END_START_PREFIX +``` +(The END_START_PREFIX before the trie describes the result of calling `state` on the root node, and the one at the end, +the result of calling state when the iteration is exhausted; these are START_END_STATE for sets that are limited on the +respective side. Also note that the inversion of an array of boundaries is the array with "null" appended on both +sides.) + +Using De Morgan's law, this weak negation also lets us perform set union. diff --git a/src/java/org/apache/cassandra/db/tries/TrieSet.java b/src/java/org/apache/cassandra/db/tries/TrieSet.java new file mode 100644 index 000000000000..a875521996dc --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/TrieSet.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// A trie that defines an infinite set of `ByteComparable`s. The convention of this package is that sets always +/// include all boundaries, all prefixes that lead to a boundary, and all descendants of all boundaries. This is done +/// to properly define reverse iteration where prefixes are listed before their descendants, and to allow for the +/// retrieval of metadata on paths pointing to specific keys. +/// +/// Trie sets represent sets of ranges of coverage by listing the boundaries between them, and providing a way to +/// identify the "covering state" for positions that are being skipped to. +/// +/// Trie sets can be constructed from ranges using [#singleton], [#range] and [#ranges], and can be manipulated via +/// the set algebra methods [#union], [#intersection] and [#weakNegation]. +public interface TrieSet extends CursorWalkable +{ + static TrieSet singleton(ByteComparable.Version version, ByteComparable b) + { + return dir -> new RangesCursor(dir, version, b, b); + } + + static TrieSet range(ByteComparable.Version version, ByteComparable left, ByteComparable right) + { + return dir -> new RangesCursor(dir, version, left, right); + } + + static TrieSet ranges(ByteComparable.Version version, ByteComparable... boundaries) + { + return dir -> new RangesCursor(dir, version, boundaries); + } + + default TrieSet union(TrieSet other) + { + return dir -> new TrieSetIntersectionCursor.UnionCursor(cursor(dir), other.cursor(dir)); + } + + default TrieSet intersection(TrieSet other) + { + return dir -> new TrieSetIntersectionCursor(cursor(dir), other.cursor(dir)); + } + + /// Represents the set inverse of the given set plus all prefixes and descendants of all boundaries of the set. + /// E.g. the inverse of the set `[a, b]` is the set `union([null, a], [b, null])`, and + /// `intersection([a, b], weakNegation([a, b]))` equals `union([a, a], [b, b])`. + /// + /// True negation is not feasible in this design (exact points are always included together with all their descendants). + default TrieSet weakNegation() + { + return dir -> new TrieSetNegatedCursor(cursor(dir)); + } + + /// Constuct a textual representation of the trie. + default String dump() + { + return cursor(Direction.FORWARD).process(new TrieDumper<>(Object::toString)); + } + + TrieSetCursor makeCursor(Direction direction); + + @Override + default TrieSetCursor cursor(Direction direction) + { + return Trie.DEBUG ? new VerificationCursor.TrieSet(makeCursor(direction)) + : makeCursor(direction); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/TrieSetCursor.java b/src/java/org/apache/cassandra/db/tries/TrieSetCursor.java new file mode 100644 index 000000000000..74281293b47f --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/TrieSetCursor.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +/// The implementation of a [TrieSet]. +/// +/// In addition to the functionality of normal trie cursors, set cursors also produce a [#state] that describes the +/// coverage of trie sections to the left, right and below the cursor position. This is necessary to be able to identify +/// coverage after a [#skipTo] operation, where the set cursor jumps to a position beyond the requested one. +interface TrieSetCursor extends Cursor +{ + /// This type describes the state at a given cursor position. It describes the coverage of the positions before and + /// after the current in forward order, whether the node is boundary (and thus applies to this point and all its + /// descendants) and also describes the type of boundary (e.g. start/end). + enum RangeState + { + // Note: the states must be ordered so that + // `values()[applicableBefore * 1 + applicableAfter * 2 + applicableAtPoint * 4]` + // produces a state with the requested flags + + /// The cursor is at a prefix of a contained range, and neither the branches to the left or right are contained. + START_END_PREFIX(false, false, false), + /// The cursor is positioned at a prefix of an end boundary, inside a covered range on the left. + END_PREFIX(true, false, false), + /// The cursor is positioned at a prefix of a start boundary. The branches to the right are covered. + START_PREFIX(false, true, false), + /// The cursor is positioned inside a covered range, on a prefix of an excluded sub-range. + END_START_PREFIX(true, true, false), + /// The cursor is positioned at a "point" boundary, i.e. only the descendants of the boundary are covered, + /// branches to the left or right are not contained. + POINT(false, false, true), + /// The cursor is positioned at an end boundary. Branches to the left, as well as descendants of this point are + /// covered by the set. + END(true, false, true), + /// The cursor is positioned at a start boundary. Branches to the right, as well as descendants of this point + /// are covered by the set. + START(false, true, true), + /// The cursor is positioned at a non-effective boundary (an end boundary for the previous range, as well as + /// a start for the next). Branches before, after and below this point is covered. + COVERED(true, true, true); + + /// Whether the set applied to positions before the cursor's in forward order. + final boolean applicableBefore; + /// Whether the set applied to positions after the cursor's in forward order. + final boolean applicableAfter; + /// The state to report as content. This converts prefix states to null to report only the boundaries + /// (e.g. for dumping to text). + final RangeState asContent; + + RangeState(boolean applicableBefore, boolean applicableAfter, boolean applicableAtPoint) + { + this.applicableBefore = applicableBefore; + this.applicableAfter = applicableAfter; + this.asContent = applicableAtPoint ? this : null; + } + + /// Whether the positions preceding the current in iteration order are included in the set. + public boolean precedingIncluded(Direction direction) + { + return direction.select(applicableBefore, applicableAfter); + } + + /// Whether the descendant branch is fully included in the set. + public boolean branchIncluded() + { + return asContent != null; + } + + public RangeState toContent() + { + return asContent; + } + + /// Return an "intersection" state for the combination of two states, i.e. the ranges covered by both states. + public RangeState intersect(RangeState other) + { + return values()[ordinal() & other.ordinal()]; + } + + /// Return a "union" state for the combination of two states, i.e. the ranges covered by at least one of the states. + public RangeState union(RangeState other) + { + return values()[ordinal() | other.ordinal()]; + } + + /// Return the "weakly negated" state, i.e. the state that corresponds to flipped areas of coverage to the left + /// and right, and the boundary points. See [TrieSet#weakNegation] for more details. + public RangeState weakNegation() + { + return values()[ordinal() ^ 3]; + } + + public static RangeState fromProperties(boolean applicableBefore, boolean applicableAfter, boolean applicableAtPoint) + { + return values()[(applicableBefore ? 1 : 0) + (applicableAfter ? 2 : 0) + (applicableAtPoint ? 4 : 0)]; + } + } + + /// The range state of the trie cursor at this point. + RangeState state(); + + /// Returns whether the set includes the positions before the current in iteration order, but after any earlier + /// position of this cursor, including any position requested by a [#skipTo] call, where this cursor advanced beyond + /// that position. + /// + /// Note that this may also be true when the cursor is in an exhausted state, as well as immediately + /// after cursor construction, signifying, respectively, right and left unbounded ranges. + default boolean precedingIncluded() + { + return state().precedingIncluded(direction()); + } + + /// Returns whether the set fully includes all descendants of the current position. This is true for all boundary + /// points. + default boolean branchIncluded() + { + return state().asContent != null; + } + + @Override + default RangeState content() + { + return state().toContent(); + } + + @Override + TrieSetCursor tailCursor(Direction direction); +} diff --git a/src/java/org/apache/cassandra/db/tries/TrieSetIntersectionCursor.java b/src/java/org/apache/cassandra/db/tries/TrieSetIntersectionCursor.java new file mode 100644 index 000000000000..e69a9290eaa6 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/TrieSetIntersectionCursor.java @@ -0,0 +1,294 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// Implementation of the intersection of two sets. +class TrieSetIntersectionCursor implements TrieSetCursor +{ + enum State + { + MATCHING, + C1_AHEAD, + C2_AHEAD; + + State swap() + { + switch(this) + { + case C1_AHEAD: + return C2_AHEAD; + case C2_AHEAD: + return C1_AHEAD; + default: + throw new AssertionError(); + } + } + } + + final Direction direction; + final TrieSetCursor c1; + final TrieSetCursor c2; + int currentDepth; + int currentTransition; + TrieSetCursor.RangeState currentRangeState; + State state; + + TrieSetIntersectionCursor(TrieSetCursor c1, TrieSetCursor c2) + { + this.direction = c1.direction(); + this.c1 = c1; + this.c2 = c2; + matchingPosition(0, -1); + } + + @Override + public int depth() + { + return currentDepth; + } + + @Override + public int incomingTransition() + { + return currentTransition; + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return c1.byteComparableVersion(); + } + + @Override + public TrieSetCursor.RangeState state() + { + return currentRangeState; + } + + /// Whether the preceding positions are in the set. Overridden by [UnionCursor]. + boolean precedingInSet(TrieSetCursor cursor) + { + return cursor.precedingIncluded(); + } + + @Override + public int advance() + { + switch(state) + { + case MATCHING: + { + int ldepth = c1.advance(); + if (precedingInSet(c1)) + return advanceWithSetAhead(c2.advance(), c2, c1, State.C1_AHEAD); + else + return advanceToIntersection(ldepth, c1, c2, State.C1_AHEAD); + } + case C1_AHEAD: + return advanceWithSetAhead(c2.advance(), c2, c1, state); + case C2_AHEAD: + return advanceWithSetAhead(c1.advance(), c1, c2, state); + default: + throw new AssertionError(); + } + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + switch(state) + { + case MATCHING: + { + int ldepth = c1.skipTo(skipDepth, skipTransition); + if (precedingInSet(c1)) + return advanceWithSetAhead(c2.skipTo(skipDepth, skipTransition), c2, c1, State.C1_AHEAD); + else + return advanceToIntersection(ldepth, c1, c2, State.C1_AHEAD); + } + case C1_AHEAD: + return advanceWithSetAhead(c2.skipTo(skipDepth, skipTransition), c2, c1, state); + case C2_AHEAD: + return advanceWithSetAhead(c1.skipTo(skipDepth, skipTransition), c1, c2, state); + default: + throw new AssertionError(); + } + } + + @Override + public int advanceMultiple(Cursor.TransitionsReceiver receiver) + { + switch(state) + { + case MATCHING: + { + // Cannot do multi-advance when cursors are at the same position. Applying advance(). + int ldepth = c1.advance(); + if (precedingInSet(c1)) + return advanceWithSetAhead(c2.advance(), c2, c1, State.C1_AHEAD); + else + return advanceToIntersection(ldepth, c1, c2, State.C1_AHEAD); + } + case C1_AHEAD: + return advanceWithSetAhead(c2.advanceMultiple(receiver), c2, c1, state); + case C2_AHEAD: + return advanceWithSetAhead(c1.advanceMultiple(receiver), c1, c2, state); + default: + throw new AssertionError(); + } + } + + /// Called to check the state and carry out any necessary advances in the case when the `ahead` cursor was known to + /// be ahead (and covering) before an operation was carried out to advance the `advancing` cursor. + private int advanceWithSetAhead(int advDepth, TrieSetCursor advancing, TrieSetCursor ahead, State state) + { + int aheadDepth = ahead.depth(); + int aheadTransition = ahead.incomingTransition(); + int advTransition = advancing.incomingTransition(); + if (advDepth > aheadDepth) + return coveredAreaWithSetAhead(advDepth, advTransition, advancing, state); + if (advDepth == aheadDepth) + { + if (direction.lt(advTransition, aheadTransition)) + return coveredAreaWithSetAhead(advDepth, advTransition, advancing, state); + if (advTransition == aheadTransition) + return matchingPosition(advDepth, advTransition); + } + + // Advancing cursor moved beyond the ahead cursor. Check if roles have reversed. + if (precedingInSet(advancing)) + return coveredAreaWithSetAhead(aheadDepth, aheadTransition, ahead, state.swap()); + else + return advanceToIntersection(advDepth, advancing, ahead, state.swap()); + } + + /// Called to advance both cursors to an intersection. When called, the `ahead` cursor is known to be ahead of the + /// `other` cursor, and the range preceding its position (which includes the current position of `other`) is not + /// contained. + private int advanceToIntersection(int aheadDepth, TrieSetCursor ahead, TrieSetCursor other, State state) + { + int aheadTransition = ahead.incomingTransition(); + while (true) + { + // At this point `ahead` is beyond `other`'s position, but the latter is outside the former's covered area. + // Skip `other` to `ahead`'s position. + int otherDepth = other.skipTo(aheadDepth, aheadTransition); + int otherTransition = other.incomingTransition(); + if (otherDepth == aheadDepth && otherTransition == aheadTransition) + return matchingPosition(aheadDepth, aheadTransition); + if (precedingInSet(other)) + return coveredAreaWithSetAhead(aheadDepth, aheadTransition, ahead, state.swap()); + + // If there's no match or coverage, the roles have reversed, swap everything and repeat. + aheadDepth = otherDepth; + aheadTransition = otherTransition; + state = state.swap(); + TrieSetCursor t = ahead; + ahead = other; + other = t; + } + } + + private int coveredAreaWithSetAhead(int depth, int transition, TrieSetCursor advancing, State state) + { + this.currentDepth = depth; + this.currentTransition = transition; + this.currentRangeState = advancing.state(); + this.state = state; + return depth; + } + + private int matchingPosition(int depth, int transition) + { + state = State.MATCHING; + currentDepth = depth; + currentTransition = transition; + currentRangeState = combineState(c1.state(), c2.state()); + return depth; + } + + TrieSetCursor.RangeState combineState(TrieSetCursor.RangeState cl, TrieSetCursor.RangeState cr) + { + assert cl.branchIncluded() == cr.branchIncluded() : "Intersection results in a prefix range"; + return cl.intersect(cr); + } + + @Override + public TrieSetCursor tailCursor(Direction direction) + { + switch (state) + { + case MATCHING: + return new TrieSetIntersectionCursor(c1.tailCursor(direction), c2.tailCursor(direction)); + case C1_AHEAD: + return c2.tailCursor(direction); + case C2_AHEAD: + return c1.tailCursor(direction); + default: + throw new AssertionError(); + } + } + + /// Implementation of the union of two sets. This is a direct application of DeMorgan's law, done by inverting the + /// meaning of [#precedingInSet] and the combination of states. + static class UnionCursor extends TrieSetIntersectionCursor + { + public UnionCursor(TrieSetCursor c1, TrieSetCursor c2) + { + super(c1, c2); + } + + @Override + boolean precedingInSet(TrieSetCursor cursor) + { + return !cursor.precedingIncluded(); + } + + @Override + TrieSetCursor.RangeState combineState(TrieSetCursor.RangeState cl, TrieSetCursor.RangeState cr) + { + assert cl.branchIncluded() == cr.branchIncluded() : "Union results in a prefix range"; + return cl.union(cr); + } + + @Override + public TrieSetCursor tailCursor(Direction direction) + { + switch (state) + { + case MATCHING: + return new UnionCursor(c1.tailCursor(direction), c2.tailCursor(direction)); + case C1_AHEAD: + return c2.tailCursor(direction); + case C2_AHEAD: + return c1.tailCursor(direction); + default: + throw new AssertionError(); + } + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/TrieSetNegatedCursor.java b/src/java/org/apache/cassandra/db/tries/TrieSetNegatedCursor.java new file mode 100644 index 000000000000..767019143864 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/TrieSetNegatedCursor.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// Negation of trie sets. +/// +/// Achieved by simply inverting the [#state()] values. +public class TrieSetNegatedCursor implements TrieSetCursor +{ + final TrieSetCursor source; + + TrieSetNegatedCursor(TrieSetCursor source) + { + this.source = source; + } + + @Override + public int depth() + { + return source.depth(); + } + + @Override + public int incomingTransition() + { + return source.incomingTransition(); + } + + @Override + public Direction direction() + { + return source.direction(); + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return source.byteComparableVersion(); + } + + @Override + public TrieSetCursor.RangeState state() + { + return source.state().weakNegation(); + } + + @Override + public int advance() + { + return source.advance(); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + return source.skipTo(skipDepth, skipTransition); + } + + // Sets don't implement advanceMultiple as they are only meant to limit data tries. + + @Override + public TrieSetCursor tailCursor(Direction direction) + { + return new TrieSetNegatedCursor(source.tailCursor(direction)); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/VerificationCursor.java b/src/java/org/apache/cassandra/db/tries/VerificationCursor.java index 697d53d19010..4a44d8388f0b 100644 --- a/src/java/org/apache/cassandra/db/tries/VerificationCursor.java +++ b/src/java/org/apache/cassandra/db/tries/VerificationCursor.java @@ -32,16 +32,14 @@ public interface VerificationCursor int EXHAUSTED_TRANSITION = -1; int INITIAL_TRANSITION = -1; - /** - * Verifies: - * - advance does advance, depth <= prevDepth + 1 and transition is higher than previous at the same depth - * (this requires path tracking) - * - skipTo is not called with earlier or equal position (including lower levels) - * - maybeSkipTo is not called with earlier position that can't be identified with depth/incomingTransition only - * (i.e. seeks to lower depth with an incoming transition that lower than the previous at that depth) - * - exhausted state is depth = -1, incomingTransition = -1 (maybe change to 0?) - * - start state is depth = 0, incomingTransition = -1 (maybe change to 0?) - */ + /// Verifies: + /// - `advance` does advance, `depth <= prevDepth + 1` and transition is higher than previous at the same depth + /// (this requires path tracking) + /// - `skipTo` is not called with earlier or equal position (including lower levels) + /// - `maybeSkipTo` is not called with earlier position that can't be identified with depth/incomingTransition only + /// (i.e. seeks to lower depth with an incoming transition that lower than the previous at that depth) + /// - exhausted state is `depth = -1, incomingTransition = -1` + /// - start state is `depth = 0, incomingTransition = -1` class Plain> implements Cursor, Cursor.TransitionsReceiver { final Direction direction; @@ -172,6 +170,7 @@ else if (depth <= returnedDepth) } @Override + @SuppressWarnings("unchecked") public Plain tailCursor(Direction direction) { return new Plain<>((C) source.tailCursor(direction), 0, 0, INITIAL_TRANSITION); @@ -229,4 +228,106 @@ public String toString() return builder.toString(); } } + + class TrieSet extends Plain implements TrieSetCursor + { + boolean currentPrecedingIncluded; + boolean nextPrecedingIncluded; + + TrieSet(TrieSetCursor source) + { + this(source, 0, 0, INITIAL_TRANSITION); + } + + TrieSet(TrieSetCursor source, int minDepth, int expectedDepth, int expectedTransition) + { + super(source, minDepth, expectedDepth, expectedTransition); + // start state can be non-null for sets + currentPrecedingIncluded = source.precedingIncluded(); + Preconditions.checkNotNull(currentPrecedingIncluded, "Covering state for trie sets must not be null"); + nextPrecedingIncluded = source.content() != null ? source.content().precedingIncluded(direction.opposite()) : currentPrecedingIncluded; + } + + void verifyEndState() + { + // end state can be non-null for sets + } + + @Override + public TrieSetCursor.RangeState state() + { + return Preconditions.checkNotNull(source.state()); + } + + @Override + public TrieSet tailCursor(Direction direction) + { + return new TrieSet(source.tailCursor(direction), 0, 0, INITIAL_TRANSITION); + } + + @Override + public int advance() + { + currentPrecedingIncluded = nextPrecedingIncluded; + return verifyState(super.advance()); + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + currentPrecedingIncluded = nextPrecedingIncluded; + return verifyState(super.advanceMultiple(receiver)); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + return verifySkipState(super.skipTo(skipDepth, skipTransition)); + } + + @Override + public boolean precedingIncluded() + { + Preconditions.checkState(currentPrecedingIncluded == source.precedingIncluded(), + "Covering state changed without advance: %s -> %s. %s", + currentPrecedingIncluded, source.precedingIncluded(), + currentPrecedingIncluded == source.precedingIncluded() + ? "The values are equal but different object. This is not permitted for performance reasons." + : ""); + // == above is correct, we do not want covering state to be recreated unless some change happened to the cursor + return currentPrecedingIncluded; + } + + private int verifyState(int depth) + { + boolean precedingIncluded = source.precedingIncluded(); + Preconditions.checkNotNull(precedingIncluded, "Covering state for trie sets must not be null"); + Preconditions.checkState(currentPrecedingIncluded == precedingIncluded, + "Unexpected change to covering state: %s -> %s", + currentPrecedingIncluded, precedingIncluded); + currentPrecedingIncluded = precedingIncluded; + + RangeState content = source.content(); + if (content != null) + { + Preconditions.checkState(currentPrecedingIncluded == content.precedingIncluded(direction), + "Range end %s does not close covering state %s", + content.precedingIncluded(direction), currentPrecedingIncluded); + nextPrecedingIncluded = content.precedingIncluded(direction.opposite()); + } + + if (depth < 0) + verifyEndState(); + return depth; + } + + private int verifySkipState(int depth) + { + // The covering state information is invalidated by a skip. + currentPrecedingIncluded = source.precedingIncluded(); + Preconditions.checkNotNull(currentPrecedingIncluded, "Covering state for trie sets must not be null"); + nextPrecedingIncluded = currentPrecedingIncluded; + return verifyState(depth); + } + } } diff --git a/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java b/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java index abee0014780c..32716a54c3a5 100644 --- a/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java +++ b/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java @@ -523,30 +523,27 @@ private Trie getSubtrie(@Nullable Expression expression) return data; ByteComparable lowerBound, upperBound; - boolean lowerInclusive, upperInclusive; if (expression.lower != null) { lowerBound = expression.getEncodedLowerBoundByteComparable(Version.current()); - lowerInclusive = expression.lower.inclusive; + // inclusivity is encoded in lowerBound } else { - lowerBound = ByteComparable.EMPTY; - lowerInclusive = false; + lowerBound = null; } if (expression.upper != null) { upperBound = expression.getEncodedUpperBoundByteComparable(Version.current()); - upperInclusive = expression.upper.inclusive; + // inclusivity is encoded in upperBound } else { upperBound = null; - upperInclusive = false; } - return data.subtrie(lowerBound, lowerInclusive, upperBound, upperInclusive); + return data.subtrie(lowerBound, upperBound); } public ByteBuffer getMinTerm() diff --git a/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.java b/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.java index 353e2532efa5..847073dbe1de 100644 --- a/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.java +++ b/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.java @@ -20,9 +20,6 @@ import java.nio.ByteBuffer; -import org.apache.cassandra.utils.FastByteOperations; - -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; /** @@ -82,13 +79,6 @@ default Preencoded preencode(Version version) // Simple factories used for testing - @VisibleForTesting - static ByteComparable of(String s) - { - // Note: This is not prefix-free - return v -> ByteSource.of(s, v); - } - static ByteComparable of(long value) { return v -> ByteSource.of(value); diff --git a/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java b/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java index 79620e079975..44136021b094 100644 --- a/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java +++ b/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java @@ -38,10 +38,10 @@ import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.concurrent.OpOrder; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.asString; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.assertMapEquals; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.byteComparableVersion; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.generateKeys; +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.apache.cassandra.db.tries.TrieUtil.asString; +import static org.apache.cassandra.db.tries.TrieUtil.assertMapEquals; +import static org.apache.cassandra.db.tries.TrieUtil.generateKeys; public class CellReuseTest { @@ -73,7 +73,7 @@ public void testCellReuseNoCopying() throws Exception public void testCellReuse(Predicate> forceCopyPredicate) throws Exception { ByteComparable[] src = generateKeys(rand, COUNT); - InMemoryTrie trieLong = makeInMemoryTrie(src, opOrder -> InMemoryTrie.longLived(byteComparableVersion, BufferType.ON_HEAP, opOrder), + InMemoryTrie trieLong = makeInMemoryTrie(src, opOrder -> InMemoryTrie.longLived(VERSION, BufferType.ON_HEAP, opOrder), forceCopyPredicate); // dump some information first @@ -121,8 +121,8 @@ public void testAbortedMutation() throws Exception { ByteComparable[] src = generateKeys(rand, COUNT); OpOrder order = new OpOrder(); - InMemoryTrie trie = InMemoryTrie.longLived(byteComparableVersion, order); - InMemoryTrie check = InMemoryTrie.shortLived(byteComparableVersion); + InMemoryTrie trie = InMemoryTrie.longLived(VERSION, order); + InMemoryTrie check = InMemoryTrie.shortLived(VERSION); int step = Math.min(100, COUNT / 100); int throwStep = (COUNT + 10) / 5; // do 4 throwing inserts int nextThrow = throwStep; @@ -247,7 +247,8 @@ private void markChild(InMemoryTrie trie, int child, BitSet set, BitSet objs) static InMemoryTrie makeInMemoryTrie(ByteComparable[] src, Function> creator, - Predicate> forceCopyPredicate) throws TrieSpaceExhaustedException + Predicate> forceCopyPredicate) + throws TrieSpaceExhaustedException { OpOrder order = new OpOrder(); InMemoryTrie trie = creator.apply(order); @@ -271,7 +272,7 @@ static void addToInMemoryTrie(ByteComparable[] src, // (so that all sources have the same value). int payload = asString(b).hashCode(); ByteBuffer v = ByteBufferUtil.bytes(payload); - Trie update = Trie.singleton(b, byteComparableVersion, v); + Trie update = Trie.singleton(b, VERSION, v); update = InMemoryTrieThreadedTest.withRootMetadata(update, Boolean.TRUE); update = update.prefixedBy(source("prefix")); applyUpdating(trie, update, forceCopyPredicate); @@ -280,7 +281,7 @@ static void addToInMemoryTrie(ByteComparable[] src, static ByteComparable source(String key) { - return ByteComparable.preencoded(byteComparableVersion, key.getBytes(StandardCharsets.UTF_8)); + return ByteComparable.preencoded(VERSION, key.getBytes(StandardCharsets.UTF_8)); } static void addThrowingEntry(ByteComparable b, @@ -289,7 +290,7 @@ static void addThrowingEntry(ByteComparable b, { int payload = asString(b).hashCode(); ByteBuffer v = ByteBufferUtil.bytes(payload); - Trie update = Trie.singleton(b, byteComparableVersion, v); + Trie update = Trie.singleton(b, VERSION, v); // Create an update with two metadata entries, so that the lower is already a copied node. // Abort processing on the lower metadata, where the new branch is not attached yet (so as not to affect the diff --git a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java index f7196a67397b..62c708018811 100644 --- a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java @@ -28,14 +28,16 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; import org.apache.cassandra.config.CassandraRelevantProperties; -import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.*; +import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.addToInMemoryTrie; +import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.makeInMemoryTrie; import static org.apache.cassandra.db.tries.MergeTrieTest.removeDuplicates; +import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; public class CollectionMergeTrieTest { @@ -46,15 +48,15 @@ public static void enableVerification() } private static final int COUNT = 15000; - Random rand = new Random(); + private static final Random rand = new Random(); @Test public void testDirect() { - ByteComparable[] src1 = generateKeys(rand, COUNT); - ByteComparable[] src2 = generateKeys(rand, COUNT); - SortedMap content1 = new TreeMap<>(forwardComparator); - SortedMap content2 = new TreeMap<>(forwardComparator); + Preencoded[] src1 = TrieUtil.generateKeys(rand, COUNT); + Preencoded[] src2 = TrieUtil.generateKeys(rand, COUNT); + SortedMap content1 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); + SortedMap content2 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); InMemoryTrie trie1 = makeInMemoryTrie(src1, content1, true); InMemoryTrie trie2 = makeInMemoryTrie(src2, content2, true); @@ -63,27 +65,27 @@ public void testDirect() // construct directly, trie.merge() will defer to mergeWith on two sources Trie union = makeCollectionMergeTrie(trie1, trie2); - assertSameContent(union, content1); + TrieUtil.assertSameContent(union, content1); } @Test public void testWithDuplicates() { - ByteComparable[] src1 = generateKeys(rand, COUNT); - ByteComparable[] src2 = generateKeys(rand, COUNT); - SortedMap content1 = new TreeMap<>(forwardComparator); - SortedMap content2 = new TreeMap<>(forwardComparator); + Preencoded[] src1 = TrieUtil.generateKeys(rand, COUNT); + Preencoded[] src2 = TrieUtil.generateKeys(rand, COUNT); + SortedMap content1 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); + SortedMap content2 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); InMemoryTrie trie1 = makeInMemoryTrie(src1, content1, true); InMemoryTrie trie2 = makeInMemoryTrie(src2, content2, true); - addToInMemoryTrie(generateKeys(new Random(5), COUNT), content1, trie1, true); - addToInMemoryTrie(generateKeys(new Random(5), COUNT), content2, trie2, true); + addToInMemoryTrie(TrieUtil.generateKeys(new Random(5), COUNT), content1, trie1, true); + addToInMemoryTrie(TrieUtil.generateKeys(new Random(5), COUNT), content2, trie2, true); content1.putAll(content2); Trie union = makeCollectionMergeTrie(trie1, trie2); - assertSameContent(union, content1); + TrieUtil.assertSameContent(union, content1); } private static Trie makeCollectionMergeTrie(InMemoryTrie... tries) @@ -94,19 +96,19 @@ private static Trie makeCollectionMergeTrie(InMemoryTrie @Test public void testDistinct() { - ByteComparable[] src1 = generateKeys(rand, COUNT); - SortedMap content1 = new TreeMap<>(forwardComparator); + Preencoded[] src1 = TrieUtil.generateKeys(rand, COUNT); + SortedMap content1 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); InMemoryTrie trie1 = makeInMemoryTrie(src1, content1, true); - ByteComparable[] src2 = generateKeys(rand, COUNT); + Preencoded[] src2 = TrieUtil.generateKeys(rand, COUNT); src2 = removeDuplicates(src2, content1); - SortedMap content2 = new TreeMap<>(forwardComparator); + SortedMap content2 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); InMemoryTrie trie2 = makeInMemoryTrie(src2, content2, true); content1.putAll(content2); Trie union = mergeDistinctTrie(ImmutableList.of(trie1, trie2)); - assertSameContent(union, content1); + TrieUtil.assertSameContent(union, content1); } private static Trie mergeDistinctTrie(Collection> sources) @@ -170,34 +172,44 @@ public void testMultiple(int mergeCount, int count) public void testMultipleDistinct(int mergeCount, int count) { List> tries = new ArrayList<>(mergeCount); - SortedMap content = new TreeMap<>(forwardComparator); + SortedMap content = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); for (int i = 0; i < mergeCount; ++i) { - ByteComparable[] src = removeDuplicates(generateKeys(rand, count), content); + Preencoded[] src = removeDuplicates(TrieUtil.generateKeys(rand, count), content); Trie trie = makeInMemoryTrie(src, content, true); tries.add(trie); } Trie union = Trie.mergeDistinct(tries); - assertSameContent(union, content); + TrieUtil.assertSameContent(union, content); } public void testMultipleWithDuplicates(int mergeCount, int count) { List> tries = new ArrayList<>(mergeCount); - SortedMap content = new TreeMap<>(forwardComparator); + SortedMap content = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); for (int i = 0; i < mergeCount; ++i) { - ByteComparable[] src = generateKeys(rand, count); + Preencoded[] src = TrieUtil.generateKeys(rand, count); Trie trie = makeInMemoryTrie(src, content, true); tries.add(trie); } Trie union = Trie.merge(tries, x -> x.iterator().next()); + TrieUtil.assertSameContent(union, content); - assertSameContent(union, content); + try + { + union = Trie.mergeDistinct(tries); + TrieUtil.assertSameContent(union, content); + Assert.fail("Expected assertion error for duplicate keys."); + } + catch (AssertionError e) + { + // correct path + } } } diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTriePutTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTriePutTest.java index 22e02087de13..7ddd571863cf 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTriePutTest.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTriePutTest.java @@ -27,6 +27,7 @@ import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; import static org.junit.Assert.fail; public class InMemoryTriePutTest extends InMemoryTrieTestBase @@ -48,7 +49,7 @@ public void testLongKey_StackOverflow() throws TrieSpaceExhaustedException try { - trie.putRecursive(ByteComparable.preencoded(byteComparableVersion, buf), "value", (x, y) -> y); + trie.putRecursive(ByteComparable.preencoded(VERSION, buf), "value", (x, y) -> y); Assert.fail("StackOverflowError expected with a recursive put for very long keys!"); } catch (StackOverflowError soe) @@ -56,7 +57,7 @@ public void testLongKey_StackOverflow() throws TrieSpaceExhaustedException // Expected. } // Using non-recursive put should work. - putSimpleResolve(trie, ByteComparable.preencoded(byteComparableVersion, buf), "value", (x, y) -> y, false); + putSimpleResolve(trie, ByteComparable.preencoded(VERSION, buf), "value", (x, y) -> y, false); } // This tests that trie space allocation works correctly close to the 2G limit. It is normally disabled because @@ -71,27 +72,27 @@ public void testOver1GSize() throws TrieSpaceExhaustedException String t1 = "test1"; String t2 = "testing2"; String t3 = "onemoretest3"; - trie.putRecursive(ByteComparable.of(t1), t1, (x, y) -> y); - Assert.assertEquals(t1, trie.get(ByteComparable.of(t1))); - Assert.assertNull(trie.get(ByteComparable.of(t2))); + trie.putRecursive(TrieUtil.comparable(t1), t1, (x, y) -> y); + Assert.assertEquals(t1, trie.get(TrieUtil.comparable(t1))); + Assert.assertNull(trie.get(TrieUtil.comparable(t2))); Assert.assertFalse(trie.reachedAllocatedSizeThreshold()); trie.advanceAllocatedPos(0x40001000); // over 1G - trie.putRecursive(ByteComparable.of(t2), t2, (x, y) -> y); - Assert.assertEquals(t1, trie.get(ByteComparable.of(t1))); - Assert.assertEquals(t2, trie.get(ByteComparable.of(t2))); - Assert.assertNull(trie.get(ByteComparable.of(t3))); + trie.putRecursive(TrieUtil.comparable(t2), t2, (x, y) -> y); + Assert.assertEquals(t1, trie.get(TrieUtil.comparable(t1))); + Assert.assertEquals(t2, trie.get(TrieUtil.comparable(t2))); + Assert.assertNull(trie.get(TrieUtil.comparable(t3))); Assert.assertTrue(trie.reachedAllocatedSizeThreshold()); trie.advanceAllocatedPos(0x7FFFFEE0); // close to 2G - Assert.assertEquals(t1, trie.get(ByteComparable.of(t1))); - Assert.assertEquals(t2, trie.get(ByteComparable.of(t2))); - Assert.assertNull(trie.get(ByteComparable.of(t3))); + Assert.assertEquals(t1, trie.get(TrieUtil.comparable(t1))); + Assert.assertEquals(t2, trie.get(TrieUtil.comparable(t2))); + Assert.assertNull(trie.get(TrieUtil.comparable(t3))); Assert.assertTrue(trie.reachedAllocatedSizeThreshold()); try { - trie.putRecursive(ByteComparable.of(t3), t3, (x, y) -> y); // should put it over the edge + trie.putRecursive(TrieUtil.comparable(t3), t3, (x, y) -> y); // should put it over the edge fail("InMemoryTrie.SpaceExhaustedError was expected"); } catch (TrieSpaceExhaustedException e) @@ -99,9 +100,9 @@ public void testOver1GSize() throws TrieSpaceExhaustedException // expected } - Assert.assertEquals(t1, trie.get(ByteComparable.of(t1))); - Assert.assertEquals(t2, trie.get(ByteComparable.of(t2))); - Assert.assertNull(trie.get(ByteComparable.of(t3))); + Assert.assertEquals(t1, trie.get(TrieUtil.comparable(t1))); + Assert.assertEquals(t2, trie.get(TrieUtil.comparable(t2))); + Assert.assertNull(trie.get(TrieUtil.comparable(t3))); Assert.assertTrue(trie.reachedAllocatedSizeThreshold()); try @@ -114,9 +115,11 @@ public void testOver1GSize() throws TrieSpaceExhaustedException // expected } - Assert.assertEquals(t1, trie.get(ByteComparable.of(t1))); - Assert.assertEquals(t2, trie.get(ByteComparable.of(t2))); - Assert.assertNull(trie.get(ByteComparable.of(t3))); + Assert.assertEquals(t1, trie.get(TrieUtil.comparable(t1))); + Assert.assertEquals(t2, trie.get(TrieUtil.comparable(t2))); + Assert.assertNull(trie.get(TrieUtil.comparable(t3))); Assert.assertTrue(trie.reachedAllocatedSizeThreshold()); + + trie.discardBuffers(); } } diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java index 974b5347e004..92ddadfc8477 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java @@ -42,6 +42,8 @@ import org.apache.cassandra.utils.ObjectSizes; import org.apache.cassandra.utils.bytecomparable.ByteSource; +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; import static org.junit.Assert.assertEquals; @RunWith(Parameterized.class) @@ -57,13 +59,7 @@ public static void enableVerification() // Do not commit the code with VERBOSE = true. private static final boolean VERBOSE = false; - // Set to true by some tests that need prefix-free keys. - static boolean prefixFree = false; - private static final int COUNT = 100000; - private static final int KEY_CHOICE = 25; - private static final int MIN_LENGTH = 10; - private static final int MAX_LENGTH = 50; Random rand = new Random(); @@ -88,12 +84,12 @@ static ByteSource invert(ByteSource src) @Test public void testSingle() { - ByteComparable e = ByteComparable.of("test"); + Preencoded e = TrieUtil.comparable("test"); InMemoryTrie trie = strategy.create(); putSimpleResolve(trie, e, "test", (x, y) -> y); System.out.println("Trie " + trie.dump()); assertEquals("test", trie.get(e)); - assertEquals(null, trie.get(ByteComparable.of("teste"))); + assertEquals(null, trie.get(TrieUtil.comparable("teste"))); } public enum ReuseStrategy @@ -102,40 +98,36 @@ public enum ReuseStrategy { InMemoryTrie create() { - return InMemoryTrie.shortLived(byteComparableVersion); + return InMemoryTrie.shortLived(VERSION); } }, LONG_LIVED { InMemoryTrie create() { - return InMemoryTrie.longLived(byteComparableVersion, BufferType.OFF_HEAP, null); + return InMemoryTrie.longLived(VERSION, BufferType.OFF_HEAP, null); } }; abstract InMemoryTrie create(); } - @Parameterized.Parameters(name="{0} version {1}") + @Parameterized.Parameters(name="{0}") public static List generateData() { var list = new ArrayList(); for (var s : ReuseStrategy.values()) - for (var v : ByteComparable.Version.values()) - list.add(new Object[] {s, v}); + list.add(new Object[] {s}); return list; } @Parameterized.Parameter(0) public static ReuseStrategy strategy = ReuseStrategy.LONG_LIVED; - @Parameterized.Parameter(1) - public static ByteComparable.Version byteComparableVersion = ByteComparable.Version.OSS50; - - public static Comparator forwardComparator = - (bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, byteComparableVersion); - public static Comparator reverseComparator = - (bytes1, bytes2) -> ByteComparable.compare(invert(bytes1), invert(bytes2), byteComparableVersion); + public static Comparator forwardComparator = + (bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION); + public static Comparator reverseComparator = + (bytes1, bytes2) -> ByteComparable.compare(invert(bytes1), invert(bytes2), VERSION); @Test public void testSplitMulti() @@ -161,7 +153,7 @@ public void testSparse00bug() InMemoryTrie trie = strategy.create(); for (String test : tests) { - ByteComparable e = ByteComparable.preencoded(byteComparableVersion, ByteBufferUtil.hexToBytes(test)); + Preencoded e = ByteComparable.preencoded(VERSION, ByteBufferUtil.hexToBytes(test)); System.out.println("Adding " + asString(e) + ": " + test); putSimpleResolve(trie, e, test, (x, y) -> y); } @@ -169,7 +161,7 @@ public void testSparse00bug() System.out.println(trie.dump()); for (String test : tests) - assertEquals(test, trie.get(ByteComparable.preencoded(byteComparableVersion, ByteBufferUtil.hexToBytes(test)))); + assertEquals(test, trie.get(ByteComparable.preencoded(VERSION, ByteBufferUtil.hexToBytes(test)))); Arrays.sort(tests); @@ -193,7 +185,7 @@ public void testUpdateContent() { String test = tests[i]; String v = values[i]; - ByteComparable e = ByteComparable.of(test); + Preencoded e = TrieUtil.comparable(test); System.out.println("Adding " + asString(e) + ": " + v); putSimpleResolve(trie, e, v, (x, y) -> "" + x + y); System.out.println("Trie " + trie.dump()); @@ -207,124 +199,10 @@ public void testUpdateContent() .filter(x -> tests[x] == test) .map(x -> values[x]) .reduce("", (x, y) -> "" + x + y), - trie.get(ByteComparable.of(test))); - } - } - - static class SpecStackEntry - { - Object[] children; - int curChild; - Object content; - SpecStackEntry parent; - - public SpecStackEntry(Object[] spec, Object content, SpecStackEntry parent, Direction direction) - { - this.children = spec; - this.content = content; - this.parent = parent; - this.curChild = direction.select(-1, spec.length); - } - } - - public static class CursorFromSpec implements Cursor - { - SpecStackEntry stack; - int depth; - Direction direction; - - CursorFromSpec(Object[] spec, Direction direction) - { - this.direction = direction; - stack = new SpecStackEntry(spec, null, null, direction); - depth = 0; - } - - public int advance() - { - SpecStackEntry current = stack; - while (current != null && !direction.inLoop(current.curChild += direction.increase, 0, current.children.length - 1)) - { - current = current.parent; - --depth; - } - if (current == null) - { - stack = current; - assert depth == -1; - return depth; - } - - Object child = current.children[current.curChild]; - if (child instanceof Object[]) - stack = new SpecStackEntry((Object[]) child, null, current, direction); - else - stack = new SpecStackEntry(new Object[0], child, current, direction); - - return ++depth; - } - - public int skipTo(int skipDepth, int skipTransition) - { - assert skipDepth <= depth + 1 : "skipTo descends more than one level"; - - while (skipDepth < depth) - { - --depth; - stack = stack.parent; - } - int index = skipTransition - 0x30; - assert direction.gt(index, stack.curChild) : "Backwards skipTo"; - if (direction.gt(index, direction.select(stack.children.length - 1, 0))) - { - --depth; - stack = stack.parent; - return advance(); - } - stack.curChild = index - direction.increase; - return advance(); - } - - public int depth() - { - return depth; - } - - public ByteBuffer content() - { - return stack != null ? (ByteBuffer) stack.content : null; - } - - public int incomingTransition() - { - SpecStackEntry parent = stack != null ? stack.parent : null; - return parent != null ? parent.curChild + 0x30 : -1; - } - - @Override - public Direction direction() - { - return direction; - } - - @Override - public ByteComparable.Version byteComparableVersion() - { - return byteComparableVersion; - } - - @Override - public Cursor tailCursor(Direction dir) - { - throw new UnsupportedOperationException("tailCursor on test cursor"); + trie.get(TrieUtil.comparable(test))); } } - static Trie specifiedTrie(Object[] nodeDef) - { - return direction -> new CursorFromSpec(nodeDef, direction); - } - @Test public void testEntriesNullChildBug() { @@ -354,7 +232,7 @@ public void testEntriesNullChildBug() ByteBufferUtil.bytes(6) // 6 }; - SortedMap expected = new TreeMap<>(forwardComparator); + SortedMap expected = new TreeMap<>(forwardComparator); expected.put(comparable("00"), ByteBufferUtil.bytes(1)); expected.put(comparable("01"), ByteBufferUtil.bytes(2)); expected.put(comparable("2"), ByteBufferUtil.bytes(3)); @@ -362,25 +240,25 @@ public void testEntriesNullChildBug() expected.put(comparable("4"), ByteBufferUtil.bytes(5)); expected.put(comparable("6"), ByteBufferUtil.bytes(6)); - Trie trie = specifiedTrie(trieDef); + Trie trie = TrieUtil.specifiedTrie(trieDef); System.out.println(trie.dump()); assertSameContent(trie, expected); } - static ByteComparable comparable(String s) + static Preencoded comparable(String s) { ByteBuffer b = ByteBufferUtil.bytes(s); - return ByteComparable.preencoded(byteComparableVersion, b); + return ByteComparable.preencoded(VERSION, b); } @Test public void testDirect() { - ByteComparable[] src = generateKeys(rand, COUNT); - SortedMap content = new TreeMap<>(forwardComparator); + Preencoded[] src = TrieUtil.generateKeys(rand, COUNT); + SortedMap content = new TreeMap<>(forwardComparator); InMemoryTrie trie = makeInMemoryTrie(src, content, usePut()); int keysize = Arrays.stream(src) - .mapToInt(src1 -> ByteComparable.length(src1, byteComparableVersion)) + .mapToInt(src1 -> ByteComparable.length(src1, VERSION)) .sum(); long ts = ObjectSizes.measureDeep(content); long onh = ObjectSizes.measureDeep(trie.contentArrays); @@ -455,9 +333,9 @@ public void testPrefixUnsafeMulti() private void testEntries(String[] tests) { - for (Function mapping : - ImmutableList.>of(ByteComparable::of, - s -> ByteComparable.preencoded(byteComparableVersion, s.getBytes()))) + for (Function mapping : + ImmutableList.>of(TrieUtil::comparable, + s -> ByteComparable.preencoded(VERSION, s.getBytes()))) { testEntries(tests, mapping); } @@ -465,18 +343,18 @@ private void testEntries(String[] tests) private void testEntriesHex(String[] tests) { - testEntries(tests, s -> ByteComparable.preencoded(byteComparableVersion, ByteBufferUtil.hexToBytes(s))); + testEntries(tests, s -> ByteComparable.preencoded(VERSION, ByteBufferUtil.hexToBytes(s))); // Run the other translations just in case. testEntries(tests); } - private void testEntries(String[] tests, Function mapping) + private void testEntries(String[] tests, Function mapping) { InMemoryTrie trie = strategy.create(); for (String test : tests) { - ByteComparable e = mapping.apply(test); + Preencoded e = mapping.apply(test); System.out.println("Adding " + asString(e) + ": " + test); putSimpleResolve(trie, e, test, (x, y) -> y); System.out.println("Trie\n" + trie.dump()); @@ -486,8 +364,8 @@ private void testEntries(String[] tests, Function mappin assertEquals(test, trie.get(mapping.apply(test))); } - static InMemoryTrie makeInMemoryTrie(ByteComparable[] src, - Map content, + static InMemoryTrie makeInMemoryTrie(Preencoded[] src, + Map content, boolean usePut) { @@ -496,18 +374,18 @@ static InMemoryTrie makeInMemoryTrie(ByteComparable[] src, return trie; } - static void addToInMemoryTrie(ByteComparable[] src, - Map content, + static void addToInMemoryTrie(Preencoded[] src, + Map content, InMemoryTrie trie, boolean usePut) { - for (ByteComparable b : src) + for (Preencoded b : src) addToInMemoryTrie(content, trie, usePut, b); } - static void addNthToInMemoryTrie(ByteComparable[] src, - Map content, + static void addNthToInMemoryTrie(Preencoded[] src, + Map content, InMemoryTrie trie, boolean usePut, int divisor, @@ -515,7 +393,7 @@ static void addNthToInMemoryTrie(ByteComparable[] src, { int i = 0; - for (ByteComparable b : src) + for (Preencoded b : src) { if (i++ % divisor != remainder) continue; @@ -524,7 +402,7 @@ static void addNthToInMemoryTrie(ByteComparable[] src, } } - private static void addToInMemoryTrie(Map content, InMemoryTrie trie, boolean usePut, ByteComparable b) + private static void addToInMemoryTrie(Map content, InMemoryTrie trie, boolean usePut, Preencoded b) { // Note: Because we don't ensure order when calling resolve, just use a hash of the key as payload // (so that all sources have the same value). @@ -538,11 +416,11 @@ private static void addToInMemoryTrie(Map content, I System.out.println(trie.dump(x -> string(x))); } - static void addToMap(ByteComparable[] src, - Map content) + static void addToMap(Preencoded[] src, + Map content) { - for (ByteComparable b : src) + for (Preencoded b : src) { // Note: Because we don't ensure order when calling resolve, just use a hash of the key as payload // (so that all sources have the same value). @@ -556,14 +434,14 @@ private static String string(Object x) { return x instanceof ByteBuffer ? ByteBufferUtil.bytesToHex((ByteBuffer) x) - : x instanceof ByteComparable - ? ((ByteComparable) x).byteComparableAsString(byteComparableVersion) + : x instanceof Preencoded + ? ((Preencoded) x).byteComparableAsString(VERSION) : x.toString(); } - static void checkGet(Trie trie, Map items) + static void checkGet(Trie trie, Map items) { - for (Map.Entry en : items.entrySet()) + for (Map.Entry en : items.entrySet()) { if (VERBOSE) System.out.println("Checking " + asString(en.getKey()) + ": " + ByteBufferUtil.bytesToHex(en.getValue())); @@ -571,7 +449,7 @@ static void checkGet(Trie trie, Map trie, SortedMap map) + static void assertSameContent(Trie trie, SortedMap map) { assertMapEquals(trie, map, Direction.FORWARD); assertForEachEntryEquals(trie, map, Direction.FORWARD); @@ -583,12 +461,12 @@ static void assertSameContent(Trie trie, SortedMap trie, SortedMap map) + private static void assertValuesEqual(Trie trie, SortedMap map) { assertIterablesEqual(trie.values(), map.values()); } - private static void assertUnorderedValuesEqual(Trie trie, SortedMap map) + private static void assertUnorderedValuesEqual(Trie trie, SortedMap map) { Multiset unordered = HashMultiset.create(); StringBuilder errors = new StringBuilder(); @@ -605,36 +483,36 @@ private static void assertUnorderedValuesEqual(Trie trie, SortedMap< assertEquals("", errors.toString()); } - static Collection maybeReversed(Direction direction, Collection data) + static Collection maybeReversed(Direction direction, Collection data) { return direction.isForward() ? data : reorderBy(data, reverseComparator); } - static Map maybeReversed(Direction direction, Map data) + static Map maybeReversed(Direction direction, Map data) { return direction.isForward() ? data : reorderBy(data, reverseComparator); } - private static Map reorderBy(Map data, Comparator comparator) + private static Map reorderBy(Map data, Comparator comparator) { - Map newMap = new TreeMap<>(comparator); + Map newMap = new TreeMap<>(comparator); newMap.putAll(data); return newMap; } - private static void assertForEachEntryEquals(Trie trie, SortedMap map, Direction direction) + private static void assertForEachEntryEquals(Trie trie, SortedMap map, Direction direction) { - Iterator> it = maybeReversed(direction, map).entrySet().iterator(); + Iterator> it = maybeReversed(direction, map).entrySet().iterator(); trie.forEachEntry(direction, (key, value) -> { Assert.assertTrue("Map exhausted first, key " + asString(key), it.hasNext()); - Map.Entry entry = it.next(); - assertEquals(0, ByteComparable.compare(entry.getKey(), key, byteComparableVersion)); + Map.Entry entry = it.next(); + assertEquals(0, ByteComparable.compare(entry.getKey(), key, VERSION)); assertEquals(entry.getValue(), value); }); Assert.assertFalse("Trie exhausted first", it.hasNext()); } - private static void assertForEachValueEquals(Trie trie, SortedMap map) + private static void assertForEachValueEquals(Trie trie, SortedMap map) { Iterator it = map.values().iterator(); trie.forEachValue(value -> { @@ -645,7 +523,7 @@ private static void assertForEachValueEquals(Trie trie, SortedMap trie, SortedMap map, Direction direction) + static void assertMapEquals(Trie trie, SortedMap map, Direction direction) { assertMapEquals(trie.entryIterator(direction), maybeReversed(direction, map).entrySet().iterator()); } @@ -657,29 +535,29 @@ static Collection reorderBy(Collection original, Comparator compara return list; } - static + static void assertMapEquals(Iterator> it1, Iterator> it2) { - List failedAt = new ArrayList<>(); + List failedAt = new ArrayList<>(); StringBuilder b = new StringBuilder(); while (it1.hasNext() && it2.hasNext()) { - Map.Entry en1 = it1.next(); - Map.Entry en2 = it2.next(); + Map.Entry en1 = it1.next(); + Map.Entry en2 = it2.next(); b.append(String.format("TreeSet %s:%s\n", asString(en2.getKey()), ByteBufferUtil.bytesToHex(en2.getValue()))); b.append(String.format("Trie %s:%s\n", asString(en1.getKey()), ByteBufferUtil.bytesToHex(en1.getValue()))); - if (ByteComparable.compare(en1.getKey(), en2.getKey(), byteComparableVersion) != 0 || ByteBufferUtil.compareUnsigned(en1.getValue(), en2.getValue()) != 0) + if (ByteComparable.compare(en1.getKey(), en2.getKey(), VERSION) != 0 || ByteBufferUtil.compareUnsigned(en1.getValue(), en2.getValue()) != 0) failedAt.add(en1.getKey()); } while (it1.hasNext()) { - Map.Entry en1 = it1.next(); + Map.Entry en1 = it1.next(); b.append(String.format("Trie %s:%s\n", asString(en1.getKey()), ByteBufferUtil.bytesToHex(en1.getValue()))); failedAt.add(en1.getKey()); } while (it2.hasNext()) { - Map.Entry en2 = it2.next(); + Map.Entry en2 = it2.next(); b.append(String.format("TreeSet %s:%s\n", asString(en2.getKey()), ByteBufferUtil.bytesToHex(en2.getValue()))); failedAt.add(en2.getKey()); } @@ -706,53 +584,13 @@ else if (actual.hasNext()) Assert.fail("Remaing values in actual, starting with " + actual.next()); } - static ByteComparable[] generateKeys(Random rand, int count) - { - ByteComparable[] sources = new ByteComparable[count]; - TreeSet added = new TreeSet<>(forwardComparator); - for (int i = 0; i < count; ++i) - { - sources[i] = generateKey(rand); - if (!added.add(sources[i])) - --i; - } - - // note: not sorted! - return sources; - } - - static ByteComparable generateKey(Random rand) - { - return generateKey(rand, MIN_LENGTH, MAX_LENGTH); - } - - static ByteComparable generateKey(Random rand, int minLength, int maxLength) - { - int len = rand.nextInt(maxLength - minLength + 1) + minLength; - byte[] bytes = new byte[len]; - int p = 0; - int length = bytes.length; - while (p < length) - { - int seed = rand.nextInt(KEY_CHOICE); - Random r2 = new Random(seed); - int m = r2.nextInt(5) + 2 + p; - if (m > length) - m = length; - while (p < m) - bytes[p++] = (byte) r2.nextInt(256); - } - return prefixFree ? v -> ByteSource.withTerminator(ByteSource.TERMINATOR, ByteSource.of(bytes, v)) - : ByteComparable.preencoded(byteComparableVersion, bytes); - } - - static String asString(ByteComparable bc) + static String asString(Preencoded bc) { - return bc != null ? bc.byteComparableAsString(byteComparableVersion) : "null"; + return bc != null ? bc.byteComparableAsString(VERSION) : "null"; } void putSimpleResolve(InMemoryTrie trie, - ByteComparable key, + Preencoded key, T value, Trie.MergeResolver resolver) { @@ -760,7 +598,7 @@ void putSimpleResolve(InMemoryTrie trie, } static void putSimpleResolve(InMemoryTrie trie, - ByteComparable key, + Preencoded key, T value, Trie.MergeResolver resolver, boolean usePut) diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java index cd7a1fee21d8..da9f1b6d527a 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java @@ -36,8 +36,8 @@ import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.apache.cassandra.utils.concurrent.OpOrder; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.byteComparableVersion; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.generateKeys; +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.apache.cassandra.db.tries.TrieUtil.generateKeys; public class InMemoryTrieThreadedTest { @@ -50,11 +50,6 @@ public class InMemoryTrieThreadedTest private static final int WALKERS = 2; private static final Random rand = new Random(); - static - { - InMemoryTrieTestBase.prefixFree = true; - } - /** * Force copy every modified cell below the partition/enumeration level. Provides atomicity of mutations within the * partition level as well as consistency. @@ -74,13 +69,13 @@ public class InMemoryTrieThreadedTest static Value value(ByteComparable b, ByteComparable cprefix, ByteComparable c, int add, int seqId) { - return new Value(b.byteComparableAsString(byteComparableVersion), - (cprefix != null ? cprefix.byteComparableAsString(byteComparableVersion) : "") + c.byteComparableAsString(byteComparableVersion), add, seqId); + return new Value(b.byteComparableAsString(VERSION), + (cprefix != null ? cprefix.byteComparableAsString(VERSION) : "") + c.byteComparableAsString(VERSION), add, seqId); } static String value(ByteComparable b) { - return b.byteComparableAsString(byteComparableVersion); + return b.byteComparableAsString(VERSION); } @Test @@ -88,7 +83,7 @@ public void testThreaded() throws InterruptedException { OpOrder readOrder = new OpOrder(); ByteComparable[] src = generateKeys(rand, COUNT + OTHERS); - InMemoryTrie trie = InMemoryTrie.longLived(byteComparableVersion, readOrder); + InMemoryTrie trie = InMemoryTrie.longLived(VERSION, readOrder); ConcurrentLinkedQueue errors = new ConcurrentLinkedQueue<>(); List threads = new ArrayList<>(); AtomicBoolean writeCompleted = new AtomicBoolean(false); @@ -107,7 +102,7 @@ public void testThreaded() throws InterruptedException for (Map.Entry en : trie.entrySet()) { String v = value(en.getKey()); - Assert.assertEquals(en.getKey().byteComparableAsString(byteComparableVersion), v, en.getValue()); + Assert.assertEquals(en.getKey().byteComparableAsString(VERSION), v, en.getValue()); ++count; } } @@ -368,7 +363,7 @@ public void testAtomicUpdates(int PER_MUTATION, OpOrder readOrder = new OpOrder(); // InMemoryTrie trie = new InMemoryTrie<>(new MemtableAllocationStrategy.NoReuseStrategy(BufferType.OFF_HEAP)); - InMemoryTrie trie = InMemoryTrie.longLived(byteComparableVersion, readOrder); + InMemoryTrie trie = InMemoryTrie.longLived(VERSION, readOrder); ConcurrentLinkedQueue errors = new ConcurrentLinkedQueue<>(); List threads = new ArrayList(); AtomicBoolean writeCompleted = new AtomicBoolean(false); @@ -422,13 +417,13 @@ public void run() try (OpOrder.Group group = readOrder.start()) { entries = trie.tailTrie(key).entrySet(); - checkEntries(" in tail " + key.byteComparableAsString(byteComparableVersion), min, false, checkAtomicity, checkSequence, PER_MUTATION, entries); + checkEntries(" in tail " + key.byteComparableAsString(VERSION), min, false, checkAtomicity, checkSequence, PER_MUTATION, entries); } try (OpOrder.Group group = readOrder.start()) { entries = trie.subtrie(key, nextBranch(key)).entrySet(); - checkEntries(" in branch " + key.byteComparableAsString(byteComparableVersion), min, true, checkAtomicity, checkSequence, PER_MUTATION, entries); + checkEntries(" in branch " + key.byteComparableAsString(VERSION), min, true, checkAtomicity, checkSequence, PER_MUTATION, entries); } } } @@ -468,7 +463,7 @@ public Content resolve(Collection contents) for (int i = 0; i < COUNT; i += PER_MUTATION) { ByteComparable b = pkeys[(i / PER_MUTATION) % pkeys.length]; - Metadata partitionMarker = new Metadata(b.byteComparableAsString(byteComparableVersion)); + Metadata partitionMarker = new Metadata(b.byteComparableAsString(VERSION)); ByteComparable cprefix = null; if (r.nextBoolean()) cprefix = ckeys[i]; // Also test branching point below the partition level @@ -478,7 +473,7 @@ public Content resolve(Collection contents) { ByteComparable k = ckeys[i + j]; - Trie row = Trie.singleton(k, byteComparableVersion, + Trie row = Trie.singleton(k, VERSION, value(b, cprefix, k, j == 0 ? -PER_MUTATION + 1 : 1, (i / PER_MUTATION / pkeys.length) * PER_MUTATION + j)); @@ -551,7 +546,7 @@ static ByteComparable nextBranch(ByteComparable key) static Trie withRootMetadata(Trie wrapped, T metadata) { - return wrapped.mergeWith(Trie.singleton(ByteComparable.EMPTY, byteComparableVersion, metadata), Trie.throwingResolver()); + return wrapped.mergeWith(Trie.singleton(ByteComparable.EMPTY, VERSION, metadata), Trie.throwingResolver()); } public void checkEntries(String location, @@ -569,7 +564,7 @@ public void checkEntries(String location, int updateCount = 0; for (var en : entries) { - String path = en.getKey().byteComparableAsString(byteComparableVersion); + String path = en.getKey().byteComparableAsString(VERSION); if (en.getValue().isPartition()) { Metadata m = (Metadata) en.getValue(); diff --git a/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java b/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java new file mode 100644 index 000000000000..4c91222b6a77 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java @@ -0,0 +1,518 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.NavigableMap; +import java.util.Random; +import java.util.TreeMap; +import java.util.stream.Collectors; + +import com.google.common.base.Predicates; +import com.google.common.collect.ImmutableList; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.googlecode.concurrenttrees.common.Iterables; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +import static java.util.Arrays.asList; +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.apache.cassandra.db.tries.TrieUtil.asString; +import static org.apache.cassandra.db.tries.TrieUtil.assertMapEquals; +import static org.apache.cassandra.db.tries.TrieUtil.generateKeys; +import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.makeInMemoryTrie; +import static org.apache.cassandra.db.tries.TrieUtil.toBound; +import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; +import static org.junit.Assert.assertEquals; + +public class IntersectionTrieTest +{ + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + + private static final int COUNT = 15000; + Random rand = new Random(); + int seed = rand.nextInt(); + final static int bitsNeeded = 4; + int bits = bitsNeeded; + + + public static final Trie.CollectionMergeResolver RESOLVER = new Trie.CollectionMergeResolver<>() + { + public Integer resolve(Collection contents) + { + return contents.iterator().next(); + } + + public Integer resolve(Integer b1, Integer b2) + { + return b1; + } + }; + + interface RangeOp + { + Trie apply(Trie t, ByteComparable left, ByteComparable right); + } + + @Test + public void testIntersectRangeDirect() throws Exception + { + testIntersectRange(COUNT, Trie::subtrie); + } + + @Test + public void testIntersectRangesOneDirect() throws Exception + { + testIntersectRange(COUNT, (t, l, r) -> t.intersect(TrieSet.ranges(VERSION, l, r))); + } + + public void testIntersectRange(int count, RangeOp op) throws Exception + { + System.out.format("intersectrange seed %d\n", ++seed); + rand.setSeed(seed); + Preencoded[] src1 = generateKeys(rand, count); + NavigableMap content1 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION)); + + InMemoryTrie trie1 = makeInMemoryTrie(src1, content1, true); + + Trie t1 = trie1; + + checkEqualRange(content1, t1, null, null, op); + checkEqualRange(content1, t1, TrieUtil.generateKeyBound(rand), null, op); + checkEqualRange(content1, t1, null, TrieUtil.generateKeyBound(rand), op); + + Preencoded l = rand.nextBoolean() ? TrieUtil.generateKeyBound(rand) : toBound(src1[rand.nextInt(src1.length)]); + Preencoded r = rand.nextBoolean() ? TrieUtil.generateKeyBound(rand) : toBound(src1[rand.nextInt(src1.length)]); + int cmp = ByteComparable.compare(l, r, VERSION); + if (cmp > 0) + { + Preencoded t = l;l = r;r = t; // swap + } + + checkEqualRange(content1, t1, l, r, op); + } + + public void checkEqualRange(NavigableMap content1, + Trie t1, + Preencoded l, + Preencoded r, + RangeOp op) throws Exception + { + System.out.format("Intersection with [%s:%s]\n", asString(l), asString(r)); + NavigableMap imap = SlicedTrieTest.boundedMap(content1, l, true, r, false); + + Trie intersection = op.apply(t1, l, r); + + assertMapEquals(intersection, imap, Direction.FORWARD); + assertMapEquals(intersection, imap, Direction.REVERSE); + } + + /** + * Extract the values of the provide trie into a list. + */ + private static List toList(Trie trie, Direction direction) + { + return Iterables.toList(trie.values(direction)); + } + + private Trie fromList(int... list) throws TrieSpaceExhaustedException + { + InMemoryTrie trie = InMemoryTrie.shortLived(VERSION); + for (int i : list) + { + trie.putRecursive(at(i), i, (ex, n) -> n); + } + return trie; + } + + /** Creates a {@link ByteComparable} for the provided value by splitting the integer in sequences of "bits" bits. */ + private ByteComparable of(int value, int terminator) + { + // TODO: Also in all other tests of this type + assert value >= 0 && value <= Byte.MAX_VALUE; + + byte[] splitBytes = new byte[(bitsNeeded + bits - 1) / bits + 1]; + int pos = 0; + int mask = (1 << bits) - 1; + for (int i = bitsNeeded - bits; i > 0; i -= bits) + splitBytes[pos++] = (byte) ((value >> i) & mask); + + splitBytes[pos++] = (byte) (value & mask); + splitBytes[pos++] = (byte) terminator; + return ByteComparable.preencoded(VERSION, splitBytes); + } + + private ByteComparable at(int value) + { + return of(value, ByteSource.TERMINATOR); + } + + private ByteComparable before(int value) + { + return of(value, ByteSource.LT_NEXT_COMPONENT); + } + + @Test + public void testSimpleSubtrie() throws TrieSpaceExhaustedException + { + for (bits = bitsNeeded; bits > 0; --bits) + { + Trie trie = fromList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); + + testIntersection("", asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + testIntersection("", asList(3, 4, 5, 6), trie, + TrieSet.range(VERSION, before(3), before(7))); + + testIntersection("", asList(0, 1, 2, 3, 4, 5, 6), trie, + TrieSet.range(VERSION, null, before(7))); + + testIntersection("", asList(3, 4, 5, 6, 7, 8, 9), trie, + TrieSet.range(VERSION, before(3), null)); + + testIntersection("", asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie, + TrieSet.range(VERSION, null, null)); + + testIntersection("", asList(), trie, + TrieSet.range(VERSION, before(7), before(7))); + } + } + + @Test + public void testRangeOnSubtrie() throws TrieSpaceExhaustedException + { + for (bits = bitsNeeded; bits > 0; --bits) + { + Trie trie = fromList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); + + // non-overlapping + testIntersection("", asList(), trie, + TrieSet.range(VERSION, before(0), before(3)), + TrieSet.range(VERSION, before(4), before(7))); + // touching, i.e. still non-overlapping + testIntersection("", asList(), trie, + TrieSet.range(VERSION, before(0), before(3)), + TrieSet.range(VERSION, before(3), before(7))); + // overlapping 1 + testIntersection("", asList(2), trie, + TrieSet.range(VERSION, before(0), before(3)), + TrieSet.range(VERSION, before(2), before(7))); + // overlapping 2 + testIntersection("", asList(1, 2), trie, + TrieSet.range(VERSION, before(0), before(3)), + TrieSet.range(VERSION, before(1), before(7))); + // covered + testIntersection("", asList(0, 1, 2), trie, + TrieSet.range(VERSION, before(0), before(3)), + TrieSet.range(VERSION, before(0), before(7))); + // covered 2 + testIntersection("", asList(1, 2), trie, + TrieSet.range(VERSION, before(1), before(3)), + TrieSet.range(VERSION, before(0), before(7))); + } + } + + @Test + public void testSimpleRanges() throws TrieSpaceExhaustedException + { + for (bits = bitsNeeded; bits > 0; --bits) + { + Trie trie = fromList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); + + testIntersection("", asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + testIntersection("", asList(3, 4, 5, 6), trie, + TrieSet.ranges(VERSION, before(3), before(7))); + + testIntersection("", asList(3), trie, + TrieSet.ranges(VERSION, before(3), before(4))); + + testIntersection("", asList(0, 1, 2, 3, 4, 5, 6), trie, + TrieSet.ranges(VERSION, null, before(7))); + + testIntersection("", asList(3, 4, 5, 6, 7, 8, 9), trie, + TrieSet.ranges(VERSION, before(3), null)); + + testIntersection("", asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie, + TrieSet.ranges(VERSION, null, null)); + + testIntersection("", asList(3, 4, 5, 7, 8), trie, + TrieSet.ranges(VERSION, before(3), before(6), before(7), before(9))); + + testIntersection("", asList(3, 7, 8), trie, + TrieSet.ranges(VERSION, before(3), before(4), before(7), before(9))); + + testIntersection("", asList(3, 7, 8), trie, + TrieSet.ranges(VERSION, before(3), before(4), before(7), before(9), before(12), before(15))); + + testIntersection("", asList(3, 4, 5, 6, 7, 8), trie, + TrieSet.ranges(VERSION, before(3), before(9))); + + testIntersection("", asList(3), trie, + TrieSet.ranges(VERSION, before(3), before(4))); + + testIntersection("", asList(0, 1, 2, 3, 4, 5, 7, 8), trie, + TrieSet.ranges(VERSION, null, before(6), before(7), before(9))); + + testIntersection("", asList(3, 4, 5, 7, 8, 9), trie, + TrieSet.ranges(VERSION, before(3), before(6), before(7), null)); + + testIntersection("", asList(0, 1, 2, 3, 4, 5, 7, 8, 9), trie, + TrieSet.ranges(VERSION, null, before(6), before(7), null)); + + testIntersection("", asList(3, 4, 5, 6, 7, 8), trie, + TrieSet.ranges(VERSION, before(3), before(6), before(6), before(9))); + + testIntersection("", asList(3, 4, 5, 7, 8), trie, + TrieSet.ranges(VERSION, before(3), before(6), before(6), before(6), before(7), before(9))); + } + } + + @Test + public void testRangesOnRangesOne() throws TrieSpaceExhaustedException + { + for (bits = bitsNeeded; bits > 0; --bits) + { + Trie trie = fromList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14); + + // non-overlapping + testIntersection("non-overlapping", asList(), trie, + TrieSet.ranges(VERSION, before(0), before(4)), + TrieSet.ranges(VERSION, before(4), before(8))); + // touching + testIntersection("touching", asList(3), trie, + TrieSet.ranges(VERSION, before(0), before(4)), + TrieSet.ranges(VERSION, before(3), before(8))); + // overlapping 1 + testIntersection("overlapping A", asList(2, 3), trie, + TrieSet.ranges(VERSION, before(0), before(4)), + TrieSet.ranges(VERSION, before(2), before(8))); + // overlapping 2 + testIntersection("overlapping B", asList(1, 2, 3), trie, + TrieSet.ranges(VERSION, before(0), before(4)), + TrieSet.ranges(VERSION, before(1), before(8))); + // covered + testIntersection("covered same end A", asList(0, 1, 2, 3), trie, + TrieSet.ranges(VERSION, before(0), before(4)), + TrieSet.ranges(VERSION, before(0), before(8))); + // covered 2 + testIntersection("covered same end B", asList(4, 5, 6, 7), trie, + TrieSet.ranges(VERSION, before(4), before(8)), + TrieSet.ranges(VERSION, before(0), before(8))); + // covered 3 + testIntersection("covered", asList(1, 2, 3), trie, + TrieSet.ranges(VERSION, before(1), before(4)), + TrieSet.ranges(VERSION, before(0), before(8))); + } + } + + @Test + public void testRangesOnRanges() throws TrieSpaceExhaustedException + { + for (bits = bitsNeeded; bits > 0; --bits) + testIntersections(fromList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)); + } + + @Test + public void testRangesOnMerge() throws TrieSpaceExhaustedException + { + + for (bits = bitsNeeded; bits > 0; --bits) + testIntersections(Trie.merge(ImmutableList.of(fromList(0, 1, 2, 3, 5, 8, 9, 13, 14), + fromList(4, 6, 7, 9, 10, 11, 12, 13)), + RESOLVER)); + } + + @Test + public void testRangesOnCollectionMerge2() throws TrieSpaceExhaustedException + { + for (bits = bitsNeeded; bits > 0; --bits) + { + List> inputs = ImmutableList.of(fromList(0, 1, 2, 3, 5, 8, 9, 13, 14), + fromList(4, 6, 7, 9, 10, 11, 12, 13)); + testIntersections(dir -> new CollectionMergeCursor<>(RESOLVER, dir, inputs, Trie::cursor)); + } + } + + @Test + public void testRangesOnCollectionMerge3() throws TrieSpaceExhaustedException + { + for (bits = bitsNeeded; bits > 0; --bits) + testIntersections(Trie.merge( + ImmutableList.of(fromList(0, 1, 2, 3, 5, 8, 9, 13, 14), + fromList(4, 6, 9, 10), + fromList(4, 7, 11, 12, 13)), + RESOLVER)); + } + + @Test + public void testRangesOnCollectionMerge10() throws TrieSpaceExhaustedException + { + for (bits = bitsNeeded; bits > 0; --bits) + testIntersections(Trie.merge( + ImmutableList.of(fromList(0, 14), + fromList(1, 2), + fromList(2, 13), + fromList(3), + fromList(4, 7), + fromList(5, 9, 12), + fromList(6, 8), + fromList(7), + fromList(8), + fromList(10, 11)), + RESOLVER)); + } + + private void testIntersections(Trie trie) + { + testIntersection("", asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14), trie); + + TrieSet set1 = TrieSet.ranges(VERSION, null, before(4), before(5), before(9), before(12), null); + TrieSet set2 = TrieSet.ranges(VERSION, before(2), before(7), before(8), before(10), before(12), before(14)); + TrieSet set3 = TrieSet.ranges(VERSION, before(1), before(2), before(3), before(4), before(5), before(6), before(7), before(8), before(9), before(10)); + + testIntersections(trie, set1, set2, set3); + + testSetAlgebraIntersection(trie); + } + + private void testSetAlgebraIntersection(Trie trie) + { + TrieSet set1 = TrieSet.range(VERSION, null, before(3)) + .union(TrieSet.range(VERSION, before(2), before(4))) + .union(TrieSet.range(VERSION, before(5), before(7))) + .union(TrieSet.range(VERSION, before(7), before(9))) + .union(TrieSet.range(VERSION, before(14), before(16))) + .union(TrieSet.range(VERSION, before(12), null)); + TrieSet set2 = TrieSet.range(VERSION, before(2), before(7)) + .union(TrieSet.ranges(VERSION, null, before(8), before(10), null).weakNegation()) + .union(TrieSet.ranges(VERSION, before(8), before(10), before(12), before(14))); + TrieSet set3 = TrieSet.range(VERSION, before(1), before(2)) + .union(TrieSet.range(VERSION, before(3), before(4))) + .union(TrieSet.range(VERSION, before(5), before(6))) + .union(TrieSet.range(VERSION, before(7), before(8))) + .union(TrieSet.range(VERSION, before(9), before(10))); + + testIntersections(trie, set1, set2, set3); + } + + private void testIntersections(Trie trie, TrieSet set1, TrieSet set2, TrieSet set3) + { + testIntersection("1", asList(0, 1, 2, 3, 5, 6, 7, 8, 12, 13, 14), trie, set1); + + testIntersection("2", asList(2, 3, 4, 5, 6, 8, 9, 12, 13), trie, set2); + + testIntersection("3", asList(1, 3, 5, 7, 9), trie, set3); + + testIntersection("12", asList(2, 3, 5, 6, 8, 12, 13), trie, set1, set2); + + testIntersection("13", asList(1, 3, 5, 7), trie, set1, set3); + + testIntersection("23", asList(3, 5, 9), trie, set2, set3); + + testIntersection("123", asList(3, 5), trie, set1, set2, set3); + } + + public void testIntersection(String message, List expected, Trie trie, TrieSet... sets) + { + testIntersectionTries(message, expected, trie, sets); + testIntersectionSets(message + " setix", expected, trie, TrieSet.range(VERSION, null, null), sets); + } + + public void checkEqual(String message, List expected, Trie trie) + { + assertEquals(message + " forward", expected, toList(trie, Direction.FORWARD)); + assertEquals(message + " reverse", expected.stream() + .sorted(Comparator.naturalOrder().reversed()) + .collect(Collectors.toList()), + toList(trie, Direction.REVERSE)); + } + + public void testIntersectionSets(String message, List expected, Trie trie, TrieSet intersectedSet, TrieSet[] sets) + { + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + checkEqual(message + " b" + bits, expected, trie.intersect(intersectedSet)); + } + else + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + TrieSet set = sets[toRemove]; + testIntersectionSets(message + " " + toRemove, expected, + trie, + intersectedSet.intersection(set), + Arrays.stream(sets) + .filter(x -> x != set) + .toArray(TrieSet[]::new) + ); + } + } + } + + public void testIntersectionTries(String message, List expected, Trie trie, TrieSet[] sets) + { + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + checkEqual(message + " b" + bits, expected, trie); + } + else + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + TrieSet set = sets[toRemove]; + testIntersectionTries(message + " " + toRemove, expected, + trie.intersect(set), + Arrays.stream(sets) + .filter(x -> x != set) + .toArray(TrieSet[]::new) + ); + } + } + } + + private static InMemoryTrie duplicateTrie(Trie trie) + { + try + { + InMemoryTrie dupe = InMemoryTrie.shortLived(VERSION); + dupe.apply(trie, (x, y) -> y, Predicates.alwaysFalse()); + return dupe; + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java b/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java index f6711f331884..def5b4650d91 100644 --- a/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java @@ -20,7 +20,6 @@ import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.List; import java.util.Random; import java.util.SortedMap; import java.util.TreeMap; @@ -29,7 +28,7 @@ import org.junit.Test; import org.apache.cassandra.config.CassandraRelevantProperties; -import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.*; @@ -47,10 +46,10 @@ public static void enableVerification() @Test public void testDirect() { - ByteComparable[] src1 = generateKeys(rand, COUNT); - ByteComparable[] src2 = generateKeys(rand, COUNT); - SortedMap content1 = new TreeMap<>(forwardComparator); - SortedMap content2 = new TreeMap<>(forwardComparator); + Preencoded[] src1 = TrieUtil.generateKeys(rand, COUNT); + Preencoded[] src2 = TrieUtil.generateKeys(rand, COUNT); + SortedMap content1 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); + SortedMap content2 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); InMemoryTrie trie1 = makeInMemoryTrie(src1, content1, true); InMemoryTrie trie2 = makeInMemoryTrie(src2, content2, true); @@ -64,45 +63,45 @@ public void testDirect() @Test public void testWithDuplicates() { - ByteComparable[] src1 = generateKeys(rand, COUNT); - ByteComparable[] src2 = generateKeys(rand, COUNT); - SortedMap content1 = new TreeMap<>(forwardComparator); - SortedMap content2 = new TreeMap<>(forwardComparator); + Preencoded[] src1 = TrieUtil.generateKeys(rand, COUNT); + Preencoded[] src2 = TrieUtil.generateKeys(rand, COUNT); + SortedMap content1 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); + SortedMap content2 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); InMemoryTrie trie1 = makeInMemoryTrie(src1, content1, true); InMemoryTrie trie2 = makeInMemoryTrie(src2, content2, true); - addToInMemoryTrie(generateKeys(new Random(5), COUNT), content1, trie1, true); - addToInMemoryTrie(generateKeys(new Random(5), COUNT), content2, trie2, true); + addToInMemoryTrie(TrieUtil.generateKeys(new Random(5), COUNT), content1, trie1, true); + addToInMemoryTrie(TrieUtil.generateKeys(new Random(5), COUNT), content2, trie2, true); content1.putAll(content2); Trie union = trie1.mergeWith(trie2, (x, y) -> y); - assertSameContent(union, content1); + TrieUtil.assertSameContent(union, content1); } @Test public void testDistinct() { - ByteComparable[] src1 = generateKeys(rand, COUNT); - SortedMap content1 = new TreeMap<>(forwardComparator); + Preencoded[] src1 = TrieUtil.generateKeys(rand, COUNT); + SortedMap content1 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); InMemoryTrie trie1 = makeInMemoryTrie(src1, content1, true); - ByteComparable[] src2 = generateKeys(rand, COUNT); + Preencoded[] src2 = TrieUtil.generateKeys(rand, COUNT); src2 = removeDuplicates(src2, content1); - SortedMap content2 = new TreeMap<>(forwardComparator); + SortedMap content2 = new TreeMap<>(TrieUtil.FORWARD_COMPARATOR); InMemoryTrie trie2 = makeInMemoryTrie(src2, content2, true); content1.putAll(content2); Trie union = Trie.mergeDistinct(trie1, trie2); - assertSameContent(union, content1); + TrieUtil.assertSameContent(union, content1); } - static ByteComparable[] removeDuplicates(ByteComparable[] keys, SortedMap content1) + static Preencoded[] removeDuplicates(Preencoded[] keys, SortedMap content1) { return Arrays.stream(keys) .filter(key -> !content1.containsKey(key)) - .toArray(ByteComparable[]::new); + .toArray(Preencoded[]::new); } } diff --git a/test/unit/org/apache/cassandra/db/tries/PrefixTailTrieTest.java b/test/unit/org/apache/cassandra/db/tries/PrefixTailTrieTest.java index ffa50b1d0273..eca0d163f8b7 100644 --- a/test/unit/org/apache/cassandra/db/tries/PrefixTailTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/PrefixTailTrieTest.java @@ -40,14 +40,15 @@ import org.apache.cassandra.utils.Hex; import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.byteComparableVersion; import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.addNthToInMemoryTrie; import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.addToInMemoryTrie; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.assertIterablesEqual; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.assertMapEquals; import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.checkGet; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.generateKey; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.generateKeys; +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.apache.cassandra.db.tries.TrieUtil.assertIterablesEqual; +import static org.apache.cassandra.db.tries.TrieUtil.assertMapEquals; +import static org.apache.cassandra.db.tries.TrieUtil.generateKey; +import static org.apache.cassandra.db.tries.TrieUtil.generateKeys; +import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -62,15 +63,9 @@ public static void enableVerification() private static final int COUNT_TAIL = 5000; private static final int COUNT_HEAD = 25; - public static final Comparator BYTE_COMPARABLE_COMPARATOR = (a, b) -> ByteComparable.compare(a, b, byteComparableVersion); + public static final Comparator BYTE_COMPARABLE_COMPARATOR = (a, b) -> ByteComparable.compare(a, b, VERSION); Random rand = new Random(); - static - { - // Use prefix-free keys to avoid putting partitions within partitions - InMemoryTrieTestBase.prefixFree = true; - } - static final InMemoryTrie.UpsertTransformer THROWING_UPSERT = (e, u) -> { if (e != null) throw new AssertionError(); return u; @@ -83,9 +78,9 @@ public static void enableVerification() static class Tail { byte[] prefix; - NavigableMap data; + NavigableMap data; - public Tail(byte[] prefix, NavigableMap map) + public Tail(byte[] prefix, NavigableMap map) { this.prefix = prefix; this.data = map; @@ -146,7 +141,7 @@ static Tail combineTails(Object x, Object y) // Cast failure is a test problem Tail tx = (Tail) x; Tail ty = (Tail) y; - var map = new TreeMap(BYTE_COMPARABLE_COMPARATOR); + var map = new TreeMap(BYTE_COMPARABLE_COMPARATOR); map.putAll(tx.data); map.putAll(ty.data); return new Tail(tx.prefix, map); @@ -154,9 +149,9 @@ static Tail combineTails(Object x, Object y) public void testPrefixTail(int splits, boolean splitInTail) throws Exception { - ByteComparable[] prefixes = generateKeys(rand, COUNT_HEAD); + Preencoded[] prefixes = generateKeys(rand, COUNT_HEAD); - NavigableMap data = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); + NavigableMap data = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); final Trie trie = splitInTail ? prepareSplitInTailTrie(splits, prefixes, data) : prepareSplitInHeadTrie(splits, prefixes, data); // System.out.println(trie.dump(CONTENT_TO_STRING)); @@ -176,7 +171,7 @@ public void testPrefixTail(int splits, boolean splitInTail) throws Exception long count = 0; for (var en : trie.tailTries(td, Tail.class)) { - System.out.println(en.getKey().byteComparableAsString(byteComparableVersion)); + System.out.println(en.getKey().byteComparableAsString(VERSION)); Trie tail = en.getValue(); Tail t = data.get(en.getKey()); assertNotNull(t); @@ -194,20 +189,18 @@ public void testPrefixTail(int splits, boolean splitInTail) throws Exception int keyCount = t.data.keySet().size(); int firstIndex = rand.nextInt(keyCount - 1); int lastIndex = firstIndex + rand.nextInt(keyCount - firstIndex); - ByteComparable first = rand.nextInt(5) > 0 ? Iterables.get(t.data.keySet(), firstIndex) : null; - ByteComparable last = rand.nextInt(5) > 0 ? Iterables.get(t.data.keySet(), lastIndex) : null; - ByteComparable prefix = prefixes[i]; - final ByteComparable leftWithPrefix = concat(prefix, first, rand.nextBoolean() ? prefix - : rand.nextBoolean() - ? data.lowerKey(prefix) - : null); + Preencoded first = rand.nextInt(5) > 0 ? Iterables.get(t.data.keySet(), firstIndex) : null; + Preencoded last = rand.nextInt(5) > 0 ? Iterables.get(t.data.keySet(), lastIndex) : null; + Preencoded prefix = prefixes[i]; + final ByteComparable leftWithPrefix = concat(prefix, first, rand.nextBoolean() ? data.lowerKey(prefix) + : null); final ByteComparable rightWithPrefix = concat(prefix, last, rand.nextBoolean() ? data.higherKey(prefix) : null); + System.out.println("Between " + (leftWithPrefix == null ? "null" : leftWithPrefix.byteComparableAsString(VERSION)) + " and " + (rightWithPrefix == null ? "null" : rightWithPrefix.byteComparableAsString(VERSION))); Trie tail = trie.subtrie(leftWithPrefix, rightWithPrefix) .tailTrie(prefixes[i]); - System.out.println("Between " + (leftWithPrefix == null ? "null" : leftWithPrefix.byteComparableAsString(byteComparableVersion)) + " and " + (rightWithPrefix == null ? "null" : rightWithPrefix.byteComparableAsString(byteComparableVersion))); - assertEquals(first == null ? t : null, getRootContent(tail)); // this behavior will change soon to report all prefixes + assertEquals(t, getRootContent(tail)); checkContent(tail, subMap(t.data, first, last)); } @@ -221,14 +214,14 @@ public void testPrefixTail(int splits, boolean splitInTail) throws Exception count.set(0); trie.forEachEntrySkippingBranches(td, (key, tail) -> { - assertArrayEquals(((Tail) tail).prefix, key.asByteComparableArray(byteComparableVersion)); + assertArrayEquals(((Tail) tail).prefix, key.asByteComparableArray(VERSION)); count.incrementAndGet(); }); assertEquals(COUNT_HEAD, count.get()); } } - private static void checkContent(Trie tail, NavigableMap data) + private static void checkContent(Trie tail, NavigableMap data) { assertMapEquals(tail.filteredEntryIterator(Direction.FORWARD, ByteBuffer.class), @@ -245,72 +238,73 @@ private static void checkContent(Trie tail, NavigableMap NavigableMap subMap(NavigableMap data, K left, K right) { + // Subtries are always inclusive. if (left == null) - return right == null ? data : data.headMap(right, false); + return right == null ? data : data.headMap(right, true); else return right == null ? data.tailMap(left, true) - : data.subMap(left, true, right, false); + : data.subMap(left, true, right, true); } private static ByteComparable concat(ByteComparable a, ByteComparable b, ByteComparable ifBNull) { if (b == null) return ifBNull; - return ByteComparable.preencoded(byteComparableVersion, - Bytes.concat(a.asByteComparableArray(byteComparableVersion), - b.asByteComparableArray(byteComparableVersion))); + return ByteComparable.preencoded(VERSION, + Bytes.concat(a.asByteComparableArray(VERSION), + b.asByteComparableArray(VERSION))); } - private Trie prepareSplitInTailTrie(int splits, ByteComparable[] prefixes, Map data) throws TrieSpaceExhaustedException + private Trie prepareSplitInTailTrie(int splits, Preencoded[] prefixes, Map data) throws TrieSpaceExhaustedException { InMemoryTrie[] tries = new InMemoryTrie[splits]; for (int i = 0; i < splits; ++i) - tries[i] = InMemoryTrie.shortLived(byteComparableVersion); + tries[i] = InMemoryTrie.shortLived(VERSION); for (int i = 0; i < COUNT_HEAD; ++i) { - ByteComparable[] src = generateKeys(rand, COUNT_TAIL); - NavigableMap allContent = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); + Preencoded[] src = generateKeys(rand, COUNT_TAIL); + NavigableMap allContent = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); for (int k = 0; k < splits; ++k) { - NavigableMap content = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); - InMemoryTrie tail = InMemoryTrie.shortLived(byteComparableVersion); + NavigableMap content = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); + InMemoryTrie tail = InMemoryTrie.shortLived(VERSION); addNthToInMemoryTrie(src, content, tail, true, splits, k); - Tail t = new Tail(prefixes[i].asByteComparableArray(byteComparableVersion), content); + Tail t = new Tail(prefixes[i].asByteComparableArray(VERSION), content); allContent.putAll(content); tail.putRecursive(ByteComparable.EMPTY, t, THROWING_UPSERT); // System.out.println(tail.dump(CONTENT_TO_STRING)); tries[k].apply(tail.prefixedBy(prefixes[i]), THROWING_UPSERT, Predicates.alwaysFalse()); } - Tail t = new Tail(prefixes[i].asByteComparableArray(byteComparableVersion), allContent); - data.put(ByteComparable.preencoded(byteComparableVersion, t.prefix), t); + Tail t = new Tail(prefixes[i].asByteComparableArray(VERSION), allContent); + data.put(ByteComparable.preencoded(VERSION, t.prefix), t); } return Trie.merge(Arrays.asList(tries), c -> c.stream().reduce(PrefixTailTrieTest::combineTails).get()); } - private Trie prepareSplitInHeadTrie(int splits, ByteComparable[] prefixes, Map data) throws TrieSpaceExhaustedException + private Trie prepareSplitInHeadTrie(int splits, Preencoded[] prefixes, Map data) throws TrieSpaceExhaustedException { InMemoryTrie[] tries = new InMemoryTrie[splits]; for (int i = 0; i < splits; ++i) - tries[i] = InMemoryTrie.shortLived(byteComparableVersion); + tries[i] = InMemoryTrie.shortLived(VERSION); int trieIndex = 0; for (int i = 0; i < prefixes.length; ++i) { - ByteComparable[] src = generateKeys(rand, COUNT_TAIL); + Preencoded[] src = generateKeys(rand, COUNT_TAIL); - NavigableMap content = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); - InMemoryTrie tail = InMemoryTrie.shortLived(byteComparableVersion); + NavigableMap content = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); + InMemoryTrie tail = InMemoryTrie.shortLived(VERSION); addToInMemoryTrie(src, content, tail, true); - Tail t = new Tail(prefixes[i].asByteComparableArray(byteComparableVersion), content); + Tail t = new Tail(prefixes[i].asByteComparableArray(VERSION), content); tail.putRecursive(ByteComparable.EMPTY, t, THROWING_UPSERT); // System.out.println(tail.dump(CONTENT_TO_STRING)); tries[trieIndex].apply(tail.prefixedBy(prefixes[i]), THROWING_UPSERT, Predicates.alwaysFalse()); - data.put(ByteComparable.preencoded(byteComparableVersion, t.prefix), t); + data.put(ByteComparable.preencoded(VERSION, t.prefix), t); trieIndex = (trieIndex + 1) % splits; } @@ -323,13 +317,13 @@ private Trie prepareSplitInHeadTrie(int splits, ByteComparable[] prefixe public void testTailMerge() throws Exception { ByteComparable prefix = generateKey(rand); - InMemoryTrie trie = InMemoryTrie.shortLived(byteComparableVersion); - NavigableMap content = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); + InMemoryTrie trie = InMemoryTrie.shortLived(VERSION); + NavigableMap content = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); for (int i = 0; i < COUNT_HEAD; ++i) { - ByteComparable[] src = generateKeys(rand, COUNT_TAIL); - InMemoryTrie tail = InMemoryTrie.shortLived(byteComparableVersion); + Preencoded[] src = generateKeys(rand, COUNT_TAIL); + InMemoryTrie tail = InMemoryTrie.shortLived(VERSION); addToInMemoryTrie(src, content, tail, true); // System.out.println(tail.dump(CONTENT_TO_STRING)); tail.putRecursive(ByteComparable.EMPTY, 1, THROWING_UPSERT); @@ -352,7 +346,7 @@ public void testTailMerge() throws Exception long count = 0; for (var en : trie.tailTries(Direction.FORWARD, Integer.class)) { - System.out.println(en.getKey().byteComparableAsString(byteComparableVersion)); + System.out.println(en.getKey().byteComparableAsString(VERSION)); Trie tt = en.getValue(); assertNotNull(tt); assertEquals(COUNT_HEAD, ((Integer) getRootContent(tail)).intValue()); @@ -376,16 +370,16 @@ public void testKeyProducer() throws Exception public void testKeyProducerMarkedRoot() throws Exception { // Check that path construction works correctly also when the root is the starting position. - testKeyProducer(new ByteComparable[] { ByteComparable.EMPTY }); + testKeyProducer(new Preencoded[] { Preencoded.EMPTY.preencode(VERSION) }); } - private void testKeyProducer(ByteComparable[] prefixes) throws TrieSpaceExhaustedException + private void testKeyProducer(Preencoded[] prefixes) throws TrieSpaceExhaustedException { - NavigableMap data = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); + NavigableMap data = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); final Trie trie = prepareSplitInHeadTrie(1, prefixes, data); // System.out.println(trie.dump(CONTENT_TO_STRING)); - InMemoryTrie dest = InMemoryTrie.shortLived(byteComparableVersion); + InMemoryTrie dest = InMemoryTrie.shortLived(VERSION); InclusionChecker checker = new InclusionChecker(); dest.apply(trie, checker, Predicates.alwaysFalse()); assertEquals("", checker.output.toString()); @@ -434,7 +428,7 @@ public Object apply(Object existing, Object update, InMemoryTrie.KeyProducer fullTrie(TrieSet s) + { + return dir -> new Cursor<>() + { + private final TrieSetCursor cursor = s.cursor(dir); + + public TrieSetCursor.RangeState content() + { + return cursor.state(); + } + + public int depth() + { + return cursor.depth(); + } + + @Override + public int incomingTransition() + { + return cursor.incomingTransition(); + } + + @Override + public int advance() + { + return cursor.advance(); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + return cursor.skipTo(skipDepth, skipTransition); + } + + @Override + public Cursor tailCursor(Direction dir) + { + throw new AssertionError(); + } + + @Override + public Direction direction() + { + return dir; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return VERSION; + } + }; + } + + static String dump(TrieSet s, Direction direction) + { + return fullTrie(s).process(direction, new TrieDumper<>(Object::toString)); + } + + static void dumpToOut(TrieSet s) + { + System.out.println("Forward:"); + System.out.println(dump(s, Direction.FORWARD)); + System.out.println("Reverse:"); + System.out.println(dump(s, Direction.REVERSE)); + } + + void check(String... boundariesAsStrings) + { + Preencoded[] boundaries = new Preencoded[boundariesAsStrings.length]; + for (int i = 0; i < boundariesAsStrings.length; ++i) + boundaries[i] = boundariesAsStrings[i] != null ? TrieUtil.comparable(boundariesAsStrings[i]) : null; + check(boundaries); + + verifySkipTo(boundariesAsStrings, TrieSet.ranges(VERSION, boundaries)); + verifyTails(boundaries, TrieSet.ranges(VERSION, boundaries)); + } + + private static TrieSet tailTrie(TrieSet set, ByteComparable prefix, Direction direction) + { + TrieSetCursor c = set.cursor(direction); + if (c.descendAlong(prefix.asComparableBytes(c.byteComparableVersion()))) + return dir -> c.tailCursor(dir); + else + return null; + } + + private static boolean startsWith(ByteComparable b, ByteComparable prefix) + { + ByteSource sb = b.asComparableBytes(VERSION); + ByteSource pb = prefix.asComparableBytes(VERSION); + int next = pb.next(); + while (next != ByteSource.END_OF_STREAM) + { + if (sb.next() != next) + return false; + next = pb.next(); + } + return true; + } + + private static void verifyTails(Preencoded[] boundaries, TrieSet set) + { + Set prefixes = new TreeSet<>(FORWARD_COMPARATOR); + for (ByteComparable b : boundaries) + { + if (b == null) + continue; + for (int i = 0; i <= ByteComparable.length(b, VERSION); ++i) + prefixes.add(ByteComparable.cut(b, i).preencode(VERSION)); + } + + for (ByteComparable prefix : prefixes) + { + List tails = null; + int prefixLength = ByteComparable.length(prefix, VERSION); + for (int i = 0; i < boundaries.length; ++i) + { + ByteComparable b = boundaries[i]; + if (b == null || !startsWith(b, prefix)) + continue; + if (tails == null) + { + tails = new ArrayList<>(); + if ((i & 1) != 0) + tails.add(null); + } + + final byte[] byteComparableArray = b.asByteComparableArray(VERSION); + tails.add(ByteComparable.preencoded(VERSION, Arrays.copyOfRange(byteComparableArray, prefixLength, byteComparableArray.length))); + } + + for (Direction dir : Direction.values()) + { + System.out.println("Tail for " + prefix.byteComparableAsString(VERSION) + " " + dir); + TrieSet tail = tailTrie(set, prefix, dir); + assertNotNull(tail); + dumpToOut(tail); + var expectations = getExpectations(tails.toArray(Preencoded[]::new)); + assertTrieEquals(fullTrie(tail), expectations); + } + } + } + + private static void verifySkipTo(String[] boundariesAsStrings, TrieSet set) + { + String arr = Arrays.toString(boundariesAsStrings); + // Verify that we get the right covering state for all positions around the boundaries. + for (int bi = 0, ei = 0; bi < boundariesAsStrings.length; bi = ei) + { + ++ei; + String s = boundariesAsStrings[bi]; + if (s == null) + continue; + while (ei < boundariesAsStrings.length && s.equals(boundariesAsStrings[ei])) + ++ei; + for (int terminator : Arrays.asList(ByteSource.LT_NEXT_COMPONENT, ByteSource.TERMINATOR, ByteSource.GT_NEXT_COMPONENT)) + for (Direction direction : Direction.values()) + { + String term = terminator == ByteSource.LT_NEXT_COMPONENT ? "<" : terminator == ByteSource.TERMINATOR ? "=" : ">"; + String dir = direction == Direction.FORWARD ? "FWD" : "REV"; + String msg = term + s + " " + dir + " in " + arr + " "; + ByteSource b = ByteSource.withTerminator(terminator, ByteSource.of(s, VERSION)); + TrieSetCursor cursor = set.cursor(direction); + // skip to nearest position in cursor + int next = b.next(); + int depth = 0; + while (next != ByteSource.END_OF_STREAM && cursor.skipTo(depth + 1, next) == depth + 1 && cursor.incomingTransition() == next) + { + next = b.next(); + ++depth; + } + // Check the resulting state. + int effectiveIndexFwd = terminator <= ByteSource.TERMINATOR ? bi : ei; + int effectiveIndexRev = terminator >= ByteSource.TERMINATOR ? ei : bi; + boolean isExact = next == ByteSource.END_OF_STREAM; + TrieSetCursor.RangeState state = isExact ? cursor.state() : (cursor.state().precedingIncluded(direction) ? TrieSetCursor.RangeState.END_START_PREFIX : TrieSetCursor.RangeState.START_END_PREFIX); + assertEquals(msg + "covering FWD", (effectiveIndexFwd & 1) != 0, state.precedingIncluded(Direction.FORWARD)); + assertEquals(msg + "covering REV", (effectiveIndexRev & 1) != 0, state.precedingIncluded(Direction.REVERSE)); + } + } + } + + void check(ByteComparable... boundaries) + { + TrieSet s = TrieSet.ranges(VERSION, boundaries); + dumpToOut(s); + var expectations = getExpectations(boundaries); + assertTrieEquals(fullTrie(s), expectations); + } + + static class PointState + { + int firstIndex = Integer.MAX_VALUE; + int lastIndex = Integer.MIN_VALUE; + boolean exact = false; + + void addIndex(int index, boolean exact) + { + firstIndex = Math.min(index, firstIndex); + lastIndex = Math.max(index, lastIndex); + this.exact |= exact; + } + + TrieSetCursor.RangeState state() + { + boolean appliesBefore = (firstIndex & 1) != 0; + boolean appliesAfter = (lastIndex & 1) == 0; + return TrieSetCursor.RangeState.values()[(appliesBefore ? 1 : 0) | (appliesAfter ? 2 : 0) | (exact ? 4 : 0)]; + } + + static PointState fullRange() + { + PointState state = new PointState(); + state.firstIndex = 1; + state.lastIndex = 2; + state.exact = false; + return state; + } + } + + static NavigableMap getExpectations(ByteComparable... boundaries) + { + var expectations = new TreeMap(FORWARD_COMPARATOR); + for (int bi = 0; bi < boundaries.length; ++bi) + { + ByteComparable b = boundaries[bi]; + if (b == null) + continue; + int len = ByteComparable.length(b, VERSION); + for (int i = 0; i <= len; ++i) + { + Preencoded v = ByteComparable.cut(b, i).preencode(VERSION); + PointState state = expectations.computeIfAbsent(v, k -> new PointState()); + state.addIndex(bi, i == len); + } + } + if (expectations.isEmpty()) + expectations.put(ByteComparable.preencoded(VERSION, new byte[0]), PointState.fullRange()); + return expectations.entrySet() + .stream() + .collect(() -> new TreeMap(FORWARD_COMPARATOR), + (m, e) -> m.put(e.getKey(), e.getValue().state()), + NavigableMap::putAll); + } + + @Test + public void testFullInterval() + { + check((String) null, null); + } + + @Test + public void testOneNull() + { + check((String) null); + } + + @Test + public void testLeftNull() + { + check(null, "afg"); + } + + @Test + public void testRightNull() + { + check("abc", null); + } + + @Test + public void testSpan() + { + check("abc", "afg"); + } + + @Test + public void testPoint() + { + check("abc", "abc"); + } + + @Test + public void testDual() + { + check("abc", "afg", "aga", "ajb"); + } + + @Test + public void testHole() + { + check(null, "abc", "afg", null); + } + + @Test + public void testRepeatLeft() + { + check("abc", "abc", "abc", null); + } + + @Test + public void testRepeatRight() + { + check(null, "abc", "abc", "abc"); + } + + @Test + public void testPointRepeat() + { + check("abc", "abc", "abc", "abc"); + } + + @Test + public void testPointInSpan() + { + check("aa", "abc", "abc", "ad"); + } + + @Test + public void testPrefixRepeatsInSpanOdd() + { + check("aaa", "abc", "abe", "aff"); + } + + @Test + public void testPrefixRepeatsInSpanEven() + { + check("abc", "abe", "aff"); + } + + @Test + public void testBothEmpty() + { + check(ByteComparable.EMPTY, ByteComparable.EMPTY); + } + + @Test + public void testLeftEmpty() + { + check(ByteComparable.EMPTY, null); + } + + @Test + public void testRightEmpty() + { + check(null, ByteComparable.EMPTY); + } + + @Test + public void testLong() + { + check("aaa", "aab", "aba", "aca", "acb", "ada", "adba", "adba", "baa", "bba", "bbb", "bbc", "bcc", "bcd"); + } + + @Test + public void testRangeStateFromProperties() + { + for (boolean applicableBefore : List.of(false, true)) + for (boolean applicableAfter : List.of(false, true)) + for (boolean applicableAt : List.of(false, true)) + { + TrieSetCursor.RangeState state = TrieSetCursor.RangeState.fromProperties(applicableBefore, applicableAfter, applicableAt); + assertEquals(applicableBefore, state.applicableBefore); + assertEquals(applicableAfter, state.applicableAfter); + assertEquals(applicableAt, state.asContent != null); + } + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java index 3c6768f77449..7d74275e22ac 100644 --- a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java @@ -19,10 +19,8 @@ package org.apache.cassandra.db.tries; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; import java.util.List; import java.util.NavigableMap; import java.util.Random; @@ -35,14 +33,18 @@ import com.googlecode.concurrenttrees.common.Iterables; import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.asString; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.assertSameContent; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.byteComparableVersion; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.generateKeys; -import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.makeInMemoryTrie; import static java.util.Arrays.asList; +import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.makeInMemoryTrie; +import static org.apache.cassandra.db.tries.TrieUtil.FORWARD_COMPARATOR; +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.apache.cassandra.db.tries.TrieUtil.asString; +import static org.apache.cassandra.db.tries.TrieUtil.assertSameContent; +import static org.apache.cassandra.db.tries.TrieUtil.generateKeys; +import static org.apache.cassandra.db.tries.TrieUtil.toBound; +import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; import static org.junit.Assert.assertEquals; public class SlicedTrieTest @@ -53,7 +55,7 @@ public static void enableVerification() CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); } - public static final ByteComparable[] BOUNDARIES = toByteComparable(new String[]{ + public static final Preencoded[] BOUNDARIES = toByteComparable(new String[]{ "test1", "test11", "test12", @@ -72,7 +74,7 @@ public static void enableVerification() "\000\000\377", "\377\377" }); - public static final ByteComparable[] KEYS = toByteComparable(new String[]{ + public static final Preencoded[] KEYS = toByteComparable(new String[]{ "test1", "test2", "test55", @@ -92,7 +94,6 @@ public static void enableVerification() "\377\377" }); - public static final Comparator BYTE_COMPARABLE_COMPARATOR = (bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, byteComparableVersion); private static final int COUNT = 15000; Random rand = new Random(); @@ -104,22 +105,22 @@ public void testIntersectRangeDirect() public void testIntersectRange(int count) { - ByteComparable[] src1 = generateKeys(rand, count); - NavigableMap content1 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, byteComparableVersion)); + Preencoded[] src1 = generateKeys(rand, count); + NavigableMap content1 = new TreeMap<>(FORWARD_COMPARATOR); InMemoryTrie trie1 = makeInMemoryTrie(src1, content1, true); checkEqualRange(content1, trie1, null, true, null, true); - checkEqualRange(content1, trie1, InMemoryTrieTestBase.generateKey(rand), true, null, true); - checkEqualRange(content1, trie1, null, true, InMemoryTrieTestBase.generateKey(rand), true); + checkEqualRange(content1, trie1, TrieUtil.generateKey(rand), true, null, true); + checkEqualRange(content1, trie1, null, true, TrieUtil.generateKey(rand), true); for (int i = 0; i < 4; ++i) { - ByteComparable l = rand.nextBoolean() ? InMemoryTrieTestBase.generateKey(rand) : src1[rand.nextInt(src1.length)]; - ByteComparable r = rand.nextBoolean() ? InMemoryTrieTestBase.generateKey(rand) : src1[rand.nextInt(src1.length)]; - int cmp = ByteComparable.compare(l, r, byteComparableVersion); + Preencoded l = rand.nextBoolean() ? TrieUtil.generateKey(rand) : src1[rand.nextInt(src1.length)]; + Preencoded r = rand.nextBoolean() ? TrieUtil.generateKey(rand) : src1[rand.nextInt(src1.length)]; + int cmp = ByteComparable.compare(l, r, VERSION); if (cmp > 0) { - ByteComparable t = l; + Preencoded t = l; l = r; r = t; // swap } @@ -132,34 +133,34 @@ public void testIntersectRange(int count) } } - private static ByteComparable[] toByteComparable(String[] keys) + private static Preencoded[] toByteComparable(String[] keys) { return Arrays.stream(keys) - .map(x -> ByteComparable.preencoded(byteComparableVersion, x.getBytes(StandardCharsets.UTF_8))) - .toArray(ByteComparable[]::new); + .map(TrieUtil::comparable) + .toArray(Preencoded[]::new); } @Test public void testSingletonSubtrie() { - Arrays.sort(BOUNDARIES, (a, b) -> ByteComparable.compare(a, b, byteComparableVersion)); + Arrays.sort(BOUNDARIES, (a, b) -> ByteComparable.compare(a, b, VERSION)); for (int li = -1; li < BOUNDARIES.length; ++li) { - ByteComparable l = li < 0 ? null : BOUNDARIES[li]; + Preencoded l = li < 0 ? null : BOUNDARIES[li]; for (int ri = Math.max(0, li); ri <= BOUNDARIES.length; ++ri) { - ByteComparable r = ri == BOUNDARIES.length ? null : BOUNDARIES[ri]; + Preencoded r = ri == BOUNDARIES.length ? null : BOUNDARIES[ri]; for (int i = li == ri ? 3 : 0; i < 4; ++i) { boolean includeLeft = (i & 1) != 0; boolean includeRight = (i & 2) != 0; - for (ByteComparable key : KEYS) + for (Preencoded key : KEYS) { - int cmp1 = l != null ? ByteComparable.compare(key, l, byteComparableVersion) : 1; - int cmp2 = r != null ? ByteComparable.compare(r, key, byteComparableVersion) : 1; - Trie ix = Trie.singleton(key, byteComparableVersion, true).subtrie(l, includeLeft, r, includeRight); + int cmp1 = l != null ? ByteComparable.compare(key, l, VERSION) : 1; + int cmp2 = r != null ? ByteComparable.compare(r, key, VERSION) : 1; + Trie ix = Trie.singleton(key, VERSION, true).subtrie(toBound(l, !includeLeft), toBound(r, includeRight)); boolean expected = true; if (cmp1 < 0 || cmp1 == 0 && !includeLeft) expected = false; @@ -172,10 +173,10 @@ public void testSingletonSubtrie() System.err.println(ix.dump()); Assert.fail(String.format("Failed on range %s%s,%s%s key %s expected %s got %s\n", includeLeft ? "[" : "(", - l != null ? l.byteComparableAsString(byteComparableVersion) : null, - r != null ? r.byteComparableAsString(byteComparableVersion) : null, + l != null ? l.byteComparableAsString(VERSION) : null, + r != null ? r.byteComparableAsString(VERSION) : null, includeRight ? "]" : ")", - key.byteComparableAsString(byteComparableVersion), + key.byteComparableAsString(VERSION), expected, actual)); } @@ -188,16 +189,16 @@ public void testSingletonSubtrie() @Test public void testMemtableSubtrie() { - Arrays.sort(BOUNDARIES, BYTE_COMPARABLE_COMPARATOR); - NavigableMap content1 = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); + Arrays.sort(BOUNDARIES, FORWARD_COMPARATOR); + NavigableMap content1 = new TreeMap<>(FORWARD_COMPARATOR); InMemoryTrie trie1 = makeInMemoryTrie(KEYS, content1, true); for (int li = -1; li < BOUNDARIES.length; ++li) { - ByteComparable l = li < 0 ? null : BOUNDARIES[li]; + Preencoded l = li < 0 ? null : BOUNDARIES[li]; for (int ri = Math.max(0, li); ri <= BOUNDARIES.length; ++ri) { - ByteComparable r = ri == BOUNDARIES.length ? null : BOUNDARIES[ri]; + Preencoded r = ri == BOUNDARIES.length ? null : BOUNDARIES[ri]; for (int i = 0; i < 4; ++i) { boolean includeLeft = (i & 1) != 0; @@ -230,8 +231,8 @@ public void testCollectionMergeSubtrie5() public void testMergeSubtrie(int mergeCount) { - Arrays.sort(BOUNDARIES, BYTE_COMPARABLE_COMPARATOR); - NavigableMap content1 = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR); + Arrays.sort(BOUNDARIES, FORWARD_COMPARATOR); + NavigableMap content1 = new TreeMap<>(FORWARD_COMPARATOR); List> tries = new ArrayList<>(); for (int i = 0; i < mergeCount; ++i) { @@ -245,10 +246,10 @@ public void testMergeSubtrie(int mergeCount) for (int li = -1; li < BOUNDARIES.length; ++li) { - ByteComparable l = li < 0 ? null : BOUNDARIES[li]; + Preencoded l = li < 0 ? null : BOUNDARIES[li]; for (int ri = Math.max(0, li); ri <= BOUNDARIES.length; ++ri) { - ByteComparable r = ri == BOUNDARIES.length ? null : BOUNDARIES[ri]; + Preencoded r = ri == BOUNDARIES.length ? null : BOUNDARIES[ri]; for (int i = 0; i < 4; ++i) { boolean includeLeft = (i & 1) != 0; @@ -261,32 +262,58 @@ public void testMergeSubtrie(int mergeCount) } } - public void checkEqualRange(NavigableMap content1, + + static NavigableMap boundedMap(NavigableMap sourceMap, Preencoded l, boolean includeLeft, Preencoded r, boolean includeRight) + { + NavigableMap imap = l == null + ? r == null + ? sourceMap + : sourceMap.headMap(r, includeRight) + : r == null + ? sourceMap.tailMap(l, includeLeft) + : sourceMap.subMap(l, includeLeft, r, includeRight); + return imap; + } + + public void checkEqualRange(NavigableMap content1, Trie t1, - ByteComparable l, + Preencoded l, boolean includeLeft, - ByteComparable r, + Preencoded r, boolean includeRight) { System.out.println(String.format("Intersection with %s%s:%s%s", includeLeft ? "[" : "(", asString(l), asString(r), includeRight ? "]" : ")")); - SortedMap imap = l == null - ? r == null - ? content1 - : content1.headMap(r, includeRight) - : r == null - ? content1.tailMap(l, includeLeft) - : content1.subMap(l, includeLeft, r, includeRight); - Trie intersection = t1.subtrie(l, includeLeft, r, includeRight); + SortedMap imap = boundedMap(content1, l, includeLeft, r, includeRight); + Trie intersection = t1.subtrie(toBound(l, !includeLeft), toBound(r, includeRight)); + try + { + assertSameContent(intersection, imap); + } + catch (AssertionError e) + { + System.out.println("\n" + t1.dump(ByteBufferUtil::bytesToHex)); + + System.out.println("\n" + intersection.dump(ByteBufferUtil::bytesToHex)); + throw e; + } + if (l == null || r == null) + return; + + // Test intersecting intersection. + intersection = t1.subtrie(toBound(l, !includeLeft), null).subtrie(null, toBound(r, includeRight)); + assertSameContent(intersection, imap); + + intersection = t1.subtrie(null, toBound(r, includeRight)).subtrie(toBound(l, !includeLeft), null); assertSameContent(intersection, imap); } /** * Extract the values of the provide trie into a list. */ - private static List toList(Trie trie) + private static List toList(Trie trie, Direction direction) { - return Iterables.toList(trie.values()); + return Iterables.toList(trie.values(direction)); } /** @@ -306,15 +333,15 @@ private static Trie singleLevelIntTrie(int childs) @Override public Cursor makeCursor(Direction direction) { - return new singleLevelCursor(direction); + return new SingleLevelCursor(direction); } - class singleLevelCursor implements Cursor + class SingleLevelCursor implements Cursor { final Direction direction; - int current = -1; + int current; - singleLevelCursor(Direction direction) + SingleLevelCursor(Direction direction) { this.direction = direction; current = direction.select(-1, childs); @@ -374,7 +401,7 @@ public Direction direction() @Override public ByteComparable.Version byteComparableVersion() { - return byteComparableVersion; + return VERSION; } @Override @@ -390,145 +417,143 @@ public Cursor tailCursor(Direction d) private static ByteComparable of(int value) { assert value >= 0 && value <= Byte.MAX_VALUE; - return ByteComparable.preencoded(byteComparableVersion, new byte[]{ (byte)value }); + return ByteComparable.preencoded(VERSION, new byte[]{ (byte)value }); } - @Test - public void testSimpleIntersectionII() + List maybeReversed(Direction direction, List list) { - Trie trie = singleLevelIntTrie(10); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(trie)); - - Trie intersection = trie.subtrie(of(3), true, of(7), true); - assertEquals(asList(3, 4, 5, 6, 7), toList(intersection)); + if (direction.isForward()) + return list; + List reversed = new ArrayList<>(list); + reversed.sort((x, y) -> x == -1 ? -1 : y == -1 ? 1 : Integer.compare(y, x)); + return reversed; } - @Test - public void testSimpleIntersectionEI() + void assertTrieEquals(List expected, Trie trie) { - Trie trie = singleLevelIntTrie(10); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(trie)); - - Trie intersection = trie.subtrie(of(3), false, of(7), true); - assertEquals(asList(4, 5, 6, 7), toList(intersection)); + assertEquals(expected, toList(trie, Direction.FORWARD)); + assertEquals(maybeReversed(Direction.REVERSE, expected), toList(trie, Direction.REVERSE)); } @Test - public void testSimpleIntersectionIE() + public void testSimpleIntersection() { Trie trie = singleLevelIntTrie(10); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(trie)); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(of(3), true, of(7), false); - assertEquals(asList(3, 4, 5, 6), toList(intersection)); + Trie intersection = trie.subtrie(of(3), of(7)); + assertTrieEquals(asList(-1, 3, 4, 5, 6, 7), intersection); } @Test - public void testSimpleIntersectionEE() + public void testSimpleLeftIntersection() { Trie trie = singleLevelIntTrie(10); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(trie)); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(of(3), false, of(7), false); - assertEquals(asList(4, 5, 6), toList(intersection)); + Trie intersection = trie.subtrie(of(3), null); + assertTrieEquals(asList(-1, 3, 4, 5, 6, 7, 8, 9), intersection); } @Test - public void testSimpleLeftIntersectionE() + public void testSimpleRightIntersection() { Trie trie = singleLevelIntTrie(10); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(trie)); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(of(3), false, null, true); - assertEquals(asList(4, 5, 6, 7, 8, 9), toList(intersection)); + Trie intersection = trie.subtrie(null, of(7)); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7), intersection); } @Test - public void testSimpleLeftIntersectionI() + public void testSimpleNoIntersection() { Trie trie = singleLevelIntTrie(10); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(trie)); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(of(3), true, null, true); - assertEquals(asList(3, 4, 5, 6, 7, 8, 9), toList(intersection)); + Trie intersection = trie.subtrie(null, null); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); } @Test - public void testSimpleRightIntersectionE() + public void testSimpleEmptyIntersectionLeft() { Trie trie = singleLevelIntTrie(10); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(trie)); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(null, true, of(7), false); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6), toList(intersection)); - } - - @Test - public void testSimpleRightIntersectionI() - { - Trie trie = singleLevelIntTrie(10); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(trie)); + Trie intersection = trie.subtrie(ByteComparable.EMPTY, null); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); - Trie intersection = trie.subtrie(null, true, of(7), true); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7), toList(intersection)); + intersection = trie.subtrie(ByteComparable.EMPTY, ByteComparable.EMPTY); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); } @Test - public void testSimpleNoIntersection() + public void testSimpleEmptyIntersectionRight() { Trie trie = singleLevelIntTrie(10); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(trie)); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(null, true, null, true); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(intersection)); + Trie intersection = trie.subtrie(null, ByteComparable.EMPTY); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); - // The two boolean flags don't have a meaning when the bound does not exist. For completeness, also test - // with them set to false. - intersection = trie.subtrie(null, false, null, false); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(intersection)); + intersection = trie.subtrie(ByteComparable.EMPTY, ByteComparable.EMPTY); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); } @Test - public void testSimpleEmptyIntersectionLeft() + public void testSubtrieOnSubtrie() { - Trie trie = singleLevelIntTrie(10); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(trie)); - - Trie intersection = trie.subtrie(ByteComparable.EMPTY, true, null, true); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(intersection)); - - intersection = trie.subtrie(ByteComparable.EMPTY, false, null, true); - assertEquals(asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(intersection)); - - intersection = trie.subtrie(ByteComparable.EMPTY, true, of(5), true); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5), toList(intersection)); - - intersection = trie.subtrie(ByteComparable.EMPTY, false, of(5), true); - assertEquals(asList(0, 1, 2, 3, 4, 5), toList(intersection)); - + Trie trie = singleLevelIntTrie(15); + + // non-overlapping + Trie intersection = trie.subtrie(of(0), of(4)).subtrie(of(5), of(8)); + assertTrieEquals(asList(-1), intersection); + // touching + intersection = trie.subtrie(of(0), of(3)).subtrie(of(3), of(8)); + assertTrieEquals(asList(-1, 3), intersection); + // overlapping 1 + intersection = trie.subtrie(of(0), of(4)).subtrie(of(2), of(8)); + assertTrieEquals(asList(-1, 2, 3, 4), intersection); + // overlapping 2 + intersection = trie.subtrie(of(0), of(4)).subtrie(of(1), of(8)); + assertTrieEquals(asList(-1, 1, 2, 3, 4), intersection); + // covered + intersection = trie.subtrie(of(0), of(4)).subtrie(of(0), of(8)); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4), intersection); + // covered 2 + intersection = trie.subtrie(of(4), of(8)).subtrie(of(0), of(8)); + assertTrieEquals(asList(-1, 4, 5, 6, 7, 8), intersection); } @Test - public void testSimpleEmptyIntersectionRight() + public void testIntersectedIntersection() { - Trie trie = singleLevelIntTrie(10); - assertEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), toList(trie)); - - Trie intersection = trie.subtrie(null, true, ByteComparable.EMPTY, true); - assertEquals(asList(-1), toList(intersection)); - - intersection = trie.subtrie(null, true, ByteComparable.EMPTY, false); - assertEquals(asList(), toList(intersection)); - - intersection = trie.subtrie(ByteComparable.EMPTY, true, ByteComparable.EMPTY, true); - assertEquals(asList(-1), toList(intersection)); - - intersection = trie.subtrie(ByteComparable.EMPTY, false, ByteComparable.EMPTY, true); - assertEquals(asList(), toList(intersection)); - - intersection = trie.subtrie(ByteComparable.EMPTY, true, ByteComparable.EMPTY, false); - assertEquals(asList(), toList(intersection)); - - // (empty, empty) is an invalid call as the "(empty" is greater than "empty)" + Trie trie = singleLevelIntTrie(15); + + // non-overlapping + Trie intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) + .intersect(TrieSet.range(VERSION, of(5), of(8))); + assertTrieEquals(asList(-1), intersection); + // touching + intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(3))) + .intersect(TrieSet.range(VERSION, of(3), of(8))); + assertTrieEquals(asList(-1, 3), intersection); + // overlapping 1 + intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) + .intersect(TrieSet.range(VERSION, of(2), of(8))); + assertTrieEquals(asList(-1, 2, 3, 4), intersection); + // overlapping 2 + intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) + .intersect(TrieSet.range(VERSION, of(1), of(8))); + assertTrieEquals(asList(-1, 1, 2, 3, 4), intersection); + // covered + intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) + .intersect(TrieSet.range(VERSION, of(0), of(8))); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4), intersection); + // covered 2 + intersection = trie.intersect(TrieSet.range(VERSION, of(4), of(8))) + .intersect(TrieSet.range(VERSION, of(0), of(8))); + assertTrieEquals(asList(-1, 4, 5, 6, 7, 8), intersection); } } diff --git a/test/unit/org/apache/cassandra/db/tries/TrieToDot.java b/test/unit/org/apache/cassandra/db/tries/TrieToDot.java new file mode 100644 index 000000000000..af42e4514ea5 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/TrieToDot.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.tries; + +import java.util.function.Function; + +import org.agrona.DirectBuffer; + +/** + * A class for dumping the structure of a trie to a graphviz/dot representation for making trie graphs. + */ +class TrieToDot extends TriePathReconstructor implements Cursor.Walker +{ + private final StringBuilder b; + private final Function contentToString; + private final Function transitionToString; + private final boolean useMultiByte; + private int prevPos; + private int currNodeTextPos; + + public TrieToDot(Function contentToString, + Function transitionToString, + boolean useMultiByte) + { + this.contentToString = contentToString; + this.transitionToString = transitionToString; + this.useMultiByte = useMultiByte; + this.b = new StringBuilder(); + b.append("digraph G {\n" + + " splines=curved"); + addNodeDefinition(nodeString(0)); + } + + @Override + public void resetPathLength(int newLength) + { + super.resetPathLength(newLength); + prevPos = newLength; + } + + private void newLineAndIndent() + { + b.append('\n'); + for (int i = 0; i < prevPos + 1; ++i) + b.append(" "); + } + + @Override + public void addPathByte(int nextByte) + { + newLineAndIndent(); + super.addPathByte(nextByte); + b.append(nodeString(prevPos)); + b.append(" -> "); + String newNode = nodeString(keyPos); + b.append(newNode); + b.append(" [label=\""); + for (int i = prevPos; i < keyPos - 1; ++i) + b.append(transitionToString.apply(keyBytes[i] & 0xFF)); + b.append(transitionToString.apply(nextByte)); + b.append("\"]"); + addNodeDefinition(newNode); + } + + private void addNodeDefinition(String newNode) + { + prevPos = keyPos; + newLineAndIndent(); + currNodeTextPos = b.length(); + b.append(String.format("%s [shape=circle label=\"\"]", newNode)); + } + + private String nodeString(int keyPos) + { + StringBuilder b = new StringBuilder(); + b.append("Node_"); + for (int i = 0; i < keyPos; ++i) + b.append(transitionToString.apply(keyBytes[i] & 0xFF)); + return b.toString(); + } + + @Override + public void addPathBytes(DirectBuffer buffer, int pos, int count) + { + if (useMultiByte) + { + super.addPathBytes(buffer, pos, count); + } + else + { + for (int i = 0; i < count; ++i) + addPathByte(buffer.getByte(pos + i) & 0xFF); + } + } + + @Override + public void content(T content) + { + b.replace(currNodeTextPos, b.length(), String.format("%s [shape=doublecircle label=\"%s\"]", nodeString(keyPos), contentToString.apply(content))); + } + + @Override + public String complete() + { + b.append("\n}\n"); + return b.toString(); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java b/test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java new file mode 100644 index 000000000000..d6e3dffdc2e2 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.junit.Test; + +public class TrieToDotTest +{ + @Test + public void testToDotContent() throws Exception + { + InMemoryTrie trie = InMemoryTrie.shortLived(TrieUtil.VERSION); + String s = "Trie node types and manipulation mechanisms. The main purpose of this is to allow for handling tries directly as" + + " they are on disk without any serialization, and to enable the creation of such files."; + s = s.toLowerCase(); + for (String word : s.split("[^a-z]+")) + trie.putRecursive(TrieUtil.directComparable(word), word, (x, y) -> y); + + System.out.println(trie.process(Direction.FORWARD, + new TrieToDot(Object::toString, + x -> Character.toString((char) ((int) x)), + true))); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/TrieToMermaid.java b/test/unit/org/apache/cassandra/db/tries/TrieToMermaid.java new file mode 100644 index 000000000000..12c1b2c2243a --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/TrieToMermaid.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.tries; + +import java.util.function.Function; + +import org.agrona.DirectBuffer; + +/** + * A class for dumping the structure of a trie to a graphviz/dot representation for making trie graphs. + */ +class TrieToMermaid extends TriePathReconstructor implements Cursor.Walker +{ + private final StringBuilder b; + private final Function contentToString; + private final Function transitionToString; + private final boolean useMultiByte; + private int prevPos; + private int currNodeTextPos; + + public TrieToMermaid(Function contentToString, + Function transitionToString, + boolean useMultiByte) + { + this.contentToString = contentToString; + this.transitionToString = transitionToString; + this.useMultiByte = useMultiByte; + this.b = new StringBuilder(); + b.append("graph TD"); + newLineAndIndent(); + addNodeDefinition(nodeString(0)); + newLineAndIndent(); + b.append("style " + nodeString(0) + " fill:darkgrey"); + } + + @Override + public void resetPathLength(int newLength) + { + super.resetPathLength(newLength); + prevPos = newLength; + } + + private void newLineAndIndent() + { + b.append('\n'); + for (int i = 0; i < prevPos + 1; ++i) + b.append(" "); + } + + @Override + public void addPathByte(int nextByte) + { + newLineAndIndent(); + super.addPathByte(nextByte); + b.append(nodeString(prevPos)); + String newNode = nodeString(keyPos); + b.append(" --\""); + for (int i = prevPos; i < keyPos - 1; ++i) + b.append(transitionToString.apply(keyBytes[i] & 0xFF)); + b.append(transitionToString.apply(nextByte)); + b.append("\"--> "); + addNodeDefinition(newNode); + } + + private void addNodeDefinition(String newNode) + { + prevPos = keyPos; + currNodeTextPos = b.length(); + b.append(String.format("%s(( ))", newNode)); + } + + private String nodeString(int keyPos) + { + StringBuilder b = new StringBuilder(); + b.append("Node_"); + for (int i = 0; i < keyPos; ++i) + b.append(transitionToString.apply(keyBytes[i] & 0xFF)); + return b.toString(); + } + + @Override + public void addPathBytes(DirectBuffer buffer, int pos, int count) + { + if (useMultiByte) + { + super.addPathBytes(buffer, pos, count); + } + else + { + for (int i = 0; i < count; ++i) + addPathByte(buffer.getByte(pos + i) & 0xFF); + } + } + + @Override + public void content(T content) + { + b.replace(currNodeTextPos, b.length(), String.format("%s(((%s)))", nodeString(keyPos), contentToString.apply(content))); + } + + @Override + public String complete() + { + b.append("\n"); + return b.toString(); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/TrieToMermaidTest.java b/test/unit/org/apache/cassandra/db/tries/TrieToMermaidTest.java new file mode 100644 index 000000000000..7989ce001570 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/TrieToMermaidTest.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.junit.Test; + +public class TrieToMermaidTest +{ + @Test + public void testToMermaidContent() throws Exception + { + InMemoryTrie trie = InMemoryTrie.shortLived(TrieUtil.VERSION); + // This was used as a basis the graphs in BTIFormat.md + String s = "a allow an and any are as node of on the this to trie types with without"; + s = s.toLowerCase(); + for (String word : s.split("[^a-z]+")) + trie.putRecursive(TrieUtil.directComparable(word), word, (x, y) -> y); + + System.out.println(trie.process(Direction.FORWARD, + new TrieToMermaid(Object::toString, + x -> Character.toString((char) ((int) x)), + false))); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/TrieUtil.java b/test/unit/org/apache/cassandra/db/tries/TrieUtil.java new file mode 100644 index 000000000000..e23a39aaa509 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/TrieUtil.java @@ -0,0 +1,533 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Objects; +import com.google.common.collect.HashMultiset; +import com.google.common.collect.Lists; +import com.google.common.collect.Multiset; +import com.google.common.collect.Sets; +import org.junit.Assert; + +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; + +public class TrieUtil +{ + // Set this to true (in combination with smaller count) to dump the tries while debugging a problem. + // Do not commit the code with VERBOSE = true. + static final boolean VERBOSE = false; + static final int COUNT = 100000; + static final ByteComparable.Version VERSION = ByteComparable.Version.OSS50; + public static final Comparator REVERSE_COMPARATOR = (bytes1, bytes2) -> ByteComparable.compare(invert(bytes1), invert(bytes2), VERSION); + public static final Comparator FORWARD_COMPARATOR = (bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION); + private static final int KEY_CHOICE = 25; + private static final int MIN_LENGTH = 10; + private static final int MAX_LENGTH = 50; + + static void assertTrieEquals(BaseTrie trie, Map map) + { + assertMapEquals(trie.entrySet(Direction.FORWARD), + map.entrySet(), + FORWARD_COMPARATOR); + assertMapEquals(trie.entrySet(Direction.REVERSE), + reorderBy(map, REVERSE_COMPARATOR).entrySet(), + REVERSE_COMPARATOR); + } + + static void assertMapEquals(Iterable> container1, + Iterable> container2, + Comparator comparator) + { + Map values1 = collectAsStrings(container1, comparator); + Map values2 = collectAsStrings(container2, comparator); + if (values1.equals(values2)) + return; + + // If the maps are not equal, we want to print out the differences in a way that is easy to read. + final Set allKeys = Sets.union(values1.keySet(), values2.keySet()); + Set keyDifference = allKeys.stream() + .filter(k -> !Objects.equal(values1.get(k), values2.get(k))) + .collect(Collectors.toCollection(() -> new TreeSet<>())); + System.err.println("All data"); + dumpDiff(values1, values2, allKeys); + System.err.println("\nDifferences"); + dumpDiff(values1, values2, keyDifference); + fail("Maps are not equal at " + keyDifference); + } + + private static void dumpDiff(Map values1, Map values2, Set set) + { + for (String key : set) + { + String v1 = values1.get(key); + if (v1 != null) + System.err.println(String.format("Trie %s:%s", key, v1)); + String v2 = values2.get(key); + if (v2 != null) + System.err.println(String.format("TreeSet %s:%s", key, v2)); + } + } + + private static Map collectAsStrings(Iterable> container, + Comparator comparator) + { + var map = new LinkedHashMap(); + Preencoded prevKey = null; + for (var e : container) + { + var key = e.getKey(); + if (prevKey != null && comparator.compare(prevKey, key) >= 0) + fail("Keys are not sorted: " + asString(prevKey) + " >= " + asString(key)); + prevKey = key; + map.put(asString(key), e.getValue().toString()); + } + return map; + } + + static ByteComparable invert(ByteComparable b) + { + return version -> invert(b.asComparableBytes(version)); + } + + static ByteSource invert(ByteSource src) + { + return () -> + { + int v = src.next(); + if (v == ByteSource.END_OF_STREAM) + return v; + return v ^ 0xFF; + }; + } + + static SpecStackEntry makeSpecStackEntry(Direction direction, Object spec, SpecStackEntry parent) + { + assert !(spec instanceof Pair); + if (spec instanceof Object[]) + { + final Object[] specArray = (Object[]) spec; + return new SpecStackEntry(specArray, null, parent, direction.select(-1, specArray.length)); + } + else + return new SpecStackEntry(new Object[0], spec, parent, direction.select(-1, 1)); + + } + + static Trie specifiedTrie(Object[] nodeDef) + { + return dir -> new CursorFromSpec<>(nodeDef, dir); + } + + static ByteComparable directComparable(String s) + { + ByteBuffer b = ByteBufferUtil.bytes(s); + return ByteComparable.preencoded(VERSION, b); + } + + @VisibleForTesting + static Preencoded comparable(String s) + { + return ((ByteComparable) (v -> ByteSource.withTerminator(ByteSource.TERMINATOR, ByteSource.of(s, v)))).preencode(VERSION); + } + + static void assertSameContent(Trie trie, SortedMap map) + { + for (Direction dir : Direction.values()) + { + assertMapEquals(trie, map, dir); + assertForEachEntryEquals(trie, map, dir); + assertValuesEqual(trie, map, dir); + assertForEachValueEquals(trie, map, dir); + } + assertUnorderedValuesEqual(trie, map); + checkGet(trie, map); + } + + static void checkGet(Trie trie, Map items) + { + for (Map.Entry en : items.entrySet()) + { + assertEquals(en.getValue(), trie.get(en.getKey())); + } + } + + private static void assertValuesEqual(Trie trie, SortedMap map, Direction direction) + { + assertIterablesEqual(trie.values(direction), maybeReversed(direction, map).values()); + } + + private static void assertUnorderedValuesEqual(Trie trie, SortedMap map) + { + Multiset unordered = HashMultiset.create(); + StringBuilder errors = new StringBuilder(); + for (ByteBuffer b : trie.valuesUnordered()) + unordered.add(b); + + for (ByteBuffer b : map.values()) + if (!unordered.remove(b)) + errors.append("\nMissing value in valuesUnordered: " + ByteBufferUtil.bytesToHex(b)); + + for (ByteBuffer b : unordered) + errors.append("\nExtra value in valuesUnordered: " + ByteBufferUtil.bytesToHex(b)); + + assertEquals("", errors.toString()); + } + + static Collection maybeReversed(Direction direction, Collection data) + { + return direction.isForward() ? data : reorderBy(data, REVERSE_COMPARATOR); + } + + static Map maybeReversed(Direction direction, Map data) + { + return direction.isForward() ? data : reorderBy(data, REVERSE_COMPARATOR); + } + + private static Map reorderBy(Map data, Comparator comparator) + { + Map newMap = new TreeMap<>(comparator); + newMap.putAll(data); + return newMap; + } + + private static void assertForEachEntryEquals(Trie trie, SortedMap map, Direction direction) + { + Iterator> it = maybeReversed(direction, map).entrySet().iterator(); + trie.forEachEntry(direction, (key, value) -> { + Assert.assertTrue("Map exhausted first, key " + asString(key), it.hasNext()); + Map.Entry entry = it.next(); + assertEquals(0, ByteComparable.compare(entry.getKey(), key, VERSION)); + assertEquals(entry.getValue(), value); + }); + Assert.assertFalse("Trie exhausted first", it.hasNext()); + } + + private static void assertForEachValueEquals(Trie trie, SortedMap map, Direction direction) + { + Iterator it = maybeReversed(direction, map).values().iterator(); + trie.forEachValue(direction, value -> { + Assert.assertTrue("Map exhausted first, value " + ByteBufferUtil.bytesToHex(value), it.hasNext()); + ByteBuffer entry = it.next(); + assertEquals("Map " + ByteBufferUtil.bytesToHex(entry) + " vs trie " + ByteBufferUtil.bytesToHex(value), entry, value); + }); + Assert.assertFalse("Trie exhausted first", it.hasNext()); + } + + static void assertMapEquals(Trie trie, SortedMap map, Direction direction) + { + assertMapEquals(trie.entryIterator(direction), maybeReversed(direction, map).entrySet().iterator()); + } + + static Collection reorderBy(Collection original, Comparator comparator) + { + List list = original.stream().collect(Collectors.toList()); + list.sort(comparator); + return list; + } + + static void assertMapEquals(Iterator> it1, + Iterator> it2) + { + List failedAt = new ArrayList<>(); + StringBuilder b = new StringBuilder(); + while (it1.hasNext() && it2.hasNext()) + { + Map.Entry en1 = it1.next(); + Map.Entry en2 = it2.next(); + b.append(String.format("TreeSet %s:%s\n", asString(en2.getKey()), ByteBufferUtil.bytesToHex(en2.getValue()))); + b.append(String.format("Trie %s:%s\n", asString(en1.getKey()), ByteBufferUtil.bytesToHex(en1.getValue()))); + if (ByteComparable.compare(en1.getKey(), en2.getKey(), VERSION) != 0 || ByteBufferUtil.compareUnsigned(en1.getValue(), en2.getValue()) != 0) + failedAt.add(en1.getKey()); + } + while (it1.hasNext()) + { + Map.Entry en1 = it1.next(); + b.append(String.format("Trie %s:%s\n", asString(en1.getKey()), ByteBufferUtil.bytesToHex(en1.getValue()))); + failedAt.add(en1.getKey()); + } + while (it2.hasNext()) + { + Map.Entry en2 = it2.next(); + b.append(String.format("TreeSet %s:%s\n", asString(en2.getKey()), ByteBufferUtil.bytesToHex(en2.getValue()))); + failedAt.add(en2.getKey()); + } + if (!failedAt.isEmpty()) + { + String message = "Failed at " + Lists.transform(failedAt, TrieUtil::asString); + System.err.println(message); + System.err.println(b); + Assert.fail(message); + } + } + + static > void assertIterablesEqual(Iterable expectedIterable, Iterable actualIterable) + { + Iterator expected = expectedIterable.iterator(); + Iterator actual = actualIterable.iterator(); + while (actual.hasNext() && expected.hasNext()) + { + Assert.assertEquals(actual.next(), expected.next()); + } + if (expected.hasNext()) + Assert.fail("Remaing values in expected, starting with " + expected.next()); + else if (actual.hasNext()) + Assert.fail("Remaing values in actual, starting with " + actual.next()); + } + + static Preencoded[] generateKeys(Random rand, int count) + { + Preencoded[] sources = new Preencoded[count]; + TreeSet added = new TreeSet<>(FORWARD_COMPARATOR); + for (int i = 0; i < count; ++i) + { + sources[i] = generateKey(rand); + if (!added.add(sources[i])) + --i; + } + + // note: not sorted! + return sources; + } + + static Preencoded generateKey(Random rand) + { + return generateKey(rand, MIN_LENGTH, MAX_LENGTH, ByteSource.TERMINATOR); + } + + static Preencoded generateKeyBound(Random rand) + { + return generateKey(rand, MIN_LENGTH, MAX_LENGTH, ByteSource.LT_NEXT_COMPONENT); + } + + static Preencoded generateKey(Random rand, int minLength, int maxLength, int terminator) + { + int len = rand.nextInt(maxLength - minLength + 1) + minLength; + byte[] bytes = new byte[len]; + int p = 0; + int length = bytes.length; + while (p < length) + { + int seed = rand.nextInt(KEY_CHOICE); + Random r2 = new Random(seed); + int m = r2.nextInt(5) + 2 + p; + if (m > length) + m = length; + while (p < m) + bytes[p++] = (byte) r2.nextInt(256); + } + return ((ByteComparable)(v -> ByteSource.withTerminator(terminator, ByteSource.of(bytes, v)))).preencode(VERSION); + } + + static Preencoded toBound(Preencoded bc) + { + return toBound(bc, false); + } + + static Preencoded toBound(Preencoded bc, boolean greater) + { + if (bc == null) + return null; + + byte[] data = bc.getPreencodedBytes().remainingBytesToArray(); + data[data.length - 1] = (byte) (greater ? ByteSource.GT_NEXT_COMPONENT : ByteSource.LT_NEXT_COMPONENT); + return ByteComparable.preencoded(bc.encodingVersion(), data); + } + + static String asString(ByteComparable bc) + { + return bc != null ? bc.byteComparableAsString(VERSION) : "null"; + } + + static class SpecStackEntry + { + Object[] children; + int curChild; + Object content; + SpecStackEntry parent; + + public SpecStackEntry(Object[] spec, Object content, SpecStackEntry parent, int curChild) + { + this.children = spec; + this.content = content; + this.parent = parent; + this.curChild = curChild; + } + } + + public static class CursorFromSpec implements Cursor + { + SpecStackEntry stack; + int depth; + int leadingTransition; + Direction direction; + + CursorFromSpec(Object[] spec, Direction direction) + { + this.direction = direction; + stack = makeSpecStackEntry(direction, spec, null); + depth = 0; + leadingTransition = -1; + } + + CursorFromSpec(SpecStackEntry stack, int depth, int leadingTransition, Direction direction) + { + this.direction = direction; + this.stack = stack; + this.depth = depth; + this.leadingTransition = leadingTransition; + } + + @Override + public int advance() + { + SpecStackEntry current = stack; + Object child; + do + { + while (current != null + && (current.children.length == 0 + || !direction.inLoop(current.curChild += direction.increase, 0, current.children.length - 1))) + { + current = current.parent; + --depth; + } + if (current == null) + { + stack = null; + leadingTransition = -1; + return depth = -1; + } + + child = current.children[current.curChild]; + } + while (child == null); + stack = makeSpecStackEntry(direction, child, current); + + return ++depth; + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + assert skipDepth <= depth + 1 : "skipTo descends more than one level"; + + while (stack != null && skipDepth <= depth) + { + --depth; + stack = stack.parent; + } + if (stack == null) + { + leadingTransition = -1; + return depth = -1; + } + + int index = skipTransition - 0x30; + assert direction.gt(index, stack.curChild) : "Backwards skipTo"; + if (direction.gt(index, direction.select(stack.children.length - 1, 0))) + { + --depth; + stack = stack.parent; + return advance(); + } + stack.curChild = index - direction.increase; + return advance(); + } + + @Override + public int depth() + { + return depth; + } + + @Override + public T content() + { + return (T) stack.content; + } + + @Override + public int incomingTransition() + { + SpecStackEntry parent = stack != null ? stack.parent : null; + return parent != null ? parent.curChild + 0x30 : leadingTransition; + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return VERSION; + } + + @Override + public Cursor tailCursor(Direction direction) + { + throw new UnsupportedOperationException("not implemented"); + } + + @Override + public String toString() + { + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append(incomingTransition()) + .append("@") + .append(depth); + if (stack.content != null) + stringBuilder.append(" content ") + .append(stack.content); + stringBuilder.append(" children "); + stringBuilder.append(IntStream.range(0, stack.children.length) + .filter(i -> stack.children[i] != null) + .mapToObj(i -> (i + 1 == stack.curChild ? "*" : "") + (char) (i + 0x30)) + .reduce("", (x, y) -> x + y)); + return stringBuilder.toString(); + } + } +} diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/trie/TrieTermsDictionaryTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/trie/TrieTermsDictionaryTest.java index ef8ee131b858..277c5a010146 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/v1/trie/TrieTermsDictionaryTest.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/trie/TrieTermsDictionaryTest.java @@ -504,7 +504,7 @@ private ByteComparable primaryKey(Function asByteCompara private void testForDifferentByteComparableEncodings(ThrowingConsumer> test) throws Exception { test.accept(s -> ByteComparable.preencoded(VERSION, ByteBufferUtil.bytes(s))); - test.accept(ByteComparable::of); + test.accept(s -> v -> ByteSource.of(s, v)); } @FunctionalInterface From 4735cdea11abf81a38bea0be6e10e6c8deb771be Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Tue, 29 Apr 2025 16:59:12 +0300 Subject: [PATCH 11/22] Extract InMemoryBaseTrie unchanged in preparation for other trie types --- .../cassandra/db/tries/InMemoryBaseTrie.java | 1546 +++++++++++++++++ .../cassandra/db/tries/InMemoryReadTrie.java | 19 +- .../cassandra/db/tries/InMemoryTrie.java | 1486 +--------------- .../cassandra/db/tries/SingletonCursor.java | 3 +- 4 files changed, 1563 insertions(+), 1491 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java new file mode 100644 index 000000000000..9028905f6918 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java @@ -0,0 +1,1546 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.tries; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicReferenceArray; +import java.util.function.Predicate; +import javax.annotation.Nonnull; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; + +import org.agrona.concurrent.UnsafeBuffer; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.ObjectSizes; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.concurrent.OpOrder; +import org.github.jamm.MemoryLayoutSpecification; + +/// In-memory trie built for fast modification and reads executing concurrently with writes from a single mutator thread. +/// +/// The main method for performing writes is [#apply(Trie,UpsertTransformer,Predicate)] which takes a trie as +/// an argument and merges it into the current trie using the methods supplied by the given [UpsertTransformer], +/// force copying anything below the points where the third argument returns true. +/// +/// +/// The predicate can be used to implement several forms of atomicity and consistency guarantees: +/// - if the predicate is `nf -> false`, neither atomicity nor sequential consistency is guaranteed - readers +/// can see any mixture of old and modified content +/// - if the predicate is `nf -> true`, full sequential consistency will be provided, i.e. if a reader sees any +/// part of a modification, it will see all of it, and all the results of all previous modifications +/// - if the predicate is `nf -> nf.isBranching()` the write will be atomic, i.e. either none or all of the +/// content of the merged trie will be visible by concurrent readers, but not sequentially consistent, i.e. there +/// may be writes that are not visible to a reader even when they precede writes that are visible. +/// - if the predicate is `nf -> (nf.content())` the write will be consistent below the identified +/// point (used e.g. by Memtable to ensure partition-level consistency) +/// +/// +/// Additionally, the class provides several simpler write methods for efficiency and convenience: +/// - [#putRecursive(ByteComparable,Object,UpsertTransformer)] inserts a single value using a recursive walk. +/// It cannot provide consistency (single-path writes are always atomic). This is more efficient as it stores the +/// walk state in the stack rather than on the heap but can cause a `StackOverflowException`. +/// - [#putSingleton(ByteComparable,Object,UpsertTransformer)] is a non-recursive version of the above, using +/// the `apply` machinery. +/// - [#putSingleton(ByteComparable,Object,UpsertTransformer,boolean)] uses the fourth argument to choose +/// between the two methods above, where some external property can be used to decide if the keys are short enough +/// to permit recursive execution. +/// +/// Because it uses 32-bit pointers in byte buffers, this trie has a fixed size limit of 2GB. +public abstract class InMemoryBaseTrie extends InMemoryReadTrie +{ + // See the trie format description in InMemoryReadTrie. + + /// Trie size limit. This is not enforced, but users must check from time to time that it is not exceeded (using + /// [#reachedAllocatedSizeThreshold()]) and start switching to a new trie if it is. + /// This must be done to avoid tries growing beyond their hard 2GB size limit (due to the 32-bit pointers). + @VisibleForTesting + static final int ALLOCATED_SIZE_THRESHOLD; + static + { + // Default threshold + 10% == 2 GB. This should give the owner enough time to react to the + // {@link #reachedAllocatedSizeThreshold()} signal and switch this trie out before it fills up. + int limitInMB = CassandraRelevantProperties.MEMTABLE_TRIE_SIZE_LIMIT.getInt(2048 * 10 / 11); + if (limitInMB < 1 || limitInMB > 2047) + throw new AssertionError(CassandraRelevantProperties.MEMTABLE_TRIE_SIZE_LIMIT.getKey() + + " must be within 1 and 2047"); + ALLOCATED_SIZE_THRESHOLD = 1024 * 1024 * limitInMB; + } + + private int allocatedPos = 0; + private int contentCount = 0; + + final BufferType bufferType; // on or off heap + final MemoryAllocationStrategy cellAllocator; + final MemoryAllocationStrategy objectAllocator; + + + // constants for space calculations + private static final long REFERENCE_ARRAY_ON_HEAP_SIZE = ObjectSizes.measureDeep(new AtomicReferenceArray<>(0)); + + enum ExpectedLifetime + { + SHORT, LONG + } + + InMemoryBaseTrie(ByteComparable.Version byteComparableVersion, BufferType bufferType, ExpectedLifetime lifetime, OpOrder opOrder) + { + super(byteComparableVersion, + new UnsafeBuffer[31 - BUF_START_SHIFT], // last one is 1G for a total of ~2G bytes + new AtomicReferenceArray[29 - CONTENTS_START_SHIFT], // takes at least 4 bytes to write pointer to one content -> 4 times smaller than buffers + NONE); + this.bufferType = bufferType; + + switch (lifetime) + { + case SHORT: + cellAllocator = new MemoryAllocationStrategy.NoReuseStrategy(new MemoryAllocationStrategy.Allocator() + { + @Override + public int allocate() throws TrieSpaceExhaustedException + { + return allocateNewCell(); + } + }); + objectAllocator = new MemoryAllocationStrategy.NoReuseStrategy(new MemoryAllocationStrategy.Allocator() + { + @Override + public int allocate() + { + return allocateNewObject(); + } + }); + break; + case LONG: + cellAllocator = new MemoryAllocationStrategy.OpOrderReuseStrategy(new MemoryAllocationStrategy.Allocator() + { + @Override + public int allocate() throws TrieSpaceExhaustedException + { + return allocateNewCell(); + } + }, opOrder); + objectAllocator = new MemoryAllocationStrategy.OpOrderReuseStrategy(new MemoryAllocationStrategy.Allocator() + { + @Override + public int allocate() + { + return allocateNewObject(); + } + }, opOrder); + break; + default: + throw new AssertionError(); + } + } + + // Buffer, content list and cell management + + private void putInt(int pos, int value) + { + getBuffer(pos).putInt(inBufferOffset(pos), value); + } + + private void putIntVolatile(int pos, int value) + { + getBuffer(pos).putIntVolatile(inBufferOffset(pos), value); + } + + private void putShort(int pos, short value) + { + getBuffer(pos).putShort(inBufferOffset(pos), value); + } + + private void putShortVolatile(int pos, short value) + { + getBuffer(pos).putShort(inBufferOffset(pos), value); + } + + private void putByte(int pos, byte value) + { + getBuffer(pos).putByte(inBufferOffset(pos), value); + } + + /// Allocate a new cell in the data buffers. This is called by the memory allocation strategy when it runs out of + /// free cells to reuse. + private int allocateNewCell() throws TrieSpaceExhaustedException + { + // Note: If this method is modified, please run InMemoryTrieTest.testOver1GSize to verify it acts correctly + // close to the 2G limit. + int v = allocatedPos; + if (inBufferOffset(v) == 0) + { + int leadBit = getBufferIdx(v, BUF_START_SHIFT, BUF_START_SIZE); + if (leadBit + BUF_START_SHIFT == 31) + throw new TrieSpaceExhaustedException(); + + ByteBuffer newBuffer = bufferType.allocate(BUF_START_SIZE << leadBit); + buffers[leadBit] = new UnsafeBuffer(newBuffer); + // Note: Since we are not moving existing data to a new buffer, we are okay with no happens-before enforcing + // writes. Any reader that sees a pointer in the new buffer may only do so after reading the volatile write + // that attached the new path. + } + + allocatedPos += CELL_SIZE; + return v; + } + + /// Allocate a cell to use for storing data. This uses the memory allocation strategy to reuse cells if any are + /// available, or to allocate new cells using [#allocateNewCell]. Because some node types rely on cells being + /// filled with 0 as initial state, any cell we get through the allocator must also be cleaned. + private int allocateCell() throws TrieSpaceExhaustedException + { + int cell = cellAllocator.allocate(); + getBuffer(cell).setMemory(inBufferOffset(cell), CELL_SIZE, (byte) 0); + return cell; + } + + private void recycleCell(int cell) + { + cellAllocator.recycle(cell & -CELL_SIZE); + } + + /// Creates a copy of a given cell and marks the original for recycling. Used when a mutation needs to force-copy + /// paths to ensure earlier states are still available for concurrent readers. + private int copyCell(int cell) throws TrieSpaceExhaustedException + { + int copy = cellAllocator.allocate(); + getBuffer(copy).putBytes(inBufferOffset(copy), getBuffer(cell), inBufferOffset(cell & -CELL_SIZE), CELL_SIZE); + recycleCell(cell); + return copy | (cell & (CELL_SIZE - 1)); + } + + /// Allocate a new position in the object array. Used by the memory allocation strategy to allocate a content spot + /// when it runs out of recycled positions. + private int allocateNewObject() + { + int index = contentCount++; + int leadBit = getBufferIdx(index, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); + AtomicReferenceArray array = contentArrays[leadBit]; + if (array == null) + { + assert inBufferOffset(index, leadBit, CONTENTS_START_SIZE) == 0 : "Error in content arrays configuration."; + contentArrays[leadBit] = new AtomicReferenceArray<>(CONTENTS_START_SIZE << leadBit); + } + return index; + } + + + /// Add a new content value. + /// + /// @return A content id that can be used to reference the content, encoded as `~index` where index is the + /// position of the value in the content array. + @Nonnull private int addContent(@Nonnull T value) throws TrieSpaceExhaustedException + { + Preconditions.checkNotNull(value, "Content value cannot be null"); + int index = objectAllocator.allocate(); + int leadBit = getBufferIdx(index, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); + int ofs = inBufferOffset(index, leadBit, CONTENTS_START_SIZE); + AtomicReferenceArray array = contentArrays[leadBit]; + // no need for a volatile set here; at this point the item is not referenced + // by any node in the trie, and a volatile set will be made to reference it. + array.setPlain(ofs, value); + return ~index; + } + + /// Change the content associated with a given content id. + /// + /// @param id content id, encoded as `~index` where index is the position in the content array + /// @param value new content value to store + private void setContent(int id, T value) + { + int leadBit = getBufferIdx(~id, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); + int ofs = inBufferOffset(~id, leadBit, CONTENTS_START_SIZE); + AtomicReferenceArray array = contentArrays[leadBit]; + array.set(ofs, value); + } + + private void releaseContent(int id) + { + objectAllocator.recycle(~id); + } + + /// Called to clean up all buffers when the trie is known to no longer be needed. + public void discardBuffers() + { + if (bufferType == BufferType.ON_HEAP) + return; // no cleaning needed + + for (UnsafeBuffer b : buffers) + { + if (b != null) + FileUtils.clean(b.byteBuffer()); + } + } + + private int copyIfOriginal(int node, int originalNode) throws TrieSpaceExhaustedException + { + return (node == originalNode) + ? copyCell(originalNode) + : node; + } + + private int getOrAllocate(int pointerAddress, int offsetWhenAllocating) throws TrieSpaceExhaustedException + { + int child = getIntVolatile(pointerAddress); + if (child != NONE) + return child; + + child = allocateCell() | offsetWhenAllocating; + // volatile writes not needed because this branch is not attached yet + putInt(pointerAddress, child); + return child; + } + + private int getCopyOrAllocate(int pointerAddress, int originalChild, int offsetWhenAllocating) throws TrieSpaceExhaustedException + { + int child = getIntVolatile(pointerAddress); + if (child == originalChild) + { + if (originalChild == NONE) + child = allocateCell() | offsetWhenAllocating; + else + child = copyCell(originalChild); + + // volatile writes not needed because this branch is not attached yet + putInt(pointerAddress, child); + } + + return child; + } + + // Write methods + + // Write visibility model: writes are not volatile, with the exception of the final write before a call returns + // the same value that was present before (e.g. content was updated in-place / existing node got a new child or had + // a child pointer updated); if the whole path including the root node changed, the root itself gets a volatile + // write. + // This final write is the point where any new cells created during the write become visible for readers for the + // first time, and such readers must pass through reading that pointer, which forces a happens-before relationship + // that extends to all values written by this thread before it. + + /// Attach a child to the given non-content node. This may be an update for an existing branch, or a new child for + /// the node. An update _is_ required (i.e. this is only called when the `newChild` pointer is not the same as the + /// existing value). + /// This method is called when the original node content must be preserved for concurrent readers (i.e. any cell to + /// be modified needs to be copied first.) + /// + /// @param node pointer to the node to update or copy + /// @param originalNode pointer to the node as it was before any updates in the current modification (i.e. apply + /// call) were started. In other words, the node that is currently reachable by readers if they + /// follow the same key, and which will become unreachable for new readers after this update + /// completes. Used to avoid copying again if already done -- if `node` is already != `originalNode` + /// (which is the case when a second or further child of a node is changed by an update), + /// then node is currently not reachable and can be safely modified or completely overwritten. + /// @param trans transition to modify/add + /// @param newChild new child pointer + /// @return pointer to the updated node + private int attachChildCopying(int node, int originalNode, int trans, int newChild) throws TrieSpaceExhaustedException + { + assert !isLeaf(node) : "attachChild cannot be used on content nodes."; + + switch (offset(node)) + { + case PREFIX_OFFSET: + assert false : "attachChild cannot be used on content nodes."; + case SPARSE_OFFSET: + // If the node is already copied (e.g. this is not the first child being modified), there's no need to copy + // it again. + return attachChildToSparseCopying(node, originalNode, trans, newChild); + case SPLIT_OFFSET: + // This call will copy the split node itself and any intermediate cells as necessary to make sure cells + // reachable from the original node are not modified. + return attachChildToSplitCopying(node, originalNode, trans, newChild); + default: + // chain nodes + return attachChildToChainCopying(node, originalNode, trans, newChild); // always copies + } + } + + /// Attach a child to the given node. This may be an update for an existing branch, or a new child for the node. + /// An update _is_ required (i.e. this is only called when the newChild pointer is not the same as the existing value). + /// + /// @param node pointer to the node to update or copy + /// @param trans transition to modify/add + /// @param newChild new child pointer + /// @return pointer to the updated node; same as node if update was in-place + private int attachChild(int node, int trans, int newChild) throws TrieSpaceExhaustedException + { + assert !isLeaf(node) : "attachChild cannot be used on content nodes."; + + switch (offset(node)) + { + case PREFIX_OFFSET: + assert false : "attachChild cannot be used on content nodes."; + case SPARSE_OFFSET: + return attachChildToSparse(node, trans, newChild); + case SPLIT_OFFSET: + return attachChildToSplit(node, trans, newChild); + default: + return attachChildToChain(node, trans, newChild); + } + } + + /// Attach a child to the given split node. This may be an update for an existing branch, or a new child for the node. + private int attachChildToSplit(int node, int trans, int newChild) throws TrieSpaceExhaustedException + { + int midPos = splitCellPointerAddress(node, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT); + int mid = getIntVolatile(midPos); + if (isNull(mid)) + { + mid = createEmptySplitNode(); + int tailPos = splitCellPointerAddress(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); + int tail = createEmptySplitNode(); + int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); + putInt(childPos, newChild); + putInt(tailPos, tail); + putIntVolatile(midPos, mid); + return node; + } + + int tailPos = splitCellPointerAddress(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); + int tail = getIntVolatile(tailPos); + if (isNull(tail)) + { + tail = createEmptySplitNode(); + int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); + putInt(childPos, newChild); + putIntVolatile(tailPos, tail); + return node; + } + + int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); + putIntVolatile(childPos, newChild); + return node; + } + + /// Non-volatile version of `attachChildToSplit`. Used when the split node is not reachable yet (during the conversion + /// from sparse). + private int attachChildToSplitNonVolatile(int node, int trans, int newChild) throws TrieSpaceExhaustedException + { + assert offset(node) == SPLIT_OFFSET : "Invalid split node in trie"; + int midPos = splitCellPointerAddress(node, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT); + int mid = getOrAllocate(midPos, SPLIT_OFFSET); + assert offset(mid) == SPLIT_OFFSET : "Invalid split node in trie"; + int tailPos = splitCellPointerAddress(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); + int tail = getOrAllocate(tailPos, SPLIT_OFFSET); + assert offset(tail) == SPLIT_OFFSET : "Invalid split node in trie"; + int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); + putInt(childPos, newChild); + return node; + } + + /// Attach a child to the given split node, copying all modified content to enable atomic visibility + /// of modification. + /// This may be an update for an existing branch, or a new child for the node. + private int attachChildToSplitCopying(int node, int originalNode, int trans, int newChild) throws TrieSpaceExhaustedException + { + if (offset(originalNode) != SPLIT_OFFSET) // includes originalNode == NONE + return attachChildToSplitNonVolatile(node, trans, newChild); + + node = copyIfOriginal(node, originalNode); + assert offset(node) == SPLIT_OFFSET : "Invalid split node in trie"; + + int midPos = splitCellPointerAddress(0, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT); + int midOriginal = originalNode != NONE ? getIntVolatile(midPos + originalNode) : NONE; + int mid = getCopyOrAllocate(node + midPos, midOriginal, SPLIT_OFFSET); + assert offset(mid) == SPLIT_OFFSET : "Invalid split node in trie"; + + int tailPos = splitCellPointerAddress(0, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); + int tailOriginal = midOriginal != NONE ? getIntVolatile(tailPos + midOriginal) : NONE; + int tail = getCopyOrAllocate(mid + tailPos, tailOriginal, SPLIT_OFFSET); + assert offset(tail) == SPLIT_OFFSET : "Invalid split node in trie"; + + int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); + putInt(childPos, newChild); + return node; + } + + /// Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. + private int attachChildToSparse(int node, int trans, int newChild) throws TrieSpaceExhaustedException + { + int index; + int smallerCount = 0; + // first check if this is an update and modify in-place if so + for (index = 0; index < SPARSE_CHILD_COUNT; ++index) + { + if (isNull(getIntVolatile(node + SPARSE_CHILDREN_OFFSET + index * 4))) + break; + final int existing = getUnsignedByte(node + SPARSE_BYTES_OFFSET + index); + if (existing == trans) + { + putIntVolatile(node + SPARSE_CHILDREN_OFFSET + index * 4, newChild); + return node; + } + else if (existing < trans) + ++smallerCount; + } + int childCount = index; + + if (childCount == SPARSE_CHILD_COUNT) + { + // Node is full. Switch to split + return upgradeSparseToSplit(node, trans, newChild); + } + + // Add a new transition. They are not kept in order, so append it at the first free position. + putByte(node + SPARSE_BYTES_OFFSET + childCount, (byte) trans); + + // Update order word. + int order = getUnsignedShortVolatile(node + SPARSE_ORDER_OFFSET); + int newOrder = insertInOrderWord(order, childCount, smallerCount); + + // Sparse nodes have two access modes: via the order word, when listing transitions, or directly to characters + // and addresses. + // To support the former, we volatile write to the order word last, and everything is correctly set up. + // The latter does not touch the order word. To support that too, we volatile write the address, as the reader + // can't determine if the position is in use based on the character byte alone (00 is also a valid transition). + // Note that this means that reader must check the transition byte AFTER the address, to ensure they get the + // correct value (see getSparseChild). + + // setting child enables reads to start seeing the new branch + putIntVolatile(node + SPARSE_CHILDREN_OFFSET + childCount * 4, newChild); + + // some readers will decide whether to check the pointer based on the order word + // write that volatile to make sure they see the new change too + putShortVolatile(node + SPARSE_ORDER_OFFSET, (short) newOrder); + return node; + } + + /// Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. + /// Resulting node is not reachable, no volatile set needed. + private int attachChildToSparseCopying(int node, int originalNode, int trans, int newChild) throws TrieSpaceExhaustedException + { + int index; + int smallerCount = 0; + // first check if this is an update and modify in-place if so + for (index = 0; index < SPARSE_CHILD_COUNT; ++index) + { + if (isNull(getIntVolatile(node + SPARSE_CHILDREN_OFFSET + index * 4))) + break; + final int existing = getUnsignedByte(node + SPARSE_BYTES_OFFSET + index); + if (existing == trans) + { + node = copyIfOriginal(node, originalNode); + putInt(node + SPARSE_CHILDREN_OFFSET + index * 4, newChild); + return node; + } + else if (existing < trans) + ++smallerCount; + } + int childCount = index; + + if (childCount == SPARSE_CHILD_COUNT) + { + // Node is full. Switch to split. + // Note that even if node != originalNode, we still have to recycle it as it was a temporary one that will + // no longer be attached. + return upgradeSparseToSplit(node, trans, newChild); + } + + node = copyIfOriginal(node, originalNode); + + // Add a new transition. They are not kept in order, so append it at the first free position. + putByte(node + SPARSE_BYTES_OFFSET + childCount, (byte) trans); + + putInt(node + SPARSE_CHILDREN_OFFSET + childCount * 4, newChild); + + // Update order word. + int order = getUnsignedShortVolatile(node + SPARSE_ORDER_OFFSET); + int newOrder = insertInOrderWord(order, childCount, smallerCount); + putShort(node + SPARSE_ORDER_OFFSET, (short) newOrder); + + return node; + } + + private int upgradeSparseToSplit(int node, int trans, int newChild) throws TrieSpaceExhaustedException + { + int split = createEmptySplitNode(); + for (int i = 0; i < SPARSE_CHILD_COUNT; ++i) + { + int t = getUnsignedByte(node + SPARSE_BYTES_OFFSET + i); + int p = getIntVolatile(node + SPARSE_CHILDREN_OFFSET + i * 4); + attachChildToSplitNonVolatile(split, t, p); + } + attachChildToSplitNonVolatile(split, trans, newChild); + recycleCell(node); + return split; + } + + /// Insert the given newIndex in the base-6 encoded order word in the correct position with respect to the ordering. + /// + /// E.g. + /// - `insertOrderWord(120, 3, 0)` must return 1203 (decimal 48*6 + 3) + /// - `insertOrderWord(120, 3, 1, ptr)` must return 1230 (decimal 8*36 + 3*6 + 0) + /// - `insertOrderWord(120, 3, 2, ptr)` must return 1320 (decimal 1*216 + 3*36 + 12) + /// - `insertOrderWord(120, 3, 3, ptr)` must return 3120 (decimal 3*216 + 48) + private static int insertInOrderWord(int order, int newIndex, int smallerCount) + { + int r = 1; + for (int i = 0; i < smallerCount; ++i) + r *= 6; + int head = order / r; + int tail = order % r; + // insert newIndex after the ones we have passed (order % r) and before the remaining (order / r) + return tail + (head * 6 + newIndex) * r; + } + + /// Attach a child to the given chain node. This may be an update for an existing branch with different target + /// address, or a second child for the node. + /// + /// This method always copies the node -- with the exception of updates that change the child of the last node in a + /// chain cell with matching transition byte (which this method is not used for, see [#attachChild]), modifications to + /// chain nodes cannot be done in place, either because we introduce a new transition byte and have to convert from + /// the single-transition chain type to sparse, or because we have to remap the child from the implicit node + 1 to + /// something else. + private int attachChildToChain(int node, int transitionByte, int newChild) throws TrieSpaceExhaustedException + { + int existingByte = getUnsignedByte(node); + if (transitionByte == existingByte) + { + // This is still a single path. Update child if possible (only if this is the last character in the chain). + if (offset(node) == LAST_POINTER_OFFSET - 1) + { + putIntVolatile(node + 1, newChild); + return node; + } + else + { + // This will only be called if new child is different from old, and the update is not on the final child + // where we can change it in place (see attachChild). We must always create something new. + // Note that since this is not the last character, we either still need this cell or we have already + // released it (a createSparseNode must have been called earlier). + // If the child is a chain, we can expand it (since it's a different value, its branch must be new and + // nothing can already reside in the rest of the cell). + return expandOrCreateChainNode(transitionByte, newChild); + } + } + + // The new transition is different, so we no longer have only one transition. Change type. + return convertChainToSparse(node, existingByte, newChild, transitionByte); + } + + /// Attach a child to the given chain node, when we are force-copying. + private int attachChildToChainCopying(int node, int originalNode, int transitionByte, int newChild) + throws TrieSpaceExhaustedException + { + int existingByte = getUnsignedByte(node); + if (transitionByte == existingByte) + { + // This is still a single path. + // Make sure we release the cell if it will no longer be referenced (if we update last reference, the whole + // path has to move as the other nodes in this chain can't be remapped). + if (offset(node) == LAST_POINTER_OFFSET - 1) + { + assert node == originalNode; // if we have already created a node, the character can't match what + // it was created with + + recycleCell(node); + } + + return expandOrCreateChainNode(transitionByte, newChild); + } + else + { + // The new transition is different, so we no longer have only one transition. Change type. + return convertChainToSparse(node, existingByte, newChild, transitionByte); + } + } + + private int convertChainToSparse(int node, int existingByte, int newChild, int transitionByte) + throws TrieSpaceExhaustedException + { + int existingChild = node + 1; + if (offset(existingChild) == LAST_POINTER_OFFSET) + { + existingChild = getIntVolatile(existingChild); + // This was a chain with just one transition which will no longer be referenced. + // The cell may contain other characters/nodes leading to this, which are also guaranteed to be + // unreferenced. + // However, these leading nodes may still be in the parent path and will be needed until the + // mutation completes. + recycleCell(node); + } + // Otherwise the sparse node we will now create references this cell, so it can't be recycled. + return createSparseNode(existingByte, existingChild, transitionByte, newChild); + } + + private boolean isExpandableChain(int newChild) + { + int newOffset = offset(newChild); + return newChild > 0 && newChild - 1 > NONE && newOffset > CHAIN_MIN_OFFSET && newOffset <= CHAIN_MAX_OFFSET; + } + + /// Create a sparse node with two children. + private int createSparseNode(int byte1, int child1, int byte2, int child2) throws TrieSpaceExhaustedException + { + assert byte1 != byte2 : "Attempted to create a sparse node with two of the same transition"; + if (byte1 > byte2) + { + // swap them so the smaller is byte1, i.e. there's always something bigger than child 0 so 0 never is + // at the end of the order + int t = byte1; byte1 = byte2; byte2 = t; + t = child1; child1 = child2; child2 = t; + } + + int node = allocateCell() + SPARSE_OFFSET; + putByte(node + SPARSE_BYTES_OFFSET + 0, (byte) byte1); + putByte(node + SPARSE_BYTES_OFFSET + 1, (byte) byte2); + putInt(node + SPARSE_CHILDREN_OFFSET + 0 * 4, child1); + putInt(node + SPARSE_CHILDREN_OFFSET + 1 * 4, child2); + putShort(node + SPARSE_ORDER_OFFSET, (short) (1 * 6 + 0)); + // Note: this does not need a volatile write as it is a new node, returning a new pointer, which needs to be + // put in an existing node or the root. That action ends in a happens-before enforcing write. + return node; + } + + /// Creates a chain node with the single provided transition (pointing to the provided child). + /// Note that to avoid creating inefficient tries with under-utilized chain nodes, this should only be called from + /// [#expandOrCreateChainNode] and other call-sites should call [#expandOrCreateChainNode]. + private int createNewChainNode(int transitionByte, int newChild) throws TrieSpaceExhaustedException + { + int newNode = allocateCell() + LAST_POINTER_OFFSET - 1; + putByte(newNode, (byte) transitionByte); + putInt(newNode + 1, newChild); + // Note: this does not need a volatile write as it is a new node, returning a new pointer, which needs to be + // put in an existing node or the root. That action ends in a happens-before enforcing write. + return newNode; + } + + /// Like [#createNewChainNode], but if the new child is already a chain node and has room, expand + /// it instead of creating a brand new node. + private int expandOrCreateChainNode(int transitionByte, int newChild) throws TrieSpaceExhaustedException + { + if (isExpandableChain(newChild)) + { + // attach as a new character in child node + int newNode = newChild - 1; + putByte(newNode, (byte) transitionByte); + return newNode; + } + + return createNewChainNode(transitionByte, newChild); + } + + private int createEmptySplitNode() throws TrieSpaceExhaustedException + { + return allocateCell() + SPLIT_OFFSET; + } + + private int createPrefixNode(int contentId, int child, boolean isSafeChain) throws TrieSpaceExhaustedException + { + assert !isNullOrLeaf(child) : "Prefix node cannot reference a childless node."; + + int offset = offset(child); + int node; + if (offset == SPLIT_OFFSET || isSafeChain && offset > (PREFIX_FLAGS_OFFSET + PREFIX_OFFSET) && offset <= CHAIN_MAX_OFFSET) + { + // We can do an embedded prefix node + // Note: for chain nodes we have a risk that the node continues beyond the current point, in which case + // creating the embedded node may overwrite information that is still needed by concurrent readers or the + // mutation process itself. + node = (child & -CELL_SIZE) | PREFIX_OFFSET; + putByte(node + PREFIX_FLAGS_OFFSET, (byte) offset); + } + else + { + // Full prefix node + node = allocateCell() + PREFIX_OFFSET; + putByte(node + PREFIX_FLAGS_OFFSET, (byte) 0xFF); + putInt(node + PREFIX_POINTER_OFFSET, child); + } + + putInt(node + PREFIX_CONTENT_OFFSET, contentId); + return node; + } + + private int updatePrefixNodeChild(int node, int child, boolean forcedCopy) throws TrieSpaceExhaustedException + { + assert offset(node) == PREFIX_OFFSET : "updatePrefix called on non-prefix node"; + assert !isNullOrLeaf(child) : "Prefix node cannot reference a childless node."; + + // We can only update in-place if we have a full prefix node + if (!isEmbeddedPrefixNode(node)) + { + if (!forcedCopy) + { + // This attaches the child branch and makes it reachable -- the write must be volatile. + putIntVolatile(node + PREFIX_POINTER_OFFSET, child); + return node; + } + else + { + node = copyCell(node); + putInt(node + PREFIX_POINTER_OFFSET, child); + return node; + } + } + else + { + // No need to recycle this cell because that is already done by the modification of the child + int contentId = getIntVolatile(node + PREFIX_CONTENT_OFFSET); + return createPrefixNode(contentId, child, true); + } + } + + private boolean isEmbeddedPrefixNode(int node) + { + return getUnsignedByte(node + PREFIX_FLAGS_OFFSET) < CELL_SIZE; + } + + /// Copy the content from an existing node, if it has any, to a newly-prepared update for its child. + /// + /// @param existingPreContentNode pointer to the existing node before skipping over content nodes, i.e. this is + /// either the same as existingPostContentNode or a pointer to a prefix or leaf node + /// whose child is `existingPostContentNode` + /// @param existingPostContentNode pointer to the existing node being updated, after any content nodes have been + /// skipped and before any modification have been applied; always a non-content node + /// @param updatedPostContentNode is the updated node, i.e. the node to which all relevant modifications have been + /// applied; if the modifications were applied in-place, this will be the same as + /// `existingPostContentNode`, otherwise a completely different pointer; always a non- + /// content node + /// @param forcedCopy whether or not we need to preserve all pre-existing data for concurrent readers + /// @return a node which has the children of updatedPostContentNode combined with the content of + /// `existingPreContentNode` + private int preserveContent(int existingPreContentNode, + int existingPostContentNode, + int updatedPostContentNode, + boolean forcedCopy) + throws TrieSpaceExhaustedException + { + if (existingPreContentNode == existingPostContentNode) + return updatedPostContentNode; // no content to preserve + + if (existingPostContentNode == updatedPostContentNode) + { + assert !forcedCopy; + return existingPreContentNode; // child didn't change, no update necessary + } + + // else we have existing prefix node, and we need to reference a new child + if (isLeaf(existingPreContentNode)) + { + return createPrefixNode(existingPreContentNode, updatedPostContentNode, true); + } + + assert offset(existingPreContentNode) == PREFIX_OFFSET : "Unexpected content in non-prefix and non-leaf node."; + return updatePrefixNodeChild(existingPreContentNode, updatedPostContentNode, forcedCopy); + } + + final ApplyState applyState = new ApplyState(); + + /// Represents the state for an [#apply] operation. Contains a stack of all nodes we descended through + /// and used to update the nodes with any new data during ascent. + /// + /// To make this as efficient and GC-friendly as possible, we use an integer array (instead of is an object stack) + /// and we reuse the same object. The latter is safe because memtable tries cannot be mutated in parallel by multiple + /// writers. + class ApplyState implements KeyProducer + { + int[] data = new int[16 * 5]; + int currentDepth = -1; + + /// Pointer to the existing node before skipping over content nodes, i.e. this is either the same as + /// existingPostContentNode or a pointer to a prefix or leaf node whose child is `existingPostContentNode`. + int existingPreContentNode() + { + return data[currentDepth * 5 + 0]; + } + void setExistingPreContentNode(int value) + { + data[currentDepth * 5 + 0] = value; + } + + /// Pointer to the existing node being updated, after any content nodes have been skipped and before any + /// modification have been applied. Always a non-content node. + int existingPostContentNode() + { + return data[currentDepth * 5 + 1]; + } + void setExistingPostContentNode(int value) + { + data[currentDepth * 5 + 1] = value; + } + + /// The updated node, i.e. the node to which the relevant modifications are being applied. This will change as + /// children are processed and attached to the node. After all children have been processed, this will contain + /// the fully updated node (i.e. the union of `existingPostContentNode` and `mutationNode`) without any content, + /// which will be processed separately and, if necessary, attached ahead of this. If the modifications were + /// applied in-place, this will be the same as `existingPostContentNode`, otherwise a completely different + /// pointer. Always a non-content node. + int updatedPostContentNode() + { + return data[currentDepth * 5 + 2]; + } + void setUpdatedPostContentNode(int value) + { + data[currentDepth * 5 + 2] = value; + } + + /// The transition we took on the way down. + int transition() + { + return data[currentDepth * 5 + 3]; + } + void setTransition(int transition) + { + data[currentDepth * 5 + 3] = transition; + } + int transitionAtDepth(int stackDepth) + { + return data[stackDepth * 5 + 3]; + } + + /// The compiled content id. Needed because we can only access a cursor's content on the way down but we can't + /// attach it until we ascend from the node. + int contentId() + { + return data[currentDepth * 5 + 4]; + } + void setContentId(int value) + { + data[currentDepth * 5 + 4] = value; + } + int contentIdAtDepth(int stackDepth) + { + return data[stackDepth * 5 + 4]; + } + + ApplyState start() + { + int existingFullNode = root; + currentDepth = 0; + + descendInto(existingFullNode); + return this; + } + + /// Returns true if the depth signals mutation cursor is exhausted. + boolean advanceTo(int depth, int transition, int forcedCopyDepth) throws TrieSpaceExhaustedException + { + while (currentDepth > Math.max(0, depth - 1)) + { + // There are no more children. Ascend to the parent state to continue walk. + attachAndMoveToParentState(forcedCopyDepth); + } + if (depth == -1) + return true; + + // We have a transition, get child to descend into + descend(transition); + return false; + } + + /// Descend to a child node. Prepares a new entry in the stack for the node. + void descend(int transition) + { + setTransition(transition); + int existingPreContentNode = getChild(existingPreContentNode(), transition); + ++currentDepth; + descendInto(existingPreContentNode); + } + + private void descendInto(int existingPreContentNode) + { + if (currentDepth * 5 >= data.length) + data = Arrays.copyOf(data, currentDepth * 5 * 2); + setExistingPreContentNode(existingPreContentNode); + + int existingContentId = NONE; + int existingPostContentNode; + if (isLeaf(existingPreContentNode)) + { + existingContentId = existingPreContentNode; + existingPostContentNode = NONE; + } + else if (offset(existingPreContentNode) == PREFIX_OFFSET) + { + existingContentId = getIntVolatile(existingPreContentNode + PREFIX_CONTENT_OFFSET); + existingPostContentNode = followContentTransition(existingPreContentNode); + } + else + existingPostContentNode = existingPreContentNode; + setExistingPostContentNode(existingPostContentNode); + setUpdatedPostContentNode(existingPostContentNode); + setContentId(existingContentId); + } + + T getContent() + { + int contentId = contentId(); + if (contentId == NONE) + return null; + return InMemoryBaseTrie.this.getContent(contentId()); + } + + void setContent(T content, boolean forcedCopy) throws TrieSpaceExhaustedException + { + int contentId = contentId(); + if (contentId == NONE) + { + if (content != null) + setContentId(InMemoryBaseTrie.this.addContent(content)); + } + else if (content == null) + { + releaseContent(contentId); + setContentId(NONE); + // At this point we are not deleting branches on the way up, just making sure we don't hold on to + // references to content. + } + else if (content == InMemoryBaseTrie.this.getContent(contentId)) + { + // no changes, nothing to do + } + else if (forcedCopy) + { + releaseContent(contentId); + setContentId(InMemoryBaseTrie.this.addContent(content)); + } + else + { + InMemoryBaseTrie.this.setContent(contentId, content); + } + } + + /// Attach a child to the current node. + private void attachChild(int transition, int child, boolean forcedCopy) throws TrieSpaceExhaustedException + { + int updatedPostContentNode = updatedPostContentNode(); + if (isNull(updatedPostContentNode)) + setUpdatedPostContentNode(expandOrCreateChainNode(transition, child)); + else if (forcedCopy) + setUpdatedPostContentNode(attachChildCopying(updatedPostContentNode, + existingPostContentNode(), + transition, + child)); + else + setUpdatedPostContentNode(InMemoryBaseTrie.this.attachChild(updatedPostContentNode, + transition, + child)); + } + + /// Apply the collected content to a node. Converts `NONE` to a leaf node, and adds or updates a prefix for all + /// others. + private int applyContent(boolean forcedCopy) throws TrieSpaceExhaustedException + { + // Note: the old content id itself is already released by setContent. Here we must release any standalone + // prefix nodes that may reference it. + int contentId = contentId(); + final int updatedPostContentNode = updatedPostContentNode(); + final int existingPreContentNode = existingPreContentNode(); + final int existingPostContentNode = existingPostContentNode(); + + // applyPrefixChange does not understand leaf nodes, handle upgrade from and to one explicitly. + if (isNull(updatedPostContentNode)) + { + if (existingPreContentNode != existingPostContentNode + && !isNullOrLeaf(existingPreContentNode) + && !isEmbeddedPrefixNode(existingPreContentNode)) + recycleCell(existingPreContentNode); + return contentId; // also fine for contentId == NONE + } + + if (isLeaf(existingPreContentNode)) + return contentId != NONE + ? createPrefixNode(contentId, updatedPostContentNode, true) + : updatedPostContentNode; + + return applyPrefixChange(updatedPostContentNode, + existingPreContentNode, + existingPostContentNode, + contentId, + forcedCopy); + } + + private int applyPrefixChange(int updatedPostPrefixNode, + int existingPrePrefixNode, + int existingPostPrefixNode, + int prefixData, + boolean forcedCopy) + throws TrieSpaceExhaustedException + { + boolean prefixWasPresent = existingPrePrefixNode != existingPostPrefixNode; + boolean prefixWasEmbedded = prefixWasPresent && isEmbeddedPrefixNode(existingPrePrefixNode); + if (prefixData == NONE) + { + if (prefixWasPresent && !prefixWasEmbedded) + recycleCell(existingPrePrefixNode); + return updatedPostPrefixNode; + } + + boolean childChanged = updatedPostPrefixNode != existingPostPrefixNode; + boolean dataChanged = !prefixWasPresent || prefixData != getIntVolatile(existingPrePrefixNode + PREFIX_CONTENT_OFFSET); + if (!childChanged && !dataChanged) + return existingPrePrefixNode; + + if (forcedCopy) + { + if (!childChanged && prefixWasEmbedded) + { + // If we directly create in this case, we will find embedding is possible and will overwrite the + // previous value. + // We could create a separate metadata node referencing the child, but in that case we'll + // use two nodes while one suffices. Instead, copy the child and embed the new metadata. + updatedPostPrefixNode = copyCell(existingPostPrefixNode); + } + else if (prefixWasPresent && !prefixWasEmbedded) + { + recycleCell(existingPrePrefixNode); + // otherwise cell is already recycled by the recycling of the child + } + return createPrefixNode(prefixData, updatedPostPrefixNode, isNull(existingPostPrefixNode)); + } + + // We can't update in-place if there was no preexisting prefix, or if the + // prefix was embedded and the target node must change. + if (!prefixWasPresent || prefixWasEmbedded && childChanged) + return createPrefixNode(prefixData, updatedPostPrefixNode, isNull(existingPostPrefixNode)); + + // Otherwise modify in place + if (childChanged) // to use volatile write but also ensure we don't corrupt embedded nodes + putIntVolatile(existingPrePrefixNode + PREFIX_POINTER_OFFSET, updatedPostPrefixNode); + if (dataChanged) + putIntVolatile(existingPrePrefixNode + PREFIX_CONTENT_OFFSET, prefixData); + return existingPrePrefixNode; + } + + /// After a node's children are processed, this is called to ascend from it. This means applying the collected + /// content to the compiled `updatedPostContentNode` and creating a mapping in the parent to it (or updating if + /// one already exists). + void attachAndMoveToParentState(int forcedCopyDepth) throws TrieSpaceExhaustedException + { + int updatedFullNode = applyContent(currentDepth >= forcedCopyDepth); + int existingFullNode = existingPreContentNode(); + --currentDepth; + + if (updatedFullNode != existingFullNode) + attachChild(transition(), updatedFullNode, currentDepth >= forcedCopyDepth); + } + + /// Ascend and update the root at the end of processing. + void attachRoot(int forcedCopyDepth) throws TrieSpaceExhaustedException + { + int updatedPreContentNode = applyContent(0 >= forcedCopyDepth); + int existingPreContentNode = existingPreContentNode(); + assert root == existingPreContentNode : "Unexpected change to root. Concurrent trie modification?"; + if (updatedPreContentNode != existingPreContentNode) + { + // Only write to root if they are different (value doesn't change, but + // we don't want to invalidate the value in other cores' caches unnecessarily). + root = updatedPreContentNode; + } + } + + public byte[] getBytes() + { + int arrSize = currentDepth; + byte[] data = new byte[arrSize]; + int pos = 0; + for (int i = 0; i < currentDepth; ++i) + { + int trans = transitionAtDepth(i); + data[pos++] = (byte) trans; + } + return data; + } + + public byte[] getBytes(Predicate shouldStop) + { + if (currentDepth == 0) + return new byte[0]; + + int arrSize = 1; + int i; + for (i = currentDepth - 1; i > 0; --i) + { + int content = contentIdAtDepth(i); + if (!isNull(content) && shouldStop.test(InMemoryBaseTrie.this.getContent(content))) + break; + ++arrSize; + } + assert i > 0 || arrSize == currentDepth; // if the loop covers the whole stack, the array must cover the full depth + + byte[] data = new byte[arrSize]; + int pos = 0; + for (; i < currentDepth; ++i) + { + int trans = transitionAtDepth(i); + data[pos++] = (byte) trans; + } + return data; + } + + public ByteComparable.Version byteComparableVersion() + { + return byteComparableVersion; + } + } + + public interface KeyProducer + { + /// Get the bytes of the path leading to this node. + byte[] getBytes(); + + /// Get the bytes of the path leading to this node from the closest ancestor whose content, after any new inserts + /// have been applied, satisfies the given predicate. + /// Note that the predicate is not called for the current position, because its content is not yet prepared when + /// the method is being called. + byte[] getBytes(Predicate shouldStop); + + ByteComparable.Version byteComparableVersion(); + } + + /// Somewhat similar to [Trie.MergeResolver], this encapsulates logic to be applied whenever new content is + /// being upserted into a [InMemoryBaseTrie]. Unlike [Trie.MergeResolver], [UpsertTransformer] will be + /// applied no matter if there's pre-existing content for that trie key/path or not. + /// + /// @param The content type for this [InMemoryBaseTrie]. + /// @param The type of the new content being applied to this [InMemoryBaseTrie]. + public interface UpsertTransformerWithKeyProducer + { + /// Called when there's content in the updating trie. + /// + /// @param existing Existing content for this key, or null if there isn't any. + /// @param update The update, always non-null. + /// @param keyState An interface that can be used to retrieve the path of the value being updated. + /// @return The combined value to use. + @Nonnull T apply(T existing, @Nonnull U update, KeyProducer keyState); + } + + /// Somewhat similar to [Trie.MergeResolver], this encapsulates logic to be applied whenever new content is + /// being upserted into a [InMemoryBaseTrie]. Unlike [Trie.MergeResolver], [UpsertTransformer] will be + /// applied no matter if there's pre-existing content for that trie key/path or not. + /// + /// A version of the above that does not use a [KeyProducer]. + /// + /// @param The content type for this [InMemoryBaseTrie]. + /// @param The type of the new content being applied to this [InMemoryBaseTrie]. + public interface UpsertTransformer extends UpsertTransformerWithKeyProducer + { + /// Called when there's content in the updating trie. + /// + /// @param existing Existing content for this key, or null if there isn't any. + /// @param update The update, always non-null. + /// @return The combined value to use. Cannot be null. + @Nonnull T apply(T existing, @Nonnull U update); + + /// Version of the above that also provides the path of a value being updated. + /// + /// @param existing Existing content for this key, or null if there isn't any. + /// @param update The update, always non-null. + /// @param keyState An interface that can be used to retrieve the path of the value being updated. + /// @return The combined value to use. Cannot be null. + @Nonnull default T apply(T existing, @Nonnull U update, @Nonnull KeyProducer keyState) + { + return apply(existing, update); + } + } + + /// Interface providing features of the mutating node during mutation done using [#apply]. + /// Effectively a subset of the [Cursor] interface which only permits operations that are safe to + /// perform before iterating the children of the mutation node to apply the branch mutation. + /// + /// This is mainly used as an argument to predicates that decide when to copy substructure when modifying tries, + /// which enables different kinds of atomicity and consistency guarantees. + /// + /// See the InMemoryTrie javadoc or InMemoryTrieThreadedTest for demonstration of the typical usages and what they + /// achieve. + public interface NodeFeatures + { + /// Whether or not the node has more than one descendant. If a checker needs mutations to be atomic, they can + /// return true when this becomes true. + boolean isBranching(); + + /// The metadata associated with the node. If readers need to see a consistent view (i.e. where older updates + /// cannot be missed if a new one is presented) below some specified point (e.g. within a partition), the checker + /// should return true when it identifies that point. + T content(); + } + + static class Mutation implements NodeFeatures + { + final UpsertTransformerWithKeyProducer transformer; + final Predicate> needsForcedCopy; + final Cursor mutationCursor; + final InMemoryBaseTrie.ApplyState state; + int forcedCopyDepth; + + Mutation(UpsertTransformerWithKeyProducer transformer, + Predicate> needsForcedCopy, + Cursor mutationCursor, + InMemoryBaseTrie.ApplyState state) + { + assert mutationCursor.depth() == 0 : "Unexpected non-fresh cursor."; + assert state.currentDepth == 0 : "Unexpected change to applyState. Concurrent trie modification?"; + this.transformer = transformer; + this.needsForcedCopy = needsForcedCopy; + this.mutationCursor = mutationCursor; + this.state = state; + } + + void apply() throws TrieSpaceExhaustedException + { + int depth = state.currentDepth; + while (true) + { + if (depth <= forcedCopyDepth) + forcedCopyDepth = needsForcedCopy.test(this) ? depth : Integer.MAX_VALUE; + + applyContent(); + + depth = mutationCursor.advance(); + if (state.advanceTo(depth, mutationCursor.incomingTransition(), forcedCopyDepth)) + break; + assert state.currentDepth == depth : "Unexpected change to applyState. Concurrent trie modification?"; + } + } + + void applyContent() throws TrieSpaceExhaustedException + { + U content = mutationCursor.content(); + if (content != null) + { + T existingContent = state.getContent(); + T combinedContent = transformer.apply(existingContent, content, state); + if (combinedContent == null) + throw new AssertionError("Transformer " + transformer + " returned null content for " + + existingContent + ", " + content); + state.setContent(combinedContent, // can be null + state.currentDepth >= forcedCopyDepth); // this is called at the start of processing + } + } + + + void complete() throws TrieSpaceExhaustedException + { + assert state.currentDepth == 0 : "Unexpected change to applyState. Concurrent trie modification?"; + state.attachRoot(forcedCopyDepth); + } + + @Override + public boolean isBranching() + { + // This is not very efficient, but we only currently use this option in tests. + // If it's needed for production use, isBranching should be implemented in the cursor interface. + Cursor dupe = mutationCursor.tailCursor(Direction.FORWARD); + int childDepth = dupe.advance(); + return childDepth > 0 && + dupe.skipTo(childDepth, dupe.incomingTransition() + 1) == childDepth; + } + + @Override + public U content() + { + return mutationCursor.content(); + } + } + + /// Map-like put method, using a fast recursive implementation through the key bytes. May run into stack overflow if + /// the trie becomes too deep. When the correct position in the trie has been reached, the value will be resolved + /// with the given function before being placed in the trie (even if there's no pre-existing content in this trie). + /// @param key the trie path/key for the given value. + /// @param value the value being put in the memtable trie. Note that it can be of type different than the element + /// type for this memtable trie. It's up to the `transformer` to return the final value that will stay in + /// the memtable trie. + /// @param transformer a function applied to the potentially pre-existing value for the given key, and the new + /// value (of a potentially different type), returning the final value that will stay in the memtable trie. Applied + /// even if there's no pre-existing value in the memtable trie. + public void putRecursive(ByteComparable key, R value, final UpsertTransformer transformer) throws TrieSpaceExhaustedException + { + try + { + int newRoot = putRecursive(root, key.asComparableBytes(byteComparableVersion), value, transformer); + if (newRoot != root) + root = newRoot; + completeMutation(); + } + catch (Throwable t) + { + abortMutation(); + throw t; + } + } + + private int putRecursive(int node, ByteSource key, R value, final UpsertTransformer transformer) throws TrieSpaceExhaustedException + { + int transition = key.next(); + if (transition == ByteSource.END_OF_STREAM) + return applyContent(node, value, transformer); + + int child = getChild(node, transition); + + int newChild = putRecursive(child, key, value, transformer); + if (newChild == child) + return node; + + int skippedContent = followContentTransition(node); + int attachedChild = !isNull(skippedContent) + ? attachChild(skippedContent, transition, newChild) // Single path, no copying required + : expandOrCreateChainNode(transition, newChild); + + return preserveContent(node, skippedContent, attachedChild, false); + } + + private int applyContent(int node, R value, UpsertTransformer transformer) throws TrieSpaceExhaustedException + { + if (isNull(node)) + return addContent(transformer.apply(null, value)); + + if (isLeaf(node)) + { + int contentId = node; + setContent(contentId, transformer.apply(getContent(contentId), value)); + return node; + } + + if (offset(node) == PREFIX_OFFSET) + { + int contentId = getIntVolatile(node + PREFIX_CONTENT_OFFSET); + setContent(contentId, transformer.apply(getContent(contentId), value)); + return node; + } + else + return createPrefixNode(addContent(transformer.apply(null, value)), node, false); + } + + void completeMutation() + { + cellAllocator.completeMutation(); + objectAllocator.completeMutation(); + } + + void abortMutation() + { + cellAllocator.abortMutation(); + objectAllocator.abortMutation(); + } + + /// Returns true if the allocation threshold has been reached. To be called by the the writing thread (ideally, just + /// after the write completes). When this returns true, the user should switch to a new trie as soon as feasible. + /// + /// The trie expects up to 10% growth above this threshold. Any growth beyond that may be done inefficiently, and + /// the trie will fail altogether when the size grows beyond 2G - 256 bytes. + public boolean reachedAllocatedSizeThreshold() + { + return allocatedPos >= ALLOCATED_SIZE_THRESHOLD; + } + + /// For tests only! Advance the allocation pointer (and allocate space) by this much to test behaviour close to + /// full. + @VisibleForTesting + int advanceAllocatedPos(int wantedPos) throws TrieSpaceExhaustedException + { + while (allocatedPos < wantedPos) + allocateCell(); + return allocatedPos; + } + + /// For tests only! Returns the current allocation position. + @VisibleForTesting + int getAllocatedPos() + { + return allocatedPos; + } + + /// Returns the off heap size of the memtable trie itself, not counting any space taken by referenced content, or + /// any space that has been allocated but is not currently in use (e.g. recycled cells or preallocated buffer). + /// The latter means we are undercounting the actual usage, but the purpose of this reporting is to decide when + /// to flush out e.g. a memtable and if we include the unused space we would almost always end up flushing out + /// immediately after allocating a large buffer and not having a chance to use it. Counting only used space makes it + /// possible to flush out before making these large allocations. + public long usedSizeOffHeap() + { + return bufferType == BufferType.ON_HEAP ? 0 : usedBufferSpace(); + } + + protected abstract long emptySizeOnHeap(); + + /// Returns the on heap size of the memtable trie itself, not counting any space taken by referenced content, or + /// any space that has been allocated but is not currently in use (e.g. recycled cells or preallocated buffer). + /// The latter means we are undercounting the actual usage, but the purpose of this reporting is to decide when + /// to flush out e.g. a memtable and if we include the unused space we would almost always end up flushing out + /// immediately after allocating a large buffer and not having a chance to use it. Counting only used space makes it + /// possible to flush out before making these large allocations. + public long usedSizeOnHeap() + { + return emptySizeOnHeap() + + usedObjectSpace() + + REFERENCE_ARRAY_ON_HEAP_SIZE * getBufferIdx(contentCount, CONTENTS_START_SHIFT, CONTENTS_START_SIZE) + + (bufferType == BufferType.ON_HEAP ? usedBufferSpace() : 0) + + REFERENCE_ARRAY_ON_HEAP_SIZE * getBufferIdx(allocatedPos, BUF_START_SHIFT, BUF_START_SIZE); + } + + private long usedBufferSpace() + { + return allocatedPos - cellAllocator.indexCountInPipeline() * CELL_SIZE; + } + + private long usedObjectSpace() + { + return (contentCount - objectAllocator.indexCountInPipeline()) * MemoryLayoutSpecification.SPEC.getReferenceSize(); + } + + /// Returns the amount of memory that has been allocated for various buffers but isn't currently in use. + /// The total on-heap space used by the trie is `usedSizeOnHeap() + unusedReservedOnHeapMemory()`. + @VisibleForTesting + public long unusedReservedOnHeapMemory() + { + int bufferOverhead = 0; + if (bufferType == BufferType.ON_HEAP) + { + int pos = this.allocatedPos; + UnsafeBuffer buffer = getBuffer(pos); + if (buffer != null) + bufferOverhead = buffer.capacity() - inBufferOffset(pos); + bufferOverhead += cellAllocator.indexCountInPipeline() * CELL_SIZE; + } + + int index = contentCount; + int leadBit = getBufferIdx(index, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); + int ofs = inBufferOffset(index, leadBit, CONTENTS_START_SIZE); + AtomicReferenceArray contentArray = contentArrays[leadBit]; + int contentOverhead = ((contentArray != null ? contentArray.length() : 0) - ofs); + contentOverhead += objectAllocator.indexCountInPipeline(); + contentOverhead *= MemoryLayoutSpecification.SPEC.getReferenceSize(); + + return bufferOverhead + contentOverhead; + } + + /// Release all recycled content references, including the ones waiting in still incomplete recycling lists. + /// This is a test method and can cause null pointer exceptions if used on a live trie. + /// + /// If similar functionality is required for non-test purposes, a version of this should be developed that only + /// releases references on barrier-complete lists. + @VisibleForTesting + public void releaseReferencesUnsafe() + { + for (int idx : objectAllocator.indexesInPipeline()) + setContent(~idx, null); + } + + /// Returns the number of values in the trie + public int valuesCount() + { + return contentCount; + } +} diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java index f9a1893f68cc..04beded9a4a5 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java @@ -28,7 +28,7 @@ /// In-memory trie built for fast modification and reads executing concurrently with writes from a single mutator thread. /// /// This class provides the read-only functionality, expanded in [InMemoryTrie] to writes. -public class InMemoryReadTrie implements Trie +public abstract class InMemoryReadTrie { /* TRIE FORMAT AND NODE TYPES @@ -288,17 +288,17 @@ T getContent(int id) Reading node content */ - boolean isNull(int node) + static boolean isNull(int node) { return node == NONE; } - boolean isLeaf(int node) + static boolean isLeaf(int node) { return node < NONE; } - boolean isNullOrLeaf(int node) + static boolean isNullOrLeaf(int node) { return node <= NONE; } @@ -1057,18 +1057,12 @@ private boolean isChainNode(int node) return !isNullOrLeaf(node) && offset(node) <= CHAIN_MAX_OFFSET; } - public InMemoryCursor makeCursor(Direction direction) - { - return new InMemoryCursor(direction); - } - /* Direct read methods */ /// Get the content mapped by the specified key. /// Fast implementation using integer node addresses. - @Override public T get(ByteComparable path) { int n = root; @@ -1095,9 +1089,10 @@ public ByteComparable.Version byteComparableVersion() return byteComparableVersion; } + abstract InMemoryCursor makeCursor(Direction direction); + /// Override of dump to provide more detailed printout that includes the type of each node in the trie. /// We do this via a wrapping cursor that returns a content string for the type of node for every node we return. - @Override public String dump(Function contentToString) { InMemoryCursor source = makeCursor(Direction.FORWARD); @@ -1205,7 +1200,7 @@ else if (isLeaf(node)) { case SPARSE_OFFSET: { - builder.append("Sparse: "); + builder.append("Sparse (Order " + Integer.toString(getUnsignedShortVolatile(node + SPARSE_ORDER_OFFSET), 6) + "): "); for (int i = 0; i < SPARSE_CHILD_COUNT; ++i) { int child = getIntVolatile(node + SPARSE_CHILDREN_OFFSET + i * 4); diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java index 8157dc6863a3..9244622e4923 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java @@ -17,26 +17,14 @@ */ package org.apache.cassandra.db.tries; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.function.Predicate; -import javax.annotation.Nonnull; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import com.google.common.base.Predicates; -import org.agrona.concurrent.UnsafeBuffer; -import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.io.compress.BufferType; -import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.utils.bytecomparable.ByteSource; -import org.apache.cassandra.utils.bytecomparable.ByteComparable; import org.apache.cassandra.utils.ObjectSizes; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; import org.apache.cassandra.utils.concurrent.OpOrder; -import org.github.jamm.MemoryLayoutSpecification; /// In-memory trie built for fast modification and reads executing concurrently with writes from a single mutator thread. /// @@ -68,102 +56,23 @@ /// to permit recursive execution. /// /// Because it uses 32-bit pointers in byte buffers, this trie has a fixed size limit of 2GB. -public class InMemoryTrie extends InMemoryReadTrie +public class InMemoryTrie extends InMemoryBaseTrie implements Trie { - // See the trie format description in InMemoryReadTrie. - - /// Trie size limit. This is not enforced, but users must check from time to time that it is not exceeded (using - /// [#reachedAllocatedSizeThreshold()]) and start switching to a new trie if it is. - /// This must be done to avoid tries growing beyond their hard 2GB size limit (due to the 32-bit pointers). - @VisibleForTesting - static final int ALLOCATED_SIZE_THRESHOLD; - static - { - // Default threshold + 10% == 2 GB. This should give the owner enough time to react to the - // {@link #reachedAllocatedSizeThreshold()} signal and switch this trie out before it fills up. - int limitInMB = CassandraRelevantProperties.MEMTABLE_TRIE_SIZE_LIMIT.getInt(2048 * 10 / 11); - if (limitInMB < 1 || limitInMB > 2047) - throw new AssertionError(CassandraRelevantProperties.MEMTABLE_TRIE_SIZE_LIMIT.getKey() + - " must be within 1 and 2047"); - ALLOCATED_SIZE_THRESHOLD = 1024 * 1024 * limitInMB; - } - - private int allocatedPos = 0; - private int contentCount = 0; - - final BufferType bufferType; // on or off heap - final MemoryAllocationStrategy cellAllocator; - final MemoryAllocationStrategy objectAllocator; - - // constants for space calculations private static final long EMPTY_SIZE_ON_HEAP; private static final long EMPTY_SIZE_OFF_HEAP; - private static final long REFERENCE_ARRAY_ON_HEAP_SIZE = ObjectSizes.measureDeep(new AtomicReferenceArray<>(0)); - static { // Measuring the empty size of long-lived tries, because these are the ones for which we want to track size. - InMemoryTrie empty = new InMemoryTrie<>(ByteComparable.Version.OSS50, BufferType.ON_HEAP, ExpectedLifetime.LONG, null); + InMemoryBaseTrie empty = new InMemoryTrie<>(ByteComparable.Version.OSS50, BufferType.ON_HEAP, ExpectedLifetime.LONG, null); EMPTY_SIZE_ON_HEAP = ObjectSizes.measureDeep(empty); empty = new InMemoryTrie<>(ByteComparable.Version.OSS50, BufferType.OFF_HEAP, ExpectedLifetime.LONG, null); EMPTY_SIZE_OFF_HEAP = ObjectSizes.measureDeep(empty); } - enum ExpectedLifetime - { - SHORT, LONG - } - InMemoryTrie(ByteComparable.Version byteComparableVersion, BufferType bufferType, ExpectedLifetime lifetime, OpOrder opOrder) { - super(byteComparableVersion, - new UnsafeBuffer[31 - BUF_START_SHIFT], // last one is 1G for a total of ~2G bytes - new AtomicReferenceArray[29 - CONTENTS_START_SHIFT], // takes at least 4 bytes to write pointer to one content -> 4 times smaller than buffers - NONE); - this.bufferType = bufferType; - - switch (lifetime) - { - case SHORT: - cellAllocator = new MemoryAllocationStrategy.NoReuseStrategy(new MemoryAllocationStrategy.Allocator() - { - @Override - public int allocate() throws TrieSpaceExhaustedException - { - return allocateNewCell(); - } - }); - objectAllocator = new MemoryAllocationStrategy.NoReuseStrategy(new MemoryAllocationStrategy.Allocator() - { - @Override - public int allocate() - { - return allocateNewObject(); - } - }); - break; - case LONG: - cellAllocator = new MemoryAllocationStrategy.OpOrderReuseStrategy(new MemoryAllocationStrategy.Allocator() - { - @Override - public int allocate() throws TrieSpaceExhaustedException - { - return allocateNewCell(); - } - }, opOrder); - objectAllocator = new MemoryAllocationStrategy.OpOrderReuseStrategy(new MemoryAllocationStrategy.Allocator() - { - @Override - public int allocate() - { - return allocateNewObject(); - } - }, opOrder); - break; - default: - throw new AssertionError(); - } + super(byteComparableVersion, bufferType, lifetime, opOrder); } public static InMemoryTrie shortLived(ByteComparable.Version byteComparableVersion) @@ -186,1207 +95,14 @@ public static InMemoryTrie longLived(ByteComparable.Version byteComparabl return new InMemoryTrie<>(byteComparableVersion, bufferType, ExpectedLifetime.LONG, opOrder); } - - // Buffer, content list and cell management - - private void putInt(int pos, int value) - { - getBuffer(pos).putInt(inBufferOffset(pos), value); - } - - private void putIntVolatile(int pos, int value) - { - getBuffer(pos).putIntVolatile(inBufferOffset(pos), value); - } - - private void putShort(int pos, short value) - { - getBuffer(pos).putShort(inBufferOffset(pos), value); - } - - private void putShortVolatile(int pos, short value) - { - getBuffer(pos).putShort(inBufferOffset(pos), value); - } - - private void putByte(int pos, byte value) - { - getBuffer(pos).putByte(inBufferOffset(pos), value); - } - - /// Allocate a new cell in the data buffers. This is called by the memory allocation strategy when it runs out of - /// free cells to reuse. - private int allocateNewCell() throws TrieSpaceExhaustedException - { - // Note: If this method is modified, please run InMemoryTrieTest.testOver1GSize to verify it acts correctly - // close to the 2G limit. - int v = allocatedPos; - if (inBufferOffset(v) == 0) - { - int leadBit = getBufferIdx(v, BUF_START_SHIFT, BUF_START_SIZE); - if (leadBit + BUF_START_SHIFT == 31) - throw new TrieSpaceExhaustedException(); - - ByteBuffer newBuffer = bufferType.allocate(BUF_START_SIZE << leadBit); - buffers[leadBit] = new UnsafeBuffer(newBuffer); - // Note: Since we are not moving existing data to a new buffer, we are okay with no happens-before enforcing - // writes. Any reader that sees a pointer in the new buffer may only do so after reading the volatile write - // that attached the new path. - } - - allocatedPos += CELL_SIZE; - return v; - } - - /// Allocate a cell to use for storing data. This uses the memory allocation strategy to reuse cells if any are - /// available, or to allocate new cells using [#allocateNewCell]. Because some node types rely on cells being - /// filled with 0 as initial state, any cell we get through the allocator must also be cleaned. - private int allocateCell() throws TrieSpaceExhaustedException - { - int cell = cellAllocator.allocate(); - getBuffer(cell).setMemory(inBufferOffset(cell), CELL_SIZE, (byte) 0); - return cell; - } - - private void recycleCell(int cell) - { - cellAllocator.recycle(cell & -CELL_SIZE); - } - - /// Creates a copy of a given cell and marks the original for recycling. Used when a mutation needs to force-copy - /// paths to ensure earlier states are still available for concurrent readers. - private int copyCell(int cell) throws TrieSpaceExhaustedException - { - int copy = cellAllocator.allocate(); - getBuffer(copy).putBytes(inBufferOffset(copy), getBuffer(cell), inBufferOffset(cell & -CELL_SIZE), CELL_SIZE); - recycleCell(cell); - return copy | (cell & (CELL_SIZE - 1)); - } - - /// Allocate a new position in the object array. Used by the memory allocation strategy to allocate a content spot - /// when it runs out of recycled positions. - private int allocateNewObject() - { - int index = contentCount++; - int leadBit = getBufferIdx(index, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); - AtomicReferenceArray array = contentArrays[leadBit]; - if (array == null) - { - assert inBufferOffset(index, leadBit, CONTENTS_START_SIZE) == 0 : "Error in content arrays configuration."; - contentArrays[leadBit] = new AtomicReferenceArray<>(CONTENTS_START_SIZE << leadBit); - } - return index; - } - - - /// Add a new content value. - /// - /// @return A content id that can be used to reference the content, encoded as `~index` where index is the - /// position of the value in the content array. - private int addContent(@Nonnull T value) throws TrieSpaceExhaustedException - { - Preconditions.checkNotNull(value, "Content value cannot be null"); - int index = objectAllocator.allocate(); - int leadBit = getBufferIdx(index, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); - int ofs = inBufferOffset(index, leadBit, CONTENTS_START_SIZE); - AtomicReferenceArray array = contentArrays[leadBit]; - // no need for a volatile set here; at this point the item is not referenced - // by any node in the trie, and a volatile set will be made to reference it. - array.setPlain(ofs, value); - return ~index; - } - - /// Change the content associated with a given content id. - /// - /// @param id content id, encoded as `~index` where index is the position in the content array - /// @param value new content value to store - private void setContent(int id, T value) - { - int leadBit = getBufferIdx(~id, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); - int ofs = inBufferOffset(~id, leadBit, CONTENTS_START_SIZE); - AtomicReferenceArray array = contentArrays[leadBit]; - array.set(ofs, value); - } - - private void releaseContent(int id) - { - objectAllocator.recycle(~id); - } - - /// Called to clean up all buffers when the trie is known to no longer be needed. - public void discardBuffers() - { - if (bufferType == BufferType.ON_HEAP) - return; // no cleaning needed - - for (UnsafeBuffer b : buffers) - { - if (b != null) - FileUtils.clean(b.byteBuffer()); - } - } - - private int copyIfOriginal(int node, int originalNode) throws TrieSpaceExhaustedException - { - return (node == originalNode) - ? copyCell(originalNode) - : node; - } - - private int getOrAllocate(int pointerAddress, int offsetWhenAllocating) throws TrieSpaceExhaustedException - { - int child = getIntVolatile(pointerAddress); - if (child != NONE) - return child; - - child = allocateCell() | offsetWhenAllocating; - // volatile writes not needed because this branch is not attached yet - putInt(pointerAddress, child); - return child; - } - - private int getCopyOrAllocate(int pointerAddress, int originalChild, int offsetWhenAllocating) throws TrieSpaceExhaustedException - { - int child = getIntVolatile(pointerAddress); - if (child == originalChild) - { - if (originalChild == NONE) - child = allocateCell() | offsetWhenAllocating; - else - child = copyCell(originalChild); - - // volatile writes not needed because this branch is not attached yet - putInt(pointerAddress, child); - } - - return child; - } - - // Write methods - - // Write visibility model: writes are not volatile, with the exception of the final write before a call returns - // the same value that was present before (e.g. content was updated in-place / existing node got a new child or had - // a child pointer updated); if the whole path including the root node changed, the root itself gets a volatile - // write. - // This final write is the point where any new cells created during the write become visible for readers for the - // first time, and such readers must pass through reading that pointer, which forces a happens-before relationship - // that extends to all values written by this thread before it. - - /// Attach a child to the given non-content node. This may be an update for an existing branch, or a new child for - /// the node. An update _is_ required (i.e. this is only called when the `newChild` pointer is not the same as the - /// existing value). - /// This method is called when the original node content must be preserved for concurrent readers (i.e. any cell to - /// be modified needs to be copied first.) - /// - /// @param node pointer to the node to update or copy - /// @param originalNode pointer to the node as it was before any updates in the current modification (i.e. apply - /// call) were started. In other words, the node that is currently reachable by readers if they - /// follow the same key, and which will become unreachable for new readers after this update - /// completes. Used to avoid copying again if already done -- if `node` is already != `originalNode` - /// (which is the case when a second or further child of a node is changed by an update), - /// then node is currently not reachable and can be safely modified or completely overwritten. - /// @param trans transition to modify/add - /// @param newChild new child pointer - /// @return pointer to the updated node - private int attachChildCopying(int node, int originalNode, int trans, int newChild) throws TrieSpaceExhaustedException - { - assert !isLeaf(node) : "attachChild cannot be used on content nodes."; - - switch (offset(node)) - { - case PREFIX_OFFSET: - assert false : "attachChild cannot be used on content nodes."; - case SPARSE_OFFSET: - // If the node is already copied (e.g. this is not the first child being modified), there's no need to copy - // it again. - return attachChildToSparseCopying(node, originalNode, trans, newChild); - case SPLIT_OFFSET: - // This call will copy the split node itself and any intermediate cells as necessary to make sure cells - // reachable from the original node are not modified. - return attachChildToSplitCopying(node, originalNode, trans, newChild); - default: - // chain nodes - return attachChildToChainCopying(node, originalNode, trans, newChild); // always copies - } - } - - /// Attach a child to the given node. This may be an update for an existing branch, or a new child for the node. - /// An update _is_ required (i.e. this is only called when the newChild pointer is not the same as the existing value). - /// - /// @param node pointer to the node to update or copy - /// @param trans transition to modify/add - /// @param newChild new child pointer - /// @return pointer to the updated node; same as node if update was in-place - private int attachChild(int node, int trans, int newChild) throws TrieSpaceExhaustedException - { - assert !isLeaf(node) : "attachChild cannot be used on content nodes."; - - switch (offset(node)) - { - case PREFIX_OFFSET: - assert false : "attachChild cannot be used on content nodes."; - case SPARSE_OFFSET: - return attachChildToSparse(node, trans, newChild); - case SPLIT_OFFSET: - return attachChildToSplit(node, trans, newChild); - default: - return attachChildToChain(node, trans, newChild); - } - } - - /// Attach a child to the given split node. This may be an update for an existing branch, or a new child for the node. - private int attachChildToSplit(int node, int trans, int newChild) throws TrieSpaceExhaustedException - { - int midPos = splitCellPointerAddress(node, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT); - int mid = getIntVolatile(midPos); - if (isNull(mid)) - { - mid = createEmptySplitNode(); - int tailPos = splitCellPointerAddress(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); - int tail = createEmptySplitNode(); - int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); - putInt(childPos, newChild); - putInt(tailPos, tail); - putIntVolatile(midPos, mid); - return node; - } - - int tailPos = splitCellPointerAddress(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); - int tail = getIntVolatile(tailPos); - if (isNull(tail)) - { - tail = createEmptySplitNode(); - int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); - putInt(childPos, newChild); - putIntVolatile(tailPos, tail); - return node; - } - - int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); - putIntVolatile(childPos, newChild); - return node; - } - - /// Non-volatile version of `attachChildToSplit`. Used when the split node is not reachable yet (during the conversion - /// from sparse). - private int attachChildToSplitNonVolatile(int node, int trans, int newChild) throws TrieSpaceExhaustedException - { - assert offset(node) == SPLIT_OFFSET : "Invalid split node in trie"; - int midPos = splitCellPointerAddress(node, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT); - int mid = getOrAllocate(midPos, SPLIT_OFFSET); - assert offset(mid) == SPLIT_OFFSET : "Invalid split node in trie"; - int tailPos = splitCellPointerAddress(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); - int tail = getOrAllocate(tailPos, SPLIT_OFFSET); - assert offset(tail) == SPLIT_OFFSET : "Invalid split node in trie"; - int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); - putInt(childPos, newChild); - return node; - } - - /// Attach a child to the given split node, copying all modified content to enable atomic visibility - /// of modification. - /// This may be an update for an existing branch, or a new child for the node. - private int attachChildToSplitCopying(int node, int originalNode, int trans, int newChild) throws TrieSpaceExhaustedException - { - if (offset(originalNode) != SPLIT_OFFSET) // includes originalNode == NONE - return attachChildToSplitNonVolatile(node, trans, newChild); - - node = copyIfOriginal(node, originalNode); - assert offset(node) == SPLIT_OFFSET : "Invalid split node in trie"; - - int midPos = splitCellPointerAddress(0, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT); - int midOriginal = originalNode != NONE ? getIntVolatile(midPos + originalNode) : NONE; - int mid = getCopyOrAllocate(node + midPos, midOriginal, SPLIT_OFFSET); - assert offset(mid) == SPLIT_OFFSET : "Invalid split node in trie"; - - int tailPos = splitCellPointerAddress(0, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); - int tailOriginal = midOriginal != NONE ? getIntVolatile(tailPos + midOriginal) : NONE; - int tail = getCopyOrAllocate(mid + tailPos, tailOriginal, SPLIT_OFFSET); - assert offset(tail) == SPLIT_OFFSET : "Invalid split node in trie"; - - int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); - putInt(childPos, newChild); - return node; - } - - /// Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. - private int attachChildToSparse(int node, int trans, int newChild) throws TrieSpaceExhaustedException - { - int index; - int smallerCount = 0; - // first check if this is an update and modify in-place if so - for (index = 0; index < SPARSE_CHILD_COUNT; ++index) - { - if (isNull(getIntVolatile(node + SPARSE_CHILDREN_OFFSET + index * 4))) - break; - final int existing = getUnsignedByte(node + SPARSE_BYTES_OFFSET + index); - if (existing == trans) - { - putIntVolatile(node + SPARSE_CHILDREN_OFFSET + index * 4, newChild); - return node; - } - else if (existing < trans) - ++smallerCount; - } - int childCount = index; - - if (childCount == SPARSE_CHILD_COUNT) - { - // Node is full. Switch to split - return upgradeSparseToSplit(node, trans, newChild); - } - - // Add a new transition. They are not kept in order, so append it at the first free position. - putByte(node + SPARSE_BYTES_OFFSET + childCount, (byte) trans); - - // Update order word. - int order = getUnsignedShortVolatile(node + SPARSE_ORDER_OFFSET); - int newOrder = insertInOrderWord(order, childCount, smallerCount); - - // Sparse nodes have two access modes: via the order word, when listing transitions, or directly to characters - // and addresses. - // To support the former, we volatile write to the order word last, and everything is correctly set up. - // The latter does not touch the order word. To support that too, we volatile write the address, as the reader - // can't determine if the position is in use based on the character byte alone (00 is also a valid transition). - // Note that this means that reader must check the transition byte AFTER the address, to ensure they get the - // correct value (see getSparseChild). - - // setting child enables reads to start seeing the new branch - putIntVolatile(node + SPARSE_CHILDREN_OFFSET + childCount * 4, newChild); - - // some readers will decide whether to check the pointer based on the order word - // write that volatile to make sure they see the new change too - putShortVolatile(node + SPARSE_ORDER_OFFSET, (short) newOrder); - return node; - } - - /// Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. - /// Resulting node is not reachable, no volatile set needed. - private int attachChildToSparseCopying(int node, int originalNode, int trans, int newChild) throws TrieSpaceExhaustedException - { - int index; - int smallerCount = 0; - // first check if this is an update and modify in-place if so - for (index = 0; index < SPARSE_CHILD_COUNT; ++index) - { - if (isNull(getIntVolatile(node + SPARSE_CHILDREN_OFFSET + index * 4))) - break; - final int existing = getUnsignedByte(node + SPARSE_BYTES_OFFSET + index); - if (existing == trans) - { - node = copyIfOriginal(node, originalNode); - putInt(node + SPARSE_CHILDREN_OFFSET + index * 4, newChild); - return node; - } - else if (existing < trans) - ++smallerCount; - } - int childCount = index; - - if (childCount == SPARSE_CHILD_COUNT) - { - // Node is full. Switch to split. - // Note that even if node != originalNode, we still have to recycle it as it was a temporary one that will - // no longer be attached. - return upgradeSparseToSplit(node, trans, newChild); - } - - node = copyIfOriginal(node, originalNode); - - // Add a new transition. They are not kept in order, so append it at the first free position. - putByte(node + SPARSE_BYTES_OFFSET + childCount, (byte) trans); - - putInt(node + SPARSE_CHILDREN_OFFSET + childCount * 4, newChild); - - // Update order word. - int order = getUnsignedShortVolatile(node + SPARSE_ORDER_OFFSET); - int newOrder = insertInOrderWord(order, childCount, smallerCount); - putShort(node + SPARSE_ORDER_OFFSET, (short) newOrder); - - return node; - } - - private int upgradeSparseToSplit(int node, int trans, int newChild) throws TrieSpaceExhaustedException - { - int split = createEmptySplitNode(); - for (int i = 0; i < SPARSE_CHILD_COUNT; ++i) - { - int t = getUnsignedByte(node + SPARSE_BYTES_OFFSET + i); - int p = getIntVolatile(node + SPARSE_CHILDREN_OFFSET + i * 4); - attachChildToSplitNonVolatile(split, t, p); - } - attachChildToSplitNonVolatile(split, trans, newChild); - recycleCell(node); - return split; - } - - /// Insert the given newIndex in the base-6 encoded order word in the correct position with respect to the ordering. - /// - /// E.g. - /// - `insertOrderWord(120, 3, 0)` must return 1203 (decimal 48*6 + 3) - /// - `insertOrderWord(120, 3, 1, ptr)` must return 1230 (decimal 8*36 + 3*6 + 0) - /// - `insertOrderWord(120, 3, 2, ptr)` must return 1320 (decimal 1*216 + 3*36 + 12) - /// - `insertOrderWord(120, 3, 3, ptr)` must return 3120 (decimal 3*216 + 48) - private static int insertInOrderWord(int order, int newIndex, int smallerCount) - { - int r = 1; - for (int i = 0; i < smallerCount; ++i) - r *= 6; - int head = order / r; - int tail = order % r; - // insert newIndex after the ones we have passed (order % r) and before the remaining (order / r) - return tail + (head * 6 + newIndex) * r; - } - - /// Attach a child to the given chain node. This may be an update for an existing branch with different target - /// address, or a second child for the node. - /// - /// This method always copies the node -- with the exception of updates that change the child of the last node in a - /// chain cell with matching transition byte (which this method is not used for, see [#attachChild]), modifications to - /// chain nodes cannot be done in place, either because we introduce a new transition byte and have to convert from - /// the single-transition chain type to sparse, or because we have to remap the child from the implicit node + 1 to - /// something else. - private int attachChildToChain(int node, int transitionByte, int newChild) throws TrieSpaceExhaustedException - { - int existingByte = getUnsignedByte(node); - if (transitionByte == existingByte) - { - // This is still a single path. Update child if possible (only if this is the last character in the chain). - if (offset(node) == LAST_POINTER_OFFSET - 1) - { - putIntVolatile(node + 1, newChild); - return node; - } - else - { - // This will only be called if new child is different from old, and the update is not on the final child - // where we can change it in place (see attachChild). We must always create something new. - // Note that since this is not the last character, we either still need this cell or we have already - // released it (a createSparseNode must have been called earlier). - // If the child is a chain, we can expand it (since it's a different value, its branch must be new and - // nothing can already reside in the rest of the cell). - return expandOrCreateChainNode(transitionByte, newChild); - } - } - - // The new transition is different, so we no longer have only one transition. Change type. - return convertChainToSparse(node, existingByte, newChild, transitionByte); - } - - /// Attach a child to the given chain node, when we are force-copying. - private int attachChildToChainCopying(int node, int originalNode, int transitionByte, int newChild) - throws TrieSpaceExhaustedException - { - int existingByte = getUnsignedByte(node); - if (transitionByte == existingByte) - { - // This is still a single path. - // Make sure we release the cell if it will no longer be referenced (if we update last reference, the whole - // path has to move as the other nodes in this chain can't be remapped). - if (offset(node) == LAST_POINTER_OFFSET - 1) - { - assert node == originalNode; // if we have already created a node, the character can't match what - // it was created with - - recycleCell(node); - } - - return expandOrCreateChainNode(transitionByte, newChild); - } - else - { - // The new transition is different, so we no longer have only one transition. Change type. - return convertChainToSparse(node, existingByte, newChild, transitionByte); - } - } - - private int convertChainToSparse(int node, int existingByte, int newChild, int transitionByte) - throws TrieSpaceExhaustedException - { - int existingChild = node + 1; - if (offset(existingChild) == LAST_POINTER_OFFSET) - { - existingChild = getIntVolatile(existingChild); - // This was a chain with just one transition which will no longer be referenced. - // The cell may contain other characters/nodes leading to this, which are also guaranteed to be - // unreferenced. - // However, these leading nodes may still be in the parent path and will be needed until the - // mutation completes. - recycleCell(node); - } - // Otherwise the sparse node we will now create references this cell, so it can't be recycled. - return createSparseNode(existingByte, existingChild, transitionByte, newChild); - } - - private boolean isExpandableChain(int newChild) - { - int newOffset = offset(newChild); - return newChild > 0 && newChild - 1 > NONE && newOffset > CHAIN_MIN_OFFSET && newOffset <= CHAIN_MAX_OFFSET; - } - - /// Create a sparse node with two children. - private int createSparseNode(int byte1, int child1, int byte2, int child2) throws TrieSpaceExhaustedException - { - assert byte1 != byte2 : "Attempted to create a sparse node with two of the same transition"; - if (byte1 > byte2) - { - // swap them so the smaller is byte1, i.e. there's always something bigger than child 0 so 0 never is - // at the end of the order - int t = byte1; byte1 = byte2; byte2 = t; - t = child1; child1 = child2; child2 = t; - } - - int node = allocateCell() + SPARSE_OFFSET; - putByte(node + SPARSE_BYTES_OFFSET + 0, (byte) byte1); - putByte(node + SPARSE_BYTES_OFFSET + 1, (byte) byte2); - putInt(node + SPARSE_CHILDREN_OFFSET + 0 * 4, child1); - putInt(node + SPARSE_CHILDREN_OFFSET + 1 * 4, child2); - putShort(node + SPARSE_ORDER_OFFSET, (short) (1 * 6 + 0)); - // Note: this does not need a volatile write as it is a new node, returning a new pointer, which needs to be - // put in an existing node or the root. That action ends in a happens-before enforcing write. - return node; - } - - /// Creates a chain node with the single provided transition (pointing to the provided child). - /// Note that to avoid creating inefficient tries with under-utilized chain nodes, this should only be called from - /// [#expandOrCreateChainNode] and other call-sites should call [#expandOrCreateChainNode]. - private int createNewChainNode(int transitionByte, int newChild) throws TrieSpaceExhaustedException - { - int newNode = allocateCell() + LAST_POINTER_OFFSET - 1; - putByte(newNode, (byte) transitionByte); - putInt(newNode + 1, newChild); - // Note: this does not need a volatile write as it is a new node, returning a new pointer, which needs to be - // put in an existing node or the root. That action ends in a happens-before enforcing write. - return newNode; - } - - /// Like [#createNewChainNode], but if the new child is already a chain node and has room, expand - /// it instead of creating a brand new node. - private int expandOrCreateChainNode(int transitionByte, int newChild) throws TrieSpaceExhaustedException - { - if (isExpandableChain(newChild)) - { - // attach as a new character in child node - int newNode = newChild - 1; - putByte(newNode, (byte) transitionByte); - return newNode; - } - - return createNewChainNode(transitionByte, newChild); - } - - private int createEmptySplitNode() throws TrieSpaceExhaustedException - { - return allocateCell() + SPLIT_OFFSET; - } - - private int createPrefixNode(int contentId, int child, boolean isSafeChain) throws TrieSpaceExhaustedException - { - assert !isNullOrLeaf(child) : "Prefix node cannot reference a childless node."; - - int offset = offset(child); - int node; - if (offset == SPLIT_OFFSET || isSafeChain && offset > (PREFIX_FLAGS_OFFSET + PREFIX_OFFSET) && offset <= CHAIN_MAX_OFFSET) - { - // We can do an embedded prefix node - // Note: for chain nodes we have a risk that the node continues beyond the current point, in which case - // creating the embedded node may overwrite information that is still needed by concurrent readers or the - // mutation process itself. - node = (child & -CELL_SIZE) | PREFIX_OFFSET; - putByte(node + PREFIX_FLAGS_OFFSET, (byte) offset); - } - else - { - // Full prefix node - node = allocateCell() + PREFIX_OFFSET; - putByte(node + PREFIX_FLAGS_OFFSET, (byte) 0xFF); - putInt(node + PREFIX_POINTER_OFFSET, child); - } - - putInt(node + PREFIX_CONTENT_OFFSET, contentId); - return node; - } - - private int updatePrefixNodeChild(int node, int child, boolean forcedCopy) throws TrieSpaceExhaustedException - { - assert offset(node) == PREFIX_OFFSET : "updatePrefix called on non-prefix node"; - assert !isNullOrLeaf(child) : "Prefix node cannot reference a childless node."; - - // We can only update in-place if we have a full prefix node - if (!isEmbeddedPrefixNode(node)) - { - if (!forcedCopy) - { - // This attaches the child branch and makes it reachable -- the write must be volatile. - putIntVolatile(node + PREFIX_POINTER_OFFSET, child); - return node; - } - else - { - node = copyCell(node); - putInt(node + PREFIX_POINTER_OFFSET, child); - return node; - } - } - else - { - // No need to recycle this cell because that is already done by the modification of the child - int contentId = getIntVolatile(node + PREFIX_CONTENT_OFFSET); - return createPrefixNode(contentId, child, true); - } - } - - private boolean isEmbeddedPrefixNode(int node) - { - return getUnsignedByte(node + PREFIX_FLAGS_OFFSET) < CELL_SIZE; - } - - /// Copy the content from an existing node, if it has any, to a newly-prepared update for its child. - /// - /// @param existingPreContentNode pointer to the existing node before skipping over content nodes, i.e. this is - /// either the same as existingPostContentNode or a pointer to a prefix or leaf node - /// whose child is `existingPostContentNode` - /// @param existingPostContentNode pointer to the existing node being updated, after any content nodes have been - /// skipped and before any modification have been applied; always a non-content node - /// @param updatedPostContentNode is the updated node, i.e. the node to which all relevant modifications have been - /// applied; if the modifications were applied in-place, this will be the same as - /// `existingPostContentNode`, otherwise a completely different pointer; always a non- - /// content node - /// @param forcedCopy whether or not we need to preserve all pre-existing data for concurrent readers - /// @return a node which has the children of updatedPostContentNode combined with the content of - /// `existingPreContentNode` - private int preserveContent(int existingPreContentNode, - int existingPostContentNode, - int updatedPostContentNode, - boolean forcedCopy) - throws TrieSpaceExhaustedException - { - if (existingPreContentNode == existingPostContentNode) - return updatedPostContentNode; // no content to preserve - - if (existingPostContentNode == updatedPostContentNode) - { - assert !forcedCopy; - return existingPreContentNode; // child didn't change, no update necessary - } - - // else we have existing prefix node, and we need to reference a new child - if (isLeaf(existingPreContentNode)) - { - return createPrefixNode(existingPreContentNode, updatedPostContentNode, true); - } - - assert offset(existingPreContentNode) == PREFIX_OFFSET : "Unexpected content in non-prefix and non-leaf node."; - return updatePrefixNodeChild(existingPreContentNode, updatedPostContentNode, forcedCopy); - } - - private final ApplyState applyState = new ApplyState(); - - /// Represents the state for an [#apply] operation. Contains a stack of all nodes we descended through - /// and used to update the nodes with any new data during ascent. - /// - /// To make this as efficient and GC-friendly as possible, we use an integer array (instead of is an object stack) - /// and we reuse the same object. The latter is safe because memtable tries cannot be mutated in parallel by multiple - /// writers. - private class ApplyState implements KeyProducer - { - int[] data = new int[16 * 5]; - int currentDepth = -1; - - /// Pointer to the existing node before skipping over content nodes, i.e. this is either the same as - /// existingPostContentNode or a pointer to a prefix or leaf node whose child is `existingPostContentNode`. - int existingPreContentNode() - { - return data[currentDepth * 5 + 0]; - } - void setExistingPreContentNode(int value) - { - data[currentDepth * 5 + 0] = value; - } - - /// Pointer to the existing node being updated, after any content nodes have been skipped and before any - /// modification have been applied. Always a non-content node. - int existingPostContentNode() - { - return data[currentDepth * 5 + 1]; - } - void setExistingPostContentNode(int value) - { - data[currentDepth * 5 + 1] = value; - } - - /// The updated node, i.e. the node to which the relevant modifications are being applied. This will change as - /// children are processed and attached to the node. After all children have been processed, this will contain - /// the fully updated node (i.e. the union of `existingPostContentNode` and `mutationNode`) without any content, - /// which will be processed separately and, if necessary, attached ahead of this. If the modifications were - /// applied in-place, this will be the same as `existingPostContentNode`, otherwise a completely different - /// pointer. Always a non-content node. - int updatedPostContentNode() - { - return data[currentDepth * 5 + 2]; - } - void setUpdatedPostContentNode(int value) - { - data[currentDepth * 5 + 2] = value; - } - - /// The transition we took on the way down. - int transition() - { - return data[currentDepth * 5 + 3]; - } - void setTransition(int transition) - { - data[currentDepth * 5 + 3] = transition; - } - int transitionAtDepth(int stackDepth) - { - return data[stackDepth * 5 + 3]; - } - - /// The compiled content id. Needed because we can only access a cursor's content on the way down but we can't - /// attach it until we ascend from the node. - int contentId() - { - return data[currentDepth * 5 + 4]; - } - void setContentId(int value) - { - data[currentDepth * 5 + 4] = value; - } - int contentIdAtDepth(int stackDepth) - { - return data[stackDepth * 5 + 4]; - } - - ApplyState start() - { - int existingFullNode = root; - currentDepth = 0; - - descendInto(existingFullNode); - return this; - } - - /// Returns true if the depth signals mutation cursor is exhausted. - boolean advanceTo(int depth, int transition, int forcedCopyDepth) throws TrieSpaceExhaustedException - { - while (currentDepth > Math.max(0, depth - 1)) - { - // There are no more children. Ascend to the parent state to continue walk. - attachAndMoveToParentState(forcedCopyDepth); - } - if (depth == -1) - return true; - - // We have a transition, get child to descend into - descend(transition); - return false; - } - - /// Descend to a child node. Prepares a new entry in the stack for the node. - void descend(int transition) - { - setTransition(transition); - int existingPreContentNode = getChild(existingPreContentNode(), transition); - ++currentDepth; - descendInto(existingPreContentNode); - } - - private void descendInto(int existingPreContentNode) - { - if (currentDepth * 5 >= data.length) - data = Arrays.copyOf(data, currentDepth * 5 * 2); - setExistingPreContentNode(existingPreContentNode); - - int existingContentId = NONE; - int existingPostContentNode; - if (isLeaf(existingPreContentNode)) - { - existingContentId = existingPreContentNode; - existingPostContentNode = NONE; - } - else if (offset(existingPreContentNode) == PREFIX_OFFSET) - { - existingContentId = getIntVolatile(existingPreContentNode + PREFIX_CONTENT_OFFSET); - existingPostContentNode = followContentTransition(existingPreContentNode); - } - else - existingPostContentNode = existingPreContentNode; - setExistingPostContentNode(existingPostContentNode); - setUpdatedPostContentNode(existingPostContentNode); - setContentId(existingContentId); - } - - T getContent() - { - int contentId = contentId(); - if (contentId == NONE) - return null; - return InMemoryTrie.this.getContent(contentId()); - } - - void setContent(T content, boolean forcedCopy) throws TrieSpaceExhaustedException - { - int contentId = contentId(); - if (contentId == NONE) - { - if (content != null) - setContentId(InMemoryTrie.this.addContent(content)); - } - else if (content == null) - { - releaseContent(contentId); - setContentId(NONE); - // At this point we are not deleting branches on the way up, just making sure we don't hold on to - // references to content. - } - else if (content == InMemoryTrie.this.getContent(contentId)) - { - // no changes, nothing to do - } - else if (forcedCopy) - { - releaseContent(contentId); - setContentId(InMemoryTrie.this.addContent(content)); - } - else - { - InMemoryTrie.this.setContent(contentId, content); - } - } - - /// Attach a child to the current node. - private void attachChild(int transition, int child, boolean forcedCopy) throws TrieSpaceExhaustedException - { - int updatedPostContentNode = updatedPostContentNode(); - if (isNull(updatedPostContentNode)) - setUpdatedPostContentNode(expandOrCreateChainNode(transition, child)); - else if (forcedCopy) - setUpdatedPostContentNode(attachChildCopying(updatedPostContentNode, - existingPostContentNode(), - transition, - child)); - else - setUpdatedPostContentNode(InMemoryTrie.this.attachChild(updatedPostContentNode, - transition, - child)); - } - - /// Apply the collected content to a node. Converts `NONE` to a leaf node, and adds or updates a prefix for all - /// others. - private int applyContent(boolean forcedCopy) throws TrieSpaceExhaustedException - { - // Note: the old content id itself is already released by setContent. Here we must release any standalone - // prefix nodes that may reference it. - int contentId = contentId(); - final int updatedPostContentNode = updatedPostContentNode(); - final int existingPreContentNode = existingPreContentNode(); - final int existingPostContentNode = existingPostContentNode(); - - // applyPrefixChange does not understand leaf nodes, handle upgrade from and to one explicitly. - if (isNull(updatedPostContentNode)) - { - if (existingPreContentNode != existingPostContentNode - && !isNullOrLeaf(existingPreContentNode) - && !isEmbeddedPrefixNode(existingPreContentNode)) - recycleCell(existingPreContentNode); - return contentId; // also fine for contentId == NONE - } - - if (isLeaf(existingPreContentNode)) - return contentId != NONE - ? createPrefixNode(contentId, updatedPostContentNode, true) - : updatedPostContentNode; - - return applyPrefixChange(updatedPostContentNode, - existingPreContentNode, - existingPostContentNode, - contentId, - forcedCopy); - } - - private int applyPrefixChange(int updatedPostPrefixNode, - int existingPrePrefixNode, - int existingPostPrefixNode, - int prefixData, - boolean forcedCopy) - throws TrieSpaceExhaustedException - { - boolean prefixWasPresent = existingPrePrefixNode != existingPostPrefixNode; - boolean prefixWasEmbedded = prefixWasPresent && isEmbeddedPrefixNode(existingPrePrefixNode); - if (prefixData == NONE) - { - if (prefixWasPresent && !prefixWasEmbedded) - recycleCell(existingPrePrefixNode); - return updatedPostPrefixNode; - } - - boolean childChanged = updatedPostPrefixNode != existingPostPrefixNode; - boolean dataChanged = !prefixWasPresent || prefixData != getIntVolatile(existingPrePrefixNode + PREFIX_CONTENT_OFFSET); - if (!childChanged && !dataChanged) - return existingPrePrefixNode; - - if (forcedCopy) - { - if (!childChanged && prefixWasEmbedded) - { - // If we directly create in this case, we will find embedding is possible and will overwrite the - // previous value. - // We could create a separate metadata node referencing the child, but in that case we'll - // use two nodes while one suffices. Instead, copy the child and embed the new metadata. - updatedPostPrefixNode = copyCell(existingPostPrefixNode); - } - else if (prefixWasPresent && !prefixWasEmbedded) - { - recycleCell(existingPrePrefixNode); - // otherwise cell is already recycled by the recycling of the child - } - return createPrefixNode(prefixData, updatedPostPrefixNode, isNull(existingPostPrefixNode)); - } - - // We can't update in-place if there was no preexisting prefix, or if the - // prefix was embedded and the target node must change. - if (!prefixWasPresent || prefixWasEmbedded && childChanged) - return createPrefixNode(prefixData, updatedPostPrefixNode, isNull(existingPostPrefixNode)); - - // Otherwise modify in place - if (childChanged) // to use volatile write but also ensure we don't corrupt embedded nodes - putIntVolatile(existingPrePrefixNode + PREFIX_POINTER_OFFSET, updatedPostPrefixNode); - if (dataChanged) - putIntVolatile(existingPrePrefixNode + PREFIX_CONTENT_OFFSET, prefixData); - return existingPrePrefixNode; - } - - /// After a node's children are processed, this is called to ascend from it. This means applying the collected - /// content to the compiled `updatedPostContentNode` and creating a mapping in the parent to it (or updating if - /// one already exists). - void attachAndMoveToParentState(int forcedCopyDepth) throws TrieSpaceExhaustedException - { - int updatedFullNode = applyContent(currentDepth >= forcedCopyDepth); - int existingFullNode = existingPreContentNode(); - --currentDepth; - - if (updatedFullNode != existingFullNode) - attachChild(transition(), updatedFullNode, currentDepth >= forcedCopyDepth); - } - - /// Ascend and update the root at the end of processing. - void attachRoot(int forcedCopyDepth) throws TrieSpaceExhaustedException - { - int updatedPreContentNode = applyContent(0 >= forcedCopyDepth); - int existingPreContentNode = existingPreContentNode(); - assert root == existingPreContentNode : "Unexpected change to root. Concurrent trie modification?"; - if (updatedPreContentNode != existingPreContentNode) - { - // Only write to root if they are different (value doesn't change, but - // we don't want to invalidate the value in other cores' caches unnecessarily). - root = updatedPreContentNode; - } - } - - public byte[] getBytes() - { - int arrSize = currentDepth; - byte[] data = new byte[arrSize]; - int pos = 0; - for (int i = 0; i < currentDepth; ++i) - { - int trans = transitionAtDepth(i); - data[pos++] = (byte) trans; - } - return data; - } - - public byte[] getBytes(Predicate shouldStop) - { - if (currentDepth == 0) - return new byte[0]; - - int arrSize = 1; - int i; - for (i = currentDepth - 1; i > 0; --i) - { - int content = contentIdAtDepth(i); - if (!isNull(content) && shouldStop.test(InMemoryTrie.this.getContent(content))) - break; - ++arrSize; - } - assert i > 0 || arrSize == currentDepth; // if the loop covers the whole stack, the array must cover the full depth - - byte[] data = new byte[arrSize]; - int pos = 0; - for (; i < currentDepth; ++i) - { - int trans = transitionAtDepth(i); - data[pos++] = (byte) trans; - } - return data; - } - - public ByteComparable.Version byteComparableVersion() - { - return byteComparableVersion; - } - } - - public interface KeyProducer + public InMemoryCursor makeCursor(Direction direction) { - /// Get the bytes of the path leading to this node. - byte[] getBytes(); - - /// Get the bytes of the path leading to this node from the closest ancestor whose content, after any new inserts - /// have been applied, satisfies the given predicate. - /// Note that the predicate is not called for the current position, because its content is not yet prepared when - /// the method is being called. - byte[] getBytes(Predicate shouldStop); - - ByteComparable.Version byteComparableVersion(); + return new InMemoryCursor(direction, root, 0, -1); } - /// Somewhat similar to [Trie.MergeResolver], this encapsulates logic to be applied whenever new content is - /// being upserted into a [InMemoryTrie]. Unlike [Trie.MergeResolver], [UpsertTransformer] will be - /// applied no matter if there's pre-existing content for that trie key/path or not. - /// - /// @param The content type for this [InMemoryTrie]. - /// @param The type of the new content being applied to this [InMemoryTrie]. - public interface UpsertTransformerWithKeyProducer + protected long emptySizeOnHeap() { - /// Called when there's content in the updating trie. - /// - /// @param existing Existing content for this key, or null if there isn't any. - /// @param update The update, always non-null. - /// @param keyState An interface that can be used to retrieve the path of the value being updated. - /// @return The combined value to use. - @Nonnull T apply(T existing, @Nonnull U update, @Nonnull KeyProducer keyState); - } - - /// Somewhat similar to [Trie.MergeResolver], this encapsulates logic to be applied whenever new content is - /// being upserted into a [InMemoryTrie]. Unlike [Trie.MergeResolver], [UpsertTransformer] will be - /// applied no matter if there's pre-existing content for that trie key/path or not. - /// - /// A version of the above that does not use a [KeyProducer]. - /// - /// @param The content type for this [InMemoryTrie]. - /// @param The type of the new content being applied to this [InMemoryTrie]. - public interface UpsertTransformer extends UpsertTransformerWithKeyProducer - { - /// Called when there's content in the updating trie. - /// - /// @param existing Existing content for this key, or null if there isn't any. - /// @param update The update, always non-null. - /// @return The combined value to use. Cannot be null. - @Nonnull T apply(T existing, @Nonnull U update); - - /// Version of the above that also provides the path of a value being updated. - /// - /// @param existing Existing content for this key, or null if there isn't any. - /// @param update The update, always non-null. - /// @param keyState An interface that can be used to retrieve the path of the value being updated. - /// @return The combined value to use. Cannot be null. - default @Nonnull T apply(T existing, @Nonnull U update, @Nonnull KeyProducer keyState) - { - return apply(existing, update); - } - } - - /// Interface providing features of the mutating node during mutation done using [#apply]. - /// Effectively a subset of the [Cursor] interface which only permits operations that are safe to - /// perform before iterating the children of the mutation node to apply the branch mutation. - /// - /// This is mainly used as an argument to predicates that decide when to copy substructure when modifying tries, - /// which enables different kinds of atomicity and consistency guarantees. - /// - /// See the InMemoryTrie javadoc or InMemoryTrieThreadedTest for demonstration of the typical usages and what they - /// achieve. - public interface NodeFeatures - { - /// Whether or not the node has more than one descendant. If a checker needs mutations to be atomic, they can - /// return true when this becomes true. - boolean isBranching(); - - /// The metadata associated with the node. If readers need to see a consistent view (i.e. where older updates - /// cannot be missed if a new one is presented) below some specified point (e.g. within a partition), the checker - /// should return true when it identifies that point. - T content(); - } - - private static class Mutation implements NodeFeatures - { - final UpsertTransformerWithKeyProducer transformer; - final Predicate> needsForcedCopy; - final Cursor mutationCursor; - final InMemoryTrie.ApplyState state; - int forcedCopyDepth; - - Mutation(UpsertTransformerWithKeyProducer transformer, - Predicate> needsForcedCopy, - Cursor mutationCursor, - InMemoryTrie.ApplyState state) - { - assert mutationCursor.depth() == 0 : "Unexpected non-fresh cursor."; - assert state.currentDepth == 0 : "Unexpected change to applyState. Concurrent trie modification?"; - this.transformer = transformer; - this.needsForcedCopy = needsForcedCopy; - this.mutationCursor = mutationCursor; - this.state = state; - } - - void apply() throws TrieSpaceExhaustedException - { - int depth = state.currentDepth; - while (true) - { - if (depth <= forcedCopyDepth) - forcedCopyDepth = needsForcedCopy.test(this) ? depth : Integer.MAX_VALUE; - - applyContent(); - - depth = mutationCursor.advance(); - if (state.advanceTo(depth, mutationCursor.incomingTransition(), forcedCopyDepth)) - break; - assert state.currentDepth == depth : "Unexpected change to applyState. Concurrent trie modification?"; - } - } - - void applyContent() throws TrieSpaceExhaustedException - { - U content = mutationCursor.content(); - if (content != null) - { - T existingContent = state.getContent(); - T combinedContent = transformer.apply(existingContent, content, state); - if (combinedContent == null) - throw new AssertionError("Transformer " + transformer + " returned null content for " - + existingContent + ", " + content); - state.setContent(combinedContent, - state.currentDepth >= forcedCopyDepth); // this is called at the start of processing - } - } - - - void complete() throws TrieSpaceExhaustedException - { - assert state.currentDepth == 0 : "Unexpected change to applyState. Concurrent trie modification?"; - state.attachRoot(forcedCopyDepth); - } - - @Override - public boolean isBranching() - { - // This is not very efficient, but we only currently use this option in tests. - // If it's needed for production use, isBranching should be implemented in the cursor interface. - Cursor dupe = mutationCursor.tailCursor(Direction.FORWARD); - int childDepth = dupe.advance(); - return childDepth > 0 && - dupe.skipTo(childDepth, dupe.incomingTransition() + 1) == childDepth; - } - - @Override - public U content() - { - return mutationCursor.content(); - } + return bufferType == BufferType.ON_HEAP ? EMPTY_SIZE_ON_HEAP : EMPTY_SIZE_OFF_HEAP; } /// Modify this trie to apply the mutation given in the form of a trie. Any content in the mutation will be resolved @@ -1465,190 +181,4 @@ public void putSingleton(ByteComparable key, else putSingleton(key, value, transformer); } - - /// Map-like put method, using a fast recursive implementation through the key bytes. May run into stack overflow if - /// the trie becomes too deep. When the correct position in the trie has been reached, the value will be resolved - /// with the given function before being placed in the trie (even if there's no pre-existing content in this trie). - /// @param key the trie path/key for the given value. - /// @param value the value being put in the memtable trie. Note that it can be of type different than the element - /// type for this memtable trie. It's up to the `transformer` to return the final value that will stay in - /// the memtable trie. - /// @param transformer a function applied to the potentially pre-existing value for the given key, and the new - /// value (of a potentially different type), returning the final value that will stay in the memtable trie. Applied - /// even if there's no pre-existing value in the memtable trie. - public void putRecursive(ByteComparable key, R value, final UpsertTransformer transformer) throws TrieSpaceExhaustedException - { - try - { - int newRoot = putRecursive(root, key.asComparableBytes(byteComparableVersion), value, transformer); - if (newRoot != root) - root = newRoot; - completeMutation(); - } - catch (Throwable t) - { - abortMutation(); - throw t; - } - } - - private int putRecursive(int node, ByteSource key, R value, final UpsertTransformer transformer) throws TrieSpaceExhaustedException - { - int transition = key.next(); - if (transition == ByteSource.END_OF_STREAM) - return applyContent(node, value, transformer); - - int child = getChild(node, transition); - - int newChild = putRecursive(child, key, value, transformer); - if (newChild == child) - return node; - - int skippedContent = followContentTransition(node); - int attachedChild = !isNull(skippedContent) - ? attachChild(skippedContent, transition, newChild) // Single path, no copying required - : expandOrCreateChainNode(transition, newChild); - - return preserveContent(node, skippedContent, attachedChild, false); - } - - private int applyContent(int node, R value, UpsertTransformer transformer) throws TrieSpaceExhaustedException - { - if (isNull(node)) - return addContent(transformer.apply(null, value)); - - if (isLeaf(node)) - { - int contentId = node; - setContent(contentId, transformer.apply(getContent(contentId), value)); - return node; - } - - if (offset(node) == PREFIX_OFFSET) - { - int contentId = getIntVolatile(node + PREFIX_CONTENT_OFFSET); - setContent(contentId, transformer.apply(getContent(contentId), value)); - return node; - } - else - return createPrefixNode(addContent(transformer.apply(null, value)), node, false); - } - - private void completeMutation() - { - cellAllocator.completeMutation(); - objectAllocator.completeMutation(); - } - - private void abortMutation() - { - cellAllocator.abortMutation(); - objectAllocator.abortMutation(); - } - - /// Returns true if the allocation threshold has been reached. To be called by the the writing thread (ideally, just - /// after the write completes). When this returns true, the user should switch to a new trie as soon as feasible. - /// - /// The trie expects up to 10% growth above this threshold. Any growth beyond that may be done inefficiently, and - /// the trie will fail altogether when the size grows beyond 2G - 256 bytes. - public boolean reachedAllocatedSizeThreshold() - { - return allocatedPos >= ALLOCATED_SIZE_THRESHOLD; - } - - /// For tests only! Advance the allocation pointer (and allocate space) by this much to test behaviour close to - /// full. - @VisibleForTesting - int advanceAllocatedPos(int wantedPos) throws TrieSpaceExhaustedException - { - while (allocatedPos < wantedPos) - allocateCell(); - return allocatedPos; - } - - /// For tests only! Returns the current allocation position. - @VisibleForTesting - int getAllocatedPos() - { - return allocatedPos; - } - - /// Returns the off heap size of the memtable trie itself, not counting any space taken by referenced content, or - /// any space that has been allocated but is not currently in use (e.g. recycled cells or preallocated buffer). - /// The latter means we are undercounting the actual usage, but the purpose of this reporting is to decide when - /// to flush out e.g. a memtable and if we include the unused space we would almost always end up flushing out - /// immediately after allocating a large buffer and not having a chance to use it. Counting only used space makes it - /// possible to flush out before making these large allocations. - public long usedSizeOffHeap() - { - return bufferType == BufferType.ON_HEAP ? 0 : usedBufferSpace(); - } - - /// Returns the on heap size of the memtable trie itself, not counting any space taken by referenced content, or - /// any space that has been allocated but is not currently in use (e.g. recycled cells or preallocated buffer). - /// The latter means we are undercounting the actual usage, but the purpose of this reporting is to decide when - /// to flush out e.g. a memtable and if we include the unused space we would almost always end up flushing out - /// immediately after allocating a large buffer and not having a chance to use it. Counting only used space makes it - /// possible to flush out before making these large allocations. - public long usedSizeOnHeap() - { - return usedObjectSpace() + - REFERENCE_ARRAY_ON_HEAP_SIZE * getBufferIdx(contentCount, CONTENTS_START_SHIFT, CONTENTS_START_SIZE) + - (bufferType == BufferType.ON_HEAP ? usedBufferSpace() + EMPTY_SIZE_ON_HEAP : EMPTY_SIZE_OFF_HEAP) + - REFERENCE_ARRAY_ON_HEAP_SIZE * getBufferIdx(allocatedPos, BUF_START_SHIFT, BUF_START_SIZE); - } - - private long usedBufferSpace() - { - return allocatedPos - cellAllocator.indexCountInPipeline() * CELL_SIZE; - } - - private long usedObjectSpace() - { - return (contentCount - objectAllocator.indexCountInPipeline()) * MemoryLayoutSpecification.SPEC.getReferenceSize(); - } - - /// Returns the amount of memory that has been allocated for various buffers but isn't currently in use. - /// The total on-heap space used by the trie is `usedSizeOnHeap() + unusedReservedOnHeapMemory()`. - @VisibleForTesting - public long unusedReservedOnHeapMemory() - { - int bufferOverhead = 0; - if (bufferType == BufferType.ON_HEAP) - { - int pos = this.allocatedPos; - UnsafeBuffer buffer = getBuffer(pos); - if (buffer != null) - bufferOverhead = buffer.capacity() - inBufferOffset(pos); - bufferOverhead += cellAllocator.indexCountInPipeline() * CELL_SIZE; - } - - int index = contentCount; - int leadBit = getBufferIdx(index, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); - int ofs = inBufferOffset(index, leadBit, CONTENTS_START_SIZE); - AtomicReferenceArray contentArray = contentArrays[leadBit]; - int contentOverhead = ((contentArray != null ? contentArray.length() : 0) - ofs); - contentOverhead += objectAllocator.indexCountInPipeline(); - contentOverhead *= MemoryLayoutSpecification.SPEC.getReferenceSize(); - - return bufferOverhead + contentOverhead; - } - - /// Release all recycled content references, including the ones waiting in still incomplete recycling lists. - /// This is a test method and can cause null pointer exceptions if used on a live trie. - /// - /// If similar functionality is required for non-test purposes, a version of this should be developed that only - /// releases references on barrier-complete lists. - @VisibleForTesting - public void releaseReferencesUnsafe() - { - for (int idx : objectAllocator.indexesInPipeline()) - setContent(~idx, null); - } - - /// Returns the number of values in the trie - public int valuesCount() - { - return contentCount; - } } diff --git a/src/java/org/apache/cassandra/db/tries/SingletonCursor.java b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java index ef6390cbe541..3f09054373a5 100644 --- a/src/java/org/apache/cassandra/db/tries/SingletonCursor.java +++ b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java @@ -80,7 +80,8 @@ public int advanceMultiple(TransitionsReceiver receiver) } currentTransition = current; nextTransition = next; - return currentDepth = ++depth; + currentDepth = ++depth; + return currentDepth; } @Override From 871f926a4623be9c722524b704b8a91d847ed49f Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Tue, 29 Apr 2025 15:30:13 +0300 Subject: [PATCH 12/22] Add deletion support for InMemoryTrie --- .../cassandra/db/tries/InMemoryBaseTrie.java | 373 ++++++++++++------ .../apache/cassandra/db/tries/InMemoryTrie.md | 12 + .../db/tries/InMemoryTrieTestBase.java | 101 ++++- 3 files changed, 366 insertions(+), 120 deletions(-) diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java index 9028905f6918..f079e0a05e62 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java @@ -17,15 +17,14 @@ */ package org.apache.cassandra.db.tries; +import com.google.common.annotations.VisibleForTesting; + import java.nio.ByteBuffer; import java.util.Arrays; import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.function.Predicate; import javax.annotation.Nonnull; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; - import org.agrona.concurrent.UnsafeBuffer; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.io.compress.BufferType; @@ -36,36 +35,8 @@ import org.apache.cassandra.utils.concurrent.OpOrder; import org.github.jamm.MemoryLayoutSpecification; -/// In-memory trie built for fast modification and reads executing concurrently with writes from a single mutator thread. -/// -/// The main method for performing writes is [#apply(Trie,UpsertTransformer,Predicate)] which takes a trie as -/// an argument and merges it into the current trie using the methods supplied by the given [UpsertTransformer], -/// force copying anything below the points where the third argument returns true. -/// -/// -/// The predicate can be used to implement several forms of atomicity and consistency guarantees: -/// - if the predicate is `nf -> false`, neither atomicity nor sequential consistency is guaranteed - readers -/// can see any mixture of old and modified content -/// - if the predicate is `nf -> true`, full sequential consistency will be provided, i.e. if a reader sees any -/// part of a modification, it will see all of it, and all the results of all previous modifications -/// - if the predicate is `nf -> nf.isBranching()` the write will be atomic, i.e. either none or all of the -/// content of the merged trie will be visible by concurrent readers, but not sequentially consistent, i.e. there -/// may be writes that are not visible to a reader even when they precede writes that are visible. -/// - if the predicate is `nf -> (nf.content())` the write will be consistent below the identified -/// point (used e.g. by Memtable to ensure partition-level consistency) -/// -/// -/// Additionally, the class provides several simpler write methods for efficiency and convenience: -/// - [#putRecursive(ByteComparable,Object,UpsertTransformer)] inserts a single value using a recursive walk. -/// It cannot provide consistency (single-path writes are always atomic). This is more efficient as it stores the -/// walk state in the stack rather than on the heap but can cause a `StackOverflowException`. -/// - [#putSingleton(ByteComparable,Object,UpsertTransformer)] is a non-recursive version of the above, using -/// the `apply` machinery. -/// - [#putSingleton(ByteComparable,Object,UpsertTransformer,boolean)] uses the fourth argument to choose -/// between the two methods above, where some external property can be used to decide if the keys are short enough -/// to permit recursive execution. -/// -/// Because it uses 32-bit pointers in byte buffers, this trie has a fixed size limit of 2GB. +/// Base class for mutable in-memory tries, providing the common infrastructure for plain, range and deletion-aware +/// in-memory tries. public abstract class InMemoryBaseTrie extends InMemoryReadTrie { // See the trie format description in InMemoryReadTrie. @@ -113,40 +84,12 @@ enum ExpectedLifetime switch (lifetime) { case SHORT: - cellAllocator = new MemoryAllocationStrategy.NoReuseStrategy(new MemoryAllocationStrategy.Allocator() - { - @Override - public int allocate() throws TrieSpaceExhaustedException - { - return allocateNewCell(); - } - }); - objectAllocator = new MemoryAllocationStrategy.NoReuseStrategy(new MemoryAllocationStrategy.Allocator() - { - @Override - public int allocate() - { - return allocateNewObject(); - } - }); + cellAllocator = new MemoryAllocationStrategy.NoReuseStrategy(this::allocateNewCell); + objectAllocator = new MemoryAllocationStrategy.NoReuseStrategy(this::allocateNewObject); break; case LONG: - cellAllocator = new MemoryAllocationStrategy.OpOrderReuseStrategy(new MemoryAllocationStrategy.Allocator() - { - @Override - public int allocate() throws TrieSpaceExhaustedException - { - return allocateNewCell(); - } - }, opOrder); - objectAllocator = new MemoryAllocationStrategy.OpOrderReuseStrategy(new MemoryAllocationStrategy.Allocator() - { - @Override - public int allocate() - { - return allocateNewObject(); - } - }, opOrder); + cellAllocator = new MemoryAllocationStrategy.OpOrderReuseStrategy(this::allocateNewCell, opOrder); + objectAllocator = new MemoryAllocationStrategy.OpOrderReuseStrategy(this::allocateNewObject, opOrder); break; default: throw new AssertionError(); @@ -249,9 +192,11 @@ private int allocateNewObject() /// /// @return A content id that can be used to reference the content, encoded as `~index` where index is the /// position of the value in the content array. - @Nonnull private int addContent(@Nonnull T value) throws TrieSpaceExhaustedException + private int addContent(T value) throws TrieSpaceExhaustedException { - Preconditions.checkNotNull(value, "Content value cannot be null"); + if (value == null) + return NONE; + int index = objectAllocator.allocate(); int leadBit = getBufferIdx(index, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); int ofs = inBufferOffset(index, leadBit, CONTENTS_START_SIZE); @@ -314,13 +259,15 @@ private int getOrAllocate(int pointerAddress, int offsetWhenAllocating) throws T private int getCopyOrAllocate(int pointerAddress, int originalChild, int offsetWhenAllocating) throws TrieSpaceExhaustedException { int child = getIntVolatile(pointerAddress); - if (child == originalChild) + if (child == NONE) { - if (originalChild == NONE) - child = allocateCell() | offsetWhenAllocating; - else - child = copyCell(originalChild); - + child = allocateCell() | offsetWhenAllocating; + // volatile writes not needed because this branch is not attached yet + putInt(pointerAddress, child); + } + else if (child == originalChild) + { + child = copyCell(originalChild); // volatile writes not needed because this branch is not attached yet putInt(pointerAddress, child); } @@ -372,7 +319,7 @@ private int attachChildCopying(int node, int originalNode, int trans, int newChi return attachChildToSplitCopying(node, originalNode, trans, newChild); default: // chain nodes - return attachChildToChainCopying(node, originalNode, trans, newChild); // always copies + return attachChildToChainCopying(node, trans, newChild); // always copies } } @@ -407,6 +354,8 @@ private int attachChildToSplit(int node, int trans, int newChild) throws TrieSpa int mid = getIntVolatile(midPos); if (isNull(mid)) { + if (isNull(newChild)) + return node; mid = createEmptySplitNode(); int tailPos = splitCellPointerAddress(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); int tail = createEmptySplitNode(); @@ -421,6 +370,8 @@ private int attachChildToSplit(int node, int trans, int newChild) throws TrieSpa int tail = getIntVolatile(tailPos); if (isNull(tail)) { + if (isNull(newChild)) + return node; tail = createEmptySplitNode(); int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); putInt(childPos, newChild); @@ -429,8 +380,61 @@ private int attachChildToSplit(int node, int trans, int newChild) throws TrieSpa } int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); - putIntVolatile(childPos, newChild); - return node; + if (isNull(newChild)) + return removeSplitChildVolatile(node, midPos, mid, tailPos, tail, childPos); + else + { + putIntVolatile(childPos, newChild); + return node; // normal path, adding data + } + } + + /// Remove a split child, propagating the removal upward if this results in an empty split node cell. + /// This version of the method is called when the node being modified is reachable and may be concurrently accessed + /// by reads. + private int removeSplitChildVolatile(int node, int midPos, int mid, int tailPos, int tail, int childPos) + throws TrieSpaceExhaustedException + { + if (isNull(getIntVolatile(childPos))) + return node; + + // Because there may be concurrent accesses to this node that have saved the path we are removing as the next + // transition and expect it to be valid, we need to copy any cell where we set the value to NONE. + if (!isSplitBlockEmptyExcept(tail, SPLIT_OTHER_LEVEL_LIMIT, childPos)) + { + int newTail = copyCell(tail); + putInt(newTail + childPos - tail, NONE); + putIntVolatile(tailPos, newTail); + return node; + } + recycleCell(tail); + if (!isSplitBlockEmptyExcept(mid, SPLIT_OTHER_LEVEL_LIMIT, tailPos)) + { + int newMid = copyCell(mid); + putInt(newMid + tailPos - mid, NONE); + putIntVolatile(midPos, newMid); + return node; + } + recycleCell(mid); + if (!isSplitBlockEmptyExcept(node, SPLIT_START_LEVEL_LIMIT, midPos)) + { + int newNode = copyCell(node); + putInt(newNode + midPos - node, NONE); + return newNode; + } + recycleCell(node); + return NONE; + } + + boolean isSplitBlockEmptyExcept(int node, int limit, int deletedPos) + { + for (int i = 0; i < limit; ++i) + { + int pos = splitCellPointerAddress(node, i, limit); + if (pos != deletedPos && !isNull(getIntVolatile(pos))) + return false; + } + return true; } /// Non-volatile version of `attachChildToSplit`. Used when the split node is not reachable yet (during the conversion @@ -446,7 +450,10 @@ private int attachChildToSplitNonVolatile(int node, int trans, int newChild) thr assert offset(tail) == SPLIT_OFFSET : "Invalid split node in trie"; int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); putInt(childPos, newChild); - return node; + if (isNull(newChild)) + return removeSplitPathNonVolatile(node, midPos, mid, tailPos, tail); + else + return node; // normal path, adding data } /// Attach a child to the given split node, copying all modified content to enable atomic visibility @@ -460,19 +467,48 @@ private int attachChildToSplitCopying(int node, int originalNode, int trans, int node = copyIfOriginal(node, originalNode); assert offset(node) == SPLIT_OFFSET : "Invalid split node in trie"; - int midPos = splitCellPointerAddress(0, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT); - int midOriginal = originalNode != NONE ? getIntVolatile(midPos + originalNode) : NONE; - int mid = getCopyOrAllocate(node + midPos, midOriginal, SPLIT_OFFSET); + int midPos = splitCellPointerAddress(node, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT); + int midOriginal = originalNode != NONE ? getIntVolatile(midPos + originalNode - node) : NONE; + int mid = getCopyOrAllocate(midPos, midOriginal, SPLIT_OFFSET); assert offset(mid) == SPLIT_OFFSET : "Invalid split node in trie"; - int tailPos = splitCellPointerAddress(0, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); - int tailOriginal = midOriginal != NONE ? getIntVolatile(tailPos + midOriginal) : NONE; - int tail = getCopyOrAllocate(mid + tailPos, tailOriginal, SPLIT_OFFSET); + int tailPos = splitCellPointerAddress(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); + int tailOriginal = midOriginal != NONE ? getIntVolatile(tailPos + midOriginal - mid) : NONE; + int tail = getCopyOrAllocate(tailPos, tailOriginal, SPLIT_OFFSET); assert offset(tail) == SPLIT_OFFSET : "Invalid split node in trie"; int childPos = splitCellPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT); putInt(childPos, newChild); - return node; + if (isNull(newChild)) + return removeSplitPathNonVolatile(node, midPos, mid, tailPos, tail); + else + return node; + } + + /// Propagate the removal of a split child upward if it resulted in an empty split node cell, + /// assuming that the node being modified is not reachable and cannot be accessed concurrently. + private int removeSplitPathNonVolatile(int node, int midPos, int mid, int tailPos, int tail) + { + if (!isSplitBlockEmpty(tail, SPLIT_OTHER_LEVEL_LIMIT)) + return node; + recycleCell(tail); + putInt(tailPos, NONE); + if (!isSplitBlockEmpty(mid, SPLIT_OTHER_LEVEL_LIMIT)) + return node; + recycleCell(mid); + putInt(midPos, NONE); + if (!isSplitBlockEmpty(node, SPLIT_START_LEVEL_LIMIT)) + return node; + recycleCell(node); + return NONE; + } + + boolean isSplitBlockEmpty(int node, int limit) + { + for (int i = 0; i < limit; ++i) + if (!isNull(getSplitCellPointer(node, i, limit))) + return false; + return true; } /// Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. @@ -488,6 +524,8 @@ private int attachChildToSparse(int node, int trans, int newChild) throws TrieSp final int existing = getUnsignedByte(node + SPARSE_BYTES_OFFSET + index); if (existing == trans) { + if (isNull(newChild)) + return removeSparseChild(node, index); putIntVolatile(node + SPARSE_CHILDREN_OFFSET + index * 4, newChild); return node; } @@ -495,6 +533,8 @@ else if (existing < trans) ++smallerCount; } int childCount = index; + if (isNull(newChild)) + return node; if (childCount == SPARSE_CHILD_COUNT) { @@ -526,8 +566,45 @@ else if (existing < trans) return node; } - /// Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. - /// Resulting node is not reachable, no volatile set needed. + /// Remove a child of the given sparse node. To ensure the safety of concurrent operations, this is always done + /// as a copying operation as we can't safely shift entries in a sparse node. + private int removeSparseChild(int node, int index) throws TrieSpaceExhaustedException + { + recycleCell(node); + int order = getUnsignedShortVolatile(node + SPARSE_ORDER_OFFSET); + if (index <= 1 && order == 6) + { + int survivingIndex = index ^ 1; + return expandOrCreateChainNode(getUnsignedByte(node + SPARSE_BYTES_OFFSET + survivingIndex), + getIntVolatile(node + SPARSE_CHILDREN_OFFSET + survivingIndex * 4)); + } + + // Because we need the smallest child to not be the last (which can happen if we just remove entries), we will + // put the remaining data in order. + int newNode = allocateCell() | SPARSE_OFFSET; + int i = 0; + int newOrder = 0; + int mul = 1; + while (order > 0) + { + int next = order % SPARSE_CHILD_COUNT; + order /= SPARSE_CHILD_COUNT; + if (next == index) + continue; + putInt(newNode + SPARSE_CHILDREN_OFFSET + i * 4, getIntVolatile(node + SPARSE_CHILDREN_OFFSET + next * 4)); + putInt(newNode + SPARSE_BYTES_OFFSET + i, getUnsignedByte(node + SPARSE_BYTES_OFFSET + next)); + newOrder += i * mul; + mul *= SPARSE_CHILD_COUNT; + ++i; + } + putShort(newNode + SPARSE_ORDER_OFFSET, (short) newOrder); + return newNode; + } + + /** + * Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. + * Resulting node is not reachable, no volatile set needed. + */ private int attachChildToSparseCopying(int node, int originalNode, int trans, int newChild) throws TrieSpaceExhaustedException { int index; @@ -540,6 +617,8 @@ private int attachChildToSparseCopying(int node, int originalNode, int trans, in final int existing = getUnsignedByte(node + SPARSE_BYTES_OFFSET + index); if (existing == trans) { + if (isNull(newChild)) + return removeSparseChild(node, index); node = copyIfOriginal(node, originalNode); putInt(node + SPARSE_CHILDREN_OFFSET + index * 4, newChild); return node; @@ -549,6 +628,9 @@ else if (existing < trans) } int childCount = index; + if (isNull(newChild)) + return node; + if (childCount == SPARSE_CHILD_COUNT) { // Node is full. Switch to split. @@ -620,11 +702,20 @@ private int attachChildToChain(int node, int transitionByte, int newChild) throw // This is still a single path. Update child if possible (only if this is the last character in the chain). if (offset(node) == LAST_POINTER_OFFSET - 1) { + if (isNull(newChild)) + { + recycleCell(node); + return NONE; + } + putIntVolatile(node + 1, newChild); return node; } else { + if (isNull(newChild)) + return NONE; + // This will only be called if new child is different from old, and the update is not on the final child // where we can change it in place (see attachChild). We must always create something new. // Note that since this is not the last character, we either still need this cell or we have already @@ -634,13 +725,15 @@ private int attachChildToChain(int node, int transitionByte, int newChild) throw return expandOrCreateChainNode(transitionByte, newChild); } } + if (isNull(newChild)) + return node; // The new transition is different, so we no longer have only one transition. Change type. return convertChainToSparse(node, existingByte, newChild, transitionByte); } /// Attach a child to the given chain node, when we are force-copying. - private int attachChildToChainCopying(int node, int originalNode, int transitionByte, int newChild) + private int attachChildToChainCopying(int node, int transitionByte, int newChild) throws TrieSpaceExhaustedException { int existingByte = getUnsignedByte(node); @@ -650,17 +743,18 @@ private int attachChildToChainCopying(int node, int originalNode, int transition // Make sure we release the cell if it will no longer be referenced (if we update last reference, the whole // path has to move as the other nodes in this chain can't be remapped). if (offset(node) == LAST_POINTER_OFFSET - 1) - { - assert node == originalNode; // if we have already created a node, the character can't match what - // it was created with - recycleCell(node); - } + + if (isNull(newChild)) + return NONE; return expandOrCreateChainNode(transitionByte, newChild); } else { + if (isNull(newChild)) + return node; + // The new transition is different, so we no longer have only one transition. Change type. return convertChainToSparse(node, existingByte, newChild, transitionByte); } @@ -843,12 +937,20 @@ private int preserveContent(int existingPreContentNode, } assert offset(existingPreContentNode) == PREFIX_OFFSET : "Unexpected content in non-prefix and non-leaf node."; - return updatePrefixNodeChild(existingPreContentNode, updatedPostContentNode, forcedCopy); + if (updatedPostContentNode != NONE) + return updatePrefixNodeChild(existingPreContentNode, updatedPostContentNode, forcedCopy); + else + { + if (!isEmbeddedPrefixNode(existingPreContentNode)) + recycleCell(existingPreContentNode); + // otherwise cell is recycled with the post-prefix node + return getIntVolatile(existingPreContentNode + PREFIX_CONTENT_OFFSET); + } } final ApplyState applyState = new ApplyState(); - /// Represents the state for an [#apply] operation. Contains a stack of all nodes we descended through + /// Represents the state for an [InMemoryTrie#apply] operation. Contains a stack of all nodes we descended through /// and used to update the nodes with any new data during ascent. /// /// To make this as efficient and GC-friendly as possible, we use an integer array (instead of is an object stack) @@ -1004,8 +1106,6 @@ else if (content == null) { releaseContent(contentId); setContentId(NONE); - // At this point we are not deleting branches on the way up, just making sure we don't hold on to - // references to content. } else if (content == InMemoryBaseTrie.this.getContent(contentId)) { @@ -1194,6 +1294,19 @@ public ByteComparable.Version byteComparableVersion() { return byteComparableVersion; } + + public String toString() + { + StringBuilder sb = new StringBuilder(); + sb.append('@'); + for (int i = 0; i < currentDepth; ++i) + sb.append(String.format("%02x", transitionAtDepth(i))); + + sb.append(" existingPostContentNode=").append(existingPostContentNode()); + sb.append(" updatedPostContentNode=").append(updatedPostContentNode()); + sb.append(" contentId=").append(contentId()); + return sb.toString(); + } } public interface KeyProducer @@ -1223,8 +1336,8 @@ public interface UpsertTransformerWithKeyProducer /// @param existing Existing content for this key, or null if there isn't any. /// @param update The update, always non-null. /// @param keyState An interface that can be used to retrieve the path of the value being updated. - /// @return The combined value to use. - @Nonnull T apply(T existing, @Nonnull U update, KeyProducer keyState); + /// @return The combined value to use. A value of null will delete the existing entry. + T apply(T existing, @Nonnull U update, KeyProducer keyState); } /// Somewhat similar to [Trie.MergeResolver], this encapsulates logic to be applied whenever new content is @@ -1241,16 +1354,16 @@ public interface UpsertTransformer extends UpsertTransformerWithKeyProduce /// /// @param existing Existing content for this key, or null if there isn't any. /// @param update The update, always non-null. - /// @return The combined value to use. Cannot be null. - @Nonnull T apply(T existing, @Nonnull U update); + /// @return The combined value to use. A value of null will delete the existing entry. + T apply(T existing, @Nonnull U update); /// Version of the above that also provides the path of a value being updated. /// /// @param existing Existing content for this key, or null if there isn't any. /// @param update The update, always non-null. /// @param keyState An interface that can be used to retrieve the path of the value being updated. - /// @return The combined value to use. Cannot be null. - @Nonnull default T apply(T existing, @Nonnull U update, @Nonnull KeyProducer keyState) + /// @return The combined value to use. A value of null will delete the existing entry. + default T apply(T existing, @Nonnull U update, @Nonnull KeyProducer keyState) { return apply(existing, update); } @@ -1322,9 +1435,7 @@ void applyContent() throws TrieSpaceExhaustedException { T existingContent = state.getContent(); T combinedContent = transformer.apply(existingContent, content, state); - if (combinedContent == null) - throw new AssertionError("Transformer " + transformer + " returned null content for " - + existingContent + ", " + content); + state.setContent(combinedContent, // can be null state.currentDepth >= forcedCopyDepth); // this is called at the start of processing } @@ -1409,16 +1520,50 @@ private int applyContent(int node, R value, UpsertTransformer transfor if (isLeaf(node)) { int contentId = node; - setContent(contentId, transformer.apply(getContent(contentId), value)); - return node; + T newContent = transformer.apply(getContent(contentId), value); + if (newContent != null) + { + setContent(contentId, newContent); + return node; + } + else + { + releaseContent(contentId); + return NONE; + } } if (offset(node) == PREFIX_OFFSET) { int contentId = getIntVolatile(node + PREFIX_CONTENT_OFFSET); - setContent(contentId, transformer.apply(getContent(contentId), value)); - return node; + T newContent = transformer.apply(getContent(contentId), value); + if (newContent != null) + { + setContent(contentId, newContent); + return node; + } + else + { + releaseContent(contentId); + + int b = getUnsignedByte(node + PREFIX_FLAGS_OFFSET); + if (b < CELL_SIZE) + { + // embedded prefix node + return node - PREFIX_OFFSET + b; + } + else + { + // separate prefix node. recycle it as it's no longer needed + recycleCell(node); + return getIntVolatile(node + PREFIX_POINTER_OFFSET); + } + } } + + T newContent = transformer.apply(null, value); + if (newContent == null) + return node; else return createPrefixNode(addContent(transformer.apply(null, value)), node, false); } @@ -1490,12 +1635,14 @@ public long usedSizeOnHeap() REFERENCE_ARRAY_ON_HEAP_SIZE * getBufferIdx(allocatedPos, BUF_START_SHIFT, BUF_START_SIZE); } - private long usedBufferSpace() + @VisibleForTesting + long usedBufferSpace() { return allocatedPos - cellAllocator.indexCountInPipeline() * CELL_SIZE; } - private long usedObjectSpace() + @VisibleForTesting + long usedObjectSpace() { return (contentCount - objectAllocator.indexCountInPipeline()) * MemoryLayoutSpecification.SPEC.getReferenceSize(); } @@ -1505,7 +1652,7 @@ private long usedObjectSpace() @VisibleForTesting public long unusedReservedOnHeapMemory() { - int bufferOverhead = 0; + long bufferOverhead = 0; if (bufferType == BufferType.ON_HEAP) { int pos = this.allocatedPos; @@ -1519,7 +1666,7 @@ public long unusedReservedOnHeapMemory() int leadBit = getBufferIdx(index, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); int ofs = inBufferOffset(index, leadBit, CONTENTS_START_SIZE); AtomicReferenceArray contentArray = contentArrays[leadBit]; - int contentOverhead = ((contentArray != null ? contentArray.length() : 0) - ofs); + long contentOverhead = ((contentArray != null ? contentArray.length() : 0) - ofs); contentOverhead += objectAllocator.indexCountInPipeline(); contentOverhead *= MemoryLayoutSpecification.SPEC.getReferenceSize(); diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md index 1952d864e056..c088f5d8ee58 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md +++ b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md @@ -804,6 +804,18 @@ Ascending back to add the child `~3`, we add a child to `NONE` and get `updatedP the existing content, we create the embedded prefix node `updatedPreContentNode = 0x0BF` with `contentIndex = 1` and pass that on to the recursion. +### Deletion + +Deletion of data in `InMemoryTrie`s is achieved by returning `null` for the value that needs to be put in a position +with existing content (both `apply` and `putSingleton` take an `UpsertTransformer` that is applied to the combination +of existing and update value; this transformer can choose to return `null`). + +This automatically results in `NONE` value for the content id. On the way up the recursive application chain, we +recognize `NONE` for the child pointer and apply this as removal of the child. Depending on the type of node, this may +be achieved by dropping the node (`Chain`), by putting the `NONE` value as the child pointer, or by duplicating a node +to switch its type or remove a child. This may in turn result in an empty node, which returns `NONE` as the child +pointer, continuing the removal upwards in the recursive chain. + ### Memory management and cell reuse As mentioned in the beginning, in order to avoid long garbage collection pauses due to large long-lasting content in diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java index 92ddadfc8477..e019cab9fba9 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java @@ -45,6 +45,7 @@ import static org.apache.cassandra.db.tries.TrieUtil.VERSION; import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; @RunWith(Parameterized.class) public abstract class InMemoryTrieTestBase @@ -316,9 +317,9 @@ public void testPrefixEvolution() } @Test - public void testPrefixUnsafeMulti() + public void testPrefixUnsafeChain() { - // Make sure prefixes on inside a multi aren't overwritten by embedded metadata node. + // Make sure prefixes on inside a chain aren't overwritten by embedded metadata node. testEntries(new String[] { "test89012345678901234567890", "test8", @@ -362,6 +363,72 @@ private void testEntries(String[] tests, Function mapping) for (String test : tests) assertEquals(test, trie.get(mapping.apply(test))); + + testDeletions(tests, mapping, trie); + + randomizedTestEntries(tests, mapping, trie); + } + + private void testDeletions(String[] tests, Function mapping, InMemoryTrie trie) + { + System.out.println("\nDeleting all entries"); + List toDelete = Arrays.stream(tests).distinct().collect(Collectors.toList()); + while (!toDelete.isEmpty()) + { + int index = rand.nextInt(toDelete.size()); + String entry = toDelete.remove(index); + Preencoded e = mapping.apply(entry); + System.out.println("Deleting " + asString(e) + ": " + entry); + delete(trie, e); + System.out.println("Trie\n" + trie.dump()); + + for (String test : toDelete) + assertEquals(test, trie.get(mapping.apply(test))); + } + assertTrue(trie.isEmpty()); + if (trie.cellAllocator instanceof MemoryAllocationStrategy.OpOrderReuseStrategy) + { + assertEquals(0L, trie.usedBufferSpace()); + assertEquals(0L, trie.usedObjectSpace()); + } + } + + private void randomizedTestEntries(String[] tests, Function mapping, InMemoryTrie trie) + { + System.out.println("\nRandomized insert and delete"); + List toInsert = Arrays.stream(tests).distinct().collect(Collectors.toList()); + List inserted = new ArrayList<>(); + + while (!toInsert.isEmpty()) + { + if (rand.nextDouble() > 0.35) + { + // Insert one value + int index = rand.nextInt(toInsert.size()); + String entry = toInsert.remove(index); + Preencoded e = mapping.apply(entry); + System.out.println("Adding " + asString(e) + ": " + entry); + putSimpleResolve(trie, e, entry, (x, y) -> y); + System.out.println("Trie\n" + trie.dump()); + inserted.add(entry); + } + else if (!inserted.isEmpty()) + { + // Delete one value + int index = rand.nextInt(inserted.size()); + String entry = inserted.remove(index); + Preencoded e = mapping.apply(entry); + System.out.println("Deleting " + asString(e) + ": " + entry); + delete(trie, e); + System.out.println("Trie\n" + trie.dump()); + toInsert.add(entry); + } + + for (String test : inserted) + assertEquals(test, trie.get(mapping.apply(test))); + for (String test: toInsert) + assertEquals(null, trie.get(mapping.apply(test))); + } } static InMemoryTrie makeInMemoryTrie(Preencoded[] src, @@ -589,15 +656,15 @@ static String asString(Preencoded bc) return bc != null ? bc.byteComparableAsString(VERSION) : "null"; } - void putSimpleResolve(InMemoryTrie trie, - Preencoded key, - T value, - Trie.MergeResolver resolver) + void putSimpleResolve(InMemoryTrie trie, + Preencoded key, + T value, + Trie.MergeResolver resolver) { putSimpleResolve(trie, key, value, resolver, usePut()); } - static void putSimpleResolve(InMemoryTrie trie, + static void putSimpleResolve(InMemoryTrie trie, Preencoded key, T value, Trie.MergeResolver resolver, @@ -615,4 +682,24 @@ static void putSimpleResolve(InMemoryTrie trie, throw Throwables.propagate(e); } } + + void delete(InMemoryTrie trie, ByteComparable key) + { + delete(trie, key, usePut()); + } + + static void delete(InMemoryTrie trie, ByteComparable key, boolean usePut) + { + try + { + trie.putSingleton(key, + Boolean.TRUE, + (existing, update) -> update ? null : existing, + usePut); + } + catch (TrieSpaceExhaustedException e) + { + throw Throwables.propagate(e); + } + } } From e9b8ce1eceee2848d17a7d203d72378ca4365b3c Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Mon, 24 Mar 2025 18:41:56 +0200 Subject: [PATCH 13/22] Add RangeTrie Range tries are tries made of ranges of coverage, which track applicable ranges and are mainly to be used to store deletions and deletion ranges. --- .../db/tries/CollectionMergeCursor.java | 197 ++++-- .../cassandra/db/tries/InMemoryBaseTrie.java | 134 +++- .../cassandra/db/tries/InMemoryRangeTrie.java | 359 +++++++++++ .../cassandra/db/tries/InMemoryReadTrie.java | 130 +++- .../cassandra/db/tries/InMemoryTrie.java | 12 +- .../db/tries/IntersectionCursor.java | 12 +- .../cassandra/db/tries/MergeCursor.java | 132 +++- .../cassandra/db/tries/PrefixedCursor.java | 74 ++- .../cassandra/db/tries/RangeCursor.java | 194 ++++++ .../db/tries/RangeIntersectionCursor.java | 301 +++++++++ .../apache/cassandra/db/tries/RangeState.java | 60 ++ .../apache/cassandra/db/tries/RangeTrie.java | 167 +++++ .../cassandra/db/tries/RangesCursor.java | 1 + .../cassandra/db/tries/SingletonCursor.java | 44 +- .../org/apache/cassandra/db/tries/Trie.java | 10 +- .../org/apache/cassandra/db/tries/Trie.md | 208 +++++++ .../apache/cassandra/db/tries/TrieSet.java | 47 ++ .../cassandra/db/tries/TrieSetCursor.java | 113 +++- .../db/tries/TrieSetIntersectionCursor.java | 4 +- .../db/tries/VerificationCursor.java | 250 ++++++-- .../db/memtable/MemtableThreadedTest.java | 2 +- .../cassandra/db/tries/CellReuseTest.java | 6 +- .../db/tries/CollectionMergeTrieTest.java | 4 +- .../db/tries/ConsistencyTestBase.java | 410 +++++++++++++ .../db/tries/InMemoryRangeTrieTest.java | 382 ++++++++++++ .../tries/InMemoryRangeTrieThreadedTest.java | 64 ++ .../db/tries/InMemoryTrieConsistencyTest.java | 205 +++++++ .../db/tries/InMemoryTrieThreadedTest.java | 578 +----------------- .../db/tries/IntersectionTrieTest.java | 42 +- .../db/tries/RangeTrieIntersectionTest.java | 464 ++++++++++++++ .../db/tries/RangeTrieMergeTest.java | 556 +++++++++++++++++ .../cassandra/db/tries/RangesTrieSetTest.java | 6 +- .../cassandra/db/tries/TestRangeState.java | 241 ++++++++ .../cassandra/db/tries/ThreadedTestBase.java | 167 +++++ .../apache/cassandra/db/tries/TrieUtil.java | 79 +++ 35 files changed, 4841 insertions(+), 814 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/tries/InMemoryRangeTrie.java create mode 100644 src/java/org/apache/cassandra/db/tries/RangeCursor.java create mode 100644 src/java/org/apache/cassandra/db/tries/RangeIntersectionCursor.java create mode 100644 src/java/org/apache/cassandra/db/tries/RangeState.java create mode 100644 src/java/org/apache/cassandra/db/tries/RangeTrie.java create mode 100644 test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java create mode 100644 test/unit/org/apache/cassandra/db/tries/InMemoryRangeTrieTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/InMemoryRangeTrieThreadedTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/RangeTrieIntersectionTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/RangeTrieMergeTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/TestRangeState.java create mode 100644 test/unit/org/apache/cassandra/db/tries/ThreadedTestBase.java diff --git a/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java b/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java index ad9ea6d2c195..40ef93856d3e 100644 --- a/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java +++ b/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java @@ -22,6 +22,8 @@ import java.util.Collection; import java.util.List; import java.util.function.BiFunction; +import java.util.function.IntFunction; + import org.apache.cassandra.utils.bytecomparable.ByteComparable; /// A merged view of multiple tries. @@ -73,34 +75,38 @@ /// /// Note: This is a simplification of the MergeIterator code from CASSANDRA-8915, without the leading ordered /// section and equalParent flag since comparisons of cursor positions are cheap. -class CollectionMergeCursor implements Cursor +abstract class CollectionMergeCursor> implements Cursor { - private final Trie.CollectionMergeResolver resolver; - private final Direction direction; + final Trie.CollectionMergeResolver resolver; + final Direction direction; /// The smallest cursor, tracked separately to improve performance in single-source sections of the trie. - private Cursor head; + C head; /// Binary heap of the remaining cursors. The smallest element is at position 0. /// Every element `i` is smaller than or equal to its two children, i.e. /// ```heap[i] <= heap[i*2 + 1] && heap[i] <= heap[i*2 + 2]``` - private final Cursor[] heap; + final C[] heap; /// A list used to collect contents during [#content()] calls. - private final List contents; + final List contents; + /// Whether content has already been collected for this position. + boolean contentCollected; + /// The collected content. + T collectedContent; - CollectionMergeCursor(Trie.CollectionMergeResolver resolver, Direction direction, Collection inputs, BiFunction> extractor) + CollectionMergeCursor(Trie.CollectionMergeResolver resolver, Direction direction, Collection inputs, IntFunction cursorArrayConstructor, BiFunction extractor) { this.resolver = resolver; this.direction = direction; int count = inputs.size(); // Get cursors for all inputs. Put one of them in head and the rest in the heap. - heap = new Cursor[count - 1]; + heap = cursorArrayConstructor.apply(count - 1); contents = new ArrayList<>(count); int i = -1; for (I src : inputs) { - Cursor cursor = extractor.apply(src, direction); + C cursor = extractor.apply(src, direction); assert cursor.depth() == 0; if (i >= 0) heap[i] = cursor; @@ -112,11 +118,11 @@ CollectionMergeCursor(Trie.CollectionMergeResolver resolver, Direction di } /// Interface for internal operations that can be applied to selected top elements of the heap. - interface HeapOp + interface HeapOp> { - void apply(CollectionMergeCursor self, Cursor cursor, int index); + void apply(CollectionMergeCursor self, C cursor, int index); - default boolean shouldContinueWithChild(Cursor child, Cursor head) + default boolean shouldContinueWithChild(C child, C head) { return equalCursor(child, head); } @@ -125,18 +131,18 @@ default boolean shouldContinueWithChild(Cursor child, Cursor head) /// Apply a non-interfering operation, i.e. one that does not change the cursor state, to all inputs in the heap /// that satisfy the [HeapOp#shouldContinueWithChild] condition (by default, being equal to the head). /// For interfering operations like advancing the cursors, use [#advanceSelectedAndRestoreHeap(AdvancingHeapOp)]. - private void applyToSelectedInHeap(HeapOp action) + void applyToSelectedInHeap(HeapOp action) { applyToSelectedElementsInHeap(action, 0); } /// Interface for internal advancing operations that can be applied to the heap cursors. This interface provides /// the code to restore the heap structure after advancing the cursors. - interface AdvancingHeapOp extends HeapOp + interface AdvancingHeapOp> extends HeapOp { - void apply(Cursor cursor); + void apply(C cursor); - default void apply(CollectionMergeCursor self, Cursor cursor, int index) + default void apply(CollectionMergeCursor self, C cursor, int index) { // Apply the operation, which should advance the position of the element. apply(cursor); @@ -152,7 +158,7 @@ default void apply(CollectionMergeCursor self, Cursor cursor, int index) /// Advance the state of all inputs in the heap that satisfy the [#shouldContinueWithChild] condition /// (by default, being equal to the head) and restore the heap invariant. - private void advanceSelectedAndRestoreHeap(AdvancingHeapOp action) + void advanceSelectedAndRestoreHeap(AdvancingHeapOp action) { applyToSelectedElementsInHeap(action, 0); } @@ -165,11 +171,11 @@ private void advanceSelectedAndRestoreHeap(AdvancingHeapOp action) /// that advances the cursor to a new state, wrapped in a [AdvancingHeapOp] ([#advance] or /// [#skipTo]). The latter interface takes care of pushing elements down in the heap after advancing /// and restores the subheap state on return from each level of the recursion. - private void applyToSelectedElementsInHeap(HeapOp action, int index) + private void applyToSelectedElementsInHeap(HeapOp action, int index) { if (index >= heap.length) return; - Cursor item = heap[index]; + C item = heap[index]; if (!action.shouldContinueWithChild(item, head)) return; @@ -183,9 +189,15 @@ private void applyToSelectedElementsInHeap(HeapOp action, int index) action.apply(this, item, index); } + void applyToAllOnHeap(HeapOp action) + { + for (int i = 0; i < heap.length; i++) + action.apply(this, heap[i], i); + } + /// Push the given state down in the heap from the given index until it finds its proper place among /// the subheap rooted at that position. - private void heapifyDown(Cursor item, int index) + private void heapifyDown(C item, int index) { while (true) { @@ -217,7 +229,7 @@ private int maybeSwapHead(int headDepth) return headDepth; // head is still smallest // otherwise we need to swap heap and heap[0] - Cursor newHeap0 = head; + C newHeap0 = head; head = heap[0]; heapifyDown(newHeap0, 0); return heap0Depth; @@ -228,8 +240,19 @@ boolean branchHasMultipleSources() return equalCursor(heap[0], head); } + boolean isExhausted() + { + return head.depth() < 0; + } + @Override public int advance() + { + contentCollected = false; + return doAdvance(); + } + + private int doAdvance() { advanceSelectedAndRestoreHeap(Cursor::advance); return maybeSwapHead(head.advance()); @@ -238,10 +261,11 @@ public int advance() @Override public int advanceMultiple(TransitionsReceiver receiver) { + contentCollected = false; // If the current position is present in just one cursor, we can safely descend multiple levels within // its branch as no one of the other tries has content for it. if (branchHasMultipleSources()) - return advance(); // More than one source at current position, do single-step advance. + return doAdvance(); // More than one source at current position, do single-step advance. // If there are no children, i.e. the cursor ascends, we have to check if it's become larger than some // other candidate. @@ -254,10 +278,10 @@ public int skipTo(int skipDepth, int skipTransition) // We need to advance all cursors that stand before the requested position. // If a child cursor does not need to advance as it is greater than the skip position, neither of the ones // below it in the heap hierarchy do as they can't have an earlier position. - class SkipTo implements AdvancingHeapOp + class SkipTo implements AdvancingHeapOp { @Override - public boolean shouldContinueWithChild(Cursor child, Cursor head) + public boolean shouldContinueWithChild(C child, C head) { // When the requested position descends, the inplicit prefix bytes are those of the head cursor, // and thus we need to check against that if it is a match. @@ -271,12 +295,13 @@ public boolean shouldContinueWithChild(Cursor child, Cursor head) } @Override - public void apply(Cursor cursor) + public void apply(C cursor) { cursor.skipTo(skipDepth, skipTransition); } } + contentCollected = false; applyToSelectedElementsInHeap(new SkipTo(), 0); return maybeSwapHead(head.skipTo(skipDepth, skipTransition)); } @@ -305,15 +330,25 @@ public ByteComparable.Version byteComparableVersion() return head.byteComparableVersion(); } - @Override - public T content() + T maybeCollectContent() { - if (!branchHasMultipleSources()) - return head.content(); + if (!contentCollected) + { + collectedContent = isExhausted() ? null : collectContent(); + contentCollected = true; + } + return collectedContent; + } + T collectContent() + { applyToSelectedInHeap(CollectionMergeCursor::collectContent); collectContent(head, -1); + return resolveContent(); + } + T resolveContent() + { T toReturn; switch (contents.size()) { @@ -331,25 +366,14 @@ public T content() return toReturn; } - private void collectContent(Cursor item, int index) + void collectContent(C item, int index) { - T itemContent = item.content(); + T itemContent = getContent(item); if (itemContent != null) contents.add(itemContent); } - @Override - public Cursor tailCursor(Direction dir) - { - if (!branchHasMultipleSources()) - return head.tailCursor(dir); - - List> inputs = new ArrayList<>(heap.length + 1); - inputs.add(head); - applyToSelectedInHeap((self, cursor, index) -> inputs.add(cursor)); - - return new CollectionMergeCursor<>(resolver, dir, inputs, Cursor::tailCursor); - } + abstract T getContent(C item); /// Compare the positions of two cursors. One is before the other when /// - its depth is greater, or @@ -367,4 +391,91 @@ static boolean equalCursor(Cursor c1, Cursor c2) { return c1.depth() == c2.depth() && c1.incomingTransition() == c2.incomingTransition(); } + + static class Plain extends CollectionMergeCursor> implements Cursor + { + public Plain(Trie.CollectionMergeResolver resolver, Direction direction, Collection inputs, BiFunction> extractor) + { + super(resolver, direction, inputs, Cursor[]::new, extractor); + } + + @Override + public T content() + { + return maybeCollectContent(); + } + + @Override + T getContent(Cursor item) + { + return item.content(); + } + + @Override + public Cursor tailCursor(Direction dir) + { + if (!branchHasMultipleSources()) + return head.tailCursor(dir); + + List> inputs = new ArrayList<>(heap.length + 1); + inputs.add(head); + applyToSelectedInHeap((self, cursor, index) -> inputs.add(cursor)); + + return new Plain<>(resolver, dir, inputs, Cursor::tailCursor); + } + } + + static class Range> extends CollectionMergeCursor> implements RangeCursor + { + Range(Trie.CollectionMergeResolver resolver, + Direction direction, + Collection inputs, + BiFunction> extractor) + { + super(resolver, direction, inputs, RangeCursor[]::new, extractor); + } + + @Override + public S state() + { + return maybeCollectContent(); + } + + @Override + S collectContent() + { + applyToAllOnHeap(CollectionMergeCursor::collectContent); + collectContent(head, -1); + return resolveContent(); + } + + @Override + S getContent(RangeCursor item) + { + return equalCursor(item, head) ? item.state() : item.precedingState(); + } + + @Override + public RangeCursor tailCursor(Direction direction) + { + List> inputs = new ArrayList<>(heap.length); + inputs.add(head); + applyToAllOnHeap((self, cursor, index) -> + { + if (equalCursor(head, cursor)) + inputs.add(cursor); + else if (cursor.precedingState() != null) + inputs.add(cursor.precedingStateCursor(direction)); + }); + + if (inputs.size() == 1) + { + assert head == inputs.get(0); + return head.tailCursor(direction); + } + + return new Range<>(resolver, direction, inputs, RangeCursor::tailCursor); + } + } + } diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java index f079e0a05e62..9e92bc38a32b 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java @@ -960,17 +960,22 @@ class ApplyState implements KeyProducer { int[] data = new int[16 * 5]; int currentDepth = -1; + int ascendLimit = -1; /// Pointer to the existing node before skipping over content nodes, i.e. this is either the same as /// existingPostContentNode or a pointer to a prefix or leaf node whose child is `existingPostContentNode`. - int existingPreContentNode() + int existingFullNode() { return data[currentDepth * 5 + 0]; } - void setExistingPreContentNode(int value) + void setExistingFullNode(int value) { data[currentDepth * 5 + 0] = value; } + int existingFullNodeAtDepth(int stackDepth) + { + return data[stackDepth * 5 + 0]; + } /// Pointer to the existing node being updated, after any content nodes have been skipped and before any /// modification have been applied. Always a non-content node. @@ -982,6 +987,10 @@ void setExistingPostContentNode(int value) { data[currentDepth * 5 + 1] = value; } + int existingPostContentNodeAtDepth(int stackDepth) + { + return data[stackDepth * 5 + 1]; + } /// The updated node, i.e. the node to which the relevant modifications are being applied. This will change as /// children are processed and attached to the node. After all children have been processed, this will contain @@ -1027,60 +1036,99 @@ int contentIdAtDepth(int stackDepth) return data[stackDepth * 5 + 4]; } + int setAscendLimit(int newLimit) + { + int prev = ascendLimit; + ascendLimit = newLimit; + return prev; + } + + ApplyState start() { int existingFullNode = root; - currentDepth = 0; + currentDepth = -1; + ascendLimit = 0; descendInto(existingFullNode); return this; } - /// Returns true if the depth signals mutation cursor is exhausted. + /// Advance to the given depth and transition. Returns false if the depth signals mutation cursor is exhausted. boolean advanceTo(int depth, int transition, int forcedCopyDepth) throws TrieSpaceExhaustedException { - while (currentDepth > Math.max(0, depth - 1)) + while (currentDepth > Math.max(ascendLimit, depth - 1)) { // There are no more children. Ascend to the parent state to continue walk. attachAndMoveToParentState(forcedCopyDepth); } if (depth == -1) - return true; + return false; // We have a transition, get child to descend into descend(transition); - return false; + return true; + } + + /// Advance to an existing position in the trie or the given limit, whichever comes first. + /// If there is an existing position before this limit, the state will be positioned on it, and true will be + /// returned. If not, we will advance and descend into the given limit position, and return false. + /// + /// The `limitDepth` and `limitTransition` parameters specify the limit position. This must be a valid + /// non-exhausted position. + boolean advanceToNextExistingOr(int limitDepth, int limitTransition, int forcedCopyDepth) throws TrieSpaceExhaustedException + { + assert limitDepth > applyState.ascendLimit; + setTransition(-1); // we have newly descended to a node, start with its first child + while (true) + { + int currentTransition = transition(); + int nextTransition = getNextTransition(existingPostContentNode(), currentTransition + 1); + if (currentDepth + 1 == limitDepth && nextTransition >= limitTransition) + { + descend(limitTransition); + return false; + } + if (nextTransition <= 0xFF) + { + descend(nextTransition); + return true; + } + + attachAndMoveToParentState(forcedCopyDepth); + } } /// Descend to a child node. Prepares a new entry in the stack for the node. void descend(int transition) { setTransition(transition); - int existingPreContentNode = getChild(existingPreContentNode(), transition); - ++currentDepth; - descendInto(existingPreContentNode); + int existingFullNode = getChild(existingFullNode(), transition); + + descendInto(existingFullNode); } - private void descendInto(int existingPreContentNode) + private void descendInto(int existingFullNode) { + ++currentDepth; if (currentDepth * 5 >= data.length) data = Arrays.copyOf(data, currentDepth * 5 * 2); - setExistingPreContentNode(existingPreContentNode); + setExistingFullNode(existingFullNode); int existingContentId = NONE; int existingPostContentNode; - if (isLeaf(existingPreContentNode)) + if (isLeaf(existingFullNode)) { - existingContentId = existingPreContentNode; + existingContentId = existingFullNode; existingPostContentNode = NONE; } - else if (offset(existingPreContentNode) == PREFIX_OFFSET) + else if (offset(existingFullNode) == PREFIX_OFFSET) { - existingContentId = getIntVolatile(existingPreContentNode + PREFIX_CONTENT_OFFSET); - existingPostContentNode = followContentTransition(existingPreContentNode); + existingContentId = getIntVolatile(existingFullNode + PREFIX_CONTENT_OFFSET); + existingPostContentNode = followContentTransition(existingFullNode); } else - existingPostContentNode = existingPreContentNode; + existingPostContentNode = existingFullNode; setExistingPostContentNode(existingPostContentNode); setUpdatedPostContentNode(existingPostContentNode); setContentId(existingContentId); @@ -1122,6 +1170,28 @@ else if (forcedCopy) } } + T getNearestContent() + { + // Assume any dead branch is deleted, thus: go upstack until first node for which we have a higher transition + // and then repeatedly descend into first child until content. + int stackPos = currentDepth; + int node = NONE; + setTransition(-1); // In the node we have just descended to, start with its first child + for (; stackPos >= 0 && node == NONE; --stackPos) + { + node = getNextChild(existingPostContentNodeAtDepth(stackPos), transitionAtDepth(stackPos) + 1); + } + + while (node != NONE) + { + T content = InMemoryBaseTrie.this.getNodeContent(node); + if (content != null) + return content; + node = getNextChild(node, 0); + } + return null; + } + /// Attach a child to the current node. private void attachChild(int transition, int child, boolean forcedCopy) throws TrieSpaceExhaustedException { @@ -1147,7 +1217,7 @@ private int applyContent(boolean forcedCopy) throws TrieSpaceExhaustedException // prefix nodes that may reference it. int contentId = contentId(); final int updatedPostContentNode = updatedPostContentNode(); - final int existingPreContentNode = existingPreContentNode(); + final int existingPreContentNode = existingFullNode(); final int existingPostContentNode = existingPostContentNode(); // applyPrefixChange does not understand leaf nodes, handle upgrade from and to one explicitly. @@ -1230,8 +1300,9 @@ else if (prefixWasPresent && !prefixWasEmbedded) void attachAndMoveToParentState(int forcedCopyDepth) throws TrieSpaceExhaustedException { int updatedFullNode = applyContent(currentDepth >= forcedCopyDepth); - int existingFullNode = existingPreContentNode(); + int existingFullNode = existingFullNode(); --currentDepth; + assert currentDepth >= 0; if (updatedFullNode != existingFullNode) attachChild(transition(), updatedFullNode, currentDepth >= forcedCopyDepth); @@ -1240,14 +1311,14 @@ void attachAndMoveToParentState(int forcedCopyDepth) throws TrieSpaceExhaustedEx /// Ascend and update the root at the end of processing. void attachRoot(int forcedCopyDepth) throws TrieSpaceExhaustedException { - int updatedPreContentNode = applyContent(0 >= forcedCopyDepth); - int existingPreContentNode = existingPreContentNode(); - assert root == existingPreContentNode : "Unexpected change to root. Concurrent trie modification?"; - if (updatedPreContentNode != existingPreContentNode) + int updatedFullNode = applyContent(0 >= forcedCopyDepth); + int existingFullNode = existingFullNode(); + assert root == existingFullNode : "Unexpected change to root. Concurrent trie modification?"; + if (updatedFullNode != existingFullNode) { // Only write to root if they are different (value doesn't change, but // we don't want to invalidate the value in other cores' caches unnecessarily). - root = updatedPreContentNode; + root = updatedFullNode; } } @@ -1264,6 +1335,7 @@ public byte[] getBytes() return data; } + @Override public byte[] getBytes(Predicate shouldStop) { if (currentDepth == 0) @@ -1309,6 +1381,7 @@ public String toString() } } + public interface KeyProducer { /// Get the bytes of the path leading to this node. @@ -1390,17 +1463,17 @@ public interface NodeFeatures T content(); } - static class Mutation implements NodeFeatures + static class Mutation> implements NodeFeatures { final UpsertTransformerWithKeyProducer transformer; final Predicate> needsForcedCopy; - final Cursor mutationCursor; + final C mutationCursor; final InMemoryBaseTrie.ApplyState state; int forcedCopyDepth; Mutation(UpsertTransformerWithKeyProducer transformer, Predicate> needsForcedCopy, - Cursor mutationCursor, + C mutationCursor, InMemoryBaseTrie.ApplyState state) { assert mutationCursor.depth() == 0 : "Unexpected non-fresh cursor."; @@ -1409,6 +1482,7 @@ static class Mutation implements NodeFeatures this.needsForcedCopy = needsForcedCopy; this.mutationCursor = mutationCursor; this.state = state; + this.forcedCopyDepth = Integer.MAX_VALUE; } void apply() throws TrieSpaceExhaustedException @@ -1416,13 +1490,13 @@ void apply() throws TrieSpaceExhaustedException int depth = state.currentDepth; while (true) { - if (depth <= forcedCopyDepth) + if (depth < forcedCopyDepth) forcedCopyDepth = needsForcedCopy.test(this) ? depth : Integer.MAX_VALUE; applyContent(); depth = mutationCursor.advance(); - if (state.advanceTo(depth, mutationCursor.incomingTransition(), forcedCopyDepth)) + if (!state.advanceTo(depth, mutationCursor.incomingTransition(), forcedCopyDepth)) break; assert state.currentDepth == depth : "Unexpected change to applyState. Concurrent trie modification?"; } diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryRangeTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryRangeTrie.java new file mode 100644 index 000000000000..fa8e6a79e029 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/InMemoryRangeTrie.java @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.function.Predicate; + +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.utils.ObjectSizes; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.concurrent.OpOrder; + +public class InMemoryRangeTrie> extends InMemoryBaseTrie implements RangeTrie +{ + // constants for space calculations + private static final long EMPTY_SIZE_ON_HEAP; + private static final long EMPTY_SIZE_OFF_HEAP; + static + { + // Measuring the empty size of long-lived tries, because these are the ones for which we want to track size. + InMemoryBaseTrie empty = new InMemoryRangeTrie<>(ByteComparable.Version.OSS50, BufferType.ON_HEAP, ExpectedLifetime.LONG, null); + EMPTY_SIZE_ON_HEAP = ObjectSizes.measureDeep(empty); + empty = new InMemoryRangeTrie<>(ByteComparable.Version.OSS50, BufferType.OFF_HEAP, ExpectedLifetime.LONG, null); + EMPTY_SIZE_OFF_HEAP = ObjectSizes.measureDeep(empty); + } + + InMemoryRangeTrie(ByteComparable.Version byteComparableVersion, BufferType bufferType, ExpectedLifetime lifetime, OpOrder opOrder) + { + super(byteComparableVersion, bufferType, lifetime, opOrder); + } + + public static > InMemoryRangeTrie shortLived(ByteComparable.Version byteComparableVersion) + { + return new InMemoryRangeTrie<>(byteComparableVersion, BufferType.ON_HEAP, ExpectedLifetime.SHORT, null); + } + + public static > InMemoryRangeTrie shortLived(ByteComparable.Version byteComparableVersion, BufferType bufferType) + { + return new InMemoryRangeTrie<>(byteComparableVersion, bufferType, ExpectedLifetime.SHORT, null); + } + + public static > InMemoryRangeTrie longLived(ByteComparable.Version byteComparableVersion, OpOrder opOrder) + { + return longLived(byteComparableVersion, BufferType.OFF_HEAP, opOrder); + } + + public static > InMemoryRangeTrie longLived(ByteComparable.Version byteComparableVersion, BufferType bufferType, OpOrder opOrder) + { + return new InMemoryRangeTrie<>(byteComparableVersion, bufferType, ExpectedLifetime.LONG, opOrder); + } + + public InMemoryRangeCursor makeCursor(Direction direction) + { + return new InMemoryRangeCursor(direction, root, 0, -1); + } + + protected long emptySizeOnHeap() + { + return bufferType == BufferType.ON_HEAP ? EMPTY_SIZE_ON_HEAP : EMPTY_SIZE_OFF_HEAP; + } + + class InMemoryRangeCursor extends InMemoryCursor implements RangeCursor + { + boolean activeIsSet; + S activeRange; // only non-null if activeIsSet + S prevContent; // can only be non-null if activeIsSet + + InMemoryRangeCursor(Direction direction, int root, int depth, int incomingTransition) + { + super(direction, root, depth, incomingTransition); + activeIsSet = true; + activeRange = null; + prevContent = null; + } + + @Override + public int advance() + { + return updateActiveAndReturn(super.advance()); + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + return updateActiveAndReturn(super.advanceMultiple(receiver)); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + activeIsSet = false; // since we are skipping, we have no idea where we will end up + activeRange = null; + prevContent = null; + return updateActiveAndReturn(super.skipTo(skipDepth, skipTransition)); + } + + @Override + public S state() + { + if (!activeIsSet) + setActiveState(); + return activeRange; + } + + private int updateActiveAndReturn(int depth) + { + if (depth >= 0) + { + // Always check if we are seeing new content; if we do, that's an easy state update. + S content = content(); + if (content != null) + { + activeRange = content; + prevContent = content; + activeIsSet = true; + } + else if (prevContent != null) + { + // If the previous state was exact, its right side is what we now have. + activeRange = prevContent.precedingState(direction.opposite()); + prevContent = null; + assert activeIsSet; + } + // otherwise the active state is either not set or still valid. + } + else + { + // exhausted + activeIsSet = true; + activeRange = null; + prevContent = null; + } + return depth; + } + + private void setActiveState() + { + assert content() == null; + S nearestContent = getNearestContent(); + // Note: the nearest content may change between the time we fetch it and when we reach that node, e.g. + // if someone deletes aa-cd where there existed an abc-acd deletion, and we fetched the latter while at "a". + // This, though, should only be possible of the preceding state of the nearest content is null. + activeRange = nearestContent != null ? nearestContent.precedingState(direction) : null; + prevContent = null; + activeIsSet = true; + } + + private S getNearestContent() + { + // Walk a copy of this cursor (non-range because we are only not doing anything smart with it) to find the + // nearest child content in the direction of the cursor. + return new InMemoryCursor(direction, currentNode, 0, -1).advanceToContent(null); + } + + @Override + public InMemoryRangeCursor tailCursor(Direction direction) + { + InMemoryRangeCursor cursor = new InMemoryRangeCursor(direction, currentFullNode, 0, -1); + if (activeIsSet) + { + // Copy the state we have already compiled to the child cursor. + cursor.activeIsSet = true; + cursor.activeRange = activeRange; + } + else + cursor.activeIsSet = false; + + return cursor; + } + } + + static class Mutation, U extends RangeState> extends InMemoryBaseTrie.Mutation> + { + Mutation(UpsertTransformerWithKeyProducer transformer, Predicate> needsForcedCopy, RangeCursor source, InMemoryRangeTrie.ApplyState state) + { + super(transformer, needsForcedCopy, source, state); + } + + @Override + void apply() throws TrieSpaceExhaustedException + { + applyRanges(); + assert state.currentDepth == 0 : "Unexpected change to applyState. Concurrent trie modification?"; + } + + void applyContent(M existingState, U mutationState) throws TrieSpaceExhaustedException + { + M combined = transformer.apply(existingState, mutationState, state); + if (combined != null) + combined = combined.isBoundary() ? combined : null; + state.setContent(combined, // can be null + state.currentDepth >= forcedCopyDepth); // this is called at the start of processing + } + + + void applyRanges() throws TrieSpaceExhaustedException + { + // While activeDeletion is not set, follow the mutation trie. + // When a deletion is found, get existing covering state, combine and apply/store. + // Get rightSideAsCovering and walk the full existing trie to apply, advancing mutation cursor in parallel + // until we see another entry in mutation trie. + // Repeat until mutation trie is exhausted. + int depth = state.currentDepth; + int prevAscendDepth = state.setAscendLimit(depth); + while (true) + { + if (depth < forcedCopyDepth) + forcedCopyDepth = needsForcedCopy.test(this) ? depth : Integer.MAX_VALUE; + + U content = mutationCursor.content(); + if (content != null) + { + final M existingCoveringState = getExistingCoveringState(); + applyContent(existingCoveringState, content); + U mutationCoveringState = content.precedingState(Direction.REVERSE); + // Several cases: + // - New deletion is point deletion: Apply it and move on to next mutation branch. + // - New deletion starts range and there is no existing or it beats the existing: Walk both tries in + // parallel to apply deletion and adjust on any change. + // - New deletion starts range and existing beats it: We still have to walk both tries in parallel, + // because existing deletion may end before the newly introduced one, and we want to apply that when + // it does. + if (mutationCoveringState != null) + applyDeletionRange(rightSideAsCovering(existingCoveringState), mutationCoveringState); + } + + depth = mutationCursor.advance(); + // Descend but do not modify anything yet. + if (!state.advanceTo(depth, mutationCursor.incomingTransition(), forcedCopyDepth)) + break; + assert depth == state.currentDepth : "Unexpected change to applyState. Concurrent trie modification?"; + } + state.setAscendLimit(prevAscendDepth); + } + + void applyDeletionRange(M existingCoveringState, + U mutationCoveringState) + throws TrieSpaceExhaustedException + { + boolean atMutation = true; + int depth = mutationCursor.depth(); + int transition = mutationCursor.incomingTransition(); + // We are walking both tries in parallel. + while (true) + { + if (atMutation) + { + depth = mutationCursor.advance(); + transition = mutationCursor.incomingTransition(); + + assert depth > 0 : "Unbounded range in mutation trie, state " + mutationCoveringState + " active when exhausted."; + if (depth < forcedCopyDepth) + forcedCopyDepth = needsForcedCopy.test(this) ? depth : Integer.MAX_VALUE; + } + atMutation = !state.advanceToNextExistingOr(depth, transition, forcedCopyDepth); + + M existingContent = state.getContent(); + U mutationContent = atMutation ? mutationCursor.content() : null; + if (existingContent != null || mutationContent != null) + { + if (existingContent == null) + existingContent = existingCoveringState; + if (mutationContent == null) + mutationContent = mutationCoveringState; + applyContent(existingContent, mutationContent); + mutationCoveringState = mutationContent.precedingState(Direction.REVERSE); + existingCoveringState = rightSideAsCovering(existingContent); + if (mutationCoveringState == null) + { + assert atMutation; // mutation covering state can only change when mutation content is present + return; // mutation deletion range was closed, we can continue normal mutation cursor iteration + } + } + } + } + + static > S rightSideAsCovering(S rangeState) + { + if (rangeState == null) + return null; + return rangeState.precedingState(Direction.REVERSE); + } + + M getExistingCoveringState() + { + // If the current node has content, use it. + M existingCoveringState = state.getContent(); + if (existingCoveringState != null) + return existingCoveringState; + + // Otherwise, we must have a descendant that will have the active state as its preceding. + existingCoveringState = state.getNearestContent(); + if (existingCoveringState != null) + return existingCoveringState.precedingState(Direction.FORWARD); + + return null; + } + } + + + /// Modify this trie to apply the mutation given in the form of a trie. Any content in the mutation will be resolved + /// with the given function before being placed in this trie (even if there's no pre-existing content in this trie). + /// @param mutation the mutation to be applied, given in the form of a trie. Note that its content can be of type + /// different than the element type for this memtable trie. + /// @param transformer a function applied to the potentially pre-existing value for the given key, and the new + /// value. Applied even if there's no pre-existing value in the memtable trie. + /// @param needsForcedCopy a predicate which decides when to fully copy a branch to provide atomicity guarantees to + /// concurrent readers. See NodeFeatures for details. + public > void apply(RangeTrie mutation, + final UpsertTransformerWithKeyProducer transformer, + Predicate> needsForcedCopy) throws TrieSpaceExhaustedException + { + try + { + Mutation m = new Mutation<>(transformer, + needsForcedCopy, + mutation.cursor(Direction.FORWARD), + applyState.start()); + m.apply(); + m.complete(); + completeMutation(); + } + catch (Throwable t) + { + abortMutation(); + throw t; + } + } + + /// Modify this trie to apply the mutation given in the form of a trie. Any content in the mutation will be resolved + /// with the given function before being placed in this trie (even if there's no pre-existing content in this trie). + /// @param mutation the mutation to be applied, given in the form of a trie. Note that its content can be of type + /// different than the element type for this memtable trie. + /// @param transformer a function applied to the potentially pre-existing value for the given key, and the new + /// value. Applied even if there's no pre-existing value in the memtable trie. + /// @param needsForcedCopy a predicate which decides when to fully copy a branch to provide atomicity guarantees to + /// concurrent readers. See NodeFeatures for details. + public > void apply(RangeTrie mutation, + final UpsertTransformer transformer, + Predicate> needsForcedCopy) throws TrieSpaceExhaustedException + { + apply(mutation, (UpsertTransformerWithKeyProducer) transformer, needsForcedCopy); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java index 04beded9a4a5..c912ca277382 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java @@ -334,6 +334,37 @@ int getChild(int node, int trans) } } + /// Returns first present transition byte in the node that is the same or greater as the given target transition. + int getNextTransition(int node, int trans) + { + if (isNullOrLeaf(node)) + return Integer.MAX_VALUE; + + node = followContentTransition(node); + + if (isNullOrLeaf(node)) + return Integer.MAX_VALUE; + + switch (offset(node)) + { + case SPARSE_OFFSET: + return getSparseNextTransition(node, trans); + case SPLIT_OFFSET: + return getSplitNextTransition(node, trans); + default: + return getChainNextTransition(node, trans); + } + } + + int getNextChild(int node, int targetTransition) + { + int nextTransition = getNextTransition(node, targetTransition); + if (nextTransition <= 0xFF) + return getChild(node, nextTransition); + else + return NONE; + } + protected int followContentTransition(int node) { if (isNullOrLeaf(node)) @@ -405,6 +436,73 @@ int getSparseChild(int node, int trans) return NONE; } + int getSparseNextTransition(int node, int targetTransition) + { + UnsafeBuffer chunk = getBuffer(node); + int inChunkNode = inBufferOffset(node); + int data = chunk.getShortVolatile(inChunkNode + SPARSE_ORDER_OFFSET) & 0xFFFF; + int index; + int transition; + do + { + // Peel off the next index. + index = data % SPARSE_CHILD_COUNT; + data = data / SPARSE_CHILD_COUNT; + transition = chunk.getByte(inChunkNode + SPARSE_BYTES_OFFSET + index) & 0xFF; + } + while (transition < targetTransition && data != 0); + + if (transition < targetTransition) + return Integer.MAX_VALUE; + else + return transition; + } + + int getChainNextTransition(int node, int targetTransition) + { + int transition = getUnsignedByte(node); + if (transition < targetTransition) + return Integer.MAX_VALUE; + else + return transition; + } + + int getSplitNextTransition(int node, int targetTransition) + { + if (targetTransition < 0) + targetTransition = 0; + int midIndex = splitNodeMidIndex(targetTransition); + int tailIndex = splitNodeTailIndex(targetTransition); + int childIndex = splitNodeChildIndex(targetTransition); + while (midIndex < SPLIT_START_LEVEL_LIMIT) + { + int mid = getSplitCellPointer(node, midIndex, SPLIT_START_LEVEL_LIMIT); + if (!isNull(mid)) + { + while (tailIndex < SPLIT_OTHER_LEVEL_LIMIT) + { + int tail = getSplitCellPointer(mid, tailIndex, SPLIT_OTHER_LEVEL_LIMIT); + if (!isNull(tail)) + { + while (childIndex < SPLIT_OTHER_LEVEL_LIMIT) + { + int child = getSplitCellPointer(tail, childIndex, SPLIT_OTHER_LEVEL_LIMIT); + if (!isNull(child)) + return childIndex | (tailIndex << 3) | (midIndex << 6); + ++childIndex; + } + } + childIndex = 0; + ++tailIndex; + } + } + tailIndex = 0; + childIndex = 0; + ++midIndex; + } + return Integer.MAX_VALUE; + } + /// Given a transition, returns the corresponding index (within the node cell) of the pointer to the mid cell of /// a split node. int splitNodeMidIndex(int trans) @@ -475,8 +573,14 @@ private static class CursorBacktrackingState { static final int BACKTRACK_INTS_PER_ENTRY = 3; static final int BACKTRACK_INITIAL_SIZE = 16; - private int[] backtrack = new int[BACKTRACK_INITIAL_SIZE * BACKTRACK_INTS_PER_ENTRY]; - int backtrackDepth = 0; + private int[] backtrack; + int backtrackDepth; + + CursorBacktrackingState() + { + backtrack = new int[BACKTRACK_INITIAL_SIZE * BACKTRACK_INTS_PER_ENTRY]; + backtrackDepth = 0; + } void addBacktrack(int node, int data, int depth) { @@ -519,17 +623,12 @@ int depth(int backtrackDepth) /// have a remaining child to advance to. When there's nothing to backtrack to, the trie is exhausted. class InMemoryCursor extends CursorBacktrackingState implements Cursor { - private int currentNode; - private int currentFullNode; - private int incomingTransition; - private T content; - private final Direction direction; + int currentNode; + int currentFullNode; + int incomingTransition; int depth; - - InMemoryCursor(Direction direction) - { - this(direction, root, 0, -1); - } + T content; + final Direction direction; InMemoryCursor(Direction direction, int root, int depth, int incomingTransition) { @@ -540,6 +639,11 @@ class InMemoryCursor extends CursorBacktrackingState implements Cursor @Override public int advance() + { + return doAdvance(); + } + + int doAdvance() { if (isNullOrLeaf(currentNode)) return backtrack(); @@ -552,7 +656,7 @@ public int advanceMultiple(TransitionsReceiver receiver) { int node = currentNode; if (!isChainNode(node)) - return advance(); + return doAdvance(); // Jump directly to the chain's child. UnsafeBuffer buffer = getBuffer(node); diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java index 9244622e4923..e0642533f145 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java @@ -120,10 +120,10 @@ public void apply(Trie mutation, { try { - Mutation m = new Mutation<>(transformer, - needsForcedCopy, - mutation.cursor(Direction.FORWARD), - applyState.start()); + Mutation> m = new Mutation<>(transformer, + needsForcedCopy, + mutation.cursor(Direction.FORWARD), + applyState.start()); m.apply(); m.complete(); completeMutation(); @@ -168,9 +168,7 @@ public void putSingleton(ByteComparable key, apply(Trie.singleton(key, byteComparableVersion, value), transformer, Predicates.alwaysFalse()); } - /** - * A version of putSingleton which uses recursive put if the last argument is true. - */ + /// A version of putSingleton which uses recursive put if the last argument is true. public void putSingleton(ByteComparable key, R value, UpsertTransformer transformer, diff --git a/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java b/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java index 3e118a4ded6c..147c10da6510 100644 --- a/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java +++ b/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java @@ -176,12 +176,6 @@ private int exhausted() return -1; } - @Override - public T content() - { - return source.content(); - } - @Override public Direction direction() { @@ -202,6 +196,12 @@ public Plain(Cursor source, TrieSetCursor set) super(source, set); } + @Override + public T content() + { + return source.content(); + } + @Override public Cursor tailCursor(Direction direction) { diff --git a/src/java/org/apache/cassandra/db/tries/MergeCursor.java b/src/java/org/apache/cassandra/db/tries/MergeCursor.java index d39270872274..f3dd0ff56c41 100644 --- a/src/java/org/apache/cassandra/db/tries/MergeCursor.java +++ b/src/java/org/apache/cassandra/db/tries/MergeCursor.java @@ -28,20 +28,21 @@ /// Crucial for the efficiency of this is the fact that when they are advanced like this, we can compare cursors' /// positions by their `depth` descending and then `incomingTransition` ascending. /// See [Trie.md](./Trie.md) for further details. -class MergeCursor implements Cursor +abstract class MergeCursor> implements Cursor { - private final Trie.MergeResolver resolver; - private final Direction direction; - private final Cursor c1; - private final Cursor c2; + final Direction direction; + final Trie.MergeResolver resolver; + + final C c1; + final C c2; boolean atC1; boolean atC2; - MergeCursor(Trie.MergeResolver resolver, Cursor c1, Cursor c2) + MergeCursor(Trie.MergeResolver resolver, C c1, C c2) { - this.resolver = resolver; this.direction = c1.direction(); + this.resolver = resolver; this.c1 = c1; this.c2 = c2; assert c1.depth() == 0; @@ -87,7 +88,7 @@ public int advanceMultiple(TransitionsReceiver receiver) return checkOrder(c1.depth(), c2.advanceMultiple(receiver)); } - private int checkOrder(int c1depth, int c2depth) + int checkOrder(int c1depth, int c2depth) { if (c1depth > c2depth) { @@ -132,33 +133,106 @@ public Direction direction() public ByteComparable.Version byteComparableVersion() { assert c1.byteComparableVersion() == c2.byteComparableVersion() : - "Merging cursors with different byteComparableVersions: " + - c1.byteComparableVersion() + " vs " + c2.byteComparableVersion(); + "Merging cursors with different byteComparableVersions: " + + c1.byteComparableVersion() + " vs " + c2.byteComparableVersion(); return c1.byteComparableVersion(); } - public T content() + /// Merge implementation for [Trie] + static class Plain extends MergeCursor> { - T mc = atC2 ? c2.content() : null; - T nc = atC1 ? c1.content() : null; - if (mc == null) - return nc; - else if (nc == null) - return mc; - else - return resolver.resolve(nc, mc); + Plain(Trie.MergeResolver resolver, Cursor c1, Cursor c2) + { + super(resolver, c1, c2); + } + + @Override + public T content() + { + T mc = atC2 ? c2.content() : null; + T nc = atC1 ? c1.content() : null; + if (mc == null) + return nc; + else if (nc == null) + return mc; + else + return resolver.resolve(nc, mc); + } + + @Override + public Cursor tailCursor(Direction direction) + { + if (atC1 && atC2) + return new Plain<>(resolver, c1.tailCursor(direction), c2.tailCursor(direction)); + else if (atC1) + return c1.tailCursor(direction); + else if (atC2) + return c2.tailCursor(direction); + else + throw new AssertionError(); + } } - @Override - public Cursor tailCursor(Direction dir) + /// Merge implementation for [RangeTrie] + static class Range> extends MergeCursor> implements RangeCursor { - if (atC1 && atC2) - return new MergeCursor<>(resolver, c1.tailCursor(dir), c2.tailCursor(dir)); - else if (atC1) - return c1.tailCursor(dir); - else if (atC2) - return c2.tailCursor(dir); - else - throw new AssertionError(); + private S state; + boolean stateCollected; + + Range(Trie.MergeResolver resolver, RangeCursor c1, RangeCursor c2) + { + super(resolver, c1, c2); + } + + @Override + public S state() + { + if (!stateCollected) + { + S state1 = atC1 ? c1.state() : c1.precedingState(); + S state2 = atC2 ? c2.state() : c2.precedingState(); + if (state1 == null) + return state2; + if (state2 == null) + return state1; + state = resolver.resolve(state1, state2); + stateCollected = true; + } + return state; + } + + @Override + public int advance() + { + stateCollected = false; + return super.advance(); + } + + @Override + public int skipTo(int depth, int incomingTransition) + { + stateCollected = false; + return super.skipTo(depth, incomingTransition); + } + + @Override + public int advanceMultiple(Cursor.TransitionsReceiver receiver) + { + stateCollected = false; + return super.advanceMultiple(receiver); + } + + @Override + public RangeCursor tailCursor(Direction direction) + { + if (atC1 && atC2) + return new Range<>(resolver, c1.tailCursor(direction), c2.tailCursor(direction)); + else if (atC1) + return new Range<>(resolver, c1.tailCursor(direction), c2.precedingStateCursor(direction)); + else if (atC2) + return new Range<>(resolver, c1.precedingStateCursor(direction), c2.tailCursor(direction)); + else + throw new AssertionError(); + } } } diff --git a/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java b/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java index a46f07988778..ce810e022477 100644 --- a/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java +++ b/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java @@ -22,25 +22,25 @@ import org.apache.cassandra.utils.bytecomparable.ByteSource; /// Prefixed cursor. Prepends the given prefix to all keys of the supplied cursor. -class PrefixedCursor implements Cursor +abstract class PrefixedCursor> implements Cursor { - final Cursor tail; + final C tail; ByteSource prefixBytes; int nextPrefixByte; int incomingTransition; int depthOfPrefix; - PrefixedCursor(ByteComparable prefix, Cursor tail) + PrefixedCursor(ByteComparable prefix, C tail) { this(prefix.asComparableBytes(tail.byteComparableVersion()), tail); } - PrefixedCursor(ByteSource prefix, Cursor tail) + PrefixedCursor(ByteSource prefix, C tail) { this(prefix.next(), prefix, tail); } - PrefixedCursor(int firstPrefixByte, ByteSource prefix, Cursor tail) + PrefixedCursor(int firstPrefixByte, ByteSource prefix, C tail) { this.tail = tail; prefixBytes = prefix; @@ -96,6 +96,10 @@ public int advanceMultiple(TransitionsReceiver receiver) if (prefixDone()) return completeAdvanceInTail(tail.advanceMultiple(receiver)); + incomingTransition = nextPrefixByte; + nextPrefixByte = prefixBytes.next(); + ++depthOfPrefix; + while (!prefixDone()) { receiver.addPathByte(incomingTransition); @@ -152,15 +156,61 @@ ByteSource.Duplicatable duplicateSource() return duplicatableSource.duplicate(); } - @Override - public Cursor tailCursor(Direction direction) + static class Plain extends PrefixedCursor> implements Cursor { - if (prefixDone()) - return tail.tailCursor(direction); - else + Plain(ByteComparable prefix, Cursor tail) + { + super(prefix, tail); + } + + Plain(int firstPrefixByte, ByteSource prefix, Cursor source) + { + super(firstPrefixByte, prefix, source); + } + + @Override + public Cursor tailCursor(Direction direction) + { + if (prefixDone()) + return tail.tailCursor(direction); + else + { + assert depthOfPrefix >= 0 : "tailTrie called on exhausted cursor"; + return new Plain<>(nextPrefixByte, duplicateSource(), tail.tailCursor(direction)); + } + } + } + + static class Range> extends PrefixedCursor> implements RangeCursor + { + Range(ByteComparable prefix, RangeCursor tail) + { + super(prefix, tail); + } + + Range(int firstPrefixByte, ByteSource prefix, RangeCursor source) + { + super(firstPrefixByte, prefix, source); + } + + @Override + public S state() + { + if (prefixDone() && tail.depth() >= 0) + return tail.state(); + return null; + } + + @Override + public RangeCursor tailCursor(Direction direction) { - assert depthOfPrefix >= 0 : "tailTrie called on exhausted cursor"; - return new PrefixedCursor<>(nextPrefixByte, duplicateSource(), tail.tailCursor(direction)); + if (prefixDone()) + return tail.tailCursor(direction); + else + { + assert depthOfPrefix >= 0 : "tailTrie called on exhausted cursor"; + return new Range<>(nextPrefixByte, duplicateSource(), tail.tailCursor(direction)); + } } } } diff --git a/src/java/org/apache/cassandra/db/tries/RangeCursor.java b/src/java/org/apache/cassandra/db/tries/RangeCursor.java new file mode 100644 index 000000000000..ef85ad0b1a20 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/RangeCursor.java @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// The cursor implementation of [RangeTrie]. +/// +/// The main difference between normal and range cursors is the addition of a [#precedingState] method, which returns a +/// range for prefixes of start or end positions and is used to determine whether a position that has been skipped to +/// falls inside one the trie's ranges. +/// +/// As an example, consider the following range trie: +/// +/// ``` +/// a -> +/// b -> start of del1 (left: null, right: del1) +/// c -> +/// d -> switch from del1 to del2 (left: del1, right: del2) +/// g -> +/// h -> end,del2 (left: del2, right: null) +/// ``` +/// +/// If we advance through this trie, it is easy to keep track of the covering deletion, as we walk through the +/// boundaries before entering the range. However, if we skip into the trie, we will not see the boundaries. +/// Imagine a cursor starts at the root and attempts to skip to "b". We need to be able to notice that "b" is covered by +/// the "ab-cd" range with deletion del1. This is achieved by using [#precedingState]. In this case skipping to "b" +/// (in forward direction) will position the cursor on "c"; because the positions to the left of "c" are covered by +/// del1, the [#precedingState] the cursor should report must be a covering non-boundary state corresponding to `del1`. +/// +/// If, on the other hand, we perform a skip in the reverse direction that reaches the same state, the cursor should +/// report `null` as its state (e.g. performing a `skipTo` from the root with character "d" will land in "c" whose state +/// correctly determines that there is no covering deletion for "d"). +/// +/// For further details, see the range trie section in [Trie.md](./Trie.md). +interface RangeCursor> extends Cursor +{ + /// Returns a range that covers positions before this in iteration order, including this position if `content()` is + /// null. This is the range that is active at (i.e. covers) a position that was skipped to, when the range trie + /// jumps past the requested position or does not have content. + /// + /// The returned value must be a non-boundary state (i.e. `precedingState().isBoundary()` must always be `false`) + /// and must return itself for its `precedingState` in both directions. + default S precedingState() + { + final S state = state(); + if (state == null) + return null; + return state.precedingState(direction()); + } + + /// The range state at the current position. This is either a reportable marker (if the cursor is positioned at a + /// range boundary), or the covering state that applies to this position and the ones preceding it (up to the + /// closest range marker preceding the current position). This carries information for both [#content] and + /// [#precedingState] and is used to reduce the amount of work done to obtain both values. + /// + /// Typically, we would have `state() == content() != null ? content() : precedingState()`, but the state could be + /// richer for some types of trie; for example, [TrieSetCursor] also includes information about the state following + /// in the states returned for prefix positions. + /// + /// This can be null when no range is active before the current position. + S state(); + + /// Content is only returned for boundary positions. + /// Note that if `content()` is non-null, `precedingState()` does not apply to this exact position. + @Override + default S content() + { + final S state = state(); + if (state == null) + return null; + return state.isBoundary() ? state : null; + } + + @Override + RangeCursor tailCursor(Direction direction); + + /// Corresponding method to tailCursor above applicable when this cursor is ahead. + /// Returns a full-range cursor returning [#precedingState()]. + default RangeCursor precedingStateCursor(Direction direction) + { + return new Empty<>(precedingState(), byteComparableVersion(), direction); + } + + class Empty> extends Cursor.Empty implements RangeCursor + { + final S coveringState; + + public Empty(S coveringState, ByteComparable.Version version, Direction direction) + { + super(direction, version); + this.coveringState = coveringState; + } + + @Override + public S state() + { + return coveringState; + } + + @Override + public S content() + { + return null; + } + + @Override + public RangeCursor tailCursor(Direction direction) + { + return new RangeCursor.Empty<>(coveringState, byteComparableVersion(), direction); + } + } + + static > RangeCursor empty(Direction direction, ByteComparable.Version version) + { + return new Empty(null, version, direction); + } + + class FromSet> implements RangeCursor + { + final TrieSetCursor source; + final S marker; + + FromSet(TrieSetCursor source, S marker) + { + this.source = source; + this.marker = marker; + } + + @Override + public S state() + { + return source.state().applyToCoveringState(marker, direction()); + } + + @Override + public int depth() + { + return source.depth(); + } + + @Override + public int incomingTransition() + { + return source.incomingTransition(); + } + + @Override + public Direction direction() + { + return source.direction(); + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return source.byteComparableVersion(); + } + + @Override + public int advance() + { + return source.advance(); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + return source.skipTo(skipDepth, skipTransition); + } + + @Override + public RangeCursor tailCursor(Direction direction) + { + return new FromSet<>(source.tailCursor(direction), marker); + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/RangeIntersectionCursor.java b/src/java/org/apache/cassandra/db/tries/RangeIntersectionCursor.java new file mode 100644 index 000000000000..7f847922c4f2 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/RangeIntersectionCursor.java @@ -0,0 +1,301 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +class RangeIntersectionCursor> implements RangeCursor +{ + enum State + { + MATCHING, + SET_AHEAD, + SOURCE_AHEAD; + } + + final Direction direction; + final RangeCursor src; + final TrieSetCursor set; + int currentDepth; + int currentTransition; + S currentState; + State state; + + public RangeIntersectionCursor(RangeCursor src, TrieSetCursor set) + { + this.direction = src.direction(); + this.set = set; + this.src = src; + matchingPosition(set.depth(), set.incomingTransition()); + } + + @Override + public int depth() + { + return currentDepth; + } + + @Override + public int incomingTransition() + { + return currentTransition; + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return set.byteComparableVersion(); + } + + @Override + public S state() + { + return currentState; + } + + @Override + public int advance() + { + switch(state) + { + case MATCHING: + { + int ldepth = set.advance(); + if (set.precedingIncluded()) + return advanceWithSetAhead(src.advance()); + else + return advanceSourceToIntersection(ldepth); + } + case SET_AHEAD: + return advanceWithSetAhead(src.advance()); + case SOURCE_AHEAD: + return advanceWithSourceAhead(set.advance()); + default: + throw new AssertionError(); + } + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + switch(state) + { + case MATCHING: + return skipBoth(skipDepth, skipTransition); + case SET_AHEAD: + { + // if the cursor ahead is at the skip point or beyond, we can advance the other cursor to the skip point + int setDepth = set.depth(); + if (setDepth < skipDepth || setDepth == skipDepth && direction.ge(set.incomingTransition(), skipTransition)) + return advanceWithSetAhead(src.skipTo(skipDepth, skipTransition)); + // otherwise we must perform a full advance + return skipBoth(skipDepth, skipTransition); + } + case SOURCE_AHEAD: + { + // if the cursor ahead is at the skip point or beyond, we can advance the other cursor to the skip point + int sourceDepth = src.depth(); + if (sourceDepth < skipDepth || sourceDepth == skipDepth && direction.ge(src.incomingTransition(), skipTransition)) + return advanceWithSourceAhead(set.skipTo(skipDepth, skipTransition)); + // otherwise we must perform a full advance + return skipBoth(skipDepth, skipTransition); + } + default: + throw new AssertionError(); + } + } + + private int skipBoth(int skipDepth, int skipTransition) + { + int ldepth = set.skipTo(skipDepth, skipTransition); + if (set.precedingIncluded()) + return advanceWithSetAhead(src.skipTo(skipDepth, skipTransition)); + else + return advanceSourceToIntersection(ldepth); + } + + @Override + public int advanceMultiple(Cursor.TransitionsReceiver receiver) + { + switch(state) + { + case MATCHING: + { + // Cannot do multi-advance when cursors are at the same position. Applying advance(). + int ldepth = set.advance(); + if (set.precedingIncluded()) + return advanceWithSetAhead(src.advance()); + else + return advanceSourceToIntersection(ldepth); + } + case SET_AHEAD: + return advanceWithSetAhead(src.advanceMultiple(receiver)); + case SOURCE_AHEAD: + return advanceWithSourceAhead(set.advanceMultiple(receiver)); + default: + throw new AssertionError(); + } + } + + private int advanceWithSetAhead(int sourceDepth) + { + int sourceTransition = src.incomingTransition(); + int setDepth = set.depth(); + int setTransition = set.incomingTransition(); + if (sourceDepth > setDepth) + return coveredAreaWithSetAhead(sourceDepth, sourceTransition); + if (sourceDepth == setDepth) + { + if (direction.lt(sourceTransition, setTransition)) + return coveredAreaWithSetAhead(sourceDepth, sourceTransition); + if (sourceTransition == setTransition) + return matchingPosition(sourceDepth, sourceTransition); + } + + // Advancing cursor moved beyond the ahead cursor. Check if roles have reversed. + if (src.precedingState() != null) + return coveredAreaWithSourceAhead(setDepth, setTransition); + else + return advanceSetToIntersection(sourceDepth); + } + + private int advanceWithSourceAhead(int setDepth) + { + int setTransition = set.incomingTransition(); + int sourceDepth = src.depth(); + int sourceTransition = src.incomingTransition(); + if (setDepth > sourceDepth) + return coveredAreaWithSourceAhead(setDepth, setTransition); + if (setDepth == sourceDepth) + { + if (direction.lt(setTransition, sourceTransition)) + return coveredAreaWithSourceAhead(setDepth, setTransition); + if (setTransition == sourceTransition) + return matchingPosition(setDepth, setTransition); + } + + // Advancing cursor moved beyond the ahead cursor. Check if roles have reversed. + if (set.precedingIncluded()) + return coveredAreaWithSetAhead(sourceDepth, sourceTransition); + else + return advanceSourceToIntersection(setDepth); + } + + private int advanceSourceToIntersection(int setDepth) + { + int setTransition = set.incomingTransition(); + while (true) + { + // Set is ahead of source, but outside the covered area. Skip source to set's position. + int sourceDepth = src.skipTo(setDepth, setTransition); + int sourceTransition = src.incomingTransition(); + if (sourceDepth == setDepth && sourceTransition == setTransition) + return matchingPosition(setDepth, setTransition); + if (src.precedingState() != null) + return coveredAreaWithSourceAhead(setDepth, setTransition); + + // Source is ahead of set, but outside the covered area. Skip set to source's position. + setDepth = set.skipTo(sourceDepth, sourceTransition); + setTransition = set.incomingTransition(); + if (setDepth == sourceDepth && setTransition == sourceTransition) + return matchingPosition(sourceDepth, sourceTransition); + if (set.precedingIncluded()) + return coveredAreaWithSetAhead(sourceDepth, sourceTransition); + } + } + + private int advanceSetToIntersection(int sourceDepth) + { + int sourceTransition = src.incomingTransition(); + while (true) + { + // Source is ahead of set, but outside the covered area. Skip set to source's position. + int setDepth = set.skipTo(sourceDepth, sourceTransition); + int setTransition = set.incomingTransition(); + if (setDepth == sourceDepth && setTransition == sourceTransition) + return matchingPosition(sourceDepth, sourceTransition); + if (set.precedingIncluded()) + return coveredAreaWithSetAhead(sourceDepth, sourceTransition); + + // Set is ahead of source, but outside the covered area. Skip source to set's position. + sourceDepth = src.skipTo(setDepth, setTransition); + sourceTransition = src.incomingTransition(); + if (sourceDepth == setDepth && sourceTransition == setTransition) + return matchingPosition(setDepth, setTransition); + if (src.precedingState() != null) + return coveredAreaWithSourceAhead(setDepth, setTransition); + } + } + + private int coveredAreaWithSetAhead(int depth, int transition) + { + return setState(State.SET_AHEAD, depth, transition, src.state()); + } + + private int coveredAreaWithSourceAhead(int depth, int transition) + { + return setState(State.SOURCE_AHEAD, depth, transition, restrict(src.precedingState(), set.state())); + } + + private int matchingPosition(int depth, int transition) + { + return setState(State.MATCHING, depth, transition, restrict(src.state(), set.state())); + } + + private S restrict(S srcState, TrieSetCursor.RangeState setState) + { + if (srcState == null) + return null; + if (srcState.isBoundary()) + return srcState.restrict(setState.applicableBefore, setState.applicableAfter); + + return setState.applyToCoveringState(srcState, direction()); + } + + private int setState(State state, int depth, int transition, S cursorState) + { + this.state = state; + this.currentDepth = depth; + this.currentTransition = transition; + this.currentState = cursorState; + return depth; + } + + @Override + public RangeCursor tailCursor(Direction direction) + { + switch (state) + { + case MATCHING: + return new RangeIntersectionCursor<>(src.tailCursor(direction), set.tailCursor(direction)); + case SET_AHEAD: + return src.tailCursor(direction); + case SOURCE_AHEAD: + return new RangeIntersectionCursor<>(src.precedingStateCursor(direction), set.tailCursor(direction)); + default: + throw new AssertionError(); + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/RangeState.java b/src/java/org/apache/cassandra/db/tries/RangeState.java new file mode 100644 index 000000000000..b94dc9a8eeda --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/RangeState.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +/// A range state interface used for range tries. +/// +/// This interface combines two logical concepts: +/// - A range marker / boundary point, which is a point in the trie that or starts or ends a range, or switches between +/// ranges. Such markers are the content of a range trie and by themselves are sufficient to define and recreate the +/// trie. +/// +/// Markers return `true` for [#isBoundary()] and usually return different values for [#precedingState] in the two +/// directions (which can be `null` if no range applies). It is also possible for a marker to specify a point of +/// coverage, in which the preceding state is the same in both directions. +/// +/// - A covering range state, which describes the range that applies to an iteration position which is inside a covered +/// range. These are necessary to be able to efficiently jump inside range tries, for example when constructing the +/// intersection between a range trie and a set trie to answer a query. When a cursor skips over a requested point, +/// the preceding state that the cursor returns is a covering state, which describes the range that applies to (i.e. +/// covers) the position the user requested. +/// +/// Covering states return `false` for [#isBoundary()] and must return themselves for [#precedingState] in both +/// directions. +/// +/// Using this combination instead of separate concepts simplifies and improves the performance of the implementation. +public interface RangeState> +{ + /// True if this is a boundary point. Boundary points are reported by `content()` and usually apply a different + /// state before and after the point (i.e. `precedingState(FORWARD) != precedingState(REVERSE)`). + boolean isBoundary(); + + /// Returns the state that applies to the positions preceding this marker in the given iteration order, if any. + /// + /// This must always be a non-boundary state (i.e. [#isBoundary()] must be `false` and the forward and reverse + /// preceding states are equal to itself). + S precedingState(Direction direction); + + /// Assuming this is a boundary, returns an intersected version of this state, which may drop parts of a marker that + /// are not covered by the intersecting range. + S restrict(boolean applicableBefore, boolean applicableAfter); + + /// Assuming this is a covering state, promote it to a boundary active in the specified direction. + S asBoundary(Direction direction); +} diff --git a/src/java/org/apache/cassandra/db/tries/RangeTrie.java b/src/java/org/apache/cassandra/db/tries/RangeTrie.java new file mode 100644 index 000000000000..e33b959d81dc --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/RangeTrie.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.tries; + +import java.util.Collection; +import java.util.Iterator; + +import com.google.common.base.Preconditions; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +/// Range trie. This type of trie represents information associated with ranges of keys. Its primary application is to +/// support deletions/tombstones. +/// +/// Range tries use [RangeState]s for their content, but they also report a [RangeCursor#state] for any prefix point, +/// so that skips inside the trie can figure out if the new position is covered by some of the trie's ranges. +/// +/// For simplicity and efficiency, range boundaries must be distinct from the keys they cover, and also distinct from +/// any prefix of a valid data key. Should a specific branch need to be covered, a range of boundaries that fall before +/// and after that key should be used -- this is easy to do with the prefix-free [ByteComparable] system we have in +/// place. +/// +/// See [RangeCursor] and [Trie.md](./Trie.md) for further details on the implementation of range tries. +public interface RangeTrie> extends BaseTrie, RangeTrie> +{ + default R process(Direction direction, Cursor.Walker walker) + { + return cursor(direction).process(walker); + } + + /// Returns a singleton trie mapping the given byte path to a marker. + /// + /// Note: Ranges are not meant to use boundaries that are distinct from data and thus a singleton range would list + /// only a boundary and always be empty in terms of covered content. However, we do want to be able to place + /// metadata in intermediate nodes of the trie and this method makes that possible. + static > RangeTrie singleton(ByteComparable key, ByteComparable.Version byteComparableVersion, S v) + { + Preconditions.checkArgument(v.isBoundary()); + Preconditions.checkArgument(v.precedingState(Direction.FORWARD) == null); + Preconditions.checkArgument(v.precedingState(Direction.REVERSE) == null); + return dir -> new SingletonCursor.Range<>(dir, key.asComparableBytes(byteComparableVersion), byteComparableVersion, v); + } + + + /// Returns a range trie covering a single range. This performs the same process as intersecting a covered + /// range by a set, converting the passed marker to the proper state depending on the set's coverage and boundaries. + /// To this end, the passed marker must be a covering state (i.e. it must not be reportable, and must have the same + /// forward and reverse `precedingState`). + static > RangeTrie range(ByteComparable left, ByteComparable right, ByteComparable.Version byteComparableVersion, S v) + { + return fromSet(TrieSet.range(byteComparableVersion, left, right), v); + } + + /// Returns a range trie covering the given set. This performs the same process as intersecting a covered + /// range by a set, converting the passed marker to the proper state depending on the set's coverage and boundaries. + /// To this end, the passed marker must be a covering state (i.e. it must not be boundary, and must return itself + /// as the forward and reverse `precedingState`). + static > RangeTrie fromSet(TrieSet set, S v) + { + Preconditions.checkArgument(!v.isBoundary()); + Preconditions.checkArgument(v.precedingState(Direction.FORWARD) == v); + Preconditions.checkArgument(v.precedingState(Direction.REVERSE) == v); + return dir -> new RangeCursor.FromSet<>(set.cursor(dir), v); + } + + /// Returns the state that applies to the given key. This is either the precise state at the given position, or + /// the range that covers it (i.e. the `precedingState` of the next marker). + default S applicableRange(ByteComparable key) + { + RangeCursor cursor = cursor(Direction.FORWARD); + final ByteSource bytes = key.asComparableBytes(cursor.byteComparableVersion()); + if (cursor.descendAlong(bytes)) + return cursor.state(); + else + return cursor.precedingState(); + } + + @Override + default RangeTrie intersect(TrieSet set) + { + return dir -> new RangeIntersectionCursor<>(cursor(dir), set.cursor(dir)); + } + + /// Constructs a view of the merge of this trie with the given one. The view is live, i.e. any write to any of the + /// sources will be reflected in the merged view. + /// + /// If there is content for a given key in both sources, the resolver will be called to obtain the combination. + /// (The resolver will not be called if there's content from only one source.) + default RangeTrie mergeWith(RangeTrie other, Trie.MergeResolver resolver) + { + return dir -> new MergeCursor.Range<>(resolver, cursor(dir), other.cursor(dir)); + } + + /// Constructs a view of the merge of multiple tries. The view is live, i.e. any write to any of the + /// sources will be reflected in the merged view. + /// + /// If there is content for a given key in more than one sources, the resolver will be called to obtain the + /// combination. (The resolver will not be called if there's content from only one source.) + static > RangeTrie merge(Collection> sources, Trie.CollectionMergeResolver resolver) + { + switch (sources.size()) + { + case 0: + throw new AssertionError(); + case 1: + return sources.iterator().next(); + case 2: + { + Iterator> it = sources.iterator(); + RangeTrie t1 = it.next(); + RangeTrie t2 = it.next(); + return t1.mergeWith(t2, resolver); + } + default: + return dir -> new CollectionMergeCursor.Range<>(resolver, dir, sources, RangeTrie::cursor); + } + } + + @SuppressWarnings("unchecked") + static > RangeTrie empty(ByteComparable.Version version) + { + return dir -> RangeCursor.empty(dir, version); + } + + @Override + default RangeTrie prefixedBy(ByteComparable prefix) + { + return dir -> new PrefixedCursor.Range<>(prefix, cursor(dir)); + } + + @Override + default RangeTrie tailTrie(ByteComparable prefix) + { + RangeCursor c = cursor(Direction.FORWARD); + if (c.descendAlong(prefix.asComparableBytes(c.byteComparableVersion()))) + return c::tailCursor; + else if (c.precedingState() != null) + return c::precedingStateCursor; + else + return null; + } + + RangeCursor makeCursor(Direction direction); + + @Override + default RangeCursor cursor(Direction direction) + { + return Trie.DEBUG ? new VerificationCursor.Range<>(makeCursor(direction)) + : makeCursor(direction); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/RangesCursor.java b/src/java/org/apache/cassandra/db/tries/RangesCursor.java index 8c59b2b6ff84..53ab241f988a 100644 --- a/src/java/org/apache/cassandra/db/tries/RangesCursor.java +++ b/src/java/org/apache/cassandra/db/tries/RangesCursor.java @@ -19,6 +19,7 @@ package org.apache.cassandra.db.tries; import java.util.Arrays; + import org.apache.cassandra.utils.bytecomparable.ByteComparable; import org.apache.cassandra.utils.bytecomparable.ByteSource; diff --git a/src/java/org/apache/cassandra/db/tries/SingletonCursor.java b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java index 3f09054373a5..be7fc910bf21 100644 --- a/src/java/org/apache/cassandra/db/tries/SingletonCursor.java +++ b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java @@ -25,9 +25,9 @@ class SingletonCursor implements Cursor { private final Direction direction; - private ByteSource src; - private final ByteComparable.Version byteComparableVersion; - private final T value; + ByteSource src; + final ByteComparable.Version byteComparableVersion; + final T value; private int currentDepth = 0; private int currentTransition = -1; protected int nextTransition; @@ -110,10 +110,15 @@ public int depth() return currentDepth; } + protected boolean atEnd() + { + return nextTransition == ByteSource.END_OF_STREAM && currentDepth >= 0; + } + @Override public T content() { - return nextTransition == ByteSource.END_OF_STREAM ? value : null; + return atEnd() ? value : null; } @Override @@ -147,4 +152,35 @@ ByteSource.Duplicatable duplicateSource() ByteSource.Duplicatable duplicatableSource = (ByteSource.Duplicatable) src; return duplicatableSource.duplicate(); } + + static class Range> extends SingletonCursor implements RangeCursor + { + public Range(Direction direction, ByteSource src, ByteComparable.Version byteComparableVersion, S value) + { + super(direction, src, byteComparableVersion, value); + } + + public Range(Direction direction, int firstByte, ByteSource src, ByteComparable.Version byteComparableVersion, S value) + { + super(direction, firstByte, src, byteComparableVersion, value); + } + + @Override + public S precedingState() + { + return null; + } + + @Override + public S state() + { + return content(); + } + + @Override + public Range tailCursor(Direction dir) + { + return new Range<>(dir, nextTransition, duplicateSource(), byteComparableVersion, value); + } + } } diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java index af1ca0b336fa..cb7ab14138c7 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.java +++ b/src/java/org/apache/cassandra/db/tries/Trie.java @@ -91,7 +91,7 @@ interface MergeResolver /// (The resolver will not be called if there's content from only one source.) default Trie mergeWith(Trie other, MergeResolver resolver) { - return dir -> new MergeCursor<>(resolver, this.cursor(dir), other.cursor(dir)); + return dir -> new MergeCursor.Plain<>(resolver, this.cursor(dir), other.cursor(dir)); } /// Resolver of content of merged nodes. @@ -133,7 +133,7 @@ static Trie merge(Collection> sources, CollectionMergeR return t1.mergeWith(t2, resolver); } default: - return dir -> new CollectionMergeCursor<>(resolver, dir, sources, Trie::cursor); + return dir -> new CollectionMergeCursor.Plain<>(resolver, dir, sources, Trie::cursor); } } @@ -171,7 +171,7 @@ static Trie mergeDistinct(Trie t1, Trie t2) @Override public Cursor makeCursor(Direction direction) { - return new MergeCursor<>(throwingResolver(), t1.cursor(direction), t2.cursor(direction)); + return new MergeCursor.Plain<>(throwingResolver(), t1.cursor(direction), t2.cursor(direction)); } @Override @@ -213,7 +213,7 @@ private static Trie mergeDistinctTrie(Collection> sourc @Override public Cursor makeCursor(Direction direction) { - return new CollectionMergeCursor<>(Trie.throwingResolver(), direction, sources, Trie::cursor); + return new CollectionMergeCursor.Plain<>(Trie.throwingResolver(), direction, sources, Trie::cursor); } @Override @@ -227,7 +227,7 @@ public Iterable valuesUnordered() @Override default Trie prefixedBy(ByteComparable prefix) { - return dir -> new PrefixedCursor(prefix, cursor(dir)); + return dir -> new PrefixedCursor.Plain<>(prefix, cursor(dir)); } @Override diff --git a/src/java/org/apache/cassandra/db/tries/Trie.md b/src/java/org/apache/cassandra/db/tries/Trie.md index 9093f5cb8a12..489c5c3c3f44 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.md +++ b/src/java/org/apache/cassandra/db/tries/Trie.md @@ -426,3 +426,211 @@ respective side. Also note that the inversion of an array of boundaries is the a sides.) Using De Morgan's law, this weak negation also lets us perform set union. + +# Range tries + +A range trie is a generalization of the trie set, where the covered ranges can come with further information. This is +achieved by replacing the `precedingIncluded` method with one that returns a state applicable to the preceding +positions. + +In their simplest, a range trie is one that returns `content` for the boundary positions of the ranges, and also +implements a `precedingState` method that returns the range state that applies to positions before the cursor's. For +a little better efficiency most of the time we combine these two into a `state` method that returns the content, if +the position is a boundary, or the preceding state otherwise. This suffices to implement the required operations, +including: +- Intersecting a range trie with a trie set, which generates boundaries that match the closer of the range trie's or + the set's. +- Combining two range tries in a union, where the applicable covering state is applied to every content position + given to the merge resolver. +- Inserting ranges into an in-memory range trie, applying new ranges to existing content as well as existing ranges to + new content to have the same result as the union above. +- The above also form the basis of the application of range tries to data, e.g. applying deletions as range tries to + content tries. + +For the examples below, consider range states that specify deletion times. For example, a range trie could be used to +describe a deletion with timestamp 555 that applies to the range `[abc, adc]` as +``` +a -> + b -> + c -> start(555) + d -> + c -> end(555) +``` + +This dump only lists the content of the range trie. If we also include the preceding state by reporting all `state` +values, the trie will look like this in the forward direction: +``` +a -> + b -> + c -> start(555) + d -> covering(555) + c -> end(555) +``` +and like this in the reverse: +``` +a -> + d -> + c -> end(555) + b -> covering(555) + c -> start(555) +``` +Note that any content must be the same in both directions, but preceding state applies to preceding positions in +iteration order and thus will be different in the two directions. + +The range state used in this representation will be such that `end(dt)` has a `null` state on the left (i.e. returned +by `precedingState(FORWARD)`) and has `covering(dt)` on the right (`precedingState(REVERSE)`), `start(dt)` has +`covering(dt)` on the left and `null` on the right. `covering(dt)` is a non-boundary state that returns itself for the +preceding state in both directions. To support touching ranges, we also need a `switch(ldt, rdt)` state that has +`covering(ldt)` on the left and `covering(rdt)` on the right. + +## Slice / set intersection of range tries + +Intersection of range tries is performed by the same process as normal trie set intersection, augmented by information +about the covering states of every position. If positions are completely covered by the set, we report the range +cursor's `state/precedingState/content` unmodified. If the position falls on a prefix or a boundary of the set, we throw +away (using the `restrict` method) parts that do not fall inside the set. The latter may also happen if the position +is not one present in the range trie, but covered by a range (i.e. where `skipTo` went beyond the set cursor's position +and the range cursor's `precedingState` returned covering state): in this case we apply `restrict` to the covering +state, which may promote it to a boundary if the set cursor's position is a boundary. + +Imagine that we want to slice the range trie above with the range `[aaa, acc]`, which would be implemented by the trie +set +``` +a -> START_END_PREFIX + a -> START_PREFIX + a -> START + c -> END_PREFIX + c -> END +``` + +The intersection cursor will first visit the root and the position "a", where in both cases it will find `null` range +cursor state, resulting in an `null` state for the intersection. The next position "aa" is present in the set, but not +in the range, thus the `skipTo` operation on the range advances to "ab", whose `precedingState` is null. This means that +there is nothing to intersect in the "aa" branch and anything before the range cursor's position, thus we continue by +skipping the set cursor to "ab". This positions it at "ac", whose state is `END_PREFIX` and thus `precedingIncluded` +is `true`. This means that we must report all branches of the range cursor that we see until we advance to or beyond the +set's position. The intersection cursor is positioned at the range cursor's "ab" position. It does not have any `state` +for it, so the intersection cursor reports `null` state as well. + +On the next advance we descend to "abc" (which by virtue of descending is known to fall before the set cursor's +position) and report the range cursor's `start(555)` state unchanged, resulting also in the same `content` and `null` +as `precedingState` (because `start(dt)` has `null` on its left (preceding in forward direction) side). + +The next advance takes the range cursor to "ad", which is beyond the current set cursor position. We check the range +cursor's `precedingState` and find that it is `covering(555)`. Since at this point we have a preceding state, we need to +walk the set branch and use it to augment and report the active covering state. The intersection cursor remains at the +set cursor's "ac" position, and must report the active `covering(555)` augmented by the set cursor's `END_PREFIX` state. +This would drop the right side of any state, but as the intersection cursor is iterating in forward direction, it must +report the _left_ side as the `precedingState`, and thus `covering(555)` is reported as the state and `null` as the +`content` (because `covering(dt)` is not a boundary state). + +On the next advance, the intersection cursor follows the earlier of the two cursors, which is the set cursor. This +advances it to "acc", which is a boundary of the set with state `END`. The active covering state is still +`covering(555)`; augmenting it with `END` turns it into the boundary `end(555)`, which is reported in `state` as +well as `content` (because `start(dt)` is a boundary state). `precedingState` reports the left side of this boundary, +which is still `covering(555)`. + +The next advance takes the set to the exhausted position with `START_END_PREFIX` state, which has `false` for +`precedingIncluded`. Therefore, there is nothing to report before this position, and the range cursor is skipped to it, +which completes the intersection. + +The resulting trie looks as expected: +``` +a -> + b -> + c -> start(555) + c -> covering(555) + c -> end(555) +``` + +## Union of range tries + +The union process is similar (with a second range trie instead of a set), but we walk all branches of both tries and +combine their states. There are two differences from the normal trie union process: +- We apply the merge resolver to states instead of content. This includes both content and preceding state, which is + necessary to be able to report the correct state for the merged trie. +- When one of the range cursors is ahead, we pass its `precedingState` as an argument to the merge resolver to modify + all reported states. + +As an example, consider once again the `[abc, adc]` range with deeltion 555, merged with the following trie for the +`[aaa, acc]` range with deletion 666: +``` +a -> + a -> + a -> start(666) + c -> covering(666) + c -> end(666) +``` + +The merge cursor will first proceed along "aaa" where the first source (advancing to "ab") does not have any +`precedingState`, and thus the merge reports "null" for "aa" and the `start(666)` state for "aaa" unchanged. On the next +advance this source moves beyond the other cursor's "ab" position. The merge thus follows the second source, but the +first has a `precedingState` of `covering(666)`, which must be reflected in the reported states. The second cursor has +no `state` for "ab", thus the merge reports `covering(666)` as the state for "ab". + +The next advance takes the second source to "abc", with `start(555)` state. The merge resolved is called with +`start(555)` and `covering(666)` as arguments. Typically, the resolvers we use drop smaller deletion timestamps, so +this returns `covering(666)` unchanged. + +The next advance takes the second source to "ad", which is beyond the current position of the first source. The merge +cursor switches to following the first source, positioned at "ac", with `covering(666)` as the `state`, but +it must also reflect the second sources `covering(555)` preceding state. The resolver is called with these two +arguments and once again returns the bigger deletion timestamp, `covering(666)`. + +The next advance takes the first source and the iteration cursor to "acc", where this source has the `end(666)` +boundary as state. The merge resolver is called with `end(666)` and `covering(555)`. This time the covering state does +not override the boundary, thus the resolver must create a state that reflects the end of the current range, as +well as the fact that we continue with the other covering state. It must thus return the boundary state +`switch(666, 555)` which the intersection cursor reports. + +The next advance takes the first source to the exhausted position and no `precedingState`. The merge thus reports all +paths and state from the other cursor unchanged until it is exhausted as well, i.e. `covering(555)` for "ad" and +`end(555)` for "adc". + +The final resulting trie looks like this: +``` +a -> + a -> + a -> start(666) + b -> covering(666) + c -> covering(666) + c -> covering(666) + c -> switch(666, 555) + d -> covering(555) + c -> end(555) +``` +Note that the "abc" path adds no information. We don't, however, know that before descending into that branch, thus we +can't easily remove it. This could be done using a special `cleanup` operation over a trie which must buffer descents +until effective content is found, which is best done as a separate transformation rather than as part of the merge. + +## In-memory range tries + +When a range trie is stored in an in-memory trie, it stores only content values. The range cursors created keep track of +the currently active covering state (which is equal to the succeeding side of any visited boundary during advance) and +report it as `precedingState`. This information, however, is no longer valid when a `skipTo` operation is performed, as +it may skip over arbitrarily many boundaries and end up in a covered range. If `precedingState` is requested after such +a skip, the cursor needs to obtain the applicable state. This is done by descending into the current branch (in +iteration order) until the closest boundary is found, and using its preceding side. For this to work, all in-memory trie +branches must terminate in a boundary state with content, which is something that in-memory tries do maintain (see +below). + +Because singletons don't really make sense for range tries (a range will have different start and end paths), all +insertions into a range trie are done using the `apply` method. The application itself is more elaborate than the case +of simple data tries: when `apply` is called with a range trie argument, the in-memory trie has to walk all existing +positions that fall under ranges of the trie and apply the active state to them. Additionally, it must track any active +existing range to combine it with incoming content. + +Because the incoming content is often expected to be a (newer) deletion, the resolver is expected to often return null +for combined content. This triggers removal of nodes and paths up the relevant branch (which may also result in changing +the type of a node e.g. from sparse to chain), which in turn guarantees that we remove branches that do not terminate in +non-null content. + +## Relation to trie sets + +`TrieSetCursor` is a subclass of `RangeTrieCursor`, and the trie set is a special case of a range trie. It uses a +richer state, which contains information for both iteration directions, which also makes it possible to present the same +state for both forward and reverse iteration directions. + +Such richer state is not forbidden, but also not necessary for a general range trie. Because it is something that is +pretty difficult to obtain (or store and maintain) for an in-memory trie, general range tries, which are meant to be +stored in in-memory tries, do not provide it. \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/tries/TrieSet.java b/src/java/org/apache/cassandra/db/tries/TrieSet.java index a875521996dc..4d6409817417 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieSet.java +++ b/src/java/org/apache/cassandra/db/tries/TrieSet.java @@ -19,6 +19,7 @@ package org.apache.cassandra.db.tries; import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; /// A trie that defines an infinite set of `ByteComparable`s. The convention of this package is that sets always /// include all boundaries, all prefixes that lead to a boundary, and all descendants of all boundaries. This is done @@ -47,6 +48,52 @@ static TrieSet ranges(ByteComparable.Version version, ByteComparable... boundari return dir -> new RangesCursor(dir, version, boundaries); } + static TrieSet empty(ByteComparable.Version byteComparableVersion) + { + return dir -> TrieSetCursor.empty(dir, byteComparableVersion); + } + + /// Returns true if the given key is strictly contained in this set, i.e. it falls inside a covered range or branch. + /// This excludes prefixes of set boundaries. + default boolean strictlyContains(ByteComparable key) + { + return contains(key) == ContainsResult.CONTAINED; + } + + /// Returns true if the given key is weaky contained in this set, i.e. it falls inside a covered range or branch, or + /// is a prefix of a set boundary. + default boolean weaklyContains(ByteComparable key) + { + return contains(key) != ContainsResult.NOT_CONTAINED; + } + + enum ContainsResult + { + CONTAINED, + PREFIX, + NOT_CONTAINED + } + + /// Returns whether the given key is contained in this set. Returns CONTAINED if it falls inside a covered range or + /// branch, PREFIX if it is a prefix of a set boundary, and NOT_CONTAINED if it is not contained in the set at all. + default ContainsResult contains(ByteComparable key) + { + TrieSetCursor cursor = cursor(Direction.FORWARD); + final ByteSource bytes = key.asComparableBytes(cursor.byteComparableVersion()); + int next = bytes.next(); + int depth = cursor.depth(); + while (next != ByteSource.END_OF_STREAM) + { + if (cursor.branchIncluded()) + return ContainsResult.CONTAINED; // The set covers a prefix of the key. + if (cursor.skipTo(++depth, next) != depth || cursor.incomingTransition() != next) + return cursor.state().precedingIncluded(Direction.FORWARD) ? ContainsResult.CONTAINED + : ContainsResult.NOT_CONTAINED; + next = bytes.next(); + } + return cursor.branchIncluded() ? ContainsResult.CONTAINED : ContainsResult.PREFIX; + } + default TrieSet union(TrieSet other) { return dir -> new TrieSetIntersectionCursor.UnionCursor(cursor(dir), other.cursor(dir)); diff --git a/src/java/org/apache/cassandra/db/tries/TrieSetCursor.java b/src/java/org/apache/cassandra/db/tries/TrieSetCursor.java index 74281293b47f..b45c459202cf 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieSetCursor.java +++ b/src/java/org/apache/cassandra/db/tries/TrieSetCursor.java @@ -18,20 +18,22 @@ package org.apache.cassandra.db.tries; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + /// The implementation of a [TrieSet]. /// /// In addition to the functionality of normal trie cursors, set cursors also produce a [#state] that describes the -/// coverage of trie sections to the left, right and below the cursor position. This is necessary to be able to identify +/// coverage of trie sections to the left and right of the cursor position. This is necessary to be able to identify /// coverage after a [#skipTo] operation, where the set cursor jumps to a position beyond the requested one. -interface TrieSetCursor extends Cursor +interface TrieSetCursor extends RangeCursor { /// This type describes the state at a given cursor position. It describes the coverage of the positions before and /// after the current in forward order, whether the node is boundary (and thus applies to this point and all its /// descendants) and also describes the type of boundary (e.g. start/end). - enum RangeState + enum RangeState implements org.apache.cassandra.db.tries.RangeState { // Note: the states must be ordered so that - // `values()[applicableBefore * 1 + applicableAfter * 2 + applicableAtPoint * 4]` + // `values()[applicableBefore * 1 + applicableAfter * 2 + isBoundary * 4]` // produces a state with the requested flags /// The cursor is at a prefix of a contained range, and neither the branches to the left or right are contained. @@ -59,15 +61,14 @@ enum RangeState final boolean applicableBefore; /// Whether the set applied to positions after the cursor's in forward order. final boolean applicableAfter; - /// The state to report as content. This converts prefix states to null to report only the boundaries - /// (e.g. for dumping to text). - final RangeState asContent; + /// Whether this marker specifies a boundary point. Boundary points are reported as content. + final boolean isBoundary; - RangeState(boolean applicableBefore, boolean applicableAfter, boolean applicableAtPoint) + RangeState(boolean applicableBefore, boolean applicableAfter, boolean isBoundary) { this.applicableBefore = applicableBefore; this.applicableAfter = applicableAfter; - this.asContent = applicableAtPoint ? this : null; + this.isBoundary = isBoundary; } /// Whether the positions preceding the current in iteration order are included in the set. @@ -76,15 +77,16 @@ public boolean precedingIncluded(Direction direction) return direction.select(applicableBefore, applicableAfter); } - /// Whether the descendant branch is fully included in the set. - public boolean branchIncluded() + /// Whether the current position is a range boundary. This also means that the descendant branch is fully + /// included in the set. + public boolean isBoundary() { - return asContent != null; + return isBoundary; } public RangeState toContent() { - return asContent; + return isBoundary ? this : null; } /// Return an "intersection" state for the combination of two states, i.e. the ranges covered by both states. @@ -106,9 +108,54 @@ public RangeState weakNegation() return values()[ordinal() ^ 3]; } - public static RangeState fromProperties(boolean applicableBefore, boolean applicableAfter, boolean applicableAtPoint) + public static RangeState fromProperties(boolean applicableBefore, boolean applicableAfter, boolean isBoundary) + { + return values()[(applicableBefore ? 1 : 0) + (applicableAfter ? 2 : 0) + (isBoundary ? 4 : 0)]; + } + + // RangeState implementations (used for verification) + + @Override + public RangeState precedingState(Direction direction) { - return values()[(applicableBefore ? 1 : 0) + (applicableAfter ? 2 : 0) + (applicableAtPoint ? 4 : 0)]; + return precedingIncluded(direction) ? END_START_PREFIX : START_END_PREFIX; + } + + @Override + public RangeState restrict(boolean applicableBefore, boolean applicableAfter) + { + return fromProperties(this.applicableBefore && applicableBefore, + this.applicableAfter && applicableAfter, + this.isBoundary); + } + + @Override + public RangeState asBoundary(Direction direction) + { + final boolean isForward = direction.isForward(); + return fromProperties(this.applicableBefore && !isForward, + this.applicableAfter && isForward, + true); + } + + + public > + S applyToCoveringState(S srcState, Direction direction) + { + switch (this) + { + case POINT: + return null; + case COVERED: + return srcState; + case START: + return srcState.asBoundary(Direction.FORWARD); + case END: + return srcState.asBoundary(Direction.REVERSE); + default: + return precedingIncluded(direction) ? srcState : null; + } + } } @@ -130,7 +177,7 @@ default boolean precedingIncluded() /// points. default boolean branchIncluded() { - return state().asContent != null; + return state().isBoundary; } @Override @@ -141,4 +188,38 @@ default RangeState content() @Override TrieSetCursor tailCursor(Direction direction); + + class Empty extends Cursor.Empty implements TrieSetCursor + { + final RangeState coveringState; + + public Empty(RangeState coveringState, ByteComparable.Version version, Direction direction) + { + super(direction, version); + this.coveringState = coveringState; + } + + @Override + public RangeState state() + { + return coveringState; + } + + @Override + public RangeState content() + { + return null; + } + + @Override + public TrieSetCursor tailCursor(Direction direction) + { + return new TrieSetCursor.Empty(coveringState, byteComparableVersion(), direction); + } + } + + static TrieSetCursor empty(Direction direction, ByteComparable.Version version) + { + return new Empty(TrieSetCursor.RangeState.START_END_PREFIX, version, direction); + } } diff --git a/src/java/org/apache/cassandra/db/tries/TrieSetIntersectionCursor.java b/src/java/org/apache/cassandra/db/tries/TrieSetIntersectionCursor.java index e69a9290eaa6..dcd51c055370 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieSetIntersectionCursor.java +++ b/src/java/org/apache/cassandra/db/tries/TrieSetIntersectionCursor.java @@ -233,7 +233,7 @@ private int matchingPosition(int depth, int transition) TrieSetCursor.RangeState combineState(TrieSetCursor.RangeState cl, TrieSetCursor.RangeState cr) { - assert cl.branchIncluded() == cr.branchIncluded() : "Intersection results in a prefix range"; + assert cl.isBoundary() == cr.isBoundary() : "Intersection results in a prefix range"; return cl.intersect(cr); } @@ -271,7 +271,7 @@ boolean precedingInSet(TrieSetCursor cursor) @Override TrieSetCursor.RangeState combineState(TrieSetCursor.RangeState cl, TrieSetCursor.RangeState cr) { - assert cl.branchIncluded() == cr.branchIncluded() : "Union results in a prefix range"; + assert cl.isBoundary() == cr.isBoundary() : "Union results in a prefix range"; return cl.union(cr); } diff --git a/src/java/org/apache/cassandra/db/tries/VerificationCursor.java b/src/java/org/apache/cassandra/db/tries/VerificationCursor.java index 4a44d8388f0b..83096d56939b 100644 --- a/src/java/org/apache/cassandra/db/tries/VerificationCursor.java +++ b/src/java/org/apache/cassandra/db/tries/VerificationCursor.java @@ -22,6 +22,7 @@ import com.google.common.base.Preconditions; +import java.util.Objects; import org.agrona.DirectBuffer; import org.apache.cassandra.utils.Hex; import org.apache.cassandra.utils.bytecomparable.ByteComparable; @@ -61,16 +62,17 @@ class Plain> implements Cursor, Cursor.TransitionsRece this.returnedTransition = expectedTransition; this.path = new byte[16]; Preconditions.checkState(source.depth() == expectedDepth && source.incomingTransition() == expectedTransition, - "Invalid initial depth %s with incoming transition %s (must be %s, %s)", + "Invalid initial depth %s with incoming transition %s (must be %s, %s)\n%s", source.depth(), source.incomingTransition(), - expectedDepth, expectedTransition); + expectedDepth, expectedTransition, this); } @Override public int depth() { Preconditions.checkState(returnedDepth == source.depth(), - "Depth changed without advance: %s -> %s", returnedDepth, source.depth()); + "Depth changed without advance: %s -> %s\n%s", + returnedDepth, source.depth(), this); return returnedDepth; } @@ -78,7 +80,8 @@ public int depth() public int incomingTransition() { Preconditions.checkState(returnedTransition == source.incomingTransition(), - "Transition changed without advance: %s -> %s", returnedTransition, source.incomingTransition()); + "Transition changed without advance: %s -> %s\n%s", + returnedTransition, source.incomingTransition(), this); return source.incomingTransition(); } @@ -114,8 +117,8 @@ public int advanceMultiple(Cursor.TransitionsReceiver receiver) int depth = source.advanceMultiple(this); chainedReceiver = null; Preconditions.checkState(!advanceMultipleCalledReceiver || depth == returnedDepth + 1, - "advanceMultiple returned depth %s did not match depth %s after added characters", - depth, returnedDepth + 1); + "advanceMultiple returned depth %s did not match depth %s after added characters\n%s", + depth, returnedDepth + 1, this); return verify(depth); } @@ -129,38 +132,40 @@ public int skipTo(int skipDepth, int skipTransition) private void verifySkipRequest(int skipDepth, int skipTransition) { Preconditions.checkState(skipDepth <= returnedDepth + 1, - "Skip descends more than one level: %s -> %s", + "Skip descends more than one level: %s -> %s\n%s", returnedDepth, - skipDepth); + skipDepth, + this); if (skipDepth <= returnedDepth && skipDepth > minDepth) Preconditions.checkState(direction.lt(getByte(skipDepth), skipTransition), - "Skip goes backwards to %s at depth %s where it already visited %s", - skipTransition, skipDepth, getByte(skipDepth)); + "Skip goes backwards to %s at depth %s where it already visited %s\n%s", + skipTransition, skipDepth, getByte(skipDepth), this); } private int verify(int depth) { Preconditions.checkState(depth <= returnedDepth + 1, - "Cursor advanced more than one level: %s -> %s", + "Cursor advanced more than one level: %s -> %s\n%s", returnedDepth, - depth); + depth, + this); Preconditions.checkState(depth < 0 || depth > minDepth, - "Cursor ascended to depth %s beyond its minimum depth %s", - depth, minDepth); + "Cursor ascended to depth %s beyond its minimum depth %s\n%s", + depth, minDepth, this); final int transition = source.incomingTransition(); if (depth < 0) { Preconditions.checkState(depth == EXHAUSTED_DEPTH && transition == EXHAUSTED_TRANSITION, - "Cursor exhausted state should be %s, %s but was %s, %s", + "Cursor exhausted state should be %s, %s but was %s, %s\n%s", EXHAUSTED_DEPTH, EXHAUSTED_TRANSITION, - depth, transition); + depth, transition, this); } else if (depth <= returnedDepth) { Preconditions.checkState(direction.lt(getByte(depth), transition), - "Cursor went backwards to %s at depth %s where it already visited %s", - transition, depth, getByte(depth)); + "Cursor went backwards to %s at depth %s where it already visited %s\n%s", + transition, depth, getByte(depth), this); } returnedDepth = depth; returnedTransition = transition; @@ -229,23 +234,22 @@ public String toString() } } - class TrieSet extends Plain implements TrieSetCursor + abstract class WithRanges, C extends RangeCursor> + extends Plain + implements RangeCursor { - boolean currentPrecedingIncluded; - boolean nextPrecedingIncluded; + S currentPrecedingState = null; + S nextPrecedingState = null; + int maxNextDepth = Integer.MAX_VALUE; - TrieSet(TrieSetCursor source) - { - this(source, 0, 0, INITIAL_TRANSITION); - } - - TrieSet(TrieSetCursor source, int minDepth, int expectedDepth, int expectedTransition) + WithRanges(C source, int minDepth, int expectedDepth, int expectedTransition) { super(source, minDepth, expectedDepth, expectedTransition); // start state can be non-null for sets - currentPrecedingIncluded = source.precedingIncluded(); - Preconditions.checkNotNull(currentPrecedingIncluded, "Covering state for trie sets must not be null"); - nextPrecedingIncluded = source.content() != null ? source.content().precedingIncluded(direction.opposite()) : currentPrecedingIncluded; + currentPrecedingState = verifyCoveringStateProperties(source.precedingState()); + final S content = source.content(); + nextPrecedingState = content != null ? verifyBoundaryStateProperties(content).precedingState(direction.opposite()) + : currentPrecedingState; } void verifyEndState() @@ -253,67 +257,78 @@ void verifyEndState() // end state can be non-null for sets } - @Override - public TrieSetCursor.RangeState state() - { - return Preconditions.checkNotNull(source.state()); - } - - @Override - public TrieSet tailCursor(Direction direction) - { - return new TrieSet(source.tailCursor(direction), 0, 0, INITIAL_TRANSITION); - } - @Override public int advance() { - currentPrecedingIncluded = nextPrecedingIncluded; + currentPrecedingState = nextPrecedingState; + checkIfDescentShouldBeForbidden(); return verifyState(super.advance()); } @Override public int advanceMultiple(TransitionsReceiver receiver) { - currentPrecedingIncluded = nextPrecedingIncluded; + currentPrecedingState = nextPrecedingState; + checkIfDescentShouldBeForbidden(); return verifyState(super.advanceMultiple(receiver)); } @Override public int skipTo(int skipDepth, int skipTransition) { + checkIfDescentShouldBeForbidden(); return verifySkipState(super.skipTo(skipDepth, skipTransition)); } + private void checkIfDescentShouldBeForbidden() + { + maxNextDepth = source.content() != null ? source.depth() : Integer.MAX_VALUE; + } + @Override - public boolean precedingIncluded() + public S precedingState() { - Preconditions.checkState(currentPrecedingIncluded == source.precedingIncluded(), - "Covering state changed without advance: %s -> %s. %s", - currentPrecedingIncluded, source.precedingIncluded(), - currentPrecedingIncluded == source.precedingIncluded() + Preconditions.checkState(currentPrecedingState == source.precedingState(), + "Preceding state changed without advance: %s -> %s. %s\n%s", + currentPrecedingState, source.precedingState(), + agree(currentPrecedingState, source.precedingState()) ? "The values are equal but different object. This is not permitted for performance reasons." - : ""); - // == above is correct, we do not want covering state to be recreated unless some change happened to the cursor - return currentPrecedingIncluded; + : "", + this); + return currentPrecedingState; } - private int verifyState(int depth) + @Override + public S state() + { + return source.state(); + } + + boolean agree(S left, S right) { - boolean precedingIncluded = source.precedingIncluded(); - Preconditions.checkNotNull(precedingIncluded, "Covering state for trie sets must not be null"); - Preconditions.checkState(currentPrecedingIncluded == precedingIncluded, - "Unexpected change to covering state: %s -> %s", - currentPrecedingIncluded, precedingIncluded); - currentPrecedingIncluded = precedingIncluded; + return Objects.equals(left, right); + } - RangeState content = source.content(); + private int verifyState(int depth) + { + S precedingState = source.precedingState(); + boolean equal = agree(currentPrecedingState, precedingState); + Preconditions.checkState(equal, + "Unexpected change to covering state: %s -> %s\n%s", + currentPrecedingState, precedingState, this); + Preconditions.checkState(depth <= maxNextDepth, + "Cursor descended after reporting an included branch\n%s", + this); + currentPrecedingState = precedingState; + + S content = source.content(); if (content != null) { - Preconditions.checkState(currentPrecedingIncluded == content.precedingIncluded(direction), - "Range end %s does not close covering state %s", - content.precedingIncluded(direction), currentPrecedingIncluded); - nextPrecedingIncluded = content.precedingIncluded(direction.opposite()); + Preconditions.checkState(agree(currentPrecedingState, content.precedingState(direction)), + "Range end %s does not close covering state %s\n%s", + content.precedingState(direction), currentPrecedingState, this); + verifyBoundaryStateProperties(content); + nextPrecedingState = content.precedingState(direction.opposite()); } if (depth < 0) @@ -324,10 +339,111 @@ private int verifyState(int depth) private int verifySkipState(int depth) { // The covering state information is invalidated by a skip. - currentPrecedingIncluded = source.precedingIncluded(); - Preconditions.checkNotNull(currentPrecedingIncluded, "Covering state for trie sets must not be null"); - nextPrecedingIncluded = currentPrecedingIncluded; + currentPrecedingState = verifyCoveringStateProperties(source.precedingState()); + nextPrecedingState = currentPrecedingState; return verifyState(depth); } + + S verifyCoveringStateProperties(S state) + { + if (state == null) + return null; + Preconditions.checkState(!state.isBoundary(), + "Boundary state %s was returned where a covering state was expected\n%s", + state, + this); + final S precedingState = state.precedingState(Direction.FORWARD); + final S succeedingState = state.precedingState(Direction.REVERSE); + Preconditions.checkState(precedingState == state && succeedingState == state, + "State %s must return itself its preceding and succeeding state (returned %s/%s)\n%s", + state, + precedingState, + succeedingState, + this); + return state; + } + + S verifyBoundaryStateProperties(S state) + { + if (state == null) + return null; + Preconditions.checkState(state.isBoundary(), + "Covering state %s was returned where a boundary state was expected\n%s", + state, + this); + final S precedingState = state.precedingState(Direction.FORWARD); + final S succeedingState = state.precedingState(Direction.REVERSE); + verifyCoveringStateProperties(precedingState); + verifyCoveringStateProperties(succeedingState); + return state; + } + + + @Override + public abstract WithRanges tailCursor(Direction direction); + + @Override + public String toString() + { + return super.toString() + " state " + state(); + } + } + + + class Range> extends WithRanges> implements RangeCursor + { + Range(RangeCursor source) + { + this(source, 0, 0, INITIAL_TRANSITION); + } + + Range(RangeCursor source, int minDepth, int expectedDepth, int expectedTransition) + { + super(source, minDepth, expectedDepth, expectedTransition); + Preconditions.checkState(currentPrecedingState == null, + "Initial preceding state %s should be null for range cursor\n%s", + currentPrecedingState, this); + } + + @Override + void verifyEndState() + { + Preconditions.checkState(currentPrecedingState == null, + "End state %s should be null for range cursor\n%s", + currentPrecedingState, this); + } + + @Override + public Range tailCursor(Direction direction) + { + return new Range<>(source.tailCursor(direction), 0, 0, INITIAL_TRANSITION); + } + } + + class TrieSet extends WithRanges implements TrieSetCursor + { + TrieSet(TrieSetCursor source) + { + this(source, 0, 0, INITIAL_TRANSITION); + } + + TrieSet(TrieSetCursor source, int minDepth, int expectedDepth, int expectedTransition) + { + super(source, minDepth, expectedDepth, expectedTransition); + // start state can be non-null for sets + Preconditions.checkNotNull(currentPrecedingState, "Preceding state for trie sets must not be null\n%s", this); + } + + @Override + public TrieSetCursor.RangeState state() + { + return Preconditions.checkNotNull(source.state()); + } + + @Override + public TrieSet tailCursor(Direction direction) + { + return new TrieSet(source.tailCursor(direction), 0, 0, INITIAL_TRANSITION); + } } } diff --git a/test/unit/org/apache/cassandra/db/memtable/MemtableThreadedTest.java b/test/unit/org/apache/cassandra/db/memtable/MemtableThreadedTest.java index 27252e52c124..a49c1894977f 100644 --- a/test/unit/org/apache/cassandra/db/memtable/MemtableThreadedTest.java +++ b/test/unit/org/apache/cassandra/db/memtable/MemtableThreadedTest.java @@ -40,7 +40,7 @@ import static org.apache.cassandra.db.ColumnFamilyStore.FlushReason.UNIT_TESTS; -/// This test is a counterpart to InMemoryTrieThreadedTest that makes sure TrieMemtable is wiring the trie consistency +/// This test is a counterpart to InMemoryTrieConsistencyTest that makes sure TrieMemtable is wiring the trie consistency /// machinery correctly. Note that this test always applies mutations the same way (with partition-level forced copying) /// and is effectively doing the same test but checking different correctness properties. /// diff --git a/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java b/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java index 44136021b094..32e9731ca278 100644 --- a/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java +++ b/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java @@ -273,7 +273,7 @@ static void addToInMemoryTrie(ByteComparable[] src, int payload = asString(b).hashCode(); ByteBuffer v = ByteBufferUtil.bytes(payload); Trie update = Trie.singleton(b, VERSION, v); - update = InMemoryTrieThreadedTest.withRootMetadata(update, Boolean.TRUE); + update = TrieUtil.withRootMetadata(update, Boolean.TRUE); update = update.prefixedBy(source("prefix")); applyUpdating(trie, update, forceCopyPredicate); } @@ -295,9 +295,9 @@ static void addThrowingEntry(ByteComparable b, // Create an update with two metadata entries, so that the lower is already a copied node. // Abort processing on the lower metadata, where the new branch is not attached yet (so as not to affect the // contents). - update = InMemoryTrieThreadedTest.withRootMetadata(update, Boolean.FALSE); + update = TrieUtil.withRootMetadata(update, Boolean.FALSE); update = update.prefixedBy(source("fix")); - update = InMemoryTrieThreadedTest.withRootMetadata(update, Boolean.TRUE); + update = TrieUtil.withRootMetadata(update, Boolean.TRUE); update = update.prefixedBy(source("pre")); trie.apply(update, diff --git a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java index 62c708018811..efaffeded19b 100644 --- a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java @@ -90,7 +90,7 @@ public void testWithDuplicates() private static Trie makeCollectionMergeTrie(InMemoryTrie... tries) { - return dir -> new CollectionMergeCursor<>(x -> x.iterator().next(), dir, List.of(tries), Trie::cursor); + return dir -> new CollectionMergeCursor.Plain<>(x -> x.iterator().next(), dir, List.of(tries), Trie::cursor); } @Test @@ -119,7 +119,7 @@ private static Trie mergeDistinctTrie(Collection> sourc @Override public Cursor makeCursor(Direction direction) { - return new CollectionMergeCursor<>(Trie.throwingResolver(), direction, sources, Trie::cursor); + return new CollectionMergeCursor.Plain<>(Trie.throwingResolver(), direction, sources, Trie::cursor); } @Override diff --git a/test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java b/test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java new file mode 100644 index 000000000000..21e754b6b88e --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java @@ -0,0 +1,410 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Predicate; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.concurrent.OpOrder; + +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.apache.cassandra.db.tries.TrieUtil.generateKeys; +import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; + +public abstract class ConsistencyTestBase, R extends BaseTrie> +{ + // Note: This should not be run by default with verification to have the higher concurrency of faster writes and reads. + + private static final int COUNT = 30000; + private static final int PROGRESS_UPDATE = COUNT / 15; + private static final int READERS = 8; + private static final int WALKERS = 2; + private static final Random rand = new Random(); + + /** + * Force copy every modified cell below the partition/enumeration level. Provides atomicity of mutations within the + * partition level as well as consistency. + */ + public final Predicate> FORCE_COPY_PARTITION = features -> isPartition(features.content()); + /** + * Force copy every modified cell below the earliest branching point. Provides atomicity of mutations at any level, + * but readers/walkers may see inconsistent views of the data, in the sense that older mutations may be missed + * while newer ones are returned. + */ + public final Predicate> FORCE_ATOMIC = features -> features.isBranching(); + /** + * Do not do any additional copying beyond what is required to build the tries safely for concurrent readers. + * Mutations may be partially seen by readers, and older mutations may be missed while newer ones are returned. + */ + public final Predicate> NO_ATOMICITY = features -> false; + + abstract R makeTrie(OpOrder readOrder); + + abstract C value(ByteComparable b, ByteComparable cprefix, ByteComparable c, int add, int seqId); + abstract C metadata(ByteComparable b); + abstract String pk(C c); + abstract String ck(C c); + abstract int seq(C c); + abstract int value(C c); + abstract int updateCount(C c); + + abstract T makeSingleton(ByteComparable b, C content); + abstract T withRootMetadata(T wrapped, C metadata); + + abstract T merge(Collection tries, Trie.CollectionMergeResolver mergeResolver); + abstract void apply(R trie, + T mutation, + InMemoryBaseTrie.UpsertTransformer mergeResolver, + Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException; + + abstract boolean isPartition(C c); + abstract C mergeMetadata(C c1, C c2); + + abstract void printStats(R trie, Predicate> forcedCopyChecker); + + @Test + public void testConsistentUpdates() throws Exception + { + // Check that multi-path updates with below-partition-level copying are safe for concurrent readers, + // and that content is atomically applied, i.e. that reader see either nothing from the update or all of it, + // and consistent, i.e. that it is not possible to receive some newer updates while missing + // older ones. (For example, if the sequence of additions is 3, 1, 5, without this requirement a reader + // could see an enumeration which lists 3 and 5 but not 1.) + testAtomicUpdates(3, FORCE_COPY_PARTITION, true, true); + // Note: using 3 per mutation, so that the first and second update fit in a sparse in-memory trie block. + } + + @Test + public void testAtomicUpdates() throws Exception + { + // Check that multi-path updates with below-branching-point copying are safe for concurrent readers, + // and that content is atomically applied, i.e. that reader see either nothing from the update or all of it. + testAtomicUpdates(3, FORCE_ATOMIC, true, false); + } + + @Test + public void testSafeUpdates() throws Exception + { + // Check that multi path updates without additional copying are safe for concurrent readers. + testAtomicUpdates(3, NO_ATOMICITY, false, false); + } + + @Test + public void testConsistentSinglePathUpdates() throws Exception + { + // Check that single path updates with below-partition-level copying are safe for concurrent readers, + // and that content is consistent, i.e. that it is not possible to receive some newer updates while missing + // older ones. (For example, if the sequence of additions is 3, 1, 5, without this requirement a reader + // could see an enumeration which lists 3 and 5 but not 1.) + testAtomicUpdates(1, FORCE_COPY_PARTITION, true, true); + } + + + @Test + public void testAtomicSinglePathUpdates() throws Exception + { + // When doing single path updates atomicity comes for free. This only checks that the branching checker is + // not doing anything funny. + testAtomicUpdates(1, FORCE_ATOMIC, true, false); + } + + @Test + public void testSafeSinglePathUpdates() throws Exception + { + // Check that single path updates without additional copying are safe for concurrent readers. + testAtomicUpdates(1, NO_ATOMICITY, true, false); + } + + // The generated keys all start with NEXT_COMPONENT, which makes it impossible to test the precise behavior of the + // partition-level force copying. Strip that byte. + private static ByteComparable[] skipFirst(ByteComparable[] keys) + { + ByteComparable[] result = new ByteComparable[keys.length]; + for (int i = 0; i < keys.length; ++i) + result[i] = skipFirst(keys[i]); + return result; + } + + private static ByteComparable skipFirst(ByteComparable key) + { + return v -> { + var bs = key.asComparableBytes(v); + int n = bs.next(); + assert n != ByteSource.END_OF_STREAM; + return bs; + }; + } + + public void testAtomicUpdates(int PER_MUTATION, + Predicate> forcedCopyChecker, + boolean checkAtomicity, + boolean checkSequence) + throws Exception + { + ByteComparable[] ckeys = skipFirst(generateKeys(rand, COUNT)); + ByteComparable[] pkeys = skipFirst(generateKeys(rand, Math.min(100, COUNT / 10))); // to guarantee repetition + + /* + * Adds COUNT partitions each with perPartition separate clusterings, where the sum of the values + * of all clusterings is 0. + * If the sum for any walk covering whole partitions is non-zero, we have had non-atomic updates. + */ + + OpOrder readOrder = new OpOrder(); + R trie = makeTrie(readOrder); + ConcurrentLinkedQueue errors = new ConcurrentLinkedQueue<>(); + List threads = new ArrayList(); + AtomicBoolean writeCompleted = new AtomicBoolean(false); + AtomicInteger writeProgress = new AtomicInteger(0); + + for (int i = 0; i < WALKERS; ++i) + threads.add(new Thread() + { + public void run() + { + try + { + Random r = ThreadLocalRandom.current(); + while (!writeCompleted.get()) + { + int min = writeProgress.get(); + try (OpOrder.Group group = readOrder.start()) + { + Iterable> entries = trie.entrySet(); + checkEntries("", min, true, checkAtomicity, false, PER_MUTATION, entries); + } + } + } + catch (Throwable t) + { + t.printStackTrace(); + errors.add(t); + } + } + }); + + for (int i = 0; i < READERS; ++i) + { + ByteComparable[] srcLocal = pkeys; + threads.add(new Thread() + { + public void run() + { + try + { + // await at least one ready partition + while (writeProgress.get() == 0) {} + + Random r = ThreadLocalRandom.current(); + while (!writeCompleted.get()) + { + ByteComparable key = srcLocal[r.nextInt(srcLocal.length)]; + int min = writeProgress.get() / (pkeys.length * PER_MUTATION) * PER_MUTATION; + Iterable> entries; + + try (OpOrder.Group group = readOrder.start()) + { + entries = trie.tailTrie(key).entrySet(); + checkEntries(" in tail " + key.byteComparableAsString(VERSION), min, false, checkAtomicity, checkSequence, PER_MUTATION, entries); + } + + try (OpOrder.Group group = readOrder.start()) + { + entries = trie.subtrie(key, nextBranch(key)).entrySet(); + checkEntries(" in branch " + key.byteComparableAsString(VERSION), min, true, checkAtomicity, checkSequence, PER_MUTATION, entries); + } + } + } + catch (Throwable t) + { + t.printStackTrace(); + errors.add(t); + } + } + }); + } + + threads.add(new Thread() + { + public void run() + { + ThreadLocalRandom r = ThreadLocalRandom.current(); + final Trie.CollectionMergeResolver mergeResolver = new Trie.CollectionMergeResolver() + { + @Override + public C resolve(C c1, C c2) + { + if (isPartition(c1) && isPartition(c2)) + return mergeMetadata(c1, c2); + throw new AssertionError("Test error, keys should be distinct."); + } + + public C resolve(Collection contents) + { + return contents.stream().reduce(this::resolve).get(); + } + }; + + try + { + int lastUpdate = 0; + for (int i = 0; i < COUNT; i += PER_MUTATION) + { + ByteComparable b = pkeys[(i / PER_MUTATION) % pkeys.length]; + C partitionMarker = metadata(b); + ByteComparable cprefix = null; + if (r.nextBoolean()) + cprefix = ckeys[i]; // Also test branching point below the partition level + + List sources = new ArrayList<>(); + for (int j = 0; j < PER_MUTATION; ++j) + { + + ByteComparable k = ckeys[i + j]; + T row = makeSingleton(k, + value(b, cprefix, k, + j == 0 ? -PER_MUTATION + 1 : 1, + (i / PER_MUTATION / pkeys.length) * PER_MUTATION + j)); + + if (cprefix != null) + row = row.prefixedBy(cprefix); + + row = withRootMetadata(row, partitionMarker); + row = row.prefixedBy(b); + sources.add(row); + } + + final T mutation = merge(sources, mergeResolver); + + apply(trie, mutation, + (existing, update) -> existing == null ? update : mergeResolver.resolve(existing, update), + forcedCopyChecker); + + if (i >= pkeys.length * PER_MUTATION && i - lastUpdate >= PROGRESS_UPDATE) + { + writeProgress.set(i); + lastUpdate = i; + } + } + } + catch (Throwable t) + { + t.printStackTrace(); + errors.add(t); + } + finally + { + writeCompleted.set(true); + } + } + }); + + for (Thread t : threads) + t.start(); + + for (Thread t : threads) + t.join(); + + printStats(trie, forcedCopyChecker); + + if (!errors.isEmpty()) + Assert.fail("Got errors:\n" + errors); + } + + static ByteComparable nextBranch(ByteComparable key) + { + return version -> { + byte[] bytes = key.asByteComparableArray(version); + int last = bytes.length - 1; + while (last >= 0 && bytes[last] == ((byte) 0xFF)) + --last; + if (last < 0) + return null; + ++bytes[last]; + return ByteSource.preencoded(bytes, 0, last + 1); + }; + } + + public void checkEntries(String location, + int min, + boolean usePk, + boolean checkAtomicity, + boolean checkConsecutiveIds, + int PER_MUTATION, + Iterable> entries) + { + long sum = 0; + int count = 0; + long idSum = 0; + long idMax = 0; + int updateCount = 0; + for (var en : entries) + { + String path = en.getKey().byteComparableAsString(VERSION); + final C v = en.getValue(); + if (isPartition(v)) + { + Assert.assertEquals("Partition metadata" + location, (usePk ? pk(v) : ""), path); + updateCount += updateCount(v); + continue; + } + String valueKey = (usePk ? pk(v) : "") + ck(v); + Assert.assertEquals(location, valueKey, path); + ++count; + sum += value(v); + int seq = seq(v); + idSum += seq; + if (seq > idMax) + idMax = seq; + } + + Assert.assertTrue("Values" + location + " should be at least " + min + ", got " + count, min <= count); + + if (checkAtomicity) + { + // If mutations apply atomically, the row count is always a multiple of the mutation size... + Assert.assertTrue("Values" + location + " should be a multiple of " + PER_MUTATION + ", got " + count, count % PER_MUTATION == 0); + // ... and the sum of the values is 0 (as the sum for each individual mutation is 0). + Assert.assertEquals("Value sum" + location, 0, sum); + } + + if (checkConsecutiveIds) + { + // The update count reflected in the partition metadata must match the row count. + Assert.assertEquals("Update count" + location, count, updateCount); + // If mutations apply consistently for the partition, for any row we see we have to have seen all rows that + // were applied before that. In other words, the id sum should be the sum of the integers from 1 to the + // highest id seen in the partition. + Assert.assertEquals("Id sum" + location, idMax * (idMax + 1) / 2, idSum); + } + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryRangeTrieTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryRangeTrieTest.java new file mode 100644 index 000000000000..a35e168e63c5 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryRangeTrieTest.java @@ -0,0 +1,382 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.function.Function; +import java.util.stream.Stream; + +import com.google.common.base.Predicates; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class InMemoryRangeTrieTest +{ + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + + int delTime; + + static TestRangeState toMarker(String string) + { + return toMarker(string, 1); + } + + static TestRangeState toMarker(String string, int delTime) + { + return new TestRangeState(TrieUtil.directComparable(string), delTime, delTime, false); + } + + static TestRangeState addMarkerStrings(TestRangeState a, TestRangeState b) + { + assert a.leftSide == b.leftSide; + assert a.rightSide == b.rightSide; + assert a.isBoundary == b.isBoundary; + return new TestRangeState(TrieUtil.directComparable(fromMarker(a) + fromMarker(b)), + a.leftSide, + a.rightSide, + a.isBoundary); + } + + static String fromMarker(TestRangeState marker) + { + if (marker == null) + return null; + return new String(marker.position.asByteComparableArray(VERSION), StandardCharsets.UTF_8); + } + + @Test + public void testSingle() + { + ByteComparable e = TrieUtil.directComparable("test"); + InMemoryRangeTrie trie = InMemoryRangeTrie.shortLived(VERSION); + putRange(trie, e, toMarker("test"), (x, y) -> y); + System.out.println("Trie " + trie.dump()); + assertEquals("test", fromMarker(trie.applicableRange(key(e)))); + assertEquals(null, fromMarker(trie.applicableRange(key(TrieUtil.directComparable("tezt"))))); + assertEquals(null, fromMarker(trie.applicableRange(key(TrieUtil.directComparable("tast"))))); + } + + @Test + public void testSplitMulti() + { + testEntries("testing", "tests", "trials", "trial", "aaaa", "aaaab", "abdddd", "abeeee"); + } + + @Test + public void testSplitMultiBug() + { + testEntriesHex(new String[]{ "0c4143aeff", "0c4143ae69ff" }); + } + + @Test + public void testUpdateContent() + { + String[] tests = new String[]{ "testing", "tests", "trials", "trial", "testing", "trial", "trial" }; + String[] values = new String[]{ "testing", "tests", "trials", "trial", "t2", "x2", "y2" }; + InMemoryRangeTrie trie = InMemoryRangeTrie.shortLived(VERSION); + for (int i = 0; i < tests.length; ++i) + { + String test = tests[i]; + String v = values[i]; + ByteComparable e = TrieUtil.directComparable(test); + System.out.println("Adding " + asString(e) + ": " + v); + putRange(trie, e, toMarker(v), InMemoryRangeTrieTest::addMarkerStrings); + System.out.println("Trie " + trie.dump()); + } + + for (int i = 0; i < tests.length; ++i) + { + String test = tests[i]; + assertEquals(Stream.iterate(0, x -> x + 1) + .limit(tests.length) + .filter(x -> tests[x] == test) + .map(x -> values[x]) + .reduce("", (x, y) -> "" + x + y), + fromMarker(trie.applicableRange(key(TrieUtil.directComparable(test))))); + } + } + + private void testEntries(String... tests) + { + for (Function mapping : + ImmutableList.>of(TrieUtil::comparable, + s -> ByteComparable.preencoded(VERSION, s.getBytes()))) + { + testEntries(tests, mapping); + } + } + + private void testEntriesHex(String[] tests) + { + testEntries(tests, s -> ByteComparable.preencoded(VERSION, ByteBufferUtil.hexToBytes(s))); + // Run the other translations just in case. + testEntries(tests); + } + + private void testEntries(String[] tests, Function mapping) + + { + InMemoryRangeTrie trie = InMemoryRangeTrie.shortLived(VERSION); + for (String test : tests) + { + ByteComparable e = mapping.apply(test); + System.out.println("Adding " + asString(e) + ": " + test); + putRange(trie, e, toMarker(test), (x, y) -> y); + System.out.println("Trie\n" + trie.dump()); + } + + for (String test : tests) + assertEquals(test, fromMarker(trie.applicableRange(key(mapping.apply(test))))); + } + + static String asString(ByteComparable bc) + { + return bc != null ? bc.byteComparableAsString(VERSION) : "null"; + } + + @Test + public void testCursorDeletionBeforeNearest() throws TrieSpaceExhaustedException + { + testCursorsWithInterveningDeletions(strings("aaebc", "aaecd"), + "aa", "aaec", + strings("aabc", "aacd")); + } + + @Test + public void testCursorRangeDeletionCoversPosition() throws TrieSpaceExhaustedException + { + testCursorsWithInterveningDeletions(strings("aaabc", "aaacd", "bcd", "cde"), + "aaa", "aaacd", + strings("a_", "ab")); + } + + @Test + public void testCursorBranchDeletionCoversPosition() throws TrieSpaceExhaustedException + { + testCursorsWithInterveningDeletions(strings("aaabc", "aaacd", "bcd", "cde"), + "aaa", "aaacd", + strings("a_", "ab")); + } + + private String[] strings(String... strings) + { + return strings; + } + + private void testCursorsWithInterveningDeletions(String[] preparations, + String leftPos, + String rightPos, + String[] insertions) + throws TrieSpaceExhaustedException + { + // New deletions supercede old + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.FORWARD, false, 1); + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.FORWARD, true, 1); + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.REVERSE, false, 1); + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.REVERSE, true, 1); + + // New deletions addition to old + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.FORWARD, false, -1); + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.FORWARD, true, -1); + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.REVERSE, false, -1); + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.REVERSE, true, -1); + + // New deletions group with old + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.FORWARD, false, 0); + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.FORWARD, true, 0); + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.REVERSE, false, 0); + testCursorsWithInterveningDeletions(preparations, leftPos, rightPos, insertions, Direction.REVERSE, true, 0); + } + + private void testCursorsWithInterveningDeletions(String[] preparations, + String leftPos, + String rightPos, + String[] insertions, + Direction dir, + boolean useSkip, + int delTimeIncrease) + throws TrieSpaceExhaustedException + { + // Note: ranges are inserted one pair at a time, with changing delTime. + delTime = 100; + if (!dir.isForward() && rightPos.startsWith(leftPos)) + { + String t = leftPos; + leftPos = rightPos; + rightPos = t; // swap left and right as prefixes are always before + } + + InMemoryRangeTrie trie = InMemoryRangeTrie.shortLived(VERSION); + insertRanges(trie, preparations, delTimeIncrease); + + final String current = dir.select(leftPos, rightPos); + RangeCursor c = trie.cursor(dir); + TriePathReconstructor paths = new TriePathReconstructor(); + boolean found; + if (useSkip) + found = c.descendAlong(TrieUtil.directComparable(current).asPeekableBytes(VERSION)); + else + found = advanceTo(c, TrieUtil.directComparable(current), paths); + + if (delTimeIncrease > 0) + assertTrue(found); + + insertRanges(trie, insertions, delTimeIncrease); + + String target = dir.select(rightPos, leftPos); + if (found) + { + if (useSkip) + found = skipByDifference(c, TrieUtil.directComparable(current), TrieUtil.directComparable(target)); + else + found = advanceTo(c, TrieUtil.directComparable(target), paths); + } + else + { + // nested entries may be gone if deleted by parent. If so, just try to skip to target for a new cursor. + c = trie.cursor(dir); + paths = new TriePathReconstructor(); + if (useSkip) + found = c.descendAlong(TrieUtil.directComparable(current).asPeekableBytes(VERSION)); + else + found = advanceTo(c, TrieUtil.directComparable(current), paths); + } + + if (delTimeIncrease > 0) + assertTrue(found); + + if (found) + while (c.advanceMultiple(null) != -1) + { + } // let the verification cursor check the correctness of the iteration + } + + ByteComparable maybeInvert(ByteComparable bc, Direction dir) + { + return dir.isForward() ? bc : InMemoryTriePutTest.invert(bc); + } + + private boolean advanceTo(RangeCursor c, ByteComparable target, TriePathReconstructor paths) + { + int cmp; + Direction dir = c.direction(); + while (true) + { + cmp = ByteComparable.compare(maybeInvert(target, dir), maybeInvert(ByteComparable.preencoded(VERSION, paths.keyBytes, 0, paths.keyPos), dir), VERSION); + if (cmp == 0) + return true; + if (cmp < 0) + return false; + if (c.advance() < 0) + return false; // exhausted + + paths.resetPathLength(c.depth() - 1); + paths.addPathByte(c.incomingTransition()); + } + } + + private boolean skipByDifference(Cursor cursor, ByteComparable a, ByteComparable b) + { + ByteSource.Peekable sa = a.asPeekableBytes(VERSION); + ByteSource.Peekable sb = b.asPeekableBytes(VERSION); + int depth = 0; + while (sa.peek() == sb.peek()) + { + sa.next(); + sb.next(); + ++depth; + } + final int nextByte = sb.next(); + int skippedDepth = cursor.skipTo(depth + 1, nextByte); + if (skippedDepth != depth + 1 || cursor.incomingTransition() != nextByte) + return false; + return cursor.descendAlong(sb); + } + + private void insertRanges(InMemoryRangeTrie trie, String[] insertions, int delTimeIncrease) throws TrieSpaceExhaustedException + { + for (int i = 0; i < insertions.length; i += 2) + { + trie.apply(RangeTrie.range(TrieUtil.directComparable(insertions[i]), + TrieUtil.directComparable(insertions[i + 1]), + VERSION, + toMarker(insertions[i], delTime)), + (existing, update) -> existing == null ? update : TestRangeState.combine(existing, update), + delTimeIncrease >= 0 ? Predicates.alwaysFalse() : Predicates.alwaysTrue()); // if we delete covered branches, we should be okay with no force copying +// Predicates.alwaysTrue()); + delTime += delTimeIncrease; + } + System.out.println("After inserting " + Arrays.toString(insertions) + ":\n" + trie.dump()); + } + + static ByteComparable withTerminator(int terminator, ByteComparable bc) + { + return ver -> ByteSource.withTerminator(terminator, bc.asComparableBytes(ver)); + } + + static ByteComparable leftBound(ByteComparable bc) + { + return withTerminator(ByteSource.LT_NEXT_COMPONENT, bc); + } + + static ByteComparable rightBound(ByteComparable bc) + { + return withTerminator(ByteSource.GT_NEXT_COMPONENT, bc); + } + + static ByteComparable key(ByteComparable bc) + { + return withTerminator(ByteSource.TERMINATOR, bc); + } + + + + static > void putRange(InMemoryRangeTrie trie, + ByteComparable key, + S value, + Trie.MergeResolver resolver) + { + try + { + trie.apply(RangeTrie.range(leftBound(key), rightBound(key), VERSION, value), + (existing, update) -> existing != null ? resolver.resolve(existing, update) : update, + Predicates.alwaysFalse()); + } + catch (TrieSpaceExhaustedException e) + { + throw Throwables.propagate(e); + } + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryRangeTrieThreadedTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryRangeTrieThreadedTest.java new file mode 100644 index 000000000000..092f36d76d18 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryRangeTrieThreadedTest.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.junit.BeforeClass; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.concurrent.OpOrder; + +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; + +public class InMemoryRangeTrieThreadedTest extends ThreadedTestBase> +{ + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + + + @Override + TestRangeState value(ByteComparable b) + { + return new TestRangeState(b, 1, 1, false); + } + + @Override + InMemoryRangeTrie makeTrie(OpOrder readOrder) + { + return InMemoryRangeTrie.longLived(VERSION, readOrder); + } + + @Override + void add(InMemoryRangeTrie trie, ByteComparable b, TestRangeState v, int iteration) throws TrieSpaceExhaustedException + { + ByteComparable left = ver -> ByteSource.withTerminator(ByteSource.LT_NEXT_COMPONENT, b.asComparableBytes(ver)); + ByteComparable right = ver -> ByteSource.withTerminator(ByteSource.GT_NEXT_COMPONENT, b.asComparableBytes(ver)); + if (iteration % 2 == 0) + { + trie.putRecursive(left, v, (x, y) -> y.asBoundary(Direction.FORWARD)); + trie.putRecursive(right, v, (x, y) -> y.asBoundary(Direction.REVERSE)); + } + else + trie.apply(RangeTrie.range(left, right, VERSION, v), (x, y) -> y, x -> true); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java new file mode 100644 index 000000000000..7eb35101721d --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Collection; +import java.util.function.Predicate; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.concurrent.OpOrder; + +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; + +public class InMemoryTrieConsistencyTest extends ConsistencyTestBase, InMemoryTrie> +{ + @Override + InMemoryTrie makeTrie(OpOrder readOrder) + { + return InMemoryTrie.longLived(VERSION, readOrder); + } + + @Override + Value value(ByteComparable b, ByteComparable cprefix, ByteComparable c, int add, int seqId) + { + return new Value(b.byteComparableAsString(VERSION), + (cprefix != null ? cprefix.byteComparableAsString(VERSION) : "") + c.byteComparableAsString(VERSION), add, seqId); + } + + @Override + Content metadata(ByteComparable b) + { + return new Metadata(b.byteComparableAsString(VERSION)); + } + + @Override + String pk(Content c) + { + return c.pk; + } + + @Override + String ck(Content c) + { + return ((Value) c).ck; + } + + @Override + int seq(Content c) + { + return ((Value) c).seq; + } + + @Override + int value(Content c) + { + return ((Value) c).value; + } + + @Override + int updateCount(Content c) + { + return ((Metadata) c).updateCount; + } + + @Override + Trie makeSingleton(ByteComparable b, Content content) + { + return Trie.singleton(b, VERSION, content); + } + + @Override + Trie withRootMetadata(Trie wrapped, Content metadata) + { + return TrieUtil.withRootMetadata(wrapped, metadata); + } + + @Override + Trie merge(Collection> tries, Trie.CollectionMergeResolver mergeResolver) + { + return Trie.merge(tries, mergeResolver); + } + + @Override + void apply(InMemoryTrie trie, Trie mutation, InMemoryBaseTrie.UpsertTransformer mergeResolver, Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException + { + trie.apply(mutation, mergeResolver, forcedCopyChecker); + } + + @Override + boolean isPartition(Content c) + { + return c != null && c.isPartition(); + } + + @Override + Content mergeMetadata(Content c1, Content c2) + { + return ((Metadata) c1).mergeWith((Metadata) c2); + } + + @Override + void printStats(InMemoryTrie trie, Predicate> forcedCopyChecker) + { + System.out.format("Reuse %s %s atomicity %s on-heap %,d (+%,d) off-heap %,d\n", + trie.cellAllocator.getClass().getSimpleName(), + trie.bufferType, + forcedCopyChecker == NO_ATOMICITY ? "none" : + forcedCopyChecker == FORCE_ATOMIC ? "atomic" : "consistent partition", + trie.usedSizeOnHeap(), + trie.unusedReservedOnHeapMemory(), + trie.usedSizeOffHeap()); + } + + abstract static class Content + { + final String pk; + + Content(String pk) + { + this.pk = pk; + } + + abstract boolean isPartition(); + } + + static class Value extends Content + { + final String ck; + final int value; + final int seq; + + Value(String pk, String ck, int value, int seq) + { + super(pk); + this.ck = ck; + this.value = value; + this.seq = seq; + } + + @Override + public String toString() + { + return "Value{" + + "pk='" + pk + '\'' + + ", ck='" + ck + '\'' + + ", value=" + value + + ", seq=" + seq + + '}'; + } + + @Override + boolean isPartition() + { + return false; + } + } + + static class Metadata extends Content + { + int updateCount; + + Metadata(String pk) + { + super(pk); + updateCount = 1; + } + + @Override + boolean isPartition() + { + return true; + } + + Metadata mergeWith(Metadata other) + { + Metadata m = new Metadata(pk); + m.updateCount = updateCount + other.updateCount; + return m; + } + + @Override + public String toString() + { + return "Metadata{" + + "pk='" + pk + '\'' + + ", updateCount=" + updateCount + + '}'; + } + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java index da9f1b6d527a..65da6edb5ed4 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java @@ -18,589 +18,29 @@ package org.apache.cassandra.db.tries; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Predicate; - -import org.junit.Assert; -import org.junit.Test; - import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.apache.cassandra.utils.concurrent.OpOrder; import static org.apache.cassandra.db.tries.TrieUtil.VERSION; -import static org.apache.cassandra.db.tries.TrieUtil.generateKeys; -public class InMemoryTrieThreadedTest +public class InMemoryTrieThreadedTest extends ThreadedTestBase> { - // Note: This should not be run by default with verification to have the higher concurrency of faster writes and reads. - - private static final int COUNT = 30000; - private static final int OTHERS = COUNT / 10; - private static final int PROGRESS_UPDATE = COUNT / 15; - private static final int READERS = 8; - private static final int WALKERS = 2; - private static final Random rand = new Random(); - - /** - * Force copy every modified cell below the partition/enumeration level. Provides atomicity of mutations within the - * partition level as well as consistency. - */ - public static final Predicate> FORCE_COPY_PARTITION = features -> isPartition(features.content()); - /** - * Force copy every modified cell below the earliest branching point. Provides atomicity of mutations at any level, - * but readers/walkers may see inconsistent views of the data, in the sense that older mutations may be missed - * while newer ones are returned. - */ - public static final Predicate> FORCE_ATOMIC = features -> features.isBranching(); - /** - * Do not do any additional copying beyond what is required to build the tries safely for concurrent readers. - * Mutations may be partially seen by readers, and older mutations may be missed while newer ones are returned. - */ - public static final Predicate> NO_ATOMICITY = features -> false; - - static Value value(ByteComparable b, ByteComparable cprefix, ByteComparable c, int add, int seqId) - { - return new Value(b.byteComparableAsString(VERSION), - (cprefix != null ? cprefix.byteComparableAsString(VERSION) : "") + c.byteComparableAsString(VERSION), add, seqId); - } - - static String value(ByteComparable b) + @Override + String value(ByteComparable b) { return b.byteComparableAsString(VERSION); } - @Test - public void testThreaded() throws InterruptedException - { - OpOrder readOrder = new OpOrder(); - ByteComparable[] src = generateKeys(rand, COUNT + OTHERS); - InMemoryTrie trie = InMemoryTrie.longLived(VERSION, readOrder); - ConcurrentLinkedQueue errors = new ConcurrentLinkedQueue<>(); - List threads = new ArrayList<>(); - AtomicBoolean writeCompleted = new AtomicBoolean(false); - AtomicInteger writeProgress = new AtomicInteger(0); - - for (int i = 0; i < WALKERS; ++i) - threads.add(new Thread(() -> { - try - { - while (!writeCompleted.get()) - { - int min = writeProgress.get(); - int count = 0; - try (OpOrder.Group group = readOrder.start()) - { - for (Map.Entry en : trie.entrySet()) - { - String v = value(en.getKey()); - Assert.assertEquals(en.getKey().byteComparableAsString(VERSION), v, en.getValue()); - ++count; - } - } - Assert.assertTrue("Got only " + count + " while progress is at " + min, count >= min); - } - } - catch (Throwable t) - { - t.printStackTrace(); - errors.add(t); - } - })); - - for (int i = 0; i < READERS; ++i) - { - threads.add(new Thread(() -> { - try - { - Random r = ThreadLocalRandom.current(); - while (!writeCompleted.get()) - { - int min = writeProgress.get(); - - for (int i1 = 0; i1 < PROGRESS_UPDATE; ++i1) - { - int index = r.nextInt(COUNT + OTHERS); - ByteComparable b = src[index]; - String v = value(b); - try (OpOrder.Group group = readOrder.start()) - { - String result = trie.get(b); - if (result != null) - { - Assert.assertTrue("Got not added " + index + " when COUNT is " + COUNT, - index < COUNT); - Assert.assertEquals("Failed " + index, v, result); - } - else if (index < min) - Assert.fail("Failed index " + index + " while progress is at " + min); - } - } - } - } - catch (Throwable t) - { - t.printStackTrace(); - errors.add(t); - } - })); - } - - threads.add(new Thread(() -> { - try - { - for (int i = 0; i < COUNT; i++) - { - ByteComparable b = src[i]; - - // Note: Because we don't ensure order when calling resolve, just use a hash of the key as payload - // (so that all sources have the same value). - String v = value(b); - trie.putSingleton(b, v, (x, y) -> y, i % 2 != 0); - - if (i % PROGRESS_UPDATE == 0) - writeProgress.set(i); - } - } - catch (Throwable t) - { - t.printStackTrace(); - errors.add(t); - } - finally - { - writeCompleted.set(true); - } - })); - - for (Thread t : threads) - t.start(); - - for (Thread t : threads) - t.join(); - - if (!errors.isEmpty()) - Assert.fail("Got errors:\n" + errors); - } - - static abstract class Content - { - final String pk; - - Content(String pk) - { - this.pk = pk; - } - - abstract boolean isPartition(); - } - - static class Value extends Content - { - final String ck; - final int value; - final int seq; - - Value(String pk, String ck, int value, int seq) - { - super(pk); - this.ck = ck; - this.value = value; - this.seq = seq; - } - - @Override - public String toString() - { - return "Value{" + - "pk='" + pk + '\'' + - ", ck='" + ck + '\'' + - ", value=" + value + - ", seq=" + seq + - '}'; - } - - @Override - boolean isPartition() - { - return false; - } - } - - static class Metadata extends Content - { - int updateCount; - - Metadata(String pk) - { - super(pk); - updateCount = 1; - } - - @Override - boolean isPartition() - { - return true; - } - - Metadata mergeWith(Metadata other) - { - Metadata m = new Metadata(pk); - m.updateCount = updateCount + other.updateCount; - return m; - } - - @Override - public String toString() - { - return "Metadata{" + - "pk='" + pk + '\'' + - ", updateCount=" + updateCount + - '}'; - } - } - - static boolean isPartition(Content c) - { - return c != null && c.isPartition(); - } - - @Test - public void testConsistentUpdates() throws Exception - { - // Check that multi-path updates with below-partition-level copying are safe for concurrent readers, - // and that content is atomically applied, i.e. that reader see either nothing from the update or all of it, - // and consistent, i.e. that it is not possible to receive some newer updates while missing - // older ones. (For example, if the sequence of additions is 3, 1, 5, without this requirement a reader - // could see an enumeration which lists 3 and 5 but not 1.) - testAtomicUpdates(3, FORCE_COPY_PARTITION, true, true); - // Note: using 3 per mutation, so that the first and second update fit in a sparse in-memory trie block. - } - - @Test - public void testAtomicUpdates() throws Exception - { - // Check that multi-path updates with below-branching-point copying are safe for concurrent readers, - // and that content is atomically applied, i.e. that reader see either nothing from the update or all of it. - testAtomicUpdates(3, FORCE_ATOMIC, true, false); - } - @Test - public void testSafeUpdates() throws Exception + @Override + InMemoryTrie makeTrie(OpOrder readOrder) { - // Check that multi path updates without additional copying are safe for concurrent readers. - testAtomicUpdates(3, NO_ATOMICITY, false, false); + return InMemoryTrie.longLived(VERSION, readOrder); } - @Test - public void testConsistentSinglePathUpdates() throws Exception + @Override + void add(InMemoryTrie trie, ByteComparable b, String v, int iteration) throws TrieSpaceExhaustedException { - // Check that single path updates with below-partition-level copying are safe for concurrent readers, - // and that content is consistent, i.e. that it is not possible to receive some newer updates while missing - // older ones. (For example, if the sequence of additions is 3, 1, 5, without this requirement a reader - // could see an enumeration which lists 3 and 5 but not 1.) - testAtomicUpdates(1, FORCE_COPY_PARTITION, true, true); - } - - - @Test - public void testAtomicSinglePathUpdates() throws Exception - { - // When doing single path updates atomicity comes for free. This only checks that the branching checker is - // not doing anything funny. - testAtomicUpdates(1, FORCE_ATOMIC, true, false); - } - - @Test - public void testSafeSinglePathUpdates() throws Exception - { - // Check that single path updates without additional copying are safe for concurrent readers. - testAtomicUpdates(1, NO_ATOMICITY, true, false); - } - - // The generated keys all start with NEXT_COMPONENT, which makes it impossible to test the precise behavior of the - // partition-level force copying. Strip that byte. - private static ByteComparable[] skipFirst(ByteComparable[] keys) - { - ByteComparable[] result = new ByteComparable[keys.length]; - for (int i = 0; i < keys.length; ++i) - result[i] = skipFirst(keys[i]); - return result; - } - - private static ByteComparable skipFirst(ByteComparable key) - { - return v -> { - var bs = key.asComparableBytes(v); - int n = bs.next(); - assert n != ByteSource.END_OF_STREAM; - return bs; - }; - } - - public void testAtomicUpdates(int PER_MUTATION, - Predicate> forcedCopyChecker, - boolean checkAtomicity, - boolean checkSequence) - throws Exception - { - ByteComparable[] ckeys = skipFirst(generateKeys(rand, COUNT)); - ByteComparable[] pkeys = skipFirst(generateKeys(rand, Math.min(100, COUNT / 10))); // to guarantee repetition - - /* - * Adds COUNT partitions each with perPartition separate clusterings, where the sum of the values - * of all clusterings is 0. - * If the sum for any walk covering whole partitions is non-zero, we have had non-atomic updates. - */ - - OpOrder readOrder = new OpOrder(); -// InMemoryTrie trie = new InMemoryTrie<>(new MemtableAllocationStrategy.NoReuseStrategy(BufferType.OFF_HEAP)); - InMemoryTrie trie = InMemoryTrie.longLived(VERSION, readOrder); - ConcurrentLinkedQueue errors = new ConcurrentLinkedQueue<>(); - List threads = new ArrayList(); - AtomicBoolean writeCompleted = new AtomicBoolean(false); - AtomicInteger writeProgress = new AtomicInteger(0); - - for (int i = 0; i < WALKERS; ++i) - threads.add(new Thread() - { - public void run() - { - try - { - Random r = ThreadLocalRandom.current(); - while (!writeCompleted.get()) - { - int min = writeProgress.get(); - try (OpOrder.Group group = readOrder.start()) - { - Iterable> entries = trie.entrySet(); - checkEntries("", min, true, checkAtomicity, false, PER_MUTATION, entries); - } - } - } - catch (Throwable t) - { - t.printStackTrace(); - errors.add(t); - } - } - }); - - for (int i = 0; i < READERS; ++i) - { - ByteComparable[] srcLocal = pkeys; - threads.add(new Thread() - { - public void run() - { - try - { - // await at least one ready partition - while (writeProgress.get() == 0) {} - - Random r = ThreadLocalRandom.current(); - while (!writeCompleted.get()) - { - ByteComparable key = srcLocal[r.nextInt(srcLocal.length)]; - int min = writeProgress.get() / (pkeys.length * PER_MUTATION) * PER_MUTATION; - Iterable> entries; - - try (OpOrder.Group group = readOrder.start()) - { - entries = trie.tailTrie(key).entrySet(); - checkEntries(" in tail " + key.byteComparableAsString(VERSION), min, false, checkAtomicity, checkSequence, PER_MUTATION, entries); - } - - try (OpOrder.Group group = readOrder.start()) - { - entries = trie.subtrie(key, nextBranch(key)).entrySet(); - checkEntries(" in branch " + key.byteComparableAsString(VERSION), min, true, checkAtomicity, checkSequence, PER_MUTATION, entries); - } - } - } - catch (Throwable t) - { - t.printStackTrace(); - errors.add(t); - } - } - }); - } - - threads.add(new Thread() - { - public void run() - { - ThreadLocalRandom r = ThreadLocalRandom.current(); - final Trie.CollectionMergeResolver mergeResolver = new Trie.CollectionMergeResolver() - { - @Override - public Content resolve(Content c1, Content c2) - { - if (c1.isPartition() && c2.isPartition()) - return ((Metadata) c1).mergeWith((Metadata) c2); - throw new AssertionError("Test error, keys should be distinct."); - } - - public Content resolve(Collection contents) - { - return contents.stream().reduce(this::resolve).get(); - } - }; - - try - { - int lastUpdate = 0; - for (int i = 0; i < COUNT; i += PER_MUTATION) - { - ByteComparable b = pkeys[(i / PER_MUTATION) % pkeys.length]; - Metadata partitionMarker = new Metadata(b.byteComparableAsString(VERSION)); - ByteComparable cprefix = null; - if (r.nextBoolean()) - cprefix = ckeys[i]; // Also test branching point below the partition level - - List> sources = new ArrayList<>(); - for (int j = 0; j < PER_MUTATION; ++j) - { - - ByteComparable k = ckeys[i + j]; - Trie row = Trie.singleton(k, VERSION, - value(b, cprefix, k, - j == 0 ? -PER_MUTATION + 1 : 1, - (i / PER_MUTATION / pkeys.length) * PER_MUTATION + j)); - - if (cprefix != null) - row = row.prefixedBy(cprefix); - - row = withRootMetadata(row, partitionMarker); - row = row.prefixedBy(b); - sources.add(row); - } - - final Trie mutation = Trie.merge(sources, mergeResolver); - - trie.apply(mutation, - (existing, update) -> existing == null ? update : mergeResolver.resolve(existing, update), - forcedCopyChecker); - - if (i >= pkeys.length * PER_MUTATION && i - lastUpdate >= PROGRESS_UPDATE) - { - writeProgress.set(i); - lastUpdate = i; - } - } - } - catch (Throwable t) - { - t.printStackTrace(); - errors.add(t); - } - finally - { - writeCompleted.set(true); - } - } - }); - - for (Thread t : threads) - t.start(); - - for (Thread t : threads) - t.join(); - - System.out.format("Reuse %s %s atomicity %s on-heap %,d (+%,d) off-heap %,d\n", - trie.cellAllocator.getClass().getSimpleName(), - trie.bufferType, - forcedCopyChecker == NO_ATOMICITY ? "none" : - forcedCopyChecker == FORCE_ATOMIC ? "atomic" : "consistent partition", - trie.usedSizeOnHeap(), - trie.unusedReservedOnHeapMemory(), - trie.usedSizeOffHeap()); - - if (!errors.isEmpty()) - Assert.fail("Got errors:\n" + errors); - } - - static ByteComparable nextBranch(ByteComparable key) - { - return version -> { - byte[] bytes = key.asByteComparableArray(version); - int last = bytes.length - 1; - while (last >= 0 && bytes[last] == ((byte) 0xFF)) - --last; - if (last < 0) - return null; - ++bytes[last]; - return ByteSource.preencoded(bytes, 0, last + 1); - }; - } - - static Trie withRootMetadata(Trie wrapped, T metadata) - { - return wrapped.mergeWith(Trie.singleton(ByteComparable.EMPTY, VERSION, metadata), Trie.throwingResolver()); - } - - public void checkEntries(String location, - int min, - boolean usePk, - boolean checkAtomicity, - boolean checkConsecutiveIds, - int PER_MUTATION, - Iterable> entries) - { - long sum = 0; - int count = 0; - long idSum = 0; - long idMax = 0; - int updateCount = 0; - for (var en : entries) - { - String path = en.getKey().byteComparableAsString(VERSION); - if (en.getValue().isPartition()) - { - Metadata m = (Metadata) en.getValue(); - Assert.assertEquals("Partition metadata" + location, (usePk ? m.pk : ""), path); - updateCount += m.updateCount; - continue; - } - final Value value = (Value) en.getValue(); - String valueKey = (usePk ? value.pk : "") + value.ck; - Assert.assertEquals(location, valueKey, path); - ++count; - sum += value.value; - int seq = value.seq; - idSum += seq; - if (seq > idMax) - idMax = seq; - } - - Assert.assertTrue("Values" + location + " should be at least " + min + ", got " + count, min <= count); - - if (checkAtomicity) - { - // If mutations apply atomically, the row count is always a multiple of the mutation size... - Assert.assertTrue("Values" + location + " should be a multiple of " + PER_MUTATION + ", got " + count, count % PER_MUTATION == 0); - // ... and the sum of the values is 0 (as the sum for each individual mutation is 0). - Assert.assertEquals("Value sum" + location, 0, sum); - } - - if (checkConsecutiveIds) - { - // The update count reflected in the partition metadata must match the row count. - Assert.assertEquals("Update count" + location, count, updateCount); - // If mutations apply consistently for the partition, for any row we see we have to have seen all rows that - // were applied before that. In other words, the id sum should be the sum of the integers from 1 to the - // highest id seen in the partition. - Assert.assertEquals("Id sum" + location, idMax * (idMax + 1) / 2, idSum); - } + trie.putSingleton(b, v, (x, y) -> y, iteration % 2 != 0); } } diff --git a/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java b/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java index 4c91222b6a77..24b1c0bc6eef 100644 --- a/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java @@ -357,7 +357,7 @@ public void testRangesOnCollectionMerge2() throws TrieSpaceExhaustedException { List> inputs = ImmutableList.of(fromList(0, 1, 2, 3, 5, 8, 9, 13, 14), fromList(4, 6, 7, 9, 10, 11, 12, 13)); - testIntersections(dir -> new CollectionMergeCursor<>(RESOLVER, dir, inputs, Trie::cursor)); + testIntersections(dir -> new CollectionMergeCursor.Plain<>(RESOLVER, dir, inputs, Trie::cursor)); } } @@ -515,4 +515,44 @@ private static InMemoryTrie duplicateTrie(Trie trie) throw new AssertionError(e); } } + + @Test + public void testReturnsContentOnPrefix() throws TrieSpaceExhaustedException + { + TrieSet set = TrieSet.singleton(VERSION, TrieUtil.directComparable("abc")); + Trie trie = TrieUtil.directTrie("a", "aa", "ab", "abc", "cd"); + Trie expected = TrieUtil.directTrie("a", "ab", "abc"); + assertMapEquals(expected.entrySet(Direction.FORWARD), trie.intersect(set).entrySet(Direction.FORWARD), TrieUtil.FORWARD_COMPARATOR); + assertMapEquals(expected.entrySet(Direction.REVERSE), trie.intersect(set).entrySet(Direction.REVERSE), TrieUtil.REVERSE_COMPARATOR); + assertEquals(expected.process(Direction.FORWARD, new TrieDumper<>(Object::toString)), trie.intersect(set).dump()); + } + + @Test + public void testReturnsBranchContents() throws TrieSpaceExhaustedException + { + TrieSet set = TrieSet.singleton(VERSION, TrieUtil.directComparable("abc")); + Trie trie = TrieUtil.directTrie("aaa", "abc", "abce", "abcfff", "bcd"); + Trie expected = TrieUtil.directTrie("abc", "abce", "abcfff"); + assertMapEquals(expected.entrySet(Direction.FORWARD), trie.intersect(set).entrySet(Direction.FORWARD), TrieUtil.FORWARD_COMPARATOR); + assertMapEquals(expected.entrySet(Direction.REVERSE), trie.intersect(set).entrySet(Direction.REVERSE), TrieUtil.REVERSE_COMPARATOR); + assertEquals(expected.process(Direction.FORWARD, new TrieDumper<>(Object::toString)), trie.intersect(set).dump()); + } + + @Test(expected = AssertionError.class) + public void testRangeUnderCoveredBranch() throws TrieSpaceExhaustedException + { + TrieSet set1 = TrieSet.singleton(VERSION, TrieUtil.directComparable("b")); + TrieSet set2 = TrieUtil.directRanges("aa", "ab", "bc", "bd", "ce", "cf"); + TrieSet expected = TrieUtil.directRanges("bc", "bd"); + assertEquals(expected.dump(), set1.intersection(set2).dump()); + } + + @Test(expected = AssertionError.class) + public void testRangeUnderCoveredRoot() throws TrieSpaceExhaustedException + { + TrieSet set1 = TrieSet.singleton(VERSION, ByteComparable.EMPTY); + TrieSet set2 = TrieUtil.directRanges("aa", "ab", "bc", "bd", "ce", "cf"); + TrieSet expected = set2; + assertEquals(expected.dump(), set1.intersection(set2).dump()); + } } diff --git a/test/unit/org/apache/cassandra/db/tries/RangeTrieIntersectionTest.java b/test/unit/org/apache/cassandra/db/tries/RangeTrieIntersectionTest.java new file mode 100644 index 000000000000..994e9bb2ca09 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/RangeTrieIntersectionTest.java @@ -0,0 +1,464 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Arrays; +import java.util.List; + +import com.google.common.collect.Lists; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +import static java.util.Arrays.asList; +import static org.apache.cassandra.db.tries.TestRangeState.fromList; +import static org.apache.cassandra.db.tries.TestRangeState.toList; +import static org.junit.Assert.assertEquals; + +public class RangeTrieIntersectionTest +{ + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + + static final int bitsNeeded = 4; + int bits = bitsNeeded; + + /** Creates a {@link ByteComparable} for the provided value by splitting the integer in sequences of "bits" bits. */ + private ByteComparable of(int value) + { + assert value >= 0 && value <= Byte.MAX_VALUE; + + byte[] splitBytes = new byte[(bitsNeeded + bits - 1) / bits]; + int pos = 0; + int mask = (1 << bits) - 1; + for (int i = bitsNeeded - bits; i > 0; i -= bits) + splitBytes[pos++] = (byte) ((value >> i) & mask); + + splitBytes[pos] = (byte) (value & mask); + return ByteComparable.preencoded(TrieUtil.VERSION, splitBytes); + } + + private TestRangeState from(int where, int value) + { + return new TestRangeState(of(where), -1, value, true); + } + + private TestRangeState to(int where, int value) + { + return new TestRangeState(of(where), value, -1, true); + } + + private TestRangeState change(int where, int from, int to) + { + return new TestRangeState(of(where), from, to, true); + } + + private TrieSet range(ByteComparable left, ByteComparable right) + { + return TrieSet.range(TrieUtil.VERSION, left, right); + } + + private TrieSet ranges(ByteComparable... bounds) + { + return TrieSet.ranges(TrieUtil.VERSION, bounds); + } + + @Test + public void testSubtrie() + { + for (bits = bitsNeeded; bits > 0; --bits) + { + RangeTrie trie = fromList(asList(from(1, 10), to(4, 10), from(6, 11), change(8, 11, 12), to(10, 12))); + + System.out.println(trie.dump()); + assertEquals("No intersection", asList(from(1, 10), to(4, 10), from(6, 11), change(8, 11, 12), to(10, 12)), toList(trie, Direction.FORWARD)); + + testIntersection("all", + asList(from(1, 10), to(4, 10), from(6, 11), change(8, 11, 12), to(10, 12)), + trie, + range(null, null)); + testIntersection("fully covered range", + asList(from(1, 10), to(4, 10)), + trie, + range(of(0), of(5))); + testIntersection("fully covered range", + asList(from(6, 11), change(8, 11, 12), to(10, 12)), + trie, + range(of(5), of(13))); + testIntersection("matching range", + asList(from(1, 10), to(4, 10)), + trie, + range(of(1), of(4))); + testIntersection("touching empty", + asList(), + trie, + range(of(4), of(6))); + + testIntersection("partial left", + asList(from(2, 10), to(4, 10)), + trie, + range(of(2), of(5))); + testIntersection("partial left on change", + asList(from(8, 12), to(10, 12)), + trie, + range(of(8), of(12))); + testIntersection("partial left with null", + asList(from(9, 12), to(10, 12)), + trie, + range(of(9), null)); + + + testIntersection("partial right", + asList(from(6, 11), to(7, 11)), + trie, + range(of(5), of(7))); + testIntersection("partial right on change", + asList(from(6, 11), to(8, 11)), + trie, + range(of(5), of(8))); + testIntersection("partial right with null", + asList(from(1, 10), to(2, 10)), + trie, + range(null, of(2))); + + testIntersection("inside range", + asList(from(2, 10), to(3, 10)), + trie, + range(of(2), of(3))); + testIntersection("inside with change", + asList(from(7, 11), change(8, 11, 12), to(9, 12)), + trie, + range(of(7), of(9))); + + testIntersection("empty range inside", + asList(), + trie, + range(of(7), of(7))); + } + } + + @Test + public void testRanges() + { + for (bits = bitsNeeded; bits > 0; --bits) + { + RangeTrie trie = fromList(asList(from(1, 10), to(4, 10), from(6, 11), change(8, 11, 12), to(10, 12))); + + testIntersection("fully covered ranges", + asList(from(1, 10), to(4, 10), from(6, 11), change(8, 11, 12), to(10, 12)), + trie, + ranges(of(0), of(5), of(5), of(13))); + testIntersection("matching ranges", + asList(from(1, 10), to(4, 10), from(6, 11), change(8, 11, 12), to(10, 12)), + trie, + ranges(of(1), of(4), of(6), of(11))); + testIntersection("touching empty", + asList(), + trie, + ranges(of(0), of(1), of(4), of(6), of(12), of(15))); + testIntersection("partial left", + asList(from(2, 10), to(4, 10), from(9, 12), to(10, 12)), + trie, + ranges(of(2), of(5), of(9), null)); + + testIntersection("partial right", + asList(from(1, 10), to(2, 10), from(6, 11), to(7, 11)), + trie, + ranges(null, of(2), of(5), of(7))); + + testIntersection("inside ranges", + asList(from(2, 10), to(3, 10), from(7, 11), change(8, 11, 12), to(9, 12)), + trie, + ranges(of(2), of(3), of(7), of(9))); + + testIntersection("jumping inside", + asList(from(1, 10), to(2, 10), from(3, 10), to(4, 10), from(7, 11), to(8, 11), from(9, 12), to(10, 12)), + trie, + ranges(of(1), of(2), of(3), of(4), of(5), of(6), of(7), of(8), of(9), of(10))); + } + } + + @Test + public void testRangeOnSubtrie() + { + for (bits = bitsNeeded; bits > 0; --bits) + { + RangeTrie trie = fromList(asList(from(1, 10), to(4, 10), from(6, 11), change(8, 11, 12), to(10, 12), from(13, 13), to(14, 13))); + + // non-overlapping + testIntersection("", asList(), trie, range(of(0), of(3)), range(of(4), of(7))); + // touching, i.e. still non-overlapping + testIntersection("", asList(), trie, range(of(0), of(3)), range(of(3), of(7))); + // overlapping 1 + testIntersection("", asList(from(2, 10), to(3, 10)), trie, range(of(0), of(3)), range(of(2), of(7))); + // overlapping 2 + testIntersection("", asList(from(1, 10), to(3, 10)), trie, range(of(0), of(3)), range(of(1), of(7))); + // covered + testIntersection("", asList(from(1, 10), to(3, 10)), trie, range(of(0), of(3)), range(of(0), of(7))); + // covered + testIntersection("", asList(from(3, 10), to(4, 10), from(6, 11), to(7, 11)), trie, range(of(3), of(7)), range(of(0), of(7))); + // covered 2 + testIntersection("", asList(from(1, 10), to(3, 10)), trie, range(of(1), of(3)), range(of(0), of(7))); + } + } + + @Test + public void testRangesOnRanges() + { + for (bits = bitsNeeded; bits > 0; --bits) + testIntersections(fromList(asList(from(1, 10), to(4, 10), from(6, 11), change(8, 11, 12), to(10, 12), from(13, 13), to(14, 13)))); + } + + private void testIntersections(RangeTrie trie) + { + System.out.println(trie.dump()); + testIntersection("", asList(from(1, 10), to(4, 10), from(6, 11), change(8, 11, 12), to(10, 12), from(13, 13), to(14, 13)), trie); + + TrieSet set1 = ranges(null, of(4), of(5), of(9), of(12), null); + TrieSet set2 = ranges(of(2), of(7), of(8), of(10), of(12), of(14)); + TrieSet set3 = ranges(of(1), of(2), of(3), of(4), of(5), of(6), of(7), of(8), of(9), of(10)); + + testIntersections(trie, set1, set2, set3); + + testSetAlgebraIntersection(trie); + } + + private void testSetAlgebraIntersection(RangeTrie trie) + { + TrieSet set1 = range(null, of(3)) + .union(range(of(2), of(4))) + .union(range(of(5), of(7))) + .union(range(of(7), of(9))) + .union(range(of(14), of(16))) + .union(range(of(12), null)); + TrieSet set2 = range(of(2), of(7)) + .union(ranges(null, of(8), of(10), null).weakNegation()) + .union(ranges(of(8), of(10), of(12), of(14))); + TrieSet set3 = range(of(1), of(2)) + .union(range(of(3), of(4))) + .union(range(of(5), of(6))) + .union(range(of(7), of(8))) + .union(range(of(9), of(10))); + + System.out.println("Set 0:\n" + set1.dump()); + System.out.println("Set 1:\n" + set2.dump()); + System.out.println("Set 2:\n" + set3.dump()); + + testIntersections(trie, set1, set2, set3); + } + + private void testIntersections(RangeTrie trie, TrieSet set1, TrieSet set2, TrieSet set3) + { + // set1 = ranges(null, of(4), of(5), of(9), of(12), null); + // set2 = ranges(of(2), of(7), of(8), of(10), of(12), of(14)); + // set3 = ranges(of(1), of(2), of(3), of(4), of(5), of(6), of(7), of(8), of(9), of(10)); + // from(1, 10), to(4, 10), from(6, 11), change(8, 11, 12), to(10, 12), from(13, 13), to(14, 13) + testIntersection("1", asList(from(1, 10), to(4, 10), + from(6, 11), change(8, 11, 12), to(9, 12), + from(13, 13), to(14,13)), trie, set1); + + testIntersection("2", asList(from(2, 10), to(4, 10), + from(6, 11), to(7, 11), + from(8, 12), to(10, 12), + from(13, 13), to(14, 13)), trie, set2); + + testIntersection("3", asList(from(1, 10), to(2, 10), + from(3, 10), to(4, 10), + from(7, 11), to(8, 11), + from(9, 12), to(10, 12)), trie, set3); + + testIntersection("12", asList(from(2, 10), to(4, 10), + from(6, 11), to(7, 11), + from(8, 12), to(9, 12), + from(13, 13), to(14, 13)), trie, set1, set2); + + testIntersection("13", asList(from(1, 10), to(2, 10), + from(3, 10), to(4, 10), + from(7, 11), to(8, 11)), trie, set1, set3); + + testIntersection("23", asList(from(3, 10), to(4, 10), + from(9, 12), to(10, 12)), trie, set2, set3); + + testIntersection("123", asList(from(3, 10), to(4, 10)), trie, set1, set2, set3); + } + + public void testIntersection(String message, List expected, RangeTrie trie, TrieSet... sets) + { + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + try + { + assertEquals(message + " forward b" + bits, expected, toList(trie, Direction.FORWARD)); + assertEquals(message + " reverse b" + bits, Lists.reverse(expected), toList(trie, Direction.REVERSE)); + } + catch (AssertionError e) + { + System.out.println("\nFORWARD:\n" + trie.dump(TestRangeState::toStringNoPosition)); + System.out.println("\nREVERSE:\n" + trie.cursor(Direction.REVERSE).process(new TrieDumper<>(TestRangeState::toStringNoPosition))); + throw e; + } + } + else + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + TrieSet set = sets[toRemove]; + testIntersection(message + " " + toRemove, expected, + trie.intersect(set), + Arrays.stream(sets) + .filter(x -> x != set) + .toArray(TrieSet[]::new) + ); + } + } + } + + @Test + public void testRangeMethod() + { + RangeTrie trie = RangeTrie.range(TrieUtil.directComparable("aa"), + TrieUtil.directComparable("bb"), + TrieUtil.VERSION, + new TestRangeState(ByteComparable.EMPTY, 1, 1, false)); + RangeTrie expected = TrieUtil.directRangeTrie("aa", "bb"); + TrieUtil.verifyEqualRangeTries(trie, expected); + } + + @Test + public void testSkipToSimple() + { + String[] ranges1 = {"aaa", "ddd"}; + String[] ranges2 = {"bbb", "eee"}; + String[] ixranges = {"bbb", "ddd"}; + String[] points = {"___", "aaa", "bbb", "ccc", "ddd", "eee"}; + testIntersectionSkipTo(ranges1, ranges2, ixranges, points); + } + + @Test + public void testRangeUnderCoveredBranchPoint() + { + String[] ranges1 = {"ba", "bc"}; + String[] ranges2 = {"aa", "ab", "bbc", "bbd", "bbfff", "bbffg", "bde", "bdf", "ce", "cf"}; + String[] expected2 = {"bbc", "bbd", "bbfff", "bbffg"}; + String[] ranges3 = {"bbfe", "bbfg"}; + String[] expected3 = {"bbfff", "bbffg"}; + testDirectIntersections(ranges1, ranges2, expected2, ranges3, expected3); + } + + @Test + public void testRangeUnderCoveredBranchRight() + { + String[] ranges1 = {"_a", "_b", "abba", "abf", "d", "e"}; + String[] ranges2 = {"aaa", "aab", "abc", "abd", "abef", "abeg", "abehhh", "abehhi", "ce", "cf"}; + String[] expected2 = {"abc", "abd", "abef", "abeg", "abehhh", "abehhi"}; + String[] ranges3 = {"abehg", "abehi"}; + String[] expected3 = {"abehhh", "abehhi"}; + testDirectIntersections(ranges1, ranges2, expected2, ranges3, expected3); + } + + @Test + public void testRangeUnderCoveredBranchLeft() + { + String[] ranges1 = {"_a", "_b", "abb_", "abe", "d", "e"}; + String[] ranges2 = {"aaa", "aab", "abbac", "abbad", "abbafff", "abbaffg", "abc", "abd", "ce", "cf"}; + String[] expected2 = {"abbac", "abbad", "abbafff", "abbaffg", "abc", "abd"}; + String[] ranges3 = {"abbafe", "abbafg"}; + String[] expected3 = {"abbafff", "abbaffg"}; + testDirectIntersections(ranges1, ranges2, expected2, ranges3, expected3); + } + + private void testDirectIntersections(String[] ranges1, String[] ranges2, String[] expected2, String[] ranges3, String[] expected3) + { + testDirectIntersectionsRangeSet(ranges1, ranges2, expected2, ranges3, expected3); + testDirectIntersectionsRangeSet(ranges2, ranges1, expected2, ranges3, expected3); + } + + private void testDirectIntersectionsRangeSet(String[] ranges1, String[] ranges2, String[] expected2, String[] ranges3, String[] expected3) + { + RangeTrie set1 = TrieUtil.directRangeTrie(ranges1); + TrieSet set2 = TrieUtil.directRanges(ranges2); + RangeTrie expected = TrieUtil.directRangeTrie(expected2); + TrieUtil.verifyEqualRangeTries(set1.intersect(set2), expected); + String[] allpoints = Arrays.asList(ranges1, ranges2, expected2, ranges3, expected3) + .stream() + .flatMap(Arrays::stream) + .distinct() + .toArray(String[]::new); + verifyIntersectionContainsCorrectness(allpoints, set1, set2); + // check skipTo in a covered branch + TrieSet set3 = TrieUtil.directRanges(ranges3); + expected = TrieUtil.directRangeTrie(expected3); + TrieUtil.verifyEqualRangeTries(set1.intersect(set2).intersect(set3), expected); + TrieUtil.verifyEqualRangeTries(set1.intersect(set3.intersection(set2)), expected); + verifyIntersectionContainsCorrectness(allpoints, set1.intersect(set2), set3); + verifyIntersectionContainsCorrectness(allpoints, set1, set3.intersection(set2)); + } + + private void testIntersectionSkipTo(String[] ranges1, String[] ranges2, String[] ixranges, String[] points) + { + testIntersectionSkipToRangeSet(ranges1, ranges2, ixranges, points); + testIntersectionSkipToRangeSet(ranges2, ranges1, ixranges, points); + } + + private void testIntersectionSkipToRangeSet(String[] ranges1, String[] ranges2, String[] ixranges, String[] points) + { + RangeTrie set1 = TrieUtil.directRangeTrie(ranges1); + TrieSet set2 = TrieUtil.directRanges(ranges2); + RangeTrie ix = TrieUtil.directRangeTrie(ixranges); + TrieUtil.verifyEqualRangeTries(set1.intersect(set2), ix); + + verifyIntersectionContainsCorrectness(points, set1, set2); + + for (int i = 1; i < 1 << points.length; i++) // at least one set bit + { + String[] ranges = new String[Integer.bitCount(i) * 2]; + int p = 0; + for (int j = 0; j < points.length; j++) + { + if ((i & (1 << j)) != 0) + { + ranges[p++] = points[j]; + ranges[p++] = points[j]; + } + } + System.out.println(Arrays.toString(ranges)); + TrieSet set3 = TrieUtil.directRanges(ranges); + RangeTrie expected = TrieUtil.directRangeTrie(Arrays.stream(ranges).filter(x -> ix.applicableRange(TrieUtil.directComparable(x)) != null).toArray(String[]::new)); + TrieUtil.verifyEqualRangeTries(set1.intersect(set2).intersect(set3), expected); + TrieUtil.verifyEqualRangeTries(set1.intersect(set3.intersection(set2)), expected); + } + } + + private static void verifyIntersectionContainsCorrectness(String[] points, RangeTrie trie, TrieSet set) + { + RangeTrie ix = trie.intersect(set); + for (String s : points) + { + ByteComparable bc = TrieUtil.directComparable(s); + assertEquals(s, set.strictlyContains(bc) ? trie.applicableRange(bc) != null : false, ix.applicableRange(bc) != null); + } + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/RangeTrieMergeTest.java b/test/unit/org/apache/cassandra/db/tries/RangeTrieMergeTest.java new file mode 100644 index 000000000000..444a9d760902 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/RangeTrieMergeTest.java @@ -0,0 +1,556 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +import com.google.common.collect.Lists; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +import static java.util.Arrays.asList; +import static org.apache.cassandra.db.tries.TestRangeState.fromList; +import static org.apache.cassandra.db.tries.TestRangeState.toList; +import static org.apache.cassandra.db.tries.TestRangeState.verify; +import static org.junit.Assert.assertEquals; + +public class RangeTrieMergeTest +{ + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + + static final int bitsNeeded = 6; + int bits = bitsNeeded; + + /** Creates a {@link ByteComparable} for the provided value by splitting the integer in sequences of "bits" bits. */ + private ByteComparable of(int value) + { + assert value >= 0 && value < 1<< bitsNeeded; + + byte[] splitBytes = new byte[(bitsNeeded + bits - 1) / bits]; + int pos = 0; + int mask = (1 << bits) - 1; + for (int i = bitsNeeded - bits; i > 0; i -= bits) + splitBytes[pos++] = (byte) ((value >> i) & mask); + + splitBytes[pos] = (byte) (value & mask); + return ByteComparable.preencoded(TrieUtil.VERSION, splitBytes); + } + + private TestRangeState from(int where, int value) + { + return new TestRangeState(of(where), -1, value, true); + } + + private TestRangeState to(int where, int value) + { + return new TestRangeState(of(where), value, -1, true); + } + + private TestRangeState change(int where, int from, int to) + { + return new TestRangeState(of(where), from, to, true); + } + + private List deletedRanges(ByteComparable... dataPoints) + { + List data = new ArrayList<>(asList(dataPoints)); + invertDataRangeList(data); + filterOutEmptyRepetitions(data); + + List markers = new ArrayList<>(); + for (int i = 0; i < data.size(); ++i) + { + ByteComparable pos = data.get(i); + if (pos == null) + pos = i % 2 == 0 ? of(0) : of((1< data) + { + // invert list + if (data.get(0) != null) + data.add(0, null); + else + data.remove(0); + if (data.get(data.size() - 1) != null) + data.add(null); + else + data.remove(data.size() - 1); + } + + private static void filterOutEmptyRepetitions(List data) + { + for (int i = 0; i < data.size() - 1; ++i) + { + if (data.get(i) != null && data.get(i + 1) != null && + ByteComparable.compare(data.get(i), data.get(i + 1), TrieUtil.VERSION) == 0) + { + data.remove(i + 1); + data.remove(i); + --i; + } + } + } + + @Test + public void testSubtrie() + { + for (bits = bitsNeeded; bits > 0; --bits) + { + testMerge("no merge"); + + testMerge("all", + deletedRanges(null, null)); + testMerge("fully covered range", + deletedRanges(of(20), of(25))); + testMerge("fully covered range", + deletedRanges(of(25), of(33))); + testMerge("matching range", + deletedRanges(of(21), of(24))); + testMerge("touching empty", + deletedRanges(of(24), of(26))); + + testMerge("partial left", + deletedRanges(of(22), of(25))); + testMerge("partial left on change", + deletedRanges(of(28), of(32))); + testMerge("partial left with null", + deletedRanges(of(29), null)); + + + testMerge("partial right", + deletedRanges(of(25), of(27))); + testMerge("partial right on change", + deletedRanges(of(25), of(28))); + testMerge("partial right with null", + deletedRanges(null, of(22))); + + testMerge("inside range", + deletedRanges(of(22), of(23))); + testMerge("inside with change", + deletedRanges(of(27), of(29))); + + testMerge("empty range inside", + deletedRanges(of(27), of(27))); + + testMerge("point covered", + deletedRanges(of(16), of(18))); + testMerge("point at range start", + deletedRanges(of(17), of(18))); + testMerge("point at range end", + deletedRanges(of(16), of(17))); + + + testMerge("start point covered", + deletedRanges(of(32), of(35))); + testMerge("start point at range start", + deletedRanges(of(33), of(35))); + testMerge("start point at range end", + deletedRanges(of(32), of(33))); + + + testMerge("end point covered", + deletedRanges(of(36), of(40))); + testMerge("end point at range start", + deletedRanges(of(38), of(40))); + testMerge("end point at range end", + deletedRanges(of(36), of(38))); + } + } + + @Test + public void testRanges() + { + for (bits = bitsNeeded; bits > 0; --bits) + { + testMerge("fully covered ranges", + deletedRanges(of(20), of(25), of(25), of(33))); + testMerge("matching ranges", + deletedRanges(of(21), of(24), of(26), of(31))); + testMerge("touching empty", + deletedRanges(of(20), of(21), of(24), of(26), of(32), of(33), of(34), of(36))); + testMerge("partial left", + deletedRanges(of(22), of(25), of(29), null)); + + testMerge("partial right", + deletedRanges(null, of(22), of(25), of(27))); + + testMerge("inside ranges", + deletedRanges(of(22), of(23), of(27), of(29))); + + testMerge("jumping inside", + deletedRanges(of(21), of(22), of(23), of(24), of(25), of(26), of(27), of(28), of(29), of(30))); + } + } + + @Test + public void testRangeOnSubtrie() + { + for (bits = bitsNeeded; bits > 0; --bits) + { + // non-overlapping + testMerge("non-overlapping", deletedRanges(of(20), of(23)), deletedRanges(of(24), of(27))); + // touching, i.e. still non-overlapping + testMerge("touching", deletedRanges(of(20), of(23)), deletedRanges(of(23), of(27))); + // overlapping 1 + testMerge("overlapping1", deletedRanges(of(20), of(23)), deletedRanges(of(22), of(27))); + // overlapping 2 + testMerge("overlapping2", deletedRanges(of(20), of(23)), deletedRanges(of(21), of(27))); + // covered + testMerge("covered1", deletedRanges(of(20), of(23)), deletedRanges(of(20), of(27))); + // covered + testMerge("covered2", deletedRanges(of(23), of(27)), deletedRanges(of(20), of(27))); + // covered 2 + testMerge("covered3", deletedRanges(of(21), of(23)), deletedRanges(of(20), of(27))); + } + } + + @Test + public void testRangesOnRanges() + { + for (bits = bitsNeeded; bits > 0; --bits) + testMerges(); + } + + private List getTestRanges() + { + TestRangeState testRangeState = to(38, 14); + TestRangeState testRangeState1 = from(33, 13); + TestRangeState testRangeState2 = change(28, 11, 12); + return asList(from(21, 10), to(24, 10), + from(26, 11), testRangeState2, to(30, 12), + testRangeState1, to(34, 13), + from(36, 14), testRangeState); + } + + private void testMerges() + { + testMerge("", fromList(getTestRanges()), getTestRanges()); + + List set1 = deletedRanges(null, of(24), of(25), of(29), of(32), null); + List set2 = deletedRanges(of(14), of(17), + of(22), of(27), + of(28), of(30), + of(32), of(34), + of(36), of(40)); + List set3 = deletedRanges(of(17), of(18), + of(19), of(20), + of(21), of(22), + of(23), of(24), + of(25), of(26), + of(27), of(28), + of(29), of(30), + of(31), of(32), + of(33), of(34), + of(35), of(36), + of(37), of(38)); + + testMerges(set1, set2, set3); + } + + private void testMerges(List set1, List set2, List set3) + { + // set1 = TrieSet.ranges(null, of(24), of(25), of(29), of(32), null); + // set2 = TrieSet.ranges(of(22), of(27), of(28), of(30), of(32), of(34)); + // set3 = TrieSet.ranges(of(21), of(22), of(23), of(24), of(25), of(26), of(27), of(28), of(29), of(30)); + // from(21, 10), to(24, 10), from(26, 11), change(28, 11, 12), to(30, 12), from(33, 13), to(34, 13) + testMerge("1", set1); + + testMerge("2", set2); + + testMerge("3", set3); + + testMerge("12", set1, set2); + + testMerge("13", set1, set3); + + testMerge("23", set2, set3); + + testMerge("123", set1, set2, set3); + } + + @SafeVarargs + public final void testMerge(String message, List... sets) + { + List testRanges = getTestRanges(); + testMerge(message, fromList(testRanges), testRanges, sets); + testCollectionMerge(message + " collection", Lists.newArrayList(fromList(testRanges)), testRanges, sets); + testMergeToInMemoryTrie(message + " inmem.apply", fromList(testRanges), testRanges, sets); + } + + + public void testMerge(String message, RangeTrie trie, List merged, List... sets) + { + System.out.println("Markers: " + merged); + verify(merged); + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + try + { + assertEquals(message + " forward b" + bits, merged, toList(trie, Direction.FORWARD)); + assertEquals(message + " reverse b" + bits, Lists.reverse(merged), toList(trie, Direction.REVERSE)); + System.out.println(message + " b" + bits + " matched."); + } + catch (AssertionError e) + { + System.out.println("\n" + trie.dump()); + throw e; + } + } + else + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + List ranges = sets[toRemove]; + System.out.println("Adding: " + ranges); + testMerge(message + " " + toRemove, + trie.mergeWith(fromList(ranges), TestRangeState::combine), + mergeLists(merged, ranges), + Arrays.stream(sets) + .filter(x -> x != ranges) + .toArray(List[]::new) + ); + } + } + } + + InMemoryRangeTrie duplicateTrie(RangeTrie trie) + { + try + { + InMemoryRangeTrie dupe = InMemoryRangeTrie.shortLived(TrieUtil.VERSION); + dupe.apply(trie, this::upsertMarkers, x -> false); + return dupe; + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + } + + public void testMergeToInMemoryTrie(String message, InMemoryRangeTrie trie, List merged, List... sets) + { + System.out.println("Markers: " + merged); + verify(merged); + System.out.println("Trie: \n" + trie.dump()); + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + try + { + assertEquals(message + " forward b" + bits, merged, toList(trie, Direction.FORWARD)); + assertEquals(message + " reverse b" + bits, Lists.reverse(merged), toList(trie, Direction.REVERSE)); + System.out.println(message + " b" + bits + " matched."); + } + catch (AssertionError e) + { + System.out.println("\n" + trie.dump()); + throw e; + } + } + else + { + try + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + List ranges = sets[toRemove]; + System.out.println("Adding: " + ranges); + InMemoryRangeTrie dupe = duplicateTrie(trie); + dupe.apply(fromList(ranges), this::upsertMarkers, x -> false); + testMergeToInMemoryTrie(message + " " + toRemove, + dupe, + mergeLists(merged, ranges), + Arrays.stream(sets) + .filter(x -> x != ranges) + .toArray(List[]::new) + ); + } + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + } + } + + TestRangeState upsertMarkers(TestRangeState left, TestRangeState right) + { + if (left == null) + return right; + if (right == null) + return left; + return TestRangeState.combine(left, right); + } + + public void testCollectionMerge(String message, List> triesToMerge, List merged, List... sets) + { + System.out.println("Markers: " + merged); + verify(merged); + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + RangeTrie trie = RangeTrie.merge(triesToMerge, TestRangeState::combineCollection); + try + { + assertEquals(message + " forward b" + bits, merged, toList(trie, Direction.FORWARD)); + assertEquals(message + " reverse b" + bits, Lists.reverse(merged), toList(trie, Direction.REVERSE)); + System.out.println(message + " b" + bits + " matched."); + } + catch (AssertionError e) + { + System.out.println("\n" + trie.dump()); + throw e; + } + } + else + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + List ranges = sets[toRemove]; + System.out.println("Adding: " + ranges); + triesToMerge.add(fromList(ranges)); + testCollectionMerge(message + " " + toRemove, + triesToMerge, + mergeLists(merged, ranges), + Arrays.stream(sets) + .filter(x -> x != ranges) + .toArray(List[]::new) + ); + triesToMerge.remove(triesToMerge.size() - 1); + } + } + } + + int delete(int deletionTime, int data) + { + if (data <= deletionTime) + return -1; + else + return data; + } + + TestRangeState delete(int deletionTime, TestRangeState marker) + { + if (deletionTime < 0) + return marker; + + int newLeft = delete(deletionTime, marker.leftSide); + int newRight = delete(deletionTime, marker.rightSide); + if (newLeft < 0 && newRight < 0 || newLeft == newRight) + return null; + if (newLeft == marker.leftSide && newRight == marker.rightSide) + return marker; + return new TestRangeState(marker.position, newLeft, newRight, marker.isBoundary); + } + + + List mergeLists(List left, List right) + { + int active = -1; + Iterator rightIt = right.iterator(); + TestRangeState nextRight = rightIt.hasNext() ? rightIt.next() : null; + List result = new ArrayList<>(); + for (TestRangeState nextLeft : left) + { + while (true) + { + int cmp; + if (nextRight == null) + cmp = -1; + else + cmp = ByteComparable.compare(nextLeft.position, nextRight.position, TrieUtil.VERSION); + + if (cmp < 0) + { + maybeAdd(result, nextRight != null ? delete(nextRight.leftSide, nextLeft) : nextLeft); + break; + } + + if (cmp == 0) + { + TestRangeState processed = TestRangeState.combine(nextRight, nextLeft).toContent(); + maybeAdd(result, processed); + nextRight = rightIt.hasNext() ? rightIt.next() : null; + break; + } + else + { + // Must close active if it becomes covered, and must open active if it is no longer covered. + if (active >= 0) + { + TestRangeState activeMarker = new TestRangeState(nextRight.position, active, active, true); + nextRight = TestRangeState.combine(activeMarker, nextRight).toContent(); + } + maybeAdd(result, nextRight); + } + + nextRight = rightIt.hasNext() ? rightIt.next() : null; + } + active = nextLeft.rightSide; + } + assert active == -1; + while (nextRight != null) + { + maybeAdd(result, delete(active, nextRight));// deletion is not needed (active == -1), do just in case + nextRight = rightIt.hasNext() ? rightIt.next() : null; + } + return result; + } + + static void maybeAdd(List list, T value) + { + if (value == null) + return; + list.add(value); + } + + @Test(expected = IllegalStateException.class) + public void testRangeUnderCoveredRange() + { + String[] ranges1 = {"ba", "bb"}; + String[] ranges2 = {"aa", "ab", "bbc", "bbd", "bbfff", "bbfff", "bce", "bcf", "ce", "cf"}; + // We don't currently handle boundaries that are prefixes of entries and we should identify this and throw an exception. + var list = toList(RangeTrie.merge(List.of(TrieUtil.directRangeTrie(1, ranges1), + TrieUtil.directRangeTrie(2, ranges2)), + TestRangeState::combineCollection), + Direction.FORWARD); + System.out.println(list); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/RangesTrieSetTest.java b/test/unit/org/apache/cassandra/db/tries/RangesTrieSetTest.java index b2be006b4b2b..5674352ea63c 100644 --- a/test/unit/org/apache/cassandra/db/tries/RangesTrieSetTest.java +++ b/test/unit/org/apache/cassandra/db/tries/RangesTrieSetTest.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db.tries; +import org.junit.BeforeClass; import org.junit.Test; import java.util.ArrayList; @@ -26,9 +27,6 @@ import java.util.NavigableMap; import java.util.Set; import java.util.TreeMap; - -import org.junit.BeforeClass; - import java.util.TreeSet; import org.apache.cassandra.config.CassandraRelevantProperties; @@ -414,7 +412,7 @@ public void testRangeStateFromProperties() TrieSetCursor.RangeState state = TrieSetCursor.RangeState.fromProperties(applicableBefore, applicableAfter, applicableAt); assertEquals(applicableBefore, state.applicableBefore); assertEquals(applicableAfter, state.applicableAfter); - assertEquals(applicableAt, state.asContent != null); + assertEquals(applicableAt, state.isBoundary); } } } diff --git a/test/unit/org/apache/cassandra/db/tries/TestRangeState.java b/test/unit/org/apache/cassandra/db/tries/TestRangeState.java new file mode 100644 index 000000000000..16fd4d0b15b0 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/TestRangeState.java @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +import com.google.common.base.Throwables; +import com.google.common.collect.Maps; +import com.google.common.collect.Streams; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +class TestRangeState implements RangeState +{ + final ByteComparable position; + final int leftSide; + final int rightSide; + + final boolean isBoundary; + final TestRangeState leftState; + final TestRangeState rightState; + + TestRangeState(ByteComparable position, int leftSide, int rightSide, boolean isBoundary) + { + this.position = position; + this.leftSide = leftSide; + this.rightSide = rightSide; + this.isBoundary = isBoundary; + if (leftSide == rightSide && !isBoundary) + { + this.leftState = this; + this.rightState = this; + } + else + { + this.leftState = leftSide >= 0 ? new TestRangeState(position, leftSide, leftSide, false) : null; + this.rightState = rightSide >= 0 ? new TestRangeState(position, rightSide, rightSide, false) : null; + } + } + + static TestRangeState combine(TestRangeState m1, TestRangeState m2) + { + int newLeft = Math.max(m1.leftSide, m2.leftSide); + int newRight = Math.max(m1.rightSide, m2.rightSide); + if (newLeft < 0 && newRight < 0) + return null; + + return new TestRangeState(m2.position, newLeft, newRight, + (m1.isBoundary || m2.isBoundary) && (newLeft != newRight)); + } + + + public static TestRangeState combineCollection(Collection rangeStates) + { + int newLeft = -1; + int newRight = -1; + boolean isReportableState = false; + ByteComparable position = null; + for (TestRangeState marker : rangeStates) + { + newLeft = Math.max(newLeft, marker.leftSide); + newRight = Math.max(newRight, marker.rightSide); + position = marker.position; + isReportableState |= marker.isBoundary; + } + if (newLeft < 0 && newRight < 0) + return null; + isReportableState &= newLeft != newRight; + + return new TestRangeState(position, newLeft, newRight, isReportableState); + } + +// @Override +// public boolean equals(Object o) +// { +// if (this == o) return true; +// if (o == null || getClass() != o.getClass()) return false; +// TestRangeState that = (TestRangeState) o; +// return ByteComparable.compare(this.position, that.position, TrieUtil.VERSION) == 0 +// && leftSide == that.leftSide +// && rightSide == that.rightSide; +// } + + @Override + public int hashCode() + { + return Objects.hash(position, leftSide, rightSide); + } + + @Override + public String toString() + { + return (leftSide >= 0 ? leftSide + "<" : "") + + '"' + toString(position) + '"' + + (rightSide >= 0 ? "<" + rightSide : "") + + (isBoundary ? "" : " not reportable"); + } + + public String toStringNoPosition() + { + return (leftSide >= 0 ? leftSide + "<" : "") + + 'X' + + (rightSide >= 0 ? "<" + rightSide : "") + + (isBoundary ? "" : " not reportable"); + } + + @Override + public boolean isBoundary() + { + return isBoundary; + } + + public TestRangeState toContent() + { + return isBoundary ? this : null; + } + + @Override + public TestRangeState precedingState(Direction direction) + { + return direction.select(leftState, rightState); + } + + @Override + public TestRangeState restrict(boolean applicableBefore, boolean applicableAfter) + { + assert isBoundary; + if ((applicableBefore || leftSide < 0) && (applicableAfter || rightSide < 0)) + return this; + int newLeft = applicableBefore ? leftSide : -1; + int newRight = applicableAfter ? rightSide : -1; + if (newLeft >= 0 || newRight >= 0) + return new TestRangeState(position, newLeft, newRight, isBoundary); + else + return null; + } + + @Override + public TestRangeState asBoundary(Direction direction) + { + assert !isBoundary; + final boolean isForward = direction.isForward(); + int newLeft = !isForward ? leftSide : -1; + int newRight = isForward ? rightSide : -1; + return new TestRangeState(position, newLeft, newRight, true); + } + + static String toString(ByteComparable position) + { + if (position == null) + return "null"; + return position.byteComparableAsString(TrieUtil.VERSION); + } + + static List verify(List markers) + { + int active = -1; + ByteComparable prev = null; + for (TestRangeState marker : markers) + { + assertTrue("Order violation " + toString(prev) + " vs " + toString(marker.position), + prev == null || ByteComparable.compare(prev, marker.position, TrieUtil.VERSION) < 0); + assertEquals("Range close violation", active, marker.leftSide); + assertTrue(marker.leftSide != marker.rightSide); + prev = marker.position; + active = marker.rightSide; + } + assertEquals("Unclosed range", -1, active); + return markers; + } + + + /** + * Extract the values of the provided trie into a list. + */ + static List toList(RangeTrie trie, Direction direction) + { + return Streams.stream(trie.entryIterator(direction)) + .map(en -> remap(en.getValue(), en.getKey())) + .collect(Collectors.toList()); + } + + static TestRangeState remap(TestRangeState dm, ByteComparable newKey) + { + return new TestRangeState(newKey, dm.leftSide, dm.rightSide, dm.isBoundary); + } + + static Map.Entry remap(Map.Entry entry) + { + return Maps.immutableEntry(entry.getKey(), remap(entry.getValue(), entry.getKey())); + } + + static InMemoryRangeTrie fromList(List list) + { + InMemoryRangeTrie trie = InMemoryRangeTrie.shortLived(TrieUtil.VERSION); + for (TestRangeState i : list) + { + try + { + trie.putRecursive(i.position, i, (ex, n) -> n); + } + catch (TrieSpaceExhaustedException e) + { + throw Throwables.propagate(e); + } + } + return trie; + } + + @Override + public boolean equals(Object other) + { + if (other == null) + return false; + TestRangeState otherMarker = (TestRangeState) other; + return otherMarker.leftSide == leftSide && otherMarker.rightSide == rightSide; + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/ThreadedTestBase.java b/test/unit/org/apache/cassandra/db/tries/ThreadedTestBase.java new file mode 100644 index 000000000000..d8a5b7cdcf01 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/ThreadedTestBase.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.concurrent.OpOrder; + +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.apache.cassandra.db.tries.TrieUtil.generateKeys; + +public abstract class ThreadedTestBase> +{ + // Note: This should not be run by default with verification to have the higher concurrency of faster writes and reads. + + private static final int COUNT = 30000; + private static final int OTHERS = COUNT / 10; + private static final int PROGRESS_UPDATE = COUNT / 15; + private static final int READERS = 0; + private static final int WALKERS = 1; + private static final Random rand = new Random(); + + abstract T value(ByteComparable b); + abstract R makeTrie(OpOrder readOrder); + abstract void add(R trie, ByteComparable b, T v, int iteration) throws TrieSpaceExhaustedException; + + @Test + public void testThreaded() throws InterruptedException + { + OpOrder readOrder = new OpOrder(); + ByteComparable[] src = generateKeys(rand, COUNT + OTHERS); + R trie = makeTrie(readOrder); + ConcurrentLinkedQueue errors = new ConcurrentLinkedQueue<>(); + List threads = new ArrayList<>(); + AtomicBoolean writeCompleted = new AtomicBoolean(false); + AtomicInteger writeProgress = new AtomicInteger(0); + + for (int i = 0; i < WALKERS; ++i) + threads.add(new Thread(() -> { + try + { + while (!writeCompleted.get()) + { + int min = writeProgress.get(); + int count = 0; + try (OpOrder.Group group = readOrder.start()) + { + for (Map.Entry en : trie.entrySet()) + { + T v = value(en.getKey()); + Assert.assertEquals(en.getKey().byteComparableAsString(VERSION), v, en.getValue()); + ++count; + } + } + Assert.assertTrue("Got only " + count + " while progress is at " + min, count >= min); + } + } + catch (Throwable t) + { + t.printStackTrace(); + errors.add(t); + } + })); + + for (int i = 0; i < READERS; ++i) + { + threads.add(new Thread(() -> { + try + { + Random r = ThreadLocalRandom.current(); + while (!writeCompleted.get()) + { + int min = writeProgress.get(); + + for (int i1 = 0; i1 < PROGRESS_UPDATE; ++i1) + { + int index = r.nextInt(COUNT + OTHERS); + ByteComparable b = src[index]; + T v = value(b); + try (OpOrder.Group group = readOrder.start()) + { + T result = trie.get(b); + if (result != null) + { + Assert.assertTrue("Got not added " + index + " when COUNT is " + COUNT, + index < COUNT); + Assert.assertEquals("Failed " + index, v, result); + } + else if (index < min) + Assert.fail("Failed index " + index + " while progress is at " + min); + } + } + } + } + catch (Throwable t) + { + t.printStackTrace(); + errors.add(t); + } + })); + } + +// threads.add +// (new Thread(() -> { + try + { + for (int i = 0; i < COUNT; i++) + { + ByteComparable b = src[i]; + + // Note: Because we don't ensure order when calling resolve, just use a hash of the key as payload + // (so that all sources have the same value). + T v = value(b); + add(trie, b, v, i); + + if (i % PROGRESS_UPDATE == 0) + writeProgress.set(i); + } + } + catch (Throwable t) + { + t.printStackTrace(); + errors.add(t); + } + finally + { + writeCompleted.set(true); + } +// })); + + for (Thread t : threads) + t.start(); + + for (Thread t : threads) + t.join(); + + if (!errors.isEmpty()) + Assert.fail("Got errors:\n" + errors); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/TrieUtil.java b/test/unit/org/apache/cassandra/db/tries/TrieUtil.java index e23a39aaa509..013730090941 100644 --- a/test/unit/org/apache/cassandra/db/tries/TrieUtil.java +++ b/test/unit/org/apache/cassandra/db/tries/TrieUtil.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Comparator; import java.util.Iterator; @@ -37,6 +38,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; import com.google.common.collect.HashMultiset; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Multiset; import com.google.common.collect.Sets; @@ -47,6 +49,7 @@ import org.apache.cassandra.utils.bytecomparable.ByteComparable; import org.apache.cassandra.utils.bytecomparable.ByteSource; +import static org.apache.cassandra.db.tries.TestRangeState.remap; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -359,6 +362,82 @@ static Preencoded generateKey(Random rand, int minLength, int maxLength, int ter return ((ByteComparable)(v -> ByteSource.withTerminator(terminator, ByteSource.of(bytes, v)))).preencode(VERSION); } + public static Trie withRootMetadata(Trie wrapped, T metadata) + { + return wrapped.mergeWith(Trie.singleton(ByteComparable.EMPTY, VERSION, metadata), Trie.throwingResolver()); + } + + static Trie directTrie(String... points) throws TrieSpaceExhaustedException + { + InMemoryTrie trie = InMemoryTrie.shortLived(VERSION); + for (String s : points) + trie.putRecursive(directComparable(s), s, (ex, n) -> n); + return trie; + } + + static TrieSet directRanges(String... ranges) + { + if (ranges.length == 0) + return TrieSet.empty(VERSION); + + // to test singleton too, special case two equal boundaries + if (ranges.length == 2 && Objects.equal(ranges[0], ranges[1])) + return TrieSet.singleton(VERSION, directComparable(ranges[0])); + + return TrieSet.ranges(VERSION, Arrays.stream(ranges) + .map(r -> directComparable(r)) + .toArray(ByteComparable[]::new)); + } + + static RangeTrie directRangeTrie(String... keys) + { + return directRangeTrie(1, keys); + } + + static RangeTrie directRangeTrie(int value, String... keys) + { + if (keys.length == 0) + return RangeTrie.empty(VERSION); + if (keys.length == 2 && Objects.equal(keys[0], keys[1])) + { + // special case to make a singleton trie + ByteComparable bc = directComparable(keys[0]); + return RangeTrie.range(bc, bc, VERSION, new TestRangeState(bc, value, value, false)); + } + + try + { + InMemoryRangeTrie trie = InMemoryRangeTrie.shortLived(VERSION); + boolean left = true; + for (String s : keys) + { + trie.putRecursive(directComparable(s), + new TestRangeState(directComparable(s), left ? -1 : value, left ? value : -1, true), + (e, n) -> e != null ? e.restrict(n.leftSide >= 0, n.rightSide >= 0) : n); + left = !left; + } + return trie; + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); // we are not inserting that much data + } + } + + static void verifyEqualRangeTries(RangeTrie trie, RangeTrie expected) + { +// System.out.println("Trie:\n" + trie.dump(TestRangeState::toStringNoPosition)); +// System.out.println("Expected:\n" + expected.cursor(Direction.FORWARD).process(new TrieDumper<>(TestRangeState::toStringNoPosition))); + assertMapEquals(Iterables.transform(trie.entrySet(Direction.FORWARD), + en -> remap(en)), + expected.entrySet(Direction.FORWARD), + FORWARD_COMPARATOR); + assertMapEquals(Iterables.transform(trie.entrySet(Direction.REVERSE), + en -> remap(en)), + expected.entrySet(Direction.REVERSE), + REVERSE_COMPARATOR); + } + static Preencoded toBound(Preencoded bc) { return toBound(bc, false); From ce9d384fdb2167655fbb47d9f58f2ec32d59526f Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Mon, 5 May 2025 18:34:31 +0300 Subject: [PATCH 14/22] Implement RangeTrie.applyTo, InMemoryTrie.delete and InMemoryTrie.apply(RangeTrie...) --- .../cassandra/db/tries/InMemoryBaseTrie.java | 21 ++ .../cassandra/db/tries/InMemoryReadTrie.java | 36 +- .../cassandra/db/tries/InMemoryTrie.java | 159 +++++++++ .../cassandra/db/tries/RangeApplyCursor.java | 167 +++++++++ .../apache/cassandra/db/tries/RangeTrie.java | 14 +- .../db/tries/ConsistencyTestBase.java | 334 +++++++++++++++--- .../db/tries/InMemoryTrieConsistencyTest.java | 26 +- .../db/tries/IntersectionTrieTest.java | 81 +++++ .../apache/cassandra/db/tries/TrieUtil.java | 5 + 9 files changed, 783 insertions(+), 60 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/tries/RangeApplyCursor.java diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java index 9e92bc38a32b..36084b1cb3fc 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java @@ -1099,6 +1099,27 @@ boolean advanceToNextExistingOr(int limitDepth, int limitTransition, int forcedC } } + /// Advance to the next existing position in the trie. + boolean advanceToNextExisting(int forcedCopyDepth) throws TrieSpaceExhaustedException + { + setTransition(-1); // we have newly descended to a node, start with its first child + while (true) + { + int currentTransition = transition(); + int nextTransition = getNextTransition(existingPostContentNode(), currentTransition + 1); + if (nextTransition <= 0xFF) + { + descend(nextTransition); + return true; + } + + if (currentDepth <= ascendLimit) + return false; + + attachAndMoveToParentState(forcedCopyDepth); + } + } + /// Descend to a child node. Prepares a new entry in the stack for the node. void descend(int transition) { diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java index c912ca277382..dd07fa684151 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java @@ -1288,6 +1288,26 @@ public String content() return new TypedNodesCursor().process(new TrieDumper<>(Function.identity())); } + private void dumpSplitNode(int node, int level, StringBuilder builder) + { + int limit = level == 0 ? SPLIT_START_LEVEL_LIMIT : SPLIT_OTHER_LEVEL_LIMIT; + for (int i = 0; i < limit; ++i) + { + int child = getIntVolatile(node - (limit - 1 - i) * 4); + if (child != NONE) + { + builder.append('\n'); + for (int ind = 0; ind < level; ++ind) + builder.append(" "); + builder.append(Integer.toBinaryString(i | 8).substring(1)) // or and substring implement %03b + .append(" -> "); + builder.append(child); + if (level < 2) + dumpSplitNode(child, level + 1, builder); + } + } + } + /// For use in debugging, dump info about the given node. @SuppressWarnings("unused") String dumpNode(int node) @@ -1304,7 +1324,7 @@ else if (isLeaf(node)) { case SPARSE_OFFSET: { - builder.append("Sparse (Order " + Integer.toString(getUnsignedShortVolatile(node + SPARSE_ORDER_OFFSET), 6) + "): "); + builder.append("Sparse (Order " + Integer.toString(getUnsignedShortVolatile(node + SPARSE_ORDER_OFFSET), 6) + "):\n"); for (int i = 0; i < SPARSE_CHILD_COUNT; ++i) { int child = getIntVolatile(node + SPARSE_CHILDREN_OFFSET + i * 4); @@ -1318,16 +1338,8 @@ else if (isLeaf(node)) } case SPLIT_OFFSET: { - builder.append("Split: "); - for (int i = 0; i < SPLIT_START_LEVEL_LIMIT; ++i) - { - int child = getIntVolatile(node - (SPLIT_START_LEVEL_LIMIT - 1 - i) * 4); - if (child != NONE) - builder.append(Integer.toBinaryString(i)) - .append(" -> ") - .append(child) - .append('\n'); - } + builder.append("Split:"); + dumpSplitNode(node, 0, builder); break; } case PREFIX_OFFSET: @@ -1343,7 +1355,7 @@ else if (isLeaf(node)) } default: { - builder.append("Chain: "); + builder.append("Chain:\n"); for (int i = 0; i < chainCellLength(node); ++i) builder.append(String.format("%02x", getUnsignedByte(node + i))); builder.append(" -> ") diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java index e0642533f145..890ad4a32ac3 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java @@ -179,4 +179,163 @@ public void putSingleton(ByteComparable key, else putSingleton(key, value, transformer); } + + /// Delete all entries covered under the specified TrieSet + public void delete(TrieSet set) throws TrieSpaceExhaustedException + { + apply(set.cursor(Direction.FORWARD), (UpsertTransformer) DeleteMutation::deleteEntry, NodeFeatures::isBranching); + } + + /// Apply the given range trie to this in-memory trie. Any existing content that falls under the ranges of the given + /// trie will be modified by applying the transformer. This is usually used to delete covered content (by returning + /// null from the transformer). + /// @param rangeTrie the ranges to be applied, given in the form of a range trie. + /// @param transformer a function applied to the potentially pre-existing value for the given key, and the new + /// value. Applied even if there's no pre-existing value in the memtable trie. + /// @param needsForcedCopy a predicate which decides when to fully copy a branch to provide atomicity guarantees to + /// concurrent readers. See NodeFeatures for details. + public > + void apply(RangeTrie rangeTrie, + final UpsertTransformerWithKeyProducer transformer, + final Predicate> needsForcedCopy) + throws TrieSpaceExhaustedException + { + apply(rangeTrie.cursor(Direction.FORWARD), transformer, needsForcedCopy); + } + + private > void apply(RangeCursor cursor, + UpsertTransformerWithKeyProducer transformer, + Predicate> needsForcedCopy) throws TrieSpaceExhaustedException + { + try + { + DeleteMutation> m = new DeleteMutation<>(transformer, + needsForcedCopy, + cursor, + applyState.start()); + m.apply(); + m.complete(); + completeMutation(); + } + catch (Throwable t) + { + abortMutation(); + throw t; + } + } + + static class DeleteMutation, C extends RangeCursor> extends Mutation + { + DeleteMutation(UpsertTransformerWithKeyProducer transformer, + Predicate> needsForcedCopy, + C mutationCursor, + InMemoryBaseTrie.ApplyState state) + { + super(transformer, needsForcedCopy, mutationCursor, state); + } + + @Override + void apply() throws TrieSpaceExhaustedException + { + // A TrieSet may start already in a deleted range. If so, pretend there's a START at the initial position. + S content = mutationCursor.precedingState(); + if (coveringStateApplies(content)) + content = content.asBoundary(Direction.FORWARD); + else + content = mutationCursor.content(); + + int depth = state.currentDepth; + int prevAscendDepth = state.setAscendLimit(depth); + while (true) + { + if (depth < forcedCopyDepth) + forcedCopyDepth = needsForcedCopy.test(this) ? depth : Integer.MAX_VALUE; + + if (content != null) + { + applyCoveringContent(content); + S mutationCoveringState = content.precedingState(Direction.REVERSE); // Use the right side of the deletion + if (coveringStateApplies(mutationCoveringState)) + { + boolean done = !applyDeletionRange(mutationCoveringState); + if (done) + break; + } + } + + depth = mutationCursor.advance(); + // Descend but do not modify anything yet. + if (!state.advanceTo(depth, mutationCursor.incomingTransition(), forcedCopyDepth)) + break; + + assert state.currentDepth == depth : "Unexpected change to applyState. Concurrent trie modification?"; + content = mutationCursor.content(); + } + state.setAscendLimit(prevAscendDepth); + } + + /// Walk all existing content covered under a deletion. Returns true if the caller needs to continue processing + /// the mutation cursor, and false if the mutation has been exhausted (i.e. the range was open on the right + /// and we have consumed all existing content). + boolean applyDeletionRange(S mutationCoveringState) throws TrieSpaceExhaustedException + { + boolean atMutation = true; + int depth = mutationCursor.depth(); + int transition = mutationCursor.incomingTransition(); + // We are walking both tries in parallel. + while (true) + { + if (atMutation) + { + depth = mutationCursor.advance(); + transition = mutationCursor.incomingTransition(); + atMutation = false; + } + + // Mutation can be open on the right (i.e. not have a closing marker). + if (depth > 0) + { + if (depth < forcedCopyDepth) + forcedCopyDepth = needsForcedCopy.test(this) ? depth : Integer.MAX_VALUE; + atMutation = !state.advanceToNextExistingOr(depth, transition, forcedCopyDepth); + } + else if (!state.advanceToNextExisting(forcedCopyDepth)) + return false; + + T existingContent = state.getContent(); + S mutationContent = atMutation ? mutationCursor.content() : null; + if (mutationContent != null) + { + applyCoveringContent(mutationContent); + mutationCoveringState = mutationContent.precedingState(Direction.REVERSE); + if (!coveringStateApplies(mutationCoveringState)) + return true; // mutation deletion range was closed, we can continue normal mutation cursor iteration + } + else if (existingContent != null) + applyCoveringContent(mutationCoveringState); + } + } + + private static > boolean coveringStateApplies(S state) + { + // Sets return non-null state (START_END_PREFIX) for regions that they do not cover. Check that too. + return state != null && state != TrieSetCursor.RangeState.START_END_PREFIX; + } + + void applyCoveringContent(S content) throws TrieSpaceExhaustedException + { + if (content != null) + { + T existingContent = state.getContent(); + T combinedContent = transformer.apply(existingContent, content, state); + state.setContent(combinedContent, // can be null + state.currentDepth >= forcedCopyDepth); // this is called at the start of processing + } + } + + private static T deleteEntry(T entry, TrieSetCursor.RangeState state) + { + return state.applicableBefore ? null : entry; + } + } } diff --git a/src/java/org/apache/cassandra/db/tries/RangeApplyCursor.java b/src/java/org/apache/cassandra/db/tries/RangeApplyCursor.java new file mode 100644 index 000000000000..1408ee9ce197 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/RangeApplyCursor.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.function.BiFunction; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// A cursor applying a range to a data cursor. The cursor will present the content of the data trie modified by any +/// applicable/covering range of the range trie. +/// +/// This is very similar to a normal merge cursor but, because it only presents content from the data trie, it does not +/// need to walk the range trie unless it matches positions from the data cursor and thus skips the range cursor +/// whenever the data one ends up ahead. +class RangeApplyCursor> implements Cursor +{ + final BiFunction resolver; + final Direction direction; + final RangeCursor range; + final Cursor data; + + boolean atRange; + + RangeApplyCursor(BiFunction resolver, RangeCursor range, Cursor data) + { + this.direction = data.direction(); + this.resolver = resolver; + this.range = range; + this.data = data; + assert data.depth() == 0; + assert range.depth() == 0; + atRange = true; + } + + @Override + public int depth() + { + return data.depth(); + } + + @Override + public int incomingTransition() + { + return data.incomingTransition(); + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + assert range.byteComparableVersion() == data.byteComparableVersion() : + "Merging cursors with different byteComparableVersions: " + + range.byteComparableVersion() + " vs " + data.byteComparableVersion(); + return range.byteComparableVersion(); + } + + @Override + public int advance() + { + int dataDepth = data.advance(); + if (atRange) + return skipRangeToDataPosition(dataDepth); + else + return maybeSkipRange(dataDepth); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + int dataDepth = data.skipTo(skipDepth, skipTransition); + if (atRange) // if both cursors were at the same position, always advance the range cursor to catch up. + return skipRangeToDataPosition(dataDepth); + else // otherwise skip range to the new data position only if it advances past the range's current position. + return maybeSkipRange(dataDepth); + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + // While we are on a shared position, we must descend one byte at a time to maintain the cursor ordering. + if (atRange) + return skipRangeToDataPosition(data.advance()); + else // atData only + return maybeSkipRange(data.advanceMultiple(receiver)); + } + + int maybeSkipRange(int dataDepth) + { + int rangeDepth = range.depth(); + // If data position is at or before the range position, we are good. + if (rangeDepth < dataDepth) + return setAtRangeAndReturnDepth(false, dataDepth); + + if (rangeDepth == dataDepth) + { + int dataTrans = data.incomingTransition(); + int rangeTrans = range.incomingTransition(); + if (direction.le(dataTrans, rangeTrans)) + return setAtRangeAndReturnDepth(dataTrans == rangeTrans, dataDepth); + } + + // Range cursor is before data cursor. Skip it ahead so that we are positioned on data. + return skipRangeToDataPosition(dataDepth); + } + + private int skipRangeToDataPosition(int dataDepth) + { + int dataTrans = data.incomingTransition(); + int rangeDepth = range.skipTo(dataDepth, dataTrans); + return setAtRangeAndReturnDepth(rangeDepth == dataDepth && range.incomingTransition() == dataTrans, + dataDepth); + } + + private int setAtRangeAndReturnDepth(boolean atRange, int depth) + { + this.atRange = atRange; + return depth; + } + + @Override + public T content() + { + T content = data.content(); + if (content == null) + return null; + + S applicableRange = atRange ? range.content() : null; + if (applicableRange == null) + { + applicableRange = range.precedingState(); + if (applicableRange == null) + return content; + } + + return resolver.apply(applicableRange, content); + } + + @Override + public Cursor tailCursor(Direction direction) + { + if (atRange) + return new RangeApplyCursor<>(resolver, range.tailCursor(direction), data.tailCursor(direction)); + else + return data.tailCursor(direction); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/RangeTrie.java b/src/java/org/apache/cassandra/db/tries/RangeTrie.java index e33b959d81dc..26b9a69675de 100644 --- a/src/java/org/apache/cassandra/db/tries/RangeTrie.java +++ b/src/java/org/apache/cassandra/db/tries/RangeTrie.java @@ -19,6 +19,7 @@ import java.util.Collection; import java.util.Iterator; +import java.util.function.BiFunction; import com.google.common.base.Preconditions; @@ -46,12 +47,12 @@ default R process(Direction direction, Cursor.Walker walker) /// Returns a singleton trie mapping the given byte path to a marker. /// - /// Note: Ranges are not meant to use boundaries that are distinct from data and thus a singleton range would list + /// Note: Ranges are meant to use boundaries that are distinct from data and thus a singleton range would list /// only a boundary and always be empty in terms of covered content. However, we do want to be able to place /// metadata in intermediate nodes of the trie and this method makes that possible. static > RangeTrie singleton(ByteComparable key, ByteComparable.Version byteComparableVersion, S v) { - Preconditions.checkArgument(v.isBoundary()); + Preconditions.checkArgument(v.isBoundary()); // make sure marker is returned for content() Preconditions.checkArgument(v.precedingState(Direction.FORWARD) == null); Preconditions.checkArgument(v.precedingState(Direction.REVERSE) == null); return dir -> new SingletonCursor.Range<>(dir, key.asComparableBytes(byteComparableVersion), byteComparableVersion, v); @@ -132,7 +133,14 @@ static > RangeTrie merge(Collection Trie applyTo(Trie source, BiFunction mapper) + { + return dir -> new RangeApplyCursor<>(mapper, cursor(dir), source.cursor(dir)); + } + static > RangeTrie empty(ByteComparable.Version version) { return dir -> RangeCursor.empty(dir, version); diff --git a/test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java b/test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java index 21e754b6b88e..369a8ed1e371 100644 --- a/test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java +++ b/test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java @@ -19,6 +19,7 @@ package org.apache.cassandra.db.tries; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collection; import java.util.List; import java.util.Map; @@ -27,6 +28,8 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.LongUnaryOperator; import java.util.function.Predicate; import org.junit.Assert; @@ -45,7 +48,7 @@ public abstract class ConsistencyTestBase, R exte // Note: This should not be run by default with verification to have the higher concurrency of faster writes and reads. private static final int COUNT = 30000; - private static final int PROGRESS_UPDATE = COUNT / 15; + private static final int PROGRESS_UPDATE = Math.max(1, COUNT / 15); private static final int READERS = 8; private static final int WALKERS = 2; private static final Random rand = new Random(); @@ -55,39 +58,73 @@ public abstract class ConsistencyTestBase, R exte * partition level as well as consistency. */ public final Predicate> FORCE_COPY_PARTITION = features -> isPartition(features.content()); + /** + * Force copy every modified cell below the partition/enumeration level. Provides atomicity of mutations within the + * partition level as well as consistency. + */ + public final Predicate> FORCE_COPY_PARTITION_RANGE_STATE = features -> isPartition(features.content()); /** * Force copy every modified cell below the earliest branching point. Provides atomicity of mutations at any level, * but readers/walkers may see inconsistent views of the data, in the sense that older mutations may be missed * while newer ones are returned. */ - public final Predicate> FORCE_ATOMIC = features -> features.isBranching(); + public final static Predicate> FORCE_ATOMIC = features -> features.isBranching(); + public final static Predicate> forceAtomic() + { + return (Predicate>) (Predicate) FORCE_ATOMIC; + } /** * Do not do any additional copying beyond what is required to build the tries safely for concurrent readers. * Mutations may be partially seen by readers, and older mutations may be missed while newer ones are returned. */ - public final Predicate> NO_ATOMICITY = features -> false; + public final static Predicate> NO_ATOMICITY = features -> false; + public final static Predicate> noAtomicity() + { + return (Predicate>) (Predicate) NO_ATOMICITY; + } abstract R makeTrie(OpOrder readOrder); abstract C value(ByteComparable b, ByteComparable cprefix, ByteComparable c, int add, int seqId); + abstract C metadata(ByteComparable b); + abstract String pk(C c); + abstract String ck(C c); + abstract int seq(C c); + abstract int value(C c); + abstract int updateCount(C c); abstract T makeSingleton(ByteComparable b, C content); + abstract T withRootMetadata(T wrapped, C metadata); abstract T merge(Collection tries, Trie.CollectionMergeResolver mergeResolver); + abstract void apply(R trie, T mutation, InMemoryBaseTrie.UpsertTransformer mergeResolver, Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException; + abstract void delete(R trie, + RangeTrie deletion, + InMemoryBaseTrie.UpsertTransformer mergeResolver, + Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException; + abstract boolean isPartition(C c); + boolean isPartition(TestRangeState c) + { + if (!(c instanceof TestStateMetadata)) + return false; + return isPartition(((TestStateMetadata) c).metadata); + } + abstract C mergeMetadata(C c1, C c2); + abstract C deleteMetadata(C existing, int entriesCount); abstract void printStats(R trie, Predicate> forcedCopyChecker); @@ -99,7 +136,7 @@ public void testConsistentUpdates() throws Exception // and consistent, i.e. that it is not possible to receive some newer updates while missing // older ones. (For example, if the sequence of additions is 3, 1, 5, without this requirement a reader // could see an enumeration which lists 3 and 5 but not 1.) - testAtomicUpdates(3, FORCE_COPY_PARTITION, true, true); + testAtomicUpdates(3, FORCE_COPY_PARTITION, FORCE_COPY_PARTITION_RANGE_STATE, true, true); // Note: using 3 per mutation, so that the first and second update fit in a sparse in-memory trie block. } @@ -108,14 +145,14 @@ public void testAtomicUpdates() throws Exception { // Check that multi-path updates with below-branching-point copying are safe for concurrent readers, // and that content is atomically applied, i.e. that reader see either nothing from the update or all of it. - testAtomicUpdates(3, FORCE_ATOMIC, true, false); + testAtomicUpdates(3, forceAtomic(), forceAtomic(), true, false); } @Test public void testSafeUpdates() throws Exception { // Check that multi path updates without additional copying are safe for concurrent readers. - testAtomicUpdates(3, NO_ATOMICITY, false, false); + testAtomicUpdates(3, noAtomicity(), noAtomicity(), false, false); } @Test @@ -125,7 +162,7 @@ public void testConsistentSinglePathUpdates() throws Exception // and that content is consistent, i.e. that it is not possible to receive some newer updates while missing // older ones. (For example, if the sequence of additions is 3, 1, 5, without this requirement a reader // could see an enumeration which lists 3 and 5 but not 1.) - testAtomicUpdates(1, FORCE_COPY_PARTITION, true, true); + testAtomicUpdates(1, FORCE_COPY_PARTITION, FORCE_COPY_PARTITION_RANGE_STATE, true, true); } @@ -134,14 +171,14 @@ public void testAtomicSinglePathUpdates() throws Exception { // When doing single path updates atomicity comes for free. This only checks that the branching checker is // not doing anything funny. - testAtomicUpdates(1, FORCE_ATOMIC, true, false); + testAtomicUpdates(1, forceAtomic(), forceAtomic(), true, false); } @Test public void testSafeSinglePathUpdates() throws Exception { // Check that single path updates without additional copying are safe for concurrent readers. - testAtomicUpdates(1, NO_ATOMICITY, true, false); + testAtomicUpdates(1, noAtomicity(), noAtomicity(), true, false); } // The generated keys all start with NEXT_COMPONENT, which makes it impossible to test the precise behavior of the @@ -164,8 +201,16 @@ private static ByteComparable skipFirst(ByteComparable key) }; } + private static ByteComparable swapTerminator(ByteComparable key, int newTerminator) + { + byte[] bytes = key.asByteComparableArray(VERSION); + bytes[bytes.length - 1] = (byte) newTerminator; + return ByteComparable.preencoded(VERSION, bytes); + } + public void testAtomicUpdates(int PER_MUTATION, Predicate> forcedCopyChecker, + Predicate> forcedCopyCheckerRanges, boolean checkAtomicity, boolean checkSequence) throws Exception @@ -185,17 +230,22 @@ public void testAtomicUpdates(int PER_MUTATION, List threads = new ArrayList(); AtomicBoolean writeCompleted = new AtomicBoolean(false); AtomicInteger writeProgress = new AtomicInteger(0); + AtomicLong writeProgressAck = new AtomicLong(0); + AtomicInteger threadIdx = new AtomicInteger(0); for (int i = 0; i < WALKERS; ++i) threads.add(new Thread() { + int threadId = threadIdx.getAndIncrement(); + LongUnaryOperator ackWriteProgress = x -> x | (1< x | (1 << threadId); - Random r = ThreadLocalRandom.current(); - while (!writeCompleted.get()) + Random r = ThreadLocalRandom.current(); + while (!writeCompleted.get()) + { + try { - ByteComparable key = srcLocal[r.nextInt(srcLocal.length)]; - int min = writeProgress.get() / (pkeys.length * PER_MUTATION) * PER_MUTATION; - Iterable> entries; - - try (OpOrder.Group group = readOrder.start()) { - entries = trie.tailTrie(key).entrySet(); - checkEntries(" in tail " + key.byteComparableAsString(VERSION), min, false, checkAtomicity, checkSequence, PER_MUTATION, entries); - } - - try (OpOrder.Group group = readOrder.start()) - { - entries = trie.subtrie(key, nextBranch(key)).entrySet(); - checkEntries(" in branch " + key.byteComparableAsString(VERSION), min, true, checkAtomicity, checkSequence, PER_MUTATION, entries); + writeProgressAck.getAndUpdate(ackWriteProgress); + ByteComparable key = srcLocal[r.nextInt(srcLocal.length)]; + int min = writeProgress.get() / (pkeys.length * PER_MUTATION) * PER_MUTATION; + Iterable> entries; + + try (OpOrder.Group group = readOrder.start()) + { + var tail = trie.tailTrie(key); + if (tail != null) + { + entries = tail.entrySet(); + checkEntries(" in tail " + key.byteComparableAsString(VERSION), min, false, checkAtomicity, checkSequence, PER_MUTATION, entries); + } + else + Assert.assertEquals("Trie key not found when there should be data for it", 0, min); + } + + try (OpOrder.Group group = readOrder.start()) + { + entries = trie.subtrie(key, key).entrySet(); + checkEntries(" in branch " + key.byteComparableAsString(VERSION), min, true, checkAtomicity, checkSequence, PER_MUTATION, entries); + } } } - } - catch (Throwable t) - { - t.printStackTrace(); - errors.add(t); + catch (Throwable t) + { + t.printStackTrace(); + errors.add(t); + } } } }); @@ -273,9 +332,11 @@ public C resolve(Collection contents) return contents.stream().reduce(this::resolve).get(); } }; + BitSet choices = new BitSet(COUNT / PER_MUTATION); try { + // Insert the data. int lastUpdate = 0; for (int i = 0; i < COUNT; i += PER_MUTATION) { @@ -283,7 +344,10 @@ public C resolve(Collection contents) C partitionMarker = metadata(b); ByteComparable cprefix = null; if (r.nextBoolean()) + { cprefix = ckeys[i]; // Also test branching point below the partition level + choices.set(i / PER_MUTATION); + } List sources = new ArrayList<>(); for (int j = 0; j < PER_MUTATION; ++j) @@ -306,8 +370,8 @@ public C resolve(Collection contents) final T mutation = merge(sources, mergeResolver); apply(trie, mutation, - (existing, update) -> existing == null ? update : mergeResolver.resolve(existing, update), - forcedCopyChecker); + (existing, update) -> existing == null ? update : mergeResolver.resolve(existing, update), + forcedCopyChecker); if (i >= pkeys.length * PER_MUTATION && i - lastUpdate >= PROGRESS_UPDATE) { @@ -315,6 +379,60 @@ public C resolve(Collection contents) lastUpdate = i; } } + + writeProgress.set(COUNT); + printStats(trie, forcedCopyChecker); + Thread.sleep(100); // Let the threads check the completed state too. + + InMemoryTrie.UpsertTransformer deleteResolver = (existing, update) -> + { + if (update instanceof TestStateMetadata) + { + assert isPartition(existing); + return deleteMetadata(existing, PER_MUTATION); + } + return null; + }; + + // Now delete the data in the reverse order of the insertion to satisfy the same constraints. + for (int i = COUNT - PER_MUTATION; i >= 0; i -= PER_MUTATION) + { + if (i < writeProgress.get()) + { + // Reduce the writeProgress so that we can start deleting a batch. + writeProgress.set(writeProgress.get() - PROGRESS_UPDATE); + // Wait until all reader threads have completed the current pass. + writeProgressAck.set(0); + while (writeProgressAck.get() + 1 < 1 << threadIdx.get()) + Thread.yield(); + } + + ByteComparable b = pkeys[(i / PER_MUTATION) % pkeys.length]; + TestRangeState partitionMarker = new TestStateMetadata<>(metadata(b)); + List> ranges = new ArrayList<>(); + ByteComparable cprefix = null; + if (choices.get(i / PER_MUTATION) && r.nextBoolean()) + { + // Delete the whole branch in one range + ranges.add(makeRangeCovering(ckeys[i])); + } + else + { + // A range for each entry + if (choices.get(i / PER_MUTATION)) + cprefix = ckeys[i]; + for (int j = 0; j < PER_MUTATION; ++j) + ranges.add(makeRangeCovering(ckeys[i + j])); + } + + RangeTrie deletion = RangeTrie.merge(ranges, Trie.throwingResolver()); + if (cprefix != null) + deletion = deletion.prefixedBy(cprefix); + deletion = TrieUtil.withRootMetadata(deletion, partitionMarker); + deletion = deletion.prefixedBy(b); + + delete(trie, deletion, deleteResolver, forcedCopyCheckerRanges); + } } catch (Throwable t) { @@ -340,6 +458,14 @@ public C resolve(Collection contents) Assert.fail("Got errors:\n" + errors); } + private static RangeTrie makeRangeCovering(ByteComparable cprefix) + { + return RangeTrie.range(swapTerminator(cprefix, ByteSource.LT_NEXT_COMPONENT), + swapTerminator(cprefix, ByteSource.GT_NEXT_COMPONENT), + VERSION, + TestRangeState.COVERED); + } + static ByteComparable nextBranch(ByteComparable key) { return version -> { @@ -407,4 +533,126 @@ public void checkEntries(String location, Assert.assertEquals("Id sum" + location, idMax * (idMax + 1) / 2, idSum); } } + + static abstract class TestRangeState implements RangeState + { + static final TestRangeState COVERED = new TestRangeCoveringState(); + static final TestRangeState RANGE_START = new TestRangeBoundary(Direction.FORWARD); + static final TestRangeState RANGE_END = new TestRangeBoundary(Direction.REVERSE); + } + + static class TestRangeCoveringState extends TestRangeState + { + @Override + public boolean isBoundary() + { + return false; + } + + @Override + public TestRangeCoveringState precedingState(Direction direction) + { + return this; + } + + @Override + public TestRangeState restrict(boolean applicableBefore, boolean applicableAfter) + { + throw new AssertionError(); + } + + @Override + public TestRangeState asBoundary(Direction direction) + { + return direction.isForward() ? RANGE_START : RANGE_END; + } + + @Override + public String toString() + { + return "COVERING"; + } + } + + static class TestRangeBoundary extends TestRangeState + { + final Direction direction; + + TestRangeBoundary(Direction direction) + { + this.direction = direction; + } + + @Override + public boolean isBoundary() + { + return true; + } + + @Override + public TestRangeState precedingState(Direction direction) + { + return direction == this.direction ? null : COVERED; + } + + @Override + public TestRangeState restrict(boolean applicableBefore, boolean applicableAfter) + { + if (direction.isForward() && !applicableBefore || !direction.isForward() && !applicableAfter) + return null; + return this; + } + + @Override + public TestRangeState asBoundary(Direction direction) + { + throw new AssertionError(); + } + + @Override + public String toString() + { + return direction.isForward() ? "START" : "END"; + } + } + + static class TestStateMetadata extends TestRangeState + { + final C metadata; + + TestStateMetadata(C metadata) + { + this.metadata = metadata; + } + + @Override + public boolean isBoundary() + { + return true; + } + + @Override + public TestRangeState precedingState(Direction direction) + { + return null; + } + + @Override + public TestRangeState restrict(boolean applicableBefore, boolean applicableAfter) + { + return this; // metadata should survive ranges + } + + @Override + public TestRangeState asBoundary(Direction direction) + { + throw new AssertionError(); + } + + @Override + public String toString() + { + return metadata.toString(); + } + } } diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java index 7eb35101721d..e23a65a6af33 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java @@ -101,6 +101,12 @@ void apply(InMemoryTrie trie, Trie mutation, InMemoryBaseTrie. trie.apply(mutation, mergeResolver, forcedCopyChecker); } + @Override + void delete(InMemoryTrie trie, RangeTrie deletion, InMemoryBaseTrie.UpsertTransformer mergeResolver, Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException + { + trie.apply(deletion, mergeResolver, forcedCopyChecker); + } + @Override boolean isPartition(Content c) { @@ -113,14 +119,20 @@ Content mergeMetadata(Content c1, Content c2) return ((Metadata) c1).mergeWith((Metadata) c2); } + @Override + Content deleteMetadata(Content c1, int entriesToRemove) + { + return ((Metadata) c1).delete(entriesToRemove); + } + @Override void printStats(InMemoryTrie trie, Predicate> forcedCopyChecker) { System.out.format("Reuse %s %s atomicity %s on-heap %,d (+%,d) off-heap %,d\n", trie.cellAllocator.getClass().getSimpleName(), trie.bufferType, - forcedCopyChecker == NO_ATOMICITY ? "none" : - forcedCopyChecker == FORCE_ATOMIC ? "atomic" : "consistent partition", + forcedCopyChecker == this.noAtomicity() ? "none" : + forcedCopyChecker == this.forceAtomic() ? "atomic" : "consistent partition", trie.usedSizeOnHeap(), trie.unusedReservedOnHeapMemory(), trie.usedSizeOffHeap()); @@ -193,6 +205,16 @@ Metadata mergeWith(Metadata other) return m; } + Metadata delete(int entriesToRemove) + { + assert updateCount >= entriesToRemove; + if (updateCount == entriesToRemove) + return null; + Metadata m = new Metadata(pk); + m.updateCount = updateCount - entriesToRemove; + return m; + } + @Override public String toString() { diff --git a/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java b/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java index 24b1c0bc6eef..9a24db95de45 100644 --- a/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java @@ -444,6 +444,8 @@ public void testIntersection(String message, List expected, Trie expected, Trie trie) @@ -502,6 +504,52 @@ public void testIntersectionTries(String message, List expected, Trie expected, Trie trie, TrieSet[] sets) + { + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + checkEqual(message + " b" + bits, expected, trie); + } + else + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + TrieSet set = sets[toRemove]; + testIntersectionTriesByRangeApplyTo(message + " " + toRemove, expected, + applySet(set, trie), + Arrays.stream(sets) + .filter(x -> x != set) + .toArray(TrieSet[]::new) + ); + } + } + } + + private Trie applySet(TrieSet set, Trie trie) + { + // Convert the set to a range trie. Do this by reinterpreting the cursor and avoiding verification + // (instead of e.g. RangeTrie.fromSet(set, TrieSetCursor.RangeState.END_START_PREFIX)), + // because some of the sets we use here are open and thus not valid range tries. + RangeTrie setAsRangeTrie = new RangeTrie<>() + { + @Override + public RangeCursor makeCursor(Direction direction) + { + throw new AssertionError(); + } + + @Override + public RangeCursor cursor(Direction direction) + { + // disable debug verification (cursor is already checked by TrieSet.cursor()) + return set.cursor(direction); + } + }; + return setAsRangeTrie.applyTo(trie, (range, value) -> range.applicableBefore ? value : null); + } + private static InMemoryTrie duplicateTrie(Trie trie) { try @@ -516,6 +564,39 @@ private static InMemoryTrie duplicateTrie(Trie trie) } } + public void testIntersectionInMemoryTrieDelete(String message, List expected, Trie trie, TrieSet[] sets) + { + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + checkEqual(message + " b" + bits, expected, trie); + } + else + { + try + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + TrieSet set = sets[toRemove]; + InMemoryTrie ix = duplicateTrie(trie); + ix.delete(set.weakNegation()); + testIntersectionInMemoryTrieDelete(message + " " + toRemove, expected, + ix, + Arrays.stream(sets) + .filter(x -> x != set) + .toArray(TrieSet[]::new) + ); + } + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + } + } + + @Test public void testReturnsContentOnPrefix() throws TrieSpaceExhaustedException { diff --git a/test/unit/org/apache/cassandra/db/tries/TrieUtil.java b/test/unit/org/apache/cassandra/db/tries/TrieUtil.java index 013730090941..08ad887174c9 100644 --- a/test/unit/org/apache/cassandra/db/tries/TrieUtil.java +++ b/test/unit/org/apache/cassandra/db/tries/TrieUtil.java @@ -367,6 +367,11 @@ public static Trie withRootMetadata(Trie wrapped, T metadata) return wrapped.mergeWith(Trie.singleton(ByteComparable.EMPTY, VERSION, metadata), Trie.throwingResolver()); } + public static > RangeTrie withRootMetadata(RangeTrie wrapped, S metadata) + { + return wrapped.mergeWith(RangeTrie.singleton(ByteComparable.EMPTY, VERSION, metadata), Trie.throwingResolver()); + } + static Trie directTrie(String... points) throws TrieSpaceExhaustedException { InMemoryTrie trie = InMemoryTrie.shortLived(VERSION); From ecd1f10f83c8c052b8378f8d3ad9caace38a24f8 Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Fri, 16 May 2025 17:05:07 +0300 Subject: [PATCH 15/22] Add DeletionAwareTrie Deletion-aware tries combine data and deletion tries. The cursor of a deletion-aware trie walks the data part of the trie, but also provides a `deletionBranchCursor` that can return a deletion/ tombstone branch covering the current position and the branch below it as a range trie. Such a branch can be given only once for any path in the trie (i.e. there cannot be a deletion branch covering another deletion branch). Deletion-aware merges and updates to in-memory tries take deletion branches into account when merging data so that deleted data is not produced in the resulting merge. --- .../cassandra/db/memtable/TrieMemtable.java | 2 +- .../apache/cassandra/db/tries/BaseTrie.java | 8 +- .../db/tries/CollectionMergeCursor.java | 370 ++++++++++- .../org/apache/cassandra/db/tries/Cursor.java | 17 +- .../db/tries/DeletionAwareCursor.java | 265 ++++++++ .../db/tries/DeletionAwareMergeSource.java | 224 +++++++ .../cassandra/db/tries/DeletionAwareTrie.java | 419 +++++++++++++ .../db/tries/DepthAdjustedCursor.java | 178 ++++++ .../db/tries/FlexibleMergeCursor.java | 272 ++++++++ .../cassandra/db/tries/InMemoryBaseTrie.java | 186 ++++-- .../db/tries/InMemoryDeletionAwareTrie.java | 459 ++++++++++++++ .../cassandra/db/tries/InMemoryRangeTrie.java | 44 +- .../cassandra/db/tries/InMemoryReadTrie.java | 264 ++++---- .../cassandra/db/tries/InMemoryTrie.java | 38 +- .../apache/cassandra/db/tries/InMemoryTrie.md | 71 ++- .../db/tries/IntersectionCursor.java | 55 +- .../cassandra/db/tries/MergeCursor.java | 212 ++++++- .../cassandra/db/tries/PrefixedCursor.java | 32 + .../cassandra/db/tries/RangeApplyCursor.java | 23 + .../cassandra/db/tries/RangeCursor.java | 73 +++ .../apache/cassandra/db/tries/RangeTrie.java | 14 +- .../cassandra/db/tries/SingletonCursor.java | 57 ++ .../org/apache/cassandra/db/tries/Trie.md | 221 ++++++- .../apache/cassandra/db/tries/TrieDumper.java | 83 ++- .../apache/cassandra/db/tries/TrieSet.java | 2 +- .../cassandra/db/tries/TrieTailsIterator.java | 140 ++++- .../db/tries/VerificationCursor.java | 120 +++- .../utils/bytecomparable/ByteComparable.java | 11 + .../cassandra/db/tries/CellReuseTest.java | 2 +- .../cassandra/db/tries/CombinedDataPoint.java | 90 +++ .../db/tries/ConsistencyTestBase.java | 233 ++++--- .../apache/cassandra/db/tries/DataPoint.java | 295 +++++++++ .../tries/DeletionAwareIntersectionTest.java | 303 +++++++++ .../db/tries/DeletionAwareMergeTest.java | 583 ++++++++++++++++++ .../db/tries/DeletionAwareRandomizedTest.java | 579 +++++++++++++++++ .../db/tries/DeletionAwareTestBase.java | 253 ++++++++ .../tries/DeletionBranchConsistencyTest.java | 304 +++++++++ .../cassandra/db/tries/DeletionMarker.java | 218 +++++++ ...emoryDeletionAwareTrieConsistencyTest.java | 278 +++++++++ ...InMemoryDeletionAwareTrieThreadedTest.java | 122 ++++ .../db/tries/InMemoryTrieConsistencyTest.java | 11 +- .../db/tries/IntersectionTrieTest.java | 4 +- .../apache/cassandra/db/tries/LivePoint.java | 105 ++++ .../db/tries/RangeTrieIntersectionTest.java | 2 +- .../cassandra/db/tries/RangesTrieSetTest.java | 2 +- .../apache/cassandra/db/tries/TrieUtil.java | 9 + 46 files changed, 6862 insertions(+), 391 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/tries/DeletionAwareCursor.java create mode 100644 src/java/org/apache/cassandra/db/tries/DeletionAwareMergeSource.java create mode 100644 src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java create mode 100644 src/java/org/apache/cassandra/db/tries/DepthAdjustedCursor.java create mode 100644 src/java/org/apache/cassandra/db/tries/FlexibleMergeCursor.java create mode 100644 src/java/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrie.java create mode 100644 test/unit/org/apache/cassandra/db/tries/CombinedDataPoint.java create mode 100644 test/unit/org/apache/cassandra/db/tries/DataPoint.java create mode 100644 test/unit/org/apache/cassandra/db/tries/DeletionAwareIntersectionTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/DeletionAwareMergeTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/DeletionAwareRandomizedTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/DeletionAwareTestBase.java create mode 100644 test/unit/org/apache/cassandra/db/tries/DeletionBranchConsistencyTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/DeletionMarker.java create mode 100644 test/unit/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrieConsistencyTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrieThreadedTest.java create mode 100644 test/unit/org/apache/cassandra/db/tries/LivePoint.java diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java index f290b526eaff..3334b21a872b 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java @@ -753,7 +753,7 @@ public DecoratedKey maxPartitionKey() } } - static class PartitionIterator extends TrieTailsIterator + static class PartitionIterator extends TrieTailsIterator.Plain { final TableMetadata metadata; final EnsureOnHeap ensureOnHeap; diff --git a/src/java/org/apache/cassandra/db/tries/BaseTrie.java b/src/java/org/apache/cassandra/db/tries/BaseTrie.java index 5984808c564a..64dad35ad0c3 100644 --- a/src/java/org/apache/cassandra/db/tries/BaseTrie.java +++ b/src/java/org/apache/cassandra/db/tries/BaseTrie.java @@ -99,13 +99,13 @@ default void forEachEntry(BiConsumer consumer) default void forEachEntry(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); - cursor.process(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); + cursor.process(new TrieEntriesWalker.WithConsumer<>(consumer, cursor.byteComparableVersion())); // Note: we can't do the ValueConsumer trick here, because the implementation requires state and cannot be // implemented with default methods alone. } /// Process the trie using the given [Cursor.Walker]. - default R process(Direction direction, Cursor.Walker walker) + default R process(Direction direction, Cursor.Walker walker) { return cursor(direction).process(walker); } @@ -121,7 +121,7 @@ default void forEachValueSkippingBranches(Direction direction, ValueConsumer default void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); - cursor.processSkippingBranches(new TrieEntriesWalker.WithConsumer(consumer, cursor.byteComparableVersion())); + cursor.processSkippingBranches(new TrieEntriesWalker.WithConsumer<>(consumer, cursor.byteComparableVersion())); // Note: we can't do the ValueConsumer trick here, because the implementation requires state and cannot be // implemented with default methods alone. } @@ -151,7 +151,7 @@ default String dump() /// Constuct a textual representation of the trie using the given content-to-string mapper. default String dump(Function contentToString) { - return process(Direction.FORWARD, new TrieDumper<>(contentToString)); + return process(Direction.FORWARD, new TrieDumper.Plain<>(contentToString)); } /// Returns the ordered entry set of this trie's content as an iterable. diff --git a/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java b/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java index 40ef93856d3e..7c38e3207600 100644 --- a/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java +++ b/src/java/org/apache/cassandra/db/tries/CollectionMergeCursor.java @@ -19,7 +19,9 @@ package org.apache.cassandra.db.tries; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.function.BiFunction; import java.util.function.IntFunction; @@ -107,7 +109,6 @@ CollectionMergeCursor(Trie.CollectionMergeResolver resolver, Direction di for (I src : inputs) { C cursor = extractor.apply(src, direction); - assert cursor.depth() == 0; if (i >= 0) heap[i] = cursor; else @@ -115,8 +116,12 @@ CollectionMergeCursor(Trie.CollectionMergeResolver resolver, Direction di ++i; } // The cursors are all currently positioned on the root and thus in valid heap order. + assert Arrays.stream(heap).allMatch(x -> equalCursor(head, x)); } + // TODO: consider if making these return boolean would be an optimization for some calls + // TODO: consider if head call can be included in these + /// Interface for internal operations that can be applied to selected top elements of the heap. interface HeapOp> { @@ -245,6 +250,55 @@ boolean isExhausted() return head.depth() < 0; } + /// Add the given source to this merger, replacing an exhausted one. Returns false if there is no place (i.e. no + /// source is exhausted). + boolean addCursor(C cursor) + { + if (isExhausted()) + { + // easy case + head = cursor; + return true; + } + + int index; + for (index = 0; index < heap.length; ++index) + if (heap[index].depth() < 0) + break; + if (index == heap.length) + return false; + + heapifyUp(cursor, index); + return true; + } + + /// Pull the given state up in the heap from the given index until it finds its proper place. + private void heapifyUp(C item, int index) + { + while (true) + { + if (index == 0) + { + if (greaterCursor(direction, head, item)) + { + heap[0] = head; + head = item; + return; + } + else + break; + } + int prev = (index - 1) / 2; + + // If the parent is lesser or equal, the invariant has been restored. + if (!greaterCursor(direction, heap[prev], item)) + break; + heap[index] = heap[prev]; + index = prev; + } + heap[index] = item; + } + @Override public int advance() { @@ -458,7 +512,7 @@ S getContent(RangeCursor item) @Override public RangeCursor tailCursor(Direction direction) { - List> inputs = new ArrayList<>(heap.length); + List> inputs = new ArrayList<>(heap.length + 1); inputs.add(head); applyToAllOnHeap((self, cursor, index) -> { @@ -478,4 +532,316 @@ else if (cursor.precedingState() != null) } } + /// Collection merge cursor for deletion-aware tries. + /// + /// This cursor efficiently merges multiple deletion-aware tries by walking their cursors in parallel + /// while properly handling both live data and deletion metadata. It extends the basic collection merge + /// functionality with deletion-aware semantics, including proper deletion application and branch management. + /// + /// The implementation maintains a separate merge cursor for deletion branches (`relevantDeletions`) and + /// coordinates between live data and deletions to ensure correct deletion application during iteration. + static class DeletionAware> + extends CollectionMergeCursor> implements DeletionAwareCursor + { + final BiFunction deleter; + final Trie.CollectionMergeResolver deletionResolver; + + /// Critical performance optimization flag. When true, guarantees that if one merge source + /// has a deletion branch at some position, the other source cannot have deletion branches + /// below or above that position. This allows us to skip walking the data trie to look for + /// lower-level deletion branches when merging. If the flag is false, we cannot know where + /// in the covered branch we may have a deletion, thus to be sure to find all we _must_ + /// walk the whole data subtrie. This can be terribly expensive. + /// + /// If we can guarantee that deletions always come at the same points in each path (e.g. at + /// partition roots), we can use this optimization. + final boolean deletionsAtFixedPoints; + + Range relevantDeletions; + int deletionBranchDepth = -1; + + // TODO: Keep track of deletion state to avoid repeated calls to `relevantDeletions.precedingState` + // TODO: Consider not applying deletions to live and making a `Shadowable` deletion-aware variation, delaying + // the deleted data removal to after transformations have been applied. + + enum DeletionState + { + NONE, + MATCHING, + AHEAD + } + DeletionState relevantDeletionsState = DeletionState.NONE; + + /// Creates a deletion-aware collection merge cursor with configurable deletion optimization. + /// + /// @param liveResolver resolver for merging live data content + /// @param deletionResolver resolver for merging deletion metadata + /// @param deleter function to apply deletions to live data + /// @param deletionsAtFixedPoints optimization flag for deletion handling + /// @param direction iteration direction (forward or reverse) + /// @param inputs collection of input sources to merge + /// @param extractor function to extract deletion-aware cursors from inputs + DeletionAware(Trie.CollectionMergeResolver liveResolver, + Trie.CollectionMergeResolver deletionResolver, + BiFunction deleter, + boolean deletionsAtFixedPoints, + Direction direction, + Collection inputs, + BiFunction> extractor) + { + super(liveResolver, + direction, + inputs, + DeletionAwareCursor[]::new, + extractor); + // We will add deletion sources to the above as we find them. + this.deletionResolver = deletionResolver; + this.deleter = deleter; + this.deletionsAtFixedPoints = deletionsAtFixedPoints; + // Initialize deletion merger as null - we'll create it lazily when needed + relevantDeletions = null; + maybeAddDeletionsBranch(this.depth()); + } + + @Override + public int advance() + { + return maybeAddDeletionsBranch(super.advance()); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + return maybeAddDeletionsBranch(super.skipTo(skipDepth, skipTransition)); + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + return (branchHasMultipleSources() || relevantDeletionsState == DeletionState.MATCHING) + ? advance() + : maybeAddDeletionsBranch(super.advanceMultiple(receiver)); + } + + /// Adjusts the deletion state based on the relative positions of deletion and content cursors. + /// This determines how deletions should be applied to live data at the current position. + void adjustDeletionState(int deletionDepth, int contentDepth, int contentTransition) + { + if (deletionDepth < 0) + relevantDeletionsState = DeletionState.NONE; + else if (deletionDepth < contentDepth) + relevantDeletionsState = DeletionState.AHEAD; + else if (direction.lt(contentTransition, relevantDeletions.incomingTransition())) + relevantDeletionsState = DeletionState.AHEAD; + else + relevantDeletionsState = DeletionState.MATCHING; + } + + /// Manages deletion branches during cursor advancement. + /// This method coordinates between live data cursors and deletion cursors to ensure + /// proper deletion application at each position. + int maybeAddDeletionsBranch(int depth) + { + int contentTransition = incomingTransition(); + int deletionDepth; + switch (relevantDeletionsState) + { + case MATCHING: + deletionDepth = relevantDeletions.skipTo(depth, contentTransition); + break; + case AHEAD: + deletionDepth = relevantDeletions.skipToWhenAhead(depth, contentTransition); + break; + default: + deletionDepth = -1; + break; + } + + if (depth <= deletionBranchDepth) + { + // ascended above the common deletions root, we can report deletion branches again + deletionBranchDepth = -1; + assert deletionDepth < 0; + } + + // If the branch is single-source, its deletions cannot affect the merge as they can't delete its own data. + // (Note that covering deletions from other sources can still affect it though.) + // Otherwise we need to get the deletions from all sources to track and apply them. However, if we are + // operating in fixed-deletion-points mode, we only need to do this if we haven't yet passed that deletion + // point on the current path. + if (branchHasMultipleSources() && (!deletionsAtFixedPoints || deletionDepth < 0)) + { + maybeAddDeletionsBranch(head, 0); + applyToSelectedInHeap((self, cursor, index) -> ((DeletionAware) self).maybeAddDeletionsBranch(cursor, index)); + // TODO: if using boolean return from above, also optimize addCursor to not do heapification, calling + // the downward process here instead. + if (relevantDeletions != null) + deletionDepth = relevantDeletions.depth(); // newly inserted cursors may have adjusted the deletion cursor's position + } + + adjustDeletionState(deletionDepth, depth, contentTransition); + return depth; + } + + /// Resolves content by applying deletions to live data. + /// This is the core method that implements deletion application during iteration. + @Override + T resolveContent() + { + T content = super.resolveContent(); + if (content == null) + return null; + + D deletion; + switch (relevantDeletionsState) + { + case MATCHING: + deletion = relevantDeletions.state(); + break; + case AHEAD: + deletion = relevantDeletions.precedingState(); + break; + default: + deletion = null; + } + if (deletion == null) + return content; + return deleter.apply(deletion, content); + } + + /// Adds deletion branches from individual cursors to the collection merge. + /// This method implements the core deletion merging logic with optimization support. + void maybeAddDeletionsBranch(DeletionAwareCursor cursor, int ignoredIndex) + { + RangeCursor deletionsBranch = cursor.deletionBranchCursor(direction); + if (deletionsBranch == null) + return; + + // Make sure the deletion branch depths are in the same basis as the data paths + deletionsBranch = DepthAdjustedCursor.make(deletionsBranch, cursor.depth(), cursor.incomingTransition()); + + // Create relevantDeletions Range if this is the first deletion branch we encounter + if (relevantDeletions == null) + { + relevantDeletions = new Range(deletionResolver, + direction, + Collections.nCopies(heap.length + 1, + RangeCursor.done(direction, byteComparableVersion())), + (c, dir) -> c); + } + + boolean succeeded = relevantDeletions.addCursor(deletionsBranch); + assert succeeded : "Too many deletion cursors added likely due to non-overlap of deletion branches violation."; + } + + /// Returns the deletion branch cursor for the current position. + /// This method implements the deletionsAtFixedPoints optimization for collection merges. + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + int depth = depth(); + if (deletionBranchDepth != -1 && depth > deletionBranchDepth) + return null; // already covered by a deletion branch, if there is any here it will be reflected in that + + if (!branchHasMultipleSources()) + return head.deletionBranchCursor(direction); + + // We are positioned at a multi-source branch. If one has a deletion branch, we must combine it with the + // deletion-tree branch of the others to make sure that we merge any lower-level deletion branch with it. + + // We have already created the merge of all present deletion branches in relevantDeletions. If that's empty, + // or not rooted here, there's no deletion to report. + if (relevantDeletions == null || relevantDeletions.depth() != depth) + return null; + + List> deletions = new ArrayList<>(heap.length + 1); + // Note: during the data path walk we add cursors to relevantDeletions as we find them. This copy, + // however, is walked independently, so we need to make sure it has the ability to walk the data trie + // to find and lower-level deletion sources. + + maybeAddDeletionTrieBranch(head, 0, deletions); + applyToSelectedInHeap((self, cursor, index) -> maybeAddDeletionTrieBranch(cursor, index, deletions)); + if (deletions.isEmpty()) + return null; + + deletionBranchDepth = depth; + return new Range(deletionResolver, direction, deletions, (c, d) -> c); + } + + /// Adds a deletion trie branch for the given cursor. This means either the deletion branch that it presents, + /// or, in the case where we accept non-aligned deletions, any deletion branch that may be present in its + /// substructure. + void maybeAddDeletionTrieBranch(DeletionAwareCursor cursor, int ignoredIndex, List> deletions) + { + RangeCursor deletionsBranch = cursor.deletionBranchCursor(direction); + if (deletionsBranch != null) + deletions.add(deletionsBranch); + else if (!deletionsAtFixedPoints) + deletions.add(new DeletionsTrieCursor<>(cursor.tailCursor(cursor.direction()))); + // Otherwise there is no need to track the subtrie. If there are deletions, they must be presented here. + } + + @Override + public T content() + { + return maybeCollectContent(); + } + + /// + /// Gets content from a specific cursor (required by CollectionMergeCursor). + /// + @Override + T getContent(DeletionAwareCursor cursor) + { + return cursor.content(); + } + + @Override + public DeletionAwareCursor tailCursor(Direction dir) + { + RangeCursor deletions = null; + switch (relevantDeletionsState) + { + case NONE: + break; + case MATCHING: + deletions = relevantDeletions.tailCursor(direction); + break; + case AHEAD: + deletions = relevantDeletions.precedingStateCursor(direction); + break; + } + + if (deletions != null) + { + // Because deletions branch is already active (and no new one can be introduced now), we treat the + // sources as plain tries. + Cursor source; + + if (!branchHasMultipleSources()) + source = head.tailCursor(dir); + else + { + List> inputs = new ArrayList<>(heap.length + 1); + inputs.add(head); + applyToSelectedInHeap((self, cursor, index) -> inputs.add(cursor)); + + source = new Plain<>(resolver, dir, inputs, DeletionAwareCursor::tailCursor); + } + + return new RangeApplyCursor.DeletionAwareDataBranch<>(deleter, deletions, source); + } + else + { + if (!branchHasMultipleSources()) + return head.tailCursor(dir); + + List> inputs = new ArrayList<>(heap.length + 1); + inputs.add(head); + applyToSelectedInHeap((self, cursor, index) -> inputs.add(cursor)); + + return new DeletionAware<>(resolver, deletionResolver, deleter, deletionsAtFixedPoints, dir, inputs, DeletionAwareCursor::tailCursor); + } + } + } } diff --git a/src/java/org/apache/cassandra/db/tries/Cursor.java b/src/java/org/apache/cassandra/db/tries/Cursor.java index bcec07d51f5b..f442bbbae7b8 100644 --- a/src/java/org/apache/cassandra/db/tries/Cursor.java +++ b/src/java/org/apache/cassandra/db/tries/Cursor.java @@ -200,6 +200,21 @@ default T advanceToContent(ResettingTransitionsReceiver receiver) /// @return the new depth, always <= previous depth + 1; -1 if the trie is exhausted int skipTo(int skipDepth, int skipTransition); + /// A version of [#skipTo] which checks if the requested position is ahead of the cursor's current position and only + /// advances if it is. This can only be used if the [#skipTo] instruction is issued from a position that is behind + /// this cursor's (i.e. if the [#skipTo] request is to descend, it is assumed to descend from a position _before_ + /// this cursor's and will not be acted on). + /// + /// Used for parallel walks when one of the source cursors is known to be ahead of the current position. + default int skipToWhenAhead(int skipDepth, int skipTransition) + { + int depth = depth(); + if (skipDepth < depth || skipDepth == depth && skipTransition > incomingTransition()) + return skipTo(skipDepth, skipTransition); + else + return depth; + } + /// Descend into the cursor with the given path. /// /// @return True if the descent is positioned at the end of the given path, false if the trie did not have a path @@ -257,7 +272,7 @@ interface Walker extends Cursor.ResettingTransitionsReceiver /// Process the trie using the given [Walker]. /// This method should only be called on a freshly constructed cursor. - default R process(Cursor.Walker walker) + default R process(Cursor.Walker walker) { assert depth() == 0 : "The provided cursor has already been advanced."; T content = content(); // handle content on the root node diff --git a/src/java/org/apache/cassandra/db/tries/DeletionAwareCursor.java b/src/java/org/apache/cassandra/db/tries/DeletionAwareCursor.java new file mode 100644 index 000000000000..c54efa68e710 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/DeletionAwareCursor.java @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.function.BiFunction; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// Cursor interface for deletion-aware tries that provides access to both live data and deletion branches. +/// +/// This cursor extends the basic [Cursor] interface to support the dual nature of deletion-aware tries, +/// where live data and deletion information coexist in a unified structure. The cursor walks the live +/// data portion of the trie while providing access to deletion branches through the +/// [#deletionBranchCursor] method. +/// +/// The cursor behaves like a standard trie cursor for live data, supporting all standard navigation and +/// content access operations inherited from [Cursor]. It can also be used as a plain trie cursor for +/// processing or iteration methods and classes, in which case only live data will be presented. +/// +/// At any position, this cursor can provide access to deletion branches through [#deletionBranchCursor], +/// which returns a [RangeCursor] covering deletion ranges rooted at the current position. The deletion +/// information is only reachable after taking and following deletion branches. When a consumer is interested +/// in the deletion information, it can be merged into the main tree using [LiveAndDeletionsMergeCursor] or +/// presented as the full range trie via [DeletionsTrieCursor]. +/// +/// Deletion-aware cursors must maintain strict structural invariants to ensure correctness and efficiency: +/// +/// **Non-Overlapping Deletion Branches**: No deletion branch can be covered by another deletion branch. +/// When a deletion branch exists at a given node, all descendants of that node must have null deletion +/// branches. This prevents nested deletion scopes and simplifies merge algorithms. +/// +/// **Well-Formed Deletion Branches**: Each deletion branch must be a properly constructed range trie: +/// - It cannot start or end with an active deletion (no open-ended ranges at boundaries). +/// - Every deletion opened by an entry must be closed by the next entry. +/// - Preceding state must be correctly reported for all positions. +/// +/// **Deletion Consistency**: There cannot be live entries in the trie that are deleted by deletion +/// branches in the same trie. This ensures that the trie represents a consistent view where deletions +/// have been properly applied. +/// +/// @param The content type for live data in the trie +/// @param The deletion marker type, must extend `RangeState` +public interface DeletionAwareCursor> extends Cursor +{ + /// Returns the deletion branch rooted at the current cursor position, if any. + /// + /// This method provides access to deletion information associated with the current position in the + /// trie and nodes below it. The deletion branch is represented as a [RangeCursor] that can cover + /// ranges of keys with deletion markers. It is presented as a tail cursor for the current position, + /// i.e. it starts with depth 0 and cannot extend beyond the current position. + /// + /// When this method returns a non-null deletion branch, the source cursor is not allowed to return another deletion + /// branch in the covered branch. In other words, for any given path in the trie there must be at most one node + /// where [#deletionBranchCursor] is non-null. + /// + /// @param direction The direction for traversing the deletion branch. + /// @return A range cursor for deletions at this position, or null if no deletion branch is defined at this level. + RangeCursor deletionBranchCursor(Direction direction); + + @Override + DeletionAwareCursor tailCursor(Direction direction); + + + /// Process the trie using the given [DeletionAwareTrie.DeletionAwareWalker], providing access to both live and + /// deletion branches. + default R process(DeletionAwareTrie.DeletionAwareWalker walker) + { + assert depth() == 0 : "The provided cursor has already been advanced."; + int prevDepth = 0; + + while (true) + { + RangeCursor deletionBranch = deletionBranchCursor(direction()); + if (deletionBranch != null && walker.enterDeletionsBranch()) + { + processDeletionBranch(walker, deletionBranch); + walker.exitDeletionsBranch(); + } + T content = content(); // handle content on the root node + if (content != null) + walker.content(content); + + int currDepth = advanceMultiple(walker); + if (currDepth < 0) + break; + if (currDepth <= prevDepth) + walker.resetPathLength(currDepth - 1); + walker.addPathByte(incomingTransition()); + prevDepth = currDepth; + } + + return walker.complete(); + } + + /// Process a deletion branch using the given walker. + private static void processDeletionBranch(DeletionAwareTrie.DeletionAwareWalker walker, Cursor cursor) + { + assert cursor.depth() == 0 : "The provided cursor has already been advanced."; + D content = cursor.content(); // handle content on the root node + if (content == null) + content = cursor.advanceToContent(walker); + + while (content != null) + { + walker.deletionMarker(content); + content = cursor.advanceToContent(walker); + } + } + + /// A cursor merging the live data and deletion markers of a deletion-aware trie into a combined trie. + class LiveAndDeletionsMergeCursor, Z> + extends FlexibleMergeCursor.WithMappedContent, RangeCursor, Z> + { + LiveAndDeletionsMergeCursor(BiFunction resolver, DeletionAwareCursor c1) + { + super(resolver, c1); + postAdvance(c1.depth()); + } + + LiveAndDeletionsMergeCursor(BiFunction resolver, DeletionAwareCursor c1, RangeCursor c2) + { + super(resolver, c1, c2); + postAdvance(c1.depth()); + } + + @Override + int postAdvance(int depth) + { + if (state == State.C1_ONLY) + { + RangeCursor deletionsBranch = c1.deletionBranchCursor(direction); + if (deletionsBranch != null) + addCursor(deletionsBranch); + } + return depth; + } + + @Override + public LiveAndDeletionsMergeCursor tailCursor(Direction direction) + { + switch (state) + { + case C1_ONLY: + return new LiveAndDeletionsMergeCursor<>(resolver, c1.tailCursor(direction)); + case AT_C2: + return new LiveAndDeletionsMergeCursor<>(resolver, new DeletionAwareCursor.Empty<>(direction, byteComparableVersion()), c2.tailCursor(direction)); + case AT_C1: + return new LiveAndDeletionsMergeCursor<>(resolver, c1.tailCursor(direction), c2.precedingStateCursor(direction)); + case AT_BOTH: + return new LiveAndDeletionsMergeCursor<>(resolver, c1.tailCursor(direction), c2.tailCursor(direction)); + default: + throw new AssertionError(); + } + } + } + + /// A cursor presenting the deletion markers of a deletion-aware trie. + /// + /// This cursor combines all deletion branches into a single trie. Because it is not known where a deletion branch + /// can be introduced, this cursor has to walk all nodes of the live trie that are not covered by a deletion branch, + /// returning (likely a lot of) unproductive branches where a deletion is not defined. + class DeletionsTrieCursor> + extends FlexibleMergeCursor, RangeCursor, D> implements RangeCursor + { + DeletionsTrieCursor(DeletionAwareCursor c1) + { + super(c1); + postAdvance(c1.depth()); + } + + @Override + public D state() + { + return c2 != null ? c2.state() : null; + } + + @Override + public D precedingState() + { + return c2 != null ? c2.precedingState() : null; + } + + @Override + public D content() + { + return c2 != null ? c2.content() : null; + } + + @Override + int postAdvance(int depth) + { + switch (state) + { + case AT_C2: + // already in deletion branch + break; + case C1_ONLY: + RangeCursor deletionsBranch = c1.deletionBranchCursor(direction); + if (deletionsBranch != null) + { + addCursor(deletionsBranch); + // deletion branches cannot be nested; skip past the current position in the main trie as we + // don't need to further track it inside this branch + c1.skipTo(depth, incomingTransition + direction.increase); + state = State.AT_C2; + } + break; + default: + throw new AssertionError("Deletion branch extends above its introduction"); + } + return depth; + } + + @Override + public RangeCursor tailCursor(Direction direction) + { + switch (state) + { + case AT_C2: + return c2.tailCursor(direction); + case C1_ONLY: + return new DeletionsTrieCursor<>(c1.tailCursor(direction)); + default: + throw new AssertionError("Deletion branch extends above its introduction"); + } + } + } + + class Empty> + extends Cursor.Empty implements DeletionAwareCursor + { + public Empty(Direction direction, ByteComparable.Version byteComparableVersion) + { + super(direction, byteComparableVersion); + } + + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + return null; + } + + @Override + public DeletionAwareCursor tailCursor(Direction direction) + { + return new DeletionAwareCursor.Empty<>(direction, byteComparableVersion()); + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/DeletionAwareMergeSource.java b/src/java/org/apache/cassandra/db/tries/DeletionAwareMergeSource.java new file mode 100644 index 000000000000..855d4c6340ff --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/DeletionAwareMergeSource.java @@ -0,0 +1,224 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.function.BiFunction; + +import javax.annotation.Nullable; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// A cursor applying deletions to a deletion-aware cursor, where the deletions can be dynamically added. +/// Based on [RangeApplyCursor] and used by [MergeCursor.DeletionAware] to process each source with the deletions of the +/// other. The cursor will present the content of the data trie modified by any applicable/covering range of the +/// deletion trie, and will leave the deletion branches unmodied (allowing the merger to process them). +class DeletionAwareMergeSource, E extends RangeState> implements DeletionAwareCursor +{ + final BiFunction resolver; + final Direction direction; + final DeletionAwareCursor data; + @Nullable RangeCursor deletions; + int deletionsDepthCorrection; + + boolean atDeletions; + + DeletionAwareMergeSource(BiFunction resolver, DeletionAwareCursor data) + { + this.direction = data.direction(); + this.resolver = resolver; + this.deletions = null; + this.data = data; + this.deletionsDepthCorrection = 0; + assert data.depth() == 0; + atDeletions = false; + } + + DeletionAwareMergeSource(BiFunction resolver, DeletionAwareCursor data, RangeCursor deletions) + { + this.direction = data.direction(); + this.resolver = resolver; + this.deletions = deletions; + this.data = data; + this.deletionsDepthCorrection = 0; + assert data.depth() == 0; + assert deletions == null || deletions.depth() == 0; + atDeletions = deletions != null; + } + + @Override + public int depth() + { + return data.depth(); + } + + @Override + public int incomingTransition() + { + return data.incomingTransition(); + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + assert deletions == null || deletions.byteComparableVersion() == data.byteComparableVersion() : + "Merging cursors with different byteComparableVersions: " + + deletions.byteComparableVersion() + " vs " + data.byteComparableVersion(); + return data.byteComparableVersion(); + } + + @Override + public int advance() + { + int newDataDepth = data.advance(); + + if (deletions == null) + return newDataDepth; + else if (atDeletions) // if both cursors were at the same position, always advance the deletions' cursor to catch up. + return skipDeletionsToDataPosition(newDataDepth); + else // otherwise skip deletions to the new data position only if it advances past the deletions' current position. + return maybeSkipDeletions(newDataDepth); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + int newDataDepth = data.skipTo(skipDepth, skipTransition); + + if (deletions == null) + return newDataDepth; + else if (atDeletions) // if both cursors were at the same position, always advance the deletions' cursor to catch up. + return skipDeletionsToDataPosition(newDataDepth); + else // otherwise skip deletions to the new data position only if it advances past the deletions' current position. + return maybeSkipDeletions(newDataDepth); + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + if (deletions == null) + return data.advanceMultiple(receiver); + + // While we are on a shared position, we must descend one byte at a time to maintain the cursor ordering. + if (atDeletions) + return skipDeletionsToDataPosition(data.advance()); + else // atData only + return maybeSkipDeletions(data.advanceMultiple(receiver)); + } + + int maybeSkipDeletions(int dataDepth) + { + int deletionsDepth = deletions.depth() + deletionsDepthCorrection; + + // If data position is at or before the deletions position, we are good. + if (deletionsDepth < dataDepth) + return setAtDeletionsAndReturnDepth(false, dataDepth); + + if (deletionsDepth == dataDepth) + { + int dataTrans = data.incomingTransition(); + int deletionsTrans = deletions.incomingTransition(); + if (direction.le(dataTrans, deletionsTrans)) + return setAtDeletionsAndReturnDepth(dataTrans == deletionsTrans, dataDepth); + } + + // Deletions cursor is before data cursor. + return skipDeletionsToDataPosition(dataDepth); + } + + private int skipDeletionsToDataPosition(int dataDepth) + { + // Skip deletions cursor to the data position; if that is beyond the branch's root, no need to skip, just leave it. + int dataTrans = data.incomingTransition(); + int deletionsSkipDepth = dataDepth - deletionsDepthCorrection; + int deletionsDepthUncorrected = deletionsSkipDepth > 0 ? deletions.skipTo(deletionsSkipDepth, dataTrans) : -1; + if (deletionsDepthUncorrected < 0) + return leaveDeletionsBranch(dataDepth); + else + return setAtDeletionsAndReturnDepth(deletionsDepthUncorrected + deletionsDepthCorrection == dataDepth && deletions.incomingTransition() == dataTrans, + dataDepth); + } + + private int leaveDeletionsBranch(int dataDepth) + { + deletions = null; + return setAtDeletionsAndReturnDepth(false, dataDepth); + } + + private int setAtDeletionsAndReturnDepth(boolean atDeletions, int depth) + { + this.atDeletions = atDeletions; + return depth; + } + + @Override + public T content() + { + T content = data.content(); + if (content == null) + return null; + if (deletions == null) + return content; + + E applicableDeletions = atDeletions ? deletions.content() : null; + if (applicableDeletions == null) + { + applicableDeletions = deletions.precedingState(); + if (applicableDeletions == null) + return content; + } + + return resolver.apply(applicableDeletions, content); + } + + @Override + public DeletionAwareMergeSource tailCursor(Direction direction) + { + if (atDeletions) + return new DeletionAwareMergeSource<>(resolver, data.tailCursor(direction), deletions.tailCursor(direction)); + else + return new DeletionAwareMergeSource<>(resolver, data.tailCursor(direction)); + } + + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + // Return unchanged, to be handled by MergeCursor. + return data.deletionBranchCursor(direction); + } + + public void addDeletions(RangeCursor deletions) + { + assert this.deletions == null; + assert deletions.depth() == 0; + this.deletions = deletions; + this.deletionsDepthCorrection = data.depth(); + this.atDeletions = true; + } + + public boolean hasDeletions() + { + return deletions != null; + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java b/src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java new file mode 100644 index 000000000000..2ca18e2cd25d --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java @@ -0,0 +1,419 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; + +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// Deletion-aware trie interface that combines live data and deletion information in a unified structure. +/// +/// This class implements the definitions of some simple deletion-aware tries (live singletons, deleted ranges), as +/// well as the main algebraic operations: +/// - intersecting a deletion-aware trie with a set/range, returning only the live paths and content covered by the +/// set, as well as any applicable range restricted to the bounds of the set; +/// - merging deletion-aware tries, applying the deletions of each set to the live data of the others and properly +/// combining the deletion branches of all sources. +/// +/// It also provides methods of processing and iterating over the live content of the trie, as well as means +/// of obtaining the full range deletion view of the trie and a combined data-with-deletions view. +/// +/// The structure of a deletion-aware trie presents the live data in its normal paths, and deleted ranges +/// in additional "deletion branches". The objective of this split it to be able to separately and efficiently +/// query the two: on one hand, to search for the closest live entry without having to walk paths leading to +/// deleted data, and on the other, to be able to find the covering deletions affecting any position in the +/// trie. With this design, both can be achieved in time proportional to the length of the key. +/// +/// For efficiency there can only be at most one deletion branch defined for any path in the trie. I.e. a deletion +/// branch cannot cover another deletion branch. To additionally improve merge performance we also support a mode +/// of operation where it is known that the depth at which a deletion can be introduced is determined in advance for +/// every path for all sources (in other words, that if one source defines a deletion branch at one point, none of the +/// other sources can define a deletion branch below it); this is the mode of operation intended for use in Cassandra +/// memtables and sstables, where deletion branches are defined at the root of each partition. +/// +/// This class expects deletions to always be specified as ranges, and the deletion boundaries to never coincide with +/// live keys or their prefixes (this property is provided by the prefix-free [ByteComparable] conversions). +/// While deletion branches can support point deletions, this support is somewhat restricted for efficiency reasons +/// (e.g. the coverage of positions under a point deletion is undefined). +/// +/// It is also expected that a deletion-aware trie does not contain any live data that is deleted by its own deletion +/// branches. If such data exists, whether it is preserved after transformations is undefined. +/// +/// See [DeletionAwareCursor] for details on cursor operations and [InMemoryDeletionAwareTrie] for the +/// concrete in-memory implementation. +/// +/// @param The content type for live data in the trie +/// @param The deletion marker type, must extend [RangeState] for range operations +public interface DeletionAwareTrie> +extends BaseTrie, DeletionAwareTrie> +{ + /// Creates a singleton deletion-aware trie containing only live data at the specified key. + /// + /// This method creates a trie with a single entry mapping the given byte-comparable key to the provided + /// content. The resulting trie contains no deletion information and behaves similarly to a regular + /// [Trie#singleton], but is compatible with deletion-aware operations. + /// + /// @param b The byte-comparable key for the content + /// @param byteComparableVersion The version to use for byte-comparable serialization + /// @param v The content to associate with the key + /// @return A deletion-aware trie containing the single key-value mapping + static > + DeletionAwareTrie singleton(ByteComparable b, ByteComparable.Version byteComparableVersion, T v) + { + return dir -> new SingletonCursor.DeletionAware<>(dir, b.asComparableBytes(byteComparableVersion), byteComparableVersion, v); + } + + /// Creates a deletion-aware trie containing a single deletion range. + /// + /// This method creates a trie that represents a deletion covering the range from `prefixInDataTrie`+`left` to + /// `prefixInDataTrie`+`right`. The deletion is presented as a deletion branch at the specified prefix, allowing + /// the user to take advantage of predefined deletion-branch positions. + /// + /// Range boundaries should be distinct from live data positions; the inclusivity of the boundary point and any + /// position below it is not specified or guaranteed to be preserved under trie transformations. + /// + /// @param prefixInDataTrie The position in the data trie where this deletion branch is rooted + /// @param left The left boundary of the deletion range + /// @param right The right boundary of the deletion range + /// @param byteComparableVersion The version to use for byte-comparable serialization + /// @param deletion A _covering_ range state that defines the deletion information + /// @return A deletion-aware trie containing the deletion range + static > + DeletionAwareTrie deletion(ByteComparable prefixInDataTrie, ByteComparable left, ByteComparable right, ByteComparable.Version byteComparableVersion, D deletion) + { + RangeTrie rangeTrie = RangeTrie.range(left, right, byteComparableVersion, deletion); + return dir -> new SingletonCursor.DeletionBranch<>(dir, + prefixInDataTrie.asComparableBytes(byteComparableVersion), byteComparableVersion, + rangeTrie); + } + + /// Creates a deletion-aware trie from an existing range trie representing deletions. + /// + /// This method allows for more complex deletion patterns by accepting a pre-constructed [RangeTrie] + /// that may contain multiple ranges, boundaries, and complex deletion states. This is useful for + /// advanced scenarios where simple range deletions are insufficient. + /// + /// @param prefixInDataTrie The position in the data trie where this deletion branch is to be rooted + /// @param byteComparableVersion The version to use for byte-comparable serialization + /// @param rangeTrie A pre-constructed range trie representing the deletion pattern + /// @return A deletion-aware trie containing the deletion branch + static > + DeletionAwareTrie deletionBranch(ByteComparable prefixInDataTrie, ByteComparable.Version byteComparableVersion, RangeTrie rangeTrie) + { + return dir -> new SingletonCursor.DeletionBranch<>(dir, + prefixInDataTrie.asComparableBytes(byteComparableVersion), byteComparableVersion, + rangeTrie); + } + + /// @inheritDoc + /// The returned deletion branches will be restricted to the bounds of the set; i.e. any ranges extending beyond + /// boundaries of the set will be cut to the confines of the set. + @Override + default DeletionAwareTrie intersect(TrieSet set) + { + return dir -> new IntersectionCursor.DeletionAware<>(cursor(dir), set.cursor(dir)); + } + + /// Specialized merge resolver for deletion-aware trie operations. + /// + /// This interface extends the basic [Trie.MergeResolver] to handle the additional complexity of + /// deletion-aware merging, including deletion marker resolution and deletion application logic. + /// + /// During merge operations, this resolver handles three types of conflicts: + /// - **Live Data Conflicts**: Resolved using inherited [#resolve] method + /// - **Deletion Marker Conflicts**: Resolved using [#resolveMarkers] method + /// - **Deletion Application**: Applied using [#applyMarker] method + /// + /// Additionally, this also provides the [#deletionsAtFixedPoints] flag, which significantly improves merge + /// performance when the user can guarantee that deletion branches are only introduced at predefined positions. + interface MergeResolver> extends Trie.MergeResolver + { + /// Resolves conflicts between deletion markers from different sources. + /// + /// It is expected that this method will return the overriding deletion marker (e.g. the one with the higher + /// timestamp), or some combination of information from the two markers. + /// + /// @param left Deletion marker from the left source (order not guaranteed) + /// @param right Deletion marker from the right source (order not guaranteed) + /// @return The resolved deletion marker, or null if deletions cancel out + D resolveMarkers(D left, D right); + + /// Applies a deletion marker to live content, potentially removing or modifying it. + /// + /// This method defines how deletions affect live data during merge operations. The + /// implementation determines whether the content should be deleted, partially modified, + /// or left unchanged based on the deletion marker's properties. + /// + /// @param marker The deletion marker to apply + /// @param content The live content that may be affected by the deletion + /// @return The content after deletion application, or null if completely deleted + T applyMarker(D marker, T content); + + /// Indicates whether deletions occur at predetermined points in the trie structure. + /// + /// This is a critical performance optimization. When true, guarantees that if one merge source + /// has a deletion branch at some position, the other source cannot have deletion branches + /// below or above that position. This allows us to skip walking the data trie to look for + /// lower-level deletion branches when merging. If the flag is false, we cannot know where + /// in the covered branch we may have a deletion, thus to be sure to find all we _must_ + /// walk the whole data subtrie. This can be terribly expensive. + boolean deletionsAtFixedPoints(); + } + + /// Constructs a view of the merge of this deletion-aware trie with another, applying deletions during the merge + /// process. The view is live, i.e. any write to any of the sources will be reflected in the merged view. + /// + /// This merge applies each source's deletions to the other source's live data, and merges deletion branches + /// to form a valid deletion-aware trie. + /// + /// The resolvers will only be called if both sources contains data for a given position, with arguments presented + /// in arbitrary order. + /// + /// @param other The other deletion-aware trie to merge with. + /// @param mergeResolver Resolver for live data conflicts between the two tries. + /// @param deletionResolver Resolver for deletion marker conflicts. See [MergeResolver#resolveMarkers]. + /// @param deleter Function to apply deletion markers to live content. See [MergeResolver#applyMarker]. + /// @param deletionsAtFixedPoints True if deletion branches are at predetermined positions. See [MergeResolver#deletionsAtFixedPoints]. + /// @return A live view of the merged tries with deletions applied + default DeletionAwareTrie mergeWith(DeletionAwareTrie other, + Trie.MergeResolver mergeResolver, + Trie.MergeResolver deletionResolver, + BiFunction deleter, + boolean deletionsAtFixedPoints) + { + return dir -> new MergeCursor.DeletionAware<>(mergeResolver, + deletionResolver, + deleter, + cursor(dir), + other.cursor(dir), + deletionsAtFixedPoints); + } + + /// Constructs a view of the merge of this deletion-aware trie with another, applying deletions during the merge + /// process. The view is live, i.e. any write to any of the sources will be reflected in the merged view. + /// + /// This merge applies each source's deletions to the other source's live data, and merges deletion branches + /// to form a valid deletion-aware trie. + /// + /// The resolvers will only be called if both sources contains data for a given position, with arguments presented + /// in arbitrary order. + /// + /// @param other The other deletion-aware trie to merge with + /// @param mergeResolver Unified [MergeResolver] providing the merge logic + /// @return A live view of the merged tries with deletions applied + default DeletionAwareTrie mergeWith(DeletionAwareTrie other, MergeResolver mergeResolver) + { + return mergeWith(other, mergeResolver, mergeResolver::resolveMarkers, mergeResolver::applyMarker, mergeResolver.deletionsAtFixedPoints()); + } + + /// See [MergeResolver] + interface CollectionMergeResolver> + extends MergeResolver, Trie.CollectionMergeResolver + { + /// Resolves conflicts between deletion markers from different sources. + /// + /// It is expected that this method will return the overriding deletion marker (e.g. the one with the higher + /// timestamp), or some combination of information from the two markers. + /// + /// @param markers A collection of all the markers that apply to a position + /// @return The resolved deletion marker, or null if deletions cancel out + D resolveMarkers(Collection markers); + + @Override + default D resolveMarkers(D c1, D c2) + { + return resolveMarkers(ImmutableList.of(c1, c2)); + } + } + + /// Constructs a view of the merge of multiple deletion-aware tries, applying deletions during the merge + /// process. The view is live, i.e. any write to any of the sources will be reflected in the merged view. + /// + /// This merge applies each source's deletions to the other sources' live data, and merges deletion branches + /// to form a valid deletion-aware trie. + /// + /// The resolvers will only be called if more than one source contains data for a given position, with arguments + /// presented in arbitrary order. + /// + /// @param sources Collection of deletion-aware tries to merge (must not be empty) + /// @param mergeResolver Unified [CollectionMergeResolver] providing the merge logic + /// @return A live view of the merged tries with deletions applied + /// @throws AssertionError if sources collection is empty. + static > + DeletionAwareTrie merge(Collection> sources, + CollectionMergeResolver mergeResolver) + { + return merge(sources, + mergeResolver, + mergeResolver::resolveMarkers, + mergeResolver::applyMarker, + mergeResolver.deletionsAtFixedPoints()); + } + + + /// Constructs a view of the merge of multiple deletion-aware tries, applying deletions during the merge + /// process. The view is live, i.e. any write to any of the sources will be reflected in the merged view. + /// + /// This merge applies each source's deletions to the other sources' live data, and merges deletion branches + /// to form a valid deletion-aware trie. + /// + /// The resolvers will only be called if more than one source contains data for a given position, with arguments + /// presented in arbitrary order. + /// + /// @param sources Collection of deletion-aware tries to merge (must not be empty). + /// @param mergeResolver Resolver for live data conflicts across all sources. + /// @param deletionResolver Resolver for deletion marker conflicts across all sources. See [CollectionMergeResolver#resolveMarkers]. + /// @param deleter Function to apply deletion markers to live content. See [MergeResolver#applyMarker]. + /// @param deletionsAtFixedPoints Optimization flag for predictable deletion patterns. See [MergeResolver#deletionsAtFixedPoints]. + /// @return A live view of the merged tries with deletions applied. + /// @throws AssertionError if sources collection is empty. + static > + DeletionAwareTrie merge(Collection> sources, + Trie.CollectionMergeResolver mergeResolver, + Trie.CollectionMergeResolver deletionResolver, + BiFunction deleter, + boolean deletionsAtFixedPoints) + { + switch (sources.size()) + { + case 0: + throw new AssertionError("Cannot merge empty collection of tries"); + case 1: + return sources.iterator().next(); + case 2: + { + Iterator> it = sources.iterator(); + DeletionAwareTrie t1 = it.next(); + DeletionAwareTrie t2 = it.next(); + return t1.mergeWith(t2, mergeResolver, deletionResolver, deleter, deletionsAtFixedPoints); + } + default: + return dir -> new CollectionMergeCursor.DeletionAware<>(mergeResolver, + deletionResolver, + deleter, + deletionsAtFixedPoints, + dir, + sources, + DeletionAwareTrie::cursor); + } + } + + /// Walker interface extended to also process deletion branches. + interface DeletionAwareWalker extends Cursor.Walker + { + /// Called when a deletion branch is found. Return null to skip over it, or the walker to use to descend inside + /// it. + /// + /// Note that the depth given by `resetPathLength` in the deletion branch will be relative to the root of the + /// deletion branch. See [TrieDumper] for an example of handling this. + boolean enterDeletionsBranch(); + + /// Called for every deletion marker found in the deletion branch. + void deletionMarker(D marker); + + /// Called when the deletion branch is exited. + void exitDeletionsBranch(); + } + + @Override + default String dump(Function contentToString) + { + return dump(contentToString, Object::toString); + } + + default String dump(Function contentToString, + Function rangeToString) + { + return process(Direction.FORWARD, new TrieDumper.DeletionAware<>(contentToString, rangeToString)); + } + + /// Process the trie using the given [DeletionAwareWalker]. + default R process(Direction direction, DeletionAwareWalker walker) + { + return cursor(direction).process(walker); + } + + /// Returns a view of the live content in this trie as a regular [Trie]. + default Trie contentOnlyTrie() + { + return this::cursor; + } + + /// Returns a view of all deletion ranges in this trie as a single [RangeTrie]. + default RangeTrie deletionOnlyTrie() + { + // Note: We must walk the main trie to find deletion branch roots. This can be inefficient. + return dir -> new DeletionAwareCursor.DeletionsTrieCursor<>(cursor(dir)); + } + + /// Returns a view of the combination of the live data and deletions in this trie as a regular [Trie], using + /// the provided mapping function to covert values to a common type. + default Trie mergedTrie(BiFunction resolver) + { + return dir -> new DeletionAwareCursor.LiveAndDeletionsMergeCursor<>(resolver, cursor(dir)); + } + + static > + DeletionAwareTrie empty(ByteComparable.Version byteComparableVersion) + { + return direction -> new DeletionAwareCursor.Empty<>(direction, byteComparableVersion); + } + + @Override + default DeletionAwareTrie prefixedBy(ByteComparable prefix) + { + return dir -> new PrefixedCursor.DeletionAware<>(prefix, cursor(dir)); + } + + /// @inheritDoc + /// + /// Note: if the cursor is positioned below a deletion branch root, the tail will not include any information about + /// that deletion branch, even if it applies to the current position. + @Override + default DeletionAwareTrie tailTrie(ByteComparable prefix) + { + DeletionAwareCursor c = cursor(Direction.FORWARD); + if (c.descendAlong(prefix.asComparableBytes(c.byteComparableVersion()))) + return c::tailCursor; + else + return null; + } + + /// Returns an entry set containing all tail tree constructed at the points that contain content of + /// the given type. + default Iterable>> tailTries(Direction direction, Class clazz) + { + return () -> new TrieTailsIterator.AsEntriesDeletionAware<>(cursor(direction), clazz); + } + + DeletionAwareCursor makeCursor(Direction direction); + + @Override + default DeletionAwareCursor cursor(Direction direction) + { + return Trie.DEBUG ? new VerificationCursor.DeletionAware<>(makeCursor(direction)) + : makeCursor(direction); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/DepthAdjustedCursor.java b/src/java/org/apache/cassandra/db/tries/DepthAdjustedCursor.java new file mode 100644 index 000000000000..0c1803bf32b6 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/DepthAdjustedCursor.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +class DepthAdjustedCursor> implements Cursor +{ + final C source; + final int depthAdjustment; + final int incomingTransitionAtRoot; + + DepthAdjustedCursor(C source, int depthAdjustment, int incomingTransitionAtRoot) + { + this.source = source; + this.depthAdjustment = depthAdjustment; + this.incomingTransitionAtRoot = incomingTransitionAtRoot; + } + + int toAdjustedDepth(int depth) + { + return depth < 0 ? depth : depth + depthAdjustment; + } + + int fromAdjustedDepth(int depth) + { + return depth < 0 ? depth : depth - depthAdjustment; + } + + @Override + public int depth() + { + return toAdjustedDepth(source.depth()); + } + + @Override + public int incomingTransition() + { + return source.depth() == 0 ? incomingTransitionAtRoot : source.incomingTransition(); + } + + @Override + public T content() + { + return source.content(); + } + + @Override + public Direction direction() + { + return source.direction(); + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return source.byteComparableVersion(); + } + + @Override + public int advance() + { + return toAdjustedDepth(source.advance()); + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + return toAdjustedDepth(source.advanceMultiple(receiver)); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + return toAdjustedDepth(source.skipTo(fromAdjustedDepth(skipDepth), skipTransition)); + } + + @Override + public int skipToWhenAhead(int skipDepth, int skipTransition) + { + int requestedDepth = fromAdjustedDepth(skipDepth); + if (requestedDepth == 0) + return direction().lt(skipTransition, incomingTransitionAtRoot) ? depthAdjustment : toAdjustedDepth(source.skipTo(requestedDepth, skipTransition)); + else + return toAdjustedDepth(source.skipToWhenAhead(fromAdjustedDepth(skipDepth), skipTransition)); + } + + @Override + public boolean descendAlong(ByteSource bytes) + { + return source.descendAlong(bytes); + } + + @Override + public Cursor tailCursor(Direction direction) + { + return source.tailCursor(direction); + } + + @Override + public R process(Walker walker) + { + throw new AssertionError("Depth-adjusted cursors cannot be walked directly."); + } + + @Override + public R processSkippingBranches(Walker walker) + { + throw new AssertionError("Depth-adjusted cursors cannot be walked directly."); + } + + static Cursor make(Cursor source, int depthAdjustment, int incomingTransitionAtRoot) + { + return depthAdjustment == 0 ? source : new Plain<>(source, depthAdjustment, incomingTransitionAtRoot); + } + + static > RangeCursor make(RangeCursor source, int depthAdjustment, int incomingTransitionAtRoot) + { + return depthAdjustment == 0 ? source : new Range<>(source, depthAdjustment, incomingTransitionAtRoot); + } + + static class Plain extends DepthAdjustedCursor> + { + public Plain(Cursor source, int depthAdjustment, int incomingTransitionAtRoot) + { + super(source, depthAdjustment, incomingTransitionAtRoot); + } + } + + static class Range> extends DepthAdjustedCursor> implements RangeCursor + { + Range(RangeCursor source, int depthAdjustment, int incomingTransitionAtRoot) + { + super(source, depthAdjustment, incomingTransitionAtRoot); + } + + @Override + public S state() + { + return source.state(); + } + + @Override + public S precedingState() + { + return source.precedingState(); + } + + @Override + public RangeCursor precedingStateCursor(Direction direction) + { + return source.precedingStateCursor(direction); + } + + @Override + public RangeCursor tailCursor(Direction direction) + { + return source.tailCursor(direction); + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/FlexibleMergeCursor.java b/src/java/org/apache/cassandra/db/tries/FlexibleMergeCursor.java new file mode 100644 index 000000000000..0642e8a700ed --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/FlexibleMergeCursor.java @@ -0,0 +1,272 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.function.BiFunction; +import javax.annotation.Nullable; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +abstract class FlexibleMergeCursor, D extends Cursor, T> implements Cursor +{ + final Direction direction; + final C c1; + @Nullable D c2; + int c2depthCorrection; + int incomingTransition; + int depth; + + enum State + { + AT_C1, + AT_C2, + AT_BOTH, + C1_ONLY, // c2 is null + } + State state; + + FlexibleMergeCursor(C c1) + { + assert c1.depth() == 0; + this.direction = c1.direction(); + this.c1 = c1; + this.c2 = null; + state = State.C1_ONLY; + incomingTransition = c1.incomingTransition(); + depth = c1.depth(); + // We can't call postAdvance here because the class may not be completely initialized. + // The concrete class should do that instead + } + + FlexibleMergeCursor(C c1, D c2) + { + assert c1.depth() == 0; + assert c2.depth() == 0; + this.direction = c1.direction(); + this.c1 = c1; + this.c2 = c2; + this.c2depthCorrection = 0; + state = c2 != null ? State.AT_BOTH : State.C1_ONLY; + incomingTransition = c1.incomingTransition(); + depth = c1.depth(); + // We can't call postAdvance here because the class may not be completely initialized. + // The concrete class should do that instead + } + + public void addCursor(D c2) + { + assert state == State.C1_ONLY : "Attempting to add further cursors to a cursor that already has two sources"; + assert c2.depth() == 0 : "Only cursors rooted at the current position can be added"; + this.c2 = c2; + this.c2depthCorrection = c1.depth(); + this.state = State.AT_BOTH; + } + + abstract int postAdvance(int depth); + + @Override + public int advance() + { + switch (state) + { + case C1_ONLY: + return inC1Only(c1.advance()); + case AT_C1: + return checkOrder(c1.advance(), c2.depth()); + case AT_C2: + return checkOrder(c1.depth(), c2.advance()); + case AT_BOTH: + return checkOrder(c1.advance(), c2.advance()); + default: + throw new AssertionError(); + } + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + if (state == State.C1_ONLY) + return inC1Only(c1.skipTo(skipDepth, skipTransition)); + + // Handle request to exit c2 branch separately for simplicity + if (skipDepth <= c2depthCorrection) + { + switch (state) + { + case AT_C1: + case AT_BOTH: + return leaveC2(c1.skipTo(skipDepth, skipTransition)); + case AT_C2: + return leaveC2(c1.skipToWhenAhead(skipDepth, skipTransition)); + default: + throw new AssertionError(); + } + } + + int c2skipDepth = skipDepth - c2depthCorrection; + switch (state) + { + case AT_C1: + return checkOrder(c1.skipTo(skipDepth, skipTransition), c2.skipToWhenAhead(c2skipDepth, skipTransition)); + case AT_C2: + return checkOrder(c1.skipToWhenAhead(skipDepth, skipTransition), c2.skipTo(c2skipDepth, skipTransition)); + case AT_BOTH: + return checkOrder(c1.skipTo(skipDepth, skipTransition), c2.skipTo(c2skipDepth, skipTransition)); + default: + throw new AssertionError(); + } + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + switch (state) + { + case C1_ONLY: + return inC1Only(c1.advanceMultiple(receiver)); + // If we are in a branch that's only covered by one of the sources, we can use its advanceMultiple as it is + // only different from advance if it takes multiple steps down, which does not change the order of the + // cursors. + // Since it might ascend, we still have to check the order after the call. + case AT_C1: + return checkOrder(c1.advanceMultiple(receiver), c2.depth()); + case AT_C2: + return checkOrder(c1.depth(), c2.advanceMultiple(receiver)); + // While we are on a shared position, we must descend one byte at a time to maintain the cursor ordering. + case AT_BOTH: + return checkOrder(c1.advance(), c2.advance()); + default: + throw new AssertionError(); + } + } + + private int inC1Only(int c1depth) + { + incomingTransition = c1.incomingTransition(); + depth = c1depth; + return postAdvance(c1depth); + } + + private int checkOrder(int c1depth, int c2depthUncorrected) + { + if (c2depthUncorrected < 0) + return leaveC2(c1depth); + + int c2depth = c2depthUncorrected + c2depthCorrection; + if (c1depth > c2depth) + { + state = State.AT_C1; + incomingTransition = c1.incomingTransition(); + depth = c1depth; + return postAdvance(c1depth); + } + if (c1depth < c2depth) + { + state = State.AT_C2; + incomingTransition = c2.incomingTransition(); + depth = c2depth; + return postAdvance(c2depth); + } + // c1depth == c2depth + int c1trans = c1.incomingTransition(); + int c2trans = c2.incomingTransition(); + boolean c1ahead = direction.gt(c1trans, c2trans); + boolean c2ahead = direction.lt(c1trans, c2trans); + state = c2ahead ? State.AT_C1 : c1ahead ? State.AT_C2 : State.AT_BOTH; + incomingTransition = c1ahead ? c2trans : c1trans; + depth = c1depth; + return postAdvance(c1depth); + } + + private int leaveC2(int c1depth) + { + state = State.C1_ONLY; + c2 = null; + incomingTransition = c1.incomingTransition(); + depth = c1depth; + return postAdvance(c1depth); + } + + @Override + public int depth() + { + return depth; + } + + @Override + public int incomingTransition() + { + return incomingTransition; + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return c1.byteComparableVersion(); + } + + static abstract class WithMappedContent, D extends Cursor, Z> extends FlexibleMergeCursor + { + final BiFunction resolver; + + WithMappedContent(BiFunction resolver, C c1) + { + super(c1); + this.resolver = resolver; + } + + WithMappedContent(BiFunction resolver, C c1, D c2) + { + super(c1, c2); + this.resolver = resolver; + } + + @Override + public Z content() + { + U mc = null; + T nc = null; + switch (state) + { + case C1_ONLY: + case AT_C1: + nc = c1.content(); + break; + case AT_C2: + mc = c2.content(); + break; + case AT_BOTH: + mc = c2.content(); + nc = c1.content(); + break; + default: + throw new AssertionError(); + } + if (nc == null && mc == null) + return null; + return resolver.apply(nc, mc); + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java index 36084b1cb3fc..1fd261b8ae6a 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryBaseTrie.java @@ -840,9 +840,15 @@ private int createEmptySplitNode() throws TrieSpaceExhaustedException return allocateCell() + SPLIT_OFFSET; } - private int createPrefixNode(int contentId, int child, boolean isSafeChain) throws TrieSpaceExhaustedException + private int createContentNode(int contentId, int child, boolean isSafeChain) throws TrieSpaceExhaustedException { - assert !isNullOrLeaf(child) : "Prefix node cannot reference a childless node."; + return createPrefixNode(contentId, NONE, child, isSafeChain); + } + + private int createPrefixNode(int contentId, int alternateBranch, int child, boolean isSafeChain) throws TrieSpaceExhaustedException + { + assert !isLeaf(child) : "Prefix node cannot reference a leaf node."; + assert !isNull(child) || !isNull(alternateBranch) : "Prefix node can only have a null child if it includes an alternate branch."; int offset = offset(child); int node; @@ -864,6 +870,7 @@ private int createPrefixNode(int contentId, int child, boolean isSafeChain) thro } putInt(node + PREFIX_CONTENT_OFFSET, contentId); + putInt(node + PREFIX_ALTERNATE_OFFSET, alternateBranch); return node; } @@ -892,7 +899,8 @@ private int updatePrefixNodeChild(int node, int child, boolean forcedCopy) throw { // No need to recycle this cell because that is already done by the modification of the child int contentId = getIntVolatile(node + PREFIX_CONTENT_OFFSET); - return createPrefixNode(contentId, child, true); + int alternateBranch = getIntVolatile(node + PREFIX_ALTERNATE_OFFSET); + return createPrefixNode(contentId, alternateBranch, child, true); } } @@ -915,9 +923,9 @@ private boolean isEmbeddedPrefixNode(int node) /// @param forcedCopy whether or not we need to preserve all pre-existing data for concurrent readers /// @return a node which has the children of updatedPostContentNode combined with the content of /// `existingPreContentNode` - private int preserveContent(int existingPreContentNode, - int existingPostContentNode, - int updatedPostContentNode, + private int preservePrefix(int existingPreContentNode, + int existingPostContentNode, + int updatedPostContentNode, boolean forcedCopy) throws TrieSpaceExhaustedException { @@ -933,11 +941,11 @@ private int preserveContent(int existingPreContentNode, // else we have existing prefix node, and we need to reference a new child if (isLeaf(existingPreContentNode)) { - return createPrefixNode(existingPreContentNode, updatedPostContentNode, true); + return createContentNode(existingPreContentNode, updatedPostContentNode, true); } assert offset(existingPreContentNode) == PREFIX_OFFSET : "Unexpected content in non-prefix and non-leaf node."; - if (updatedPostContentNode != NONE) + if (updatedPostContentNode != NONE || getIntVolatile(existingPreContentNode + PREFIX_ALTERNATE_OFFSET) != NONE) return updatePrefixNodeChild(existingPreContentNode, updatedPostContentNode, forcedCopy); else { @@ -960,7 +968,6 @@ class ApplyState implements KeyProducer { int[] data = new int[16 * 5]; int currentDepth = -1; - int ascendLimit = -1; /// Pointer to the existing node before skipping over content nodes, i.e. this is either the same as /// existingPostContentNode or a pointer to a prefix or leaf node whose child is `existingPostContentNode`. @@ -1006,6 +1013,10 @@ void setUpdatedPostContentNode(int value) { data[currentDepth * 5 + 2] = value; } + int updatedPostContentNodeAtDepth(int stackDepth) + { + return data[stackDepth * 5 + 2]; + } /// The transition we took on the way down. int transition() @@ -1020,6 +1031,10 @@ int transitionAtDepth(int stackDepth) { return data[stackDepth * 5 + 3]; } + int incomingTransition() + { + return transitionAtDepth(currentDepth - 1); + } /// The compiled content id. Needed because we can only access a cursor's content on the way down but we can't /// attach it until we ascend from the node. @@ -1036,19 +1051,17 @@ int contentIdAtDepth(int stackDepth) return data[stackDepth * 5 + 4]; } - int setAscendLimit(int newLimit) + int alternateBranchToAttach = NONE; + + ApplyState start() { - int prev = ascendLimit; - ascendLimit = newLimit; - return prev; + return start(root); } - - ApplyState start() + ApplyState start(int root) { int existingFullNode = root; currentDepth = -1; - ascendLimit = 0; descendInto(existingFullNode); return this; @@ -1057,12 +1070,17 @@ ApplyState start() /// Advance to the given depth and transition. Returns false if the depth signals mutation cursor is exhausted. boolean advanceTo(int depth, int transition, int forcedCopyDepth) throws TrieSpaceExhaustedException { - while (currentDepth > Math.max(ascendLimit, depth - 1)) + return advanceTo(depth, transition, forcedCopyDepth, 0); + } + /// Advance to the given depth and transition. Returns false if the depth signals mutation cursor is exhausted. + boolean advanceTo(int depth, int transition, int forcedCopyDepth, int ascendLimit) throws TrieSpaceExhaustedException + { + while (currentDepth >= Math.max(ascendLimit + 1, depth)) { // There are no more children. Ascend to the parent state to continue walk. attachAndMoveToParentState(forcedCopyDepth); } - if (depth == -1) + if (depth <= ascendLimit) return false; // We have a transition, get child to descend into @@ -1078,12 +1096,11 @@ boolean advanceTo(int depth, int transition, int forcedCopyDepth) throws TrieSpa /// non-exhausted position. boolean advanceToNextExistingOr(int limitDepth, int limitTransition, int forcedCopyDepth) throws TrieSpaceExhaustedException { - assert limitDepth > applyState.ascendLimit; - setTransition(-1); // we have newly descended to a node, start with its first child + assert limitDepth > 0; while (true) { int currentTransition = transition(); - int nextTransition = getNextTransition(existingPostContentNode(), currentTransition + 1); + int nextTransition = getNextTransition(updatedPostContentNode(), currentTransition + 1); if (currentDepth + 1 == limitDepth && nextTransition >= limitTransition) { descend(limitTransition); @@ -1102,11 +1119,15 @@ boolean advanceToNextExistingOr(int limitDepth, int limitTransition, int forcedC /// Advance to the next existing position in the trie. boolean advanceToNextExisting(int forcedCopyDepth) throws TrieSpaceExhaustedException { - setTransition(-1); // we have newly descended to a node, start with its first child + return advanceToNextExisting(forcedCopyDepth, 0); + } + /// Advance to the next existing position in the trie. + boolean advanceToNextExisting(int forcedCopyDepth, int ascendLimit) throws TrieSpaceExhaustedException + { while (true) { int currentTransition = transition(); - int nextTransition = getNextTransition(existingPostContentNode(), currentTransition + 1); + int nextTransition = getNextTransition(updatedPostContentNode(), currentTransition + 1); if (nextTransition <= 0xFF) { descend(nextTransition); @@ -1124,7 +1145,7 @@ boolean advanceToNextExisting(int forcedCopyDepth) throws TrieSpaceExhaustedExce void descend(int transition) { setTransition(transition); - int existingFullNode = getChild(existingFullNode(), transition); + int existingFullNode = getChild(updatedPostContentNode(), transition); descendInto(existingFullNode); } @@ -1146,13 +1167,14 @@ private void descendInto(int existingFullNode) else if (offset(existingFullNode) == PREFIX_OFFSET) { existingContentId = getIntVolatile(existingFullNode + PREFIX_CONTENT_OFFSET); - existingPostContentNode = followContentTransition(existingFullNode); + existingPostContentNode = followPrefixTransition(existingFullNode); } else existingPostContentNode = existingFullNode; setExistingPostContentNode(existingPostContentNode); setUpdatedPostContentNode(existingPostContentNode); setContentId(existingContentId); + setTransition(-1); } T getContent() @@ -1200,7 +1222,7 @@ T getNearestContent() setTransition(-1); // In the node we have just descended to, start with its first child for (; stackPos >= 0 && node == NONE; --stackPos) { - node = getNextChild(existingPostContentNodeAtDepth(stackPos), transitionAtDepth(stackPos) + 1); + node = getNextChild(updatedPostContentNodeAtDepth(stackPos), transitionAtDepth(stackPos) + 1); } while (node != NONE) @@ -1213,6 +1235,11 @@ T getNearestContent() return null; } + public int alternateBranch() + { + return getAlternateBranch(existingFullNode()); + } + /// Attach a child to the current node. private void attachChild(int transition, int child, boolean forcedCopy) throws TrieSpaceExhaustedException { @@ -1234,6 +1261,13 @@ else if (forcedCopy) /// others. private int applyContent(boolean forcedCopy) throws TrieSpaceExhaustedException { + if (alternateBranchToAttach != NONE) + { + int alternateBranch = alternateBranchToAttach; + alternateBranchToAttach = NONE; + return applyContentWithAlternateBranch(alternateBranch, forcedCopy); + } + // Note: the old content id itself is already released by setContent. Here we must release any standalone // prefix nodes that may reference it. int contentId = contentId(); @@ -1253,26 +1287,51 @@ private int applyContent(boolean forcedCopy) throws TrieSpaceExhaustedException if (isLeaf(existingPreContentNode)) return contentId != NONE - ? createPrefixNode(contentId, updatedPostContentNode, true) + ? createContentNode(contentId, updatedPostContentNode, true) : updatedPostContentNode; return applyPrefixChange(updatedPostContentNode, existingPreContentNode, existingPostContentNode, contentId, + NONE, forcedCopy); } + /// Apply the collected content to a node. Converts `NONE` to a leaf node, and adds or updates a prefix for all + /// others. + private int applyContentWithAlternateBranch(int alternateBranch, boolean forcedCopy) throws TrieSpaceExhaustedException + { + int contentId = contentId(); + final int updatedPostContentNode = updatedPostContentNode(); + final int existingPreContentNode = existingFullNode(); + final int existingPostContentNode = existingPostContentNode(); + + // applyPrefixChange does not understand leaf nodes, handle upgrade from one explicitly. + if (isLeaf(existingPreContentNode)) + return contentId != NONE + ? createPrefixNode(contentId, alternateBranch, updatedPostContentNode, true) + : updatedPostContentNode; + + return applyPrefixChange(updatedPostContentNode, + existingPreContentNode, + existingPostContentNode, + contentId, + alternateBranch, + forcedCopy); + } + private int applyPrefixChange(int updatedPostPrefixNode, int existingPrePrefixNode, int existingPostPrefixNode, - int prefixData, + int contentId, + int alternateBranch, boolean forcedCopy) throws TrieSpaceExhaustedException { boolean prefixWasPresent = existingPrePrefixNode != existingPostPrefixNode; boolean prefixWasEmbedded = prefixWasPresent && isEmbeddedPrefixNode(existingPrePrefixNode); - if (prefixData == NONE) + if (contentId == NONE && alternateBranch == NONE) { if (prefixWasPresent && !prefixWasEmbedded) recycleCell(existingPrePrefixNode); @@ -1280,7 +1339,8 @@ private int applyPrefixChange(int updatedPostPrefixNode, } boolean childChanged = updatedPostPrefixNode != existingPostPrefixNode; - boolean dataChanged = !prefixWasPresent || prefixData != getIntVolatile(existingPrePrefixNode + PREFIX_CONTENT_OFFSET); + boolean dataChanged = !prefixWasPresent || contentId != getIntVolatile(existingPrePrefixNode + PREFIX_CONTENT_OFFSET) + || alternateBranch != getIntVolatile(existingPrePrefixNode + PREFIX_ALTERNATE_OFFSET); if (!childChanged && !dataChanged) return existingPrePrefixNode; @@ -1299,19 +1359,22 @@ else if (prefixWasPresent && !prefixWasEmbedded) recycleCell(existingPrePrefixNode); // otherwise cell is already recycled by the recycling of the child } - return createPrefixNode(prefixData, updatedPostPrefixNode, isNull(existingPostPrefixNode)); + return createPrefixNode(contentId, alternateBranch, updatedPostPrefixNode, isNull(existingPostPrefixNode)); } // We can't update in-place if there was no preexisting prefix, or if the // prefix was embedded and the target node must change. if (!prefixWasPresent || prefixWasEmbedded && childChanged) - return createPrefixNode(prefixData, updatedPostPrefixNode, isNull(existingPostPrefixNode)); + return createPrefixNode(contentId, alternateBranch, updatedPostPrefixNode, isNull(existingPostPrefixNode)); // Otherwise modify in place if (childChanged) // to use volatile write but also ensure we don't corrupt embedded nodes putIntVolatile(existingPrePrefixNode + PREFIX_POINTER_OFFSET, updatedPostPrefixNode); if (dataChanged) - putIntVolatile(existingPrePrefixNode + PREFIX_CONTENT_OFFSET, prefixData); + { + putIntVolatile(existingPrePrefixNode + PREFIX_CONTENT_OFFSET, contentId); + putIntVolatile(existingPrePrefixNode + PREFIX_ALTERNATE_OFFSET, alternateBranch); + } return existingPrePrefixNode; } @@ -1320,7 +1383,10 @@ else if (prefixWasPresent && !prefixWasEmbedded) /// one already exists). void attachAndMoveToParentState(int forcedCopyDepth) throws TrieSpaceExhaustedException { - int updatedFullNode = applyContent(currentDepth >= forcedCopyDepth); + attachBranchAndMoveToParentState(applyContent(currentDepth >= forcedCopyDepth), forcedCopyDepth); + } + + void attachBranchAndMoveToParentState(int updatedFullNode, int forcedCopyDepth) throws TrieSpaceExhaustedException { int existingFullNode = existingFullNode(); --currentDepth; assert currentDepth >= 0; @@ -1329,11 +1395,21 @@ void attachAndMoveToParentState(int forcedCopyDepth) throws TrieSpaceExhaustedEx attachChild(transition(), updatedFullNode, currentDepth >= forcedCopyDepth); } + int completeBranch(int forcedCopyDepth) throws TrieSpaceExhaustedException + { + return applyContent(currentDepth >= forcedCopyDepth); + } + /// Ascend and update the root at the end of processing. - void attachRoot(int forcedCopyDepth) throws TrieSpaceExhaustedException + void attachAndUpdateRoot(int forcedCopyDepth) throws TrieSpaceExhaustedException + { + attachRoot(applyContent(0 >= forcedCopyDepth), forcedCopyDepth); + } + + void attachRoot(int updatedFullNode, int ignoredForcedCopyDepth) throws TrieSpaceExhaustedException { - int updatedFullNode = applyContent(0 >= forcedCopyDepth); int existingFullNode = existingFullNode(); + --currentDepth; assert root == existingFullNode : "Unexpected change to root. Concurrent trie modification?"; if (updatedFullNode != existingFullNode) { @@ -1343,6 +1419,11 @@ void attachRoot(int forcedCopyDepth) throws TrieSpaceExhaustedException } } + void prepareToWalkBranchAgain() throws TrieSpaceExhaustedException + { + setTransition(-1); + } + public byte[] getBytes() { int arrSize = currentDepth; @@ -1390,6 +1471,8 @@ public ByteComparable.Version byteComparableVersion() public String toString() { + if (data == null) + return "uninitialized"; StringBuilder sb = new StringBuilder(); sb.append('@'); for (int i = 0; i < currentDepth; ++i) @@ -1400,6 +1483,11 @@ public String toString() sb.append(" contentId=").append(contentId()); return sb.toString(); } + + public InMemoryBaseTrie trie() + { + return InMemoryBaseTrie.this; + } } @@ -1498,7 +1586,6 @@ static class Mutation> implements NodeFeatures InMemoryBaseTrie.ApplyState state) { assert mutationCursor.depth() == 0 : "Unexpected non-fresh cursor."; - assert state.currentDepth == 0 : "Unexpected change to applyState. Concurrent trie modification?"; this.transformer = transformer; this.needsForcedCopy = needsForcedCopy; this.mutationCursor = mutationCursor; @@ -1540,7 +1627,7 @@ void applyContent() throws TrieSpaceExhaustedException void complete() throws TrieSpaceExhaustedException { assert state.currentDepth == 0 : "Unexpected change to applyState. Concurrent trie modification?"; - state.attachRoot(forcedCopyDepth); + state.attachAndUpdateRoot(forcedCopyDepth); } @Override @@ -1599,12 +1686,12 @@ private int putRecursive(int node, ByteSource key, R value, final UpsertTran if (newChild == child) return node; - int skippedContent = followContentTransition(node); + int skippedContent = followPrefixTransition(node); int attachedChild = !isNull(skippedContent) ? attachChild(skippedContent, transition, newChild) // Single path, no copying required : expandOrCreateChainNode(transition, newChild); - return preserveContent(node, skippedContent, attachedChild, false); + return preservePrefix(node, skippedContent, attachedChild, false); } private int applyContent(int node, R value, UpsertTransformer transformer) throws TrieSpaceExhaustedException @@ -1631,15 +1718,26 @@ private int applyContent(int node, R value, UpsertTransformer transfor if (offset(node) == PREFIX_OFFSET) { int contentId = getIntVolatile(node + PREFIX_CONTENT_OFFSET); - T newContent = transformer.apply(getContent(contentId), value); + T newContent = transformer.apply(isNull(contentId) ? null : getContent(contentId), value); if (newContent != null) { - setContent(contentId, newContent); + if (!isNull(contentId)) + setContent(contentId, newContent); + else + putIntVolatile(node + PREFIX_CONTENT_OFFSET, addContent(newContent)); return node; } else { - releaseContent(contentId); + if (!isNull(contentId)) + releaseContent(contentId); + + if (!isNull(getIntVolatile(node + PREFIX_ALTERNATE_OFFSET))) + { + // keep the prefix node for the alternate path + putIntVolatile(node + PREFIX_CONTENT_OFFSET, NONE); + return node; + } int b = getUnsignedByte(node + PREFIX_FLAGS_OFFSET); if (b < CELL_SIZE) @@ -1660,7 +1758,7 @@ private int applyContent(int node, R value, UpsertTransformer transfor if (newContent == null) return node; else - return createPrefixNode(addContent(transformer.apply(null, value)), node, false); + return createContentNode(addContent(transformer.apply(null, value)), node, false); } void completeMutation() diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrie.java new file mode 100644 index 000000000000..92c3d3efbedb --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrie.java @@ -0,0 +1,459 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Predicate; + +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.utils.ObjectSizes; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.concurrent.OpOrder; + +/// In-memory implementation of deletion-aware tries with concurrent access support. +/// +/// This class provides a concrete implementation of [DeletionAwareTrie] that stores both live data +/// and deletion information in a unified in-memory structure. It extends [InMemoryBaseTrie] to +/// inherit the efficient cell-based memory management, concurrent access patterns, and performance +/// optimizations of the base trie implementation. +/// +/// This class stores deletion branches in the "alternate branch" field of prefix nodes. All the +/// machinery to support this is already provided by [InMemoryBaseTrie]. This class implements the +/// relevant cursor and mutation methods. +/// +/// See [InMemoryTrie] for information on the consistency model. +/// +/// @param The content type for live data stored in the trie +/// @param The deletion marker type, must extend [RangeState] for range operations +public class InMemoryDeletionAwareTrie> +extends InMemoryBaseTrie implements DeletionAwareTrie +{ + // constants for space calculations + private static final long EMPTY_SIZE_ON_HEAP; + private static final long EMPTY_SIZE_OFF_HEAP; + static + { + // Measuring the empty size of long-lived tries, because these are the ones for which we want to track size. + InMemoryBaseTrie empty = new InMemoryDeletionAwareTrie<>(ByteComparable.Version.OSS50, BufferType.ON_HEAP, ExpectedLifetime.LONG, null); + EMPTY_SIZE_ON_HEAP = ObjectSizes.measureDeep(empty); + empty = new InMemoryDeletionAwareTrie<>(ByteComparable.Version.OSS50, BufferType.OFF_HEAP, ExpectedLifetime.LONG, null); + EMPTY_SIZE_OFF_HEAP = ObjectSizes.measureDeep(empty); + } + + InMemoryDeletionAwareTrie(ByteComparable.Version byteComparableVersion, BufferType bufferType, ExpectedLifetime lifetime, OpOrder opOrder) + { + super(byteComparableVersion, bufferType, lifetime, opOrder); + } + + public static > + InMemoryDeletionAwareTrie shortLived(ByteComparable.Version byteComparableVersion) + { + return new InMemoryDeletionAwareTrie<>(byteComparableVersion, BufferType.ON_HEAP, ExpectedLifetime.SHORT, null); + } + + public static > + InMemoryDeletionAwareTrie shortLived(ByteComparable.Version byteComparableVersion, BufferType bufferType) + { + return new InMemoryDeletionAwareTrie<>(byteComparableVersion, bufferType, ExpectedLifetime.SHORT, null); + } + + public static > + InMemoryDeletionAwareTrie longLived(ByteComparable.Version byteComparableVersion, OpOrder opOrder) + { + return longLived(byteComparableVersion, BufferType.OFF_HEAP, opOrder); + } + + public static > + InMemoryDeletionAwareTrie longLived(ByteComparable.Version byteComparableVersion, BufferType bufferType, OpOrder opOrder) + { + return new InMemoryDeletionAwareTrie<>(byteComparableVersion, bufferType, ExpectedLifetime.LONG, opOrder); + } + + static class DeletionAwareInMemoryCursor> + extends InMemoryCursor implements DeletionAwareCursor + { + DeletionAwareInMemoryCursor(InMemoryReadTrie trie, Direction direction, int root, int depth, int incomingTransition) + { + super(trie, direction, root, depth, incomingTransition); + } + + @SuppressWarnings("unchecked") + @Override + public T content() + { + return content; + } + + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + int alternateBranch = trie.getAlternateBranch(currentFullNode); + return ((InMemoryDeletionAwareTrie) trie).makeRangeCursor(direction, alternateBranch); + } + + @Override + public DeletionAwareCursor tailCursor(Direction direction) + { + return new DeletionAwareInMemoryCursor<>(trie, direction, currentFullNode, 0, -1); + } + } + + @SuppressWarnings("rawtypes") + private RangeCursor makeRangeCursor(Direction direction, int alternateBranch) { + return isNull(alternateBranch) + ? null + : new InMemoryRangeTrie.InMemoryRangeCursor<>((InMemoryReadTrie) this, direction, alternateBranch, 0, -1); + } + + @Override + public DeletionAwareInMemoryCursor makeCursor(Direction direction) + { + return new DeletionAwareInMemoryCursor<>(this, direction, root, 0, -1); + } + + protected long emptySizeOnHeap() + { + return bufferType == BufferType.ON_HEAP ? EMPTY_SIZE_ON_HEAP : EMPTY_SIZE_OFF_HEAP; + } + + @SuppressWarnings("unchecked") + InMemoryTrie.ApplyState deletionState = (InMemoryTrie.ApplyState) new ApplyState(); + + static class Mutation, V, E extends RangeState> + extends InMemoryBaseTrie.Mutation> + { + final UpsertTransformerWithKeyProducer deletionTransformer; + final UpsertTransformerWithKeyProducer deleter; + final boolean deletionsAtFixedPoints; + final InMemoryTrie.ApplyState deletionState; + + Mutation(UpsertTransformerWithKeyProducer dataTransformer, + UpsertTransformerWithKeyProducer deletionTransformer, + UpsertTransformerWithKeyProducer existingDeleter, + BiFunction insertedDeleter, + Predicate> needsForcedCopy, + boolean deletionsAtFixedPoints, + DeletionAwareCursor mutationCursor, + InMemoryBaseTrie.ApplyState state, + InMemoryBaseTrie.ApplyState deletionState) + { + super(dataTransformer, needsForcedCopy, new DeletionAwareMergeSource<>(insertedDeleter, mutationCursor), state); + this.deletionTransformer = deletionTransformer; + this.deleter = existingDeleter; + this.deletionsAtFixedPoints = deletionsAtFixedPoints; + this.deletionState = deletionState; + } + + @Override + void apply() throws TrieSpaceExhaustedException + { + // TODO: Consider walking both data and deletion branches in parallel. + int depth = state.currentDepth; + while (true) + { + if (depth < forcedCopyDepth) + forcedCopyDepth = needsForcedCopy.test(this) ? depth : Integer.MAX_VALUE; + + // Content must be applied before descending into the branch to make sure we call the transformers + // in the right order. + applyContent(); + + int existingAlternateBranch = state.alternateBranch(); + RangeCursor incomingAlternateBranch = mutationCursor.deletionBranchCursor(Direction.FORWARD); + if (incomingAlternateBranch != null || existingAlternateBranch != NONE) + { + int updatedAlternateBranch = existingAlternateBranch; + RangeCursor ourDeletionBranch; + if (!deletionsAtFixedPoints && existingAlternateBranch == NONE && state.existingFullNode() != NONE) + { + // Move any covered deletion branches up to this depth so that we can correctly merge the + // incoming deletions. + updatedAlternateBranch = hoistOurDeletionBranches(); + } + ourDeletionBranch = ((InMemoryDeletionAwareTrie) state.trie()).makeRangeCursor(Direction.FORWARD, updatedAlternateBranch); + + if (!deletionsAtFixedPoints && incomingAlternateBranch == null) + { + // The incoming cursor has no deletions here, but it may have some below this point. + // Switch to deletion branch to transform them to be rooted here. + // (Note: this will cause a lot of processing of unproductive branches.) + incomingAlternateBranch = new DeletionAwareCursor.DeletionsTrieCursor<>(mutationCursor.tailCursor(Direction.FORWARD)); + } + + if (incomingAlternateBranch != null) + { + // Duplicate cursor as we need it for both deletion and data branches. + RangeCursor deletionBranch = incomingAlternateBranch.tailCursor(Direction.FORWARD); + + // Delete data that is covered by the new deletions. + applyDeletions(incomingAlternateBranch); + + // Merge the deletions into our deletion branch. + updatedAlternateBranch = mergeDeletionBranch(updatedAlternateBranch, deletionBranch); + } + + // Continue processing to also insert the incoming data at this branch. + // Note that this will also apply the incoming content to this node and advance the mutation cursor + // to the position after this branch. + applyDataUnderDeletion(ourDeletionBranch); + + // Ascend and apply alternate branch. + state.alternateBranchToAttach = updatedAlternateBranch; + if (state.currentDepth == 0) + break; // to be attached to root by complete() + state.attachAndMoveToParentState(forcedCopyDepth); + depth = mutationCursor.depth(); + } + else + depth = mutationCursor.advance(); + + if (!state.advanceTo(depth, mutationCursor.incomingTransition(), forcedCopyDepth)) + break; + assert state.currentDepth == depth : "Unexpected change to applyState. Concurrent trie modification?"; + } + } + + private void applyDataUnderDeletion(RangeCursor ourDeletionBranch) throws TrieSpaceExhaustedException + { + // Add our deletion to DeletionAwareMergeSource to apply them to incoming data. + if (ourDeletionBranch != null) + mutationCursor.addDeletions(ourDeletionBranch); + int initialDepth = state.currentDepth; + + // The first forcedCopyDepth and applyContent are already called. + int depth = mutationCursor.advance(); + + // Below is the same as the main loop in `apply`, slightly rearranged and ignoring deletion branches. + while (state.advanceTo(depth, mutationCursor.incomingTransition(), forcedCopyDepth, initialDepth)) + { + assert state.currentDepth == depth : "Unexpected change to applyState. Concurrent trie modification?"; + + if (depth < forcedCopyDepth) + forcedCopyDepth = needsForcedCopy.test(this) ? depth : Integer.MAX_VALUE; + + applyContent(); + depth = mutationCursor.advance(); + } + assert state.currentDepth == initialDepth; + } + + private void applyDeletions(RangeCursor incomingAlternateBranch) throws TrieSpaceExhaustedException + { + // Apply the deletion branch to our data. + InMemoryTrie.DeleteMutation> deleteMutation = new InMemoryTrie.DeleteMutation<>( + deleter, + (Predicate>) (Predicate) needsForcedCopy, + incomingAlternateBranch, + state, + forcedCopyDepth); + deleteMutation.apply(); + + // Make sure the data pass that follows walks the updated branch. + state.prepareToWalkBranchAgain(); + } + + private int mergeDeletionBranch(int existingAlternateBranch, RangeCursor deletionBranch) throws TrieSpaceExhaustedException + { + // If forced copying is in force, apply it to the deletion branch too. + int deletionForcedCopyDepth = forcedCopyDepth <= state.currentDepth ? 0 : Integer.MAX_VALUE; + + InMemoryRangeTrie.Mutation rangeMutation = new InMemoryRangeTrie.Mutation<>( + deletionTransformer, + (Predicate>) (Predicate) needsForcedCopy, + deletionBranch, + deletionState.start(existingAlternateBranch), + deletionForcedCopyDepth); + rangeMutation.apply(); + return deletionState.completeBranch(deletionForcedCopyDepth); + } + + private int hoistOurDeletionBranches() throws TrieSpaceExhaustedException + { + // Walk all of our data branch and build new branches corresponding to it. When we reach a deletion + // branch, link it. If a branch is walked without finding a deletion branch, the returned NONEs should + // propagate up. + // We need to walk both the deletion-aware/data trie, as well as the deletion branch being built, so that + // the existing deletion branch mappings can be removed. + deletionState.start(NONE); + int initialDepth = state.currentDepth; + + int depth = state.currentDepth; + while (true) + { + if (depth < forcedCopyDepth) + forcedCopyDepth = needsForcedCopy.test(this) ? depth : Integer.MAX_VALUE; + + int existingAlternateBranch = state.alternateBranch(); + if (existingAlternateBranch != NONE) + { + deletionState.attachBranchAndMoveToParentState(existingAlternateBranch, forcedCopyDepth); + // Drop the existing alternate branch from the main state and ascend. + // The normal applyContent() method uses alternate branch value of NONE. + state.attachAndMoveToParentState(forcedCopyDepth); + } + + if (!state.advanceToNextExisting(forcedCopyDepth, initialDepth)) + break; + depth = state.currentDepth; + deletionState.advanceTo(depth - initialDepth, state.incomingTransition(), forcedCopyDepth - initialDepth); + } + if (deletionState.currentDepth > 0) + deletionState.advanceTo(-1, -1, forcedCopyDepth - initialDepth); + + // Make sure the walks over the data branch that follow use the updated branch. + state.prepareToWalkBranchAgain(); + return deletionState.completeBranch(forcedCopyDepth - initialDepth); + } + } + + + /// Modify this trie to apply the mutation given in the form of a trie. Any content in the mutation will be resolved + /// with the given function before being placed in this trie (even if there's no pre-existing content in this trie). + /// All of the deletions in the given mutation trie will be applied, removing any content and trie paths that become + /// empty as a result of the deletions and releasing any of the trie cells that they occupied. The deletion branches + /// of the trie will be combined with the incoming deletions. + /// + /// @param mutation the mutation to be applied, given in the form of a trie. Note that its content can be of type + /// different than the element type for this memtable trie. + /// @param dataTransformer a function applied to the potentially pre-existing value for the given key, and the new + /// value. Applied even if there's no pre-existing value in the memtable trie. The transformer can return null + /// if the entry should not be added or preserved. + /// @param deletionTransformer a function applied to combine overlapping deletions into a consistent view. Called + /// even if there is no pre-existing deletion to convert the marker type. For code simplicity this transformer is + /// not provided with the path to the modified markers. The transformer can return null if deletions cancel out + /// or should not be preserved. + /// @param existingDeleter a function used to apply a deletion marker to potentially delete live data. This is + /// only called if there is both content and deletion at a given covered point. It should return null if the entry + /// is to be deleted. + /// @param insertedDeleter a function used to filter incoming entries that are covered by existing deletions + /// in this trie, called only if both an entry and a deletion apply to a given point. This function is not provided + /// with a path to the modified data. + /// @param deletionsAtFixedPoints True if deletion branches are at predetermined positions. + /// @see DeletionAwareTrie.MergeResolver#deletionsAtFixedPoints + public > + void apply(DeletionAwareTrie mutation, + final UpsertTransformerWithKeyProducer dataTransformer, + final UpsertTransformer deletionTransformer, + final UpsertTransformerWithKeyProducer existingDeleter, + final BiFunction insertedDeleter, + boolean deletionsAtFixedPoints, + Predicate> needsForcedCopy) + throws TrieSpaceExhaustedException + { + // TODO: track hasDeletions and do plain Trie merges if neither this nor mutation has deletions. + try + { + Mutation m = new Mutation<>(dataTransformer, + deletionTransformer, + existingDeleter, + insertedDeleter, + needsForcedCopy, + deletionsAtFixedPoints, + mutation.cursor(Direction.FORWARD), + applyState.start(), + deletionState); + m.apply(); + m.complete(); + completeMutation(); + } + catch (Throwable t) + { + abortMutation(); + throw t; + } + } + + /// Modify this trie to apply the mutation given in the form of a trie. Any content in the mutation will be resolved + /// with the given function before being placed in this trie (even if there's no pre-existing content in this trie). + /// All of the deletions in the given mutation trie will be applied, removing any content and trie paths that become + /// empty as a result of the deletions and releasing any of the trie cells that they occupied. The deletion branches + /// of the trie will be combined with the incoming deletions. + /// + /// @param mutation the mutation to be applied, given in the form of a trie. Note that its content can be of type + /// different than the element type for this memtable trie. + /// @param dataTransformer a function applied to the potentially pre-existing value for the given key, and the new + /// value. Applied even if there's no pre-existing value in the memtable trie. The transformer can return null + /// if the entry should not be added or preserved. + /// @param deletionTransformer a function applied to combine overlapping deletions into a consistent view. Called + /// even if there is no pre-existing deletion to convert the marker type. For code simplicity this transformer is + /// not provided with the path to the modified markers. The transformer can return null if deletions cancel out + /// or should not be preserved. + /// @param existingDeleter a function used to apply a deletion marker to potentially delete live data. This is + /// only called if there is both content and deletion at a given covered point. It should return null if the entry + /// is to be deleted. + /// @param insertedDeleter a function used to filter incoming entries that are covered by existing deletions + /// in this trie, called only if both an entry and a deletion apply to a given point. This function is not provided + /// with a path to the modified data. + /// @param deletionsAtFixedPoints True if deletion branches are at predetermined positions. + /// @see DeletionAwareTrie.MergeResolver#deletionsAtFixedPoints + public > + void apply(DeletionAwareTrie mutation, + final UpsertTransformer dataTransformer, + final UpsertTransformer deletionTransformer, + final UpsertTransformer existingDeleter, + final BiFunction insertedDeleter, + boolean deletionsAtFixedPoints, + Predicate> needsForcedCopy) + throws TrieSpaceExhaustedException + { + apply(mutation, + (UpsertTransformerWithKeyProducer) dataTransformer, + deletionTransformer, existingDeleter, insertedDeleter, deletionsAtFixedPoints, needsForcedCopy); + } + + class DumpCursor extends InMemoryReadTrie.DumpCursor> implements DeletionAwareCursor + { + DumpCursor(DeletionAwareInMemoryCursor source, Function contentToString) + { + super(source, contentToString); + } + + + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + return source.deletionBranchCursor(direction); + } + + @Override + public DumpCursor tailCursor(Direction direction) + { + throw new AssertionError(); + } + } + + public String dump(Function contentToString) + { + return dump(contentToString, Object::toString); + } + + /// Override of dump to provide more detailed printout that includes the type of each node in the trie. + /// We do this via a wrapping cursor that returns a content string for the type of node for every node we return. + public String dump(Function contentToString, Function rangeToString) + { + return new DumpCursor(makeCursor(Direction.FORWARD), contentToString).process(new TrieDumper.DeletionAware<>(Function.identity(), rangeToString)); + } + + private String dumpBranch(int branchRoot) + { + return new DumpCursor(new DeletionAwareInMemoryCursor<>(this, Direction.FORWARD, branchRoot, 0, -1), Object::toString) + .process(new TrieDumper.DeletionAware<>(Function.identity(), Object::toString)); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryRangeTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryRangeTrie.java index fa8e6a79e029..5bd28baa1939 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryRangeTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryRangeTrie.java @@ -64,9 +64,9 @@ public static > InMemoryRangeTrie longLived(ByteCompa return new InMemoryRangeTrie<>(byteComparableVersion, bufferType, ExpectedLifetime.LONG, opOrder); } - public InMemoryRangeCursor makeCursor(Direction direction) + public InMemoryRangeCursor makeCursor(Direction direction) { - return new InMemoryRangeCursor(direction, root, 0, -1); + return new InMemoryRangeCursor<>(this, direction, root, 0, -1); } protected long emptySizeOnHeap() @@ -74,15 +74,15 @@ protected long emptySizeOnHeap() return bufferType == BufferType.ON_HEAP ? EMPTY_SIZE_ON_HEAP : EMPTY_SIZE_OFF_HEAP; } - class InMemoryRangeCursor extends InMemoryCursor implements RangeCursor + static class InMemoryRangeCursor> extends InMemoryCursor implements RangeCursor { boolean activeIsSet; S activeRange; // only non-null if activeIsSet S prevContent; // can only be non-null if activeIsSet - InMemoryRangeCursor(Direction direction, int root, int depth, int incomingTransition) + InMemoryRangeCursor(InMemoryReadTrie trie, Direction direction, int root, int depth, int incomingTransition) { - super(direction, root, depth, incomingTransition); + super(trie, direction, root, depth, incomingTransition); activeIsSet = true; activeRange = null; prevContent = null; @@ -164,13 +164,13 @@ private S getNearestContent() { // Walk a copy of this cursor (non-range because we are only not doing anything smart with it) to find the // nearest child content in the direction of the cursor. - return new InMemoryCursor(direction, currentNode, 0, -1).advanceToContent(null); + return new InMemoryCursor<>(trie, direction, currentNode, 0, -1).advanceToContent(null); } @Override - public InMemoryRangeCursor tailCursor(Direction direction) + public InMemoryRangeCursor tailCursor(Direction direction) { - InMemoryRangeCursor cursor = new InMemoryRangeCursor(direction, currentFullNode, 0, -1); + InMemoryRangeCursor cursor = new InMemoryRangeCursor<>(trie, direction, currentFullNode, 0, -1); if (activeIsSet) { // Copy the state we have already compiled to the child cursor. @@ -184,11 +184,17 @@ public InMemoryRangeCursor tailCursor(Direction direction) } } - static class Mutation, U extends RangeState> extends InMemoryBaseTrie.Mutation> + static class Mutation, U extends RangeState> extends InMemoryBaseTrie.Mutation> { - Mutation(UpsertTransformerWithKeyProducer transformer, Predicate> needsForcedCopy, RangeCursor source, InMemoryRangeTrie.ApplyState state) + Mutation(UpsertTransformerWithKeyProducer transformer, Predicate> needsForcedCopy, RangeCursor source, InMemoryRangeTrie.ApplyState state) + { + this(transformer, needsForcedCopy, source, state, Integer.MAX_VALUE); + } + + Mutation(UpsertTransformerWithKeyProducer transformer, Predicate> needsForcedCopy, RangeCursor source, InMemoryRangeTrie.ApplyState state, int forcedCopyDepth) { super(transformer, needsForcedCopy, source, state); + this.forcedCopyDepth = forcedCopyDepth; } @Override @@ -198,9 +204,9 @@ void apply() throws TrieSpaceExhaustedException assert state.currentDepth == 0 : "Unexpected change to applyState. Concurrent trie modification?"; } - void applyContent(M existingState, U mutationState) throws TrieSpaceExhaustedException + void applyContent(S existingState, U mutationState) throws TrieSpaceExhaustedException { - M combined = transformer.apply(existingState, mutationState, state); + S combined = transformer.apply(existingState, mutationState, state); if (combined != null) combined = combined.isBoundary() ? combined : null; state.setContent(combined, // can be null @@ -216,7 +222,6 @@ void applyRanges() throws TrieSpaceExhaustedException // until we see another entry in mutation trie. // Repeat until mutation trie is exhausted. int depth = state.currentDepth; - int prevAscendDepth = state.setAscendLimit(depth); while (true) { if (depth < forcedCopyDepth) @@ -225,7 +230,7 @@ void applyRanges() throws TrieSpaceExhaustedException U content = mutationCursor.content(); if (content != null) { - final M existingCoveringState = getExistingCoveringState(); + final S existingCoveringState = getExistingCoveringState(); applyContent(existingCoveringState, content); U mutationCoveringState = content.precedingState(Direction.REVERSE); // Several cases: @@ -245,11 +250,10 @@ void applyRanges() throws TrieSpaceExhaustedException break; assert depth == state.currentDepth : "Unexpected change to applyState. Concurrent trie modification?"; } - state.setAscendLimit(prevAscendDepth); } - void applyDeletionRange(M existingCoveringState, - U mutationCoveringState) + void applyDeletionRange(S existingCoveringState, + U mutationCoveringState) throws TrieSpaceExhaustedException { boolean atMutation = true; @@ -269,7 +273,7 @@ void applyDeletionRange(M existingCoveringState, } atMutation = !state.advanceToNextExistingOr(depth, transition, forcedCopyDepth); - M existingContent = state.getContent(); + S existingContent = state.getContent(); U mutationContent = atMutation ? mutationCursor.content() : null; if (existingContent != null || mutationContent != null) { @@ -296,10 +300,10 @@ static > S rightSideAsCovering(S rangeState) return rangeState.precedingState(Direction.REVERSE); } - M getExistingCoveringState() + S getExistingCoveringState() { // If the current node has content, use it. - M existingCoveringState = state.getContent(); + S existingCoveringState = state.getContent(); if (existingCoveringState != null) return existingCoveringState; diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java index dd07fa684151..bfa648c8c259 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java @@ -166,9 +166,11 @@ Cell offsets used to identify node types (by comparing them to the node 'pointer static final int SPARSE_ORDER_OFFSET = SPARSE_CHILD_COUNT * 5 - SPARSE_OFFSET; // 0 // Offset of the flag byte in a prefix node. In shared cells, this contains the offset of the next node. - static final int PREFIX_FLAGS_OFFSET = 4 - PREFIX_OFFSET; + static final int PREFIX_FLAGS_OFFSET = 9 - PREFIX_OFFSET; // Offset of the content id static final int PREFIX_CONTENT_OFFSET = 0 - PREFIX_OFFSET; + // Offset of the alternate branch pointer + static final int PREFIX_ALTERNATE_OFFSET = 4 - PREFIX_OFFSET; // Offset of the next pointer in a non-shared prefix node static final int PREFIX_POINTER_OFFSET = LAST_POINTER_OFFSET - PREFIX_OFFSET; @@ -249,7 +251,7 @@ int inBufferOffset(int pos) /// Pointer offset for a node pointer. - int offset(int pos) + static int offset(int pos) { return pos & (CELL_SIZE - 1); } @@ -304,7 +306,7 @@ static boolean isNullOrLeaf(int node) } /// Returns the number of transitions in a chain cell entered with the given pointer. - private int chainCellLength(int node) + static int chainCellLength(int node) { return LAST_POINTER_OFFSET - offset(node); } @@ -315,7 +317,7 @@ int getChild(int node, int trans) if (isNullOrLeaf(node)) return NONE; - node = followContentTransition(node); + node = followPrefixTransition(node); switch (offset(node)) { @@ -340,7 +342,7 @@ int getNextTransition(int node, int trans) if (isNullOrLeaf(node)) return Integer.MAX_VALUE; - node = followContentTransition(node); + node = followPrefixTransition(node); if (isNullOrLeaf(node)) return Integer.MAX_VALUE; @@ -365,7 +367,7 @@ int getNextChild(int node, int targetTransition) return NONE; } - protected int followContentTransition(int node) + protected int followPrefixTransition(int node) { if (isNullOrLeaf(node)) return NONE; @@ -392,7 +394,7 @@ int advance(int node, int first, ByteSource rest) if (isNullOrLeaf(node)) return NONE; - node = followContentTransition(node); + node = followPrefixTransition(node); switch (offset(node)) { @@ -505,7 +507,7 @@ int getSplitNextTransition(int node, int targetTransition) /// Given a transition, returns the corresponding index (within the node cell) of the pointer to the mid cell of /// a split node. - int splitNodeMidIndex(int trans) + static int splitNodeMidIndex(int trans) { // first 2 bits of the 2-3-3 split return (trans >> 6) & 0x3; @@ -513,7 +515,7 @@ int splitNodeMidIndex(int trans) /// Given a transition, returns the corresponding index (within the mid cell) of the pointer to the tail cell of /// a split node. - int splitNodeTailIndex(int trans) + static int splitNodeTailIndex(int trans) { // second 3 bits of the 2-3-3 split return (trans >> 3) & 0x7; @@ -521,7 +523,7 @@ int splitNodeTailIndex(int trans) /// Given a transition, returns the corresponding index (within the tail cell) of the pointer to the child of /// a split node. - int splitNodeChildIndex(int trans) + static int splitNodeChildIndex(int trans) { // third 3 bits of the 2-3-3 split return trans & 0x7; @@ -555,6 +557,15 @@ T getNodeContent(int node) : null; } + int getAlternateBranch(int node) + { + if (isNullOrLeaf(node)) + return NONE; + if (offset(node) != PREFIX_OFFSET) + return NONE; + return getIntVolatile(node + PREFIX_ALTERNATE_OFFSET); + } + int splitCellPointerAddress(int node, int childIndex, int subLevelLimit) { return node - SPLIT_OFFSET + (8 - subLevelLimit + childIndex) * 4; @@ -621,17 +632,19 @@ int depth(int backtrackDepth) /// When the cursor is asked to advance it first checks the current node for children, and if there aren't any /// (i.e. it is positioned on a leaf node), it goes one level up the backtracking chain, where we are guaranteed to /// have a remaining child to advance to. When there's nothing to backtrack to, the trie is exhausted. - class InMemoryCursor extends CursorBacktrackingState implements Cursor + static class InMemoryCursor extends CursorBacktrackingState implements Cursor { + final InMemoryReadTrie trie; int currentNode; int currentFullNode; - int incomingTransition; - int depth; - T content; + private int incomingTransition; + private int depth; + protected T content; final Direction direction; - InMemoryCursor(Direction direction, int root, int depth, int incomingTransition) + InMemoryCursor(InMemoryReadTrie trie, Direction direction, int root, int depth, int incomingTransition) { + this.trie = trie; this.depth = depth - 1; this.direction = direction; descendInto(root, incomingTransition); @@ -659,8 +672,8 @@ public int advanceMultiple(TransitionsReceiver receiver) return doAdvance(); // Jump directly to the chain's child. - UnsafeBuffer buffer = getBuffer(node); - int inBufferNode = inBufferOffset(node); + UnsafeBuffer buffer = trie.getBuffer(node); + int inBufferNode = trie.inBufferOffset(node); int bytesJumped = chainCellLength(node) - 1; // leave the last byte for incomingTransition if (receiver != null && bytesJumped > 0) receiver.addPathBytes(buffer, inBufferNode, bytesJumped); @@ -736,17 +749,17 @@ public Direction direction() @Override public ByteComparable.Version byteComparableVersion() { - return byteComparableVersion; + return trie.byteComparableVersion; } @Override public Cursor tailCursor(Direction dir) { assert depth >= 0 : "tailCursor called on exhausted cursor"; - return new InMemoryCursor(dir, currentFullNode, 0, -1); + return new InMemoryCursor<>(trie, dir, currentFullNode, 0, -1); } - private int exhausted() + int exhausted() { depth = -1; incomingTransition = -1; @@ -848,7 +861,7 @@ int descendInSplitSublevel(int node, int limit, int collected, int shift) direction.inLoop(childIndex, 0, limit - 1); childIndex += direction.increase) { - child = getSplitCellPointer(node, childIndex, limit); + child = trie.getSplitCellPointer(node, childIndex, limit); if (!isNull(child)) break; } @@ -890,7 +903,7 @@ private int descendInSplitSublevelWithTarget(int node, int limit, int collected, direction.inLoop(childIndex, 0, limit - 1); childIndex += direction.increase) { - child = getSplitCellPointer(node, childIndex, limit); + child = trie.getSplitCellPointer(node, childIndex, limit); if (!isNull(child)) break; isExact = false; @@ -925,7 +938,7 @@ int nextValidSplitTransition(int node, int data) { // Note: This is equivalent to return advanceToSplitTransition(node, data, data) but quicker. assert data >= 0 && data <= 0xFF; - int childIndex = splitNodeChildIndex(data); + int childIndex = trie.splitNodeChildIndex(data); if (childIndex != direction.select(0, SPLIT_OTHER_LEVEL_LIMIT - 1)) { maybeAddSplitBacktrack(node, @@ -933,10 +946,10 @@ int nextValidSplitTransition(int node, int data) SPLIT_OTHER_LEVEL_LIMIT, data & -(1 << (SPLIT_LEVEL_SHIFT * 1)), SPLIT_LEVEL_SHIFT * 0); - int child = getSplitCellPointer(node, childIndex, SPLIT_OTHER_LEVEL_LIMIT); + int child = trie.getSplitCellPointer(node, childIndex, SPLIT_OTHER_LEVEL_LIMIT); return descendInto(child, data); } - int tailIndex = splitNodeTailIndex(data); + int tailIndex = trie.splitNodeTailIndex(data); if (tailIndex != direction.select(0, SPLIT_OTHER_LEVEL_LIMIT - 1)) { maybeAddSplitBacktrack(node, @@ -944,7 +957,7 @@ int nextValidSplitTransition(int node, int data) SPLIT_OTHER_LEVEL_LIMIT, data & -(1 << (SPLIT_LEVEL_SHIFT * 2)), SPLIT_LEVEL_SHIFT * 1); - int tail = getSplitCellPointer(node, tailIndex, SPLIT_OTHER_LEVEL_LIMIT); + int tail = trie.getSplitCellPointer(node, tailIndex, SPLIT_OTHER_LEVEL_LIMIT); return descendInSplitSublevel(tail, SPLIT_OTHER_LEVEL_LIMIT, data & -(1 << SPLIT_LEVEL_SHIFT * 1), @@ -957,7 +970,7 @@ int nextValidSplitTransition(int node, int data) SPLIT_START_LEVEL_LIMIT, 0, SPLIT_LEVEL_SHIFT * 2); - int mid = getSplitCellPointer(node, midIndex, SPLIT_START_LEVEL_LIMIT); + int mid = trie.getSplitCellPointer(node, midIndex, SPLIT_START_LEVEL_LIMIT); return descendInSplitSublevel(mid, SPLIT_OTHER_LEVEL_LIMIT, data & -(1 << SPLIT_LEVEL_SHIFT * 2), @@ -972,7 +985,7 @@ private int advanceToSplitTransition(int node, int data, int skipTransition) if (direction.lt(skipTransition, data)) return nextValidSplitTransition(node, data); // already went over the target in lower sublevel, just advance - int childIndex = splitNodeChildIndex(data); + int childIndex = trie.splitNodeChildIndex(data); if (childIndex != direction.select(0, SPLIT_OTHER_LEVEL_LIMIT - 1)) { int sublevelMask = -(1 << (SPLIT_LEVEL_SHIFT * 1)); @@ -980,7 +993,7 @@ private int advanceToSplitTransition(int node, int data, int skipTransition) int sublevelLimit = SPLIT_OTHER_LEVEL_LIMIT; return descendInSplitSublevelWithTarget(node, sublevelLimit, data & sublevelMask, sublevelShift, skipTransition); } - int tailIndex = splitNodeTailIndex(data); + int tailIndex = trie.splitNodeTailIndex(data); if (tailIndex != direction.select(0, SPLIT_OTHER_LEVEL_LIMIT - 1)) { int sublevelMask = -(1 << (SPLIT_LEVEL_SHIFT * 2)); @@ -1002,7 +1015,7 @@ private void maybeAddSplitBacktrack(int node, int startAfter, int limit, int col direction.inLoop(nextChildIndex, 0, limit - 1); nextChildIndex += direction.increase) { - if (!isNull(getSplitCellPointer(node, nextChildIndex, limit))) + if (!isNull(trie.getSplitCellPointer(node, nextChildIndex, limit))) break; } if (direction.inLoop(nextChildIndex, 0, limit - 1)) @@ -1024,8 +1037,8 @@ private int nextValidSparseTransition(int node, int data) int index = data % SPARSE_CHILD_COUNT; data = data / SPARSE_CHILD_COUNT; - UnsafeBuffer buffer = getBuffer(node); - int inBufferNode = inBufferOffset(node); + UnsafeBuffer buffer = trie.getBuffer(node); + int inBufferNode = trie.inBufferOffset(node); // If there are remaining transitions, add backtracking entry. if (data != exhaustedOrderWord()) @@ -1041,7 +1054,7 @@ private int nextValidSparseTransition(int node, int data) /// For reverse, we also invert the data so that the peeling code above still works. int prepareOrderWord(int node) { - int fwdState = getUnsignedShortVolatile(node + SPARSE_ORDER_OFFSET); + int fwdState = trie.getUnsignedShortVolatile(node + SPARSE_ORDER_OFFSET); if (direction.isForward()) return fwdState; else @@ -1082,8 +1095,8 @@ int exhaustedOrderWord() private int advanceToSparseTransition(int node, int data, int skipTransition) { - UnsafeBuffer buffer = getBuffer(node); - int inBufferNode = inBufferOffset(node); + UnsafeBuffer buffer = trie.getBuffer(node); + int inBufferNode = trie.inBufferOffset(node); int index; int transition; do @@ -1109,8 +1122,8 @@ private int advanceToSparseTransition(int node, int data, int skipTransition) private int getChainTransition(int node) { // No backtracking needed. - UnsafeBuffer buffer = getBuffer(node); - int inBufferNode = inBufferOffset(node); + UnsafeBuffer buffer = trie.getBuffer(node); + int inBufferNode = trie.inBufferOffset(node); int transition = buffer.getByte(inBufferNode) & 0xFF; int next = node + 1; if (offset(next) <= CHAIN_MAX_OFFSET) @@ -1122,8 +1135,8 @@ private int getChainTransition(int node) private int advanceToChainTransition(int node, int skipTransition) { // No backtracking needed. - UnsafeBuffer buffer = getBuffer(node); - int inBufferNode = inBufferOffset(node); + UnsafeBuffer buffer = trie.getBuffer(node); + int inBufferNode = trie.inBufferOffset(node); int transition = buffer.getByte(inBufferNode) & 0xFF; if (direction.gt(skipTransition, transition)) return -1; @@ -1139,9 +1152,9 @@ int descendInto(int child, int transition) { ++depth; incomingTransition = transition; - content = getNodeContent(child); + content = trie.getNodeContent(child); currentFullNode = child; - currentNode = followContentTransition(child); + currentNode = trie.followPrefixTransition(child); return depth; } @@ -1156,7 +1169,7 @@ int descendIntoChain(int child, int transition) } } - private boolean isChainNode(int node) + static boolean isChainNode(int node) { return !isNullOrLeaf(node) && offset(node) <= CHAIN_MAX_OFFSET; } @@ -1193,99 +1206,108 @@ public ByteComparable.Version byteComparableVersion() return byteComparableVersion; } - abstract InMemoryCursor makeCursor(Direction direction); + abstract InMemoryCursor makeCursor(Direction direction); - /// Override of dump to provide more detailed printout that includes the type of each node in the trie. - /// We do this via a wrapping cursor that returns a content string for the type of node for every node we return. - public String dump(Function contentToString) + /// Dump cursor, augmented to show the type of node + class DumpCursor> implements Cursor { - InMemoryCursor source = makeCursor(Direction.FORWARD); - class TypedNodesCursor implements Cursor + final C source; + private final Function contentToString; + + DumpCursor(C source, Function contentToString) { - @Override - public int advance() - { - return source.advance(); - } + this.source = source; + this.contentToString = contentToString; + } + @Override + public int advance() + { + return source.advance(); + } - @Override - public int advanceMultiple(TransitionsReceiver receiver) - { - return source.advanceMultiple(receiver); - } + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + return source.advanceMultiple(receiver); + } - @Override - public int skipTo(int skipDepth, int skipTransition) - { - return source.skipTo(skipDepth, skipTransition); - } + @Override + public int skipTo(int skipDepth, int skipTransition) + { + return source.skipTo(skipDepth, skipTransition); + } - @Override - public int depth() - { - return source.depth(); - } + @Override + public int depth() + { + return source.depth(); + } - @Override - public int incomingTransition() - { - return source.incomingTransition(); - } + @Override + public int incomingTransition() + { + return source.incomingTransition(); + } - @Override - public Direction direction() - { - return source.direction(); - } + @Override + public Direction direction() + { + return source.direction(); + } - @Override - public ByteComparable.Version byteComparableVersion() - { - return source.byteComparableVersion(); - } + @Override + public ByteComparable.Version byteComparableVersion() + { + return source.byteComparableVersion(); + } - @Override - public Cursor tailCursor(Direction direction) - { - throw new AssertionError(); - } + @Override + public DumpCursor tailCursor(Direction direction) + { + throw new AssertionError(); + } - @Override - public String content() + @Override + public String content() + { + String type = null; + int node = source.currentNode; + if (!isNullOrLeaf(node)) { - String type = null; - int node = source.currentNode; - if (!isNullOrLeaf(node)) - { - switch (offset(node)) - { - case SPARSE_OFFSET: - type = "[SPARSE]"; - break; - case SPLIT_OFFSET: - type = "[SPLIT]"; - break; - case PREFIX_OFFSET: - throw new AssertionError("Unexpected prefix as cursor currentNode."); - default: - type = "[CHAIN]"; - break; - } - } - T content = source.content(); - if (content != null) + switch (offset(node)) { - if (type != null) - return contentToString.apply(content) + " -> " + type; - else - return contentToString.apply(content); + case SPARSE_OFFSET: + type = "[SPARSE]"; + break; + case SPLIT_OFFSET: + type = "[SPLIT]"; + break; + case PREFIX_OFFSET: + throw new AssertionError("Unexpected prefix as cursor currentNode."); + default: + type = "[CHAIN]"; + break; } + } + T content = source.content(); + if (content != null) + { + if (type != null) + return contentToString.apply(content) + " -> " + type; else - return type; + return contentToString.apply(content); } + else + return type; } - return new TypedNodesCursor().process(new TrieDumper<>(Function.identity())); + } + + /// Override of dump to provide more detailed printout that includes the type of each node in the trie. + /// We do this via a wrapping cursor that returns a content string for the type of node for every node we return. + public String dump(Function contentToString) + { + return new DumpCursor<>(makeCursor(Direction.FORWARD), contentToString).process(new TrieDumper.Plain<>(Function.identity())); } private void dumpSplitNode(int node, int level, StringBuilder builder) @@ -1347,10 +1369,13 @@ else if (isLeaf(node)) builder.append("Prefix: "); int flags = getUnsignedByte(node + PREFIX_FLAGS_OFFSET); final int content = getIntVolatile(node + PREFIX_CONTENT_OFFSET); + final int alternate = getIntVolatile(node + PREFIX_ALTERNATE_OFFSET); builder.append(content < 0 ? "~" + (~content) : "" + content); - int child = followContentTransition(node); + if (alternate != NONE) + builder.append(" alt:" + alternate); + int child = followPrefixTransition(node); builder.append(" -> ") - .append(child); + .append(dumpNode(child)); break; } default: @@ -1367,3 +1392,4 @@ else if (isLeaf(node)) } } } + diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java index 890ad4a32ac3..eebfed5e1a94 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java @@ -95,9 +95,9 @@ public static InMemoryTrie longLived(ByteComparable.Version byteComparabl return new InMemoryTrie<>(byteComparableVersion, bufferType, ExpectedLifetime.LONG, opOrder); } - public InMemoryCursor makeCursor(Direction direction) + public InMemoryCursor makeCursor(Direction direction) { - return new InMemoryCursor(direction, root, 0, -1); + return new InMemoryCursor<>(this, direction, root, 0, -1); } protected long emptySizeOnHeap() @@ -226,12 +226,25 @@ private > void apply(RangeCursor cursor, static class DeleteMutation, C extends RangeCursor> extends Mutation { + final int initialDepth; + DeleteMutation(UpsertTransformerWithKeyProducer transformer, Predicate> needsForcedCopy, C mutationCursor, InMemoryBaseTrie.ApplyState state) + { + this(transformer, needsForcedCopy, mutationCursor, state, Integer.MAX_VALUE); + } + + DeleteMutation(UpsertTransformerWithKeyProducer transformer, + Predicate> needsForcedCopy, + C mutationCursor, + InMemoryBaseTrie.ApplyState state, + int initialForcedCopyDepth) { super(transformer, needsForcedCopy, mutationCursor, state); + initialDepth = state.currentDepth; + forcedCopyDepth = initialForcedCopyDepth; } @Override @@ -245,7 +258,6 @@ void apply() throws TrieSpaceExhaustedException content = mutationCursor.content(); int depth = state.currentDepth; - int prevAscendDepth = state.setAscendLimit(depth); while (true) { if (depth < forcedCopyDepth) @@ -263,15 +275,16 @@ void apply() throws TrieSpaceExhaustedException } } - depth = mutationCursor.advance(); + depth = mutationCursor.advance() + initialDepth; // Descend but do not modify anything yet. - if (!state.advanceTo(depth, mutationCursor.incomingTransition(), forcedCopyDepth)) + if (!state.advanceTo(depth, mutationCursor.incomingTransition(), forcedCopyDepth, initialDepth)) break; assert state.currentDepth == depth : "Unexpected change to applyState. Concurrent trie modification?"; content = mutationCursor.content(); } - state.setAscendLimit(prevAscendDepth); + + assert state.currentDepth == initialDepth; } /// Walk all existing content covered under a deletion. Returns true if the caller needs to continue processing @@ -280,14 +293,14 @@ void apply() throws TrieSpaceExhaustedException boolean applyDeletionRange(S mutationCoveringState) throws TrieSpaceExhaustedException { boolean atMutation = true; - int depth = mutationCursor.depth(); + int depth = mutationCursor.depth() + initialDepth; int transition = mutationCursor.incomingTransition(); // We are walking both tries in parallel. while (true) { if (atMutation) { - depth = mutationCursor.advance(); + depth = mutationCursor.advance() + initialDepth; transition = mutationCursor.incomingTransition(); atMutation = false; } @@ -327,9 +340,12 @@ void applyCoveringContent(S content) throws TrieSpaceExhaustedException if (content != null) { T existingContent = state.getContent(); - T combinedContent = transformer.apply(existingContent, content, state); - state.setContent(combinedContent, // can be null - state.currentDepth >= forcedCopyDepth); // this is called at the start of processing + if (existingContent != null) + { + T combinedContent = transformer.apply(existingContent, content, state); + state.setContent(combinedContent, // can be null + state.currentDepth >= forcedCopyDepth); // this is called at the start of processing + } } } diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md index c088f5d8ee58..202fc13d8c75 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md +++ b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md @@ -22,9 +22,10 @@ The `InMemoryTrie` is one of the main components of the trie infrastructure, a m modification and reads executing concurrently with writes from a single mutator thread. The main features of its implementation are: -- full support of the `Trie` interface +- full support of the `Trie` hierarchy of interfaces - using nodes of several different types for efficiency - support for content on any node, including intermediate (prefix) +- support for alternate branch pointers - support for writes from a single mutator thread concurrent with multiple readers - various consistency and atomicity guarantees for readers - memory management, off-heap or on-heap @@ -377,14 +378,15 @@ reachable with pointers, they only make sense as substructure of the split node. ![graph](InMemoryTrie.md.g3.svg) -#### Content `Prefix` +#### `Prefix` nodes Prefix nodes are not nodes in themselves, but they add information to the node they lead to. Specifically, they -encode an index in the content array, and a pointer to the node to which this content is attached. In anything other -than the content, they are equivalent to the linked node — i.e. a prefix node pointer has the same children as -the node it links to (another way to see this is as a content-carrying node is one that has an _ε_ transition to the -linked node and no other features except added content). We do not allow more than one prefix to a node (i.e. prefix -can't point to another prefix), and the child of a prefix node cannot be a leaf. +encode an index in the content array, a pointer to any alternate branch, and a pointer to the node to which this +additional information is attached. In anything other than the content/alternate, they are equivalent to the linked node +— i.e. a prefix node pointer has the same children as the node it links to (another way to see this is as a +content-carrying node that has an _ε_ transition to the linked node and no other features except added content and/or +alternate branch). We do not allow more than one prefix to a node (i.e. prefix can't point to another prefix), and the +child of a prefix node cannot be a leaf. There are two types of prefixes: - standalone, which has a full 32-bit pointer to the linked node @@ -393,42 +395,52 @@ of the linked node Standalone prefixes have this layout: -offset|content|example ----|---|--- -00 - 03|content index|00000001 -04|standalone flag, 0xFF|FF -05 - 1B|unused| -1C - 1F|linked node pointer|0000025E +offset | content |example +-------|--------------------------|--- +00 - 03| content pointer |FFFFFFFE ~1 +04 - 07| alternate branch pointer |00000000 NONE +08 | standalone flag, 0xFF |FF +09 - 1B| unused | +1C - 1F| linked node pointer |0000025E and pointer offset `0x1F`. The sample values above will be the ones used to link a prefix node to our `Sparse` example, where a prefix cannot be embedded as all the bytes of the cell are in use. If we want to attach the same prefix to the `Split` example, we will place this -offset|content|example ----|---|--- -00 - 03|content index|00000001 -04|embedded offset within cell|1C -05 - 1F|unused| +offset | content |example +-------|-----------------------------|--- +00 - 03| content pointer |FFFFFFFE ~1 +04 - 07| alternate branch pointer |00000000 NONE +08 | embedded offset within cell |1C +09 - 1F| unused | _inside_ the leading split cell, with pointer `0x1F`. Since this is an embedded node, the augmented one resides within the same cell, and thus we need only 5 bits to encode the pointer (the other 27 are the same as the prefix's). -The combined content of the cell at `0x500-0x51F` will then be `00000001 1C000000 00000000 00000000 00000520 00000560 +The combined content of the cell at `0x500-0x51F` will then be `FFFFFFFE 00000000 1C000000 00000000 00000520 00000560 00000000 00000000`: -offset|content|example ----|---|--- -00 - 03|content index|00000001 -04|embedded offset within cell|1C -05 - 0F|unused| -10 - 13|mid-cell for leading 00|00000520 -14 - 17|mid-cell for leading 01|00000560 -18 - 1B|mid-cell for leading 10|00000000 NONE -1C - 1F|mid-cell for leading 11|00000000 NONE +offset | content |example +-------|-----------------------------|--- +00 - 03| content pointer |FFFFFFFE ~1 +04 - 07| alternate branch pointer |00000000 NONE +08 | embedded offset within cell |1C +09 - 0F| unused | +10 - 13| mid-cell for leading 00 |00000520 +14 - 17| mid-cell for leading 01 |00000560 +18 - 1B| mid-cell for leading 10 |00000000 NONE +1C - 1F| mid-cell for leading 11 |00000000 NONE Both `0x51C` and `0x51F` are valid pointers in this cell. The former refers to the plain split node, the latter to its content-augmented version. The only difference between the two is the result of a call to `content()`. +Note that for code simplicity we store content indexes as pointers, i.e. with the same value as the leaf node for the +given content index. In the example above, `contentArray[1]` is encoded as `~1` i.e. `0xFFFFFFFE`. + +Alternate branch pointers can store a link to another branch of the trie. They are used by deletion-aware tries to store +deletion branches and are ignored by other types of tries. Another possible application of these would be to implement +non-deterministic tries. + ![graph](InMemoryTrie.md.g4.svg) @@ -443,7 +455,8 @@ interface implemented by `InMemoryTrie` (see `Trie.md` for a description of curs ![graph](InMemoryTrie.md.wc1.svg) -(Edges in black show the trie's structure, and the ones in light blue the path the cursor walk takes.) +(Edges in black show the trie's structure, and the ones in light blue the path the +cursor walk takes.) ### Cursors over `InMemoryTrie` diff --git a/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java b/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java index 147c10da6510..d112225e462e 100644 --- a/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java +++ b/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java @@ -176,6 +176,12 @@ private int exhausted() return -1; } + @Override + public T content() + { + return source.content(); + } + @Override public Direction direction() { @@ -197,23 +203,64 @@ public Plain(Cursor source, TrieSetCursor set) } @Override - public T content() + public Cursor tailCursor(Direction direction) { - return source.content(); + switch (state) + { + case MATCHING: + return new Plain<>(source.tailCursor(direction), set.tailCursor(direction)); + case SET_AHEAD: + return source.tailCursor(direction); + default: + throw new AssertionError(); + } + } + } + + static class DeletionAware> + extends IntersectionCursor> + implements DeletionAwareCursor + { + RangeCursor applicableDeletionBranch; + + public DeletionAware(DeletionAwareCursor source, TrieSetCursor set) + { + super(source, set); + applicableDeletionBranch = null; } @Override - public Cursor tailCursor(Direction direction) + public DeletionAwareCursor tailCursor(Direction direction) { switch (state) { case MATCHING: - return new Plain<>(source.tailCursor(direction), set.tailCursor(direction)); + return new DeletionAware<>(source.tailCursor(direction), set.tailCursor(direction)); case SET_AHEAD: return source.tailCursor(direction); default: throw new AssertionError(); } } + + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + RangeCursor deletions = source.deletionBranchCursor(direction); + if (deletions == null) + return null; + + switch (state) + { + case SET_AHEAD: + // Since the deletion branch cannot extend outside this branch, it is fully covered by the set. + return deletions; + case MATCHING: + return new RangeIntersectionCursor<>(deletions, + set.tailCursor(direction)); + default: + throw new AssertionError(); + } + } } } diff --git a/src/java/org/apache/cassandra/db/tries/MergeCursor.java b/src/java/org/apache/cassandra/db/tries/MergeCursor.java index f3dd0ff56c41..94311dd11a39 100644 --- a/src/java/org/apache/cassandra/db/tries/MergeCursor.java +++ b/src/java/org/apache/cassandra/db/tries/MergeCursor.java @@ -18,6 +18,8 @@ package org.apache.cassandra.db.tries; +import java.util.function.BiFunction; + import org.apache.cassandra.utils.bytecomparable.ByteComparable; /// A merged view of two trie cursors. @@ -45,8 +47,8 @@ abstract class MergeCursor> implements Cursor this.resolver = resolver; this.c1 = c1; this.c2 = c2; - assert c1.depth() == 0; - assert c2.depth() == 0; + assert c1.depth() == c2.depth(); + assert c1.incomingTransition() == c2.incomingTransition(); atC1 = atC2 = true; } @@ -235,4 +237,210 @@ else if (atC2) throw new AssertionError(); } } + + /// Deletion-aware merge cursor that efficiently merges two deletion-aware tries. + /// This cursor handles the complex task of merging both live data and deletion metadata + /// from two deletion-aware sources. It supports an important optimization via the + /// `deletionsAtFixedPoints` flag. + static class DeletionAware> + extends MergeCursor> implements DeletionAwareCursor + { + final Trie.MergeResolver deletionResolver; + + /// Tracks the depth at which deletion branches were introduced to avoid redundant processing. + /// Set to -1 when no deletion branches are active. + int deletionBranchDepth = -1; + + /// @see DeletionAwareTrie.MergeResolver#deletionsAtFixedPoints + final boolean deletionsAtFixedPoints; + + /// Creates a deletion-aware merge cursor with configurable deletion optimization. + /// + /// @param mergeResolver resolver for merging live data content + /// @param deletionResolver resolver for merging deletion metadata + /// @param deleter function to apply deletions to live data + /// @param c1 first deletion-aware cursor + /// @param c2 second deletion-aware cursor + /// @param deletionsAtFixedPoints See [DeletionAwareTrie.MergeResolver#deletionsAtFixedPoints] + DeletionAware(Trie.MergeResolver mergeResolver, + Trie.MergeResolver deletionResolver, + BiFunction deleter, + DeletionAwareCursor c1, + DeletionAwareCursor c2, + boolean deletionsAtFixedPoints) + { + this(mergeResolver, + deletionResolver, + new DeletionAwareMergeSource<>(deleter, c1), + new DeletionAwareMergeSource<>(deleter, c2), + deletionsAtFixedPoints); + // We will add deletion sources to the above as we find them. + maybeAddDeletionsBranch(this.c1.depth()); + } + + DeletionAware(Trie.MergeResolver mergeResolver, + Trie.MergeResolver deletionResolver, + DeletionAwareMergeSource c1, + DeletionAwareMergeSource c2, + boolean deletionsAtFixedPoints) + { + super(mergeResolver, c1, c2); + this.deletionResolver = deletionResolver; + this.deletionsAtFixedPoints = deletionsAtFixedPoints; + } + + @Override + public T content() + { + T mc = atC2 ? c2.content() : null; + T nc = atC1 ? c1.content() : null; + if (mc == null) + return nc; + else if (nc == null) + return mc; + else + return resolver.resolve(nc, mc); + } + + @Override + public int advance() + { + return maybeAddDeletionsBranch(super.advance()); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + return maybeAddDeletionsBranch(super.skipTo(skipDepth, skipTransition)); + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + return maybeAddDeletionsBranch(super.advanceMultiple(receiver)); + } + + int maybeAddDeletionsBranch(int depth) + { + if (depth <= deletionBranchDepth) // ascending above common deletions root + { + deletionBranchDepth = -1; + assert !c1.hasDeletions(); + assert !c2.hasDeletions(); + } + + if (atC1 && atC2) + { + maybeAddDeletionsBranch(c1, c2); + maybeAddDeletionsBranch(c2, c1); + } // otherwise even if there is deletion, the other cursor is ahead of it and can't be affected + return depth; + } + + /// Attempts to add deletion branches from one source to another. + /// This method implements the core deletion merging logic. When `deletionsAtFixedPoints` + /// is true, it can skip expensive operations because we know deletion branches are + /// mutually exclusive between sources. + /// + /// @param tgt target merge source that may receive deletions + /// @param src source merge source that may provide deletions + void maybeAddDeletionsBranch(DeletionAwareMergeSource tgt, + DeletionAwareMergeSource src) + { + // If tgt already has deletions applied, no need to add more (we cannot have a deletion branch covering + // another deletion branch). + if (tgt.hasDeletions()) + return; + // Additionally, if `deletionsAtFixedPoints` is in force, we don't need to look for deletions below this + // point when we already have applied tdt's deletions to src. + if (deletionsAtFixedPoints && src.hasDeletions()) + return; + + RangeCursor deletionsBranch = src.deletionBranchCursor(direction); + if (deletionsBranch != null) + tgt.addDeletions(deletionsBranch); // apply all src deletions to tgt + } + + + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + int depth = depth(); + if (deletionBranchDepth != -1 && depth > deletionBranchDepth) + return null; // already covered by a deletion branch, if there is any here it will be reflected in that + + // if one of the two cursors is ahead, it can't affect this deletion branch + if (!atC1) + return maybeSetDeletionsDepth(c2.deletionBranchCursor(direction), depth); + if (!atC2) + return maybeSetDeletionsDepth(c1.deletionBranchCursor(direction), depth); + + // We are positioned at a common branch. If one has a deletion branch, we must combine it with the + // deletion-tree branch of the other to make sure that we merge any higher-depth deletion branch with it. + RangeCursor b1 = c1.deletionBranchCursor(direction); + RangeCursor b2 = c2.deletionBranchCursor(direction); + if (b1 == null && b2 == null) + return null; + + deletionBranchDepth = depth; + + // OPTIMIZATION: When deletionsAtFixedPoints=true, we know that both sources would + // have deletions at the same depth, i.e. if one source has a deletion + // branch at this position, the other cannot have any deletion branches below this + // point. We can thus avoid reproducing the data trie in the deletion branch. + if (deletionsAtFixedPoints) + { + // With the optimization, we can directly return the existing deletion branch + // without needing to create expensive DeletionsTrieCursor instances + if (b1 != null && b2 != null) + { + // Both have deletion branches - merge them directly + return new Range<>(deletionResolver, b1, b2); + } + else + { + // Only one has a deletion branch - return it directly + // The optimization guarantees the other source has no conflicting deletions + return b1 != null ? b1 : b2; + } + } + else + { + // Safe path: create DeletionsTrieCursor for missing deletion branches + // This ensures we capture any deletion branches that might exist deeper + // in the trie structure, but is expensive for large tries because we have + // to list the whole data trie (minus content). + if (b1 == null) + b1 = new DeletionAwareCursor.DeletionsTrieCursor(c1.data.tailCursor(direction)); + if (b2 == null) + b2 = new DeletionAwareCursor.DeletionsTrieCursor(c2.data.tailCursor(direction)); + + return new Range<>(deletionResolver, b1, b2); + } + } + + private RangeCursor maybeSetDeletionsDepth(RangeCursor deletionBranchCursor, int depth) + { + if (deletionBranchCursor != null) + deletionBranchDepth = depth; + return deletionBranchCursor; + } + + @Override + public DeletionAwareCursor tailCursor(Direction direction) + { + if (atC1 && atC2) + return new DeletionAware<>(resolver, + deletionResolver, + c1.tailCursor(direction), + c2.tailCursor(direction), + deletionsAtFixedPoints); + else if (atC1) + return c1.tailCursor(direction); + else if (atC2) + return c2.tailCursor(direction); + else + throw new AssertionError(); + } + } } diff --git a/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java b/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java index ce810e022477..77b1ce5d7e7f 100644 --- a/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java +++ b/src/java/org/apache/cassandra/db/tries/PrefixedCursor.java @@ -213,4 +213,36 @@ public RangeCursor tailCursor(Direction direction) } } } + + static class DeletionAware> + extends PrefixedCursor> implements DeletionAwareCursor + { + DeletionAware(ByteComparable prefix, DeletionAwareCursor tail) + { + super(prefix, tail); + } + + DeletionAware(int firstPrefixByte, ByteSource prefix, DeletionAwareCursor tail) + { + super(firstPrefixByte, prefix, tail); + } + + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + return prefixDone() ? tail.deletionBranchCursor(direction) : null; + } + + @Override + public DeletionAwareCursor tailCursor(Direction direction) + { + if (prefixDone()) + return tail.tailCursor(direction); + else + { + assert depthOfPrefix >= 0 : "tailTrie called on exhausted cursor"; + return new DeletionAware<>(nextPrefixByte, duplicateSource(), tail.tailCursor(direction)); + } + } + } } diff --git a/src/java/org/apache/cassandra/db/tries/RangeApplyCursor.java b/src/java/org/apache/cassandra/db/tries/RangeApplyCursor.java index 1408ee9ce197..d5181e433b7a 100644 --- a/src/java/org/apache/cassandra/db/tries/RangeApplyCursor.java +++ b/src/java/org/apache/cassandra/db/tries/RangeApplyCursor.java @@ -164,4 +164,27 @@ public Cursor tailCursor(Direction direction) else return data.tailCursor(direction); } + + static class DeletionAwareDataBranch> extends RangeApplyCursor implements DeletionAwareCursor + { + DeletionAwareDataBranch(BiFunction resolver, RangeCursor range, Cursor data) + { + super(resolver, range, data); + } + + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + return null; + } + + @Override + public DeletionAwareCursor tailCursor(Direction direction) + { + return new DeletionAwareDataBranch<>(resolver, + atRange ? range.tailCursor(direction) + : RangeCursor.empty(direction, byteComparableVersion()), + data.tailCursor(direction)); + } + } } diff --git a/src/java/org/apache/cassandra/db/tries/RangeCursor.java b/src/java/org/apache/cassandra/db/tries/RangeCursor.java index ef85ad0b1a20..27470802b480 100644 --- a/src/java/org/apache/cassandra/db/tries/RangeCursor.java +++ b/src/java/org/apache/cassandra/db/tries/RangeCursor.java @@ -95,6 +95,8 @@ default S content() /// Returns a full-range cursor returning [#precedingState()]. default RangeCursor precedingStateCursor(Direction direction) { + // Note: this uses `precedingState` in the current direction, which must be the same as the preceding state we + // would get if we walked to the same state in the opposite direction. return new Empty<>(precedingState(), byteComparableVersion(), direction); } @@ -132,6 +134,77 @@ static > RangeCursor empty(Direction direction, ByteC return new Empty(null, version, direction); } + class Done> implements RangeCursor + { + final Direction direction; + final ByteComparable.Version version; + + public Done(Direction direction, ByteComparable.Version version) + { + this.direction = direction; + this.version = version; + } + + @Override + public S state() + { + return null; + } + + @Override + public int depth() + { + return -1; + } + + @Override + public int incomingTransition() + { + return -1; + } + + @Override + public S content() + { + return null; + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return version; + } + + @Override + public int advance() + { + return -1; + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + return -1; + } + + @Override + public RangeCursor tailCursor(Direction direction) + { + return this; + } + } + + static > RangeCursor done(Direction direction, ByteComparable.Version version) + { + return new Done<>(direction, version); + } + class FromSet> implements RangeCursor { final TrieSetCursor source; diff --git a/src/java/org/apache/cassandra/db/tries/RangeTrie.java b/src/java/org/apache/cassandra/db/tries/RangeTrie.java index 26b9a69675de..e955abdbe802 100644 --- a/src/java/org/apache/cassandra/db/tries/RangeTrie.java +++ b/src/java/org/apache/cassandra/db/tries/RangeTrie.java @@ -19,6 +19,7 @@ import java.util.Collection; import java.util.Iterator; +import java.util.Map; import java.util.function.BiFunction; import com.google.common.base.Preconditions; @@ -40,11 +41,6 @@ /// See [RangeCursor] and [Trie.md](./Trie.md) for further details on the implementation of range tries. public interface RangeTrie> extends BaseTrie, RangeTrie> { - default R process(Direction direction, Cursor.Walker walker) - { - return cursor(direction).process(walker); - } - /// Returns a singleton trie mapping the given byte path to a marker. /// /// Note: Ranges are meant to use boundaries that are distinct from data and thus a singleton range would list @@ -58,7 +54,6 @@ static > RangeTrie singleton(ByteComparable key, Byte return dir -> new SingletonCursor.Range<>(dir, key.asComparableBytes(byteComparableVersion), byteComparableVersion, v); } - /// Returns a range trie covering a single range. This performs the same process as intersecting a covered /// range by a set, converting the passed marker to the proper state depending on the set's coverage and boundaries. /// To this end, the passed marker must be a covering state (i.e. it must not be reportable, and must have the same @@ -164,6 +159,13 @@ else if (c.precedingState() != null) return null; } + /// Returns an entry set containing all tail tree constructed at the points that contain content of + /// the given type. + default Iterable>> tailTries(Direction direction, Class clazz) + { + return () -> new TrieTailsIterator.AsEntriesRange<>(cursor(direction), clazz); + } + RangeCursor makeCursor(Direction direction); @Override diff --git a/src/java/org/apache/cassandra/db/tries/SingletonCursor.java b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java index be7fc910bf21..1aff55183e88 100644 --- a/src/java/org/apache/cassandra/db/tries/SingletonCursor.java +++ b/src/java/org/apache/cassandra/db/tries/SingletonCursor.java @@ -183,4 +183,61 @@ public Range tailCursor(Direction dir) return new Range<>(dir, nextTransition, duplicateSource(), byteComparableVersion, value); } } + + static class DeletionAware> + extends SingletonCursor implements DeletionAwareCursor + { + DeletionAware(Direction direction, ByteSource src, ByteComparable.Version byteComparableVersion, T value) + { + super(direction, src, byteComparableVersion, value); + } + + DeletionAware(Direction direction, int firstByte, ByteSource src, ByteComparable.Version byteComparableVersion, T value) + { + super(direction, firstByte, src, byteComparableVersion, value); + } + + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + return null; + } + + @Override + public DeletionAware tailCursor(Direction dir) + { + return new DeletionAware<>(dir, nextTransition, duplicateSource(), byteComparableVersion, value); + } + } + + static class DeletionBranch> + extends SingletonCursor implements DeletionAwareCursor + { + RangeTrie deletionBranch; + + DeletionBranch(Direction direction, ByteSource src, ByteComparable.Version byteComparableVersion, RangeTrie deletionBranch) + { + super(direction, src, byteComparableVersion, null); + this.deletionBranch = deletionBranch; + } + + DeletionBranch(Direction direction, int firstByte, ByteSource src, ByteComparable.Version byteComparableVersion, RangeTrie deletionBranch) + { + + super(direction, firstByte, src, byteComparableVersion, null); + this.deletionBranch = deletionBranch; + } + + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + return atEnd() ? deletionBranch.cursor(direction) : null; + } + + @Override + public DeletionBranch tailCursor(Direction dir) + { + return new DeletionBranch<>(dir, nextTransition, duplicateSource(), byteComparableVersion, deletionBranch); + } + } } diff --git a/src/java/org/apache/cassandra/db/tries/Trie.md b/src/java/org/apache/cassandra/db/tries/Trie.md index 489c5c3c3f44..f6b6cf4622c8 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.md +++ b/src/java/org/apache/cassandra/db/tries/Trie.md @@ -342,7 +342,7 @@ present positions before and after any valid key (both provided by our byte-comp correctly define ranges between any two keys, with the posibility of inclusive or exclusive boundaries as needed. As we also would also like to retrieve metadata on the paths leading to queried keys (e.g. a partition marker and stats -when we query information for a subset of rows in that partition), the fact that these sets always produce prefixes can +when we query information for a subset of rows in that partition), the fact that these sets always include prefixes can be seen as an advantage. ## Converting ranges to trie sets @@ -545,8 +545,8 @@ a -> ## Union of range tries -The union process is similar (with a second range trie instead of a set), but we walk all branches of both tries and -combine their states. There are two differences from the normal trie union process: +The merge process is similar (with a second range trie instead of a set), but we walk all branches of both tries and +combine their states. There are two differences from the normal trie merge process: - We apply the merge resolver to states instead of content. This includes both content and preceding state, which is necessary to be able to report the correct state for the merged trie. - When one of the range cursors is ahead, we pass its `precedingState` as an argument to the merge resolver to modify @@ -633,4 +633,217 @@ state for both forward and reverse iteration directions. Such richer state is not forbidden, but also not necessary for a general range trie. Because it is something that is pretty difficult to obtain (or store and maintain) for an in-memory trie, general range tries, which are meant to be -stored in in-memory tries, do not provide it. \ No newline at end of file +stored in in-memory tries, do not provide it. + +# Deletion-Aware Tries + +Deletion-aware tries are designed to store live data together with ranges of deletions (aka tombstones) in a single +structure, and be able to apply operations over them that properly restrict deletion ranges on intersections and apply +the deletions of one source to the live content of others in merges. + +Our deletion-aware tries implement this by allowing nodes in the trie to offer a "deletions branch" which specifies +and encloses the deletion ranges applicable to the branch rooted at that node. This can be provided at any level of the +trie, but only once for any given path (i.e. there cannot be a deletion branch under another deletion branch). In many +practical usecases the depth at which this deletion path is introduced will also be predetermined for any given path; +merges implement an option that exploits this property to avoid some inefficiencies. + +It is also forbidden for live branches to contain data that is deleted by the trie's own deletion branches (aka +shadowed data). + +## Alternatives + +Perhaps the easiest way to describe why we chose this approach is to discuss the alternatives and discuss the reasons +we chose the structure and features of the option we went with. + +### Why not mix deletion ranges with live data? + +In this approach we store deletions as ranges, and live data as point ranges in the single structure. They are ordered +together and, to facilitate an efficient `precedingState`, points need to specify the applicable deletions before and +after the point. This approach is an evolution of Cassandra's `UnfilteredRowIterator` that mixes rows and tombstone +markers. + +The example below represents a trie that contains a deletion from `aaa` to `acc` with timestamp 666, and a live point at +`abb` with timestamp 700 in this fashion: +``` +a -> + a -> + a -> start(666) + b -> covering(666) + b -> data(value, 700) + switch(666, 666) + c -> covering(666) + c -> end(666) +``` + +Having the point also declare the state before and after makes it easy to obtain the covering deletion e.g. for `aba`, +`abc` or `ab`. This is a very acceptable amount of overhead that isn't a problem for the approach. + +The greatest strength of this approach is that it makes it very easy to perform merges because all of the necessary +information is present at any position that the merging cursor visits. + +The reason to avoid this approach is that we often want to find only the live data between a given range of keys, or the +closest live entry after a given key. In an approach like this we can have many thousands of deletion markers that +precede the live entries, and to find it we have to filter these deletions out. + +In fact, we have found this situation to occur often in many practical applications of Cassandra. Solving this problem +is one of the main reasons to implement the `Trie` machinery. + +This problem could be worked around by storing metadata at parent nodes whose branches don't contain live data; we went +with a more flexible approach. + +### Why not store live data and deletions separately? + +In the other extreme, we can have two separate tries. For the example above, it could look like this: +``` +LIVE +b -> + b -> data(value, 700) +DELETIONS +a -> + a -> + a -> start(666) + c -> covering(666) + c -> end(666) +``` + +To perform a merge, we have to apply the DELETIONS trie of each source to the other's LIVE trie. In other words, a merge +can be implemented as +``` +merge(a, b).LIVE = merge(apply(b.DELETIONS, a.LIVE), apply(a.DELETIONS, b.LIVE)) +merge(a, b).DELETIONS = merge(a.DELETIONS, b.DELETIONS) +``` +or (which makes better sense when multiple sources are merged): +``` +d = merge(a.DELETIONS, b.DELETIONS) +merge(a, b).LIVE = apply(d, merge(a.LIVE, b.LIVE)) +merge(a, b).DELETIONS = d +``` + +This can create extra complexity when multiple merge operations are applied on top of one another, but if we select all +sources in advance and merge them with a single collection merge the method's performance is good. + +This solves the issue above: because we query live and deletions separately, we can efficiently get the first live item +after a point. We can also get the preceding state of a deletion without storing extra information at live points. + +The approach we ultimately took is an evolution of this to avoid a couple of performance weaknesses. On one hand, it is +a little inefficient to walk the same path in two separate tries, and it would be helpful if we can do this only once +for at least some part of the key. On the other, there is a concurrency chokepoint at the root of this structure, because +whenever a deletion actually finds live data to remove in an in-memory trie, to achieve atomicity of the operation we +need to prepare and swap snapshots for the two full tries, which can waste work and limits caching efficiency. + +In Cassandra we use partitions as the unit of consistency, and also the limit that range deletions are not allowed to +cross. It is natural, then, to split the live and deletion branches at the partition level rather than at the trie root. + +### Why not allow shadowed data, i.e. data deleted by the same trie's deletion branches? + +One way to avoid the concurrency issue above is to leave the live data in place and apply the deletion trie on every +query. This does ease the atomicity problem, and in addition makes the merge process simpler as we can independently +merge data and deletion branches. + +However, we pay a cost on live data read that is not insignificant, and the amount of space and effort we must spend +to deal with the retained data items can quickly compound unless we apply garbage collection at some points. We prefer +to do that garbage collection as early as possible, by not introducing the garbage in the first place. + +There is a potential application of relaxing this for intermediate states of transformation, e.g. by letting a merge +delay the application of the deletions until the end of a chain of transformations. This is an internal implementation +detail that would not change the requirements for the user. + +### Why not allow nested deletion branches? + +If it makes sense to permit deletion branches, then we could have them at multiple levels, reducing the amount of path +duplication in the trie. + +For example, using a deletion branch we can represent the example above as +``` +a -> + *** start deletion branch + a -> + a -> start(666) + c -> covering(666) + c -> end(666) + *** end deletion branch + b -> + b -> data(value, 700) +``` + +and if we then delete `aba-abc` with timestamp 777, represented as +``` +a -> + b -> + *** start deletion branch + a -> start(777) + c -> end(777) + *** end deletion branch +``` +we could merge it into the in-memory trie as +``` +a -> + *** start deletion branch + a -> + a -> start(666) + c -> covering(666) + c -> end(666) + *** end deletion branch + b -> + *** start deletion branch + a -> start(777) + c -> end(777) + *** end deletion branch +``` + +This can work well for point queries and has some simplicity advantages for merging, but introduces a +complication tracking the state when we want to walk over a range and apply deletion branches to data. The problem is +that we don't easily know what deletion state applies e.g. when we advance from `abc` in the trie above; we either have +to keep a stack of applicable ranges, or store the deletion to return to in the nested deletion branch, which would +cancel out the simplicity advantages. + +### Why predetermined deletion levels (`deletionsAtFixedPoints`) are important + +The case above (nested deletion branches) is something that can naturally occur in merges, including as shown in the +example. If we don't do anything special, this merge would create a nesting of branches. + +We fix this problem by applying "hoisting" during merges, i.e. by bringing other sources' covered deletion branches to +the highest level that one source defines it. For the example above, this means that when the merged cursor encounters +the in-memory deletion branch at `a`, it has to hoist the mutation's deletion to be rooted at `a` rather than `ab`. + +In other words, the mutation is changed to effectively become +``` +a -> + *** start deletion branch + b -> + a -> start(777) + c -> end(777) + *** end deletion branch +``` + +which can then be correctly combined into +``` +a -> + *** start deletion branch + a -> + a -> start(666) + b -> covering(666) + a -> switch(666, 777) + c -> switch(777, 666) + c -> covering(666) + c -> end(666) + *** end deletion branch +``` + +The hoisting process can be very inefficient. The reason for this is that we do not know where in the source trie a +deletion branch is defined, and to bring them all to a certain level we must walk the whole live branch. If e.g. this +in-memory trie never had a deletion before, this could mean walking all the live data in the trie, potentially millions +of nodes. + +Provided that the result of this hoisting becomes a new deletion branch, which would be the case for in-memory tries, +one can say that the amortized cost is still O(1) because once we hoist a branch we never have to walk that branch +again. The delay of doing that pass could still cause problems; more importantly, in merge views we may have to do that +multiple times, especially on nested merges. + +To avoid this issue, deletion-aware merges accept a flag called `deletionsAtFixedPoints`. When this flag is true, the +merge expects that all sources can only define deletion branches at matching points. If this is guaranteed, we do not +need to do any hoisting, because a covered deletion branch cannot exist. We expect most practical uses of this class to +perform all merges with this flag set to true. + +This means preparing deletions so that they always share the same point of introduction of the deletion branch. For the +example above it means preparing the deletion in the hoisted form. In Cassandra, for example, this can be guaranteed +by wiring the deletion branches to always be on the partition level. \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/tries/TrieDumper.java b/src/java/org/apache/cassandra/db/tries/TrieDumper.java index 5839e34b0b96..d04b672972c8 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieDumper.java +++ b/src/java/org/apache/cassandra/db/tries/TrieDumper.java @@ -22,20 +22,19 @@ import org.agrona.DirectBuffer; /// Simple utility class for dumping the structure of a trie to string. -class TrieDumper implements Cursor.Walker +abstract class TrieDumper implements Cursor.Walker { - private final StringBuilder b; - private final Function contentToString; + protected final StringBuilder b; int needsIndent = -1; int currentLength = 0; + int depthAdjustment = 0; - TrieDumper(Function contentToString) + TrieDumper() { - this.contentToString = contentToString; this.b = new StringBuilder(); } - private void endLineAndSetIndent(int newIndent) + protected void endLineAndSetIndent(int newIndent) { needsIndent = newIndent; } @@ -43,11 +42,11 @@ private void endLineAndSetIndent(int newIndent) @Override public void resetPathLength(int newLength) { - currentLength = newLength; - endLineAndSetIndent(newLength); + currentLength = newLength + depthAdjustment; + endLineAndSetIndent(currentLength); } - private void maybeIndent() + protected void maybeIndent() { if (needsIndent >= 0) { @@ -76,16 +75,68 @@ public void addPathBytes(DirectBuffer buffer, int pos, int count) } @Override - public void content(T content) + public String complete() { - b.append(" -> "); - b.append(contentToString.apply(content)); - endLineAndSetIndent(currentLength); + return b.toString(); } - @Override - public String complete() + static class Plain extends TrieDumper { - return b.toString(); + protected final Function contentToString; + + public Plain(Function contentToString) + { + super(); + this.contentToString = contentToString; + } + + @Override + public void content(T content) + { + b.append(" -> "); + b.append(contentToString.apply(content)); + endLineAndSetIndent(currentLength); + } + } + + static class DeletionAware> extends Plain + implements DeletionAwareTrie.DeletionAwareWalker + { + final Function rangeToString; + + public DeletionAware(Function contentToString, + Function rangeToString) + { + super(contentToString); + this.rangeToString = rangeToString; + } + + @Override + public void deletionMarker(D content) + { + b.append(" -> "); + b.append(rangeToString.apply(content)); + endLineAndSetIndent(currentLength); + } + + @Override + public boolean enterDeletionsBranch() + { + b.append("*** Start deletion branch"); + endLineAndSetIndent(currentLength); + depthAdjustment = currentLength; + currentLength = 0; + return true; + } + + @Override + public void exitDeletionsBranch() + { + endLineAndSetIndent(depthAdjustment); + maybeIndent(); + b.append("*** End deletion branch"); + resetPathLength(0); + depthAdjustment = 0; + } } } diff --git a/src/java/org/apache/cassandra/db/tries/TrieSet.java b/src/java/org/apache/cassandra/db/tries/TrieSet.java index 4d6409817417..275d2834fc77 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieSet.java +++ b/src/java/org/apache/cassandra/db/tries/TrieSet.java @@ -117,7 +117,7 @@ default TrieSet weakNegation() /// Constuct a textual representation of the trie. default String dump() { - return cursor(Direction.FORWARD).process(new TrieDumper<>(Object::toString)); + return cursor(Direction.FORWARD).process(new TrieDumper.Plain<>(Object::toString)); } TrieSetCursor makeCursor(Direction direction); diff --git a/src/java/org/apache/cassandra/db/tries/TrieTailsIterator.java b/src/java/org/apache/cassandra/db/tries/TrieTailsIterator.java index dff6ca87c2bb..7c906765a404 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieTailsIterator.java +++ b/src/java/org/apache/cassandra/db/tries/TrieTailsIterator.java @@ -26,21 +26,14 @@ /// Iterator of trie entries that constructs tail tries for the content-bearing branches that satisfy the given predicate /// and skips over the returned branches. -public abstract class TrieTailsIterator extends TriePathReconstructor implements Iterator +public abstract class TrieTailsIterator> extends TriePathReconstructor implements Iterator { - final Cursor cursor; + final C cursor; private final Predicate predicate; private T next; private boolean gotNext; - protected TrieTailsIterator(Trie trie, Direction direction, Predicate predicate) - { - this.cursor = trie.cursor(direction); - this.predicate = predicate; - assert cursor.depth() == 0; - } - - TrieTailsIterator(Cursor cursor, Predicate predicate) + TrieTailsIterator(C cursor, Predicate predicate) { this.cursor = cursor; this.predicate = predicate; @@ -84,20 +77,104 @@ public V next() gotNext = false; T v = next; next = null; - return mapContent(v, dir -> cursor.tailCursor(dir), keyBytes, keyPos); + return getContent(v); } + protected abstract V getContent(T v); + ByteComparable.Version byteComparableVersion() { return cursor.byteComparableVersion(); } - protected abstract V mapContent(T value, Trie tailTrie, byte[] bytes, int byteLength); + public static abstract class Plain extends TrieTailsIterator> + { + Plain(Cursor cursor, Predicate predicate) + { + super(cursor, predicate); + } + + /// Public constructor accepting a Trie and creating a cursor from it + public Plain(Trie trie, Predicate predicate) + { + this(trie.cursor(Direction.FORWARD), predicate); + } + + /// Public constructor accepting a Trie, Direction, and creating a cursor from it + public Plain(Trie trie, Direction direction, Predicate predicate) + { + this(trie.cursor(direction), predicate); + } + + @Override + protected V getContent(T v) + { + return mapContent(v, dir -> cursor.tailCursor(dir), keyBytes, keyPos); + } + + protected abstract V mapContent(T value, Trie tailTrie, byte[] bytes, int byteLength); + } + + public static abstract class Range, V> extends TrieTailsIterator> + { + Range(RangeCursor cursor, Predicate predicate) + { + super(cursor, predicate); + } + + /// Public constructor accepting a RangeTrie and creating a cursor from it + public Range(RangeTrie trie, Predicate predicate) + { + this(trie.cursor(Direction.FORWARD), predicate); + } + + /// Public constructor accepting a RangeTrie, Direction, and creating a cursor from it + public Range(RangeTrie trie, Direction direction, Predicate predicate) + { + this(trie.cursor(direction), predicate); + } + + @Override + protected V getContent(S v) + { + return mapContent(v, dir -> cursor.tailCursor(dir), keyBytes, keyPos); + } + + protected abstract V mapContent(S value, RangeTrie tailTrie, byte[] bytes, int byteLength); + } + + public static abstract class DeletionAware, V> extends TrieTailsIterator> + { + DeletionAware(DeletionAwareCursor cursor, Predicate predicate) + { + super(cursor, predicate); + } + + /// Public constructor accepting a DeletionAwareTrie and creating a cursor from it + public DeletionAware(DeletionAwareTrie trie, Predicate predicate) + { + this(trie.cursor(Direction.FORWARD), predicate); + } + + /// Public constructor accepting a DeletionAwareTrie, Direction, and creating a cursor from it + public DeletionAware(DeletionAwareTrie trie, Direction direction, Predicate predicate) + { + this(trie.cursor(direction), predicate); + } + + @Override + protected V getContent(T v) + { + return mapContent(v, dir -> cursor.tailCursor(dir), keyBytes, keyPos); + } + + protected abstract V mapContent(T value, DeletionAwareTrie tailTrie, byte[] bytes, int byteLength); + } /// Iterator representing the selected content of the trie a sequence of `(path, tail)` pairs, where /// `tail` is the branch of the trie rooted at the selected content node (reachable by following /// `path`). The tail trie will have the selected content at its root. - static class AsEntries extends TrieTailsIterator>> + static class AsEntries extends Plain>> { public AsEntries(Cursor cursor, Class clazz) { @@ -111,4 +188,41 @@ protected Map.Entry> mapContent(T value, Trie return new AbstractMap.SimpleImmutableEntry<>(key, tailTrie); } } + + /// Iterator representing the selected content of the trie a sequence of `(path, tail)` pairs, where + /// `tail` is the branch of the trie rooted at the selected content node (reachable by following + /// `path`). The tail trie will have the selected content at its root. + static class AsEntriesRange> extends Range>> + { + public AsEntriesRange(RangeCursor cursor, Class clazz) + { + super(cursor, clazz::isInstance); + } + + @Override + protected Map.Entry> mapContent(S value, RangeTrie tailTrie, byte[] bytes, int byteLength) + { + ByteComparable key = toByteComparable(byteComparableVersion(), bytes, byteLength); + return new AbstractMap.SimpleImmutableEntry<>(key, tailTrie); + } + } + + + /// Iterator representing the selected content of the trie a sequence of `(path, tail)` pairs, where + /// `tail` is the branch of the trie rooted at the selected content node (reachable by following + /// `path`). The tail trie will have the selected content at its root. + static class AsEntriesDeletionAware> extends DeletionAware>> + { + public AsEntriesDeletionAware(DeletionAwareCursor cursor, Class clazz) + { + super(cursor, clazz::isInstance); + } + + @Override + protected Map.Entry> mapContent(T value, DeletionAwareTrie tailTrie, byte[] bytes, int byteLength) + { + ByteComparable key = toByteComparable(byteComparableVersion(), bytes, byteLength); + return new AbstractMap.SimpleImmutableEntry<>(key, tailTrie); + } + } } diff --git a/src/java/org/apache/cassandra/db/tries/VerificationCursor.java b/src/java/org/apache/cassandra/db/tries/VerificationCursor.java index 83096d56939b..ed6b8d7444d2 100644 --- a/src/java/org/apache/cassandra/db/tries/VerificationCursor.java +++ b/src/java/org/apache/cassandra/db/tries/VerificationCursor.java @@ -53,6 +53,11 @@ class Plain> implements Cursor, Cursor.TransitionsRece Cursor.TransitionsReceiver chainedReceiver = null; boolean advanceMultipleCalledReceiver; + Plain(C cursor) + { + this(cursor, 0, 0, INITIAL_TRANSITION); + } + Plain(C cursor, int minDepth, int expectedDepth, int expectedTransition) { this.direction = cursor.direction(); @@ -178,7 +183,7 @@ else if (depth <= returnedDepth) @SuppressWarnings("unchecked") public Plain tailCursor(Direction direction) { - return new Plain<>((C) source.tailCursor(direction), 0, 0, INITIAL_TRANSITION); + return new Plain<>((C) source.tailCursor(direction)); } @@ -242,9 +247,9 @@ abstract class WithRanges, C extends RangeCursor> S nextPrecedingState = null; int maxNextDepth = Integer.MAX_VALUE; - WithRanges(C source, int minDepth, int expectedDepth, int expectedTransition) + WithRanges(C source) { - super(source, minDepth, expectedDepth, expectedTransition); + super(source); // start state can be non-null for sets currentPrecedingState = verifyCoveringStateProperties(source.precedingState()); final S content = source.content(); @@ -288,12 +293,10 @@ private void checkIfDescentShouldBeForbidden() @Override public S precedingState() { - Preconditions.checkState(currentPrecedingState == source.precedingState(), - "Preceding state changed without advance: %s -> %s. %s\n%s", + Preconditions.checkState(currentPrecedingState == source.precedingState() || + currentPrecedingState != null && currentPrecedingState.equals(source.precedingState()), + "Preceding state changed without advance: %s -> %s.\n%s", currentPrecedingState, source.precedingState(), - agree(currentPrecedingState, source.precedingState()) - ? "The values are equal but different object. This is not permitted for performance reasons." - : "", this); return currentPrecedingState; } @@ -394,12 +397,7 @@ class Range> extends WithRanges> imple { Range(RangeCursor source) { - this(source, 0, 0, INITIAL_TRANSITION); - } - - Range(RangeCursor source, int minDepth, int expectedDepth, int expectedTransition) - { - super(source, minDepth, expectedDepth, expectedTransition); + super(source); Preconditions.checkState(currentPrecedingState == null, "Initial preceding state %s should be null for range cursor\n%s", currentPrecedingState, this); @@ -416,7 +414,7 @@ void verifyEndState() @Override public Range tailCursor(Direction direction) { - return new Range<>(source.tailCursor(direction), 0, 0, INITIAL_TRANSITION); + return new Range<>(source.tailCursor(direction)); } } @@ -424,12 +422,7 @@ class TrieSet extends WithRanges implem { TrieSet(TrieSetCursor source) { - this(source, 0, 0, INITIAL_TRANSITION); - } - - TrieSet(TrieSetCursor source, int minDepth, int expectedDepth, int expectedTransition) - { - super(source, minDepth, expectedDepth, expectedTransition); + super(source); // start state can be non-null for sets Preconditions.checkNotNull(currentPrecedingState, "Preceding state for trie sets must not be null\n%s", this); } @@ -443,7 +436,90 @@ public TrieSetCursor.RangeState state() @Override public TrieSet tailCursor(Direction direction) { - return new TrieSet(source.tailCursor(direction), 0, 0, INITIAL_TRANSITION); + return new TrieSet(source.tailCursor(direction)); + } + } + + class DeletionAware> + extends VerificationCursor.Plain> + implements DeletionAwareCursor + { + int deletionBranchDepth; + + DeletionAware(DeletionAwareCursor source) + { + super(source); + this.deletionBranchDepth = -1; + verifyDeletionBranch(0); + } + + @Override + public int incomingTransition() + { + return source.incomingTransition(); + } + + @Override + public int advance() + { + return verifyDeletionBranch(super.advance()); + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + return verifyDeletionBranch(super.advanceMultiple(receiver)); + } + + @Override + public int skipTo(int skipDepth, int skipTransition) + { + return verifyDeletionBranch(super.skipTo(skipDepth, skipTransition)); + } + + @Override + public RangeCursor deletionBranchCursor(Direction direction) + { + // deletionBranch is already verified + final RangeCursor deletionBranch = source.deletionBranchCursor(direction); + if (deletionBranch == null) + return null; + return new Range<>(deletionBranch); + } + + int verifyDeletionBranch(int depth) + { + if (depth <= deletionBranchDepth) + deletionBranchDepth = -1; + + var deletionBranch = source.deletionBranchCursor(direction); + if (deletionBranch != null) + { + Preconditions.checkState(deletionBranchDepth == -1, + "Deletion branch at depth %s covered by another deletion branch at parent depth %s", + depth, deletionBranchDepth); + Preconditions.checkState(deletionBranch.depth() == 0, + "Invalid deletion branch initial depth %s", + deletionBranch.depth()); + Preconditions.checkState(deletionBranch.incomingTransition() == INITIAL_TRANSITION, + "Invalid deletion branch initial transition %s", + deletionBranch.incomingTransition()); + Preconditions.checkState(deletionBranch.precedingState() == null, + "Deletion branch starts with active deletion %s", + deletionBranch.precedingState()); + deletionBranch.skipTo(EXHAUSTED_DEPTH, EXHAUSTED_TRANSITION); + Preconditions.checkState(deletionBranch.precedingState() == null, + "Deletion branch ends with active deletion %s", + deletionBranch.precedingState()); + deletionBranchDepth = depth; + } + return depth; + } + + @Override + public DeletionAware tailCursor(Direction direction) + { + return new DeletionAware<>(source.tailCursor(direction)); } } } diff --git a/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.java b/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.java index 847073dbe1de..9cfb347c1f89 100644 --- a/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.java +++ b/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.java @@ -171,6 +171,17 @@ static ByteComparable cut(ByteComparable src, int cutoff) return version -> ByteSource.cut(src.asComparableBytes(version), cutoff); } + static ByteComparable skipFirst(ByteComparable src, int bytesToSkip) + { + return version -> + { + ByteSource bsrc = src.asComparableBytes(version); + for (int i = 0; i < bytesToSkip; i++) + bsrc.next(); + return bsrc; + }; + } + /** * Return the length of a byte comparable, not including the terminator byte. */ diff --git a/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java b/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java index 32e9731ca278..f9c5c3772ad6 100644 --- a/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java +++ b/test/unit/org/apache/cassandra/db/tries/CellReuseTest.java @@ -226,7 +226,7 @@ private void mark(InMemoryTrie trie, int node, BitSet set, BitSet objs) else markChild(trie, content, set, objs); - markChild(trie, trie.followContentTransition(node), set, objs); + markChild(trie, trie.followPrefixTransition(node), set, objs); break; default: assert trie.offset(node) <= InMemoryTrie.CHAIN_MAX_OFFSET && trie.offset(node) >= InMemoryTrie.CHAIN_MIN_OFFSET; diff --git a/test/unit/org/apache/cassandra/db/tries/CombinedDataPoint.java b/test/unit/org/apache/cassandra/db/tries/CombinedDataPoint.java new file mode 100644 index 000000000000..33f5e06fa318 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/CombinedDataPoint.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Objects; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +class CombinedDataPoint implements DataPoint +{ + final LivePoint livePoint; + final DeletionMarker marker; + + public CombinedDataPoint(LivePoint livePoint, DeletionMarker marker) + { + this.livePoint = livePoint; + this.marker = marker; + } + + @Override + public DeletionMarker marker() + { + return marker; + } + + @Override + public LivePoint live() + { + return livePoint; + } + + @Override + public ByteComparable position() + { + return livePoint.position(); + } + + @Override + public DataPoint withMarker(DeletionMarker newMarker) + { + if (newMarker == null) + return livePoint; + else + return new CombinedDataPoint(livePoint, newMarker); + } + + @Override + public DataPoint remap(ByteComparable newKey) + { + return new CombinedDataPoint(livePoint.remap(newKey), marker.remap(newKey)); + } + + @Override + public String toString() + { + return marker.toString() + 'L' + livePoint.timestamp; + } + + public DataPoint toContent() + { + if (marker.isBoundary) + return this; + return livePoint; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + CombinedDataPoint that = (CombinedDataPoint) o; + return Objects.equals(livePoint, that.livePoint) && Objects.equals(marker, that.marker); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java b/test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java index 369a8ed1e371..0ce4cfb6e69a 100644 --- a/test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java +++ b/test/unit/org/apache/cassandra/db/tries/ConsistencyTestBase.java @@ -19,7 +19,6 @@ package org.apache.cassandra.db.tries; import java.util.ArrayList; -import java.util.BitSet; import java.util.Collection; import java.util.List; import java.util.Map; @@ -29,9 +28,11 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; import java.util.function.LongUnaryOperator; import java.util.function.Predicate; +import com.google.common.collect.Iterables; import org.junit.Assert; import org.junit.Test; @@ -42,6 +43,7 @@ import static org.apache.cassandra.db.tries.TrieUtil.VERSION; import static org.apache.cassandra.db.tries.TrieUtil.generateKeys; import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; +import static org.junit.Assert.assertTrue; public abstract class ConsistencyTestBase, R extends BaseTrie> { @@ -111,6 +113,8 @@ abstract void apply(R trie, Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException; abstract void delete(R trie, + ByteComparable deletionPrefix, + TestRangeState partitionMarker, RangeTrie deletion, InMemoryBaseTrie.UpsertTransformer mergeResolver, Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException; @@ -126,6 +130,13 @@ boolean isPartition(TestRangeState c) abstract C mergeMetadata(C c1, C c2); abstract C deleteMetadata(C existing, int entriesCount); + // To overridden by deletion branch testing. + Iterable> getEntrySet(BaseTrie trie) + { + return trie.entrySet(); + } + + abstract void printStats(R trie, Predicate> forcedCopyChecker); @Test @@ -136,7 +147,7 @@ public void testConsistentUpdates() throws Exception // and consistent, i.e. that it is not possible to receive some newer updates while missing // older ones. (For example, if the sequence of additions is 3, 1, 5, without this requirement a reader // could see an enumeration which lists 3 and 5 but not 1.) - testAtomicUpdates(3, FORCE_COPY_PARTITION, FORCE_COPY_PARTITION_RANGE_STATE, true, true); + testUpdateConsistency(3, FORCE_COPY_PARTITION, FORCE_COPY_PARTITION_RANGE_STATE, true, true); // Note: using 3 per mutation, so that the first and second update fit in a sparse in-memory trie block. } @@ -145,14 +156,14 @@ public void testAtomicUpdates() throws Exception { // Check that multi-path updates with below-branching-point copying are safe for concurrent readers, // and that content is atomically applied, i.e. that reader see either nothing from the update or all of it. - testAtomicUpdates(3, forceAtomic(), forceAtomic(), true, false); + testUpdateConsistency(3, forceAtomic(), forceAtomic(), true, false); } @Test public void testSafeUpdates() throws Exception { // Check that multi path updates without additional copying are safe for concurrent readers. - testAtomicUpdates(3, noAtomicity(), noAtomicity(), false, false); + testUpdateConsistency(3, noAtomicity(), noAtomicity(), false, false); } @Test @@ -162,7 +173,7 @@ public void testConsistentSinglePathUpdates() throws Exception // and that content is consistent, i.e. that it is not possible to receive some newer updates while missing // older ones. (For example, if the sequence of additions is 3, 1, 5, without this requirement a reader // could see an enumeration which lists 3 and 5 but not 1.) - testAtomicUpdates(1, FORCE_COPY_PARTITION, FORCE_COPY_PARTITION_RANGE_STATE, true, true); + testUpdateConsistency(1, FORCE_COPY_PARTITION, FORCE_COPY_PARTITION_RANGE_STATE, true, true); } @@ -171,14 +182,14 @@ public void testAtomicSinglePathUpdates() throws Exception { // When doing single path updates atomicity comes for free. This only checks that the branching checker is // not doing anything funny. - testAtomicUpdates(1, forceAtomic(), forceAtomic(), true, false); + testUpdateConsistency(1, forceAtomic(), forceAtomic(), true, false); } @Test public void testSafeSinglePathUpdates() throws Exception { // Check that single path updates without additional copying are safe for concurrent readers. - testAtomicUpdates(1, noAtomicity(), noAtomicity(), true, false); + testUpdateConsistency(1, noAtomicity(), noAtomicity(), true, false); } // The generated keys all start with NEXT_COMPONENT, which makes it impossible to test the precise behavior of the @@ -208,13 +219,37 @@ private static ByteComparable swapTerminator(ByteComparable key, int newTerminat return ByteComparable.preencoded(VERSION, bytes); } - public void testAtomicUpdates(int PER_MUTATION, - Predicate> forcedCopyChecker, - Predicate> forcedCopyCheckerRanges, - boolean checkAtomicity, - boolean checkSequence) + static class ThreadWithProgressAck extends Thread + { + final int threadId; + final LongUnaryOperator ackWriteProgress; + final Consumer runnable; + + ThreadWithProgressAck(AtomicInteger threadIdx, Consumer runnable) + { + threadId = threadIdx.getAndIncrement(); + ackWriteProgress = x -> x | (1<> forcedCopyChecker, + Predicate> forcedCopyCheckerRanges, + boolean checkAtomicity, + boolean checkSequence) throws Exception { + long seed = rand.nextLong(); + System.out.println("Seed: " + seed); + rand.setSeed(seed); + ByteComparable[] ckeys = skipFirst(generateKeys(rand, COUNT)); ByteComparable[] pkeys = skipFirst(generateKeys(rand, Math.min(100, COUNT / 10))); // to guarantee repetition @@ -233,90 +268,80 @@ public void testAtomicUpdates(int PER_MUTATION, AtomicLong writeProgressAck = new AtomicLong(0); AtomicInteger threadIdx = new AtomicInteger(0); - for (int i = 0; i < WALKERS; ++i) - threads.add(new Thread() + Consumer walkTrie = ackWriteProgress -> + { + while (!writeCompleted.get()) { - int threadId = threadIdx.getAndIncrement(); - LongUnaryOperator ackWriteProgress = x -> x | (1<> entries = trie.entrySet(); - checkEntries("", min, true, checkAtomicity, false, PER_MUTATION, entries); - } - } - catch (Throwable t) - { - t.printStackTrace(); - errors.add(t); - } + Iterable> entries = getEntrySet(trie); + checkEntries("", min, true, checkAtomicity, false, PER_MUTATION, entries); } } - }); + catch (Throwable t) + { + t.printStackTrace(); + errors.add(t); + } + } + }; - for (int i = 0; i < READERS; ++i) + Consumer readTrie = ackWriteProgress -> { - ByteComparable[] srcLocal = pkeys; - threads.add(new Thread() + Random r = ThreadLocalRandom.current(); + while (!writeCompleted.get()) { - public void run() + try { - int threadId = threadIdx.getAndIncrement(); - LongUnaryOperator ackWriteProgress = x -> x | (1 << threadId); - - Random r = ThreadLocalRandom.current(); - while (!writeCompleted.get()) { - try + writeProgressAck.getAndUpdate(ackWriteProgress); + ByteComparable key = pkeys[r.nextInt(pkeys.length)]; + int min = writeProgress.get() / (pkeys.length * PER_MUTATION) * PER_MUTATION; + Iterable> entries; + + try (OpOrder.Group group = readOrder.start()) { + var tail = trie.tailTrie(key); + if (tail != null) { - writeProgressAck.getAndUpdate(ackWriteProgress); - ByteComparable key = srcLocal[r.nextInt(srcLocal.length)]; - int min = writeProgress.get() / (pkeys.length * PER_MUTATION) * PER_MUTATION; - Iterable> entries; - - try (OpOrder.Group group = readOrder.start()) - { - var tail = trie.tailTrie(key); - if (tail != null) - { - entries = tail.entrySet(); - checkEntries(" in tail " + key.byteComparableAsString(VERSION), min, false, checkAtomicity, checkSequence, PER_MUTATION, entries); - } - else - Assert.assertEquals("Trie key not found when there should be data for it", 0, min); - } - - try (OpOrder.Group group = readOrder.start()) - { - entries = trie.subtrie(key, key).entrySet(); - checkEntries(" in branch " + key.byteComparableAsString(VERSION), min, true, checkAtomicity, checkSequence, PER_MUTATION, entries); - } + entries = getEntrySet(tail); + checkEntries(" in tail " + key.byteComparableAsString(VERSION), min, false, checkAtomicity, checkSequence, PER_MUTATION, entries); } + else + Assert.assertEquals("Trie key not found when there should be data for it", 0, min); } - catch (Throwable t) + + try (OpOrder.Group group = readOrder.start()) { - t.printStackTrace(); - errors.add(t); + entries = getEntrySet(trie.subtrie(key, key)); + checkEntries(" in branch " + key.byteComparableAsString(VERSION), min, true, checkAtomicity, checkSequence, PER_MUTATION, entries); } } } - }); - } + catch (Throwable t) + { + t.printStackTrace(); + errors.add(t); + } + } + }; + + for (int i = 0; i < WALKERS; ++i) + threads.add(new ThreadWithProgressAck(threadIdx, walkTrie)); + + for (int i = 0; i < READERS; ++i) + threads.add(new ThreadWithProgressAck(threadIdx, readTrie)); + byte[] choices = new byte[COUNT / PER_MUTATION]; + rand.nextBytes(choices); threads.add(new Thread() { public void run() { - ThreadLocalRandom r = ThreadLocalRandom.current(); final Trie.CollectionMergeResolver mergeResolver = new Trie.CollectionMergeResolver() { @Override @@ -332,7 +357,6 @@ public C resolve(Collection contents) return contents.stream().reduce(this::resolve).get(); } }; - BitSet choices = new BitSet(COUNT / PER_MUTATION); try { @@ -343,11 +367,8 @@ public C resolve(Collection contents) ByteComparable b = pkeys[(i / PER_MUTATION) % pkeys.length]; C partitionMarker = metadata(b); ByteComparable cprefix = null; - if (r.nextBoolean()) - { + if ((choices[i / PER_MUTATION] & 1) == 1) cprefix = ckeys[i]; // Also test branching point below the partition level - choices.set(i / PER_MUTATION); - } List sources = new ArrayList<>(); for (int j = 0; j < PER_MUTATION; ++j) @@ -384,6 +405,14 @@ public C resolve(Collection contents) printStats(trie, forcedCopyChecker); Thread.sleep(100); // Let the threads check the completed state too. + // Make sure we can read everything we have inserted from this thread (if this fails, the problem + // is not concurrency). + try (OpOrder.Group group = readOrder.start()) + { + Iterable> entries = getEntrySet(trie); + checkEntries("", COUNT, true, checkAtomicity, false, PER_MUTATION, entries); + } + InMemoryTrie.UpsertTransformer deleteResolver = (existing, update) -> { if (update instanceof TestStateMetadata) @@ -411,7 +440,7 @@ public C resolve(Collection contents) TestRangeState partitionMarker = new TestStateMetadata<>(metadata(b)); List> ranges = new ArrayList<>(); ByteComparable cprefix = null; - if (choices.get(i / PER_MUTATION) && r.nextBoolean()) + if ((choices[i / PER_MUTATION] & 3) == 3) { // Delete the whole branch in one range ranges.add(makeRangeCovering(ckeys[i])); @@ -419,7 +448,7 @@ public C resolve(Collection contents) else { // A range for each entry - if (choices.get(i / PER_MUTATION)) + if ((choices[i / PER_MUTATION] & 1) == 1) cprefix = ckeys[i]; for (int j = 0; j < PER_MUTATION; ++j) ranges.add(makeRangeCovering(ckeys[i + j])); @@ -428,11 +457,11 @@ public C resolve(Collection contents) RangeTrie deletion = RangeTrie.merge(ranges, Trie.throwingResolver()); if (cprefix != null) deletion = deletion.prefixedBy(cprefix); - deletion = TrieUtil.withRootMetadata(deletion, partitionMarker); - deletion = deletion.prefixedBy(b); - delete(trie, deletion, deleteResolver, forcedCopyCheckerRanges); + delete(trie, b, partitionMarker, deletion, deleteResolver, forcedCopyCheckerRanges); } + + writeProgress.set(0); } catch (Throwable t) { @@ -454,8 +483,30 @@ public C resolve(Collection contents) printStats(trie, forcedCopyChecker); + Assert.assertEquals("Writer did not complete", 0, writeProgress.get()); + + assertTrue(Iterables.isEmpty(getEntrySet(trie))); + if (!errors.isEmpty()) + { + System.out.println(trie.dump()); + for (byte b : choices) + switch (b & 3) + { + case 0: + case 2: + System.out.print("."); + break; + case 1: + System.out.print("-"); + break; + case 3: + System.out.print("#"); + break; + } + System.out.println(); Assert.fail("Got errors:\n" + errors); + } } private static RangeTrie makeRangeCovering(ByteComparable cprefix) @@ -513,12 +564,12 @@ public void checkEntries(String location, idMax = seq; } - Assert.assertTrue("Values" + location + " should be at least " + min + ", got " + count, min <= count); + assertTrue("Values" + location + " should be at least " + min + ", got " + count, min <= count); if (checkAtomicity) { // If mutations apply atomically, the row count is always a multiple of the mutation size... - Assert.assertTrue("Values" + location + " should be a multiple of " + PER_MUTATION + ", got " + count, count % PER_MUTATION == 0); + assertTrue("Values" + location + " should be a multiple of " + PER_MUTATION + ", got " + count, count % PER_MUTATION == 0); // ... and the sum of the values is 0 (as the sum for each individual mutation is 0). Assert.assertEquals("Value sum" + location, 0, sum); } @@ -539,6 +590,18 @@ static abstract class TestRangeState implements RangeState static final TestRangeState COVERED = new TestRangeCoveringState(); static final TestRangeState RANGE_START = new TestRangeBoundary(Direction.FORWARD); static final TestRangeState RANGE_END = new TestRangeBoundary(Direction.REVERSE); + + public static TestRangeState combine(TestRangeState existing, TestRangeState incoming) + { + // This can only be called for TestRangeBoundary as other types should not end up in any persisted tries. + TestRangeBoundary be = (TestRangeBoundary) existing; + TestRangeBoundary bi = (TestRangeBoundary) incoming; + if (be == null) + return bi; + if (be.direction == bi.direction) + return be; + return null; // switch from covered to covered, we should not store anything + } } static class TestRangeCoveringState extends TestRangeState diff --git a/test/unit/org/apache/cassandra/db/tries/DataPoint.java b/test/unit/org/apache/cassandra/db/tries/DataPoint.java new file mode 100644 index 000000000000..50b5b018237b --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/DataPoint.java @@ -0,0 +1,295 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.List; +import java.util.function.BiFunction; +import java.util.stream.Collectors; + +import com.google.common.collect.Streams; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +interface DataPoint +{ + static LivePoint combineLive(LivePoint a, LivePoint b) + { + if (a == null) + return b; + if (b == null) + return a; + return LivePoint.combine(a, b); + } + + static DeletionMarker combineDeletion(DeletionMarker a, DeletionMarker b) + { + if (a == null) + return b; + if (b == null) + return a; + return DeletionMarker.combine(a, b); + } + + static LivePoint deleteLive(DeletionMarker deletion, LivePoint live) + { + if (deletion == null || live == null) + return live; + return deletion.applyTo(live); + } + + static LivePoint deleteLive(LivePoint live, DeletionMarker deletion) + { + if (deletion == null || live == null) + return live; + return deletion.applyTo(live); + } + + DeletionMarker marker(); + LivePoint live(); + ByteComparable position(); + + DataPoint withMarker(DeletionMarker newMarker); + DataPoint remap(ByteComparable newKey); + + static String toString(ByteComparable position) + { + if (position == null) + return "null"; + return position.byteComparableAsString(VERSION); + } + + static List verify(List dataPoints) + { + int active = -1; + ByteComparable prev = null; + for (DataPoint dp : dataPoints) + { + DeletionMarker marker = dp.marker(); + if (marker == null) + continue; + assertTrue("Order violation " + toString(prev) + " vs " + toString(marker.position), + prev == null || ByteComparable.compare(prev, marker.position, VERSION) < 0); + assertEquals("Range close violation", active, marker.leftSide); + assertTrue(marker.at != marker.leftSide || marker.at != marker.rightSide); + prev = marker.position; + active = marker.rightSide; + } + assertEquals(-1, active); + return dataPoints; + } + + static DataPoint resolve(LivePoint a, DeletionMarker m) + { + if (a == null) + return m; + if (m == null) + return a; + return new CombinedDataPoint(a, m); + } + + static DataPoint combine(DataPoint a, DataPoint b) + { + LivePoint live = combine(a.live(), b.live(), LivePoint::combine); + DeletionMarker marker = combine(a.marker(), b.marker(), DeletionMarker::combine); + if (marker != null && live != null) + live = marker.applyTo(live); + return resolve(live, marker); + } + + static T combine(T a, T b, BiFunction combiner) + { + if (a == null) + return b; + if (b == null) + return a; + return combiner.apply(a, b); + } + + DataPoint toContent(); + + /** + * Extract the values of the provided trie into a list. + */ + static List toList(DeletionAwareTrie trie) + { + return Streams.stream(trie.mergedTrie(DataPoint::resolve).entryIterator()) + .map(en -> en.getValue().remap(en.getKey())) + .collect(Collectors.toList()); + } + + /** + * Extract the values of the provided trie into a list. + */ + static List contentOnlyList(DeletionAwareTrie trie) + { + return Streams.stream(trie.contentOnlyTrie().entryIterator()) + .map(en -> en.getValue().remap(en.getKey())) + .collect(Collectors.toList()); + } + + /** + * Extract the values of the provided trie into a list. + */ + static List deletionOnlyList(DeletionAwareTrie trie) + { + return Streams.stream(trie.deletionOnlyTrie().entryIterator()) + .map(en -> en.getValue().remap(en.getKey())) + .collect(Collectors.toList()); + } + + static InMemoryDeletionAwareTrie fromList(List list) + { + return fromList(list, false); + } + + static InMemoryDeletionAwareTrie fromList(List list, boolean forceCopy) + { + InMemoryDeletionAwareTrie trie = InMemoryDeletionAwareTrie.shortLived(VERSION); + try + { + // If we put a deletion first, the deletion branch will start at the root which works but isn't interesting + // enough as a test. So put the live data first. + for (DataPoint i : list) + { + LivePoint live = i.live(); + if (live != null) + { + trie.apply( + DeletionAwareTrie.singleton(live.position, VERSION, live), + DataPoint::combineLive, + DataPoint::combineDeletion, + DataPoint::deleteLive, + DataPoint::deleteLive, + false, + v -> forceCopy + ); + } + } + + // If we simply put all deletions with putAlternativeRecursive, we won't get correct branches as they + // won't always close the intervals they open. Deletions need to be put as ranges instead. + int active = -1; + int activeStartedAt = -1; + for (int i = 0; i < list.size(); ++i) + { + DeletionMarker marker = list.get(i).marker(); + if (marker == null || marker.leftSide == marker.rightSide) + continue; + assert marker.leftSide == active; + if (active != -1) + { + if (marker == null || marker.leftSide == marker.rightSide) + continue; + + DeletionMarker startMarker = list.get(activeStartedAt).marker(); + assert startMarker != null; + int prefixLength = ByteComparable.diffPoint(startMarker.position, marker.position, VERSION) - 1; + trie.apply( + DeletionAwareTrie.deletion(ByteComparable.cut(startMarker.position, prefixLength), + ByteComparable.skipFirst(startMarker.position, prefixLength), + ByteComparable.skipFirst(marker.position, prefixLength), + VERSION, marker.leftSideAsCovering), + DataPoint::combineLive, + DataPoint::combineDeletion, + DataPoint::deleteLive, + DataPoint::deleteLive, + false, + v -> forceCopy + ); + } + + active = marker.rightSide; + activeStartedAt = i; + } + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + return trie; + } + + static DeletionAwareTrie fromListToMerge(List list) + { + DeletionAwareTrie trie = DeletionAwareTrie.empty(VERSION); + { + // If we put a deletion first, the deletion branch will start at the root which works but isn't interesting + // enough as a test. So put the live data first. + for (DataPoint i : list) + { + LivePoint live = i.live(); + if (live != null) + { + trie = trie.mergeWith( + DeletionAwareTrie.singleton(live.position, VERSION, live), + LivePoint::combine, + DeletionMarker::combine, + DeletionMarker::applyTo, + false); + } + } + // If we simply put all deletions with putAlternativeRecursive, we won't get correct branches as they + // won't always close the intervals they open. Deletions need to be put as ranges instead. + int active = -1; + int activeStartedAt = -1; + for (int i = 0; i < list.size(); ++i) + { + DeletionMarker marker = list.get(i).marker(); + if (marker == null || marker.leftSide == marker.rightSide) + continue; + assert marker.leftSide == active; + if (active != -1) + { + if (marker == null || marker.leftSide == marker.rightSide) + continue; + + DeletionMarker startMarker = list.get(activeStartedAt).marker(); + assert startMarker != null; + int prefixLength = ByteComparable.diffPoint(startMarker.position, marker.position, VERSION) - 1; + trie = trie.mergeWith( + DeletionAwareTrie.deletion(ByteComparable.cut(startMarker.position, prefixLength), + ByteComparable.skipFirst(startMarker.position, prefixLength), + ByteComparable.skipFirst(marker.position, prefixLength), + VERSION, marker.leftSideAsCovering), + LivePoint::combine, + DeletionMarker::combine, + DeletionMarker::applyTo, + false); + } + + active = marker.rightSide; + activeStartedAt = i; + } + } + return trie; + } + + static DeletionAwareTrie dumpDeletionAwareTrie(DeletionAwareTrie trie) + { + System.out.println("DeletionAware"); + System.out.println(trie.dump()); + System.out.println("Merged"); + System.out.println(trie.mergedTrie(DataPoint::resolve).dump()); + return trie; + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/DeletionAwareIntersectionTest.java b/test/unit/org/apache/cassandra/db/tries/DeletionAwareIntersectionTest.java new file mode 100644 index 000000000000..4594dac569c3 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/DeletionAwareIntersectionTest.java @@ -0,0 +1,303 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Arrays; +import java.util.List; + +import org.junit.Test; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +import static java.util.Arrays.asList; +import static org.apache.cassandra.db.tries.DataPoint.fromList; +import static org.apache.cassandra.db.tries.DataPoint.verify; + +public class DeletionAwareIntersectionTest extends DeletionAwareTestBase +{ + + @Test + public void testSubtrie() + { + for (bits = bitsNeeded; bits > 0; --bits) + { + testIntersection("no intersection"); + + testIntersection("all", + array(null, null)); + testIntersection("fully covered range", + array(before(20), before(25))); + testIntersection("fully covered range", + array(before(25), before(33))); + testIntersection("matching range", + array(before(21), before(24))); + testIntersection("touching empty", + array(before(24), before(26))); + + testIntersection("partial left", + array(before(22), before(25))); + testIntersection("partial left on change", + array(before(28), before(32))); + testIntersection("partial left with null", + array(before(29), null)); + + + testIntersection("partial right", + array(before(25), before(27))); + testIntersection("partial right on change", + array(before(25), before(28))); + testIntersection("partial right with null", + array(null, before(22))); + + testIntersection("inside range", + array(before(22), before(23))); + testIntersection("inside with change", + array(before(27), before(29))); + +// testIntersection("empty range inside", +// array(before(27), before(27))); + + testIntersection("point covered", + array(before(16), before(18))); + testIntersection("point at range start", + array(before(17), before(18))); + testIntersection("point at range end", + array(before(16), before(17))); + + + testIntersection("start point covered", + array(before(32), before(35))); + testIntersection("start point at range start", + array(before(33), before(35))); + testIntersection("start point at range end", + array(before(32), before(33))); + + + testIntersection("end point covered", + array(before(36), before(40))); + testIntersection("end point at range start", + array(before(38), before(40))); + testIntersection("end point at range end", + array(before(36), before(38))); + } + } + + @Test + public void testRanges() + { + for (bits = bitsNeeded; bits > 0; --bits) + { + testIntersection("fully covered ranges", + array(before(20), before(25), before(25), before(33))); + testIntersection("matching ranges", + array(before(21), before(24), before(26), before(31))); + testIntersection("touching empty", + array(before(20), before(21), before(24), before(26), before(32), before(33), before(34), before(36))); + testIntersection("partial left", + array(before(22), before(25), before(29), null)); + + testIntersection("partial right", + array(null, before(22), before(25), before(27))); + + testIntersection("inside ranges", + array(before(22), before(23), before(27), before(29))); + + testIntersection("jumping inside", + array(before(21), before(22), before(23), before(24), before(25), before(26), before(27), before(28), before(29), before(30))); + } + } + + @Test + public void testRangeOnSubtrie() + { + for (bits = bitsNeeded; bits > 0; --bits) + { + // non-overlapping + testIntersection("", array(before(20), before(23)), array(before(24), before(27))); + // touching, i.e. still non-overlapping + testIntersection("", array(before(20), before(23)), array(before(23), before(27))); + // overlapping 1 + testIntersection("", array(before(20), before(23)), array(before(22), before(27))); + // overlapping 2 + testIntersection("", array(before(20), before(23)), array(before(21), before(27))); + // covered + testIntersection("", array(before(20), before(23)), array(before(20), before(27))); + // covered + testIntersection("", array(before(23), before(27)), array(before(20), before(27))); + // covered 2 + testIntersection("", array(before(21), before(23)), array(before(20), before(27))); + } + } + + @Test + public void testRangesOnRanges() + { + for (bits = bitsNeeded; bits > 0; --bits) + testIntersections(); + } + + private List getTestRanges() + { + return flatten(asList(deletedPoint(17, 20), + livePoint(19, 30), + from(21, 10), deletedPointInside(22, 21, 10), livePoint(23, 31), to(24, 10), + from(26, 11), livePoint(27, 32), change(28, 11, 12).withPoint(22), livePoint(29, 33), to(30, 12), + livePoint(32, 34), from(33, 13).withPoint(23), to(34, 13), + from(36, 14), to(38, 14).withPoint(24), livePoint(39, 35))); + } + + private DeletionAwareTrie mergeGeneratedRanges() + { + return fromList(asList(from(21, 10), to(24, 10), + from(26, 11), to(29, 11), + from(33, 13), to(34, 13), + from(36, 14), to(38, 14))) + .mergeWith(fromList(asList(from(28, 12), to(30, 12))), + LivePoint::combine, + DeletionMarker::combine, + DeletionMarker::applyTo, false) + .mergeWith(fromList(flatten(asList(deletedPoint(17, 20), + deletedPoint(22, 21), + deletedPoint(28, 22), + deletedPoint(33, 23), + deletedPoint(38, 24)))), + LivePoint::combine, + DeletionMarker::combine, + DeletionMarker::applyTo, false) + .mergeWith(fromList(asList(livePoint(19, 30), + livePoint(23, 31), + livePoint(27, 32), + livePoint(29, 33), + livePoint(32, 34), + livePoint(39, 35))), + LivePoint::combine, + DeletionMarker::combine, + DeletionMarker::applyTo, false) + ; + } + + private DeletionAwareTrie collectionMergeGeneratedRanges() + { + return DeletionAwareTrie.merge(asList( + fromList(asList(from(21, 10), to(24, 10), + from(26, 11), to(29, 11), + from(33, 13), to(34, 13), + from(36, 14), to(38, 14))), + fromList(asList(from(28, 12), to(30, 12))), + fromList(flatten(asList(deletedPoint(17, 20), + deletedPoint(22, 21), + deletedPoint(28, 22), + deletedPoint(33, 23), + deletedPoint(38, 24)))), + fromList(asList(livePoint(19, 30), + livePoint(23, 31), + livePoint(27, 32), + livePoint(29, 33), + livePoint(32, 34), + livePoint(39, 35))) + ), + LivePoint::combineCollection, + DeletionMarker::combineCollection, + DeletionMarker::applyTo, + false); + } + + private void testIntersections() + { + testIntersection(""); + + ByteComparable[] set1 = array(null, before(24), + before(25), before(29), + before(32), null); + ByteComparable[] set2 = array(before(14), before(17), + before(22), before(27), + before(28), before(30), + before(32), before(34), + before(36), before(40)); + ByteComparable[] set3 = array(before(17), before(18), + before(19), before(20), + before(21), before(22), + before(23), before(24), + before(25), before(26), + before(27), before(28), + before(29), before(30), + before(31), before(32), + before(33), before(34), + before(35), before(36), + before(37), before(38)); + + testIntersections(set1, set2, set3); + } + + private void testIntersections(ByteComparable[] set1, ByteComparable[] set2, ByteComparable[] set3) + { + testIntersection("1", set1); + + testIntersection("2", set2); + + testIntersection("3", set3); + + testIntersection("12", set1, set2); + + testIntersection("13", set1, set3); + + testIntersection("23", set2, set3); + + testIntersection("123", set1, set2, set3); + } + + public void testIntersection(String message, ByteComparable[]... sets) + { + final List testRanges = getTestRanges(); + testIntersection(message, fromList(testRanges), testRanges, sets); + testIntersection(message + " on merge ", mergeGeneratedRanges(), testRanges, sets); // Mainly tests MergeCursor's skipTo + testIntersection(message + " on collection merge ", collectionMergeGeneratedRanges(), testRanges, sets); // Mainly tests MergeCursor's skipTo + } + + public void testIntersection(String message, DeletionAwareTrie trie, List intersected, ByteComparable[]... sets) + { + if (VERBOSE) + { + System.out.println("Markers: " + intersected); + DataPoint.dumpDeletionAwareTrie(trie); + } + verify(intersected); + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + assertDeletionAwareEqual(message + " forward b" + bits, intersected, trie); + } + else + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + ByteComparable[] ranges = sets[toRemove]; + System.out.println("Ranges: " + toString(ranges)); + testIntersection(message + " " + toRemove, + trie.intersect(TrieSet.ranges(TrieUtil.VERSION, ranges)), + intersect(intersected, ranges), + Arrays.stream(sets) + .filter(x -> x != ranges) + .toArray(ByteComparable[][]::new) + ); + } + } + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/DeletionAwareMergeTest.java b/test/unit/org/apache/cassandra/db/tries/DeletionAwareMergeTest.java new file mode 100644 index 000000000000..5f23a5c27a5f --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/DeletionAwareMergeTest.java @@ -0,0 +1,583 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +import com.google.common.collect.Lists; +import org.junit.Test; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +import static java.util.Arrays.asList; +import static org.apache.cassandra.db.tries.DataPoint.dumpDeletionAwareTrie; +import static org.apache.cassandra.db.tries.DataPoint.fromList; +import static org.apache.cassandra.db.tries.DataPoint.verify; +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; + +public class DeletionAwareMergeTest extends DeletionAwareTestBase +{ + int deletionPoint = 100; + + private List deletedRanges(ByteComparable... dataPoints) + { + List data = new ArrayList<>(asList(dataPoints)); + invertDataRangeList(data); + filterOutEmptyRepetitions(data); + + List markers = new ArrayList<>(); + for (int i = 0; i < data.size(); ++i) + { + ByteComparable pos = data.get(i); + if (pos == null) + pos = i % 2 == 0 ? before(0) : before((1< data) + { + // invert list + if (data.get(0) != null) + data.add(0, null); + else + data.remove(0); + if (data.get(data.size() - 1) != null) + data.add(null); + else + data.remove(data.size() - 1); + } + + private static void filterOutEmptyRepetitions(List data) + { + for (int i = 0; i < data.size() - 1; ++i) + { + if (data.get(i) != null && data.get(i + 1) != null && + ByteComparable.compare(data.get(i), data.get(i + 1), VERSION) == 0) + { + data.remove(i + 1); + data.remove(i); + --i; + } + } + } + + @Test + public void testSubtrie() + { + for (bits = bitsNeeded; bits > 0; --bits) + for (deletionPoint = 4; deletionPoint <= 40; deletionPoint += 9) + { + testMerge("no merge"); + + testMerge("all", + deletedRanges(null, null)); + testMerge("fully covered range", + deletedRanges(before(20), before(25))); + testMerge("fully covered range", + deletedRanges(before(25), before(33))); + testMerge("matching range", + deletedRanges(before(21), before(24))); + testMerge("touching empty", + deletedRanges(before(24), before(26))); + + testMerge("partial left", + deletedRanges(before(22), before(25))); + testMerge("partial left on change", + deletedRanges(before(28), before(32))); + testMerge("partial left with null", + deletedRanges(before(29), null)); + + + testMerge("partial right", + deletedRanges(before(25), before(27))); + testMerge("partial right on change", + deletedRanges(before(25), before(28))); + testMerge("partial right with null", + deletedRanges(null, before(22))); + + testMerge("inside range", + deletedRanges(before(22), before(23))); + testMerge("inside with change", + deletedRanges(before(27), before(29))); + + testMerge("empty range inside", + deletedRanges(before(27), before(27))); + + testMerge("point covered", + deletedRanges(before(16), before(18))); + testMerge("point at range start", + deletedRanges(before(17), before(18))); + testMerge("point at range end", + deletedRanges(before(16), before(17))); + + + testMerge("start point covered", + deletedRanges(before(32), before(35))); + testMerge("start point at range start", + deletedRanges(before(33), before(35))); + testMerge("start point at range end", + deletedRanges(before(32), before(33))); + + + testMerge("end point covered", + deletedRanges(before(36), before(40))); + testMerge("end point at range start", + deletedRanges(before(38), before(40))); + testMerge("end point at range end", + deletedRanges(before(36), before(38))); + } + } + + @Test + public void testRanges() + { + for (bits = bitsNeeded; bits > 0; --bits) + for (deletionPoint = 4; deletionPoint <= 40; deletionPoint += 9) + { + testMerge("fully covered ranges", + deletedRanges(before(20), before(25), before(25), before(33))); + testMerge("matching ranges", + deletedRanges(before(21), before(24), before(26), before(31))); + testMerge("touching empty", + deletedRanges(before(20), before(21), before(24), before(26), before(32), before(33), before(34), before(36))); + testMerge("partial left", + deletedRanges(before(22), before(25), before(29), null)); + + testMerge("partial right", + deletedRanges(null, before(22), before(25), before(27))); + + testMerge("inside ranges", + deletedRanges(before(22), before(23), before(27), before(29))); + + testMerge("jumping inside", + deletedRanges(before(21), before(22), before(23), before(24), before(25), before(26), before(27), before(28), before(29), before(30))); + } + } + + @Test + public void testRangeOnSubtrie() + { + for (bits = bitsNeeded; bits > 0; --bits) + for (deletionPoint = 4; deletionPoint <= 40; deletionPoint += 9) + { + // non-overlapping + testMerge("non-overlapping", deletedRanges(before(20), before(23)), deletedRanges(before(24), before(27))); + // touching, i.e. still non-overlapping + testMerge("touching", deletedRanges(before(20), before(23)), deletedRanges(before(23), before(27))); + // overlapping 1 + testMerge("overlapping1", deletedRanges(before(20), before(23)), deletedRanges(before(22), before(27))); + // overlapping 2 + testMerge("overlapping2", deletedRanges(before(20), before(23)), deletedRanges(before(21), before(27))); + // covered + testMerge("covered1", deletedRanges(before(20), before(23)), deletedRanges(before(20), before(27))); + // covered 2 + testMerge("covered2", deletedRanges(before(23), before(27)), deletedRanges(before(20), before(27))); + // covered 3 + testMerge("covered3", deletedRanges(before(21), before(23)), deletedRanges(before(20), before(27))); + } + } + + @Test + public void testRangesOnRanges() + { + for (bits = bitsNeeded; bits > 0; --bits) + for (deletionPoint = 4; deletionPoint <= 40; deletionPoint += 9) + testMerges(); + } + + private List getTestRanges() + { + return flatten(asList(deletedPoint(17, 20), + livePoint(19, 30), + from(21, 10), deletedPointInside(22, 21, 10), livePoint(23, 31), to(24, 10), + from(26, 11), livePoint(27, 32), change(28, 11, 12).withPoint(22), livePoint(29, 33), to(30, 12), + livePoint(32, 34), from(33, 13).withPoint(23), to(34, 13), + from(36, 14), to(38, 14).withPoint(24), livePoint(39, 35))); + } + + private void testMerges() + { + testMergeWith("", fromList(getTestRanges()), getTestRanges()); + + List set1 = deletedRanges(null, before(24), before(25), before(29), before(32), null); + List set2 = deletedRanges(before(14), before(17), + before(22), before(27), + before(28), before(30), + before(32), before(34), + before(36), before(40)); + List set3 = deletedRanges(before(17), before(18), + before(19), before(20), + before(21), before(22), + before(23), before(24), + before(25), before(26), + before(27), before(28), + before(29), before(30), + before(31), before(32), + before(33), before(34), + before(35), before(36), + before(37), before(38)); + + testMerges(set1, set2, set3); + } + + private void testMerges(List set1, List set2, List set3) + { + testMerge("1", set1); + + testMerge("2", set2); + + testMerge("3", set3); + + testMerge("12", set1, set2); + + testMerge("13", set1, set3); + + testMerge("23", set2, set3); + + testMerge("123", set1, set2, set3); + } + + @SafeVarargs + public final void testMerge(String message, List... sets) + { + List testRanges = getTestRanges(); + testMergeWith(message, fromList(testRanges), testRanges, sets); + testCollectionMerge(message + " collection", Lists.newArrayList(fromList(testRanges)), testRanges, sets); + testMergeInMemoryTrie(message + " inmem.apply", fromList(testRanges), testRanges, sets); + testMergeInMemoryTrieIntoSet(message + " inmem.apply into set", fromList(testRanges), testRanges, sets); + } + + + public void testMergeWith(String message, DeletionAwareTrie trie, List merged, List... sets) + { + if (VERBOSE) + { + System.out.println("Markers: " + merged); + dumpDeletionAwareTrie(trie); + } + verify(merged); + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + assertDeletionAwareEqual(message + " forward b" + bits, merged, trie); + } + else + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + List ranges = sets[toRemove]; + InMemoryDeletionAwareTrie adding = fromList(ranges); + if (VERBOSE) + { + System.out.println("Adding: " + ranges); + dumpDeletionAwareTrie(adding); + } + testMergeWith(message + " " + toRemove, + trie.mergeWith(adding, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, false), + mergeLists(merged, ranges), + Arrays.stream(sets) + .filter(x -> x != ranges) + .toArray(List[]::new) + ); + } + } + } + + public void testCollectionMerge(String message, List> triesToMerge, List merged, List... sets) + { + if (VERBOSE) + System.out.println("Markers: " + merged); + verify(merged); + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + if (VERBOSE) + { + System.out.println("Sources:"); + triesToMerge.forEach(DataPoint::dumpDeletionAwareTrie); + } + + DeletionAwareTrie trie = DeletionAwareTrie.merge(triesToMerge, + LivePoint::combineCollection, + DeletionMarker::combineCollection, + DeletionMarker::applyTo, + false); + if (VERBOSE) + { + System.out.println("Result:"); + dumpDeletionAwareTrie(trie); + } + + assertDeletionAwareEqual(message + " forward b" + bits, merged, trie); + } + else + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + List ranges = sets[toRemove]; + if (VERBOSE) + System.out.println("Adding: " + ranges); + triesToMerge.add(fromList(ranges)); + testCollectionMerge(message + " " + toRemove, + triesToMerge, + mergeLists(merged, ranges), + Arrays.stream(sets) + .filter(x -> x != ranges) + .toArray(List[]::new) + ); + triesToMerge.remove(triesToMerge.size() - 1); + } + } + } + + public void testMergeInMemoryTrie(String message, DeletionAwareTrie trie, List merged, List... sets) + { + if (VERBOSE) + { + System.out.println("Markers: " + merged); + dumpDeletionAwareTrie(trie); + } + verify(merged); + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + assertDeletionAwareEqual(message + " forward b" + bits, merged, trie); + } + else + { + try + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + List ranges = sets[toRemove]; + InMemoryDeletionAwareTrie adding = fromList(ranges); + if (VERBOSE) + { + System.out.println("Adding: " + ranges); + dumpDeletionAwareTrie(adding); + } + var dupe = duplicateTrie(trie); + dupe.apply(adding, + DataPoint::combineLive, + DataPoint::combineDeletion, + DataPoint::deleteLive, + DataPoint::deleteLive, + false, + v -> false); + testMergeInMemoryTrie(message + " " + toRemove, + dupe, + mergeLists(merged, ranges), + Arrays.stream(sets) + .filter(x -> x != ranges) + .toArray(List[]::new) + ); + } + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + } + } + + public void testMergeInMemoryTrieIntoSet(String message, DeletionAwareTrie trie, List merged, List... sets) + { + if (VERBOSE) + { + System.out.println("Markers: " + merged); + dumpDeletionAwareTrie(trie); + } + verify(merged); + // Test that intersecting the given trie with the given sets, in any order, results in the expected list. + // Checks both forward and reverse iteration direction. + if (sets.length == 0) + { + assertDeletionAwareEqual(message + " forward b" + bits, merged, trie); + } + else + { + try + { + for (int toRemove = 0; toRemove < sets.length; ++toRemove) + { + List ranges = sets[toRemove]; + var set = fromList(ranges); + if (VERBOSE) + { + System.out.println("Adding: " + ranges); + dumpDeletionAwareTrie(set); + } + set.apply(trie, + DataPoint::combineLive, + DataPoint::combineDeletion, + DataPoint::deleteLive, + DataPoint::deleteLive, + false, + v -> false); + testMergeInMemoryTrieIntoSet(message + " " + toRemove, + set, + mergeLists(merged, ranges), + Arrays.stream(sets) + .filter(x -> x != ranges) + .toArray(List[]::new) + ); + } + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + } + } + + InMemoryDeletionAwareTrie duplicateTrie(DeletionAwareTrie trie) + { + try + { + InMemoryDeletionAwareTrie copy = InMemoryDeletionAwareTrie.shortLived(VERSION); + copy.apply(trie, + DataPoint::combineLive, + DataPoint::combineDeletion, + DataPoint::deleteLive, + DataPoint::deleteLive, + false, + v -> false); + return copy; + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + } + + DeletionMarker delete(int deletionTime, DeletionMarker marker) + { + if (deletionTime < 0 || marker == null) + return marker; + + int newLeft = Math.max(deletionTime, marker.leftSide); + int newAt = Math.max(deletionTime, marker.at); + int newRight = Math.max(deletionTime, marker.rightSide); + if (newLeft < 0 && newAt < 0 && newRight < 0 || newAt == newLeft && newLeft == newRight) + return null; + if (newLeft == marker.leftSide && newAt == marker.at && newRight == marker.rightSide) + return marker; + return new DeletionMarker(marker.position, newLeft, newAt, newRight); + } + + LivePoint delete(int deletionTime, LivePoint marker) + { + if (deletionTime < 0 || marker == null) + return marker; + return marker.delete(deletionTime); + } + + DataPoint delete(int deletionTime, DataPoint marker) + { + LivePoint live = delete(deletionTime, marker.live()); + DeletionMarker deletion = delete(deletionTime, marker.marker()); + return DataPoint.resolve(live, deletion); + } + + int leftSide(DataPoint point) + { + if (point.marker() == null) + return -1; + return point.marker().leftSide; + } + + int rightSide(DataPoint point) + { + if (point.marker() == null) + return -1; + return point.marker().rightSide; + } + + List mergeLists(List left, List right) + { + int active = -1; + Iterator rightIt = right.iterator(); + DataPoint nextRight = rightIt.hasNext() ? rightIt.next() : null; + List result = new ArrayList<>(); + for (DataPoint nextLeft : left) + { + while (true) + { + int cmp; + if (nextRight == null) + cmp = -1; + else + cmp = ByteComparable.compare(nextLeft.position(), nextRight.position(), VERSION); + + if (cmp < 0) + { + maybeAdd(result, nextRight != null ? delete(leftSide(nextRight), nextLeft) : nextLeft); + break; + } + + if (cmp == 0) + { + if (nextLeft.marker() == null) + nextRight = delete(active, nextRight); + if (nextRight != null) + maybeAdd(result, DataPoint.combine(nextRight, nextLeft).toContent()); + else + maybeAdd(result, nextLeft); + + nextRight = rightIt.hasNext() ? rightIt.next() : null; + break; + } + else + { + // Must close active if it becomes covered, and must open active if it is no longer covered. + maybeAdd(result, delete(active, nextRight)); + } + + nextRight = rightIt.hasNext() ? rightIt.next() : null; + } + if (nextLeft.marker() != null) + active = nextLeft.marker().rightSide; + } + assert active == -1; + while (nextRight != null) + { + maybeAdd(result, delete(active, nextRight));// deletion is not needed (active == -1), do just in case + nextRight = rightIt.hasNext() ? rightIt.next() : null; + } + return result; + } + + static void maybeAdd(List list, T value) + { + if (value == null) + return; + list.add(value); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/DeletionAwareRandomizedTest.java b/test/unit/org/apache/cassandra/db/tries/DeletionAwareRandomizedTest.java new file mode 100644 index 000000000000..11f2e078354d --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/DeletionAwareRandomizedTest.java @@ -0,0 +1,579 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.quicktheories.core.Gen; + +import static java.util.Arrays.asList; +import static org.apache.cassandra.db.tries.DataPoint.fromList; +import static org.apache.cassandra.db.tries.DataPoint.toList; +import static org.apache.cassandra.db.tries.DataPoint.verify; +import static org.junit.Assert.assertEquals; +import static org.quicktheories.QuickTheory.qt; +import static org.quicktheories.generators.SourceDSL.booleans; +import static org.quicktheories.generators.SourceDSL.integers; +import static org.quicktheories.generators.SourceDSL.lists; + +/// +/// Randomized property-based testing for deletion-aware tries using QuickTheories. +/// +/// This test class uses the existing deletion-aware trie test infrastructure to perform +/// comprehensive randomized testing of trie operations, merging, and deletion handling. +/// It complements the structured tests in `DeletionAwareMergeTest` with property-based +/// testing to catch edge cases and verify invariants across a wide range of inputs. +/// +public class DeletionAwareRandomizedTest extends DeletionAwareTestBase +{ + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + + private static final int MAX_POINTS = 20; + private static final int MAX_VALUE = 63; // Fits in 6 bits (bitsNeeded) + private static final int MAX_TIMESTAMP = 100; + + /// + /// Generator for random live data points. + /// Creates `LivePoint` instances with random positions and timestamps. + /// + private Gen livePointGen() + { + return integers().between(0, MAX_VALUE) + .zip(integers().between(1, MAX_TIMESTAMP), + (pos, ts) -> new LivePoint(at(pos), ts)); + } + + /// + /// Generator for random deletion markers. + /// Creates `DeletionMarker` instances with random positions and deletion values. + /// + private Gen deletionMarkerGen() + { + return integers().between(0, MAX_VALUE) + .zip(integers().between(1, MAX_TIMESTAMP), + integers().between(-1, MAX_TIMESTAMP), + integers().between(-1, MAX_TIMESTAMP), + (pos, left, at, right) -> new DeletionMarker(before(pos), left, at, right)); + } + + /// + /// Generator for random live point lists. + /// Creates sorted lists of `LivePoint` instances for trie construction. + /// + private Gen> dataPointListGen() + { + return lists().of(livePointGen().map(lp -> (DataPoint) lp)) + .ofSizeBetween(0, MAX_POINTS) + .map(this::sortAndValidateDataPoints); + } + + /** + * Generator for pairs of data point lists for merge testing. + */ + private Gen>> dataPointPairGen() + { + return dataPointListGen().zip(dataPointListGen(), Arrays::asList); + } + + /// + /// Sorts data points by position and ensures they form a valid deletion-aware sequence. + /// + private List sortAndValidateDataPoints(List points) + { + if (points.isEmpty()) + return points; + + // Sort by position + List sorted = points.stream() + .sorted((a, b) -> ByteComparable.compare(a.position(), b.position(), TrieUtil.VERSION)) + .collect(Collectors.toList()); + + // Remove duplicates at same position, keeping the last one + List deduplicated = new ArrayList<>(); + DataPoint prev = null; + for (DataPoint dp : sorted) + { + if (prev == null || ByteComparable.compare(prev.position(), dp.position(), TrieUtil.VERSION) != 0) + deduplicated.add(dp); + else + deduplicated.set(deduplicated.size() - 1, dp); // Replace with newer + prev = dp; + } + + try + { + return verify(deduplicated); + } + catch (AssertionError e) + { + // If verification fails, return empty list to avoid invalid test data + return Collections.emptyList(); + } + } + + /// + /// Test that trie construction and iteration are consistent. + /// + /// **Property**: Converting a list to a trie and back should yield the same list. + /// + @Test + public void testTrieConstructionConsistency() + { + qt().forAll(dataPointListGen(), booleans().all()) + .checkAssert((dataPoints, forceCopy) -> { + if (dataPoints.isEmpty()) + return; // Skip empty lists + + DeletionAwareTrie trie = fromList(dataPoints, forceCopy); + List reconstructed = toList(trie); + + assertEquals("Trie construction should be consistent with iteration", + dataPoints, reconstructed); + }); + } + + /// + /// Test that merging tries is commutative for the same resolver. + /// + /// **Property**: `merge(A, B)` should equal `merge(B, A)` when using the same merge resolver. + /// + @Test + public void testMergeCommutativity() + { + qt().forAll(dataPointPairGen(), booleans().all()) + .checkAssert((pair, forcedCopy) -> { + List list1 = pair.get(0); + List list2 = pair.get(1); + + if (list1.isEmpty() && list2.isEmpty()) + return; // Skip empty merges + + DeletionAwareTrie trie1 = fromList(list1, forcedCopy); + DeletionAwareTrie trie2 = fromList(list2, forcedCopy); + + // Merge in both directions + DeletionAwareTrie merged1to2 = + trie1.mergeWith(trie2, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, false); + DeletionAwareTrie merged2to1 = + trie2.mergeWith(trie1, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, false); + + List result1 = toList(merged1to2); + List result2 = toList(merged2to1); + + assertEquals("Merge should be commutative", result1, result2); + }); + } + + /// + /// Test that merging with an empty trie is an identity operation. + /// + /// **Property**: `merge(A, empty)` should equal `A`. + /// + @Test + public void testMergeIdentity() + { + qt().forAll(dataPointListGen(), booleans().all()) + .checkAssert((dataPoints, forcedCopy) -> { + if (dataPoints.isEmpty()) + return; // Skip empty lists + + DeletionAwareTrie trie = fromList(dataPoints, forcedCopy); + DeletionAwareTrie empty = fromList(Collections.emptyList(), forcedCopy); + + DeletionAwareTrie merged = + trie.mergeWith(empty, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, false); + + List original = toList(trie); + List result = toList(merged); + + assertEquals("Merging with empty trie should be identity", original, result); + }); + } + + /// + /// Test that subtrie operations preserve ordering and boundaries. + /// + /// **Property**: A subtrie should contain only elements within the specified range. + /// + @Test + public void testSubtrieRangeInvariant() + { + qt().forAll(dataPointListGen() + .zip(integers().between(0, MAX_VALUE), + integers().between(0, MAX_VALUE), + (points, start, end) -> { + int left = Math.min(start, end); + int right = Math.max(start, end); + return asList(points, left, right); + }), booleans().all()) + .checkAssert((params, forcedCopy) -> { + @SuppressWarnings("unchecked") + List dataPoints = (List) params.get(0); + int left = (Integer) params.get(1); + int right = (Integer) params.get(2); + + if (dataPoints.isEmpty()) + return; // Skip empty lists + + DeletionAwareTrie trie = fromList(dataPoints, forcedCopy); + DeletionAwareTrie subtrie = + trie.subtrie(before(left), before(right)); + + List subtriePoints = toList(subtrie); + + // Verify all points in subtrie are within range + for (DataPoint dp : subtriePoints) + { + ByteComparable pos = dp.position(); + int cmp1 = ByteComparable.compare(pos, before(left), TrieUtil.VERSION); + int cmp2 = ByteComparable.compare(pos, before(right), TrieUtil.VERSION); + + if (cmp1 < 0 || cmp2 >= 0) + { + throw new AssertionError( + String.format("Point %s outside subtrie range [%s, %s)", + pos.byteComparableAsString(TrieUtil.VERSION), + before(left).byteComparableAsString(TrieUtil.VERSION), + before(right).byteComparableAsString(TrieUtil.VERSION))); + } + } + }); + } + + /// + /// Test that the optimized `MergeCursor` produces the same results as the safe version. + /// + /// **Property**: Optimized and safe merge cursors should produce identical results. + /// + @Test + public void testOptimizedMergeCursorEquivalence() + { + qt().forAll(dataPointPairGen(), booleans().all()) + .checkAssert((pair, forcedCopy) -> { + List list1 = pair.get(0); + List list2 = pair.get(1); + + if (list1.isEmpty() || list2.isEmpty()) + return; // Skip cases with empty lists + + DeletionAwareTrie trie1 = fromList(list1, forcedCopy); + DeletionAwareTrie trie2 = fromList(list2, forcedCopy); + + // Test both optimized and safe merge cursors + DeletionAwareCursor cursor1 = trie1.cursor(Direction.FORWARD); + DeletionAwareCursor cursor2 = trie2.cursor(Direction.FORWARD); + + // Test both optimized and safe merge using the trie API + DeletionAwareTrie safeMerge = + trie1.mergeWith(trie2, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, false); + + // Create optimized merge + DeletionAwareTrie optimizedMerge = + trie1.mergeWith(trie2, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, true); + + List safeResult = toList(safeMerge); + List optimizedResult = toList(optimizedMerge); + + assertEquals("Optimized and safe merge cursors should produce identical results", + safeResult, optimizedResult); + }); + } + + /// + /// Test that deletion markers properly delete live data within their ranges. + /// + /// **Property**: Live data covered by deletion ranges should be removed or modified. + /// + @Test + public void testDeletionApplicationInvariant() + { + qt().forAll(integers().between(0, MAX_VALUE) + .zip(integers().between(1, MAX_TIMESTAMP), + integers().between(1, MAX_TIMESTAMP), + (pos, liveTs, deleteTs) -> asList(pos, liveTs, deleteTs))) + .checkAssert(params -> { + int pos = (Integer) params.get(0); + int liveTs = (Integer) params.get(1); + int deleteTs = (Integer) params.get(2); + + // Create a live point and a deletion that should affect it + LivePoint live = new LivePoint(at(pos), liveTs); + DeletionMarker deletion = new DeletionMarker(before(pos), -1, deleteTs, deleteTs); + + // Apply deletion to live data + LivePoint result = deletion.applyTo(live); + + if (deleteTs > liveTs) + { + // Deletion should remove the live data (return null) + assertEquals("Live data should be deleted when deletion timestamp > live timestamp", + null, result); + } + else + { + // Deletion should not affect live data with same or newer timestamp + assertEquals("Live data should not be affected by older or equal deletions", live, result); + } + }); + } + + /// + /// Test that trie operations maintain structural invariants. + /// + /// **Property**: All trie operations should preserve the trie's structural integrity. + /// + @Test + public void testTrieStructuralInvariants() + { + qt().forAll(dataPointListGen(), booleans().all()) + .checkAssert((dataPoints, forcedCopy) -> { + if (dataPoints.isEmpty()) + return; // Skip empty lists + + DeletionAwareTrie trie = fromList(dataPoints, forcedCopy); + + // Test that trie construction is consistent + List reconstructed = toList(trie); + + // Verify that the reconstructed list maintains ordering + for (int i = 1; i < reconstructed.size(); i++) + { + ByteComparable prev = reconstructed.get(i - 1).position(); + ByteComparable curr = reconstructed.get(i).position(); + int cmp = ByteComparable.compare(prev, curr, TrieUtil.VERSION); + + if (cmp > 0) + { + throw new AssertionError( + String.format("Trie ordering violation: %s > %s", + prev.byteComparableAsString(TrieUtil.VERSION), + curr.byteComparableAsString(TrieUtil.VERSION))); + } + } + }); + } + + /// + /// Test that range operations are consistent with full trie operations. + /// + /// **Property**: Operating on a range should be equivalent to filtering the full result. + /// + @Test + public void testRangeOperationConsistency() + { + qt().forAll(dataPointListGen() + .zip(integers().between(0, MAX_VALUE), + integers().between(0, MAX_VALUE), + (points, start, end) -> { + int left = Math.min(start, end); + int right = Math.max(start, end); + return asList(points, left, right); + }), + booleans().all()) + .checkAssert((params, forcedCopy) -> { + @SuppressWarnings("unchecked") + List dataPoints = (List) params.get(0); + int left = (Integer) params.get(1); + int right = (Integer) params.get(2); + + if (dataPoints.isEmpty()) + return; // Skip empty lists + + DeletionAwareTrie trie = fromList(dataPoints, forcedCopy); + + // Get subtrie result + DeletionAwareTrie subtrie = + trie.subtrie(before(left), before(right)); + List subtrieResult = toList(subtrie); + + // Get filtered full result + List fullResult = toList(trie); + List filteredResult = fullResult.stream() + .filter(dp -> { + ByteComparable pos = dp.position(); + int cmp1 = ByteComparable.compare(pos, before(left), TrieUtil.VERSION); + int cmp2 = ByteComparable.compare(pos, before(right), TrieUtil.VERSION); + return cmp1 >= 0 && cmp2 < 0; + }) + .collect(Collectors.toList()); + + assertEquals("Subtrie should be equivalent to filtering full trie", + filteredResult, subtrieResult); + }); + } + + /// + /// Test that merge operations are associative. + /// + /// **Property**: `merge(merge(A, B), C)` should equal `merge(A, merge(B, C))`. + /// + @Test + public void testMergeAssociativity() + { + qt().forAll(dataPointListGen() + .zip(dataPointListGen(), dataPointListGen(), Arrays::asList), + booleans().all()) + .checkAssert((triple, forcedCopy) -> { + List list1 = triple.get(0); + List list2 = triple.get(1); + List list3 = triple.get(2); + + if (list1.isEmpty() && list2.isEmpty() && list3.isEmpty()) + return; // Skip all empty + + DeletionAwareTrie trie1 = fromList(list1, forcedCopy); + DeletionAwareTrie trie2 = fromList(list2, forcedCopy); + DeletionAwareTrie trie3 = fromList(list3); + + // Test (A merge B, forcedCopy) merge C + DeletionAwareTrie ab = + trie1.mergeWith(trie2, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, false); + DeletionAwareTrie ab_c = + ab.mergeWith(trie3, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, false); + + // Test A merge (B merge C) + DeletionAwareTrie bc = + trie2.mergeWith(trie3, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, false); + DeletionAwareTrie a_bc = + trie1.mergeWith(bc, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, false); + + List result1 = toList(ab_c); + List result2 = toList(a_bc); + + assertEquals("Merge should be associative", result1, result2); + }); + } + + + /// + /// Test collection merge functionality using randomized property-based testing. + /// This test verifies that merging multiple tries using collection merge produces + /// the same result as sequential pairwise merges. + /// + @Test + public void testCollectionMerge() + { + qt().forAll(dataPointListGen().zip(dataPointListGen(), dataPointListGen(), Arrays::asList), booleans().all()) + .checkAssert((triple, forcedCopy) -> { + List list1 = triple.get(0); + List list2 = triple.get(1); + List list3 = triple.get(2); + + // Skip cases where all tries are empty or where we have empty tries mixed with non-empty ones + // Collection merge requires at least one non-empty trie and can't handle mixed empty/non-empty + boolean hasEmpty = list1.isEmpty() || list2.isEmpty() || list3.isEmpty(); + boolean hasNonEmpty = !list1.isEmpty() || !list2.isEmpty() || !list3.isEmpty(); + + if (!hasNonEmpty || hasEmpty) + return; // Skip if all empty or if we have any empty tries + + DeletionAwareTrie trie1 = fromList(list1, forcedCopy); + DeletionAwareTrie trie2 = fromList(list2, forcedCopy); + DeletionAwareTrie trie3 = fromList(list3, forcedCopy); + + // Test collection merge + DeletionAwareTrie collectionMerged = + DeletionAwareTrie.merge(Arrays.asList(trie1, trie2, trie3), + LivePoint::combineCollection, + DeletionMarker::combineCollection, + DeletionMarker::applyTo, + false); + + List collectionResult = toList(collectionMerged); + + // Test pairwise merge for comparison + DeletionAwareTrie pairwise12 = + trie1.mergeWith(trie2, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, false); + DeletionAwareTrie pairwiseMerged = + pairwise12.mergeWith(trie3, LivePoint::combine, DeletionMarker::combine, DeletionMarker::applyTo, false); + + List pairwiseResult = toList(pairwiseMerged); + + assertEquals("Collection merge should equal pairwise merge", pairwiseResult, collectionResult); + }); + } + + /// + /// Test that the optimized collection merge produces the same results as the safe version. + /// This verifies that the deletionsAtFixedPoints optimization works correctly for collection merges. + /// + @Test + public void testOptimizedCollectionMerge() + { + qt().forAll(dataPointListGen().zip(dataPointListGen(), dataPointListGen(), Arrays::asList), booleans().all()) + .checkAssert((triple, forcedCopy) -> { + List list1 = triple.get(0); + List list2 = triple.get(1); + List list3 = triple.get(2); + + // Skip cases where all tries are empty or where we have empty tries mixed with non-empty ones + // Collection merge requires at least one non-empty trie and can't handle mixed empty/non-empty + boolean hasEmpty = list1.isEmpty() || list2.isEmpty() || list3.isEmpty(); + boolean hasNonEmpty = !list1.isEmpty() || !list2.isEmpty() || !list3.isEmpty(); + + if (!hasNonEmpty || hasEmpty) + return; // Skip if all empty or if we have any empty tries + + DeletionAwareTrie trie1 = fromList(list1, forcedCopy); + DeletionAwareTrie trie2 = fromList(list2, forcedCopy); + DeletionAwareTrie trie3 = fromList(list3); + + // Test safe collection merge (deletionsAtFixedPoints = false, forcedCopy) + DeletionAwareTrie safeMerged = dir -> + new CollectionMergeCursor.DeletionAware<>( + LivePoint::combineCollection, + DeletionMarker::combineCollection, + DeletionMarker::applyTo, + false, + dir, + Arrays.asList(trie1, trie2, trie3), + DeletionAwareTrie::cursor); + + // Test optimized collection merge (deletionsAtFixedPoints = true) + DeletionAwareTrie optimizedMerged = dir -> + new CollectionMergeCursor.DeletionAware<>( + LivePoint::combineCollection, + DeletionMarker::combineCollection, + DeletionMarker::applyTo, + true, + dir, + Arrays.asList(trie1, trie2, trie3), + DeletionAwareTrie::cursor); + + List safeResult = toList(safeMerged); + List optimizedResult = toList(optimizedMerged); + + assertEquals("Optimized and safe collection merge should produce identical results", + safeResult, optimizedResult); + }); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/DeletionAwareTestBase.java b/test/unit/org/apache/cassandra/db/tries/DeletionAwareTestBase.java new file mode 100644 index 000000000000..b9a244b24c80 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/DeletionAwareTestBase.java @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.junit.BeforeClass; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +import static org.apache.cassandra.db.tries.DataPoint.contentOnlyList; +import static org.apache.cassandra.db.tries.DataPoint.deletionOnlyList; +import static org.apache.cassandra.db.tries.DataPoint.toList; +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.junit.Assert.assertEquals; + +public class DeletionAwareTestBase +{ + /// Change to true to pring debug info + static final boolean VERBOSE = false; + + static final int bitsNeeded = 6; + int bits = bitsNeeded; + + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + + private static String toString(ByteComparable ranges) + { + if (ranges == null) + return "null"; + return ranges.byteComparableAsString(TrieUtil.VERSION); + } + + private static DeletionMarker makeActiveMarker(int active, int rangeIndex, ByteComparable nextRange) + { + if (active >= 0) // cmp > 0, must covert active to marker + { + if ((rangeIndex & 1) != 0) + return new DeletionMarker(nextRange, active, -1, -1); + else + return new DeletionMarker(nextRange, -1, active, active); + } + return null; + } + + protected static void assertDeletionAwareEqual(String msg, List merged, DeletionAwareTrie trie) + { + try + { + assertEquals(msg, merged, toList(trie)); + assertEquals(msg + " live", + merged.stream().map(DataPoint::live).filter(x -> x != null).collect(Collectors.toList()), + contentOnlyList(trie)); + assertEquals(msg + " deletions", + merged.stream().map(DataPoint::marker).filter(x -> x != null).collect(Collectors.toList()), + deletionOnlyList(trie)); + System.out.println(msg + " matched."); + } + catch (AssertionError e) + { + System.out.println(); + DataPoint.dumpDeletionAwareTrie(trie); + throw e; + } + } + + static void maybeAdd(List list, T value) + { + if (value == null) + return; + list.add(value); + } + + /** + * Creates a {@link ByteComparable} for the provided value by splitting the integer in sequences of "bits" bits. + */ + private ByteComparable of(int value, int terminator) + { + // TODO: Also in all other tests of this type + assert value >= 0 && value <= Byte.MAX_VALUE; + + byte[] splitBytes = new byte[(bitsNeeded + bits - 1) / bits + 1]; + int pos = 0; + int mask = (1 << bits) - 1; + for (int i = bitsNeeded - bits; i > 0; i -= bits) + splitBytes[pos++] = (byte) ((value >> i) & mask); + + splitBytes[pos++] = (byte) (value & mask); + splitBytes[pos++] = (byte) terminator; + return ByteComparable.preencoded(VERSION, splitBytes); + } + + ByteComparable at(int value) + { + return of(value, ByteSource.TERMINATOR); + } + + ByteComparable before(int value) + { + return of(value, ByteSource.LT_NEXT_COMPONENT); + } + + ByteComparable after(int value) + { + return of(value, ByteSource.GT_NEXT_COMPONENT); + } + + DeletionMarker from(int where, int value) + { + return new DeletionMarker(before(where), -1, value, value); + } + + DeletionMarker to(int where, int value) + { + return new DeletionMarker(before(where), value, -1, -1); + } + + DeletionMarker change(int where, int from, int to) + { + return new DeletionMarker(before(where), from, to, to); + } + + DeletionMarker[] deletedPoint(int where, int value) + { + return deletedPointInside(where, value, -1); + } + + DeletionMarker[] deletedPointInside(int where, int value, int active) + { + return new DeletionMarker[] + { + new DeletionMarker(before(where), active, value, value), + new DeletionMarker(after(where), value, active, active) + }; + } + + DataPoint livePoint(int where, int timestamp) + { + return new LivePoint(at(where), timestamp); + } + + protected ByteComparable[] array(ByteComparable... data) + { + return data; + } + + protected List flatten(List pointsOrArrays) + { + return pointsOrArrays.stream() + .flatMap(x -> x instanceof DataPoint ? Stream.of((DataPoint) x) : Arrays.stream((DeletionMarker[]) x)) + .collect(Collectors.toList()); + } + + String toString(ByteComparable[] ranges) + { + StringBuilder b = new StringBuilder(); + for (int i = 0; i < ranges.length; i += 2) + { + b.append('['); + b.append(DeletionAwareTestBase.toString(ranges[i])); + b.append(';'); + b.append(DeletionAwareTestBase.toString(ranges[i + 1])); + b.append(')'); + } + return b.toString(); + } + + List intersect(List dataPoints, ByteComparable... ranges) + { + int rangeIndex = 0; + int active = -1; + ByteComparable nextRange = ranges[0]; + if (nextRange == null) + nextRange = ++rangeIndex < ranges.length ? ranges[rangeIndex] : null; + List result = new ArrayList<>(); + for (DataPoint dp : dataPoints) + { + DeletionMarker marker = dp.marker(); + while (true) + { + int cmp; + if (nextRange == null) + cmp = -1; + else + cmp = ByteComparable.compare(dp.position(), nextRange, TrieUtil.VERSION); + + if (cmp < 0) + { + if ((rangeIndex & 1) != 0) + DeletionAwareTestBase.maybeAdd(result, dp); + break; + } + + if (cmp == 0) + { + DeletionMarker adjustedMarker = marker != null ? marker : DeletionAwareTestBase.makeActiveMarker(active, rangeIndex, nextRange); + + if ((rangeIndex & 1) == 0) + DeletionAwareTestBase.maybeAdd(result, dp.withMarker(startOf(adjustedMarker))); + else + DeletionAwareTestBase.maybeAdd(result, dp.withMarker(endOf(adjustedMarker))); // live points are included at starts as well as ends + + nextRange = ++rangeIndex < ranges.length ? ranges[rangeIndex] : null; + break; + } + else + DeletionAwareTestBase.maybeAdd(result, DeletionAwareTestBase.makeActiveMarker(active, rangeIndex, nextRange)); + + nextRange = ++rangeIndex < ranges.length ? ranges[rangeIndex] : null; + } + if (marker != null) + active = marker.rightSide; + } + assert active == -1; + return result; + } + + DeletionMarker startOf(DeletionMarker marker) + { + return marker != null ? marker.restrict(false, true) : null; + } + + DeletionMarker endOf(DeletionMarker marker) + { + return marker != null ? marker.restrict(true, false) : null; + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/DeletionBranchConsistencyTest.java b/test/unit/org/apache/cassandra/db/tries/DeletionBranchConsistencyTest.java new file mode 100644 index 000000000000..65593058c4b6 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/DeletionBranchConsistencyTest.java @@ -0,0 +1,304 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Collection; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Predicate; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.concurrent.OpOrder; + +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; + +/** + * Consistency test for InMemoryDeletionAwareTrie that validates concurrent operations + * with both live data and deletion markers under different atomicity guarantees. + * + * This test extends ConsistencyTestBase to verify that InMemoryDeletionAwareTrie maintains + * correctness and consistency under concurrent access patterns typical of Cassandra's + * memtable operations with deletions. + */ +public class DeletionBranchConsistencyTest +extends ConsistencyTestBase, + InMemoryDeletionAwareTrie> +{ + + @SuppressWarnings("rawtypes") // type does not matter, we are always throwing an exception + static final InMemoryBaseTrie.UpsertTransformer UPSERT_THROW = (x, y) -> { throw new AssertionError(); }; + @SuppressWarnings("rawtypes") // type does not matter, we are always throwing an exception + static final BiFunction BIFUNCTION_THROW = (x, y) -> { throw new AssertionError(); }; + + @Override + InMemoryDeletionAwareTrie makeTrie(OpOrder readOrder) + { + return InMemoryDeletionAwareTrie.longLived(VERSION, readOrder); + } + + @Override + TestStateMetadata value(ByteComparable b, ByteComparable cprefix, ByteComparable c, int add, int seqId) + { + String pk = b.byteComparableAsString(VERSION); + String ck = (cprefix != null ? cprefix.byteComparableAsString(VERSION) : "") + c.byteComparableAsString(VERSION); + return new TestStateMetadata<>(new Value(pk, ck, add, seqId)); + } + + @Override + TestStateMetadata metadata(ByteComparable b) + { + return new TestStateMetadata<>(new Metadata(b.byteComparableAsString(VERSION))); + } + + @Override + String pk(TestStateMetadata c) + { + return ((Content)c.metadata).pk; + } + + @Override + String ck(TestStateMetadata c) + { + return ((Value) c.metadata).ck; + } + + @Override + int seq(TestStateMetadata c) + { + return ((Value) c.metadata).seq; + } + + @Override + int value(TestStateMetadata c) + { + return ((Value) c.metadata).value; + } + + @Override + int updateCount(TestStateMetadata c) + { + return ((Metadata) c.metadata).updateCount; + } + + @Override + DeletionAwareTrie makeSingleton(ByteComparable b, TestStateMetadata content) + { + return DeletionAwareTrie.deletionBranch(ByteComparable.EMPTY, VERSION, RangeTrie.singleton(b, VERSION, content)); + } + + @Override + DeletionAwareTrie withRootMetadata(DeletionAwareTrie wrapped, TestStateMetadata metadata) + { + return TrieUtil.withRootMetadata(wrapped, metadata); + } + + @Override + DeletionAwareTrie merge(Collection> tries, + Trie.CollectionMergeResolver mergeResolver) + { + return DeletionAwareTrie.merge(tries, + mergeResolver, + Trie.throwingResolver(), + BIFUNCTION_THROW, + true); // deletionsAtFixedPoints = true for consistency + } + + @Override + void apply(InMemoryDeletionAwareTrie trie, + DeletionAwareTrie mutation, + InMemoryBaseTrie.UpsertTransformer mergeResolver, + Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException + { + trie.apply(mutation, + mergeResolver, // Use the provided merge resolver for content + (x, y) -> mergeResolver.apply((TestStateMetadata) x, (TestStateMetadata) y), + UPSERT_THROW, + BIFUNCTION_THROW, + false, // deletionsAtFixedPoints = true for consistency + forcedCopyChecker); // Use the provided forced copy checker + } + + @Override + void delete(InMemoryDeletionAwareTrie trie, + ByteComparable deletionPrefix, + TestRangeState partitionMarker, + RangeTrie deletionBranch, + InMemoryBaseTrie.UpsertTransformer mergeResolver, + Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException + { + DeletionAwareTrie deletion = DeletionAwareTrie.deletionBranch(ByteComparable.EMPTY, VERSION, deletionBranch); + deletion = TrieUtil.withRootMetadata(deletion, partitionMarker); + deletion = deletion.prefixedBy(deletionPrefix); + + trie.apply(deletion, + mergeResolver, + (existing, incoming) -> (existing instanceof TestStateMetadata) + ? mergeResolver.apply((TestStateMetadata) existing, incoming) + : TestRangeState.combine(existing, incoming), + mergeResolver, + BIFUNCTION_THROW, + false, + forcedCopyChecker); + } + + @Override + boolean isPartition(TestStateMetadata c) + { + return c != null && ((Content) c.metadata).isPartition(); + } + + @Override + TestStateMetadata mergeMetadata(TestStateMetadata c1, TestStateMetadata c2) + { + if (c1 == null) return c2; + if (c2 == null) return c1; + return toTestStateMetadata(((Metadata) c1.metadata).mergeWith((Metadata) c2.metadata)); + } + + @Override + TestStateMetadata deleteMetadata(TestStateMetadata existing, int entriesToRemove) + { + if (existing == null) return null; + return toTestStateMetadata(((Metadata) existing.metadata).delete(entriesToRemove)); + } + + @Override + Iterable> getEntrySet(BaseTrie trie) + { + return ((DeletionAwareTrie) trie) + .mergedTrie(DeletionBranchConsistencyTest::mergeStateAndMetadata) + .entrySet(); + } + + static TestStateMetadata mergeStateAndMetadata(TestStateMetadata m, TestRangeState s) + { + if (!(s instanceof TestStateMetadata)) + return m; + TestStateMetadata m2 = (TestStateMetadata) s; + if (m == null) + return m2; + return toTestStateMetadata(((Metadata) m.metadata).mergeWith((Metadata) m2.metadata)); + } + + static TestStateMetadata toTestStateMetadata(Content c) + { + return c != null ? new TestStateMetadata(c) : null; + } + + @Override + void printStats(InMemoryDeletionAwareTrie trie, + Predicate> forcedCopyChecker) + { + System.out.format("DeletionAware Reuse %s %s on-heap %,d (+%,d) off-heap %,d\n", + trie.cellAllocator.getClass().getSimpleName(), + trie.bufferType, + trie.usedSizeOnHeap(), + trie.unusedReservedOnHeapMemory(), + trie.usedSizeOffHeap()); + } + + // TestStateMetadata hierarchy for deletion-aware consistency testing + abstract static class Content + { + final String pk; + + Content(String pk) + { + this.pk = pk; + } + + abstract boolean isPartition(); + } + + static class Value extends Content + { + final String ck; + final int value; + final int seq; + + Value(String pk, String ck, int value, int seq) + { + super(pk); + this.ck = ck; + this.value = value; + this.seq = seq; + } + + @Override + public String toString() + { + return "Value{" + + "pk='" + pk + '\'' + + ", ck='" + ck + '\'' + + ", value=" + value + + ", seq=" + seq + + '}'; + } + + @Override + boolean isPartition() + { + return false; + } + } + + static class Metadata extends Content + { + int updateCount; + + Metadata(String pk) + { + super(pk); + updateCount = 1; + } + + @Override + boolean isPartition() + { + return true; + } + + Metadata mergeWith(Metadata other) + { + Metadata m = new Metadata(pk); + m.updateCount = updateCount + other.updateCount; + return m; + } + + Metadata delete(int entriesToRemove) + { + assert updateCount >= entriesToRemove; + if (updateCount == entriesToRemove) + return null; + Metadata m = new Metadata(pk); + m.updateCount = updateCount - entriesToRemove; + return m; + } + + @Override + public String toString() + { + return "Metadata{" + + "pk='" + pk + '\'' + + ", updateCount=" + updateCount + + '}'; + } + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/DeletionMarker.java b/test/unit/org/apache/cassandra/db/tries/DeletionMarker.java new file mode 100644 index 000000000000..e3cfcf434883 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/DeletionMarker.java @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Collection; +import java.util.Objects; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +class DeletionMarker implements DataPoint, RangeState +{ + final ByteComparable position; + final int leftSide; + final int rightSide; + + final int at; + final boolean isBoundary; + + final DeletionMarker leftSideAsCovering; + final DeletionMarker rightSideAsCovering; + + DeletionMarker(ByteComparable position, int leftSide, int at, int rightSide) + { + this.position = position; + this.leftSide = leftSide; + this.rightSide = rightSide; + this.at = at; + this.isBoundary = at != leftSide || leftSide != rightSide; + + if (!isBoundary) + leftSideAsCovering = rightSideAsCovering = this; + else + { + if (this.leftSide < 0) + leftSideAsCovering = null; + else + leftSideAsCovering = new DeletionMarker(this.position, this.leftSide, this.leftSide, this.leftSide); + + if (this.rightSide < 0) + rightSideAsCovering = null; + else + rightSideAsCovering = new DeletionMarker(this.position, this.rightSide, this.rightSide, this.rightSide); + } + } + + static DeletionMarker combine(DeletionMarker m1, DeletionMarker m2) + { + int newLeft = Math.max(m1.leftSide, m2.leftSide); + int newAt = Math.max(m1.at, m2.at); + int newRight = Math.max(m1.rightSide, m2.rightSide); + if (newLeft < 0 && newAt < 0 && newRight < 0) + return null; + + return new DeletionMarker(m2.position, newLeft, newAt, newRight); + } + + + public static DeletionMarker combineCollection(Collection rangeMarkers) + { + int newLeft = -1; + int newAt = -1; + int newRight = -1; + ByteComparable position = null; + for (DeletionMarker marker : rangeMarkers) + { + newLeft = Math.max(newLeft, marker.leftSide); + newAt = Math.max(newAt, marker.at); + newRight = Math.max(newRight, marker.rightSide); + position = marker.position; + } + if (newLeft < 0 && newAt < 0 && newRight < 0) + return null; + + return new DeletionMarker(position, newLeft, newAt, newRight); + } + + DeletionMarker[] withPoint(int value) + { + return new DeletionMarker[] + { + new DeletionMarker(position, leftSide, value, value), + new DeletionMarker(replaceTerminator(position, ByteSource.GT_NEXT_COMPONENT), value, rightSide, rightSide) + }; + } + + ByteComparable replaceTerminator(ByteComparable c, int terminator) + { + byte[] key = c.asByteComparableArray(TrieUtil.VERSION); + key[key.length - 1] = (byte) terminator; + return ByteComparable.preencoded(TrieUtil.VERSION, key); + } + + @Override + public DeletionMarker marker() + { + return this; + } + + @Override + public LivePoint live() + { + return null; + } + + @Override + public ByteComparable position() + { + return position; + } + + @Override + public DeletionMarker withMarker(DeletionMarker newMarker) + { + return newMarker; + } + + @Override + public DeletionMarker remap(ByteComparable newKey) + { + return new DeletionMarker(newKey, leftSide, at, rightSide); + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DeletionMarker that = (DeletionMarker) o; + return leftSide == that.leftSide + && rightSide == that.rightSide + && at == that.at; + } + + @Override + public int hashCode() + { + return Objects.hash(position, leftSide, at, rightSide); + } + + @Override + public String toString() + { + boolean hasAt = at >= 0 && at != leftSide && at != rightSide; + String left = leftSide != at ? "<" : "<="; + String right = rightSide != at ? "<" : "<="; + + return (leftSide >= 0 ? leftSide + left : "") + + '{' + DataPoint.toString(position) + '}' + + (hasAt ? "=" + at : "") + + (rightSide >= 0 ? right + rightSide : ""); + } + + @Override + public DeletionMarker toContent() + { + return isBoundary ? this : null; + } + + @Override + public DeletionMarker restrict(boolean applicableBefore, boolean applicableAfter) + { + assert isBoundary; + if ((applicableBefore || leftSide < 0) && (applicableAfter || rightSide < 0)) + return this; + int newLeft = applicableBefore ? leftSide : -1; + int newRight = applicableAfter ? rightSide : -1; + int newAt = applicableAfter ? at : -1; + if (newLeft >= 0 || newRight >= 0 || newAt >= 0) + return new DeletionMarker(position, newLeft, newRight, newAt); + else + return null; + } + + @Override + public DeletionMarker precedingState(Direction direction) + { + return direction.select(leftSideAsCovering, rightSideAsCovering); + } + + @Override + public DeletionMarker asBoundary(Direction direction) + { + assert !isBoundary; + final boolean isForward = direction.isForward(); + int newLeft = !isForward ? leftSide : -1; + int newRight = isForward ? rightSide : -1; + int newAt = isForward ? at : -1; + return new DeletionMarker(position, newLeft, newRight, newAt); + } + + @Override + public boolean isBoundary() + { + return isBoundary; + } + + public LivePoint applyTo(LivePoint content) + { + return content.delete(at); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrieConsistencyTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrieConsistencyTest.java new file mode 100644 index 000000000000..12aac5469fbb --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrieConsistencyTest.java @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Collection; +import java.util.function.BiFunction; +import java.util.function.Predicate; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.concurrent.OpOrder; + +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; + +/** + * Consistency test for InMemoryDeletionAwareTrie that validates concurrent operations + * with both live data and deletion markers under different atomicity guarantees. + * + * This test extends ConsistencyTestBase to verify that InMemoryDeletionAwareTrie maintains + * correctness and consistency under concurrent access patterns typical of Cassandra's + * memtable operations with deletions. + */ +public class InMemoryDeletionAwareTrieConsistencyTest +extends ConsistencyTestBase, + InMemoryDeletionAwareTrie> +{ + + @SuppressWarnings("rawtypes") // type does not matter, we are always throwing an exception + static final InMemoryBaseTrie.UpsertTransformer UPSERT_THROW = (x, y) -> { throw new AssertionError(); }; + @SuppressWarnings("rawtypes") // type does not matter, we are always throwing an exception + static final BiFunction BIFUNCTION_THROW = (x, y) -> { throw new AssertionError(); }; + + @Override + InMemoryDeletionAwareTrie makeTrie(OpOrder readOrder) + { + return InMemoryDeletionAwareTrie.longLived(VERSION, readOrder); + } + + @Override + Content value(ByteComparable b, ByteComparable cprefix, ByteComparable c, int add, int seqId) + { + String pk = b.byteComparableAsString(VERSION); + String ck = (cprefix != null ? cprefix.byteComparableAsString(VERSION) : "") + c.byteComparableAsString(VERSION); + return new Value(pk, ck, add, seqId); + } + + @Override + Content metadata(ByteComparable b) + { + return new Metadata(b.byteComparableAsString(VERSION)); + } + + @Override + String pk(Content c) + { + return c.pk; + } + + @Override + String ck(Content c) + { + return ((Value) c).ck; + } + + @Override + int seq(Content c) + { + return ((Value) c).seq; + } + + @Override + int value(Content c) + { + return ((Value) c).value; + } + + @Override + int updateCount(Content c) + { + return ((Metadata) c).updateCount; + } + + @Override + DeletionAwareTrie makeSingleton(ByteComparable b, Content content) + { + return DeletionAwareTrie.singleton(b, VERSION, content); + } + + @Override + DeletionAwareTrie withRootMetadata(DeletionAwareTrie wrapped, Content metadata) + { + return TrieUtil.withRootMetadata(wrapped, metadata); + } + + @Override + DeletionAwareTrie merge(Collection> tries, + Trie.CollectionMergeResolver mergeResolver) + { + return DeletionAwareTrie.merge(tries, + mergeResolver, + Trie.throwingResolver(), + BIFUNCTION_THROW, + true); // deletionsAtFixedPoints = true for consistency + } + + @Override + void apply(InMemoryDeletionAwareTrie trie, + DeletionAwareTrie mutation, + InMemoryBaseTrie.UpsertTransformer mergeResolver, + Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException + { + trie.apply(mutation, + mergeResolver, // Use the provided merge resolver for content + (del, incoming) -> { throw new AssertionError(); }, + (del, incoming) -> { throw new AssertionError(); }, + (del, incoming) -> { throw new AssertionError(); }, + true, // deletionsAtFixedPoints = true for consistency + forcedCopyChecker); // Use the provided forced copy checker + } + + @Override + void delete(InMemoryDeletionAwareTrie trie, + ByteComparable deletionPrefix, + TestRangeState partitionMarker, + RangeTrie deletionBranch, + InMemoryBaseTrie.UpsertTransformer mergeResolver, + Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException + { + DeletionAwareTrie deletion = DeletionAwareTrie.deletionBranch(ByteComparable.EMPTY, VERSION, deletionBranch); + deletion = TrieUtil.withRootMetadata(deletion, partitionMarker); + deletion = deletion.prefixedBy(deletionPrefix); + + trie.apply(deletion, + mergeResolver, + (existing, incoming) -> TestRangeState.combine(existing, incoming), + mergeResolver, + BIFUNCTION_THROW, + true, + forcedCopyChecker); + } + + @Override + boolean isPartition(Content c) + { + return c != null && c.isPartition(); + } + + @Override + Content mergeMetadata(Content c1, Content c2) + { + if (c1 == null) return c2; + if (c2 == null) return c1; + return ((Metadata) c1).mergeWith((Metadata) c2); + } + + @Override + Content deleteMetadata(Content existing, int entriesToRemove) + { + if (existing == null) return null; + return ((Metadata) existing).delete(entriesToRemove); + } + + @Override + void printStats(InMemoryDeletionAwareTrie trie, + Predicate> forcedCopyChecker) + { + System.out.format("DeletionAware Reuse %s %s on-heap %,d (+%,d) off-heap %,d\n", + trie.cellAllocator.getClass().getSimpleName(), + trie.bufferType, + trie.usedSizeOnHeap(), + trie.unusedReservedOnHeapMemory(), + trie.usedSizeOffHeap()); + } + + // Content hierarchy for deletion-aware consistency testing + abstract static class Content + { + final String pk; + + Content(String pk) + { + this.pk = pk; + } + + abstract boolean isPartition(); + } + + static class Value extends Content + { + final String ck; + final int value; + final int seq; + + Value(String pk, String ck, int value, int seq) + { + super(pk); + this.ck = ck; + this.value = value; + this.seq = seq; + } + + @Override + public String toString() + { + return "Value{" + + "pk='" + pk + '\'' + + ", ck='" + ck + '\'' + + ", value=" + value + + ", seq=" + seq + + '}'; + } + + @Override + boolean isPartition() + { + return false; + } + } + + static class Metadata extends Content + { + int updateCount; + + Metadata(String pk) + { + super(pk); + updateCount = 1; + } + + @Override + boolean isPartition() + { + return true; + } + + Metadata mergeWith(Metadata other) + { + Metadata m = new Metadata(pk); + m.updateCount = updateCount + other.updateCount; + return m; + } + + Metadata delete(int entriesToRemove) + { + assert updateCount >= entriesToRemove; + if (updateCount == entriesToRemove) + return null; + Metadata m = new Metadata(pk); + m.updateCount = updateCount - entriesToRemove; + return m; + } + + @Override + public String toString() + { + return "Metadata{" + + "pk='" + pk + '\'' + + ", updateCount=" + updateCount + + '}'; + } + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrieThreadedTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrieThreadedTest.java new file mode 100644 index 000000000000..ae2256b59022 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrieThreadedTest.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import org.junit.BeforeClass; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.concurrent.OpOrder; + +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; + +/** + * Multithreaded test for InMemoryDeletionAwareTrie that creates both data and deletion entries. + * + * This test extends ThreadedTestBase to verify that InMemoryDeletionAwareTrie works correctly + * under concurrent access with both live data points and deletion markers being added. + */ +public class InMemoryDeletionAwareTrieThreadedTest extends ThreadedTestBase> +{ + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + + @Override + LivePoint value(ByteComparable b) + { + // Create a live point with a timestamp based on the key's hash + // This ensures all threads will create the same value for the same key + String keyStr = b.byteComparableAsString(VERSION); + int timestamp = Math.abs(keyStr.hashCode()) % 1000 + 1; // Ensure positive timestamp + return new LivePoint(b, timestamp); + } + + @Override + InMemoryDeletionAwareTrie makeTrie(OpOrder readOrder) + { + return InMemoryDeletionAwareTrie.longLived(VERSION, readOrder); + } + + @Override + void add(InMemoryDeletionAwareTrie trie, ByteComparable b, LivePoint v, int iteration) throws TrieSpaceExhaustedException + { + // Alternate between adding live data and deletion markers + if (iteration % 3 == 0) + { + // Add live data point using apply with singleton trie + DeletionAwareTrie singletonTrie = DeletionAwareTrie.singleton(b, VERSION, v); + trie.apply(singletonTrie, + DataPoint::combineLive, // Combine live data using DataPoint utility + DataPoint::combineDeletion, // Combine deletion markers using DataPoint utility + DataPoint::deleteLive, // Apply deletions to existing data using DataPoint utility + DataPoint::deleteLive, // Apply deletions to incoming data using DataPoint utility + true, // deletionsAtFixedPoints = true (singleton deletions satisfy invariant) + x -> false); // needsForcedCopy = never force copy for this test + } + else if (iteration % 3 == 1) + { + // Add deletion marker using DeletionAwareTrie.deletion + // Create a deletion marker that deletes data with timestamp less than current + int deletionTime = v.timestamp + 10; // Delete older data + DeletionMarker marker = new DeletionMarker(b, deletionTime, deletionTime, deletionTime); + + DeletionAwareTrie deletionTrie = + DeletionAwareTrie.deletion(b, b, b, VERSION, marker); + + trie.apply(deletionTrie, + (existing, incoming) -> existing, // Keep existing live data (no incoming live data in deletion trie) + DataPoint::combineDeletion, // Combine deletion markers using DataPoint utility + DataPoint::deleteLive, // Apply deletions to existing data using DataPoint utility + DataPoint::deleteLive, // Apply deletions to incoming data using DataPoint utility + true, // deletionsAtFixedPoints = true (singleton deletions satisfy invariant) + x -> false); // needsForcedCopy = never force copy for this test + } + else + { + // Add a merge of singleton and deletion + DeletionAwareTrie singletonTrie = DeletionAwareTrie.singleton(b, VERSION, v); + + // Create a deletion marker that deletes data with timestamp less than current + int deletionTime = v.timestamp + 5; // Delete slightly older data + DeletionMarker marker = new DeletionMarker(b, deletionTime, deletionTime, deletionTime); + DeletionAwareTrie deletionTrie = + DeletionAwareTrie.deletion(b, b, b, VERSION, marker); + + // Merge singleton and deletion into a combined trie + DeletionAwareTrie combinedTrie = + singletonTrie.mergeWith(deletionTrie, + DataPoint::combineLive, + DataPoint::combineDeletion, + DataPoint::deleteLive, + true); // deletionsAtFixedPoints = true + + trie.apply(combinedTrie, + DataPoint::combineLive, // Combine live data using DataPoint utility + DataPoint::combineDeletion, // Combine deletion markers using DataPoint utility + DataPoint::deleteLive, // Apply deletions to existing data using DataPoint utility + DataPoint::deleteLive, // Apply deletions to incoming data using DataPoint utility + true, // deletionsAtFixedPoints = true (singleton deletions satisfy invariant) + x -> false); // needsForcedCopy = never force copy for this test + } + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java index e23a65a6af33..1f5243d68be9 100644 --- a/test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java +++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieConsistencyTest.java @@ -102,8 +102,15 @@ void apply(InMemoryTrie trie, Trie mutation, InMemoryBaseTrie. } @Override - void delete(InMemoryTrie trie, RangeTrie deletion, InMemoryBaseTrie.UpsertTransformer mergeResolver, Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException - { + void delete(InMemoryTrie trie, + ByteComparable deletionPrefix, + TestRangeState partitionMarker, + RangeTrie deletion, + InMemoryBaseTrie.UpsertTransformer mergeResolver, + Predicate> forcedCopyChecker) throws TrieSpaceExhaustedException + { + deletion = TrieUtil.withRootMetadata(deletion, partitionMarker); + deletion = deletion.prefixedBy(deletionPrefix); trie.apply(deletion, mergeResolver, forcedCopyChecker); } diff --git a/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java b/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java index 9a24db95de45..5997c0ae4e49 100644 --- a/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/IntersectionTrieTest.java @@ -605,7 +605,7 @@ public void testReturnsContentOnPrefix() throws TrieSpaceExhaustedException Trie expected = TrieUtil.directTrie("a", "ab", "abc"); assertMapEquals(expected.entrySet(Direction.FORWARD), trie.intersect(set).entrySet(Direction.FORWARD), TrieUtil.FORWARD_COMPARATOR); assertMapEquals(expected.entrySet(Direction.REVERSE), trie.intersect(set).entrySet(Direction.REVERSE), TrieUtil.REVERSE_COMPARATOR); - assertEquals(expected.process(Direction.FORWARD, new TrieDumper<>(Object::toString)), trie.intersect(set).dump()); + assertEquals(expected.process(Direction.FORWARD, new TrieDumper.Plain<>(Object::toString)), trie.intersect(set).dump()); } @Test @@ -616,7 +616,7 @@ public void testReturnsBranchContents() throws TrieSpaceExhaustedException Trie expected = TrieUtil.directTrie("abc", "abce", "abcfff"); assertMapEquals(expected.entrySet(Direction.FORWARD), trie.intersect(set).entrySet(Direction.FORWARD), TrieUtil.FORWARD_COMPARATOR); assertMapEquals(expected.entrySet(Direction.REVERSE), trie.intersect(set).entrySet(Direction.REVERSE), TrieUtil.REVERSE_COMPARATOR); - assertEquals(expected.process(Direction.FORWARD, new TrieDumper<>(Object::toString)), trie.intersect(set).dump()); + assertEquals(expected.process(Direction.FORWARD, new TrieDumper.Plain<>(Object::toString)), trie.intersect(set).dump()); } @Test(expected = AssertionError.class) diff --git a/test/unit/org/apache/cassandra/db/tries/LivePoint.java b/test/unit/org/apache/cassandra/db/tries/LivePoint.java new file mode 100644 index 000000000000..81ccc96c5994 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/LivePoint.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.util.Collection; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +class LivePoint implements DataPoint +{ + final ByteComparable position; + final int timestamp; + + public LivePoint(ByteComparable position, int timestamp) + { + this.position = position; + this.timestamp = timestamp; + } + + public LivePoint delete(int timestamp) + { + return this.timestamp < timestamp ? null : this; + } + + @Override + public DeletionMarker marker() + { + return null; + } + + @Override + public LivePoint live() + { + return this; + } + + @Override + public ByteComparable position() + { + return position; + } + + @Override + public DataPoint withMarker(DeletionMarker newMarker) + { + if (newMarker == null) + return this; + else + return new CombinedDataPoint(this, newMarker); + } + + @Override + public LivePoint remap(ByteComparable newKey) + { + return new LivePoint(newKey, timestamp); + } + + @Override + public DataPoint toContent() + { + return this; + } + + @Override + public String toString() + { + return '{' + DataPoint.toString(position) + "}L" + timestamp; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + LivePoint livePoint = (LivePoint) o; + return timestamp == livePoint.timestamp + && ByteComparable.compare(this.position, livePoint.position, TrieUtil.VERSION) == 0; + } + + static LivePoint combine(LivePoint a, LivePoint b) + { + return a.timestamp >= b.timestamp ? a : b; + } + + static LivePoint combineCollection(Collection values) + { + return values.stream().reduce(LivePoint::combine).orElseThrow(); + } +} diff --git a/test/unit/org/apache/cassandra/db/tries/RangeTrieIntersectionTest.java b/test/unit/org/apache/cassandra/db/tries/RangeTrieIntersectionTest.java index 994e9bb2ca09..313a3ff0a943 100644 --- a/test/unit/org/apache/cassandra/db/tries/RangeTrieIntersectionTest.java +++ b/test/unit/org/apache/cassandra/db/tries/RangeTrieIntersectionTest.java @@ -317,7 +317,7 @@ public void testIntersection(String message, List expected, Rang catch (AssertionError e) { System.out.println("\nFORWARD:\n" + trie.dump(TestRangeState::toStringNoPosition)); - System.out.println("\nREVERSE:\n" + trie.cursor(Direction.REVERSE).process(new TrieDumper<>(TestRangeState::toStringNoPosition))); + System.out.println("\nREVERSE:\n" + trie.cursor(Direction.REVERSE).process(new TrieDumper.Plain<>(TestRangeState::toStringNoPosition))); throw e; } } diff --git a/test/unit/org/apache/cassandra/db/tries/RangesTrieSetTest.java b/test/unit/org/apache/cassandra/db/tries/RangesTrieSetTest.java index 5674352ea63c..8d93a44c5d57 100644 --- a/test/unit/org/apache/cassandra/db/tries/RangesTrieSetTest.java +++ b/test/unit/org/apache/cassandra/db/tries/RangesTrieSetTest.java @@ -104,7 +104,7 @@ public ByteComparable.Version byteComparableVersion() static String dump(TrieSet s, Direction direction) { - return fullTrie(s).process(direction, new TrieDumper<>(Object::toString)); + return fullTrie(s).process(direction, new TrieDumper.Plain<>(Object::toString)); } static void dumpToOut(TrieSet s) diff --git a/test/unit/org/apache/cassandra/db/tries/TrieUtil.java b/test/unit/org/apache/cassandra/db/tries/TrieUtil.java index 08ad887174c9..a9069d0d392d 100644 --- a/test/unit/org/apache/cassandra/db/tries/TrieUtil.java +++ b/test/unit/org/apache/cassandra/db/tries/TrieUtil.java @@ -372,6 +372,15 @@ public static > RangeTrie withRootMetadata(RangeTrie< return wrapped.mergeWith(RangeTrie.singleton(ByteComparable.EMPTY, VERSION, metadata), Trie.throwingResolver()); } + public static > DeletionAwareTrie withRootMetadata(DeletionAwareTrie wrapped, T metadata) + { + return wrapped.mergeWith(DeletionAwareTrie.singleton(ByteComparable.EMPTY, VERSION, metadata), + Trie.throwingResolver(), + Trie.throwingResolver(), + (d,t) -> { throw new AssertionError(); }, + false); + } + static Trie directTrie(String... points) throws TrieSpaceExhaustedException { InMemoryTrie trie = InMemoryTrie.shortLived(VERSION); From 806924295e07cefe8cfaeb69bd70ccdf57952baa Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Tue, 15 Jul 2025 16:59:19 +0300 Subject: [PATCH 16/22] Add "Stage2" versions of trie memtable and partition classes --- .../cassandra/db/memtable/TrieMemtable.java | 2 +- .../db/memtable/TrieMemtableStage1.java | 39 +- .../db/memtable/TrieMemtableStage2.java | 944 ++++++++++++++++++ .../partitions/TrieBackedPartitionStage2.java | 709 +++++++++++++ .../partitions/TriePartitionUpdateStage2.java | 586 +++++++++++ .../TriePartitionUpdaterStage2.java | 169 ++++ .../cassandra/cql3/MemtableQuickTest.java | 1 + .../db/memtable/MemtableSizeTestBase.java | 1 + .../db/memtable/MemtableThreadedTest.java | 1 + 9 files changed, 2430 insertions(+), 22 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java create mode 100644 src/java/org/apache/cassandra/db/partitions/TrieBackedPartitionStage2.java create mode 100644 src/java/org/apache/cassandra/db/partitions/TriePartitionUpdateStage2.java create mode 100644 src/java/org/apache/cassandra/db/partitions/TriePartitionUpdaterStage2.java diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java index 3334b21a872b..4a2f0e7d052d 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java @@ -408,7 +408,7 @@ public MemtableUnfilteredPartitionIterator makePartitionIterator(final ColumnFil private static ByteComparable toComparableBound(PartitionPosition position, boolean before) { - return position.isMinimum() ? null : position.asComparableBound(before); + return position == null || position.isMinimum() ? null : position.asComparableBound(before); } public Partition getPartition(DecoratedKey key) diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java index 6323f0d54e1c..d77b86b20bd1 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java @@ -75,26 +75,23 @@ import org.apache.cassandra.utils.memory.MemtableAllocator; import org.github.jamm.Unmetered; -/** - * Previous TrieMemtable implementation, provided for two reasons: - *
      - *
    • to easily compare current and earlier implementations of the trie memtable - *
    • to have an option to change a database back to the older implementation if we find a bug or a performance problem - * with the new code. - *
    - *

    - * To switch a table to this version, use - *

    - *   ALTER TABLE ... WITH memtable = {'class': 'TrieMemtableStage1'}
    - * 
    - * or add - *
    - *   memtable:
    - *     class: TrieMemtableStage1
    - * 
    - * in cassandra.yaml to switch a node to it as default. - * - */ +/// Previous TrieMemtable implementation, provided for two reasons: +/// +/// - to easily compare current and earlier implementations of the trie memtable +/// - to have an option to change a database back to the older implementation if we find a bug or a performance +/// problem with the new code. +/// +/// +/// To switch a table to this version, use +/// ``` +/// ALTER TABLE ... WITH memtable = {'class': 'TrieMemtableStage1'} +/// ``` +/// or add +/// ``` +/// memtable: +/// class: TrieMemtableStage1 +/// ``` +/// in `cassandra.yaml` to switch a node to it as default. public class TrieMemtableStage1 extends AbstractAllocatorMemtable { private static final Logger logger = LoggerFactory.getLogger(TrieMemtableStage1.class); @@ -399,7 +396,7 @@ public MemtableUnfilteredPartitionIterator makePartitionIterator(final ColumnFil private static ByteComparable toComparableBound(PartitionPosition position, boolean before) { - return position.isMinimum() ? null : position.asComparableBound(before); + return position == null || position.isMinimum() ? null : position.asComparableBound(before); } public Partition getPartition(DecoratedKey key) diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java new file mode 100644 index 000000000000..e286bce472a5 --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java @@ -0,0 +1,944 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.memtable; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Predicate; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.BufferDecoratedKey; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.MutableDeletionInfo; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.commitlog.CommitLogPosition; +import org.apache.cassandra.db.filter.ClusteringIndexFilter; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.partitions.AbstractUnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.Partition; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.partitions.TrieBackedPartitionStage2; +import org.apache.cassandra.db.partitions.TriePartitionUpdateStage2; +import org.apache.cassandra.db.partitions.TriePartitionUpdaterStage2; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.tries.Direction; +import org.apache.cassandra.db.tries.InMemoryTrie; +import org.apache.cassandra.db.tries.Trie; +import org.apache.cassandra.db.tries.TrieEntriesWalker; +import org.apache.cassandra.db.tries.TrieSpaceExhaustedException; +import org.apache.cassandra.db.tries.TrieTailsIterator; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.dht.IncludingExcludingBounds; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.index.transactions.UpdateTransaction; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.metrics.TableMetrics; +import org.apache.cassandra.metrics.TrieMemtableMetricsView; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.TableMetadataRef; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.MBeanWrapper; +import org.apache.cassandra.utils.ObjectSizes; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.concurrent.OpOrder; +import org.apache.cassandra.utils.memory.EnsureOnHeap; +import org.apache.cassandra.utils.memory.HeapCloner; +import org.apache.cassandra.utils.memory.MemtableAllocator; +import org.github.jamm.Unmetered; + +/// Previous TrieMemtable implementation, provided for two reasons: +/// +/// - to easily compare current and earlier implementations of the trie memtable +/// - to have an option to change a database back to the older implementation if we find a bug or a performance +/// problem with the new code. +/// +/// +/// To switch a table to this version, use +/// ``` +/// ALTER TABLE ... WITH memtable = {'class': 'TrieMemtableStage2'} +/// ``` +/// or add +/// ``` +/// memtable: +/// class: TrieMemtableStage2 +/// ``` +/// in `cassandra.yaml` to switch a node to it as default. +public class TrieMemtableStage2 extends AbstractAllocatorMemtable +{ + private static final Logger logger = LoggerFactory.getLogger(TrieMemtableStage2.class); + public static final String TRIE_MEMTABLE_CONFIG_OBJECT_NAME = "org.apache.cassandra.db:type=TrieMemtableConfig"; + + public static final Factory FACTORY = new TrieMemtableStage2.Factory(); + + /** Buffer type to use for memtable tries (on- vs off-heap) */ + public static final BufferType BUFFER_TYPE; + + static + { + switch (DatabaseDescriptor.getMemtableAllocationType()) + { + case unslabbed_heap_buffers: + case heap_buffers: + BUFFER_TYPE = BufferType.ON_HEAP; + break; + case offheap_buffers: + case offheap_objects: + BUFFER_TYPE = BufferType.OFF_HEAP; + break; + default: + throw new AssertionError(); + } + + MBeanWrapper.instance.registerMBean(new TrieMemtableConfig(), TRIE_MEMTABLE_CONFIG_OBJECT_NAME, MBeanWrapper.OnException.LOG); + } + + /** + * Force copy checker (see InMemoryTrie.ApplyState) ensuring all modifications apply atomically and consistently to + * the whole partition. + */ + public static final Predicate> FORCE_COPY_PARTITION_BOUNDARY = features -> isPartitionBoundary(features.content()); + + public static final Predicate IS_PARTITION_BOUNDARY = TrieMemtableStage2::isPartitionBoundary; + + // Set to true when the memtable requests a switch (e.g. for trie size limit being reached) to ensure only one + // thread calls cfs.switchMemtableIfCurrent. + private AtomicBoolean switchRequested = new AtomicBoolean(false); + + + // The boundaries for the keyspace as they were calculated when the memtable is created. + // The boundaries will be NONE for system keyspaces or if StorageService is not yet initialized. + // The fact this is fixed for the duration of the memtable lifetime, guarantees we'll always pick the same core + // for the a given key, even if we race with the StorageService initialization or with topology changes. + @Unmetered + private final ShardBoundaries boundaries; + + /** + * Core-specific memtable regions. All writes must go through the specific core. The data structures used + * are concurrent-read safe, thus reads can be carried out from any thread. + */ + private final MemtableShard[] shards; + + /** + * A merged view of the memtable map. Used for partition range queries and flush. + * For efficiency we serve single partition requests off the shard which offers more direct InMemoryTrie methods. + */ + private final Trie mergedTrie; + + @Unmetered + private final TrieMemtableMetricsView metrics; + + /** + * Keeps an estimate of the average row size in this memtable, computed from a small sample of rows. + * Because computing this estimate is potentially costly, as it requires iterating the rows, + * the estimate is updated only whenever the number of operations on the memtable increases significantly from the + * last update. This estimate is not very accurate but should be ok for planning or diagnostic purposes. + */ + private volatile MemtableAverageRowSize estimatedAverageRowSize; + + public static volatile int SHARD_COUNT = CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_COUNT.getInt(autoShardCount()); + public static volatile boolean SHARD_LOCK_FAIRNESS = CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_LOCK_FAIRNESS.getBoolean(); + + private static int autoShardCount() + { + return 4 * FBUtilities.getAvailableProcessors(); + } + + // only to be used by init(), to setup the very first memtable for the cfs + TrieMemtableStage2(AtomicReference commitLogLowerBound, TableMetadataRef metadataRef, Owner owner) + { + super(commitLogLowerBound, metadataRef, owner); + this.boundaries = owner.localRangeSplits(SHARD_COUNT); + this.metrics = TrieMemtableMetricsView.getOrCreate(metadataRef.keyspace, metadataRef.name); + this.shards = generatePartitionShards(boundaries.shardCount(), metadataRef, metrics, owner.readOrdering()); + this.mergedTrie = makeMergedTrie(shards); + logger.trace("Created memtable with {} shards", this.shards.length); + } + + private static MemtableShard[] generatePartitionShards(int splits, + TableMetadataRef metadata, + TrieMemtableMetricsView metrics, + OpOrder opOrder) + { + if (splits == 1) + return new MemtableShard[] { new MemtableShard(metadata, metrics, opOrder) }; + + MemtableShard[] partitionMapContainer = new MemtableShard[splits]; + for (int i = 0; i < splits; i++) + partitionMapContainer[i] = new MemtableShard(metadata, metrics, opOrder); + + return partitionMapContainer; + } + + private static Trie makeMergedTrie(MemtableShard[] shards) + { + List> tries = new ArrayList<>(shards.length); + for (MemtableShard shard : shards) + tries.add(shard.data); + return Trie.mergeDistinct(tries); + } + + protected Factory factory() + { + return FACTORY; + } + + public boolean isClean() + { + for (MemtableShard shard : shards) + if (!shard.isEmpty()) + return false; + return true; + } + + @VisibleForTesting + @Override + public void switchOut(OpOrder.Barrier writeBarrier, AtomicReference commitLogUpperBound) + { + super.switchOut(writeBarrier, commitLogUpperBound); + + for (MemtableShard shard : shards) + shard.allocator.setDiscarding(); + } + + @Override + public void discard() + { + super.discard(); + // metrics here are not thread safe, but I think we can live with that + metrics.lastFlushShardDataSizes.reset(); + for (MemtableShard shard : shards) + { + metrics.lastFlushShardDataSizes.update(shard.liveDataSize()); + } + for (MemtableShard shard : shards) + { + shard.allocator.setDiscarded(); + shard.data.discardBuffers(); + } + } + + /** + * Should only be called by ColumnFamilyStore.apply via Keyspace.apply, which supplies the appropriate + * OpOrdering. + * + * commitLogSegmentPosition should only be null if this is a secondary index, in which case it is *expected* to be null + */ + public long put(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) + { + DecoratedKey key = update.partitionKey(); + MemtableShard shard = shards[boundaries.getShardForKey(key)]; + long colUpdateTimeDelta = shard.put(update, indexer, opGroup); + + if (shard.data.reachedAllocatedSizeThreshold()) + signalFlushRequired(ColumnFamilyStore.FlushReason.TRIE_LIMIT, true); + + return colUpdateTimeDelta; + } + + @Override + public void signalFlushRequired(ColumnFamilyStore.FlushReason flushReason, boolean skipIfSignaled) + { + if (!switchRequested.getAndSet(true) || !skipIfSignaled) + { + logger.info("Scheduling flush for table {} due to {}", this.metadata.get(), flushReason); + owner.signalFlushRequired(this, flushReason); + } + } + + @Override + public void addMemoryUsageTo(MemoryUsage stats) + { + super.addMemoryUsageTo(stats); + for (MemtableShard shard : shards) + { + stats.ownsOnHeap += shard.allocator.onHeap().owns(); + stats.ownsOffHeap += shard.allocator.offHeap().owns(); + stats.ownershipRatioOnHeap += shard.allocator.onHeap().ownershipRatio(); + stats.ownershipRatioOffHeap += shard.allocator.offHeap().ownershipRatio(); + } + } + + @Override + public long getLiveDataSize() + { + long total = 0L; + for (MemtableShard shard : shards) + total += shard.liveDataSize(); + return total; + } + + @Override + public long getOperations() + { + long total = 0L; + for (MemtableShard shard : shards) + total += shard.currentOperations(); + return total; + } + + @Override + public long partitionCount() + { + int total = 0; + for (MemtableShard shard : shards) + total += shard.partitionCount(); + return total; + } + + public int getShardCount() + { + return shards.length; + } + + @Override + public long getEstimatedAverageRowSize() + { + if (estimatedAverageRowSize == null || currentOperations.get() > estimatedAverageRowSize.operations * 1.5) + estimatedAverageRowSize = new MemtableAverageRowSize(this); + return estimatedAverageRowSize.rowSize; + } + + /** + * Returns the minTS if one available, otherwise NO_MIN_TIMESTAMP. + * + * EncodingStats uses a synthetic epoch TS at 2015. We don't want to leak that (CASSANDRA-18118) so we return NO_MIN_TIMESTAMP instead. + * + * @return The minTS or NO_MIN_TIMESTAMP if none available + */ + @Override + public long getMinTimestamp() + { + long min = Long.MAX_VALUE; + for (MemtableShard shard : shards) + min = EncodingStats.mergeMinTimestamp(min, shard.stats); + return min != EncodingStats.NO_STATS.minTimestamp ? min : NO_MIN_TIMESTAMP; + } + + public int getMinLocalDeletionTime() + { + int min = Integer.MAX_VALUE; + for (MemtableShard shard : shards) + min = EncodingStats.mergeMinLocalDeletionTime(min, shard.stats); + return min; + } + + @Override + public DecoratedKey minPartitionKey() + { + for (int i = 0; i < shards.length; i++) + { + MemtableShard shard = shards[i]; + if (!shard.isEmpty()) + return shard.minPartitionKey(); + } + return null; + } + + @Override + public DecoratedKey maxPartitionKey() + { + for (int i = shards.length - 1; i >= 0; i--) + { + MemtableShard shard = shards[i]; + if (!shard.isEmpty()) + return shard.maxPartitionKey(); + } + return null; + } + + @Override + RegularAndStaticColumns columns() + { + for (MemtableShard shard : shards) + columnsCollector.update(shard.columns); + return columnsCollector.get(); + } + + @Override + EncodingStats encodingStats() + { + for (MemtableShard shard : shards) + statsCollector.update(shard.stats); + return statsCollector.get(); + } + + static boolean isPartitionBoundary(Object content) + { + // In the trie we use PartitionData for the root of a partition, but PartitionUpdates come with DeletionInfo. + // Both are descendants of DeletionInfo. + return content instanceof DeletionInfo; + } + + public MemtableUnfilteredPartitionIterator makePartitionIterator(final ColumnFilter columnFilter, final DataRange dataRange) + { + AbstractBounds keyRange = dataRange.keyRange(); + + boolean isBound = keyRange instanceof Bounds; + boolean includeStart = isBound || keyRange instanceof IncludingExcludingBounds; + boolean includeStop = isBound || keyRange instanceof Range; + + Trie subMap = mergedTrie.subtrie(toComparableBound(keyRange.left, includeStart), + toComparableBound(keyRange.right, !includeStop)); + + return new MemtableUnfilteredPartitionIterator(metadata(), + allocator.ensureOnHeap(), + subMap, + columnFilter, + dataRange, + getMinLocalDeletionTime()); + // Note: the minLocalDeletionTime reported by the iterator is the memtable's minLocalDeletionTime. This is okay + // because we only need to report a lower bound that will eventually advance, and calculating a more precise + // bound would be an unnecessary expense. + } + + private static ByteComparable toComparableBound(PartitionPosition position, boolean before) + { + return position == null || position.isMinimum() ? null : position.asComparableBound(before); + } + + public Partition getPartition(DecoratedKey key) + { + int shardIndex = boundaries.getShardForKey(key); + Trie trie = shards[shardIndex].data.tailTrie(key); + return createPartition(metadata(), allocator.ensureOnHeap(), key, trie); + } + + private static TrieBackedPartitionStage2 createPartition(TableMetadata metadata, EnsureOnHeap ensureOnHeap, DecoratedKey key, Trie trie) + { + if (trie == null) + return null; + PartitionData holder = (PartitionData) trie.get(ByteComparable.EMPTY); + // If we found a matching path in the trie, it must be the root of this partition (because partition keys are + // prefix-free, it can't be a prefix for a different path, or have another partition key as prefix) and contain + // PartitionData (because the attachment of a new or modified partition to the trie is atomic). + assert holder != null : "Entry for " + key + " without associated PartitionData"; + + return TrieBackedPartitionStage2.create(key, + holder.columns(), + holder.stats(), + holder.rowCountIncludingStatic(), + trie, + metadata, + ensureOnHeap); + } + + private static DecoratedKey getPartitionKeyFromPath(TableMetadata metadata, ByteComparable path) + { + return BufferDecoratedKey.fromByteComparable(path, + TrieBackedPartitionStage2.BYTE_COMPARABLE_VERSION, + metadata.partitioner); + } + + /** + * Metadata object signifying the root node of a partition. Holds the deletion information as well as a link + * to the owning subrange, which is used for compiling statistics and column sets. + * + * Descends from MutableDeletionInfo to permit tail tries to be passed directly to TrieBackedPartitionStage2. + */ + public static class PartitionData extends MutableDeletionInfo + { + @Unmetered + public final MemtableShard owner; + + private int rowCountIncludingStatic; + + public static final long HEAP_SIZE = ObjectSizes.measure(new PartitionData(DeletionInfo.LIVE, null)); + + public PartitionData(DeletionInfo deletion, + MemtableShard owner) + { + super(deletion.getPartitionDeletion(), deletion.copyRanges(HeapCloner.instance)); + this.owner = owner; + this.rowCountIncludingStatic = 0; + } + + public PartitionData(PartitionData existing, + DeletionInfo update) + { + // Start with the update content, to properly copy it + this(update, existing.owner); + rowCountIncludingStatic = existing.rowCountIncludingStatic; + add(existing); + } + + public RegularAndStaticColumns columns() + { + return owner.columns; + } + + public EncodingStats stats() + { + return owner.stats; + } + + public int rowCountIncludingStatic() + { + return rowCountIncludingStatic; + } + + public void markInsertedRows(int howMany) + { + rowCountIncludingStatic += howMany; + } + + @Override + public String toString() + { + return "partition " + super.toString(); + } + + @Override + public long unsharedHeapSize() + { + return super.unsharedHeapSize() + HEAP_SIZE - MutableDeletionInfo.EMPTY_SIZE; + } + } + + + class KeySizeAndCountCollector extends TrieEntriesWalker + { + long keySize = 0; + int keyCount = 0; + + @Override + public Void complete() + { + return null; + } + + @Override + protected void content(Object content, byte[] bytes, int byteLength) + { + // This is used with processSkippingBranches which should ensure that we only see the partition roots. + assert content instanceof PartitionData; + ++keyCount; + byte[] keyBytes = DecoratedKey.keyFromByteSource(ByteSource.preencoded(bytes, 0, byteLength), + TrieBackedPartitionStage2.BYTE_COMPARABLE_VERSION, + metadata().partitioner); + keySize += keyBytes.length; + } + } + + public FlushCollection getFlushSet(PartitionPosition from, PartitionPosition to) + { + Trie toFlush = mergedTrie.subtrie(toComparableBound(from, true), toComparableBound(to, true)); + + var counter = new KeySizeAndCountCollector(); // need to jump over tails keys + toFlush.processSkippingBranches(Direction.FORWARD, counter); + int partitionCount = counter.keyCount; + long partitionKeySize = counter.keySize; + + return new AbstractFlushCollection() + { + public Memtable memtable() + { + return TrieMemtableStage2.this; + } + + public PartitionPosition from() + { + return from; + } + + public PartitionPosition to() + { + return to; + } + + public long partitionCount() + { + return partitionCount; + } + + public Iterator iterator() + { + return new PartitionIterator(toFlush, metadata(), EnsureOnHeap.NOOP); + } + + public long partitionKeySize() + { + return partitionKeySize; + } + }; + } + + public static class MemtableShard + { + // The following fields are volatile as we have to make sure that when we + // collect results from all sub-ranges, the thread accessing the value + // is guaranteed to see the changes to the values. + + // The smallest timestamp for all partitions stored in this shard + private volatile long minTimestamp = Long.MAX_VALUE; + + private volatile long liveDataSize = 0; + + private volatile long currentOperations = 0; + + private volatile int partitionCount = 0; + + @Unmetered + private ReentrantLock writeLock = new ReentrantLock(SHARD_LOCK_FAIRNESS); + + // Content map for the given shard. This is implemented as a memtable trie which uses the prefix-free + // byte-comparable ByteSource representations of the keys to address the partitions. + // + // This map is used in a single-producer, multi-consumer fashion: only one thread will insert items but + // several threads may read from it and iterate over it. Iterators (especially partition range iterators) + // may operate for a long period of time and thus iterators should not throw ConcurrentModificationExceptions + // if the underlying map is modified during iteration, they should provide a weakly consistent view of the map + // instead. + // + // Also, this data is backed by memtable memory, when accessing it callers must specify if it can be accessed + // unsafely, meaning that the memtable will not be discarded as long as the data is used, or whether the data + // should be copied on heap for off-heap allocators. + @VisibleForTesting + final InMemoryTrie data; + + RegularAndStaticColumns columns; + + EncodingStats stats; + + private final MemtableAllocator allocator; + + @Unmetered + private final TrieMemtableMetricsView metrics; + + private final TableMetadataRef metadata; + + MemtableShard(TableMetadataRef metadata, TrieMemtableMetricsView metrics, OpOrder opOrder) + { + this(metadata, AbstractAllocatorMemtable.MEMORY_POOL.newAllocator(), metrics, opOrder); + } + + @VisibleForTesting + MemtableShard(TableMetadataRef metadata, MemtableAllocator allocator, TrieMemtableMetricsView metrics, OpOrder opOrder) + { + this.metadata = metadata; + this.data = InMemoryTrie.longLived(TrieBackedPartitionStage2.BYTE_COMPARABLE_VERSION, BUFFER_TYPE, opOrder); + this.columns = RegularAndStaticColumns.NONE; + this.stats = EncodingStats.NO_STATS; + this.allocator = allocator; + this.metrics = metrics; + } + + public long put(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) + { + TriePartitionUpdaterStage2 updater = new TriePartitionUpdaterStage2(allocator.cloner(opGroup), indexer, metadata.get(), this); + boolean locked = writeLock.tryLock(); + if (locked) + { + metrics.uncontendedPuts.inc(); + } + else + { + metrics.contendedPuts.inc(); + long lockStartTime = System.nanoTime(); + writeLock.lock(); + metrics.contentionTime.addNano(System.nanoTime() - lockStartTime); + } + try + { + try + { + indexer.start(); + // Add the initial trie size on the first operation. This technically isn't correct (other shards + // do take their memory share even if they are empty) but doing it during construction may cause + // the allocator to block while we are trying to flush a memtable and become a deadlock. + long onHeap = data.isEmpty() ? 0 : data.usedSizeOnHeap(); + long offHeap = data.isEmpty() ? 0 : data.usedSizeOffHeap(); + // Use the fast recursive put if we know the key is small enough to not cause a stack overflow. + try + { + data.apply(TriePartitionUpdateStage2.asMergableTrie(update), + updater, + FORCE_COPY_PARTITION_BOUNDARY); + } + catch (TrieSpaceExhaustedException e) + { + // This should never really happen as a flush would be triggered long before this limit is reached. + throw new AssertionError(e); + } + allocator.offHeap().adjust(data.usedSizeOffHeap() - offHeap, opGroup); + allocator.onHeap().adjust((data.usedSizeOnHeap() - onHeap) + updater.heapSize, opGroup); + partitionCount += updater.partitionsAdded; + } + finally + { + indexer.commit(); + updateMinTimestamp(update.stats().minTimestamp); + updateLiveDataSize(updater.dataSize); + updateCurrentOperations(update.operationCount()); + + columns = columns.mergeTo(update.columns()); + stats = stats.mergeWith(update.stats()); + } + } + finally + { + writeLock.unlock(); + } + return updater.colUpdateTimeDelta; + } + + public boolean isEmpty() + { + return data.isEmpty(); + } + + private void updateMinTimestamp(long timestamp) + { + if (timestamp < minTimestamp) + minTimestamp = timestamp; + } + + void updateLiveDataSize(long size) + { + liveDataSize = liveDataSize + size; + } + + private void updateCurrentOperations(long op) + { + currentOperations = currentOperations + op; + } + + public int partitionCount() + { + return partitionCount; + } + + long liveDataSize() + { + return liveDataSize; + } + + long currentOperations() + { + return currentOperations; + } + + private DecoratedKey firstPartitionKey(Direction direction) + { + Iterator> iter = data.filteredEntryIterator(direction, PartitionData.class); + if (!iter.hasNext()) + return null; + + Map.Entry entry = iter.next(); + return getPartitionKeyFromPath(metadata.get(), entry.getKey()); + } + + public DecoratedKey minPartitionKey() + { + return firstPartitionKey(Direction.FORWARD); + } + + public DecoratedKey maxPartitionKey() + { + return firstPartitionKey(Direction.REVERSE); + } + } + + static class PartitionIterator extends TrieTailsIterator.Plain + { + final TableMetadata metadata; + final EnsureOnHeap ensureOnHeap; + PartitionIterator(Trie source, TableMetadata metadata, EnsureOnHeap ensureOnHeap) + { + super(source, Direction.FORWARD, PartitionData.class::isInstance); + this.metadata = metadata; + this.ensureOnHeap = ensureOnHeap; + } + + @Override + protected TrieBackedPartitionStage2 mapContent(Object content, Trie tailTrie, byte[] bytes, int byteLength) + { + PartitionData pd = (PartitionData) content; + DecoratedKey key = getPartitionKeyFromPath(metadata, + ByteComparable.preencoded(TrieBackedPartitionStage2.BYTE_COMPARABLE_VERSION, + bytes, 0, byteLength)); + return TrieBackedPartitionStage2.create(key, + pd.columns(), + pd.stats(), + pd.rowCountIncludingStatic(), + tailTrie, + metadata, + ensureOnHeap); + } + } + + static class MemtableUnfilteredPartitionIterator extends AbstractUnfilteredPartitionIterator implements Memtable.MemtableUnfilteredPartitionIterator + { + private final TableMetadata metadata; + private final Iterator iter; + private final ColumnFilter columnFilter; + private final DataRange dataRange; + private final int minLocalDeletionTime; + + public MemtableUnfilteredPartitionIterator(TableMetadata metadata, + EnsureOnHeap ensureOnHeap, + Trie source, + ColumnFilter columnFilter, + DataRange dataRange, + int minLocalDeletionTime) + { + this.iter = new PartitionIterator(source, metadata, ensureOnHeap); + this.metadata = metadata; + this.columnFilter = columnFilter; + this.dataRange = dataRange; + this.minLocalDeletionTime = minLocalDeletionTime; + } + + public int getMinLocalDeletionTime() + { + return minLocalDeletionTime; + } + + public TableMetadata metadata() + { + return metadata; + } + + public boolean hasNext() + { + return iter.hasNext(); + } + + public UnfilteredRowIterator next() + { + Partition partition = iter.next(); + DecoratedKey key = partition.partitionKey(); + ClusteringIndexFilter filter = dataRange.clusteringIndexFilter(key); + + return filter.getUnfilteredRowIterator(columnFilter, partition); + } + } + + static class Factory implements Memtable.Factory + { + public Memtable create(AtomicReference commitLogLowerBound, + TableMetadataRef metadaRef, + Owner owner) + { + return new TrieMemtableStage2(commitLogLowerBound, metadaRef, owner); + } + + @Override + public PartitionUpdate.Factory partitionUpdateFactory() + { + return TriePartitionUpdateStage2.FACTORY; + } + + @Override + public TableMetrics.ReleasableMetric createMemtableMetrics(TableMetadataRef metadataRef) + { + TrieMemtableMetricsView metrics = TrieMemtableMetricsView.getOrCreate(metadataRef.keyspace, metadataRef.name); + return metrics::release; + } + } + + @Override + public long unusedReservedOnHeapMemory() + { + long size = 0; + for (MemtableShard shard : shards) + { + size += shard.data.unusedReservedOnHeapMemory(); + size += shard.allocator.unusedReservedOnHeapMemory(); + } + size += this.allocator.unusedReservedOnHeapMemory(); + return size; + } + + /** + * Release all recycled content references, including the ones waiting in still incomplete recycling lists. + * This is a test method and can cause null pointer exceptions if used on a live trie. + */ + @VisibleForTesting + void releaseReferencesUnsafe() + { + for (MemtableShard shard : shards) + shard.data.releaseReferencesUnsafe(); + } + + @VisibleForTesting + public static class TrieMemtableConfig implements TrieMemtableConfigMXBean + { + @Override + public void setShardCount(String shardCount) + { + if ("auto".equalsIgnoreCase(shardCount)) + { + SHARD_COUNT = autoShardCount(); + CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_COUNT.setInt(SHARD_COUNT); + } + else + { + try + { + SHARD_COUNT = Integer.valueOf(shardCount); + CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_COUNT.setInt(SHARD_COUNT); + } + catch (NumberFormatException ex) + { + logger.warn("Unable to parse {} as valid value for shard count; leaving it as {}", + shardCount, SHARD_COUNT); + return; + } + } + logger.info("Requested setting shard count to {}; set to: {}", shardCount, SHARD_COUNT); + } + + @Override + public String getShardCount() + { + return "" + SHARD_COUNT; + } + + @Override + public void setLockFairness(String fairness) + { + SHARD_LOCK_FAIRNESS = Boolean.parseBoolean(fairness); + CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_LOCK_FAIRNESS.setBoolean(SHARD_LOCK_FAIRNESS); + logger.info("Requested setting shard lock fairness to {}; set to: {}", fairness, SHARD_LOCK_FAIRNESS); + } + + @Override + public String getLockFairness() + { + return "" + SHARD_LOCK_FAIRNESS; + } + } + +} diff --git a/src/java/org/apache/cassandra/db/partitions/TrieBackedPartitionStage2.java b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartitionStage2.java new file mode 100644 index 000000000000..c7c2136f0be5 --- /dev/null +++ b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartitionStage2.java @@ -0,0 +1,709 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.partitions; + +import java.util.Iterator; +import java.util.NavigableSet; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.primitives.Ints; + +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringBound; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.ClusteringPrefix; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.LivenessInfo; +import org.apache.cassandra.db.MutableDeletionInfo; +import org.apache.cassandra.db.RangeTombstone; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.Slice; +import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.ByteBufferAccessor; +import org.apache.cassandra.db.rows.AbstractUnfilteredRowIterator; +import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.rows.ColumnData; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.RangeTombstoneMarker; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.RowAndDeletionMergeIterator; +import org.apache.cassandra.db.rows.Rows; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.rows.UnfilteredRowIterators; +import org.apache.cassandra.db.tries.Direction; +import org.apache.cassandra.db.tries.InMemoryTrie; +import org.apache.cassandra.db.tries.Trie; +import org.apache.cassandra.db.tries.TrieEntriesIterator; +import org.apache.cassandra.db.tries.TrieSpaceExhaustedException; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.AbstractIterator; +import org.apache.cassandra.utils.ObjectSizes; +import org.apache.cassandra.utils.btree.BTree; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.memory.Cloner; +import org.apache.cassandra.utils.memory.EnsureOnHeap; + +/** + * In-memory partition backed by a trie. The rows of the partition are values in the leaves of the trie, where the key + * to the row is only stored as the path to reach that leaf; static rows are also treated as a row with STATIC_CLUSTERING + * path; the deletion information is placed as a metadata object at the root of the trie -- this matches how Memtable + * stores partitions within the larger map, so that TrieBackedPartition objects can be created directly from Memtable + * tail tries. + * + * This object also holds the partition key, as well as some metadata (columns and statistics). + * + * Currently all descendants and instances of this class are immutable (even tail tries from mutable memtables are + * guaranteed to not change as we use forced copying below the partition level), though this may change in the future. + */ +public class TrieBackedPartitionStage2 implements Partition +{ + /** + * If keys are below this length, we will use a recursive procedure for inserting data when building the backing + * trie. + */ + @VisibleForTesting + public static final int MAX_RECURSIVE_KEY_LENGTH = 128; + + public static final ByteComparable.Version BYTE_COMPARABLE_VERSION = ByteComparable.Version.OSS50; + + /** Pre-made path for STATIC_CLUSTERING, to avoid creating path object when querying static path. */ + public static final ByteComparable STATIC_CLUSTERING_PATH = v -> ByteSource.oneByte(ClusteringPrefix.Kind.STATIC_CLUSTERING.asByteComparableValue(v)); + /** Pre-made path for BOTTOM, to avoid creating path object when iterating rows. */ + public static final ByteComparable BOTTOM_PATH = v -> ByteSource.oneByte(ClusteringPrefix.Kind.INCL_START_BOUND.asByteComparableValue(v)); + + /** + * The representation of a row stored at the leaf of a trie. Does not contain the row key. + * + * The methods toRow and copyToOnHeapRow combine this with a clustering for the represented Row. + */ + public static class RowData + { + final Object[] columnsBTree; + final LivenessInfo livenessInfo; + final DeletionTime deletion; + final int minLocalDeletionTime; + + RowData(Object[] columnsBTree, LivenessInfo livenessInfo, DeletionTime deletion) + { + this(columnsBTree, livenessInfo, deletion, BTreeRow.minDeletionTime(columnsBTree, livenessInfo, deletion)); + } + + RowData(Object[] columnsBTree, LivenessInfo livenessInfo, DeletionTime deletion, int minLocalDeletionTime) + { + this.columnsBTree = columnsBTree; + this.livenessInfo = livenessInfo; + this.deletion = deletion; + this.minLocalDeletionTime = minLocalDeletionTime; + } + + Row toRow(Clustering clustering) + { + return BTreeRow.create(clustering, + livenessInfo, + Row.Deletion.regular(deletion), + columnsBTree, + minLocalDeletionTime); + } + + public int dataSize() + { + int dataSize = livenessInfo.dataSize() + deletion.dataSize(); + + return Ints.checkedCast(BTree.accumulate(columnsBTree, (ColumnData cd, long v) -> v + cd.dataSize(), dataSize)); + } + + public long unsharedHeapSizeExcludingData() + { + long heapSize = EMPTY_ROWDATA_SIZE + + BTree.sizeOfStructureOnHeap(columnsBTree) + + livenessInfo.unsharedHeapSize() + + deletion.unsharedHeapSize(); + + return BTree.accumulate(columnsBTree, (ColumnData cd, long v) -> v + cd.unsharedHeapSizeExcludingData(), heapSize); + } + + public String toString() + { + return "row " + livenessInfo + " size " + dataSize(); + } + + public RowData clone(Cloner cloner) + { + Object[] tree = BTree.transform(columnsBTree, c -> c.clone(cloner)); + return new RowData(tree, livenessInfo, deletion, minLocalDeletionTime); + } + } + + private static final long EMPTY_ROWDATA_SIZE = ObjectSizes.measure(new RowData(null, null, null, 0)); + + protected final Trie trie; + protected final DecoratedKey partitionKey; + protected final TableMetadata metadata; + protected final RegularAndStaticColumns columns; + protected final EncodingStats stats; + protected final int rowCountIncludingStatic; + protected final boolean canHaveShadowedData; + + public TrieBackedPartitionStage2(DecoratedKey partitionKey, + RegularAndStaticColumns columns, + EncodingStats stats, + int rowCountIncludingStatic, + Trie trie, + TableMetadata metadata, + boolean canHaveShadowedData) + { + this.partitionKey = partitionKey; + this.trie = trie; + this.metadata = metadata; + this.columns = columns; + this.stats = stats; + this.rowCountIncludingStatic = rowCountIncludingStatic; + this.canHaveShadowedData = canHaveShadowedData; + // There must always be deletion info metadata. + // Note: we can't use deletionInfo() because WithEnsureOnHeap's override is not yet set up. + assert trie.get(ByteComparable.EMPTY) != null; + assert stats != null; + } + + public static TrieBackedPartitionStage2 fromIterator(UnfilteredRowIterator iterator) + { + ContentBuilder builder = build(iterator, false); + return new TrieBackedPartitionStage2(iterator.partitionKey(), + iterator.columns(), + iterator.stats(), + builder.rowCountIncludingStatic(), + builder.trie(), + iterator.metadata(), + false); + } + + protected static ContentBuilder build(UnfilteredRowIterator iterator, boolean collectDataSize) + { + try + { + ContentBuilder builder = new ContentBuilder(iterator.metadata(), iterator.partitionLevelDeletion(), iterator.isReverseOrder(), collectDataSize); + + builder.addStatic(iterator.staticRow()); + + while (iterator.hasNext()) + builder.addUnfiltered(iterator.next()); + + return builder.complete(); + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + } + + /** + * Create a row with the given properties and content, making sure to copy all off-heap data to keep it alive when + * the given access mode requires it. + */ + public static TrieBackedPartitionStage2 create(DecoratedKey partitionKey, + RegularAndStaticColumns columnMetadata, + EncodingStats encodingStats, + int rowCountIncludingStatic, + Trie trie, + TableMetadata metadata, + EnsureOnHeap ensureOnHeap) + { + return ensureOnHeap == EnsureOnHeap.NOOP + ? new TrieBackedPartitionStage2(partitionKey, columnMetadata, encodingStats, rowCountIncludingStatic, trie, metadata, true) + : new WithEnsureOnHeap(partitionKey, columnMetadata, encodingStats, rowCountIncludingStatic, trie, metadata, true, ensureOnHeap); + } + + class RowIterator extends TrieEntriesIterator + { + public RowIterator(Trie trie, Direction direction) + { + super(trie, direction, RowData.class::isInstance); + } + + @Override + protected Row mapContent(Object content, byte[] bytes, int byteLength) + { + var rd = (RowData) content; + return toRow(rd, + metadata.comparator.clusteringFromByteComparable( + ByteBufferAccessor.instance, + ByteComparable.preencoded(BYTE_COMPARABLE_VERSION, bytes, 0, byteLength))); + } + } + + private Iterator rowIterator(Trie trie, Direction direction) + { + return new RowIterator(trie, direction); + } + + static RowData rowToData(Row row) + { + BTreeRow brow = (BTreeRow) row; + return new RowData(brow.getBTree(), row.primaryKeyLivenessInfo(), row.deletion().time(), brow.getMinLocalDeletionTime()); + } + + /** + * Conversion from RowData to Row. TrieBackedPartitionOnHeap overrides this to do the necessary copying + * (hence the non-static method). + */ + Row toRow(RowData data, Clustering clustering) + { + return data.toRow(clustering); + } + + /** + * Put the given unfiltered in the trie. + * @param comparator for converting key to byte-comparable + * @param useRecursive whether the key length is guaranteed short and recursive put can be used + * @param trie destination + * @param row content to put + */ + protected static void putInTrie(ClusteringComparator comparator, boolean useRecursive, InMemoryTrie trie, Row row) throws TrieSpaceExhaustedException + { + trie.putSingleton(comparator.asByteComparable(row.clustering()), rowToData(row), NO_CONFLICT_RESOLVER, useRecursive); + } + + /** + * Check if we can use recursive operations when putting a value in tries. + * True if all types in the clustering keys are fixed length, and total size is small enough. + */ + protected static boolean useRecursive(ClusteringComparator comparator) + { + int length = 1; // terminator + for (AbstractType type : comparator.subtypes()) + if (!type.isValueLengthFixed()) + return false; + else + length += 1 + type.valueLengthIfFixed(); // separator + value + + return length <= MAX_RECURSIVE_KEY_LENGTH; + } + + public TableMetadata metadata() + { + return metadata; + } + + public DecoratedKey partitionKey() + { + return partitionKey; + } + + public DeletionTime partitionLevelDeletion() + { + return deletionInfo().getPartitionDeletion(); + } + + public RegularAndStaticColumns columns() + { + return columns; + } + + public EncodingStats stats() + { + return stats; + } + + public int rowCount() + { + return rowCountIncludingStatic - (hasStaticRow() ? 1 : 0); + } + + public DeletionInfo deletionInfo() + { + return (DeletionInfo) trie.get(ByteComparable.EMPTY); + } + + public ByteComparable path(ClusteringPrefix clustering) + { + return metadata.comparator.asByteComparable(clustering); + } + + public Row staticRow() + { + RowData staticRow = (RowData) trie.get(STATIC_CLUSTERING_PATH); + + if (staticRow != null) + return toRow(staticRow, Clustering.STATIC_CLUSTERING); + else + return Rows.EMPTY_STATIC_ROW; + } + + public boolean isEmpty() + { + return rowCountIncludingStatic == 0 && deletionInfo().isLive(); + } + + private boolean hasStaticRow() + { + return trie.get(STATIC_CLUSTERING_PATH) != null; + } + + public boolean hasRows() + { + return rowCountIncludingStatic > 1 || rowCountIncludingStatic > 0 && !hasStaticRow(); + } + + /** + * Provides read access to the trie for users that can take advantage of it directly (e.g. Memtable). + */ + public Trie trie() + { + return trie; + } + + private Trie nonStaticSubtrie() + { + // skip static row if present - the static clustering sorts before BOTTOM so that it's never included in + // any slices (we achieve this by using the byte ByteSource.EXCLUDED for its representation, which is lower + // than BOTTOM's ByteSource.LT_NEXT_COMPONENT). + return trie.subtrie(BOTTOM_PATH, null); + } + + public Iterator rowIterator() + { + return rowIterator(nonStaticSubtrie(), Direction.FORWARD); + } + + public Iterator rowsIncludingStatic() + { + return rowIterator(trie, Direction.FORWARD); + } + + @Override + public Row lastRow() + { + Iterator reverseIterator = rowIterator(nonStaticSubtrie(), Direction.REVERSE); + return reverseIterator.hasNext() ? reverseIterator.next() : null; + } + + public Row getRow(Clustering clustering) + { + RowData data = (RowData) trie.get(path(clustering)); + + DeletionInfo deletionInfo = deletionInfo(); + RangeTombstone rt = deletionInfo.rangeCovering(clustering); + + // The trie only contains rows, so it doesn't allow to directly account for deletion that should apply to row + // (the partition deletion or the deletion of a range tombstone that covers it). So if needs be, reuse the row + // deletion to carry the proper deletion on the row. + DeletionTime partitionDeletion = deletionInfo.getPartitionDeletion(); + DeletionTime activeDeletion = partitionDeletion; + if (rt != null && rt.deletionTime().supersedes(activeDeletion)) + activeDeletion = rt.deletionTime(); + + if (data == null) + { + // this means our partition level deletion supersedes all other deletions and we don't have to keep the row deletions + if (activeDeletion == partitionDeletion) + return null; + // no need to check activeDeletion.isLive here - if anything superseedes the partitionDeletion + // it must be non-live + return BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(activeDeletion)); + } + + Row row = toRow(data, clustering); + if (!activeDeletion.isLive()) + row = row.filter(ColumnFilter.selection(columns()), activeDeletion, true, metadata()); + return row; + } + + public UnfilteredRowIterator unfilteredIterator() + { + return unfilteredIterator(ColumnFilter.selection(columns()), Slices.ALL, false); + } + + public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, Slices slices, boolean reversed) + { + Row staticRow = staticRow(selection, false); + if (slices.size() == 0) + { + DeletionTime partitionDeletion = deletionInfo().getPartitionDeletion(); + return UnfilteredRowIterators.noRowsIterator(metadata(), partitionKey(), staticRow, partitionDeletion, reversed); + } + + return slices.size() == 1 + ? sliceIterator(selection, slices.get(0), reversed, staticRow) + : new SlicesIterator(selection, slices, reversed, staticRow); + } + + public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, NavigableSet> clusteringsInQueryOrder, boolean reversed) + { + Row staticRow = staticRow(selection, false); + if (clusteringsInQueryOrder.isEmpty()) + { + DeletionTime partitionDeletion = deletionInfo().getPartitionDeletion(); + return UnfilteredRowIterators.noRowsIterator(metadata(), partitionKey(), staticRow, partitionDeletion, reversed); + } + + Iterator rowIter = new AbstractIterator() { + + Iterator> clusterings = clusteringsInQueryOrder.iterator(); + + @Override + protected Row computeNext() + { + while (clusterings.hasNext()) + { + Clustering clustering = clusterings.next(); + Object rowData = trie.get(path(clustering)); + if (rowData instanceof RowData) + return toRow((RowData) rowData, clustering); + } + return endOfData(); + } + }; + + // not using DeletionInfo.rangeCovering(Clustering), because it returns the original range tombstone, + // but we need DeletionInfo.rangeIterator(Set) that generates tombstones based on given clustering bound. + Iterator deleteIter = deletionInfo().rangeIterator(clusteringsInQueryOrder, reversed); + + return merge(rowIter, deleteIter, selection, reversed, staticRow); + } + + private UnfilteredRowIterator sliceIterator(ColumnFilter selection, Slice slice, boolean reversed, Row staticRow) + { + ClusteringBound start = slice.start(); + ClusteringBound end = slice.end() == ClusteringBound.TOP ? null : slice.end(); + Iterator rowIter = slice(start, end, reversed); + Iterator deleteIter = deletionInfo().rangeIterator(slice, reversed); + return merge(rowIter, deleteIter, selection, reversed, staticRow); + } + + private Iterator slice(ClusteringBound start, ClusteringBound end, boolean reversed) + { + ByteComparable endPath = end != null ? path(end) : null; + // use BOTTOM as bound to skip over static rows + ByteComparable startPath = start != null ? path(start) : BOTTOM_PATH; + return rowIterator(trie.subtrie(startPath, endPath), Direction.fromBoolean(reversed)); + } + + private Row staticRow(ColumnFilter columns, boolean setActiveDeletionToRow) + { + DeletionTime partitionDeletion = deletionInfo().getPartitionDeletion(); + Row staticRow = staticRow(); + if (columns.fetchedColumns().statics.isEmpty() || (staticRow.isEmpty() && partitionDeletion.isLive())) + return Rows.EMPTY_STATIC_ROW; + + Row row = staticRow.filter(columns, partitionDeletion, setActiveDeletionToRow, metadata()); + return row == null ? Rows.EMPTY_STATIC_ROW : row; + } + + private RowAndDeletionMergeIterator merge(Iterator rowIter, Iterator deleteIter, + ColumnFilter selection, boolean reversed, Row staticRow) + { + return new RowAndDeletionMergeIterator(metadata(), partitionKey(), deletionInfo().getPartitionDeletion(), + selection, staticRow, reversed, stats(), + rowIter, deleteIter, canHaveShadowedData); + } + + + @Override + public String toString() + { + return Partition.toString(this); + } + + class SlicesIterator extends AbstractUnfilteredRowIterator + { + private final Slices slices; + + private int idx; + private Iterator currentSlice; + private final ColumnFilter selection; + + private SlicesIterator(ColumnFilter selection, + Slices slices, + boolean isReversed, + Row staticRow) + { + super(TrieBackedPartitionStage2.this.metadata(), TrieBackedPartitionStage2.this.partitionKey(), + TrieBackedPartitionStage2.this.partitionLevelDeletion(), + selection.fetchedColumns(), staticRow, isReversed, TrieBackedPartitionStage2.this.stats()); + this.selection = selection; + this.slices = slices; + } + + protected Unfiltered computeNext() + { + while (true) + { + if (currentSlice == null) + { + if (idx >= slices.size()) + return endOfData(); + + int sliceIdx = isReverseOrder ? slices.size() - idx - 1 : idx; + currentSlice = sliceIterator(selection, slices.get(sliceIdx), isReverseOrder, Rows.EMPTY_STATIC_ROW); + idx++; + } + + if (currentSlice.hasNext()) + return currentSlice.next(); + + currentSlice = null; + } + } + } + + + /** + * An snapshot of the current TrieBackedPartition data, copied on heap when retrieved. + */ + private static final class WithEnsureOnHeap extends TrieBackedPartitionStage2 + { + final DeletionInfo onHeapDeletion; + EnsureOnHeap ensureOnHeap; + + public WithEnsureOnHeap(DecoratedKey partitionKey, + RegularAndStaticColumns columns, + EncodingStats stats, + int rowCountIncludingStatic, + Trie trie, + TableMetadata metadata, + boolean canHaveShadowedData, + EnsureOnHeap ensureOnHeap) + { + super(partitionKey, columns, stats, rowCountIncludingStatic, trie, metadata, canHaveShadowedData); + this.ensureOnHeap = ensureOnHeap; + this.onHeapDeletion = ensureOnHeap.applyToDeletionInfo(super.deletionInfo()); + } + + @Override + public Row toRow(RowData data, Clustering clustering) + { + return ensureOnHeap.applyToRow(super.toRow(data, clustering)); + } + + @Override + public DeletionInfo deletionInfo() + { + return onHeapDeletion; + } + } + + /** + * Resolver for operations with trie-backed partitions. We don't permit any overwrites/merges. + */ + public static final InMemoryTrie.UpsertTransformer NO_CONFLICT_RESOLVER = + (existing, update) -> + { + if (existing != null) + throw new AssertionError("Unique rows expected."); + return update; + }; + + /** + * Helper class for constructing tries and deletion info from an iterator or flowable partition. + * + * Note: This is not collecting any stats or columns! + */ + public static class ContentBuilder + { + final TableMetadata metadata; + final ClusteringComparator comparator; + + private final MutableDeletionInfo.Builder deletionBuilder; + private final InMemoryTrie trie; + + private final boolean useRecursive; + private final boolean collectDataSize; + + private int rowCountIncludingStatic; + private long dataSize; + + public ContentBuilder(TableMetadata metadata, DeletionTime partitionLevelDeletion, boolean isReverseOrder, boolean collectDataSize) + { + this.metadata = metadata; + this.comparator = metadata.comparator; + + this.deletionBuilder = MutableDeletionInfo.builder(partitionLevelDeletion, + comparator, + isReverseOrder); + this.trie = InMemoryTrie.shortLived(BYTE_COMPARABLE_VERSION); + + this.useRecursive = useRecursive(comparator); + this.collectDataSize = collectDataSize; + + rowCountIncludingStatic = 0; + dataSize = 0; + } + + public ContentBuilder addStatic(Row staticRow) throws TrieSpaceExhaustedException + { + if (!staticRow.isEmpty()) + return addRow(staticRow); + else + return this; + } + + public ContentBuilder addRow(Row row) throws TrieSpaceExhaustedException + { + putInTrie(comparator, useRecursive, trie, row); + ++rowCountIncludingStatic; + if (collectDataSize) + dataSize += row.dataSize(); + return this; + } + + public ContentBuilder addRangeTombstoneMarker(RangeTombstoneMarker unfiltered) + { + deletionBuilder.add(unfiltered); + return this; + } + + public ContentBuilder addUnfiltered(Unfiltered unfiltered) throws TrieSpaceExhaustedException + { + if (unfiltered.kind() == Unfiltered.Kind.ROW) + return addRow((Row) unfiltered); + else + return addRangeTombstoneMarker((RangeTombstoneMarker) unfiltered); + } + + public ContentBuilder complete() throws TrieSpaceExhaustedException + { + MutableDeletionInfo deletionInfo = deletionBuilder.build(); + trie.putRecursive(ByteComparable.EMPTY, deletionInfo, NO_CONFLICT_RESOLVER); // will throw if called more than once + // dataSize does not include the deletion info bytes + return this; + } + + public Trie trie() + { + return trie; + } + + public int rowCountIncludingStatic() + { + return rowCountIncludingStatic; + } + + public int dataSize() + { + assert collectDataSize; + return Ints.saturatedCast(dataSize); + } + } +} diff --git a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdateStage2.java b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdateStage2.java new file mode 100644 index 000000000000..b6ab8198e7da --- /dev/null +++ b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdateStage2.java @@ -0,0 +1,586 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.partitions; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.primitives.Ints; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.Columns; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.LivenessInfo; +import org.apache.cassandra.db.MutableDeletionInfo; +import org.apache.cassandra.db.RangeTombstone; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.ColumnData; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Rows; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.rows.UnfilteredRowIterators; +import org.apache.cassandra.db.tries.InMemoryTrie; +import org.apache.cassandra.db.tries.Trie; +import org.apache.cassandra.db.tries.TrieSpaceExhaustedException; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.btree.BTree; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/** + * A trie-backed PartitionUpdate. Immutable. + *

    + * Provides factories for simple variations (e.g. singleRowUpdate) and a mutable builder for constructing one. + * The builder holds a mutable trie to which content may be added in any order, also taking care of + * merging any duplicate rows, and keeping track of statistics and column coverage. + */ +public class TriePartitionUpdateStage2 extends TrieBackedPartitionStage2 implements PartitionUpdate +{ + protected static final Logger logger = LoggerFactory.getLogger(TriePartitionUpdateStage2.class); + + public static final Factory FACTORY = new TrieFactory(); + + final int dataSize; + + private TriePartitionUpdateStage2(TableMetadata metadata, + DecoratedKey key, + RegularAndStaticColumns columns, + EncodingStats stats, + int rowCountIncludingStatic, + int dataSize, + Trie trie, + boolean canHaveShadowedData) + { + super(key, columns, stats, rowCountIncludingStatic, trie, metadata, canHaveShadowedData); + this.dataSize = dataSize; + } + + private static InMemoryTrie newTrie(DeletionInfo deletion) + { + InMemoryTrie trie = InMemoryTrie.shortLived(BYTE_COMPARABLE_VERSION); + try + { + trie.putRecursive(ByteComparable.EMPTY, deletion, NO_CONFLICT_RESOLVER); + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + return trie; + } + + /** + * Creates a empty immutable partition update. + * + * @param metadata the metadata for the created update. + * @param key the partition key for the created update. + * + * @return the newly created empty (and immutable) update. + */ + public static TriePartitionUpdateStage2 emptyUpdate(TableMetadata metadata, DecoratedKey key) + { + return new TriePartitionUpdateStage2(metadata, + key, + RegularAndStaticColumns.NONE, + EncodingStats.NO_STATS, + 0, + 0, + newTrie(MutableDeletionInfo.live()), + false); + } + + /** + * Creates an immutable partition update that entirely deletes a given partition. + * + * @param metadata the metadata for the created update. + * @param key the partition key for the partition that the created update should delete. + * @param timestamp the timestamp for the deletion. + * @param nowInSec the current time in seconds to use as local deletion time for the partition deletion. + * + * @return the newly created partition deletion update. + */ + public static TriePartitionUpdateStage2 fullPartitionDelete(TableMetadata metadata, DecoratedKey key, long timestamp, int nowInSec) + { + MutableDeletionInfo deletion = new MutableDeletionInfo(timestamp, nowInSec); + return new TriePartitionUpdateStage2(metadata, + key, + RegularAndStaticColumns.NONE, + new EncodingStats(timestamp, nowInSec, LivenessInfo.NO_TTL), + 0, + 0, + newTrie(deletion), + false); + } + + /** + * Creates an immutable partition update that contains a single row update. + * + * @param metadata the metadata for the created update. + * @param key the partition key for the partition to update. + * @param row the row for the update, may be a regular or static row and cannot be null. + * + * @return the newly created partition update containing only {@code row}. + */ + public static TriePartitionUpdateStage2 singleRowUpdate(TableMetadata metadata, DecoratedKey key, Row row) + { + EncodingStats stats = EncodingStats.Collector.forRow(row); + InMemoryTrie trie = newTrie(DeletionInfo.LIVE); + + RegularAndStaticColumns columns; + if (row.isStatic()) + columns = new RegularAndStaticColumns(Columns.from(row.columns()), Columns.NONE); + else + columns = new RegularAndStaticColumns(Columns.NONE, Columns.from(row.columns())); + + try + { + putInTrie(metadata.comparator, useRecursive(metadata.comparator), trie, row); + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + + return new TriePartitionUpdateStage2(metadata, key, columns, stats, 1, row.dataSize(), trie, false); + } + + /** + * Creates an immutable partition update that contains a single row update. + * + * @param metadata the metadata for the created update. + * @param key the partition key for the partition to update. + * @param row the row for the update. + * + * @return the newly created partition update containing only {@code row}. + */ + public static TriePartitionUpdateStage2 singleRowUpdate(TableMetadata metadata, ByteBuffer key, Row row) + { + return singleRowUpdate(metadata, metadata.partitioner.decorateKey(key), row); + } + + /** + * Turns the given iterator into an update. + * + * @param iterator the iterator to turn into updates. + * + * Warning: this method does not close the provided iterator, it is up to + * the caller to close it. + */ + @SuppressWarnings("resource") + public static TriePartitionUpdateStage2 fromIterator(UnfilteredRowIterator iterator) + { + ContentBuilder builder = build(iterator, true); + + return new TriePartitionUpdateStage2(iterator.metadata(), + iterator.partitionKey(), + iterator.columns(), + iterator.stats(), + builder.rowCountIncludingStatic(), + builder.dataSize(), + builder.trie(), + false); + } + + public static TriePartitionUpdateStage2 asTrieUpdate(PartitionUpdate update) + { + if (update instanceof TriePartitionUpdateStage2) + return (TriePartitionUpdateStage2) update; + + try (UnfilteredRowIterator iterator = update.unfilteredIterator()) + { + return fromIterator(iterator); + } + } + + public static Trie asMergableTrie(PartitionUpdate update) + { + return asTrieUpdate(update).trie.prefixedBy(update.partitionKey()); + } + + /** + * Modify this update to set every timestamp for live data to {@code newTimestamp} and + * every deletion timestamp to {@code newTimestamp - 1}. + * + * There is no reason to use that except on the Paxos code path, where we need to ensure that + * anything inserted uses the ballot timestamp (to respect the order of updates decided by + * the Paxos algorithm). We use {@code newTimestamp - 1} for deletions because tombstones + * always win on timestamp equality and we don't want to delete our own insertions + * (typically, when we overwrite a collection, we first set a complex deletion to delete the + * previous collection before adding new elements. If we were to set that complex deletion + * to the same timestamp that the new elements, it would delete those elements). And since + * tombstones always wins on timestamp equality, using -1 guarantees our deletion will still + * delete anything from a previous update. + */ + @Override + public TriePartitionUpdateStage2 withUpdatedTimestamps(long newTimestamp) + { + + InMemoryTrie t = InMemoryTrie.shortLived(BYTE_COMPARABLE_VERSION); + try + { + t.apply(trie, new InMemoryTrie.UpsertTransformer() + { + public Object apply(Object shouldBeNull, Object o) + { + assert shouldBeNull == null; + if (o instanceof RowData) + return applyRowData((RowData) o); + else + return applyDeletion((DeletionInfo) o); + } + + public RowData applyRowData(RowData update) + { + LivenessInfo newInfo = update.livenessInfo.isEmpty() + ? update.livenessInfo + : update.livenessInfo.withUpdatedTimestamp(newTimestamp); + DeletionTime newDeletion = update.deletion.isLive() + ? DeletionTime.LIVE + : new DeletionTime(newTimestamp - 1, update.deletion.localDeletionTime()); + + return new RowData(BTree.transformAndFilter(update.columnsBTree, + (ColumnData cd) -> cd.updateAllTimestamp(newTimestamp)), + newInfo, newDeletion); + } + + public DeletionInfo applyDeletion(DeletionInfo update) + { + if (update.isLive()) + return update; + + MutableDeletionInfo mdi = update.mutableCopy(); + mdi.updateAllTimestamp(newTimestamp - 1); + return mdi; + } + }, x -> false); + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + return new TriePartitionUpdateStage2(metadata, partitionKey, columns, stats, rowCountIncludingStatic, dataSize, t, canHaveShadowedData); + } + + /** + * The number of "operations" contained in the update. + *

    + * This is used by {@code Memtable} to approximate how much work this update does. In practice, this + * count how many rows are updated and how many ranges are deleted by the partition update. + * + * @return the number of "operations" performed by the update. + */ + @Override + public int operationCount() + { + return rowCountIncludingStatic + + deletionInfo().rangeCount() + + (deletionInfo().getPartitionDeletion().isLive() ? 0 : 1); + } + + /** + * The size of the data contained in this update. + * + * @return the size of the data contained in this update. + */ + @Override + public int dataSize() + { + return dataSize; + } + + /** + * Validates the data contained in this update. + * + * @throws org.apache.cassandra.serializers.MarshalException if some of the data contained in this update is corrupted. + */ + @Override + public void validate() + { + for (Iterator it = rowsIncludingStatic(); it.hasNext();) + { + Row row = it.next(); + metadata().comparator.validate(row.clustering()); + for (ColumnData cd : row) + cd.validate(); + } + } + + /** + * The maximum timestamp used in this update. + * + * @return the maximum timestamp used in this update. + */ + @Override + public long maxTimestamp() + { + long maxTimestamp = deletionInfo().maxTimestamp(); + for (Iterator it = rowsIncludingStatic(); it.hasNext();) + maxTimestamp = Math.max(maxTimestamp, Rows.collectMaxTimestamp(it.next())); + + return maxTimestamp; + } + + /** + * For an update on a counter table, returns a list containing a {@code CounterMark} for + * every counter contained in the update. + * + * @return a list with counter marks for every counter in this update. + */ + @Override + public List collectCounterMarks() + { + assert metadata().isCounter(); + // We will take aliases on the rows of this update, and update them in-place. So we should be sure the + // update is now immutable for all intent and purposes. + List marks = new ArrayList<>(); + for (Iterator it = rowsIncludingStatic(); it.hasNext();) + { + Row row = it.next(); + addMarksForRow(row, marks); + } + return marks; + } + + private static void addMarksForRow(Row row, List marks) + { + for (Cell cell : row.cells()) + { + if (cell.isCounterCell()) + marks.add(new CounterMark(row, cell.column(), cell.path())); + } + } + + /** + * Builder for PartitionUpdates + * + * This class is not thread safe, but the PartitionUpdate it produces is (since it is immutable). + */ + public static class Builder implements PartitionUpdate.Builder + { + private final TableMetadata metadata; + private final DecoratedKey key; + private final MutableDeletionInfo deletionInfo; + private final boolean canHaveShadowedData; + private final RegularAndStaticColumns columns; + private final InMemoryTrie trie = InMemoryTrie.shortLived(BYTE_COMPARABLE_VERSION); + private final EncodingStats.Collector statsCollector = new EncodingStats.Collector(); + private final boolean useRecursive; + private int rowCountIncludingStatic; + private long dataSize; + + public Builder(TableMetadata metadata, + DecoratedKey key, + RegularAndStaticColumns columns) + { + this(metadata, key, columns, true, Rows.EMPTY_STATIC_ROW, DeletionInfo.LIVE); + } + + private Builder(TableMetadata metadata, + DecoratedKey key, + RegularAndStaticColumns columns, + boolean canHaveShadowedData, + Row staticRow, + DeletionInfo deletionInfo) + { + this.metadata = metadata; + this.key = key; + this.columns = columns; + this.canHaveShadowedData = canHaveShadowedData; + this.deletionInfo = deletionInfo.mutableCopy(); + useRecursive = useRecursive(metadata.comparator); + rowCountIncludingStatic = 0; + dataSize = 0; + add(staticRow); + } + + // This is wasteful, only to be used for testing. + @VisibleForTesting + public Builder(TriePartitionUpdateStage2 base) + { + this(base.metadata, base.partitionKey, base.columns(), base.canHaveShadowedData, Rows.EMPTY_STATIC_ROW, base.deletionInfo()); + for (Iterator it = base.rowsIncludingStatic(); it.hasNext();) + add(it.next()); + } + + /** + * Adds a row to this update. + *

    + * There is no particular assumption made on the order of row added to a partition update. It is further + * allowed to add the same row (more precisely, multiple row objects for the same clustering). + *

    + * Note however that the columns contained in the added row must be a subset of the columns used when + * creating this update. + * + * @param row the row to add. + */ + public void add(Row row) + { + if (row.isEmpty()) + return; + + // this assert is expensive, and possibly of limited value; we should consider removing it + // or introducing a new class of assertions for test purposes + assert (row.isStatic() ? columns().statics : columns().regulars).containsAll(row.columns()) + : (row.isStatic() ? columns().statics : columns().regulars) + " is not superset of " + row.columns(); + + try + { + trie.putSingleton(metadata.comparator.asByteComparable(row.clustering()), + row, + this::merge, + useRecursive); + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + Rows.collectStats(row, statsCollector); + } + + public void addPartitionDeletion(DeletionTime deletionTime) + { + deletionInfo.add(deletionTime); + } + + public void add(RangeTombstone range) + { + deletionInfo.add(range, metadata.comparator); + } + + public DecoratedKey partitionKey() + { + return key; + } + + public TableMetadata metadata() + { + return metadata; + } + + public TriePartitionUpdateStage2 build() + { + try + { + trie.putRecursive(ByteComparable.EMPTY, deletionInfo, NO_CONFLICT_RESOLVER); + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + deletionInfo.collectStats(statsCollector); + TriePartitionUpdateStage2 pu = new TriePartitionUpdateStage2(metadata, + partitionKey(), + columns, + statsCollector.get(), + rowCountIncludingStatic, + Ints.saturatedCast(dataSize), + trie, + canHaveShadowedData); + + return pu; + } + + RowData merge(Object existing, Row update) + { + if (existing != null) + { + // this is not expected to happen much, so going through toRow and the existing size is okay + RowData rowData = (RowData) existing; + update = Rows.merge(rowData.toRow(update.clustering()), update); + dataSize += update.dataSize() - rowData.dataSize(); + } + else + { + ++rowCountIncludingStatic; + dataSize += update.dataSize(); + } + + return rowToData(update); + } + + public RegularAndStaticColumns columns() + { + return columns; + } + + public DeletionTime partitionLevelDeletion() + { + return deletionInfo.getPartitionDeletion(); + } + + @Override + public String toString() + { + return "Builder{" + + "metadata=" + metadata + + ", key=" + key + + ", deletionInfo=" + deletionInfo + + ", canHaveShadowedData=" + canHaveShadowedData + + ", columns=" + columns + + '}'; + } + } + + public static class TrieFactory implements Factory + { + + @Override + public PartitionUpdate.Builder builder(TableMetadata metadata, DecoratedKey partitionKey, RegularAndStaticColumns columns, int initialRowCapacity) + { + return new TriePartitionUpdateStage2.Builder(metadata, partitionKey, columns); + } + + @Override + public PartitionUpdate emptyUpdate(TableMetadata metadata, DecoratedKey partitionKey) + { + return TriePartitionUpdateStage2.emptyUpdate(metadata, partitionKey); + } + + @Override + public PartitionUpdate singleRowUpdate(TableMetadata metadata, DecoratedKey valueKey, Row row) + { + return TriePartitionUpdateStage2.singleRowUpdate(metadata, valueKey, row); + } + + @Override + public PartitionUpdate fullPartitionDelete(TableMetadata metadata, DecoratedKey key, long timestamp, int nowInSec) + { + return TriePartitionUpdateStage2.fullPartitionDelete(metadata, key, timestamp, nowInSec); + } + + @Override + public PartitionUpdate fromIterator(UnfilteredRowIterator iterator) + { + return TriePartitionUpdateStage2.fromIterator(iterator); + } + + @Override + public PartitionUpdate fromIterator(UnfilteredRowIterator iterator, ColumnFilter filter) + { + return TriePartitionUpdateStage2.fromIterator(UnfilteredRowIterators.withOnlyQueriedData(iterator, filter)); + } + } +} diff --git a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdaterStage2.java b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdaterStage2.java new file mode 100644 index 000000000000..fdc489289e90 --- /dev/null +++ b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdaterStage2.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.partitions; + +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.LivenessInfo; +import org.apache.cassandra.db.marshal.ByteArrayAccessor; +import org.apache.cassandra.db.memtable.TrieMemtableStage2; +import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.tries.InMemoryTrie; +import org.apache.cassandra.index.transactions.UpdateTransaction; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.memory.Cloner; + +import static org.apache.cassandra.db.partitions.TrieBackedPartitionStage2.RowData; + +/** + * The function we provide to the trie utilities to perform any partition and row inserts and updates + */ +public final class TriePartitionUpdaterStage2 +extends BasePartitionUpdater +implements InMemoryTrie.UpsertTransformerWithKeyProducer +{ + private final UpdateTransaction indexer; + private final TableMetadata metadata; + private TrieMemtableStage2.PartitionData currentPartition; + private final TrieMemtableStage2.MemtableShard owner; + public int partitionsAdded = 0; + + public TriePartitionUpdaterStage2(Cloner cloner, + UpdateTransaction indexer, + TableMetadata metadata, + TrieMemtableStage2.MemtableShard owner) + { + super(cloner); + this.indexer = indexer; + this.metadata = metadata; + this.owner = owner; + } + + @Override + public Object apply(Object existing, Object update, InMemoryTrie.KeyProducer keyState) + { + if (update instanceof RowData) + return applyRow((RowData) existing, (RowData) update, keyState); + else if (update instanceof DeletionInfo) + return applyDeletion((TrieMemtableStage2.PartitionData) existing, (DeletionInfo) update); + else + throw new AssertionError("Unexpected update type: " + update.getClass()); + } + + /** + * Called when a row needs to be copied to the Memtable trie. + * + * @param existing Existing RowData for this clustering, or null if there isn't any. + * @param insert RowData to be inserted. + * @param keyState Used to obtain the path through which this node was reached. + * @return the insert row, or the merged row, copied using our allocator + */ + private RowData applyRow(RowData existing, RowData insert, InMemoryTrie.KeyProducer keyState) + { + if (existing == null) + { + RowData data = insert.clone(cloner); + + if (indexer != UpdateTransaction.NO_OP) + indexer.onInserted(data.toRow(clusteringFor(keyState))); + + this.dataSize += data.dataSize(); + this.heapSize += data.unsharedHeapSizeExcludingData(); + currentPartition.markInsertedRows(1); // null pointer here means a problem in applyDeletion + return data; + } + else + { + // data and heap size are updated during merge through the PostReconciliationFunction interface + RowData reconciled = merge(existing, insert); + + if (indexer != UpdateTransaction.NO_OP) + { + Clustering clustering = clusteringFor(keyState); + indexer.onUpdated(existing.toRow(clustering), reconciled.toRow(clustering)); + } + + return reconciled; + } + } + + private RowData merge(RowData existing, RowData update) + { + + LivenessInfo livenessInfo = LivenessInfo.merge(update.livenessInfo, existing.livenessInfo); + DeletionTime deletion = DeletionTime.merge(update.deletion, existing.deletion); + if (deletion.deletes(livenessInfo)) + livenessInfo = LivenessInfo.EMPTY; + + Object[] tree = BTreeRow.mergeRowBTrees(this, + existing.columnsBTree, update.columnsBTree, + deletion, existing.deletion); + return new RowData(tree, livenessInfo, deletion); + } + + private Clustering clusteringFor(InMemoryTrie.KeyProducer keyState) + { + return metadata.comparator.clusteringFromByteComparable( + ByteArrayAccessor.instance, + ByteComparable.preencoded(TrieBackedPartitionStage2.BYTE_COMPARABLE_VERSION, + keyState.getBytes(TrieMemtableStage2.IS_PARTITION_BOUNDARY))); + } + + /** + * Called at the partition boundary to merge the existing and new metadata associated with the partition. This needs + * to update the deletion time with any new deletion introduced by the update, but also make sure that the + * statistics we track for the partition (dataSize) are updated for the changes caused by merging the update's rows + * (note that this is called _after_ the rows of the partition have been merged, on the return path of the + * recursion). + * + * @param existing Any partition data already associated with the partition. + * @param update The update, always non-null. + * @return the combined partition data, copying any updated deletion information to heap. + */ + private TrieMemtableStage2.PartitionData applyDeletion(TrieMemtableStage2.PartitionData existing, DeletionInfo update) + { + if (indexer != UpdateTransaction.NO_OP) + { + if (!update.getPartitionDeletion().isLive()) + indexer.onPartitionDeletion(update.getPartitionDeletion()); + if (update.hasRanges()) + update.rangeIterator(false).forEachRemaining(indexer::onRangeTombstone); + } + + if (existing == null) + { + // Note: Always on-heap, regardless of cloner + TrieMemtableStage2.PartitionData newRef = new TrieMemtableStage2.PartitionData(update, owner); + this.heapSize += newRef.unsharedHeapSize(); + ++this.partitionsAdded; + return currentPartition = newRef; + } + + assert owner == existing.owner; + if (update.isLive() || !update.mayModify(existing)) + return currentPartition = existing; + + // Note: Always on-heap, regardless of cloner + TrieMemtableStage2.PartitionData merged = new TrieMemtableStage2.PartitionData(existing, update); + this.heapSize += merged.unsharedHeapSize() - existing.unsharedHeapSize(); + return currentPartition = merged; + } +} diff --git a/test/unit/org/apache/cassandra/cql3/MemtableQuickTest.java b/test/unit/org/apache/cassandra/cql3/MemtableQuickTest.java index 4ef9a890f033..b66ef6b922c7 100644 --- a/test/unit/org/apache/cassandra/cql3/MemtableQuickTest.java +++ b/test/unit/org/apache/cassandra/cql3/MemtableQuickTest.java @@ -63,6 +63,7 @@ public static List parameters() return ImmutableList.of("SkipListMemtable", "TrieMemtable", "TrieMemtableStage1", + "TrieMemtableStage2", "PersistentMemoryMemtable"); } diff --git a/test/unit/org/apache/cassandra/db/memtable/MemtableSizeTestBase.java b/test/unit/org/apache/cassandra/db/memtable/MemtableSizeTestBase.java index 52ab207d0a54..bf90e737024d 100644 --- a/test/unit/org/apache/cassandra/db/memtable/MemtableSizeTestBase.java +++ b/test/unit/org/apache/cassandra/db/memtable/MemtableSizeTestBase.java @@ -63,6 +63,7 @@ public static List parameters() { return ImmutableList.of("SkipListMemtable", "TrieMemtableStage1", + "TrieMemtableStage2", "TrieMemtable"); } diff --git a/test/unit/org/apache/cassandra/db/memtable/MemtableThreadedTest.java b/test/unit/org/apache/cassandra/db/memtable/MemtableThreadedTest.java index a49c1894977f..26e01548301c 100644 --- a/test/unit/org/apache/cassandra/db/memtable/MemtableThreadedTest.java +++ b/test/unit/org/apache/cassandra/db/memtable/MemtableThreadedTest.java @@ -57,6 +57,7 @@ public static List parameters() return ImmutableList.of("SkipListMemtable", "TrieMemtable", "TrieMemtableStage1", + "TrieMemtableStage2", "PersistentMemoryMemtable"); } From 83394bb059c9181ebfa926045ee9669763cb36d3 Mon Sep 17 00:00:00 2001 From: Branimir Lambov Date: Fri, 18 Jul 2025 16:40:38 +0300 Subject: [PATCH 17/22] TrieMemtable Stage 3 Implements a row-level trie memtable that uses deletion-aware tries to store deletions separately from live data, together with the associated TrieBackedPartition and TriePartitionUpdate. Every deletion is first converted to its range version (e.g. deleted rows are now represented as a WHERE ck <= x AND ck >= x, deleted partitions -- as deletions covering from LT_EXCLUDED to GT_NEXT_COMPONENT to include static and all normal rows) and then stored in the deletion path of the trie. To make tests work, all such ranges are converted back to rows and partition deletion times on conversion to UnfiteredPartitionIterator. --- .../cassandra/db/memtable/TrieMemtable.java | 219 +++--- .../db/memtable/TrieMemtableStage1.java | 2 +- .../db/memtable/TrieMemtableStage2.java | 69 +- .../RecombiningUnfilteredRowIterator.java | 231 ++++++ .../db/partitions/TrieBackedPartition.java | 672 +++++++++++------- .../db/partitions/TriePartitionUpdate.java | 257 ++++--- .../db/partitions/TriePartitionUpdater.java | 163 +++-- .../TriePartitionUpdaterStage2.java | 10 +- .../apache/cassandra/db/rows/BTreeRow.java | 5 + .../cassandra/db/rows/ComplexColumnData.java | 12 +- .../org/apache/cassandra/db/rows/Row.java | 10 +- .../db/rows/TrieTombstoneMarker.java | 63 ++ .../db/rows/TrieTombstoneMarkerImpl.java | 299 ++++++++ .../cassandra/db/tries/DeletionAwareTrie.java | 70 ++ .../db/tries/InMemoryDeletionAwareTrie.java | 8 +- .../db/tries/TrieEntriesIterator.java | 78 ++ .../index/sai/memory/TrieMemtableIndex.java | 2 +- .../index/sai/utils/RowWithSourceTable.java | 6 + .../utils/bytecomparable/ByteSource.java | 1 + .../cassandra/index/sai/LongBM25Test.java | 17 +- .../cassandra/index/sai/LongVectorTest.java | 17 +- .../entities/SecondaryIndexTest.java | 31 +- .../cassandra/db/RangeTombstoneTest.java | 52 +- .../sai/cql/FeaturesVersionSupportTest.java | 5 +- .../index/sai/cql/LuceneUpdateDeleteTest.java | 23 +- .../sai/memory/TrieMemtableIndexTestBase.java | 8 +- 26 files changed, 1664 insertions(+), 666 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java create mode 100644 src/java/org/apache/cassandra/db/rows/TrieTombstoneMarker.java create mode 100644 src/java/org/apache/cassandra/db/rows/TrieTombstoneMarkerImpl.java diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java index 4a2f0e7d052d..d5e91e66b197 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java @@ -36,8 +36,6 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DataRange; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.DeletionInfo; -import org.apache.cassandra.db.MutableDeletionInfo; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.RegularAndStaticColumns; import org.apache.cassandra.db.commitlog.CommitLogPosition; @@ -50,10 +48,13 @@ import org.apache.cassandra.db.partitions.TriePartitionUpdate; import org.apache.cassandra.db.partitions.TriePartitionUpdater; import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.TrieTombstoneMarker; import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.tries.DeletionAwareTrie; import org.apache.cassandra.db.tries.Direction; +import org.apache.cassandra.db.tries.InMemoryBaseTrie; +import org.apache.cassandra.db.tries.InMemoryDeletionAwareTrie; import org.apache.cassandra.db.tries.InMemoryTrie; -import org.apache.cassandra.db.tries.Trie; import org.apache.cassandra.db.tries.TrieEntriesWalker; import org.apache.cassandra.db.tries.TrieSpaceExhaustedException; import org.apache.cassandra.db.tries.TrieTailsIterator; @@ -74,7 +75,6 @@ import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.apache.cassandra.utils.concurrent.OpOrder; import org.apache.cassandra.utils.memory.EnsureOnHeap; -import org.apache.cassandra.utils.memory.HeapCloner; import org.apache.cassandra.utils.memory.MemtableAllocator; import org.github.jamm.Unmetered; @@ -85,7 +85,7 @@ public class TrieMemtable extends AbstractAllocatorMemtable public static final Factory FACTORY = new TrieMemtable.Factory(); - /** Buffer type to use for memtable tries (on- vs off-heap) */ + /// Buffer type to use for memtable tries (on- vs off-heap) public static final BufferType BUFFER_TYPE; static @@ -107,57 +107,58 @@ public class TrieMemtable extends AbstractAllocatorMemtable MBeanWrapper.instance.registerMBean(new TrieMemtableConfig(), TRIE_MEMTABLE_CONFIG_OBJECT_NAME, MBeanWrapper.OnException.LOG); } - /** - * Force copy checker (see InMemoryTrie.ApplyState) ensuring all modifications apply atomically and consistently to - * the whole partition. - */ - public static final Predicate> FORCE_COPY_PARTITION_BOUNDARY = features -> isPartitionBoundary(features.content()); + /// Force copy checker (see [InMemoryTrie#apply]) ensuring all modifications apply atomically and consistently to + /// the whole partition. + public static final Predicate> FORCE_COPY_PARTITION_BOUNDARY = + features -> TrieBackedPartition.isPartitionBoundary(features.content()); - public static final Predicate IS_PARTITION_BOUNDARY = TrieMemtable::isPartitionBoundary; - - // Set to true when the memtable requests a switch (e.g. for trie size limit being reached) to ensure only one - // thread calls cfs.switchMemtableIfCurrent. + /// Set to true when the memtable requests a switch (e.g. for trie size limit being reached) to ensure only one + /// thread calls cfs.switchMemtableIfCurrent. private AtomicBoolean switchRequested = new AtomicBoolean(false); - // The boundaries for the keyspace as they were calculated when the memtable is created. - // The boundaries will be NONE for system keyspaces or if StorageService is not yet initialized. - // The fact this is fixed for the duration of the memtable lifetime, guarantees we'll always pick the same core - // for the a given key, even if we race with the StorageService initialization or with topology changes. + /// The boundaries for the keyspace as they were calculated when the memtable is created. + /// The boundaries will be `NONE` for system keyspaces or if `StorageService` is not yet initialized. + /// The fact this is fixed for the duration of the memtable lifetime, guarantees we'll always pick the same core + /// for the a given key, even if we race with the `StorageService` initialization or with topology changes. @Unmetered private final ShardBoundaries boundaries; - /** - * Core-specific memtable regions. All writes must go through the specific core. The data structures used - * are concurrent-read safe, thus reads can be carried out from any thread. - */ + /// Core-specific memtable regions. All writes must go through the specific core. The data structures used + /// are concurrent-read safe, thus reads can be carried out from any thread. private final MemtableShard[] shards; - /** - * A merged view of the memtable map. Used for partition range queries and flush. - * For efficiency we serve single partition requests off the shard which offers more direct InMemoryTrie methods. - */ - private final Trie mergedTrie; + /// A merged view of the memtable map. Used for partition range queries and flush. + /// For efficiency we serve single partition requests off the shard which offers more direct [InMemoryTrie] methods. + private final DeletionAwareTrie mergedTrie; @Unmetered private final TrieMemtableMetricsView metrics; - /** - * Keeps an estimate of the average row size in this memtable, computed from a small sample of rows. - * Because computing this estimate is potentially costly, as it requires iterating the rows, - * the estimate is updated only whenever the number of operations on the memtable increases significantly from the - * last update. This estimate is not very accurate but should be ok for planning or diagnostic purposes. - */ + /// Keeps an estimate of the average row size in this memtable, computed from a small sample of rows. + /// Because computing this estimate is potentially costly, as it requires iterating the rows, + /// the estimate is updated only whenever the number of operations on the memtable increases significantly from the + /// last update. This estimate is not very accurate but should be ok for planning or diagnostic purposes. private volatile MemtableAverageRowSize estimatedAverageRowSize; - public static volatile int SHARD_COUNT = CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_COUNT.getInt(autoShardCount()); - public static volatile boolean SHARD_LOCK_FAIRNESS = CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_LOCK_FAIRNESS.getBoolean(); + private static volatile int SHARD_COUNT = CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_COUNT.getInt(autoShardCount()); + private static volatile boolean SHARD_LOCK_FAIRNESS = CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_LOCK_FAIRNESS.getBoolean(); private static int autoShardCount() { return 4 * FBUtilities.getAvailableProcessors(); } + public static int shardCount() + { + return SHARD_COUNT; + } + + public static boolean shardLockFairness() + { + return SHARD_LOCK_FAIRNESS; + } + // only to be used by init(), to setup the very first memtable for the cfs TrieMemtable(AtomicReference commitLogLowerBound, TableMetadataRef metadataRef, Owner owner) { @@ -184,12 +185,12 @@ private static MemtableShard[] generatePartitionShards(int splits, return partitionMapContainer; } - private static Trie makeMergedTrie(MemtableShard[] shards) + private static DeletionAwareTrie makeMergedTrie(MemtableShard[] shards) { - List> tries = new ArrayList<>(shards.length); + List> tries = new ArrayList<>(shards.length); for (MemtableShard shard : shards) tries.add(shard.data); - return Trie.mergeDistinct(tries); + return DeletionAwareTrie.mergeDistinct(tries); } protected Factory factory() @@ -232,12 +233,12 @@ public void discard() } } - /** - * Should only be called by ColumnFamilyStore.apply via Keyspace.apply, which supplies the appropriate - * OpOrdering. - * - * commitLogSegmentPosition should only be null if this is a secondary index, in which case it is *expected* to be null - */ + /// Should only be called by [ColumnFamilyStore#apply] via `Keyspace#apply`, which supplies the appropriate + /// [OpOrder.Group]. + /// + /// `commitLogSegmentPosition` should only be null if this is a secondary index, in which case it is *expected* to + /// be null. + @Override public long put(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) { DecoratedKey key = update.partitionKey(); @@ -313,13 +314,11 @@ public long getEstimatedAverageRowSize() return estimatedAverageRowSize.rowSize; } - /** - * Returns the minTS if one available, otherwise NO_MIN_TIMESTAMP. - * - * EncodingStats uses a synthetic epoch TS at 2015. We don't want to leak that (CASSANDRA-18118) so we return NO_MIN_TIMESTAMP instead. - * - * @return The minTS or NO_MIN_TIMESTAMP if none available - */ + /// Returns the minimum timestamp if one available, otherwise `NO_MIN_TIMESTAMP`. + /// [EncodingStats] uses a synthetic epoch TS at 2015. We don't want to leak that (CASSANDRA-18118) so we return + /// `NO_MIN_TIMESTAMP` instead. + /// + /// @return The minTS or `NO_MIN_TIMESTAMP` if none available @Override public long getMinTimestamp() { @@ -377,13 +376,6 @@ EncodingStats encodingStats() return statsCollector.get(); } - static boolean isPartitionBoundary(Object content) - { - // In the trie we use PartitionData for the root of a partition, but PartitionUpdates come with DeletionInfo. - // Both are descendants of DeletionInfo. - return content instanceof DeletionInfo; - } - public MemtableUnfilteredPartitionIterator makePartitionIterator(final ColumnFilter columnFilter, final DataRange dataRange) { AbstractBounds keyRange = dataRange.keyRange(); @@ -392,8 +384,9 @@ public MemtableUnfilteredPartitionIterator makePartitionIterator(final ColumnFil boolean includeStart = isBound || keyRange instanceof IncludingExcludingBounds; boolean includeStop = isBound || keyRange instanceof Range; - Trie subMap = mergedTrie.subtrie(toComparableBound(keyRange.left, includeStart), - toComparableBound(keyRange.right, !includeStop)); + DeletionAwareTrie subMap = + mergedTrie.subtrie(toComparableBound(keyRange.left, includeStart), + toComparableBound(keyRange.right, !includeStop)); return new MemtableUnfilteredPartitionIterator(metadata(), allocator.ensureOnHeap(), @@ -414,11 +407,11 @@ private static ByteComparable toComparableBound(PartitionPosition position, bool public Partition getPartition(DecoratedKey key) { int shardIndex = boundaries.getShardForKey(key); - Trie trie = shards[shardIndex].data.tailTrie(key); + DeletionAwareTrie trie = shards[shardIndex].data.tailTrie(key); return createPartition(metadata(), allocator.ensureOnHeap(), key, trie); } - private static TrieBackedPartition createPartition(TableMetadata metadata, EnsureOnHeap ensureOnHeap, DecoratedKey key, Trie trie) + private static TrieBackedPartition createPartition(TableMetadata metadata, EnsureOnHeap ensureOnHeap, DecoratedKey key, DeletionAwareTrie trie) { if (trie == null) return null; @@ -432,6 +425,7 @@ private static TrieBackedPartition createPartition(TableMetadata metadata, Ensur holder.columns(), holder.stats(), holder.rowCountIncludingStatic(), + holder.tombstoneCount(), trie, metadata, ensureOnHeap); @@ -444,36 +438,26 @@ private static DecoratedKey getPartitionKeyFromPath(TableMetadata metadata, Byte metadata.partitioner); } - /** - * Metadata object signifying the root node of a partition. Holds the deletion information as well as a link - * to the owning subrange, which is used for compiling statistics and column sets. - * - * Descends from MutableDeletionInfo to permit tail tries to be passed directly to TrieBackedPartition. - */ - public static class PartitionData extends MutableDeletionInfo + /// Metadata object signifying the root node of a partition. Holds row and tombstone counts as well as a link + /// to the owning subrange, which is used for compiling encoding statistics and column sets. + /// + /// Descends from [TrieBackedPartition.PartitionMarker] to permit tail tries to be passed directly to + /// [TrieBackedPartition]. + public static class PartitionData implements TrieBackedPartition.PartitionMarker { @Unmetered public final MemtableShard owner; private int rowCountIncludingStatic; + private int tombstoneCount; - public static final long HEAP_SIZE = ObjectSizes.measure(new PartitionData(DeletionInfo.LIVE, null)); + public static final long HEAP_SIZE = ObjectSizes.measure(new PartitionData((MemtableShard) null)); - public PartitionData(DeletionInfo deletion, - MemtableShard owner) + public PartitionData(MemtableShard owner) { - super(deletion.getPartitionDeletion(), deletion.copyRanges(HeapCloner.instance)); this.owner = owner; this.rowCountIncludingStatic = 0; - } - - public PartitionData(PartitionData existing, - DeletionInfo update) - { - // Start with the update content, to properly copy it - this(update, existing.owner); - rowCountIncludingStatic = existing.rowCountIncludingStatic; - add(existing); + this.tombstoneCount = 0; } public RegularAndStaticColumns columns() @@ -491,25 +475,33 @@ public int rowCountIncludingStatic() return rowCountIncludingStatic; } + public int tombstoneCount() + { + return tombstoneCount; + } + public void markInsertedRows(int howMany) { rowCountIncludingStatic += howMany; } + public void markAddedTombstones(int howMany) + { + tombstoneCount += howMany; + } + @Override public String toString() { - return "partition " + super.toString(); + return String.format("partition with %d rows and %d tombstones", rowCountIncludingStatic, tombstoneCount); } - @Override public long unsharedHeapSize() { - return super.unsharedHeapSize() + HEAP_SIZE - MutableDeletionInfo.EMPTY_SIZE; + return HEAP_SIZE; } } - class KeySizeAndCountCollector extends TrieEntriesWalker { long keySize = 0; @@ -534,9 +526,10 @@ protected void content(Object content, byte[] bytes, int byteLength) } } + @Override public FlushCollection getFlushSet(PartitionPosition from, PartitionPosition to) { - Trie toFlush = mergedTrie.subtrie(toComparableBound(from, true), toComparableBound(to, true)); + DeletionAwareTrie toFlush = mergedTrie.subtrie(toComparableBound(from, true), toComparableBound(to, true)); var counter = new KeySizeAndCountCollector(); // need to jump over tails keys toFlush.processSkippingBranches(Direction.FORWARD, counter); @@ -595,20 +588,21 @@ public static class MemtableShard @Unmetered private ReentrantLock writeLock = new ReentrantLock(SHARD_LOCK_FAIRNESS); - // Content map for the given shard. This is implemented as a memtable trie which uses the prefix-free - // byte-comparable ByteSource representations of the keys to address the partitions. - // - // This map is used in a single-producer, multi-consumer fashion: only one thread will insert items but - // several threads may read from it and iterate over it. Iterators (especially partition range iterators) - // may operate for a long period of time and thus iterators should not throw ConcurrentModificationExceptions - // if the underlying map is modified during iteration, they should provide a weakly consistent view of the map - // instead. - // - // Also, this data is backed by memtable memory, when accessing it callers must specify if it can be accessed - // unsafely, meaning that the memtable will not be discarded as long as the data is used, or whether the data - // should be copied on heap for off-heap allocators. + /// Content map for the given shard. This is implemented as an in-memory trie which uses the prefix-free + /// byte-comparable [ByteSource] representations of keys to address partitions and individual rows within + /// partitions. + /// + /// This map is used in a single-producer, multi-consumer fashion: only one thread will insert items but + /// several threads may read from it and iterate over it. Iterators (especially partition range iterators) + /// may operate for a long period of time and thus iterators should not throw `ConcurrentModificationException`s + /// if the underlying map is modified during iteration, they should provide a weakly consistent view of the map + /// instead. + /// + /// Also, this data is backed by memtable memory, when accessing it callers must specify if it can be accessed + /// unsafely, meaning that the memtable will not be discarded as long as the data is used, or whether the data + /// should be copied on heap for off-heap allocators. @VisibleForTesting - final InMemoryTrie data; + final InMemoryDeletionAwareTrie data; RegularAndStaticColumns columns; @@ -630,7 +624,7 @@ public static class MemtableShard MemtableShard(TableMetadataRef metadata, MemtableAllocator allocator, TrieMemtableMetricsView metrics, OpOrder opOrder) { this.metadata = metadata; - this.data = InMemoryTrie.longLived(TrieBackedPartition.BYTE_COMPARABLE_VERSION, BUFFER_TYPE, opOrder); + this.data = InMemoryDeletionAwareTrie.longLived(TrieBackedPartition.BYTE_COMPARABLE_VERSION, BUFFER_TYPE, opOrder); this.columns = RegularAndStaticColumns.NONE; this.stats = EncodingStats.NO_STATS; this.allocator = allocator; @@ -639,7 +633,7 @@ public static class MemtableShard public long put(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) { - TriePartitionUpdater updater = new TriePartitionUpdater(allocator.cloner(opGroup), indexer, metadata.get(), this); + TriePartitionUpdater updater = new TriePartitionUpdater(allocator.cloner(opGroup), indexer, update.partitionLevelDeletion(), metadata.get(), this); boolean locked = writeLock.tryLock(); if (locked) { @@ -662,11 +656,14 @@ public long put(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group // the allocator to block while we are trying to flush a memtable and become a deadlock. long onHeap = data.isEmpty() ? 0 : data.usedSizeOnHeap(); long offHeap = data.isEmpty() ? 0 : data.usedSizeOffHeap(); - // Use the fast recursive put if we know the key is small enough to not cause a stack overflow. try { data.apply(TriePartitionUpdate.asMergableTrie(update), updater, + updater::mergeMarkers, + updater::applyMarker, + updater::applyMarker, + true, FORCE_COPY_PARTITION_BOUNDARY); } catch (TrieSpaceExhaustedException e) @@ -734,6 +731,7 @@ long currentOperations() private DecoratedKey firstPartitionKey(Direction direction) { + // Note: there is no need to skip tails here as this will only be run until we find the first partition. Iterator> iter = data.filteredEntryIterator(direction, PartitionData.class); if (!iter.hasNext()) return null; @@ -753,11 +751,11 @@ public DecoratedKey maxPartitionKey() } } - static class PartitionIterator extends TrieTailsIterator.Plain + static class PartitionIterator extends TrieTailsIterator.DeletionAware { final TableMetadata metadata; final EnsureOnHeap ensureOnHeap; - PartitionIterator(Trie source, TableMetadata metadata, EnsureOnHeap ensureOnHeap) + PartitionIterator(DeletionAwareTrie source, TableMetadata metadata, EnsureOnHeap ensureOnHeap) { super(source, Direction.FORWARD, PartitionData.class::isInstance); this.metadata = metadata; @@ -765,7 +763,7 @@ static class PartitionIterator extends TrieTailsIterator.Plain tailTrie, byte[] bytes, int byteLength) + protected TrieBackedPartition mapContent(Object content, DeletionAwareTrie tailTrie, byte[] bytes, int byteLength) { PartitionData pd = (PartitionData) content; DecoratedKey key = getPartitionKeyFromPath(metadata, @@ -775,6 +773,7 @@ protected TrieBackedPartition mapContent(Object content, Trie tailTrie, pd.columns(), pd.stats(), pd.rowCountIncludingStatic(), + pd.tombstoneCount(), tailTrie, metadata, ensureOnHeap); @@ -791,7 +790,7 @@ static class MemtableUnfilteredPartitionIterator extends AbstractUnfilteredParti public MemtableUnfilteredPartitionIterator(TableMetadata metadata, EnsureOnHeap ensureOnHeap, - Trie source, + DeletionAwareTrie source, ColumnFilter columnFilter, DataRange dataRange, int minLocalDeletionTime) @@ -864,10 +863,8 @@ public long unusedReservedOnHeapMemory() return size; } - /** - * Release all recycled content references, including the ones waiting in still incomplete recycling lists. - * This is a test method and can cause null pointer exceptions if used on a live trie. - */ + /// Release all recycled content references, including the ones waiting in still incomplete recycling lists. + /// This is a test method and can cause null pointer exceptions if used on a live trie. @VisibleForTesting void releaseReferencesUnsafe() { diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java index d77b86b20bd1..f9e98266a112 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage1.java @@ -143,7 +143,7 @@ public class TrieMemtableStage1 extends AbstractAllocatorMemtable TrieMemtableStage1(AtomicReference commitLogLowerBound, TableMetadataRef metadataRef, Owner owner) { super(commitLogLowerBound, metadataRef, owner); - this.boundaries = owner.localRangeSplits(TrieMemtable.SHARD_COUNT); + this.boundaries = owner.localRangeSplits(TrieMemtable.shardCount()); this.metrics = TrieMemtableMetricsView.getOrCreate(metadataRef.keyspace, metadataRef.name); this.shards = generatePartitionShards(boundaries.shardCount(), metadataRef, metrics); this.mergedTrie = makeMergedTrie(shards); diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java index e286bce472a5..e81f43ee4321 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java @@ -52,6 +52,7 @@ import org.apache.cassandra.db.rows.EncodingStats; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.tries.Direction; +import org.apache.cassandra.db.tries.InMemoryBaseTrie; import org.apache.cassandra.db.tries.InMemoryTrie; import org.apache.cassandra.db.tries.Trie; import org.apache.cassandra.db.tries.TrieEntriesWalker; @@ -67,8 +68,6 @@ import org.apache.cassandra.metrics.TrieMemtableMetricsView; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableMetadataRef; -import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.MBeanWrapper; import org.apache.cassandra.utils.ObjectSizes; import org.apache.cassandra.utils.bytecomparable.ByteComparable; import org.apache.cassandra.utils.bytecomparable.ByteSource; @@ -120,15 +119,13 @@ public class TrieMemtableStage2 extends AbstractAllocatorMemtable default: throw new AssertionError(); } - - MBeanWrapper.instance.registerMBean(new TrieMemtableConfig(), TRIE_MEMTABLE_CONFIG_OBJECT_NAME, MBeanWrapper.OnException.LOG); } /** * Force copy checker (see InMemoryTrie.ApplyState) ensuring all modifications apply atomically and consistently to * the whole partition. */ - public static final Predicate> FORCE_COPY_PARTITION_BOUNDARY = features -> isPartitionBoundary(features.content()); + public static final Predicate> FORCE_COPY_PARTITION_BOUNDARY = features -> isPartitionBoundary(features.content()); public static final Predicate IS_PARTITION_BOUNDARY = TrieMemtableStage2::isPartitionBoundary; @@ -167,19 +164,11 @@ public class TrieMemtableStage2 extends AbstractAllocatorMemtable */ private volatile MemtableAverageRowSize estimatedAverageRowSize; - public static volatile int SHARD_COUNT = CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_COUNT.getInt(autoShardCount()); - public static volatile boolean SHARD_LOCK_FAIRNESS = CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_LOCK_FAIRNESS.getBoolean(); - - private static int autoShardCount() - { - return 4 * FBUtilities.getAvailableProcessors(); - } - // only to be used by init(), to setup the very first memtable for the cfs TrieMemtableStage2(AtomicReference commitLogLowerBound, TableMetadataRef metadataRef, Owner owner) { super(commitLogLowerBound, metadataRef, owner); - this.boundaries = owner.localRangeSplits(SHARD_COUNT); + this.boundaries = owner.localRangeSplits(TrieMemtable.shardCount()); this.metrics = TrieMemtableMetricsView.getOrCreate(metadataRef.keyspace, metadataRef.name); this.shards = generatePartitionShards(boundaries.shardCount(), metadataRef, metrics, owner.readOrdering()); this.mergedTrie = makeMergedTrie(shards); @@ -610,7 +599,7 @@ public static class MemtableShard private volatile int partitionCount = 0; @Unmetered - private ReentrantLock writeLock = new ReentrantLock(SHARD_LOCK_FAIRNESS); + private ReentrantLock writeLock = new ReentrantLock(TrieMemtable.shardLockFairness()); // Content map for the given shard. This is implemented as a memtable trie which uses the prefix-free // byte-comparable ByteSource representations of the keys to address the partitions. @@ -891,54 +880,4 @@ void releaseReferencesUnsafe() for (MemtableShard shard : shards) shard.data.releaseReferencesUnsafe(); } - - @VisibleForTesting - public static class TrieMemtableConfig implements TrieMemtableConfigMXBean - { - @Override - public void setShardCount(String shardCount) - { - if ("auto".equalsIgnoreCase(shardCount)) - { - SHARD_COUNT = autoShardCount(); - CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_COUNT.setInt(SHARD_COUNT); - } - else - { - try - { - SHARD_COUNT = Integer.valueOf(shardCount); - CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_COUNT.setInt(SHARD_COUNT); - } - catch (NumberFormatException ex) - { - logger.warn("Unable to parse {} as valid value for shard count; leaving it as {}", - shardCount, SHARD_COUNT); - return; - } - } - logger.info("Requested setting shard count to {}; set to: {}", shardCount, SHARD_COUNT); - } - - @Override - public String getShardCount() - { - return "" + SHARD_COUNT; - } - - @Override - public void setLockFairness(String fairness) - { - SHARD_LOCK_FAIRNESS = Boolean.parseBoolean(fairness); - CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_LOCK_FAIRNESS.setBoolean(SHARD_LOCK_FAIRNESS); - logger.info("Requested setting shard lock fairness to {}; set to: {}", fairness, SHARD_LOCK_FAIRNESS); - } - - @Override - public String getLockFairness() - { - return "" + SHARD_LOCK_FAIRNESS; - } - } - } diff --git a/src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java new file mode 100644 index 000000000000..93faf962f84e --- /dev/null +++ b/src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.partitions; + +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringPrefix; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.rows.RangeTombstoneBoundaryMarker; +import org.apache.cassandra.db.rows.RangeTombstoneMarker; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.rows.WrappingUnfilteredRowIterator; + +/// An [UnfilteredRowIterator] that recombines sequences of range tombstones for the same key into deleted rows. +/// +/// The objective of this class is to reverse the transformation made by [TriePartitionUpdate] that implements row +/// deletions as pairs of tombstones around the row (which are then placed in the deletion branch of the trie). +/// This transformation is valid, but a lot of tests rely on row deletions being represented by empty deleted rows. +/// For the time being we thus do the reverse transformation on conversion from trie to unfiltered iterator. +class RecombiningUnfilteredRowIterator extends WrappingUnfilteredRowIterator +{ + Unfiltered bufferedOne; + Unfiltered bufferedTwo; + Unfiltered next; + boolean nextPrepared; + + protected RecombiningUnfilteredRowIterator(UnfilteredRowIterator wrapped) + { + super(wrapped); + bufferedOne = null; + nextPrepared = false; + } + + @Override + public boolean hasNext() + { + return computeNext() != null; + } + + @Override + public Unfiltered next() + { + Unfiltered item = computeNext(); + nextPrepared = false; + return item; + } + + private Unfiltered computeNext() + { + if (nextPrepared) + return next; + + // If we have two buffered entries, report the first one directly (there's no need to process it as we already + // know it is a row) and shift the second one to the first position. + if (bufferedTwo != null) + { + Unfiltered unfiltered2 = bufferedTwo; + bufferedTwo = null; + return setNextAndBufferedAndReturn(bufferedOne, unfiltered2); + } + + // If we have a buffered entry, use it for the following processing, otherwise get one from the source. + Unfiltered unfiltered1; + if (bufferedOne != null) + { + unfiltered1 = bufferedOne; + bufferedOne = null; + } + else + { + if (!wrapped.hasNext()) + return setNextAndReturn(null); + + unfiltered1 = wrapped.next(); + } + + // The pattern we are looking for is + // open_incusive(clustering, del) + row(clustering) + close_inclusive(clustering, del) + // where the row is optional + + if (unfiltered1.isRow()) + return setNextAndReturn(unfiltered1); + + RangeTombstoneMarker marker1 = (RangeTombstoneMarker) unfiltered1; + boolean reversed = isReverseOrder(); + int clusteringSize = metadata().comparator.size(); + // The first marker must be open, inclusive, and a fully specified clustering. + if (!marker1.isOpen(reversed) + || !marker1.openIsInclusive(reversed) + || marker1.clustering().size() != clusteringSize + || (clusteringSize > 0 && marker1.clustering().get(clusteringSize - 1) == null)) + return setNextAndReturn(marker1); + + if (!wrapped.hasNext()) + return setNextAndReturn(marker1); + + Unfiltered unfiltered2 = wrapped.next(); + final DeletionTime deletionTime = marker1.openDeletionTime(reversed); + if (unfiltered2.isRangeTombstoneMarker()) + { + RangeTombstoneMarker marker2 = (RangeTombstoneMarker) unfiltered2; + assert marker2.isClose(reversed); + assert marker2.closeDeletionTime(reversed).equals(deletionTime); + if (!marker2.closeIsInclusive(reversed) || !clusteringPositionsEqual(marker1, marker2)) + return setNextAndBufferedAndReturn(marker1, marker2); + + // The recombination applies. We have to transform the open side of marker1 and the close side + // of marker2 into an empty row with deletion time. + return processOtherSidesAndReturn(BTreeRow.emptyDeletedRow(clusteringPositionOf(marker1), Row.Deletion.regular(deletionTime)), + reversed, marker1, marker2, deletionTime); + } + + BTreeRow row2 = (BTreeRow) unfiltered2; + + if (!clusteringPositionsEqual(marker1, row2)) + return setNextAndBufferedAndReturn(marker1, row2); + + if (!wrapped.hasNext()) + return setNextAndBufferedAndReturn(marker1, row2); + + Unfiltered unfiltered3 = wrapped.next(); + if (unfiltered3.isRow()) + return setNextAndBufferedAndReturn(marker1, row2, unfiltered3); + + RangeTombstoneMarker marker3 = (RangeTombstoneMarker) unfiltered3; + assert marker3.isClose(reversed); + assert marker3.closeDeletionTime(reversed).equals(deletionTime); + if (!marker3.closeIsInclusive(reversed) || !clusteringPositionsEqual(marker1, marker3)) + return setNextAndBufferedAndReturn(marker1, row2, marker3); + + // The recombination applies. We have to transform the open side of marker1 and the close side + // of marker3 into a deletion time for row2. + return processOtherSidesAndReturn(BTreeRow.create(row2.clustering(), row2.primaryKeyLivenessInfo(), Row.Deletion.regular(deletionTime), row2.getBTree()), + reversed, marker1, marker3, deletionTime); + } + + private Unfiltered processOtherSidesAndReturn(Row row, + boolean reversed, + RangeTombstoneMarker markerLeft, + RangeTombstoneMarker markerRight, + DeletionTime deletionTime) + { + // Check if any of the markers is a boundary, and if so, report the other side. + if (!markerLeft.isClose(reversed)) + { + if (!markerRight.isOpen(reversed)) + return setNextAndReturn(row); + + return setNextAndBufferedAndReturn(row, + ((RangeTombstoneBoundaryMarker) markerRight).createCorrespondingOpenMarker(reversed)); + } + + if (!markerRight.isOpen(reversed)) + return setNextAndBufferedAndReturn(((RangeTombstoneBoundaryMarker) markerLeft).createCorrespondingCloseMarker(reversed), + row); + + // We have surviving markers on both sides. + final DeletionTime closeDeletionTime = markerLeft.closeDeletionTime(reversed); + if (markerRight.openDeletionTime(reversed).equals(closeDeletionTime) && !closeDeletionTime.supersedes(deletionTime)) + { + // The row interrupts a covering deletion, we can still drop both markers and report a deleted row. + return setNextAndReturn(row); + } + + return setNextAndBufferedAndReturn(((RangeTombstoneBoundaryMarker) markerLeft).createCorrespondingCloseMarker(reversed), + row, + ((RangeTombstoneBoundaryMarker) markerRight).createCorrespondingOpenMarker(reversed)); + } + + private Unfiltered setNextAndReturn(Unfiltered next) + { + this.next = next; + this.nextPrepared = true; + return next; + } + + private Unfiltered setNextAndBufferedAndReturn(Unfiltered next, Unfiltered bufferedOne) + { + this.bufferedOne = bufferedOne; + return setNextAndReturn(next); + } + + private Unfiltered setNextAndBufferedAndReturn(Unfiltered next, Row bufferedOne, Unfiltered bufferedTwo) + { + this.bufferedTwo = bufferedTwo; + return setNextAndBufferedAndReturn(next, bufferedOne); + } + + static boolean clusteringPositionsEqual(Unfiltered l, Unfiltered r) + { + return clusteringPositionsEqual(l.clustering(), r.clustering()); + } + + static boolean clusteringPositionsEqual(ClusteringPrefix cl, ClusteringPrefix cr) + { + if (cl.size() != cr.size()) + return false; + for (int i = cl.size() - 1; i >= 0; --i) + if (cl.accessor().compare(cl.get(i), cr.get(i), cr.accessor()) != 0) + return false; + return true; + } + + static Clustering clusteringPositionOf(Unfiltered unfiltered) + { + return clusteringPositionOf(unfiltered.clustering()); + } + + static Clustering clusteringPositionOf(ClusteringPrefix prefix) + { + return prefix.accessor().factory().clustering(prefix.getRawValues()); + } +} diff --git a/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java index 11aff6a5f438..8a8981303a92 100644 --- a/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java +++ b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java @@ -20,44 +20,45 @@ import java.util.Iterator; import java.util.NavigableSet; +import java.util.function.BiFunction; +import java.util.function.Predicate; -import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Ints; import org.apache.cassandra.db.Clustering; -import org.apache.cassandra.db.ClusteringBound; import org.apache.cassandra.db.ClusteringComparator; import org.apache.cassandra.db.ClusteringPrefix; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.DeletionInfo; import org.apache.cassandra.db.DeletionTime; import org.apache.cassandra.db.LivenessInfo; -import org.apache.cassandra.db.MutableDeletionInfo; -import org.apache.cassandra.db.RangeTombstone; import org.apache.cassandra.db.RegularAndStaticColumns; import org.apache.cassandra.db.Slice; import org.apache.cassandra.db.Slices; import org.apache.cassandra.db.filter.ColumnFilter; -import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.ByteBufferAccessor; -import org.apache.cassandra.db.rows.AbstractUnfilteredRowIterator; import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.rows.Cell; import org.apache.cassandra.db.rows.ColumnData; +import org.apache.cassandra.db.rows.ComplexColumnData; import org.apache.cassandra.db.rows.EncodingStats; import org.apache.cassandra.db.rows.RangeTombstoneMarker; import org.apache.cassandra.db.rows.Row; -import org.apache.cassandra.db.rows.RowAndDeletionMergeIterator; import org.apache.cassandra.db.rows.Rows; +import org.apache.cassandra.db.rows.TrieTombstoneMarker; import org.apache.cassandra.db.rows.Unfiltered; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.rows.UnfilteredRowIterators; +import org.apache.cassandra.db.tries.DeletionAwareTrie; import org.apache.cassandra.db.tries.Direction; +import org.apache.cassandra.db.tries.InMemoryBaseTrie; +import org.apache.cassandra.db.tries.InMemoryDeletionAwareTrie; import org.apache.cassandra.db.tries.InMemoryTrie; import org.apache.cassandra.db.tries.Trie; import org.apache.cassandra.db.tries.TrieEntriesIterator; +import org.apache.cassandra.db.tries.TrieSet; import org.apache.cassandra.db.tries.TrieSpaceExhaustedException; +import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.utils.AbstractIterator; import org.apache.cassandra.utils.ObjectSizes; import org.apache.cassandra.utils.btree.BTree; import org.apache.cassandra.utils.bytecomparable.ByteComparable; @@ -65,27 +66,17 @@ import org.apache.cassandra.utils.memory.Cloner; import org.apache.cassandra.utils.memory.EnsureOnHeap; -/** - * In-memory partition backed by a trie. The rows of the partition are values in the leaves of the trie, where the key - * to the row is only stored as the path to reach that leaf; static rows are also treated as a row with STATIC_CLUSTERING - * path; the deletion information is placed as a metadata object at the root of the trie -- this matches how Memtable - * stores partitions within the larger map, so that TrieBackedPartition objects can be created directly from Memtable - * tail tries. - * - * This object also holds the partition key, as well as some metadata (columns and statistics). - * - * Currently all descendants and instances of this class are immutable (even tail tries from mutable memtables are - * guaranteed to not change as we use forced copying below the partition level), though this may change in the future. - */ +/// In-memory partition backed by a deletion-aware trie. The rows of the partition are values in the leaves of the trie, +/// where the key to the row is only stored as the path to reach that leaf; static rows are also treated as a row with +/// `STATIC_CLUSTERING` path; the deletion information is placed in a deletion branch of the trie which starts at the +/// root of the partition. This matches how `TrieMemtable` stores partitions within the larger map, so that +/// `TrieBackedPartition` objects can be created directly from `TrieMemtable` tail tries. +/// +/// This object also holds the partition key, as well as some metadata (columns and statistics). +/// Currently, all descendants and instances of this class are immutable (even tail tries from mutable memtables are +/// guaranteed to not change as we use forced copying below the partition level), though this may change in the future. public class TrieBackedPartition implements Partition { - /** - * If keys are below this length, we will use a recursive procedure for inserting data when building the backing - * trie. - */ - @VisibleForTesting - public static final int MAX_RECURSIVE_KEY_LENGTH = 128; - public static final ByteComparable.Version BYTE_COMPARABLE_VERSION = ByteComparable.Version.OSS50; /** Pre-made path for STATIC_CLUSTERING, to avoid creating path object when querying static path. */ @@ -93,32 +84,56 @@ public class TrieBackedPartition implements Partition /** Pre-made path for BOTTOM, to avoid creating path object when iterating rows. */ public static final ByteComparable BOTTOM_PATH = v -> ByteSource.oneByte(ClusteringPrefix.Kind.INCL_START_BOUND.asByteComparableValue(v)); - /** - * The representation of a row stored at the leaf of a trie. Does not contain the row key. - * - * The methods toRow and copyToOnHeapRow combine this with a clustering for the represented Row. - */ + /// Pre-made path for partition deletions + public static final ByteComparable PARTITION_DELETION_START = v -> ByteSource.oneByte(ByteSource.LT_EXCLUDED); + public static final ByteComparable PARTITION_DELETION_END = v -> ByteSource.oneByte(ByteSource.GT_NEXT_COMPONENT); + + /// Interface implemented by partition markers, both the singleton below used for standalone [TrieBackedPartition], + /// and the marker used in tail tries in `TrieMemtable`s. + public interface PartitionMarker {} + + /// Singleton partition marker used for standalone [TrieBackedPartition] and [TriePartitionUpdate] objects. + public static final PartitionMarker PARTITION_MARKER = new PartitionMarker() + { + public String toString() + { + return "PARTITION_MARKER"; + } + }; + + /// Predicate to identify partition boundaries in tries. This accepts any [PartitionMarker], not just the + /// [#PARTITION_MARKER] used for standalone trie-backed partitions. + public static final Predicate IS_PARTITION_BOUNDARY = TrieBackedPartition::isPartitionBoundary; + + /// Returns true if the given content is a partition marker. + public static boolean isPartitionBoundary(Object content) + { + return content instanceof TrieBackedPartition.PartitionMarker; + } + + /// The representation of a row stored at the leaf of a trie. Does not contain the row key. + /// + /// The method [#toRow] combines this with a clustering for the represented [Row]. public static class RowData { final Object[] columnsBTree; final LivenessInfo livenessInfo; - final DeletionTime deletion; final int minLocalDeletionTime; + // TODO track minTimestamp to avoid applying deletions that do not do anything - RowData(Object[] columnsBTree, LivenessInfo livenessInfo, DeletionTime deletion) + RowData(Object[] columnsBTree, LivenessInfo livenessInfo) { - this(columnsBTree, livenessInfo, deletion, BTreeRow.minDeletionTime(columnsBTree, livenessInfo, deletion)); + this(columnsBTree, livenessInfo, BTreeRow.minDeletionTime(columnsBTree, livenessInfo, DeletionTime.LIVE)); } - RowData(Object[] columnsBTree, LivenessInfo livenessInfo, DeletionTime deletion, int minLocalDeletionTime) + RowData(Object[] columnsBTree, LivenessInfo livenessInfo, int minLocalDeletionTime) { this.columnsBTree = columnsBTree; this.livenessInfo = livenessInfo; - this.deletion = deletion; this.minLocalDeletionTime = minLocalDeletionTime; } - Row toRow(Clustering clustering) + Row toRow(Clustering clustering, DeletionTime deletion) { return BTreeRow.create(clustering, livenessInfo, @@ -129,7 +144,7 @@ Row toRow(Clustering clustering) public int dataSize() { - int dataSize = livenessInfo.dataSize() + deletion.dataSize(); + int dataSize = livenessInfo.dataSize(); return Ints.checkedCast(BTree.accumulate(columnsBTree, (ColumnData cd, long v) -> v + cd.dataSize(), dataSize)); } @@ -138,41 +153,63 @@ public long unsharedHeapSizeExcludingData() { long heapSize = EMPTY_ROWDATA_SIZE + BTree.sizeOfStructureOnHeap(columnsBTree) - + livenessInfo.unsharedHeapSize() - + deletion.unsharedHeapSize(); + + livenessInfo.unsharedHeapSize(); return BTree.accumulate(columnsBTree, (ColumnData cd, long v) -> v + cd.unsharedHeapSizeExcludingData(), heapSize); } public String toString() { - return "row " + livenessInfo + " size " + dataSize(); + return "row " + livenessInfo + " size " + dataSize() + ": " + BTree.toString(columnsBTree); } public RowData clone(Cloner cloner) { Object[] tree = BTree.transform(columnsBTree, c -> c.clone(cloner)); - return new RowData(tree, livenessInfo, deletion, minLocalDeletionTime); + return new RowData(tree, livenessInfo, minLocalDeletionTime); + } + + public RowData delete(DeletionTime activeDeletion) + { + LivenessInfo newLiveness = livenessInfo; + if (activeDeletion.deletes(livenessInfo.timestamp())) + newLiveness = LivenessInfo.EMPTY; + + Object[] newBTree = BTree.transformAndFilter(columnsBTree, cd -> + { + ColumnMetadata column = cd.column(); + if (column.isComplex()) + return ((ComplexColumnData) cd).delete(activeDeletion); + + Cell cell = (Cell) cd; + return activeDeletion.deletes(cell) ? null : cell; + }); + + if (newLiveness == livenessInfo && newBTree == columnsBTree) + return this; + if (newLiveness.isEmpty() && newBTree == BTree.empty()) + return null; + return new RowData(newBTree, newLiveness); } } - private static final long EMPTY_ROWDATA_SIZE = ObjectSizes.measure(new RowData(null, null, null, 0)); + private static final long EMPTY_ROWDATA_SIZE = ObjectSizes.measure(new RowData(null, null, 0)); - protected final Trie trie; + protected final DeletionAwareTrie trie; protected final DecoratedKey partitionKey; protected final TableMetadata metadata; protected final RegularAndStaticColumns columns; protected final EncodingStats stats; protected final int rowCountIncludingStatic; - protected final boolean canHaveShadowedData; + protected final int tombstoneCount; public TrieBackedPartition(DecoratedKey partitionKey, RegularAndStaticColumns columns, EncodingStats stats, int rowCountIncludingStatic, - Trie trie, - TableMetadata metadata, - boolean canHaveShadowedData) + int tombstoneCount, + DeletionAwareTrie trie, + TableMetadata metadata) { this.partitionKey = partitionKey; this.trie = trie; @@ -180,9 +217,8 @@ public TrieBackedPartition(DecoratedKey partitionKey, this.columns = columns; this.stats = stats; this.rowCountIncludingStatic = rowCountIncludingStatic; - this.canHaveShadowedData = canHaveShadowedData; - // There must always be deletion info metadata. - // Note: we can't use deletionInfo() because WithEnsureOnHeap's override is not yet set up. + this.tombstoneCount = tombstoneCount; + // There must always be a partition marker. assert trie.get(ByteComparable.EMPTY) != null; assert stats != null; } @@ -194,9 +230,9 @@ public static TrieBackedPartition fromIterator(UnfilteredRowIterator iterator) iterator.columns(), iterator.stats(), builder.rowCountIncludingStatic(), + builder.tombstoneCount(), builder.trie(), - iterator.metadata(), - false); + iterator.metadata()); } protected static ContentBuilder build(UnfilteredRowIterator iterator, boolean collectDataSize) @@ -218,28 +254,27 @@ protected static ContentBuilder build(UnfilteredRowIterator iterator, boolean co } } - /** - * Create a row with the given properties and content, making sure to copy all off-heap data to keep it alive when - * the given access mode requires it. - */ + /// Create a row with the given properties and content, making sure to copy all off-heap data to keep it alive when + /// the given access mode requires it. public static TrieBackedPartition create(DecoratedKey partitionKey, RegularAndStaticColumns columnMetadata, EncodingStats encodingStats, int rowCountIncludingStatic, - Trie trie, + int tombstoneCount, + DeletionAwareTrie trie, TableMetadata metadata, EnsureOnHeap ensureOnHeap) { return ensureOnHeap == EnsureOnHeap.NOOP - ? new TrieBackedPartition(partitionKey, columnMetadata, encodingStats, rowCountIncludingStatic, trie, metadata, true) - : new WithEnsureOnHeap(partitionKey, columnMetadata, encodingStats, rowCountIncludingStatic, trie, metadata, true, ensureOnHeap); + ? new TrieBackedPartition(partitionKey, columnMetadata, encodingStats, rowCountIncludingStatic, tombstoneCount, trie, metadata) + : new WithEnsureOnHeap(partitionKey, columnMetadata, encodingStats, rowCountIncludingStatic, tombstoneCount, trie, metadata, ensureOnHeap); } class RowIterator extends TrieEntriesIterator { - public RowIterator(Trie trie, Direction direction) + public RowIterator(DeletionAwareTrie trie, Direction direction) { - super(trie, direction, RowData.class::isInstance); + super(trie.contentOnlyTrie(), direction, RowData.class::isInstance); } @Override @@ -253,7 +288,7 @@ protected Row mapContent(Object content, byte[] bytes, int byteLength) } } - private Iterator rowIterator(Trie trie, Direction direction) + private Iterator rowIterator(DeletionAwareTrie trie, Direction direction) { return new RowIterator(trie, direction); } @@ -261,46 +296,99 @@ private Iterator rowIterator(Trie trie, Direction direction) static RowData rowToData(Row row) { BTreeRow brow = (BTreeRow) row; - return new RowData(brow.getBTree(), row.primaryKeyLivenessInfo(), row.deletion().time(), brow.getMinLocalDeletionTime()); + return new RowData(brow.getBTree(), row.primaryKeyLivenessInfo(), brow.getMinLocalDeletionTime()); } - /** - * Conversion from RowData to Row. TrieBackedPartitionOnHeap overrides this to do the necessary copying - * (hence the non-static method). - */ + /// Conversion from [RowData] to [Row]. [WithEnsureOnHeap] overrides this to do the necessary copying + /// (hence the non-static method). Row toRow(RowData data, Clustering clustering) { - return data.toRow(clustering); + return data.toRow(clustering, DeletionTime.LIVE); } - /** - * Put the given unfiltered in the trie. - * @param comparator for converting key to byte-comparable - * @param useRecursive whether the key length is guaranteed short and recursive put can be used - * @param trie destination - * @param row content to put - */ - protected static void putInTrie(ClusteringComparator comparator, boolean useRecursive, InMemoryTrie trie, Row row) throws TrieSpaceExhaustedException + /// Put the given unfiltered in the trie, used by methods to build stand-alone partitions. + /// + /// @param comparator for converting key to byte-comparable + /// @param trie destination + /// @param row content to put + protected static void putInTrie(ClusteringComparator comparator, InMemoryDeletionAwareTrie trie, Row row) + throws TrieSpaceExhaustedException { - trie.putSingleton(comparator.asByteComparable(row.clustering()), rowToData(row), NO_CONFLICT_RESOLVER, useRecursive); + // We do not look for atomicity here, so can do the two steps separately. + // TODO: Direct insertion methods (singleton known to not be deleted, deletion known to not delete anything) + Clustering clustering = row.clustering(); + DeletionTime deletionTime = row.deletion().time(); + + if (!deletionTime.isLive()) + { + putDeletionInTrie(trie, + comparator.asByteComparable(clustering.asStartBound()), + comparator.asByteComparable(clustering.asEndBound()), + deletionTime); + } + if (!row.isEmptyAfterDeletion()) + { + trie.apply(DeletionAwareTrie.singleton(comparator.asByteComparable(clustering), + BYTE_COMPARABLE_VERSION, + rowToData(row)), + noConflictInData(), + mergeTombstoneRanges(), + noIncomingSelfDeletion(), + noExistingSelfDeletion(), + true, + x -> false); + } } - /** - * Check if we can use recursive operations when putting a value in tries. - * True if all types in the clustering keys are fixed length, and total size is small enough. - */ - protected static boolean useRecursive(ClusteringComparator comparator) + protected static void putMarkerInTrie(ClusteringComparator comparator, + InMemoryDeletionAwareTrie trie, + RangeTombstoneMarker openMarker, + RangeTombstoneMarker closeMarker) { - int length = 1; // terminator - for (AbstractType type : comparator.subtypes()) - if (!type.isValueLengthFixed()) - return false; - else - length += 1 + type.valueLengthIfFixed(); // separator + value + // TODO: Standalone partitions would not delete their own data, we could tell the trie not to go over the live path. + DeletionTime deletionTime = openMarker.openDeletionTime(false); + assert deletionTime.equals(closeMarker.closeDeletionTime(false)); + putDeletionInTrie(trie, + comparator.asByteComparable(openMarker.clustering()), + comparator.asByteComparable(closeMarker.clustering()), + deletionTime); + } - return length <= MAX_RECURSIVE_KEY_LENGTH; + protected static void putPartitionDeletionInTrie(InMemoryDeletionAwareTrie trie, + DeletionTime deletionTime) + { + putDeletionInTrie(trie, + PARTITION_DELETION_START, + PARTITION_DELETION_END, + deletionTime); } + static void putDeletionInTrie(InMemoryDeletionAwareTrie trie, + ByteComparable start, + ByteComparable end, + DeletionTime deletionTime) + { + try + { + trie.apply(DeletionAwareTrie.deletion(ByteComparable.EMPTY, + start, + end, + BYTE_COMPARABLE_VERSION, + TrieTombstoneMarker.covering(deletionTime)), + noConflictInData(), + mergeTombstoneRanges(), + noIncomingSelfDeletion(), + noExistingSelfDeletion(), + true, + x -> false); + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + } + + public TableMetadata metadata() { return metadata; @@ -313,7 +401,8 @@ public DecoratedKey partitionKey() public DeletionTime partitionLevelDeletion() { - return deletionInfo().getPartitionDeletion(); + TrieTombstoneMarker applicableRange = trie.deletionOnlyTrie().applicableRange(STATIC_CLUSTERING_PATH); + return applicableRange != null ? applicableRange.deletionTime() : DeletionTime.LIVE; } public RegularAndStaticColumns columns() @@ -331,11 +420,6 @@ public int rowCount() return rowCountIncludingStatic - (hasStaticRow() ? 1 : 0); } - public DeletionInfo deletionInfo() - { - return (DeletionInfo) trie.get(ByteComparable.EMPTY); - } - public ByteComparable path(ClusteringPrefix clustering) { return metadata.comparator.asByteComparable(clustering); @@ -343,17 +427,14 @@ public ByteComparable path(ClusteringPrefix clustering) public Row staticRow() { + // Static rows can only be deleted via the partition deletion. There is no need to check and apply that here. RowData staticRow = (RowData) trie.get(STATIC_CLUSTERING_PATH); - - if (staticRow != null) - return toRow(staticRow, Clustering.STATIC_CLUSTERING); - else - return Rows.EMPTY_STATIC_ROW; + return staticRow != null ? staticRow.toRow(Clustering.STATIC_CLUSTERING, DeletionTime.LIVE) : Rows.EMPTY_STATIC_ROW; } public boolean isEmpty() { - return rowCountIncludingStatic == 0 && deletionInfo().isLive(); + return rowCountIncludingStatic + tombstoneCount == 0; } private boolean hasStaticRow() @@ -366,15 +447,13 @@ public boolean hasRows() return rowCountIncludingStatic > 1 || rowCountIncludingStatic > 0 && !hasStaticRow(); } - /** - * Provides read access to the trie for users that can take advantage of it directly (e.g. Memtable). - */ - public Trie trie() + /// Provides read access to the trie for users that can take advantage of it directly (e.g. `TrieMemtable`). + public DeletionAwareTrie trie() { return trie; } - private Trie nonStaticSubtrie() + private DeletionAwareTrie nonStaticSubtrie() { // skip static row if present - the static clustering sorts before BOTTOM so that it's never included in // any slices (we achieve this by using the byte ByteSource.EXCLUDED for its representation, which is lower @@ -401,33 +480,19 @@ public Row lastRow() public Row getRow(Clustering clustering) { - RowData data = (RowData) trie.get(path(clustering)); - - DeletionInfo deletionInfo = deletionInfo(); - RangeTombstone rt = deletionInfo.rangeCovering(clustering); - - // The trie only contains rows, so it doesn't allow to directly account for deletion that should apply to row - // (the partition deletion or the deletion of a range tombstone that covers it). So if needs be, reuse the row - // deletion to carry the proper deletion on the row. - DeletionTime partitionDeletion = deletionInfo.getPartitionDeletion(); - DeletionTime activeDeletion = partitionDeletion; - if (rt != null && rt.deletionTime().supersedes(activeDeletion)) - activeDeletion = rt.deletionTime(); - - if (data == null) - { - // this means our partition level deletion supersedes all other deletions and we don't have to keep the row deletions - if (activeDeletion == partitionDeletion) - return null; - // no need to check activeDeletion.isLive here - if anything superseedes the partitionDeletion - // it must be non-live - return BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(activeDeletion)); - } + return getRow(clustering, path(clustering)); + } - Row row = toRow(data, clustering); - if (!activeDeletion.isLive()) - row = row.filter(ColumnFilter.selection(columns()), activeDeletion, true, metadata()); - return row; + public Row getRow(Clustering clustering, ByteComparable path) + { + RowData data = (RowData) trie.get(path); + TrieTombstoneMarker marker = trie.applicableDeletion(path); + if (data != null) + return data.toRow(clustering, marker != null ? marker.deletionTime() : DeletionTime.LIVE); + else if (marker != null) + return BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(marker.deletionTime())); + else + return null; } public UnfilteredRowIterator unfilteredIterator() @@ -435,160 +500,161 @@ public UnfilteredRowIterator unfilteredIterator() return unfilteredIterator(ColumnFilter.selection(columns()), Slices.ALL, false); } - public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, Slices slices, boolean reversed) + /// Implementation of [UnfilteredRowIterator] for this partition. + /// + /// Currently, this implementation is pretty involved because it has to revert the transformations done to row and + /// partition-level deletions. To do the former, we apply a [RecombiningUnfilteredRowIterator] on top of this. To + /// do the latter, we extract the partition-level deletion from its coverage of the static row and filter out + /// tombstone ranges that switch to it. + class UnfilteredIterator + extends TrieEntriesIterator.WithNullFiltering + implements UnfilteredRowIterator { - Row staticRow = staticRow(selection, false); - if (slices.size() == 0) + final boolean reversed; + final ColumnFilter selection; + final DeletionTime partitionLevelDeletion; + final Row staticRow; + + protected UnfilteredIterator(ColumnFilter selection, Trie trie, boolean reversed) { - DeletionTime partitionDeletion = deletionInfo().getPartitionDeletion(); - return UnfilteredRowIterators.noRowsIterator(metadata(), partitionKey(), staticRow, partitionDeletion, reversed); + this(selection, trie, reversed, TrieBackedPartition.this.partitionLevelDeletion()); } - return slices.size() == 1 - ? sliceIterator(selection, slices.get(0), reversed, staticRow) - : new SlicesIterator(selection, slices, reversed, staticRow); - } - - public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, NavigableSet> clusteringsInQueryOrder, boolean reversed) - { - Row staticRow = staticRow(selection, false); - if (clusteringsInQueryOrder.isEmpty()) + private UnfilteredIterator(ColumnFilter selection, Trie trie, boolean reversed, DeletionTime partitionLevelDeletion) { - DeletionTime partitionDeletion = deletionInfo().getPartitionDeletion(); - return UnfilteredRowIterators.noRowsIterator(metadata(), partitionKey(), staticRow, partitionDeletion, reversed); + super(trie, Direction.fromBoolean(reversed)); + this.selection = selection; + this.reversed = reversed; + this.partitionLevelDeletion = partitionLevelDeletion; + Row staticRow = TrieBackedPartition.this.staticRow().filter(selection, metadata()); + this.staticRow = staticRow != null ? staticRow : Rows.EMPTY_STATIC_ROW; } - Iterator rowIter = new AbstractIterator() { + @Override + protected Unfiltered mapContent(Object content, byte[] bytes, int byteLength) + { + if (content instanceof RowData) + return toRow((RowData) content, + metadata.comparator.clusteringFromByteComparable(ByteBufferAccessor.instance, + ByteComparable.preencoded(BYTE_COMPARABLE_VERSION, + bytes, 0, byteLength), + BYTE_COMPARABLE_VERSION)) // deletion is given as range tombstone + .filter(selection, metadata()); + else + return ((TrieTombstoneMarker) content).toRangeTombstoneMarker( + ByteComparable.preencoded(BYTE_COMPARABLE_VERSION, bytes, 0, byteLength), + BYTE_COMPARABLE_VERSION, + metadata.comparator, + partitionLevelDeletion); + } - Iterator> clusterings = clusteringsInQueryOrder.iterator(); + @Override + public DeletionTime partitionLevelDeletion() + { + return partitionLevelDeletion; + } - @Override - protected Row computeNext() - { - while (clusterings.hasNext()) - { - Clustering clustering = clusterings.next(); - Object rowData = trie.get(path(clustering)); - if (rowData instanceof RowData) - return toRow((RowData) rowData, clustering); - } - return endOfData(); - } - }; + @Override + public EncodingStats stats() + { + return stats; + } - // not using DeletionInfo.rangeCovering(Clustering), because it returns the original range tombstone, - // but we need DeletionInfo.rangeIterator(Set) that generates tombstones based on given clustering bound. - Iterator deleteIter = deletionInfo().rangeIterator(clusteringsInQueryOrder, reversed); + @Override + public TableMetadata metadata() + { + return metadata; + } - return merge(rowIter, deleteIter, selection, reversed, staticRow); - } + @Override + public boolean isReverseOrder() + { + return reversed; + } - private UnfilteredRowIterator sliceIterator(ColumnFilter selection, Slice slice, boolean reversed, Row staticRow) - { - ClusteringBound start = slice.start(); - ClusteringBound end = slice.end() == ClusteringBound.TOP ? null : slice.end(); - Iterator rowIter = slice(start, end, reversed); - Iterator deleteIter = deletionInfo().rangeIterator(slice, reversed); - return merge(rowIter, deleteIter, selection, reversed, staticRow); - } + @Override + public RegularAndStaticColumns columns() + { + return columns; + } - private Iterator slice(ClusteringBound start, ClusteringBound end, boolean reversed) - { - ByteComparable endPath = end != null ? path(end) : null; - // use BOTTOM as bound to skip over static rows - ByteComparable startPath = start != null ? path(start) : BOTTOM_PATH; - return rowIterator(trie.subtrie(startPath, endPath), Direction.fromBoolean(reversed)); - } + @Override + public DecoratedKey partitionKey() + { + return partitionKey; + } - private Row staticRow(ColumnFilter columns, boolean setActiveDeletionToRow) - { - DeletionTime partitionDeletion = deletionInfo().getPartitionDeletion(); - Row staticRow = staticRow(); - if (columns.fetchedColumns().statics.isEmpty() || (staticRow.isEmpty() && partitionDeletion.isLive())) - return Rows.EMPTY_STATIC_ROW; + @Override + public Row staticRow() + { + return staticRow; + } - Row row = staticRow.filter(columns, partitionDeletion, setActiveDeletionToRow, metadata()); - return row == null ? Rows.EMPTY_STATIC_ROW : row; + @Override + public void close() + { + // nothing to close + } } - private RowAndDeletionMergeIterator merge(Iterator rowIter, Iterator deleteIter, - ColumnFilter selection, boolean reversed, Row staticRow) + public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, ByteComparable[] bounds, boolean reversed) { - return new RowAndDeletionMergeIterator(metadata(), partitionKey(), deletionInfo().getPartitionDeletion(), - selection, staticRow, reversed, stats(), - rowIter, deleteIter, canHaveShadowedData); - } - + if (bounds.length == 0) + return UnfilteredRowIterators.noRowsIterator(metadata, partitionKey, staticRow(), partitionLevelDeletion(), reversed); - @Override - public String toString() - { - return Partition.toString(this); + DeletionAwareTrie slicedTrie = trie.intersect(TrieSet.ranges(BYTE_COMPARABLE_VERSION, bounds)); + Trie mergedTrie = slicedTrie.mergedTrie((x, y) -> x instanceof RowData ? x : y); + return new RecombiningUnfilteredRowIterator(new UnfilteredIterator(selection, mergedTrie, reversed)); } - class SlicesIterator extends AbstractUnfilteredRowIterator + public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, Slices slices, boolean reversed) { - private final Slices slices; - - private int idx; - private Iterator currentSlice; - private final ColumnFilter selection; - - private SlicesIterator(ColumnFilter selection, - Slices slices, - boolean isReversed, - Row staticRow) + ByteComparable[] bounds = new ByteComparable[slices.size() * 2]; + int index = 0; + for (Slice slice : slices) { - super(TrieBackedPartition.this.metadata(), TrieBackedPartition.this.partitionKey(), - TrieBackedPartition.this.partitionLevelDeletion(), - selection.fetchedColumns(), staticRow, isReversed, TrieBackedPartition.this.stats()); - this.selection = selection; - this.slices = slices; + bounds[index++] = metadata.comparator.asByteComparable(slice.start()); + bounds[index++] = metadata.comparator.asByteComparable(slice.end()); } + return unfilteredIterator(selection, bounds, reversed); + } - protected Unfiltered computeNext() + public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, NavigableSet> clusteringsInQueryOrder, boolean reversed) + { + ByteComparable[] bounds = new ByteComparable[clusteringsInQueryOrder.size() * 2]; + int index = reversed ? (clusteringsInQueryOrder.size() - 1) * 2 : 0; + int indexInc = reversed ? -2 : +2; + for (Clustering clustering : clusteringsInQueryOrder) { - while (true) - { - if (currentSlice == null) - { - if (idx >= slices.size()) - return endOfData(); - - int sliceIdx = isReverseOrder ? slices.size() - idx - 1 : idx; - currentSlice = sliceIterator(selection, slices.get(sliceIdx), isReverseOrder, Rows.EMPTY_STATIC_ROW); - idx++; - } - - if (currentSlice.hasNext()) - return currentSlice.next(); - - currentSlice = null; - } + bounds[index + 0] = metadata.comparator.asByteComparable(clustering.asStartBound()); + bounds[index + 1] = metadata.comparator.asByteComparable(clustering.asEndBound()); + index += indexInc; } + return unfilteredIterator(selection, bounds, reversed); } + @Override + public String toString() + { + return Partition.toString(this); + } - /** - * An snapshot of the current TrieBackedPartition data, copied on heap when retrieved. - */ + /// A snapshot of the current [TrieBackedPartition] data, copied on heap when retrieved. private static final class WithEnsureOnHeap extends TrieBackedPartition { - final DeletionInfo onHeapDeletion; EnsureOnHeap ensureOnHeap; public WithEnsureOnHeap(DecoratedKey partitionKey, RegularAndStaticColumns columns, EncodingStats stats, int rowCountIncludingStatic, - Trie trie, + int tombstoneCount, + DeletionAwareTrie trie, TableMetadata metadata, - boolean canHaveShadowedData, EnsureOnHeap ensureOnHeap) { - super(partitionKey, columns, stats, rowCountIncludingStatic, trie, metadata, canHaveShadowedData); + super(partitionKey, columns, stats, rowCountIncludingStatic, tombstoneCount, trie, metadata); this.ensureOnHeap = ensureOnHeap; - this.onHeapDeletion = ensureOnHeap.applyToDeletionInfo(super.deletionInfo()); } @Override @@ -596,18 +662,11 @@ public Row toRow(RowData data, Clustering clustering) { return ensureOnHeap.applyToRow(super.toRow(data, clustering)); } - - @Override - public DeletionInfo deletionInfo() - { - return onHeapDeletion; - } } - /** - * Resolver for operations with trie-backed partitions. We don't permit any overwrites/merges. - */ - public static final InMemoryTrie.UpsertTransformer NO_CONFLICT_RESOLVER = + /// Resolver for operations with trie-backed partitions. We don't permit any overwrites/merges. + @SuppressWarnings("rawtypes") + private static final InMemoryTrie.UpsertTransformer NO_CONFLICT_RESOLVER = (existing, update) -> { if (existing != null) @@ -615,40 +674,79 @@ public DeletionInfo deletionInfo() return update; }; - /** - * Helper class for constructing tries and deletion info from an iterator or flowable partition. - * - * Note: This is not collecting any stats or columns! - */ + /// Resolver for data in trie-backed partitions. We don't permit any overwrites/merges. + @SuppressWarnings("rawtypes") + public static InMemoryTrie.UpsertTransformer noConflictInData() + { + return NO_CONFLICT_RESOLVER; + } + + /// Tombstone merging resolver. Even though we don't support overwrites, we get requests to add the two sides + /// of a boundary separately and must join them. + private static final InMemoryTrie.UpsertTransformer MERGE_TOMBSTONE_RANGES = + (existing, update) -> existing != null ? existing.mergeWith(update) : update; + + /// Tombstone merging resolver. Even though we don't support overwrites, we get requests to add the two sides + /// of a boundary separately and must join them. + public static InMemoryTrie.UpsertTransformer mergeTombstoneRanges() + { + return MERGE_TOMBSTONE_RANGES; + } + + private static InMemoryBaseTrie.UpsertTransformer IGNORE_UPDATE = (left, right) -> left; + + /// Resolver for applying incoming deletions to existing data in trie-backed partitions. We assume that the data is + /// not affected by the deletion. + public static InMemoryTrie.UpsertTransformer noIncomingSelfDeletion() + { + return IGNORE_UPDATE; + } + + private static BiFunction IGNORE_EXISTING = (left, right) -> right; + + /// Resolver for applying existing deletions to incoming data in trie-backed partitions. We assume that the data is + /// not affected by the deletion. + public static BiFunction noExistingSelfDeletion() + { + return IGNORE_EXISTING; + } + + /// Helper class for constructing tries and deletion info from an iterator. public static class ContentBuilder { final TableMetadata metadata; final ClusteringComparator comparator; - private final MutableDeletionInfo.Builder deletionBuilder; - private final InMemoryTrie trie; + private final InMemoryDeletionAwareTrie trie; - private final boolean useRecursive; private final boolean collectDataSize; private int rowCountIncludingStatic; + private int tombstoneCount; private long dataSize; + private RangeTombstoneMarker openMarker = null; + private final boolean isReverseOrder; + public ContentBuilder(TableMetadata metadata, DeletionTime partitionLevelDeletion, boolean isReverseOrder, boolean collectDataSize) { this.metadata = metadata; this.comparator = metadata.comparator; - this.deletionBuilder = MutableDeletionInfo.builder(partitionLevelDeletion, - comparator, - isReverseOrder); - this.trie = InMemoryTrie.shortLived(BYTE_COMPARABLE_VERSION); + this.trie = InMemoryDeletionAwareTrie.shortLived(BYTE_COMPARABLE_VERSION); - this.useRecursive = useRecursive(comparator); this.collectDataSize = collectDataSize; rowCountIncludingStatic = 0; + tombstoneCount = 0; dataSize = 0; + this.isReverseOrder = isReverseOrder; + + if (!partitionLevelDeletion.isLive()) + { + putPartitionDeletionInTrie(trie, partitionLevelDeletion); + ++tombstoneCount; + } } public ContentBuilder addStatic(Row staticRow) throws TrieSpaceExhaustedException @@ -661,16 +759,34 @@ public ContentBuilder addStatic(Row staticRow) throws TrieSpaceExhaustedExceptio public ContentBuilder addRow(Row row) throws TrieSpaceExhaustedException { - putInTrie(comparator, useRecursive, trie, row); + putInTrie(comparator, trie, row); ++rowCountIncludingStatic; if (collectDataSize) dataSize += row.dataSize(); + if (!row.deletion().isLive()) + ++tombstoneCount; return this; } public ContentBuilder addRangeTombstoneMarker(RangeTombstoneMarker unfiltered) { - deletionBuilder.add(unfiltered); + if (openMarker != null) + { + // This will check that unfiltered closes openMarker + putMarkerInTrie(comparator, trie, + isReverseOrder ? unfiltered : openMarker, + isReverseOrder ? openMarker : unfiltered); + ++tombstoneCount; + if (unfiltered.isOpen(isReverseOrder)) + openMarker = unfiltered; + else + openMarker = null; + } + else + { + assert unfiltered.isOpen(isReverseOrder); + openMarker = unfiltered; + } return this; } @@ -684,13 +800,12 @@ public ContentBuilder addUnfiltered(Unfiltered unfiltered) throws TrieSpaceExhau public ContentBuilder complete() throws TrieSpaceExhaustedException { - MutableDeletionInfo deletionInfo = deletionBuilder.build(); - trie.putRecursive(ByteComparable.EMPTY, deletionInfo, NO_CONFLICT_RESOLVER); // will throw if called more than once - // dataSize does not include the deletion info bytes + assert openMarker == null; + trie.putRecursive(ByteComparable.EMPTY, PARTITION_MARKER, noConflictInData()); // will throw if called more than once return this; } - public Trie trie() + public DeletionAwareTrie trie() { return trie; } @@ -700,6 +815,11 @@ public int rowCountIncludingStatic() return rowCountIncludingStatic; } + public int tombstoneCount() + { + return tombstoneCount; + } + public int dataSize() { assert collectDataSize; diff --git a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdate.java b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdate.java index a910b4e94a8e..58bb7f22ea0c 100644 --- a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdate.java +++ b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdate.java @@ -21,12 +21,13 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.Map; -import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Ints; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.db.Clustering; import org.apache.cassandra.db.Columns; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.DeletionInfo; @@ -39,12 +40,14 @@ import org.apache.cassandra.db.rows.Cell; import org.apache.cassandra.db.rows.ColumnData; import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.RangeTombstoneMarker; import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.db.rows.Rows; +import org.apache.cassandra.db.rows.TrieTombstoneMarker; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.rows.UnfilteredRowIterators; -import org.apache.cassandra.db.tries.InMemoryTrie; -import org.apache.cassandra.db.tries.Trie; +import org.apache.cassandra.db.tries.DeletionAwareTrie; +import org.apache.cassandra.db.tries.InMemoryDeletionAwareTrie; import org.apache.cassandra.db.tries.TrieSpaceExhaustedException; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.btree.BTree; @@ -70,20 +73,20 @@ private TriePartitionUpdate(TableMetadata metadata, RegularAndStaticColumns columns, EncodingStats stats, int rowCountIncludingStatic, + int tombstoneCount, int dataSize, - Trie trie, - boolean canHaveShadowedData) + DeletionAwareTrie trie) { - super(key, columns, stats, rowCountIncludingStatic, trie, metadata, canHaveShadowedData); + super(key, columns, stats, rowCountIncludingStatic, tombstoneCount, trie, metadata); this.dataSize = dataSize; } - private static InMemoryTrie newTrie(DeletionInfo deletion) + private static InMemoryDeletionAwareTrie newTrie() { - InMemoryTrie trie = InMemoryTrie.shortLived(BYTE_COMPARABLE_VERSION); + InMemoryDeletionAwareTrie trie = InMemoryDeletionAwareTrie.shortLived(BYTE_COMPARABLE_VERSION); try { - trie.putRecursive(ByteComparable.EMPTY, deletion, NO_CONFLICT_RESOLVER); + trie.putRecursive(ByteComparable.EMPTY, PARTITION_MARKER, noConflictInData()); } catch (TrieSpaceExhaustedException e) { @@ -108,8 +111,8 @@ public static TriePartitionUpdate emptyUpdate(TableMetadata metadata, DecoratedK EncodingStats.NO_STATS, 0, 0, - newTrie(MutableDeletionInfo.live()), - false); + 0, + newTrie()); } /** @@ -124,15 +127,16 @@ public static TriePartitionUpdate emptyUpdate(TableMetadata metadata, DecoratedK */ public static TriePartitionUpdate fullPartitionDelete(TableMetadata metadata, DecoratedKey key, long timestamp, int nowInSec) { - MutableDeletionInfo deletion = new MutableDeletionInfo(timestamp, nowInSec); + InMemoryDeletionAwareTrie trie = newTrie(); + putPartitionDeletionInTrie(trie, new DeletionTime(timestamp, nowInSec)); return new TriePartitionUpdate(metadata, key, RegularAndStaticColumns.NONE, new EncodingStats(timestamp, nowInSec, LivenessInfo.NO_TTL), 0, + 1, 0, - newTrie(deletion), - false); + trie); } /** @@ -147,7 +151,7 @@ public static TriePartitionUpdate fullPartitionDelete(TableMetadata metadata, De public static TriePartitionUpdate singleRowUpdate(TableMetadata metadata, DecoratedKey key, Row row) { EncodingStats stats = EncodingStats.Collector.forRow(row); - InMemoryTrie trie = newTrie(DeletionInfo.LIVE); + InMemoryDeletionAwareTrie trie = newTrie(); RegularAndStaticColumns columns; if (row.isStatic()) @@ -157,14 +161,14 @@ public static TriePartitionUpdate singleRowUpdate(TableMetadata metadata, Decora try { - putInTrie(metadata.comparator, useRecursive(metadata.comparator), trie, row); + putInTrie(metadata.comparator, trie, row); } catch (TrieSpaceExhaustedException e) { throw new AssertionError(e); } - return new TriePartitionUpdate(metadata, key, columns, stats, 1, row.dataSize(), trie, false); + return new TriePartitionUpdate(metadata, key, columns, stats, 1, row.deletion().isLive() ? 0 : 1, row.dataSize(), trie); } /** @@ -199,9 +203,9 @@ public static TriePartitionUpdate fromIterator(UnfilteredRowIterator iterator) iterator.columns(), iterator.stats(), builder.rowCountIncludingStatic(), + builder.tombstoneCount(), builder.dataSize(), - builder.trie(), - false); + builder.trie()); } public static TriePartitionUpdate asTrieUpdate(PartitionUpdate update) @@ -215,7 +219,7 @@ public static TriePartitionUpdate asTrieUpdate(PartitionUpdate update) } } - public static Trie asMergableTrie(PartitionUpdate update) + public static DeletionAwareTrie asMergableTrie(PartitionUpdate update) { return asTrieUpdate(update).trie.prefixedBy(update.partitionKey()); } @@ -238,50 +242,55 @@ public static Trie asMergableTrie(PartitionUpdate update) public TriePartitionUpdate withUpdatedTimestamps(long newTimestamp) { - InMemoryTrie t = InMemoryTrie.shortLived(BYTE_COMPARABLE_VERSION); + InMemoryDeletionAwareTrie t = InMemoryDeletionAwareTrie.shortLived(BYTE_COMPARABLE_VERSION); try { - t.apply(trie, new InMemoryTrie.UpsertTransformer() - { - public Object apply(Object shouldBeNull, Object o) - { - assert shouldBeNull == null; - if (o instanceof RowData) - return applyRowData((RowData) o); - else - return applyDeletion((DeletionInfo) o); - } - - public RowData applyRowData(RowData update) - { - LivenessInfo newInfo = update.livenessInfo.isEmpty() - ? update.livenessInfo - : update.livenessInfo.withUpdatedTimestamp(newTimestamp); - DeletionTime newDeletion = update.deletion.isLive() - ? DeletionTime.LIVE - : new DeletionTime(newTimestamp - 1, update.deletion.localDeletionTime()); - - return new RowData(BTree.transformAndFilter(update.columnsBTree, - (ColumnData cd) -> cd.updateAllTimestamp(newTimestamp)), - newInfo, newDeletion); - } - - public DeletionInfo applyDeletion(DeletionInfo update) - { - if (update.isLive()) - return update; - - MutableDeletionInfo mdi = update.mutableCopy(); - mdi.updateAllTimestamp(newTimestamp - 1); - return mdi; - } - }, x -> false); + t.apply(trie, + (shouldBeNull, o) -> + { + assert shouldBeNull == null; + if (!(o instanceof RowData)) + return o; + RowData update = (RowData) o; + + LivenessInfo newInfo = update.livenessInfo.isEmpty() + ? update.livenessInfo + : update.livenessInfo.withUpdatedTimestamp(newTimestamp); + + return new RowData(BTree.transformAndFilter(update.columnsBTree, + (ColumnData cd) -> cd.updateAllTimestamp(newTimestamp)), + newInfo); + }, + (shouldBeNull, o) -> + { + assert shouldBeNull == null; + return o.withUpdatedTimestamp(newTimestamp - 1); + }, + noIncomingSelfDeletion(), + noExistingSelfDeletion(), + true, + x -> false); } catch (TrieSpaceExhaustedException e) { throw new AssertionError(e); } - return new TriePartitionUpdate(metadata, partitionKey, columns, stats, rowCountIncludingStatic, dataSize, t, canHaveShadowedData); + return new TriePartitionUpdate(metadata, partitionKey, columns, stats, rowCountIncludingStatic, tombstoneCount, dataSize, t); + } + + @Override + public DeletionInfo deletionInfo() + { + // Collect deletion info from the trie. + DeletionTime partitionLevelDeletion = partitionLevelDeletion(); + MutableDeletionInfo.Builder builder = MutableDeletionInfo.builder(partitionLevelDeletion, metadata.comparator, false); + for (Map.Entry entry : trie.deletionOnlyTrie().entrySet()) + { + RangeTombstoneMarker marker = entry.getValue().toRangeTombstoneMarker(entry.getKey(), BYTE_COMPARABLE_VERSION, metadata.comparator, partitionLevelDeletion); + if (marker != null) + builder.add(marker); + } + return builder.build(); } /** @@ -295,9 +304,7 @@ public DeletionInfo applyDeletion(DeletionInfo update) @Override public int operationCount() { - return rowCountIncludingStatic - + deletionInfo().rangeCount() - + (deletionInfo().getPartitionDeletion().isLive() ? 0 : 1); + return rowCountIncludingStatic + tombstoneCount; } /** @@ -336,7 +343,9 @@ public void validate() @Override public long maxTimestamp() { - long maxTimestamp = deletionInfo().maxTimestamp(); + long maxTimestamp = Long.MIN_VALUE; + for (Iterator it = trie.deletionOnlyTrie().valueIterator(); it.hasNext();) + maxTimestamp = Math.max(maxTimestamp, it.next().deletionTime().markedForDeleteAt()); for (Iterator it = rowsIncludingStatic(); it.hasNext();) maxTimestamp = Math.max(maxTimestamp, Rows.collectMaxTimestamp(it.next())); @@ -381,48 +390,26 @@ private static void addMarksForRow(Row row, List marks) public static class Builder implements PartitionUpdate.Builder { private final TableMetadata metadata; + private final ColumnFilter cf; private final DecoratedKey key; - private final MutableDeletionInfo deletionInfo; - private final boolean canHaveShadowedData; private final RegularAndStaticColumns columns; - private final InMemoryTrie trie = InMemoryTrie.shortLived(BYTE_COMPARABLE_VERSION); + private final InMemoryDeletionAwareTrie trie = InMemoryDeletionAwareTrie.shortLived(BYTE_COMPARABLE_VERSION); private final EncodingStats.Collector statsCollector = new EncodingStats.Collector(); - private final boolean useRecursive; private int rowCountIncludingStatic; + private int tombstoneCount; private long dataSize; public Builder(TableMetadata metadata, DecoratedKey key, RegularAndStaticColumns columns) - { - this(metadata, key, columns, true, Rows.EMPTY_STATIC_ROW, DeletionInfo.LIVE); - } - - private Builder(TableMetadata metadata, - DecoratedKey key, - RegularAndStaticColumns columns, - boolean canHaveShadowedData, - Row staticRow, - DeletionInfo deletionInfo) { this.metadata = metadata; this.key = key; this.columns = columns; - this.canHaveShadowedData = canHaveShadowedData; - this.deletionInfo = deletionInfo.mutableCopy(); - useRecursive = useRecursive(metadata.comparator); rowCountIncludingStatic = 0; + tombstoneCount = 0; dataSize = 0; - add(staticRow); - } - - // This is wasteful, only to be used for testing. - @VisibleForTesting - public Builder(TriePartitionUpdate base) - { - this(base.metadata, base.partitionKey, base.columns(), base.canHaveShadowedData, Rows.EMPTY_STATIC_ROW, base.deletionInfo()); - for (Iterator it = base.rowsIncludingStatic(); it.hasNext();) - add(it.next()); + cf = ColumnFilter.all(metadata); } /** @@ -448,10 +435,29 @@ public void add(Row row) try { - trie.putSingleton(metadata.comparator.asByteComparable(row.clustering()), - row, - this::merge, - useRecursive); + // We do not look for atomicity here, so can do the two steps separately. + // TODO: Direct insertion methods (singleton known to not be deleted, deletion known to not delete anything) + Clustering clustering = row.clustering(); + DeletionTime deletionTime = row.deletion().time(); + + if (!deletionTime.isLive()) + { + putDeletionInTrie(metadata.comparator.asByteComparable(clustering.asStartBound()), + metadata.comparator.asByteComparable(clustering.asEndBound()), + deletionTime); + } + if (!row.isEmptyAfterDeletion()) + { + trie.apply(DeletionAwareTrie.singleton(metadata.comparator.asByteComparable(clustering), + BYTE_COMPARABLE_VERSION, + row), + this::merge, + this::mergeTombstones, + this::applyTombstone, + this::applyTombstone, + true, + x -> false); + } } catch (TrieSpaceExhaustedException e) { @@ -460,14 +466,40 @@ public void add(Row row) Rows.collectStats(row, statsCollector); } + private void putDeletionInTrie(ByteComparable start, ByteComparable end, DeletionTime deletionTime) + { + try + { + trie.apply(DeletionAwareTrie.deletion(ByteComparable.EMPTY, + start, + end, + BYTE_COMPARABLE_VERSION, + TrieTombstoneMarker.covering(deletionTime)), + this::merge, + this::mergeTombstones, + this::applyTombstone, + this::applyTombstone, + true, + x -> false); + statsCollector.update(deletionTime); + } + catch (TrieSpaceExhaustedException e) + { + throw new AssertionError(e); + } + } + public void addPartitionDeletion(DeletionTime deletionTime) { - deletionInfo.add(deletionTime); + if (!deletionTime.isLive()) + putDeletionInTrie(PARTITION_DELETION_START, PARTITION_DELETION_END, deletionTime); } public void add(RangeTombstone range) { - deletionInfo.add(range, metadata.comparator); + putDeletionInTrie(metadata.comparator.asByteComparable(range.deletedSlice().start()), + metadata.comparator.asByteComparable(range.deletedSlice().end()), + range.deletionTime()); } public DecoratedKey partitionKey() @@ -484,21 +516,20 @@ public TriePartitionUpdate build() { try { - trie.putRecursive(ByteComparable.EMPTY, deletionInfo, NO_CONFLICT_RESOLVER); + trie.putRecursive(ByteComparable.EMPTY, PARTITION_MARKER, noConflictInData()); } catch (TrieSpaceExhaustedException e) { throw new AssertionError(e); } - deletionInfo.collectStats(statsCollector); TriePartitionUpdate pu = new TriePartitionUpdate(metadata, partitionKey(), columns, statsCollector.get(), rowCountIncludingStatic, + tombstoneCount, Ints.saturatedCast(dataSize), - trie, - canHaveShadowedData); + trie); return pu; } @@ -509,7 +540,7 @@ RowData merge(Object existing, Row update) { // this is not expected to happen much, so going through toRow and the existing size is okay RowData rowData = (RowData) existing; - update = Rows.merge(rowData.toRow(update.clustering()), update); + update = Rows.merge(rowData.toRow(update.clustering(), DeletionTime.LIVE), update); dataSize += update.dataSize() - rowData.dataSize(); } else @@ -521,14 +552,38 @@ RowData merge(Object existing, Row update) return rowToData(update); } + private Row applyTombstone(TrieTombstoneMarker trieTombstoneMarker, Row o) + { + return o.filter(cf, trieTombstoneMarker.deletionTime(), false, metadata); + } + + private Object applyTombstone(Object o, TrieTombstoneMarker trieTombstoneMarker) + { + RowData row = (RowData) o; + return row.delete(trieTombstoneMarker.deletionTime()); + } + + private TrieTombstoneMarker mergeTombstones(TrieTombstoneMarker existing, TrieTombstoneMarker update) + { + if (existing == null) + { + ++tombstoneCount; + return update; + } + else + return update.mergeWith(existing); + } + public RegularAndStaticColumns columns() { return columns; } + @Override public DeletionTime partitionLevelDeletion() { - return deletionInfo.getPartitionDeletion(); + TrieTombstoneMarker applicableRange = trie.deletionOnlyTrie().applicableRange(STATIC_CLUSTERING_PATH); + return applicableRange != null ? applicableRange.deletionTime() : DeletionTime.LIVE; } @Override @@ -537,8 +592,6 @@ public String toString() return "Builder{" + "metadata=" + metadata + ", key=" + key + - ", deletionInfo=" + deletionInfo + - ", canHaveShadowedData=" + canHaveShadowedData + ", columns=" + columns + '}'; } diff --git a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdater.java b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdater.java index 832c38dbf3de..8d2dff0b2b96 100644 --- a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdater.java +++ b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdater.java @@ -18,14 +18,21 @@ package org.apache.cassandra.db.partitions; +import javax.annotation.Nullable; + import org.apache.cassandra.db.Clustering; -import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.ClusteringBound; import org.apache.cassandra.db.DeletionTime; import org.apache.cassandra.db.LivenessInfo; +import org.apache.cassandra.db.RangeTombstone; +import org.apache.cassandra.db.Slice; import org.apache.cassandra.db.marshal.ByteArrayAccessor; import org.apache.cassandra.db.memtable.TrieMemtable; import org.apache.cassandra.db.rows.BTreeRow; -import org.apache.cassandra.db.tries.InMemoryTrie; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.TrieTombstoneMarker; +import org.apache.cassandra.db.tries.Direction; +import org.apache.cassandra.db.tries.InMemoryBaseTrie; import org.apache.cassandra.index.transactions.UpdateTransaction; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.bytecomparable.ByteComparable; @@ -38,16 +45,19 @@ */ public final class TriePartitionUpdater extends BasePartitionUpdater -implements InMemoryTrie.UpsertTransformerWithKeyProducer +implements InMemoryBaseTrie.UpsertTransformerWithKeyProducer { private final UpdateTransaction indexer; private final TableMetadata metadata; private TrieMemtable.PartitionData currentPartition; private final TrieMemtable.MemtableShard owner; + private ClusteringBound rangeTombstoneOpenPosition = null; + private final DeletionTime partitionLevelDeletion; public int partitionsAdded = 0; public TriePartitionUpdater(Cloner cloner, UpdateTransaction indexer, + DeletionTime partitionLevelDeletion, TableMetadata metadata, TrieMemtable.MemtableShard owner) { @@ -55,19 +65,107 @@ public TriePartitionUpdater(Cloner cloner, this.indexer = indexer; this.metadata = metadata; this.owner = owner; + this.partitionLevelDeletion = partitionLevelDeletion; + if (!partitionLevelDeletion.isLive()) + indexer.onPartitionDeletion(partitionLevelDeletion); } @Override - public Object apply(Object existing, Object update, InMemoryTrie.KeyProducer keyState) + public Object apply(@Nullable Object existing, Object update, InMemoryBaseTrie.KeyProducer keyState) { - if (update instanceof RowData) + if (update == TrieBackedPartition.PARTITION_MARKER) + return mergePartitionMarkers((TrieMemtable.PartitionData) existing); + else if (update instanceof RowData) return applyRow((RowData) existing, (RowData) update, keyState); - else if (update instanceof DeletionInfo) - return applyDeletion((TrieMemtable.PartitionData) existing, (DeletionInfo) update); else throw new AssertionError("Unexpected update type: " + update.getClass()); } + public TrieTombstoneMarker mergeMarkers(@Nullable TrieTombstoneMarker existing, TrieTombstoneMarker update, InMemoryBaseTrie.KeyProducer keyState) + { + if (indexer != UpdateTransaction.NO_OP) + { + if (update.isBoundary()) + { + if (rangeTombstoneOpenPosition != null) + { + TrieTombstoneMarker preceding = update.precedingState(Direction.FORWARD); + assert preceding != null; // open markers are always closed + DeletionTime deletionTime = preceding.deletionTime(); + ClusteringBound bound = metadata.comparator.boundFromByteComparable( + ByteArrayAccessor.instance, + ByteComparable.preencoded(TrieBackedPartition.BYTE_COMPARABLE_VERSION, + keyState.getBytes()), + true); + indexer.onRangeTombstone(new RangeTombstone(Slice.make(rangeTombstoneOpenPosition, + bound), + deletionTime)); + } + + TrieTombstoneMarker succeeding = update.precedingState(Direction.REVERSE); + // Ignore the partition deletion. + if (succeeding != null && !succeeding.deletionTime().equals(partitionLevelDeletion)) + { + rangeTombstoneOpenPosition = metadata.comparator.boundFromByteComparable( + ByteArrayAccessor.instance, + ByteComparable.preencoded(TrieBackedPartition.BYTE_COMPARABLE_VERSION, + keyState.getBytes()), + false); + } + else + { + rangeTombstoneOpenPosition = null; + } + } + } + + if (existing == null) + { + currentPartition.markAddedTombstones(1); + this.heapSize += update.unsharedHeapSize(); + return update; + } + else + { + TrieTombstoneMarker merged = update.mergeWith(existing); + this.heapSize += (merged != null ? merged.unsharedHeapSize() : 0) - existing.unsharedHeapSize(); + return merged; + } + } + + public Object applyMarker(Object existingContent, TrieTombstoneMarker updateMarker, InMemoryBaseTrie.KeyProducer keyState) + { + assert existingContent instanceof RowData; // must be non-null, and can't be partition root + RowData existing = (RowData) existingContent; + RowData updated = existing.delete(updateMarker.deletionTime()); + if (updated != existing) + this.heapSize += (updated != null ? updated.unsharedHeapSizeExcludingData() : 0) - existing.unsharedHeapSizeExcludingData(); + if (updated == null) + currentPartition.markInsertedRows(-1); + + if (indexer != UpdateTransaction.NO_OP && updated != existingContent) + { + Clustering clustering = clusteringFor(keyState); + if (updated != null) + indexer.onUpdated(existing.toRow(clustering, DeletionTime.LIVE), + updated.toRow(clustering, DeletionTime.LIVE)); + else if (clustering != Clustering.STATIC_CLUSTERING) + indexer.onUpdated(existing.toRow(clustering, DeletionTime.LIVE), + BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(updateMarker.deletionTime()))); + else + indexer.onPartitionDeletion(updateMarker.deletionTime()); // static clustering is deleted only on partition deletion + } + return updated; + } + + public Object applyMarker(TrieTombstoneMarker marker, Object content) + { + // This is called to apply an existing tombstone to incoming data, before applyRow is called on the result. + // No size tracking is needed, because the result of this then gets applied to the trie with applyRow. + assert content instanceof RowData; // must be non-null, and can't be partition root + return ((RowData) content).delete(marker.deletionTime()); + } + /** * Called when a row needs to be copied to the Memtable trie. * @@ -76,14 +174,14 @@ else if (update instanceof DeletionInfo) * @param keyState Used to obtain the path through which this node was reached. * @return the insert row, or the merged row, copied using our allocator */ - private RowData applyRow(RowData existing, RowData insert, InMemoryTrie.KeyProducer keyState) + private RowData applyRow(@Nullable RowData existing, RowData insert, InMemoryBaseTrie.KeyProducer keyState) { if (existing == null) { RowData data = insert.clone(cloner); if (indexer != UpdateTransaction.NO_OP) - indexer.onInserted(data.toRow(clusteringFor(keyState))); + indexer.onInserted(data.toRow(clusteringFor(keyState), DeletionTime.LIVE)); this.dataSize += data.dataSize(); this.heapSize += data.unsharedHeapSizeExcludingData(); @@ -98,7 +196,8 @@ private RowData applyRow(RowData existing, RowData insert, InMemoryTrie.KeyProdu if (indexer != UpdateTransaction.NO_OP) { Clustering clustering = clusteringFor(keyState); - indexer.onUpdated(existing.toRow(clustering), reconciled.toRow(clustering)); + indexer.onUpdated(existing.toRow(clustering, DeletionTime.LIVE), + reconciled.toRow(clustering, DeletionTime.LIVE)); } return reconciled; @@ -108,62 +207,46 @@ private RowData applyRow(RowData existing, RowData insert, InMemoryTrie.KeyProdu private RowData merge(RowData existing, RowData update) { - LivenessInfo livenessInfo = LivenessInfo.merge(update.livenessInfo, existing.livenessInfo); - DeletionTime deletion = DeletionTime.merge(update.deletion, existing.deletion); - if (deletion.deletes(livenessInfo)) - livenessInfo = LivenessInfo.EMPTY; + LivenessInfo existingLiveness = existing.livenessInfo; + LivenessInfo livenessInfo = LivenessInfo.merge(update.livenessInfo, existingLiveness); + this.heapSize += livenessInfo.unsharedHeapSize() - existingLiveness.unsharedHeapSize(); Object[] tree = BTreeRow.mergeRowBTrees(this, existing.columnsBTree, update.columnsBTree, - deletion, existing.deletion); - return new RowData(tree, livenessInfo, deletion); + DeletionTime.LIVE, DeletionTime.LIVE); + return new RowData(tree, livenessInfo); } - private Clustering clusteringFor(InMemoryTrie.KeyProducer keyState) + private Clustering clusteringFor(InMemoryBaseTrie.KeyProducer keyState) { return metadata.comparator.clusteringFromByteComparable( ByteArrayAccessor.instance, ByteComparable.preencoded(TrieBackedPartition.BYTE_COMPARABLE_VERSION, - keyState.getBytes(TrieMemtable.IS_PARTITION_BOUNDARY))); + keyState.getBytes(TrieBackedPartition.IS_PARTITION_BOUNDARY))); } /** * Called at the partition boundary to merge the existing and new metadata associated with the partition. This needs - * to update the deletion time with any new deletion introduced by the update, but also make sure that the - * statistics we track for the partition (dataSize) are updated for the changes caused by merging the update's rows - * (note that this is called _after_ the rows of the partition have been merged, on the return path of the - * recursion). + * to make sure that the statistics we track for the partition (dataSize) are updated for the changes caused by + * merging the update's rows. * * @param existing Any partition data already associated with the partition. - * @param update The update, always non-null. - * @return the combined partition data, copying any updated deletion information to heap. + * @return the combined partition data, creating a new marker if one did not already exist. */ - private TrieMemtable.PartitionData applyDeletion(TrieMemtable.PartitionData existing, DeletionInfo update) + private TrieMemtable.PartitionData mergePartitionMarkers(@Nullable TrieMemtable.PartitionData existing) { - if (indexer != UpdateTransaction.NO_OP) - { - if (!update.getPartitionDeletion().isLive()) - indexer.onPartitionDeletion(update.getPartitionDeletion()); - if (update.hasRanges()) - update.rangeIterator(false).forEachRemaining(indexer::onRangeTombstone); - } + // TODO: Check if we need to call onPartitionDeletion if (existing == null) { // Note: Always on-heap, regardless of cloner - TrieMemtable.PartitionData newRef = new TrieMemtable.PartitionData(update, owner); + TrieMemtable.PartitionData newRef = new TrieMemtable.PartitionData(owner); this.heapSize += newRef.unsharedHeapSize(); ++this.partitionsAdded; return currentPartition = newRef; } assert owner == existing.owner; - if (update.isLive() || !update.mayModify(existing)) - return currentPartition = existing; - - // Note: Always on-heap, regardless of cloner - TrieMemtable.PartitionData merged = new TrieMemtable.PartitionData(existing, update); - this.heapSize += merged.unsharedHeapSize() - existing.unsharedHeapSize(); - return currentPartition = merged; + return currentPartition = existing; } } diff --git a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdaterStage2.java b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdaterStage2.java index fdc489289e90..af995db32cb7 100644 --- a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdaterStage2.java +++ b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdaterStage2.java @@ -25,7 +25,7 @@ import org.apache.cassandra.db.marshal.ByteArrayAccessor; import org.apache.cassandra.db.memtable.TrieMemtableStage2; import org.apache.cassandra.db.rows.BTreeRow; -import org.apache.cassandra.db.tries.InMemoryTrie; +import org.apache.cassandra.db.tries.InMemoryBaseTrie; import org.apache.cassandra.index.transactions.UpdateTransaction; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.bytecomparable.ByteComparable; @@ -38,7 +38,7 @@ */ public final class TriePartitionUpdaterStage2 extends BasePartitionUpdater -implements InMemoryTrie.UpsertTransformerWithKeyProducer +implements InMemoryBaseTrie.UpsertTransformerWithKeyProducer { private final UpdateTransaction indexer; private final TableMetadata metadata; @@ -58,7 +58,7 @@ public TriePartitionUpdaterStage2(Cloner cloner, } @Override - public Object apply(Object existing, Object update, InMemoryTrie.KeyProducer keyState) + public Object apply(Object existing, Object update, InMemoryBaseTrie.KeyProducer keyState) { if (update instanceof RowData) return applyRow((RowData) existing, (RowData) update, keyState); @@ -76,7 +76,7 @@ else if (update instanceof DeletionInfo) * @param keyState Used to obtain the path through which this node was reached. * @return the insert row, or the merged row, copied using our allocator */ - private RowData applyRow(RowData existing, RowData insert, InMemoryTrie.KeyProducer keyState) + private RowData applyRow(RowData existing, RowData insert, InMemoryBaseTrie.KeyProducer keyState) { if (existing == null) { @@ -119,7 +119,7 @@ private RowData merge(RowData existing, RowData update) return new RowData(tree, livenessInfo, deletion); } - private Clustering clusteringFor(InMemoryTrie.KeyProducer keyState) + private Clustering clusteringFor(InMemoryBaseTrie.KeyProducer keyState) { return metadata.comparator.clusteringFromByteComparable( ByteArrayAccessor.instance, diff --git a/src/java/org/apache/cassandra/db/rows/BTreeRow.java b/src/java/org/apache/cassandra/db/rows/BTreeRow.java index c2a5f0fbd3a4..c6bbd7f414ce 100644 --- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java +++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java @@ -285,6 +285,11 @@ && deletion().isLive() && BTree.isEmpty(btree); } + public boolean isEmptyAfterDeletion() + { + return primaryKeyLivenessInfo().isEmpty() && BTree.isEmpty(btree); + } + public Deletion deletion() { return deletion; diff --git a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java index 38b84f7015bd..d3f5723cbfd0 100644 --- a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java +++ b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java @@ -28,11 +28,8 @@ import org.apache.cassandra.db.DeletionTime; import org.apache.cassandra.db.Digest; import org.apache.cassandra.db.LivenessInfo; -import org.apache.cassandra.db.context.CounterContext; import org.apache.cassandra.db.filter.ColumnFilter; -import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.ByteType; -import org.apache.cassandra.db.marshal.CollectionType; import org.apache.cassandra.db.marshal.SetType; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.DroppedColumn; @@ -215,6 +212,15 @@ public ComplexColumnData filter(ColumnFilter filter, DeletionTime activeDeletion }); } + public ComplexColumnData delete(DeletionTime activeDeletion) + { + if (activeDeletion.isLive() || !activeDeletion.supersedes(complexDeletion)) + return this; + + DeletionTime newDeletion = activeDeletion.supersedes(complexDeletion) ? DeletionTime.LIVE : complexDeletion; + return transformAndFilter(newDeletion, (cell) -> activeDeletion.deletes(cell) ? null : cell); + } + public ComplexColumnData purge(DeletionPurger purger, int nowInSec) { DeletionTime newDeletion = complexDeletion.isLive() || purger.shouldPurge(complexDeletion) ? DeletionTime.LIVE : complexDeletion; diff --git a/src/java/org/apache/cassandra/db/rows/Row.java b/src/java/org/apache/cassandra/db/rows/Row.java index 7d4d58c50985..4e1b6318b689 100644 --- a/src/java/org/apache/cassandra/db/rows/Row.java +++ b/src/java/org/apache/cassandra/db/rows/Row.java @@ -113,12 +113,20 @@ public interface Row extends Unfiltered, Iterable */ public boolean isEmpty(); + /** + * Whether the row has no live data. This means no PK liveness info, no cells + * and no complex deletion info. + * + * @return {@code true} if the row has no data, {@code false} otherwise. + */ + public boolean isEmptyAfterDeletion(); + /** * Whether the row has some live information (i.e. it's not just deletion informations). * * @param nowInSec the current time to decide what is deleted and what isn't * @param enforceStrictLiveness whether the row should be purged if there is no PK liveness info, - * normally retrieved from {@link CFMetaData#enforceStrictLiveness()} + * normally retrieved from {@link TableMetadata#enforceStrictLiveness()} * @return true if there is some live information */ public boolean hasLiveData(int nowInSec, boolean enforceStrictLiveness); diff --git a/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarker.java b/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarker.java new file mode 100644 index 000000000000..a07a3ab1cd92 --- /dev/null +++ b/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarker.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.rows; + +import org.apache.cassandra.cache.IMeasurableMemory; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.tries.RangeState; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// A tombstone marker used in trie-backed structures. Unlike [RangeTombstoneMarker], this does not include a position +/// in the clustering order, only deletion times. +/// There are two kinds of trie markers: +/// - covering markers, which have a deletion time applicable to some position that is not a boundary in the deletions +/// branch, +/// - boundary markers, which switch from one deletion time to a different one (either of which may be null). +/// +/// See [RangeState] for further explanation of the types of markers. +public interface TrieTombstoneMarker extends RangeState, IMeasurableMemory +{ + /// Returns the deletion time applicable at this point. Normally this would be called on non-boundary states only, + /// but generally makes sense on any boundary as the boundary itself is included. + DeletionTime deletionTime(); + + /// Converts this marker to [RangeTombstoneMarker], assigning it a clustering position from its byte-comparable + /// path in the trie. This is only applicable to boundary markers and will throw if called on covering ones. + /// + /// The given `deletionToOmit` can be used to omit deletion times that are already covered by some higher-level + /// marker (e.g. partition deletion). + RangeTombstoneMarker toRangeTombstoneMarker(ByteComparable clusteringPrefixAsByteComparable, + ByteComparable.Version byteComparableVersion, + ClusteringComparator comparator, + DeletionTime deletionToOmit); + + + /// Combine two markers and return the applicable combined state, obtained by getting the higher of the deletion + /// times on both sides of the marker. For boundaries this may result in a covering state (when both sides become + /// equal) which is not stored or reported. + TrieTombstoneMarker mergeWith(TrieTombstoneMarker existing); + + static TrieTombstoneMarker covering(DeletionTime deletionTime) + { + return TrieTombstoneMarkerImpl.covering(deletionTime); + } + + TrieTombstoneMarker withUpdatedTimestamp(long l); +} diff --git a/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarkerImpl.java b/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarkerImpl.java new file mode 100644 index 000000000000..3dacd6786231 --- /dev/null +++ b/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarkerImpl.java @@ -0,0 +1,299 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.rows; + +import java.util.Objects; +import javax.annotation.Nullable; + +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.marshal.ByteArrayAccessor; +import org.apache.cassandra.db.tries.Direction; +import org.apache.cassandra.utils.ObjectSizes; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/// The implementation of trie tombstone markers. +/// +/// To save some object creation, the `Covering` subtype extends `DeletionTime`, and the `Boundary` subtypes stores the +/// sides as instances of `Covering`. +interface TrieTombstoneMarkerImpl extends TrieTombstoneMarker +{ + Covering leftDeletion(); + Covering rightDeletion(); + + static Covering covering(DeletionTime deletionTime) + { + return new Covering(deletionTime); + } + + static Covering combine(Covering left, Covering right) + { + if (left == null) + return right; + if (right == null) + return left; + if (right.supersedes(left)) + return right; + else + return left; + } + + static TrieTombstoneMarker make(Covering left, Covering right) + { + if (left == right) // includes both being null + return left; + + if (left != null && left.equals(right)) + return left; + + return new Boundary(left, right); + } + + static class Covering extends DeletionTime implements TrieTombstoneMarkerImpl + { + static final long HEAP_SIZE = ObjectSizes.measure(new Covering(DeletionTime.LIVE)); + + private Covering(DeletionTime deletionTime) + { + super(deletionTime.markedForDeleteAt(), deletionTime.localDeletionTime()); + } + + private Covering(long markedForDeleteAt, int localDeletionTime) + { + super(markedForDeleteAt, localDeletionTime); + } + + @Override + public RangeTombstoneMarker toRangeTombstoneMarker(ByteComparable clusteringPrefixAsByteComparable, + ByteComparable.Version byteComparableVersion, + ClusteringComparator comparator, + DeletionTime deletionToOmit) + { + throw new AssertionError("Covering trie tombstone cannot be converted to a RangeTombstoneMarker"); + } + + @Override + public Covering leftDeletion() + { + return this; + } + + @Override + public Covering rightDeletion() + { + return this; + } + + @Override + public TrieTombstoneMarker mergeWith(TrieTombstoneMarker other) + { + if (other instanceof Boundary) + return other.mergeWith(this); + + return combine(this, (Covering) other); + } + + @Override + public Covering withUpdatedTimestamp(long l) + { + return new Covering(l, localDeletionTime()); + } + + @Override + public boolean isBoundary() + { + return false; + } + + @Override + public TrieTombstoneMarker precedingState(Direction direction) + { + return this; + } + + @Override + public TrieTombstoneMarker restrict(boolean applicableBefore, boolean applicableAfter) + { + throw new AssertionError("Restrict is only applicable to boundary markers"); + } + + @Override + public TrieTombstoneMarker asBoundary(Direction direction) + { + return direction.isForward() ? new Boundary(null, this) : new Boundary(this, null); + } + + @Override + public DeletionTime deletionTime() + { + return this; + } + + @Override + public long unsharedHeapSize() + { + // Note: HEAP_SIZE is used directly by Point and Boundary. Make sure to apply any changes there too. + return HEAP_SIZE; + } + } + + static class Boundary implements TrieTombstoneMarkerImpl + { + // Every boundary contains one side of a deletion, and for simplicity we assume that any covering deletion we + // interrupt is already accounted for by its end boundaries, so with every new Boundary we add this object's + // size plus one half of a Covering. + static final long UNSHARED_HEAP_SIZE = + ObjectSizes.measure(new Boundary(new Covering(0, 0), null)) + + Covering.HEAP_SIZE / 2; + + final @Nullable Covering leftDeletion; + final @Nullable Covering rightDeletion; + + private Boundary(@Nullable Covering left, @Nullable Covering right) + { + assert left != null || right != null; + assert left == null || !left.isLive(); + assert right == null || !right.isLive(); + this.leftDeletion = left; + this.rightDeletion = right; + } + + @Override + public DeletionTime deletionTime() + { + // Report the higher deletion, to avoid dropping the other side of boundaries that switch to any omitted + // deletion time. + return leftDeletion == null ? rightDeletion + : rightDeletion == null ? leftDeletion + : rightDeletion.supersedes(leftDeletion) ? rightDeletion + : leftDeletion; + } + + @Override + public RangeTombstoneMarker toRangeTombstoneMarker(ByteComparable clusteringPrefixAsByteComparable, + ByteComparable.Version byteComparableVersion, + ClusteringComparator comparator, + DeletionTime deletionToOmit) + { + assert byteComparableVersion == ByteComparable.Version.OSS50; + if (leftDeletion == null || leftDeletion.equals(deletionToOmit)) + { + if (rightDeletion == null || rightDeletion.equals(deletionToOmit)) + return null; + else + return new RangeTombstoneBoundMarker(comparator.boundFromByteComparable(ByteArrayAccessor.instance, + clusteringPrefixAsByteComparable, + false), + rightDeletion); + } + + if (rightDeletion == null || rightDeletion.equals(deletionToOmit)) + return new RangeTombstoneBoundMarker(comparator.boundFromByteComparable(ByteArrayAccessor.instance, + clusteringPrefixAsByteComparable, + true), + leftDeletion); + + return new RangeTombstoneBoundaryMarker(comparator.boundaryFromByteComparable(ByteArrayAccessor.instance, + clusteringPrefixAsByteComparable), + leftDeletion, + rightDeletion); + } + + @Override + public TrieTombstoneMarker mergeWith(TrieTombstoneMarker existing) + { + if (existing == null) + return this; + + TrieTombstoneMarkerImpl other = (TrieTombstoneMarkerImpl) existing; + Covering otherLeft = other.leftDeletion(); + Covering newLeft = combine(leftDeletion, otherLeft); + Covering otherRight = other.rightDeletion(); + Covering newRight = combine(rightDeletion, otherRight); + if (leftDeletion == newLeft && rightDeletion == newRight) + return this; + if (otherLeft == newLeft && otherRight == newRight) + return other; + return make(newLeft, newRight); + } + + @Override + public TrieTombstoneMarker withUpdatedTimestamp(long l) + { + Covering newLeft = leftDeletion != null ? leftDeletion.withUpdatedTimestamp(l) : null; + Covering newRight = rightDeletion != null ? rightDeletion.withUpdatedTimestamp(l) : null; + if (Objects.equals(newLeft, newRight)) + return null; + return new Boundary(newLeft, newRight); + } + + @Override + public boolean isBoundary() + { + return true; + } + + @Override + public TrieTombstoneMarker precedingState(Direction dir) + { + return dir.isForward() ? leftDeletion : rightDeletion; + } + + @Override + public TrieTombstoneMarker restrict(boolean applicableBefore, boolean applicableAfter) + { + if (!applicableAfter && leftDeletion == null || !applicableBefore && rightDeletion == null) + return null; + if (applicableBefore && applicableAfter) + return this; + return new Boundary(applicableBefore ? leftDeletion : null, + applicableAfter ? rightDeletion : null); + } + + @Override + public TrieTombstoneMarker asBoundary(Direction direction) + { + throw new AssertionError("Already a boundary"); + } + + @Override + public Covering leftDeletion() + { + return leftDeletion; + } + + @Override + public Covering rightDeletion() + { + return rightDeletion; + } + + @Override + public String toString() + { + return (leftDeletion != null ? leftDeletion : "LIVE") + " -> " + (rightDeletion != null ? rightDeletion : "LIVE"); + } + + @Override + public long unsharedHeapSize() + { + return UNSHARED_HEAP_SIZE; + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java b/src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java index 2ca18e2cd25d..71703c7091cc 100644 --- a/src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java +++ b/src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.function.BiFunction; import java.util.function.Function; @@ -27,6 +28,7 @@ import com.google.common.collect.ImmutableList; import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; /// Deletion-aware trie interface that combines live data and deletion information in a unified structure. /// @@ -320,6 +322,45 @@ DeletionAwareTrie merge(Collection> sour } } + static > DeletionAwareTrie mergeDistinct(List> tries) + { + return merge(tries, throwingResolver()); + } + + @SuppressWarnings("unchecked") + static > CollectionMergeResolver throwingResolver() + { + return THROWING_RESOLVER; + } + + @SuppressWarnings("rawtypes") + static final CollectionMergeResolver THROWING_RESOLVER = new CollectionMergeResolver() + { + @Override + public Object resolve(Collection contents) + { + throw new AssertionError("Distinct tries expected"); + } + + @Override + public Object applyMarker(RangeState marker, Object content) + { + throw new AssertionError("Distinct tries expected"); + } + + @Override + public RangeState resolveMarkers(Collection markers) + { + throw new AssertionError("Distinct tries expected"); + } + + @Override + public boolean deletionsAtFixedPoints() + { + return true; + } + }; + /// Walker interface extended to also process deletion branches. interface DeletionAwareWalker extends Cursor.Walker { @@ -355,6 +396,35 @@ default R process(Direction direction, DeletionAwareWalker dac = cursor(Direction.FORWARD); + final ByteSource bytes = key.asComparableBytes(dac.byteComparableVersion()); + int next; + int depth = dac.depth(); + RangeCursor rc; + while (true) + { + rc = dac.deletionBranchCursor(Direction.FORWARD); + if (rc != null) + break; + next = bytes.next(); + if (next == ByteSource.END_OF_STREAM) + return null; // no deletion branch found + if (dac.skipTo(++depth, next) != depth || dac.incomingTransition() != next) + return null; + } + + if (rc.descendAlong(bytes)) + return rc.state(); + else + return rc.precedingState(); + } + + /// Returns a view of the live content in this trie as a regular [Trie]. default Trie contentOnlyTrie() { diff --git a/src/java/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrie.java index 92c3d3efbedb..b9f8a8dd9e7a 100644 --- a/src/java/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrie.java +++ b/src/java/org/apache/cassandra/db/tries/InMemoryDeletionAwareTrie.java @@ -336,9 +336,9 @@ private int hoistOurDeletionBranches() throws TrieSpaceExhaustedException /// value. Applied even if there's no pre-existing value in the memtable trie. The transformer can return null /// if the entry should not be added or preserved. /// @param deletionTransformer a function applied to combine overlapping deletions into a consistent view. Called - /// even if there is no pre-existing deletion to convert the marker type. For code simplicity this transformer is - /// not provided with the path to the modified markers. The transformer can return null if deletions cancel out - /// or should not be preserved. + /// even if there is no pre-existing deletion to convert the marker type. The transformer can return null if + /// deletions cancel out or should not be preserved. + /// **Note: for code simplicity this transformer is provided only the path to the root of the deletion branch.** /// @param existingDeleter a function used to apply a deletion marker to potentially delete live data. This is /// only called if there is both content and deletion at a given covered point. It should return null if the entry /// is to be deleted. @@ -350,7 +350,7 @@ private int hoistOurDeletionBranches() throws TrieSpaceExhaustedException public > void apply(DeletionAwareTrie mutation, final UpsertTransformerWithKeyProducer dataTransformer, - final UpsertTransformer deletionTransformer, + final UpsertTransformerWithKeyProducer deletionTransformer, final UpsertTransformerWithKeyProducer existingDeleter, final BiFunction insertedDeleter, boolean deletionsAtFixedPoints, diff --git a/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java b/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java index 0909579ad982..2ebf51ec319f 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java +++ b/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java @@ -28,6 +28,9 @@ /// Convertor of trie entries to iterator where each entry is passed through [#mapContent] (to be implemented by /// descendants). +/// +/// If [#mapContent] returns null, this version of the class will pass on that null upstream. If this is not the desired +/// behaviour, see [WithNullFiltering]. public abstract class TrieEntriesIterator extends TriePathReconstructor implements Iterator { private final Cursor cursor; @@ -79,6 +82,9 @@ ByteComparable.Version byteComparableVersion() return cursor.byteComparableVersion(); } + /// To be implemented by descendants to map the content value and path to the required entry. If callers need to + /// save the path, they must copy the `bytes` array, which will be overwritten when the iteration continues. + /// If this method returns null, the null will be passed on as an entry in the iteration. protected abstract V mapContent(T content, byte[] bytes, int byteLength); /** @@ -120,4 +126,76 @@ static java.util.Map.Entry toEntry(ByteCompara { return new AbstractMap.SimpleImmutableEntry<>(toByteComparable(version, bytes, byteLength), content); } + + /// Convertor of trie entries to iterator where each entry is passed through [#mapContent] (to be implemented by + /// descendants). This is the same as [TrieEntriesIterator], but instead of accepting a predicate to filter out entries, + /// it skips over ones where [#mapContent] returns null. + public static abstract class WithNullFiltering extends TriePathReconstructor implements Iterator + { + protected final Cursor cursor; + V next; + boolean gotNext; + + protected WithNullFiltering(Trie trie, Direction direction) + { + this(trie.cursor(direction)); + } + + WithNullFiltering(Cursor cursor) + { + this.cursor = cursor; + assert cursor.depth() == 0; + T nextContent = cursor.content(); + if (nextContent != null) + { + next = mapContent(nextContent, keyBytes, keyPos); + gotNext = next != null; + } + else + gotNext = false; + } + + public boolean hasNext() + { + while (!gotNext) + { + T nextContent = cursor.advanceToContent(this); + if (nextContent != null) + { + next = mapContent(nextContent, keyBytes, keyPos); + gotNext = next != null; + } + else + gotNext = true; + } + + return next != null; + } + + public V next() + { + if (!hasNext()) + throw new IllegalStateException("next without hasNext"); + + return consumeNext(); + } + + protected V consumeNext() + { + gotNext = false; + V v = next; + next = null; + return v; + } + + protected V peekNextIfAvailable() + { + return next; // null if not prepared + } + + /// To be implemented by descendants to map the content value and path to the required entry. If callers need to + /// save the path, they must copy the `bytes` array, which will be overwritten when the iteration continues. + /// If this method returns null, the iteration will skip over the current position. + protected abstract V mapContent(T content, byte[] bytes, int byteLength); + } } diff --git a/src/java/org/apache/cassandra/index/sai/memory/TrieMemtableIndex.java b/src/java/org/apache/cassandra/index/sai/memory/TrieMemtableIndex.java index c16336c476a2..2babad66162f 100644 --- a/src/java/org/apache/cassandra/index/sai/memory/TrieMemtableIndex.java +++ b/src/java/org/apache/cassandra/index/sai/memory/TrieMemtableIndex.java @@ -87,7 +87,7 @@ public class TrieMemtableIndex extends AbstractMemtableIndex public TrieMemtableIndex(IndexContext indexContext, Memtable memtable) { - this(indexContext, memtable, TrieMemtable.SHARD_COUNT); + this(indexContext, memtable, TrieMemtable.shardCount()); } @VisibleForTesting diff --git a/src/java/org/apache/cassandra/index/sai/utils/RowWithSourceTable.java b/src/java/org/apache/cassandra/index/sai/utils/RowWithSourceTable.java index 797a4ca374b8..38bec262fdf2 100644 --- a/src/java/org/apache/cassandra/index/sai/utils/RowWithSourceTable.java +++ b/src/java/org/apache/cassandra/index/sai/utils/RowWithSourceTable.java @@ -131,6 +131,12 @@ public boolean isEmpty() return row.isEmpty(); } + @Override + public boolean isEmptyAfterDeletion() + { + return row.isEmptyAfterDeletion(); + } + @Override public String toString(TableMetadata metadata) { diff --git a/src/java/org/apache/cassandra/utils/bytecomparable/ByteSource.java b/src/java/org/apache/cassandra/utils/bytecomparable/ByteSource.java index 9b73c6d5454b..24ddc489e886 100644 --- a/src/java/org/apache/cassandra/utils/bytecomparable/ByteSource.java +++ b/src/java/org/apache/cassandra/utils/bytecomparable/ByteSource.java @@ -108,6 +108,7 @@ default int nextBytes(byte[] dest) // Special value for components that should be excluded from the normal min/max span. (static rows) int EXCLUDED = 0x18; + int LT_EXCLUDED = 0x10; /** * Encodes byte-accessible data as a byte-comparable source that has 0s escaped and finishes in an escaped diff --git a/test/burn/org/apache/cassandra/index/sai/LongBM25Test.java b/test/burn/org/apache/cassandra/index/sai/LongBM25Test.java index 49b5b5118540..b894054c83d6 100644 --- a/test/burn/org/apache/cassandra/index/sai/LongBM25Test.java +++ b/test/burn/org/apache/cassandra/index/sai/LongBM25Test.java @@ -32,10 +32,13 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.junit.Before; +import org.junit.Assert; +import org.junit.BeforeClass; import org.junit.Test; import org.slf4j.Logger; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.memtable.TrieMemtable; public class LongBM25Test extends SAITester @@ -81,14 +84,13 @@ public class LongBM25Test extends SAITester } KeySet keysInserted = new KeySet(); - private final int threadCount = 12; + private static final int threadCount = 12; - @Before - public void setup() throws Throwable + @BeforeClass + public static void setUpClass() { - // we don't get loaded until after TM, so we can't affect the very first memtable, - // but this will affect all subsequent ones - TrieMemtable.SHARD_COUNT = 4 * threadCount; + CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_COUNT.setInt(4 * threadCount); + SAITester.setUpClass(); } @FunctionalInterface @@ -99,6 +101,7 @@ private interface Op public void testConcurrentOps(Op op) throws ExecutionException, InterruptedException { + Assert.assertEquals(4 * threadCount, TrieMemtable.shardCount()); createTable("CREATE TABLE %s (key int primary key, value text)"); // Create analyzed index following BM25Test pattern createIndex("CREATE CUSTOM INDEX ON %s(value) " + diff --git a/test/burn/org/apache/cassandra/index/sai/LongVectorTest.java b/test/burn/org/apache/cassandra/index/sai/LongVectorTest.java index 7123b30fa140..4d23fef1d625 100644 --- a/test/burn/org/apache/cassandra/index/sai/LongVectorTest.java +++ b/test/burn/org/apache/cassandra/index/sai/LongVectorTest.java @@ -28,10 +28,13 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.junit.Before; +import org.junit.Assert; +import org.junit.BeforeClass; import org.junit.Test; import org.slf4j.Logger; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.memtable.TrieMemtable; import static org.assertj.core.api.Assertions.assertThat; @@ -44,14 +47,13 @@ public class LongVectorTest extends SAITester int dimension = 16; // getRandom().nextIntBetween(128, 768); KeySet keysInserted = new KeySet(); - private final int threadCount = 12; + private final static int threadCount = 12; - @Before - public void setup() throws Throwable + @BeforeClass + public static void setUpClass() { - // we don't get loaded until after TM, so we can't affect the very first memtable, - // but this will affect all subsequent ones - TrieMemtable.SHARD_COUNT = 4 * threadCount; + CassandraRelevantProperties.TRIE_MEMTABLE_SHARD_COUNT.setInt(4 * threadCount); + SAITester.setUpClass(); } @FunctionalInterface @@ -62,6 +64,7 @@ private interface Op public void testConcurrentOps(Op op) throws ExecutionException, InterruptedException { + Assert.assertEquals(4 * threadCount, TrieMemtable.shardCount()); createTable(String.format("CREATE TABLE %%s (key int primary key, value vector)", dimension)); createIndex("CREATE CUSTOM INDEX ON %s(value) USING 'StorageAttachedIndex' WITH OPTIONS = { 'similarity_function': 'dot_product' }"); diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java index b0aa80803b6b..cf0c6cea5f76 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java @@ -1003,20 +1003,27 @@ public void testDeletions() // delete the same row again execute("DELETE FROM %s USING TIMESTAMP 3 WHERE a = ? AND b = ?", 0, 0); - assertEquals(2, index1.rowsUpdated.size()); - update = index1.rowsUpdated.get(1); - existingRow = update.left; - newRow = update.right; - // check the new row from the update call - assertFalse(existingRow.deletion().isLive()); - assertEquals(2L, existingRow.deletion().time().markedForDeleteAt()); - assertFalse(existingRow.cells().iterator().hasNext()); + // TrieMemtable (stage 3) no longer has a row to issue an update for and the deletion is treated as a range deletion. + // Legacy memtable implementations would still keep a mapping for the deleted row and issue an update. + if (index1.rowsUpdated.size() > 1) + { + assertEquals(2, index1.rowsUpdated.size()); + update = index1.rowsUpdated.get(1); + existingRow = update.left; + newRow = update.right; - // check the new row from the update call - assertFalse(newRow.deletion().isLive()); - assertEquals(3L, newRow.deletion().time().markedForDeleteAt()); - assertFalse(newRow.cells().iterator().hasNext()); + + // check the new row from the update call + assertFalse(existingRow.deletion().isLive()); + assertEquals(2L, existingRow.deletion().time().markedForDeleteAt()); + assertFalse(existingRow.cells().iterator().hasNext()); + + // check the new row from the update call + assertFalse(newRow.deletion().isLive()); + assertEquals(3L, newRow.deletion().time().markedForDeleteAt()); + assertFalse(newRow.cells().iterator().hasNext()); + } } @Test diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java index e83502233d7d..01a79ee41970 100644 --- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java +++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java @@ -26,6 +26,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterators; @@ -43,8 +44,8 @@ import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.memtable.TrieMemtable; import org.apache.cassandra.db.partitions.FilteredPartition; -import org.apache.cassandra.db.partitions.ImmutableBTreePartition; import org.apache.cassandra.db.partitions.Partition; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; @@ -61,6 +62,7 @@ import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.MemtableParams; import org.apache.cassandra.schema.SchemaTestUtil; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.ByteBufferUtil; @@ -81,11 +83,13 @@ public class RangeTombstoneTest private static final String CFNAME_INDEXED = "StandardIntegerIndexed"; public static final int GC_GRACE = 5000; - @Parameterized.Parameters(name = "compaction={0}") - public static Iterable compactionParamSets() + @Parameterized.Parameters(name = "compaction={0} memtable={1}") + public static Iterable compactionParamSets() { - return ImmutableSet.of(CompactionParams.stcs(ImmutableMap.of()), - CompactionParams.ucs(ImmutableMap.of())); + return ImmutableSet.of(new Object[] {CompactionParams.stcs(ImmutableMap.of()), "SkipListMemtable"}, + new Object[] {CompactionParams.lcs(ImmutableMap.of()), "TrieMemtableStage1"}, + new Object[] {CompactionParams.ucs(ImmutableMap.of()), "TrieMemtableStage2"}, + new Object[] {CompactionParams.ucs(ImmutableMap.of()), "TrieMemtable"}); } @BeforeClass @@ -98,14 +102,15 @@ public static void defineSchema() throws ConfigurationException standardCFMD(KSNAME, CFNAME_INDEXED, 1, UTF8Type.instance, Int32Type.instance, Int32Type.instance)); } - public RangeTombstoneTest(CompactionParams compactionParams) + public RangeTombstoneTest(CompactionParams compactionParams, String memtableClass) { + MemtableParams memtableParams = MemtableParams.fromMap(ImmutableMap.of("class", memtableClass)); Keyspace ks = Keyspace.open(KSNAME); ColumnFamilyStore cfs = ks.getColumnFamilyStore(CFNAME); - SchemaTestUtil.announceTableUpdate(cfs.metadata().unbuild().compaction(compactionParams).build()); + SchemaTestUtil.announceTableUpdate(cfs.metadata().unbuild().compaction(compactionParams).memtable(memtableParams).build()); cfs.disableAutoCompaction(); // don't trigger compaction at 4 sstables cfs = ks.getColumnFamilyStore(CFNAME_INDEXED); - SchemaTestUtil.announceTableUpdate(cfs.metadata().unbuild().compaction(compactionParams).build()); + SchemaTestUtil.announceTableUpdate(cfs.metadata().unbuild().compaction(compactionParams).memtable(memtableParams).build()); cfs.disableAutoCompaction(); // don't trigger compaction at 4 sstables } @@ -193,11 +198,12 @@ public void rangeTombstoneFilteringTest() throws Exception Collection rt = rangeTombstones(partition); assertEquals(0, rt.size()); - partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).fromIncl(11).toIncl(15).build()); + // Note: if the slice start matches the tombstone end, the result is transformed to deleted row. + partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).fromIncl(11).toIncl(16).build()); rt = rangeTombstones(partition); assertEquals(1, rt.size()); - partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).fromIncl(20).toIncl(25).build()); + partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).fromIncl(19).toIncl(25).build()); rt = rangeTombstones(partition); assertEquals(1, rt.size()); @@ -229,7 +235,7 @@ public void rangeTombstoneFilteringTest() throws Exception rt = rangeTombstones(partition); assertEquals(0, rt.size()); - partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).fromIncl(1).toIncl(5).build()); + partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).fromIncl(1).toIncl(6).build()); rt = rangeTombstones(partition); assertEquals(1, rt.size()); @@ -237,7 +243,7 @@ public void rangeTombstoneFilteringTest() throws Exception rt = rangeTombstones(partition); assertEquals(1, rt.size()); - partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).fromIncl(5).toIncl(6).build()); + partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).fromIncl(4).toIncl(6).build()); rt = rangeTombstones(partition); assertEquals(1, rt.size()); @@ -733,18 +739,28 @@ public void testOverwritesToDeletedColumns() throws Exception UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0).newRow(1).add("val", 1).applyUnsafe(); - // add a RT which hides the column we just inserted + // add a RT which hides the row we just inserted new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(0, 1).build().applyUnsafe(); - // now re-insert that column + // now re-insert that row UpdateBuilder.create(cfs.metadata(), key).withTimestamp(2).newRow(1).add("val", 1).applyUnsafe(); cfs.forceBlockingFlush(UNIT_TESTS); - // We should have 1 insert and 1 update to the indexed "1" column - // CASSANDRA-6640 changed index update to just update, not insert then delete - assertEquals(1, index.rowsInserted.size()); - assertEquals(1, index.rowsUpdated.size()); + if (cfs.getCurrentMemtable() instanceof TrieMemtable) + { + // TrieMemtable deletes the row on receiving the tombstone (issuing an update), and then insert it a second + // time. + assertEquals(2, index.rowsInserted.size()); + assertEquals(1, index.rowsUpdated.size()); + } + else + { + // Legacy memtables will keep the row shadowed and update it. + assertEquals(1, index.rowsInserted.size()); + assertEquals(1, index.rowsUpdated.size()); + } + } private static ByteBuffer bb(int i) diff --git a/test/unit/org/apache/cassandra/index/sai/cql/FeaturesVersionSupportTest.java b/test/unit/org/apache/cassandra/index/sai/cql/FeaturesVersionSupportTest.java index e294c341609a..67c0b416ae6c 100644 --- a/test/unit/org/apache/cassandra/index/sai/cql/FeaturesVersionSupportTest.java +++ b/test/unit/org/apache/cassandra/index/sai/cql/FeaturesVersionSupportTest.java @@ -270,7 +270,8 @@ public void testIndexMetaForNumRows() assertNumRowsMemtable(bodyIndexName, DATASET.length, totalTermsCount); assertNumRowsMemtable(mapIndexName, DATASET.length); execute("DELETE FROM %s WHERE id = ?", 4); - // Deletion is not tracked by Memindex + // Since TrieMemtable stage 3, deletion is tracked by Memindex + totalTermsCount -= calculateTotalTermsForRow(4); assertNumRowsMemtable(bodyIndexName, DATASET.length, totalTermsCount); // Test an update to a different value for the analyzed index execute("UPDATE %s SET body = ? WHERE id = ?", DATASET[10][DATASET_BODY_COLUMN], 6); @@ -280,8 +281,6 @@ public void testIndexMetaForNumRows() execute("UPDATE %s SET body = ? WHERE id = ?", DATASET[6][DATASET_BODY_COLUMN], 10); totalTermsCount += calculateTotalTermsForRow(6) - calculateTotalTermsForRow(10); assertNumRowsMemtable(bodyIndexName, DATASET.length, totalTermsCount); - // Flush will account for the deleted row - totalTermsCount -= calculateTotalTermsForRow(4); flush(); assertNumRowsAndTotalTermsSSTable(scoreIndexName, DATASET.length - 1, DATASET.length - 1); assertNumRowsAndTotalTermsSSTable(bodyIndexName, DATASET.length - 1, totalTermsCount); diff --git a/test/unit/org/apache/cassandra/index/sai/cql/LuceneUpdateDeleteTest.java b/test/unit/org/apache/cassandra/index/sai/cql/LuceneUpdateDeleteTest.java index cd8641754f8f..ba483003d76b 100644 --- a/test/unit/org/apache/cassandra/index/sai/cql/LuceneUpdateDeleteTest.java +++ b/test/unit/org/apache/cassandra/index/sai/cql/LuceneUpdateDeleteTest.java @@ -605,9 +605,20 @@ public void testRangeDeletionThenOverwrite() throws Throwable // delete range execute("DELETE FROM %s WHERE pk = 0"); - // Still expect both rows to be in the index because range deletion doesn't remove from index - searchMemtable(indexName, "indexed", 0); - searchMemtable(indexName, "random", 1, 0); + if (version.equals(Version.AA)) + { + // Still expect both rows to be in the index because range deletion doesn't remove from index + searchMemtable(indexName, "indexed", 0); + searchMemtable(indexName, "random", 1, 0); + searchMemtable(indexName, "something", 0); // range deleted, but not yet removed + } + else + { + // The range deletion causes an update since trie memtable stage 3 + searchMemtable(indexName, "indexed"); + searchMemtable(indexName, "random", 1); + searchMemtable(indexName, "something"); + } // Overwrite the value for the first of the 2 rows in partition 0 execute("INSERT INTO %s (pk, x, val) VALUES (0, 0, 'random')"); @@ -619,15 +630,15 @@ public void testRangeDeletionThenOverwrite() throws Throwable searchMemtable(indexName, "phrase", 1, 0); // random is in all 3 memtable index rows, but only 2 partitions, and AA indexes partition keys searchMemtable(indexName, "random", 1, 0); + searchMemtable(indexName, "something", 0); // range deleted, but not yet removed } else { searchMemtable(indexName, "indexed"); // overwritten, and the update removes the value searchMemtable(indexName, "phrase", 1); // was deleted/overwritten in 0, so just in 1 now - searchMemtable(indexName, "random", 1, 0, 0); // random is in all 3 memtable index rows + searchMemtable(indexName, "random", 1, 0); // random is in all 3 memtable index rows + searchMemtable(indexName, "something"); } - // True for all versions - searchMemtable(indexName, "something", 0); // range deleted, but not yet removed } @Test diff --git a/test/unit/org/apache/cassandra/index/sai/memory/TrieMemtableIndexTestBase.java b/test/unit/org/apache/cassandra/index/sai/memory/TrieMemtableIndexTestBase.java index afa742f768a2..9324065acc2f 100644 --- a/test/unit/org/apache/cassandra/index/sai/memory/TrieMemtableIndexTestBase.java +++ b/test/unit/org/apache/cassandra/index/sai/memory/TrieMemtableIndexTestBase.java @@ -119,7 +119,7 @@ public static void setup(Config.MemtableAllocationType allocationType) @Before public void setup() throws Throwable { - assertEquals(8, TrieMemtable.SHARD_COUNT); + assertEquals(8, TrieMemtable.shardCount()); TokenMetadata metadata = StorageService.instance.getTokenMetadata(); metadata.updateNormalTokens(BootStrapper.getRandomTokens(metadata, 10), FBUtilities.getBroadcastAddressAndPort()); @@ -144,9 +144,9 @@ public void setup() throws Throwable @Test public void allocation() throws Throwable { - assertEquals(8, TrieMemtable.SHARD_COUNT); + assertEquals(8, TrieMemtable.shardCount()); memtableIndex = new TrieMemtableIndex(indexContext, memtable); - assertEquals(TrieMemtable.SHARD_COUNT, memtableIndex.shardCount()); + assertEquals(TrieMemtable.shardCount(), memtableIndex.shardCount()); assertEquals(0, memtable.getAllocator().onHeap().owns()); assertEquals(0, memtable.getAllocator().offHeap().owns()); @@ -168,7 +168,7 @@ public void allocation() throws Throwable public void randomQueryTest() throws Exception { memtableIndex = new TrieMemtableIndex(indexContext, memtable); - assertEquals(TrieMemtable.SHARD_COUNT, memtableIndex.shardCount()); + assertEquals(TrieMemtable.shardCount(), memtableIndex.shardCount()); for (int row = 0; row < getRandom().nextIntBetween(1000, 5000); row++) { From 45bc6b15b3ceb2f1e106d44e5160cd2bb616fae8 Mon Sep 17 00:00:00 2001 From: blambov Date: Thu, 4 Sep 2025 17:05:21 +0300 Subject: [PATCH 18/22] Implement, test and benchmark stopIssuingTombstones Adds a new method to UnfilteredRowIterator that is implemented by the new trie-backed partitions to ask them to stop issuing tombstones. This is done on filtering (i.e. conversion from UnfilteredRowIterator to RowIterator) where tombstones have already done their job and are no longer needed. Adds JMH tests of tombstones that demonstrate the improvement. --- .../RecombiningUnfilteredRowIterator.java | 19 +++ .../db/partitions/TrieBackedPartition.java | 23 +++- .../db/rows/UnfilteredRowIterator.java | 10 ++ .../apache/cassandra/db/transform/Filter.java | 3 +- .../cassandra/db/transform/FilteredRows.java | 1 + .../db/transform/UnfilteredRows.java | 12 ++ .../db/tries/DeletionAwareCursor.java | 37 ++++++ .../cassandra/db/tries/DeletionAwareTrie.java | 17 +++ .../db/tries/TrieEntriesIterator.java | 2 +- .../test/microbench/instance/ReadTest.java | 121 ++++++++++++++++++ .../instance/ReadTestWidePartitions.java | 6 + .../miscellaneous/TombstonesTest.java | 74 +++++++++-- 12 files changed, 308 insertions(+), 17 deletions(-) diff --git a/src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java index 93faf962f84e..6eb973c4ab41 100644 --- a/src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java +++ b/src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java @@ -49,6 +49,25 @@ protected RecombiningUnfilteredRowIterator(UnfilteredRowIterator wrapped) nextPrepared = false; } + @Override + public boolean stopIssuingTombstones() + { + if (!wrapped.stopIssuingTombstones()) + return false; + + if (nextPrepared && next == null) + return true; // this is already exhausted + + // clear all buffered state that includes tombstones + if (bufferedTwo != null && bufferedTwo.isRangeTombstoneMarker()) + bufferedTwo = null; + if (bufferedOne != null && bufferedOne.isRangeTombstoneMarker()) + bufferedOne = bufferedTwo; // can be null + if (nextPrepared && next.isRangeTombstoneMarker()) + nextPrepared = false; + return true; + } + @Override public boolean hasNext() { diff --git a/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java index 8a8981303a92..e174bb0049e7 100644 --- a/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java +++ b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java @@ -53,7 +53,6 @@ import org.apache.cassandra.db.tries.InMemoryBaseTrie; import org.apache.cassandra.db.tries.InMemoryDeletionAwareTrie; import org.apache.cassandra.db.tries.InMemoryTrie; -import org.apache.cassandra.db.tries.Trie; import org.apache.cassandra.db.tries.TrieEntriesIterator; import org.apache.cassandra.db.tries.TrieSet; import org.apache.cassandra.db.tries.TrieSpaceExhaustedException; @@ -513,16 +512,18 @@ class UnfilteredIterator final boolean reversed; final ColumnFilter selection; final DeletionTime partitionLevelDeletion; + final DeletionAwareTrie trie; final Row staticRow; - protected UnfilteredIterator(ColumnFilter selection, Trie trie, boolean reversed) + protected UnfilteredIterator(ColumnFilter selection, DeletionAwareTrie trie, boolean reversed) { this(selection, trie, reversed, TrieBackedPartition.this.partitionLevelDeletion()); } - private UnfilteredIterator(ColumnFilter selection, Trie trie, boolean reversed, DeletionTime partitionLevelDeletion) + private UnfilteredIterator(ColumnFilter selection, DeletionAwareTrie trie, boolean reversed, DeletionTime partitionLevelDeletion) { - super(trie, Direction.fromBoolean(reversed)); + super(trie.mergedTrieSwitchable((x, y) -> x instanceof RowData ? x : y), Direction.fromBoolean(reversed)); + this.trie = trie; this.selection = selection; this.reversed = reversed; this.partitionLevelDeletion = partitionLevelDeletion; @@ -595,6 +596,17 @@ public void close() { // nothing to close } + + @Override + public boolean stopIssuingTombstones() + { + ((DeletionAwareTrie.DeletionsStopControl) cursor).stopIssuingDeletions(this); + + Unfiltered next = peekNextIfAvailable(); + if (next != null && next.isRangeTombstoneMarker()) + consumeNext(); + return true; + } } public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, ByteComparable[] bounds, boolean reversed) @@ -603,8 +615,7 @@ public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, ByteComp return UnfilteredRowIterators.noRowsIterator(metadata, partitionKey, staticRow(), partitionLevelDeletion(), reversed); DeletionAwareTrie slicedTrie = trie.intersect(TrieSet.ranges(BYTE_COMPARABLE_VERSION, bounds)); - Trie mergedTrie = slicedTrie.mergedTrie((x, y) -> x instanceof RowData ? x : y); - return new RecombiningUnfilteredRowIterator(new UnfilteredIterator(selection, mergedTrie, reversed)); + return new RecombiningUnfilteredRowIterator(new UnfilteredIterator(selection, slicedTrie, reversed)); } public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, Slices slices, boolean reversed) diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterator.java index bbef11e8495a..9cd2f1b4da2b 100644 --- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterator.java +++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterator.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db.rows; import org.apache.cassandra.db.*; +import org.apache.cassandra.db.transform.FilteredRows; /** * An iterator over the rows of a given partition that also includes deletion informations. @@ -66,4 +67,13 @@ public default boolean isEmpty() && staticRow().isEmpty() && !hasNext(); } + + /// Ask the iterator to stop issuing tombstones because they are no longer useful to the consumer. + /// Usually done just before filtering out tombstones in [FilteredRows]. + /// If the iterator will not gain efficiency from dropping tombstones, it can reject this call by returning false. + public default boolean stopIssuingTombstones() + { + // ignored in base class + return false; + } } diff --git a/src/java/org/apache/cassandra/db/transform/Filter.java b/src/java/org/apache/cassandra/db/transform/Filter.java index 0bd3eabe6a33..759695759857 100644 --- a/src/java/org/apache/cassandra/db/transform/Filter.java +++ b/src/java/org/apache/cassandra/db/transform/Filter.java @@ -38,9 +38,10 @@ public Filter(int nowInSec, boolean enforceStrictLiveness) @SuppressWarnings("resource") protected RowIterator applyToPartition(BaseRowIterator iterator) { + ((UnfilteredRowIterator) iterator).stopIssuingTombstones(); return iterator instanceof UnfilteredRows ? new FilteredRows(this, (UnfilteredRows) iterator) - : new FilteredRows((UnfilteredRowIterator) iterator, this); + : new FilteredRows(((UnfilteredRowIterator) iterator), this); } @Override diff --git a/src/java/org/apache/cassandra/db/transform/FilteredRows.java b/src/java/org/apache/cassandra/db/transform/FilteredRows.java index 349183ca6e68..883b96389575 100644 --- a/src/java/org/apache/cassandra/db/transform/FilteredRows.java +++ b/src/java/org/apache/cassandra/db/transform/FilteredRows.java @@ -55,6 +55,7 @@ public boolean isEmpty() */ public static RowIterator filter(UnfilteredRowIterator iterator, int nowInSecs) { + iterator.stopIssuingTombstones(); return new Filter(nowInSecs, iterator.metadata().enforceStrictLiveness()).applyToPartition(iterator); } } diff --git a/src/java/org/apache/cassandra/db/transform/UnfilteredRows.java b/src/java/org/apache/cassandra/db/transform/UnfilteredRows.java index b8720fcdaa12..0acd866d8aab 100644 --- a/src/java/org/apache/cassandra/db/transform/UnfilteredRows.java +++ b/src/java/org/apache/cassandra/db/transform/UnfilteredRows.java @@ -73,4 +73,16 @@ public boolean isEmpty() { return staticRow().isEmpty() && partitionLevelDeletion().isLive() && !hasNext(); } + + @Override + public boolean stopIssuingTombstones() + { + if (!input.stopIssuingTombstones()) + return false; + + // If we are stopping tombstones, we must check if any already prepared `next` is a tombstone and drop it if so. + if (next != null && next.isRangeTombstoneMarker()) + next = null; + return true; + } } diff --git a/src/java/org/apache/cassandra/db/tries/DeletionAwareCursor.java b/src/java/org/apache/cassandra/db/tries/DeletionAwareCursor.java index c54efa68e710..3a30e2d10dab 100644 --- a/src/java/org/apache/cassandra/db/tries/DeletionAwareCursor.java +++ b/src/java/org/apache/cassandra/db/tries/DeletionAwareCursor.java @@ -170,6 +170,43 @@ public LiveAndDeletionsMergeCursor tailCursor(Direction direction) } } + /// A variant of [LiveAndDeletionsMergeCursor] that can be asked to stop issuing deletion markers. + class SwitchableLiveAndDeletionsMergeCursor, Z> + extends LiveAndDeletionsMergeCursor + implements DeletionAwareTrie.DeletionsStopControl + { + boolean stopIssuingDeletions = false; + + SwitchableLiveAndDeletionsMergeCursor(BiFunction resolver, DeletionAwareCursor c1) + { + super(resolver, c1); + } + + public void stopIssuingDeletions(ResettingTransitionsReceiver receiver) + { + stopIssuingDeletions = true; + // drop any already open deletion branch + switch (state) + { + case AT_C2: + // we need to exit the deletion branch at the next advance + c2 = RangeCursor.empty(direction, byteComparableVersion()); + break; + default: + state = state.C1_ONLY; + break; + } + } + + @Override + int postAdvance(int depth) + { + if (stopIssuingDeletions) + return depth; + return super.postAdvance(depth); + } + } + /// A cursor presenting the deletion markers of a deletion-aware trie. /// /// This cursor combines all deletion branches into a single trie. Because it is not known where a deletion branch diff --git a/src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java b/src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java index 71703c7091cc..bb1c84518e7e 100644 --- a/src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java +++ b/src/java/org/apache/cassandra/db/tries/DeletionAwareTrie.java @@ -445,6 +445,23 @@ default Trie mergedTrie(BiFunction resolver) return dir -> new DeletionAwareCursor.LiveAndDeletionsMergeCursor<>(resolver, cursor(dir)); } + /// Interface used to ask a cursor to stop issuing deletions. Provided by the cursor implementing + /// [#mergedTrieSwitchable]. + interface DeletionsStopControl + { + void stopIssuingDeletions(Cursor.ResettingTransitionsReceiver receiver); + } + + /// Returns a view of the combination of the live data and deletions in this trie as a regular [Trie], using + /// the provided mapping function to covert values to a common type. + /// + /// The only difference with [#mergedTrie] is that this cursor can be asked to stop visiting deletion branches + /// via the [DeletionsStopControl] interface. + default Trie mergedTrieSwitchable(BiFunction resolver) + { + return dir -> new DeletionAwareCursor.SwitchableLiveAndDeletionsMergeCursor<>(resolver, cursor(dir)); + } + static > DeletionAwareTrie empty(ByteComparable.Version byteComparableVersion) { diff --git a/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java b/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java index 2ebf51ec319f..173f2ebfd892 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java +++ b/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java @@ -33,7 +33,7 @@ /// behaviour, see [WithNullFiltering]. public abstract class TrieEntriesIterator extends TriePathReconstructor implements Iterator { - private final Cursor cursor; + protected final Cursor cursor; private final Predicate predicate; T next; boolean gotNext; diff --git a/test/microbench/org/apache/cassandra/test/microbench/instance/ReadTest.java b/test/microbench/org/apache/cassandra/test/microbench/instance/ReadTest.java index f2eff8f67de0..c4f52f90dc51 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/instance/ReadTest.java +++ b/test/microbench/org/apache/cassandra/test/microbench/instance/ReadTest.java @@ -20,6 +20,7 @@ import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Random; import java.util.concurrent.ExecutorService; @@ -30,6 +31,8 @@ import com.google.common.base.Throwables; +import org.apache.commons.codec.digest.MurmurHash3; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.ColumnFamilyStore; @@ -76,6 +79,52 @@ public enum Flush @Param({"1"}) int threadCount = 1; + @Param({"0"}) + double deletionsRatio = 0; + + @Param({"EQUAL"}) + DeletionSpec deletionSpec = DeletionSpec.EQUAL; + + public enum DeletionPattern + { + RANDOM, + FROM_START, + SPREAD; + } + + public enum DeletionSpec + { + EQUAL("picid = ?", 0), + SINGLETON_RANGE("picid >= ? AND picid <= ?", 0, 0), + RANGE_TO_NEXT("picid >= ? AND picid < ?", 0, -1), + RANGE_10("picid >= ? AND picid < ?", 0, 10), + RANGE_FROM_START("picid <= ?", 0); + + final int[] argumentShifts; + final String spec; + + DeletionSpec(String spec, int... argumentShifts) + { + this.argumentShifts = argumentShifts; + this.spec = spec; + } + + Object[] convertArgs(Object[] args, ReadTest test) + { + Object[] result = new Object[1 + argumentShifts.length]; + result[0] = args[0]; + long column = (Long) args[1]; + for (int i = 0; i < argumentShifts.length; ++i) + result[i + 1] = column + (argumentShifts[i] >= 0 ? argumentShifts[i] : test.getDiffToNext() * -argumentShifts[i]); + return result; + } + } + + @Param({"RANDOM"}) + DeletionPattern deletionPattern = DeletionPattern.RANDOM; + + long deletionCount; + ExecutorService executorService; @Setup(Level.Trial) @@ -118,6 +167,19 @@ public void setup() throws Throwable performWrite(writeStatement, i, BATCH); if (i < count) performWrite(writeStatement, i, count - i); + + deletionCount = Math.min((long) (count * deletionsRatio), count); + if (deletionCount > 0) + { + String deleteStatement = "DELETE FROM " + table + " WHERE userid = ? AND " + deletionSpec.spec; + System.err.println("Deleting " + deletionCount + " using " + deleteStatement); + + for (i = 0; i <= deletionCount - BATCH; i += BATCH) + performDelete(deleteStatement, i, BATCH); + if (i < deletionCount) + performDelete(deleteStatement, i, deletionCount - i); + } + long writeLength = System.currentTimeMillis() - writeStart; System.err.format("... done in %.3f s.\n", writeLength / 1000.0); @@ -154,6 +216,11 @@ public void setup() throws Throwable abstract Object[] writeArguments(long i); + long getDiffToNext() + { + return 1; + } + public void performWrite(String writeStatement, long ofs, long count) throws Throwable { if (threadCount == 1) @@ -193,6 +260,60 @@ public void performWriteThreads(String writeStatement, long ofs, long count) thr assert count == done; } + long deleteIndex(long index) + { + switch (deletionPattern) + { + case FROM_START: + return index; + case RANDOM: + return Integer.remainderUnsigned(MurmurHash3.hash32(index), count); + case SPREAD: + return index * count / deletionCount; + default: + throw new AssertionError(); + } + } + + public void performDelete(String deleteStatement, long ofs, long count) throws Throwable + { + if (threadCount == 1) + performDeleteSerial(deleteStatement, ofs, count); + else + performDeleteThreads(deleteStatement, ofs, count); + } + + public void performDeleteSerial(String deleteStatement, long ofs, long count) throws Throwable + { + for (long i = ofs; i < ofs + count; ++i) + execute(deleteStatement, deletionSpec.convertArgs(writeArguments(deleteIndex(i)), this)); + } + + public void performDeleteThreads(String deleteStatement, long ofs, long count) throws Throwable + { + List> futures = new ArrayList<>(); + for (long i = 0; i < count; ++i) + { + long pos = ofs + i; + futures.add(executorService.submit(() -> + { + try + { + execute(deleteStatement, deletionSpec.convertArgs(writeArguments(deleteIndex(pos)), ReadTest.this)); + return 1; + } + catch (Throwable throwable) + { + throw Throwables.propagate(throwable); + } + })); + } + long done = 0; + for (Future f : futures) + done += f.get(); + assert count == done; + } + @TearDown(Level.Trial) public void teardown() throws InterruptedException { diff --git a/test/microbench/org/apache/cassandra/test/microbench/instance/ReadTestWidePartitions.java b/test/microbench/org/apache/cassandra/test/microbench/instance/ReadTestWidePartitions.java index c36e09fb7a35..d428a4139592 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/instance/ReadTestWidePartitions.java +++ b/test/microbench/org/apache/cassandra/test/microbench/instance/ReadTestWidePartitions.java @@ -33,6 +33,12 @@ public Object[] writeArguments(long i) return new Object[] { i % partitions, i, i }; } + @Override + long getDiffToNext() + { + return partitions; + } + Object[] readArguments(long i, long offset) { return new Object[] { (i + offset) % partitions, i }; diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java index 7983fdc820a7..373a3701ad54 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.cql3.validation.miscellaneous; +import java.util.Arrays; import java.util.concurrent.TimeUnit; import com.google.common.base.Throwables; @@ -24,22 +25,43 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.filter.TombstoneOverwhelmingException; +import org.apache.cassandra.db.memtable.TrieMemtable; import static junit.framework.Assert.assertEquals; import static junit.framework.Assert.assertTrue; import static junit.framework.Assert.fail; +import static org.junit.Assert.assertFalse; /** * Test that TombstoneOverwhelmingException gets thrown when it should be and doesn't when it shouldn't be. */ +@RunWith(Parameterized.class) public class TombstonesTest extends CQLTester { + @Parameterized.Parameter + public String memtableClass; + + @Parameterized.Parameter(1) + public boolean flush; + + @Parameterized.Parameters(name = "{0} flush: {1}") + public static Iterable parameters() + { + return Arrays.asList(new Object[] {"SkipListMemtable", false}, + new Object[] {"TrieMemtableStage1", true}, // this uses the same partition code as SkipListMemtable + new Object[] {"TrieMemtableStage2", false}, // this flushes like SkipListMemtable, no need to test flushed + new Object[] {"TrieMemtable", false}, + new Object[] {"TrieMemtable", true}); + } + static final int ORIGINAL_FAILURE_THRESHOLD = DatabaseDescriptor.getGuardrailsConfig().tombstone_failure_threshold; static final int FAILURE_THRESHOLD = 100; @@ -65,20 +87,23 @@ public static void tearDown() public void testBelowThresholdSelect() throws Throwable { - String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));"); + String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b)) WITH memtable = {'class': '" + memtableClass + "'};"); ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName); long oldFailures = cfs.metric.tombstoneFailures.getCount(); long oldWarnings = cfs.metric.tombstoneWarnings.getCount(); + boolean tombstonesCountTowardsThresholds = flush || !(cfs.getCurrentMemtable() instanceof TrieMemtable); // insert exactly the amount of tombstones that shouldn't trigger an exception for (int i = 0; i < FAILURE_THRESHOLD; i++) execute("DELETE FROM %s WHERE a = 'key' and b = '" + i + "'"); + if (flush) + flush(); try { execute("SELECT * FROM %s WHERE a = 'key';"); assertEquals(oldFailures, cfs.metric.tombstoneFailures.getCount()); - assertEquals(oldWarnings + 1, cfs.metric.tombstoneWarnings.getCount()); + assertEquals(oldWarnings + (tombstonesCountTowardsThresholds ? 1 : 0), cfs.metric.tombstoneWarnings.getCount()); } catch (Throwable e) { @@ -89,22 +114,27 @@ public void testBelowThresholdSelect() throws Throwable @Test public void testBeyondThresholdSelect() throws Throwable { - String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));"); + String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b)) WITH memtable = {'class': '" + memtableClass + "'};"); ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName); long oldFailures = cfs.metric.tombstoneFailures.getCount(); long oldWarnings = cfs.metric.tombstoneWarnings.getCount(); + boolean tombstonesCountTowardsThresholds = flush || !(cfs.getCurrentMemtable() instanceof TrieMemtable); // insert exactly the amount of tombstones that *SHOULD* trigger an exception for (int i = 0; i < FAILURE_THRESHOLD + 1; i++) execute("DELETE FROM %s WHERE a = 'key' and b = '" + i + "'"); + if (flush) + flush(); try { execute("SELECT * FROM %s WHERE a = 'key';"); - fail("SELECT with tombstones beyond the threshold should have failed, but hasn't"); + assertFalse("SELECT with tombstones beyond the threshold should have failed, but hasn't", tombstonesCountTowardsThresholds); } catch (Throwable e) { + assertTrue(memtableClass + " should not be affected by the number of tombstones", tombstonesCountTowardsThresholds); + String error = "Expected exception instanceof TombstoneOverwhelmingException instead got " + System.lineSeparator() + Throwables.getStackTraceAsString(e); @@ -117,7 +147,18 @@ public void testBeyondThresholdSelect() throws Throwable @Test public void testAllShadowedSelect() throws Throwable { - String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));"); + testAllShadowedSelect(false); + } + + @Test + public void testAllShadowedInSeparateSSTable() throws Throwable + { + testAllShadowedSelect(true); + } + + public void testAllShadowedSelect(boolean flushBetween) throws Throwable + { + String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b)) WITH memtable = {'class': '" + memtableClass + "'};"); ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName); long oldFailures = cfs.metric.tombstoneFailures.getCount(); long oldWarnings = cfs.metric.tombstoneWarnings.getCount(); @@ -126,9 +167,15 @@ public void testAllShadowedSelect() throws Throwable for (int i = 0; i < FAILURE_THRESHOLD + 1; i++) execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);"); + if (flushBetween) + flush(); + // delete all with a partition level tombstone execute("DELETE FROM %s WHERE a = 'key'"); + if (flush) + flush(); + try { execute("SELECT * FROM %s WHERE a = 'key';"); @@ -144,7 +191,7 @@ public void testAllShadowedSelect() throws Throwable @Test public void testLiveShadowedCellsSelect() throws Throwable { - String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));"); + String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b)) WITH memtable = {'class': '" + memtableClass + "'};"); ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName); long oldFailures = cfs.metric.tombstoneFailures.getCount(); long oldWarnings = cfs.metric.tombstoneWarnings.getCount(); @@ -155,6 +202,9 @@ public void testLiveShadowedCellsSelect() throws Throwable // delete all with a partition level tombstone execute("DELETE FROM %s WHERE a = 'key'"); + if (flush) + flush(); + try { execute("SELECT * FROM %s WHERE a = 'key';"); @@ -170,13 +220,15 @@ public void testLiveShadowedCellsSelect() throws Throwable @Test public void testExpiredTombstones() throws Throwable { - String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b)) WITH gc_grace_seconds = 1;"); + String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b)) WITH gc_grace_seconds = 1 AND memtable = {'class': '" + memtableClass + "'};"); ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName); long oldFailures = cfs.metric.tombstoneFailures.getCount(); long oldWarnings = cfs.metric.tombstoneWarnings.getCount(); for (int i = 0; i < FAILURE_THRESHOLD + 1; i++) execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);"); + if (flush) + flush(); // not yet past gc grace - must throw a TOE try @@ -211,18 +263,22 @@ public void testExpiredTombstones() throws Throwable @Test public void testBeyondWarnThresholdSelect() throws Throwable { - String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a,b));"); + String tableName = createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a,b)) WITH memtable = {'class': '" + memtableClass + "'};"); ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName); long oldFailures = cfs.metric.tombstoneFailures.getCount(); long oldWarnings = cfs.metric.tombstoneWarnings.getCount(); + boolean tombstonesCountTowardsThresholds = flush || !(cfs.getCurrentMemtable() instanceof TrieMemtable); // insert the number of tombstones that *SHOULD* trigger an Warning for (int i = 0; i < WARN_THRESHOLD + 1; i++) execute("DELETE FROM %s WHERE a = 'key' and b = '" + i + "'"); + if (flush) + flush(); + try { execute("SELECT * FROM %s WHERE a = 'key';"); - assertEquals(oldWarnings + 1, cfs.metric.tombstoneWarnings.getCount()); + assertEquals(oldWarnings + (tombstonesCountTowardsThresholds ? 1 : 0), cfs.metric.tombstoneWarnings.getCount()); assertEquals(oldFailures, cfs.metric.tombstoneFailures.getCount()); } catch (Throwable e) From 855f58714e3b1fd9c9eae407ec02643659bc2edb Mon Sep 17 00:00:00 2001 From: blambov Date: Thu, 25 Sep 2025 16:44:53 +0300 Subject: [PATCH 19/22] Add trie slicing support for SAI uses --- .../db/tries/IntersectionCursor.java | 105 +++- .../org/apache/cassandra/db/tries/Trie.java | 20 + .../index/sai/memory/TrieMemoryIndex.java | 23 +- .../cassandra/db/tries/SlicedTrieTest.java | 225 ++++--- .../cassandra/db/tries/SubtrieTest.java | 559 ++++++++++++++++++ 5 files changed, 828 insertions(+), 104 deletions(-) create mode 100644 test/unit/org/apache/cassandra/db/tries/SubtrieTest.java diff --git a/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java b/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java index d112225e462e..2c5ced48a2d6 100644 --- a/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java +++ b/src/java/org/apache/cassandra/db/tries/IntersectionCursor.java @@ -41,7 +41,7 @@ enum State this.direction = source.direction(); this.source = source; this.set = set; - matchingPosition(depth()); + setInitialState(); } @Override @@ -159,7 +159,7 @@ private int coveredAreaWithSetAhead(int depth) return depth; } - private int matchingPosition(int depth) + int matchingPosition(int depth) { // If we are matching a boundary of the set, include all its children by using a set-ahead state, ensuring that // the set will only be advanced once the source ascends to its depth again. @@ -170,6 +170,11 @@ private int matchingPosition(int depth) return depth; } + void setInitialState() + { + matchingPosition(depth()); + } + private int exhausted() { state = State.MATCHING; @@ -194,6 +199,79 @@ public ByteComparable.Version byteComparableVersion() return source.byteComparableVersion(); } + /// A variation of the intersection cursor that supports boundary inclusivity control and does not report content + /// in prefixes. + /// + /// Note: Exclusivity is ignored for empty bounds, i.e. if a boundary is empty, it is treated like null regardless + /// of the inclusivity flag. + abstract static class Slice> extends IntersectionCursor + { + final boolean startsInclusive; + final boolean endsInclusive; + + Slice(C source, TrieSetCursor set, boolean startsInclusive, boolean endsInclusive) + { + super(source, set); + + this.startsInclusive = startsInclusive; + this.endsInclusive = endsInclusive; + } + + @Override + void setInitialState() + { + // Check if the set is fully unbounded, and make sure the empty position is reported if this is the case. + TrieSetCursor.RangeState setState = set.state(); + if (setState == TrieSetCursor.RangeState.END_START_PREFIX || setState.isBoundary) + state = State.SET_AHEAD; + else + state = State.MATCHING; + } + + @Override + int matchingPosition(int depth) + { + TrieSetCursor.RangeState setState = set.state(); + if (!setState.isBoundary) + { + // This is a prefix, we still have set path bytes to follow. + state = State.MATCHING; + return depth; + } + + // If the boundary is a start (for the direction of iteration), and we include starts, we should include branch. + // Also, if the boundary is an end (for the direction of travel), and we include ends. + if ((setState.precedingIncluded(Direction.FORWARD) || startsInclusive) && + (setState.precedingIncluded(Direction.REVERSE) || endsInclusive)) + { + // Report the content, and include all the branch's children by using a set-ahead state, ensuring that + // the set will only be advanced once the source ascends to this depth again. + state = State.SET_AHEAD; + return depth; + } + + // Otherwise we need to skip this node and its branch by jumping to the next position on the same depth. + // Note that we can't mess up any `advanceMultiple` path reporting, as that cannot end up on a matching + // position while it is reporting bytes for a descending chain. + return skipTo(depth, incomingTransition() + direction.increase); + } + + @Override + public T content() + { + switch (state) + { + case SET_AHEAD: + return source.content(); + case MATCHING: + // This is a prefix (boundaries we either skip or mark as SET_AHEAD). Report if it leads to an end bound. + return set.state().precedingIncluded(Direction.FORWARD) ? source.content() : null; + default: + throw new AssertionError(); + } + } + } + /// Intersection cursor for [Trie]. static class Plain extends IntersectionCursor> { @@ -217,6 +295,29 @@ public Cursor tailCursor(Direction direction) } } + /// Slice cursor for [Trie]. + static class PlainSlice extends Slice> + { + public PlainSlice(Cursor source, TrieSetCursor set, boolean startsInclusive, boolean endsInclusive) + { + super(source, set, startsInclusive, endsInclusive); + } + + @Override + public Cursor tailCursor(Direction direction) + { + switch (state) + { + case MATCHING: + return new PlainSlice<>(source.tailCursor(direction), set.tailCursor(direction), startsInclusive, endsInclusive); + case SET_AHEAD: + return source.tailCursor(direction); + default: + throw new AssertionError(); + } + } + } + static class DeletionAware> extends IntersectionCursor> implements DeletionAwareCursor diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java index cb7ab14138c7..b18bd26e5a37 100644 --- a/src/java/org/apache/cassandra/db/tries/Trie.java +++ b/src/java/org/apache/cassandra/db/tries/Trie.java @@ -70,6 +70,26 @@ default Trie intersect(TrieSet set) return dir -> new IntersectionCursor.Plain<>(cursor(dir), set.cursor(dir)); } + /// A version of subtrie that supports control over the inclusivity of bounds. + /// Neither of the two bounds can be a prefix of the other. + /// + /// Unlike `subtrie`, prefixes of the boundaries are reported only if they fall in the span (i.e. are prefixes of a + /// right bound), and the branches rooted at the relevant boundary are covered if and only if the boundary itself is + /// included. + /// + /// For example, `slice(20, false, 40, true)` excludes `2020` (which is a descendant of the excluded `20`) but + /// includes `4040` (a descendant of the included `40`). + default Trie slice(ByteComparable left, boolean inclusiveLeft, ByteComparable right, boolean inclusiveRight) + { + return dir -> { + Cursor cursor = cursor(dir); + return new IntersectionCursor.PlainSlice<>(cursor, + new RangesCursor(dir, cursor.byteComparableVersion(), left, right), + inclusiveLeft, + inclusiveRight); + }; + } + /// Returns the values in any order. For some tries this is much faster than the ordered iterable. default Iterable valuesUnordered() { diff --git a/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java b/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java index 32716a54c3a5..218c18cb113c 100644 --- a/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java +++ b/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java @@ -385,17 +385,17 @@ private KeyRangeIterator rangeMatch(Expression expression, AbstractBounds { + subtrie.forEachEntry((keyComparable, value) -> { // Before version DB, we encoded composite types using a non order-preserving function. In order to // perform a range query on a map, we use the bounds to get all entries for a given map key and then // only keep the map entries that satisfy the expression. - assert entry.getKey().encodingVersion() == TypeUtil.BYTE_COMPARABLE_VERSION || Version.current() == Version.AA; - byte[] key = ByteSourceInverse.readBytes(entry.getKey().getPreencodedBytes()); + assert keyComparable.encodingVersion() == TypeUtil.BYTE_COMPARABLE_VERSION || Version.current() == Version.AA; + byte[] key = ByteSourceInverse.readBytes(keyComparable.getPreencodedBytes()); if (expression.isSatisfiedBy(ByteBuffer.wrap(key))) - mergingIteratorBuilder.add(entry.getValue()); + mergingIteratorBuilder.add(value); }); else - subtrie.values().forEach(mergingIteratorBuilder::add); + subtrie.forEachValue(mergingIteratorBuilder::add); return mergingIteratorBuilder.isEmpty() ? KeyRangeIterator.empty() @@ -523,27 +523,34 @@ private Trie getSubtrie(@Nullable Expression expression) return data; ByteComparable lowerBound, upperBound; + boolean lowerInclusive, upperInclusive; if (expression.lower != null) { lowerBound = expression.getEncodedLowerBoundByteComparable(Version.current()); - // inclusivity is encoded in lowerBound + lowerInclusive = expression.lower.inclusive || + // Pre-D versions use first-component prefixes and post-filtering for composite lookups. + // To make sure we get all the content in the relevant branch, we force inclusivity on the + // start bound (otherwise Trie would skip the branch, see [Trie#slice]. + !Version.current().onOrAfter(Version.DB) && TypeUtil.isComposite(expression.validator); } else { lowerBound = null; + lowerInclusive = false; } if (expression.upper != null) { upperBound = expression.getEncodedUpperBoundByteComparable(Version.current()); - // inclusivity is encoded in upperBound + upperInclusive = expression.upper.inclusive; } else { upperBound = null; + upperInclusive = false; } - return data.subtrie(lowerBound, upperBound); + return data.slice(lowerBound, lowerInclusive, upperBound, upperInclusive); } public ByteBuffer getMinTerm() diff --git a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java index 7d74275e22ac..b548463bb05b 100644 --- a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java +++ b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java @@ -19,6 +19,7 @@ package org.apache.cassandra.db.tries; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -43,7 +44,6 @@ import static org.apache.cassandra.db.tries.TrieUtil.asString; import static org.apache.cassandra.db.tries.TrieUtil.assertSameContent; import static org.apache.cassandra.db.tries.TrieUtil.generateKeys; -import static org.apache.cassandra.db.tries.TrieUtil.toBound; import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; import static org.junit.Assert.assertEquals; @@ -136,12 +136,17 @@ public void testIntersectRange(int count) private static Preencoded[] toByteComparable(String[] keys) { return Arrays.stream(keys) - .map(TrieUtil::comparable) + .map(x -> ByteComparable.preencoded(VERSION, x.getBytes(StandardCharsets.UTF_8))) .toArray(Preencoded[]::new); } + static boolean isTruePrefix(ByteComparable b, ByteComparable key) + { + return ByteComparable.compare(b, key, VERSION) != 0 && key.byteComparableAsString(VERSION).startsWith(b.byteComparableAsString(VERSION)); + } + @Test - public void testSingletonSubtrie() + public void testSingletonSlice() { Arrays.sort(BOUNDARIES, (a, b) -> ByteComparable.compare(a, b, VERSION)); for (int li = -1; li < BOUNDARIES.length; ++li) @@ -150,6 +155,8 @@ public void testSingletonSubtrie() for (int ri = Math.max(0, li); ri <= BOUNDARIES.length; ++ri) { Preencoded r = ri == BOUNDARIES.length ? null : BOUNDARIES[ri]; + if (l != null && r != null && (isTruePrefix(l, r) || isTruePrefix(r, l))) + continue; // prefixes in bounds are not supported for (int i = li == ri ? 3 : 0; i < 4; ++i) { @@ -158,9 +165,9 @@ public void testSingletonSubtrie() for (Preencoded key : KEYS) { - int cmp1 = l != null ? ByteComparable.compare(key, l, VERSION) : 1; - int cmp2 = r != null ? ByteComparable.compare(r, key, VERSION) : 1; - Trie ix = Trie.singleton(key, VERSION, true).subtrie(toBound(l, !includeLeft), toBound(r, includeRight)); + int cmp1 = l != null ? !isTruePrefix(l, key) ? ByteComparable.compare(key, l, VERSION) : 0 : 1; + int cmp2 = r != null ? !isTruePrefix(r, key) ? ByteComparable.compare(r, key, VERSION) : 0 : 1; + Trie ix = Trie.singleton(key, VERSION, true).slice(l, includeLeft, r, includeRight); boolean expected = true; if (cmp1 < 0 || cmp1 == 0 && !includeLeft) expected = false; @@ -187,7 +194,7 @@ public void testSingletonSubtrie() } @Test - public void testMemtableSubtrie() + public void testMemtableSlice() { Arrays.sort(BOUNDARIES, FORWARD_COMPARATOR); NavigableMap content1 = new TreeMap<>(FORWARD_COMPARATOR); @@ -212,24 +219,24 @@ public void testMemtableSubtrie() } @Test - public void testMergeSubtrie() + public void testMergeSlice() { - testMergeSubtrie(2); + testMergeSlice(2); } @Test - public void testCollectionMergeSubtrie3() + public void testCollectionMergeSlice3() { - testMergeSubtrie(3); + testMergeSlice(3); } @Test - public void testCollectionMergeSubtrie5() + public void testCollectionMergeSlice5() { - testMergeSubtrie(5); + testMergeSlice(5); } - public void testMergeSubtrie(int mergeCount) + public void testMergeSlice(int mergeCount) { Arrays.sort(BOUNDARIES, FORWARD_COMPARATOR); NavigableMap content1 = new TreeMap<>(FORWARD_COMPARATOR); @@ -262,17 +269,38 @@ public void testMergeSubtrie(int mergeCount) } } + static Preencoded nudge(Preencoded v) + { + if (v == null) + return null; + + byte[] data = v.getPreencodedBytes().remainingBytesToArray(); + int len = data.length; + while (len > 0 && data[len-1] == -1) + --len; + + if (len == 0) + return null; + + ++data[len - 1]; + return ByteComparable.preencoded(v.encodingVersion(), data, 0, len); + } - static NavigableMap boundedMap(NavigableMap sourceMap, Preencoded l, boolean includeLeft, Preencoded r, boolean includeRight) + static NavigableMap boundedMap(NavigableMap sourceMap, Preencoded ll, boolean includeLeft, Preencoded rr, boolean includeRight) { - NavigableMap imap = l == null - ? r == null - ? sourceMap - : sourceMap.headMap(r, includeRight) - : r == null - ? sourceMap.tailMap(l, includeLeft) - : sourceMap.subMap(l, includeLeft, r, includeRight); - return imap; + // Our slice has somewhat different semantics: + // - prefixes are not supported, i.e. a range like (a, aaa) cannot be used + // - inclusivity extends to the branches of each bound + Preencoded l = !includeLeft ? nudge(ll) : ll; + Preencoded r = includeRight ? nudge(rr) : rr; + + return l == null + ? r == null + ? sourceMap + : sourceMap.headMap(r, false) + : r == null + ? sourceMap.tailMap(l, true) + : sourceMap.subMap(l, true, r, false); } public void checkEqualRange(NavigableMap content1, @@ -282,9 +310,12 @@ public void checkEqualRange(NavigableMap content1, Preencoded r, boolean includeRight) { + if (l != null && r != null && (isTruePrefix(l, r) || isTruePrefix(r, l))) + return; // prefixes not supported in key bounds + System.out.println(String.format("Intersection with %s%s:%s%s", includeLeft ? "[" : "(", asString(l), asString(r), includeRight ? "]" : ")")); SortedMap imap = boundedMap(content1, l, includeLeft, r, includeRight); - Trie intersection = t1.subtrie(toBound(l, !includeLeft), toBound(r, includeRight)); + Trie intersection = t1.slice(l, includeLeft, r, includeRight); try { assertSameContent(intersection, imap); @@ -301,10 +332,10 @@ public void checkEqualRange(NavigableMap content1, return; // Test intersecting intersection. - intersection = t1.subtrie(toBound(l, !includeLeft), null).subtrie(null, toBound(r, includeRight)); + intersection = t1.slice(l, includeLeft, null, false).slice(null, false, r, includeRight); assertSameContent(intersection, imap); - intersection = t1.subtrie(null, toBound(r, includeRight)).subtrie(toBound(l, !includeLeft), null); + intersection = t1.slice(null, false, r, includeRight).slice(l, includeLeft, null, false); assertSameContent(intersection, imap); } @@ -436,124 +467,130 @@ void assertTrieEquals(List expected, Trie trie) } @Test - public void testSimpleIntersection() + public void testSimpleIntersectionII() { Trie trie = singleLevelIntTrie(10); assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(of(3), of(7)); - assertTrieEquals(asList(-1, 3, 4, 5, 6, 7), intersection); + Trie intersection = trie.slice(of(3), true, of(7), true); + assertTrieEquals(asList(3, 4, 5, 6, 7), intersection); } @Test - public void testSimpleLeftIntersection() + public void testSimpleIntersectionEI() { Trie trie = singleLevelIntTrie(10); assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(of(3), null); - assertTrieEquals(asList(-1, 3, 4, 5, 6, 7, 8, 9), intersection); + Trie intersection = trie.slice(of(3), false, of(7), true); + assertTrieEquals(asList(4, 5, 6, 7), intersection); } @Test - public void testSimpleRightIntersection() + public void testSimpleIntersectionIE() { Trie trie = singleLevelIntTrie(10); assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(null, of(7)); - assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7), intersection); + Trie intersection = trie.slice(of(3), true, of(7), false); + assertTrieEquals(asList(3, 4, 5, 6), intersection); } @Test - public void testSimpleNoIntersection() + public void testSimpleIntersectionEE() { Trie trie = singleLevelIntTrie(10); assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(null, null); - assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); + Trie intersection = trie.slice(of(3), false, of(7), false); + assertTrieEquals(asList(4, 5, 6), intersection); } @Test - public void testSimpleEmptyIntersectionLeft() + public void testSimpleLeftIntersectionE() { Trie trie = singleLevelIntTrie(10); assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(ByteComparable.EMPTY, null); - assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); + Trie intersection = trie.slice(of(3), false, null, true); + assertTrieEquals(asList(4, 5, 6, 7, 8, 9), intersection); + } - intersection = trie.subtrie(ByteComparable.EMPTY, ByteComparable.EMPTY); - assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); + @Test + public void testSimpleLeftIntersectionI() + { + Trie trie = singleLevelIntTrie(10); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + Trie intersection = trie.slice(of(3), true, null, true); + assertTrieEquals(asList(3, 4, 5, 6, 7, 8, 9), intersection); } @Test - public void testSimpleEmptyIntersectionRight() + public void testSimpleRightIntersectionE() { Trie trie = singleLevelIntTrie(10); assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); - Trie intersection = trie.subtrie(null, ByteComparable.EMPTY); + Trie intersection = trie.slice(null, true, of(7), false); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6), intersection); + } + + @Test + public void testSimpleRightIntersectionI() + { + Trie trie = singleLevelIntTrie(10); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + Trie intersection = trie.slice(null, true, of(7), true); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7), intersection); + } + + @Test + public void testSimpleNoIntersection() + { + Trie trie = singleLevelIntTrie(10); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + Trie intersection = trie.slice(null, true, null, true); assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); - intersection = trie.subtrie(ByteComparable.EMPTY, ByteComparable.EMPTY); + // The two boolean flags don't have a meaning when the bound does not exist. For completeness, also test + // with them set to false. + intersection = trie.slice(null, false, null, false); assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); } @Test - public void testSubtrieOnSubtrie() + public void testSimpleEmptyIntersectionLeft() { - Trie trie = singleLevelIntTrie(15); - - // non-overlapping - Trie intersection = trie.subtrie(of(0), of(4)).subtrie(of(5), of(8)); - assertTrieEquals(asList(-1), intersection); - // touching - intersection = trie.subtrie(of(0), of(3)).subtrie(of(3), of(8)); - assertTrieEquals(asList(-1, 3), intersection); - // overlapping 1 - intersection = trie.subtrie(of(0), of(4)).subtrie(of(2), of(8)); - assertTrieEquals(asList(-1, 2, 3, 4), intersection); - // overlapping 2 - intersection = trie.subtrie(of(0), of(4)).subtrie(of(1), of(8)); - assertTrieEquals(asList(-1, 1, 2, 3, 4), intersection); - // covered - intersection = trie.subtrie(of(0), of(4)).subtrie(of(0), of(8)); - assertTrieEquals(asList(-1, 0, 1, 2, 3, 4), intersection); - // covered 2 - intersection = trie.subtrie(of(4), of(8)).subtrie(of(0), of(8)); - assertTrieEquals(asList(-1, 4, 5, 6, 7, 8), intersection); + Trie trie = singleLevelIntTrie(10); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + Trie intersection = trie.slice(ByteComparable.EMPTY, true, null, true); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); + + // Not currently supported +// intersection = trie.slice(ByteComparable.EMPTY, false, null, true); +// assertTrieEquals(asList(), intersection); } @Test - public void testIntersectedIntersection() + public void testSimpleEmptyIntersectionRight() { - Trie trie = singleLevelIntTrie(15); - - // non-overlapping - Trie intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) - .intersect(TrieSet.range(VERSION, of(5), of(8))); - assertTrieEquals(asList(-1), intersection); - // touching - intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(3))) - .intersect(TrieSet.range(VERSION, of(3), of(8))); - assertTrieEquals(asList(-1, 3), intersection); - // overlapping 1 - intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) - .intersect(TrieSet.range(VERSION, of(2), of(8))); - assertTrieEquals(asList(-1, 2, 3, 4), intersection); - // overlapping 2 - intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) - .intersect(TrieSet.range(VERSION, of(1), of(8))); - assertTrieEquals(asList(-1, 1, 2, 3, 4), intersection); - // covered - intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) - .intersect(TrieSet.range(VERSION, of(0), of(8))); - assertTrieEquals(asList(-1, 0, 1, 2, 3, 4), intersection); - // covered 2 - intersection = trie.intersect(TrieSet.range(VERSION, of(4), of(8))) - .intersect(TrieSet.range(VERSION, of(0), of(8))); - assertTrieEquals(asList(-1, 4, 5, 6, 7, 8), intersection); + Trie trie = singleLevelIntTrie(10); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + Trie intersection = trie.slice(null, true, ByteComparable.EMPTY, true); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); + + // Not currently supported +// intersection = trie.slice(null, true, ByteComparable.EMPTY, false); +// assertTrieEquals(asList(), intersection); + + intersection = trie.slice(ByteComparable.EMPTY, true, ByteComparable.EMPTY, true); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); + + // (empty, empty) is an invalid call as the "(empty" is greater than "empty)" } } diff --git a/test/unit/org/apache/cassandra/db/tries/SubtrieTest.java b/test/unit/org/apache/cassandra/db/tries/SubtrieTest.java new file mode 100644 index 000000000000..60b7246d88b1 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/tries/SubtrieTest.java @@ -0,0 +1,559 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.tries; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.NavigableMap; +import java.util.Random; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.googlecode.concurrenttrees.common.Iterables; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +import static java.util.Arrays.asList; +import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.makeInMemoryTrie; +import static org.apache.cassandra.db.tries.TrieUtil.FORWARD_COMPARATOR; +import static org.apache.cassandra.db.tries.TrieUtil.VERSION; +import static org.apache.cassandra.db.tries.TrieUtil.asString; +import static org.apache.cassandra.db.tries.TrieUtil.assertSameContent; +import static org.apache.cassandra.db.tries.TrieUtil.generateKeys; +import static org.apache.cassandra.db.tries.TrieUtil.toBound; +import static org.apache.cassandra.utils.bytecomparable.ByteComparable.Preencoded; +import static org.junit.Assert.assertEquals; + +public class SubtrieTest +{ + @BeforeClass + public static void enableVerification() + { + CassandraRelevantProperties.TRIE_DEBUG.setBoolean(true); + } + + public static final Preencoded[] BOUNDARIES = toByteComparable(new String[]{ + "test1", + "test11", + "test12", + "test13", + "test2", + "test21", + "te", + "s", + "q", + "\000", + "\377", + "\377\000", + "\000\377", + "\000\000", + "\000\000\000", + "\000\000\377", + "\377\377" + }); + public static final Preencoded[] KEYS = toByteComparable(new String[]{ + "test1", + "test2", + "test55", + "test123", + "test124", + "test12", + "test21", + "tease", + "sort", + "sorting", + "square", + "\377\000", + "\000\377", + "\000\000", + "\000\000\000", + "\000\000\377", + "\377\377" + }); + + private static final int COUNT = 15000; + Random rand = new Random(); + + @Test + public void testIntersectRangeDirect() + { + testIntersectRange(COUNT); + } + + public void testIntersectRange(int count) + { + Preencoded[] src1 = generateKeys(rand, count); + NavigableMap content1 = new TreeMap<>(FORWARD_COMPARATOR); + + InMemoryTrie trie1 = makeInMemoryTrie(src1, content1, true); + + checkEqualRange(content1, trie1, null, true, null, true); + checkEqualRange(content1, trie1, TrieUtil.generateKey(rand), true, null, true); + checkEqualRange(content1, trie1, null, true, TrieUtil.generateKey(rand), true); + for (int i = 0; i < 4; ++i) + { + Preencoded l = rand.nextBoolean() ? TrieUtil.generateKey(rand) : src1[rand.nextInt(src1.length)]; + Preencoded r = rand.nextBoolean() ? TrieUtil.generateKey(rand) : src1[rand.nextInt(src1.length)]; + int cmp = ByteComparable.compare(l, r, VERSION); + if (cmp > 0) + { + Preencoded t = l; + l = r; + r = t; // swap + } + + boolean includeLeft = (i & 1) != 0 || cmp == 0; + boolean includeRight = (i & 2) != 0 || cmp == 0; + checkEqualRange(content1, trie1, l, includeLeft, r, includeRight); + checkEqualRange(content1, trie1, null, includeLeft, r, includeRight); + checkEqualRange(content1, trie1, l, includeLeft, null, includeRight); + } + } + + private static Preencoded[] toByteComparable(String[] keys) + { + return Arrays.stream(keys) + .map(TrieUtil::comparable) + .toArray(Preencoded[]::new); + } + + @Test + public void testSingletonSubtrie() + { + Arrays.sort(BOUNDARIES, (a, b) -> ByteComparable.compare(a, b, VERSION)); + for (int li = -1; li < BOUNDARIES.length; ++li) + { + Preencoded l = li < 0 ? null : BOUNDARIES[li]; + for (int ri = Math.max(0, li); ri <= BOUNDARIES.length; ++ri) + { + Preencoded r = ri == BOUNDARIES.length ? null : BOUNDARIES[ri]; + + for (int i = li == ri ? 3 : 0; i < 4; ++i) + { + boolean includeLeft = (i & 1) != 0; + boolean includeRight = (i & 2) != 0; + + for (Preencoded key : KEYS) + { + int cmp1 = l != null ? ByteComparable.compare(key, l, VERSION) : 1; + int cmp2 = r != null ? ByteComparable.compare(r, key, VERSION) : 1; + Trie ix = Trie.singleton(key, VERSION, true).subtrie(toBound(l, !includeLeft), toBound(r, includeRight)); + boolean expected = true; + if (cmp1 < 0 || cmp1 == 0 && !includeLeft) + expected = false; + if (cmp2 < 0 || cmp2 == 0 && !includeRight) + expected = false; + boolean actual = com.google.common.collect.Iterables.getFirst(ix.values(), false); + if (expected != actual) + { + System.err.println("Intersection"); + System.err.println(ix.dump()); + Assert.fail(String.format("Failed on range %s%s,%s%s key %s expected %s got %s\n", + includeLeft ? "[" : "(", + l != null ? l.byteComparableAsString(VERSION) : null, + r != null ? r.byteComparableAsString(VERSION) : null, + includeRight ? "]" : ")", + key.byteComparableAsString(VERSION), + expected, + actual)); + } + } + } + } + } + } + + @Test + public void testMemtableSubtrie() + { + Arrays.sort(BOUNDARIES, FORWARD_COMPARATOR); + NavigableMap content1 = new TreeMap<>(FORWARD_COMPARATOR); + InMemoryTrie trie1 = makeInMemoryTrie(KEYS, content1, true); + + for (int li = -1; li < BOUNDARIES.length; ++li) + { + Preencoded l = li < 0 ? null : BOUNDARIES[li]; + for (int ri = Math.max(0, li); ri <= BOUNDARIES.length; ++ri) + { + Preencoded r = ri == BOUNDARIES.length ? null : BOUNDARIES[ri]; + for (int i = 0; i < 4; ++i) + { + boolean includeLeft = (i & 1) != 0; + boolean includeRight = (i & 2) != 0; + if ((!includeLeft || !includeRight) && li == ri) + continue; + checkEqualRange(content1, trie1, l, includeLeft, r, includeRight); + } + } + } + } + + @Test + public void testMergeSubtrie() + { + testMergeSubtrie(2); + } + + @Test + public void testCollectionMergeSubtrie3() + { + testMergeSubtrie(3); + } + + @Test + public void testCollectionMergeSubtrie5() + { + testMergeSubtrie(5); + } + + public void testMergeSubtrie(int mergeCount) + { + Arrays.sort(BOUNDARIES, FORWARD_COMPARATOR); + NavigableMap content1 = new TreeMap<>(FORWARD_COMPARATOR); + List> tries = new ArrayList<>(); + for (int i = 0; i < mergeCount; ++i) + { + tries.add(makeInMemoryTrie(Arrays.copyOfRange(KEYS, + KEYS.length * i / mergeCount, + KEYS.length * (i + 1) / mergeCount), + content1, + true)); + } + Trie trie1 = Trie.mergeDistinct(tries); + + for (int li = -1; li < BOUNDARIES.length; ++li) + { + Preencoded l = li < 0 ? null : BOUNDARIES[li]; + for (int ri = Math.max(0, li); ri <= BOUNDARIES.length; ++ri) + { + Preencoded r = ri == BOUNDARIES.length ? null : BOUNDARIES[ri]; + for (int i = 0; i < 4; ++i) + { + boolean includeLeft = (i & 1) != 0; + boolean includeRight = (i & 2) != 0; + if ((!includeLeft || !includeRight) && li == ri) + continue; + checkEqualRange(content1, trie1, l, includeLeft, r, includeRight); + } + } + } + } + + + static NavigableMap boundedMap(NavigableMap sourceMap, Preencoded l, boolean includeLeft, Preencoded r, boolean includeRight) + { + NavigableMap imap = l == null + ? r == null + ? sourceMap + : sourceMap.headMap(r, includeRight) + : r == null + ? sourceMap.tailMap(l, includeLeft) + : sourceMap.subMap(l, includeLeft, r, includeRight); + return imap; + } + + public void checkEqualRange(NavigableMap content1, + Trie t1, + Preencoded l, + boolean includeLeft, + Preencoded r, + boolean includeRight) + { + System.out.println(String.format("Intersection with %s%s:%s%s", includeLeft ? "[" : "(", asString(l), asString(r), includeRight ? "]" : ")")); + SortedMap imap = boundedMap(content1, l, includeLeft, r, includeRight); + Trie intersection = t1.subtrie(toBound(l, !includeLeft), toBound(r, includeRight)); + try + { + assertSameContent(intersection, imap); + } + catch (AssertionError e) + { + System.out.println("\n" + t1.dump(ByteBufferUtil::bytesToHex)); + + System.out.println("\n" + intersection.dump(ByteBufferUtil::bytesToHex)); + throw e; + } + + if (l == null || r == null) + return; + + // Test intersecting intersection. + intersection = t1.subtrie(toBound(l, !includeLeft), null).subtrie(null, toBound(r, includeRight)); + assertSameContent(intersection, imap); + + intersection = t1.subtrie(null, toBound(r, includeRight)).subtrie(toBound(l, !includeLeft), null); + assertSameContent(intersection, imap); + } + + /** + * Extract the values of the provide trie into a list. + */ + private static List toList(Trie trie, Direction direction) + { + return Iterables.toList(trie.values(direction)); + } + + /** + * Creates a simple trie with a root having the provided number of childs, where each child is a leaf whose content + * is simply the value of the transition leading to it. + * + * In other words, {@code singleLevelIntTrie(4)} creates the following trie: + * Root + * t= 0 1 2 3 + * | | | | + * 0 1 2 3 + */ + private static Trie singleLevelIntTrie(int childs) + { + return new Trie() + { + @Override + public Cursor makeCursor(Direction direction) + { + return new SingleLevelCursor(direction); + } + + class SingleLevelCursor implements Cursor + { + final Direction direction; + int current; + + SingleLevelCursor(Direction direction) + { + this.direction = direction; + current = direction.select(-1, childs); + } + + @Override + public int advance() + { + current += direction.increase; + return depth(); + } + + @Override + public int skipTo(int depth, int transition) + { + if (depth > 1) + return advance(); + if (depth < 1) + transition = direction.select(childs, -1); + + if (direction.isForward()) + current = Math.max(0, transition); + else + current = Math.min(childs - 1, transition); + + return depth(); + } + + @Override + public int depth() + { + if (current == direction.select(-1, childs)) + return 0; + if (direction.inLoop(current, 0, childs - 1)) + return 1; + return -1; + } + + @Override + public int incomingTransition() + { + return current >= childs ? -1 : current; + } + + @Override + public Integer content() + { + return current == direction.select(-1, childs) ? -1 : current; + } + + @Override + public Direction direction() + { + return direction; + } + + @Override + public ByteComparable.Version byteComparableVersion() + { + return VERSION; + } + + @Override + public Cursor tailCursor(Direction d) + { + throw new UnsupportedOperationException("tailTrie on test cursor"); + } + } + }; + } + + /** Creates a single byte {@link ByteComparable} with the provide value */ + private static ByteComparable of(int value) + { + assert value >= 0 && value <= Byte.MAX_VALUE; + return ByteComparable.preencoded(VERSION, new byte[]{ (byte)value }); + } + + List maybeReversed(Direction direction, List list) + { + if (direction.isForward()) + return list; + List reversed = new ArrayList<>(list); + reversed.sort((x, y) -> x == -1 ? -1 : y == -1 ? 1 : Integer.compare(y, x)); + return reversed; + } + + void assertTrieEquals(List expected, Trie trie) + { + assertEquals(expected, toList(trie, Direction.FORWARD)); + assertEquals(maybeReversed(Direction.REVERSE, expected), toList(trie, Direction.REVERSE)); + } + + @Test + public void testSimpleIntersection() + { + Trie trie = singleLevelIntTrie(10); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + Trie intersection = trie.subtrie(of(3), of(7)); + assertTrieEquals(asList(-1, 3, 4, 5, 6, 7), intersection); + } + + @Test + public void testSimpleLeftIntersection() + { + Trie trie = singleLevelIntTrie(10); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + Trie intersection = trie.subtrie(of(3), null); + assertTrieEquals(asList(-1, 3, 4, 5, 6, 7, 8, 9), intersection); + } + + @Test + public void testSimpleRightIntersection() + { + Trie trie = singleLevelIntTrie(10); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + Trie intersection = trie.subtrie(null, of(7)); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7), intersection); + } + + @Test + public void testSimpleNoIntersection() + { + Trie trie = singleLevelIntTrie(10); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + Trie intersection = trie.subtrie(null, null); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); + } + + @Test + public void testSimpleEmptyIntersectionLeft() + { + Trie trie = singleLevelIntTrie(10); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + Trie intersection = trie.subtrie(ByteComparable.EMPTY, null); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); + + intersection = trie.subtrie(ByteComparable.EMPTY, ByteComparable.EMPTY); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); + } + + @Test + public void testSimpleEmptyIntersectionRight() + { + Trie trie = singleLevelIntTrie(10); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), trie); + + Trie intersection = trie.subtrie(null, ByteComparable.EMPTY); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); + + intersection = trie.subtrie(ByteComparable.EMPTY, ByteComparable.EMPTY); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9), intersection); + } + + @Test + public void testSubtrieOnSubtrie() + { + Trie trie = singleLevelIntTrie(15); + + // non-overlapping + Trie intersection = trie.subtrie(of(0), of(4)).subtrie(of(5), of(8)); + assertTrieEquals(asList(-1), intersection); + // touching + intersection = trie.subtrie(of(0), of(3)).subtrie(of(3), of(8)); + assertTrieEquals(asList(-1, 3), intersection); + // overlapping 1 + intersection = trie.subtrie(of(0), of(4)).subtrie(of(2), of(8)); + assertTrieEquals(asList(-1, 2, 3, 4), intersection); + // overlapping 2 + intersection = trie.subtrie(of(0), of(4)).subtrie(of(1), of(8)); + assertTrieEquals(asList(-1, 1, 2, 3, 4), intersection); + // covered + intersection = trie.subtrie(of(0), of(4)).subtrie(of(0), of(8)); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4), intersection); + // covered 2 + intersection = trie.subtrie(of(4), of(8)).subtrie(of(0), of(8)); + assertTrieEquals(asList(-1, 4, 5, 6, 7, 8), intersection); + } + + @Test + public void testIntersectedIntersection() + { + Trie trie = singleLevelIntTrie(15); + + // non-overlapping + Trie intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) + .intersect(TrieSet.range(VERSION, of(5), of(8))); + assertTrieEquals(asList(-1), intersection); + // touching + intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(3))) + .intersect(TrieSet.range(VERSION, of(3), of(8))); + assertTrieEquals(asList(-1, 3), intersection); + // overlapping 1 + intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) + .intersect(TrieSet.range(VERSION, of(2), of(8))); + assertTrieEquals(asList(-1, 2, 3, 4), intersection); + // overlapping 2 + intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) + .intersect(TrieSet.range(VERSION, of(1), of(8))); + assertTrieEquals(asList(-1, 1, 2, 3, 4), intersection); + // covered + intersection = trie.intersect(TrieSet.range(VERSION, of(0), of(4))) + .intersect(TrieSet.range(VERSION, of(0), of(8))); + assertTrieEquals(asList(-1, 0, 1, 2, 3, 4), intersection); + // covered 2 + intersection = trie.intersect(TrieSet.range(VERSION, of(4), of(8))) + .intersect(TrieSet.range(VERSION, of(0), of(8))); + assertTrieEquals(asList(-1, 4, 5, 6, 7, 8), intersection); + } +} From b0519d66816d201b62a037c2290decb6294a7fb5 Mon Sep 17 00:00:00 2001 From: blambov Date: Fri, 26 Sep 2025 16:11:46 +0300 Subject: [PATCH 20/22] Switch row deletions to point tombstones In the initial implementation row deletions were mapped to range tombstones, which works but isn't compatible with the multitude of tests, which require deletions to be returned in the form they were made. This commit changes the representation of deleted rows to use point tombstones. In addition to making the tests pass, this improves the memory usage of memtables with row deletions. Although they only add complexity at this stage, point tombstones (expanded to apply to the covered branch) will be needed in the next stage of development. --- .../RecombiningUnfilteredRowIterator.java | 250 ------------------ .../db/partitions/TrieBackedPartition.java | 85 ++++-- .../db/partitions/TriePartitionUpdate.java | 7 +- .../db/partitions/TriePartitionUpdater.java | 14 +- .../db/rows/TrieTombstoneMarker.java | 2 + .../db/rows/TrieTombstoneMarkerImpl.java | 157 +++++++++++ .../apache/cassandra/db/tries/RangeState.java | 6 + .../cassandra/db/tries/TrieSetCursor.java | 2 +- .../apache/cassandra/db/RowUpdateBuilder.java | 11 + .../PartitionImplementationTest.java | 48 +++- .../service/reads/DataResolverTest.java | 3 +- 11 files changed, 307 insertions(+), 278 deletions(-) delete mode 100644 src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java diff --git a/src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java deleted file mode 100644 index 6eb973c4ab41..000000000000 --- a/src/java/org/apache/cassandra/db/partitions/RecombiningUnfilteredRowIterator.java +++ /dev/null @@ -1,250 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.db.partitions; - -import org.apache.cassandra.db.Clustering; -import org.apache.cassandra.db.ClusteringPrefix; -import org.apache.cassandra.db.DeletionTime; -import org.apache.cassandra.db.rows.BTreeRow; -import org.apache.cassandra.db.rows.RangeTombstoneBoundaryMarker; -import org.apache.cassandra.db.rows.RangeTombstoneMarker; -import org.apache.cassandra.db.rows.Row; -import org.apache.cassandra.db.rows.Unfiltered; -import org.apache.cassandra.db.rows.UnfilteredRowIterator; -import org.apache.cassandra.db.rows.WrappingUnfilteredRowIterator; - -/// An [UnfilteredRowIterator] that recombines sequences of range tombstones for the same key into deleted rows. -/// -/// The objective of this class is to reverse the transformation made by [TriePartitionUpdate] that implements row -/// deletions as pairs of tombstones around the row (which are then placed in the deletion branch of the trie). -/// This transformation is valid, but a lot of tests rely on row deletions being represented by empty deleted rows. -/// For the time being we thus do the reverse transformation on conversion from trie to unfiltered iterator. -class RecombiningUnfilteredRowIterator extends WrappingUnfilteredRowIterator -{ - Unfiltered bufferedOne; - Unfiltered bufferedTwo; - Unfiltered next; - boolean nextPrepared; - - protected RecombiningUnfilteredRowIterator(UnfilteredRowIterator wrapped) - { - super(wrapped); - bufferedOne = null; - nextPrepared = false; - } - - @Override - public boolean stopIssuingTombstones() - { - if (!wrapped.stopIssuingTombstones()) - return false; - - if (nextPrepared && next == null) - return true; // this is already exhausted - - // clear all buffered state that includes tombstones - if (bufferedTwo != null && bufferedTwo.isRangeTombstoneMarker()) - bufferedTwo = null; - if (bufferedOne != null && bufferedOne.isRangeTombstoneMarker()) - bufferedOne = bufferedTwo; // can be null - if (nextPrepared && next.isRangeTombstoneMarker()) - nextPrepared = false; - return true; - } - - @Override - public boolean hasNext() - { - return computeNext() != null; - } - - @Override - public Unfiltered next() - { - Unfiltered item = computeNext(); - nextPrepared = false; - return item; - } - - private Unfiltered computeNext() - { - if (nextPrepared) - return next; - - // If we have two buffered entries, report the first one directly (there's no need to process it as we already - // know it is a row) and shift the second one to the first position. - if (bufferedTwo != null) - { - Unfiltered unfiltered2 = bufferedTwo; - bufferedTwo = null; - return setNextAndBufferedAndReturn(bufferedOne, unfiltered2); - } - - // If we have a buffered entry, use it for the following processing, otherwise get one from the source. - Unfiltered unfiltered1; - if (bufferedOne != null) - { - unfiltered1 = bufferedOne; - bufferedOne = null; - } - else - { - if (!wrapped.hasNext()) - return setNextAndReturn(null); - - unfiltered1 = wrapped.next(); - } - - // The pattern we are looking for is - // open_incusive(clustering, del) + row(clustering) + close_inclusive(clustering, del) - // where the row is optional - - if (unfiltered1.isRow()) - return setNextAndReturn(unfiltered1); - - RangeTombstoneMarker marker1 = (RangeTombstoneMarker) unfiltered1; - boolean reversed = isReverseOrder(); - int clusteringSize = metadata().comparator.size(); - // The first marker must be open, inclusive, and a fully specified clustering. - if (!marker1.isOpen(reversed) - || !marker1.openIsInclusive(reversed) - || marker1.clustering().size() != clusteringSize - || (clusteringSize > 0 && marker1.clustering().get(clusteringSize - 1) == null)) - return setNextAndReturn(marker1); - - if (!wrapped.hasNext()) - return setNextAndReturn(marker1); - - Unfiltered unfiltered2 = wrapped.next(); - final DeletionTime deletionTime = marker1.openDeletionTime(reversed); - if (unfiltered2.isRangeTombstoneMarker()) - { - RangeTombstoneMarker marker2 = (RangeTombstoneMarker) unfiltered2; - assert marker2.isClose(reversed); - assert marker2.closeDeletionTime(reversed).equals(deletionTime); - if (!marker2.closeIsInclusive(reversed) || !clusteringPositionsEqual(marker1, marker2)) - return setNextAndBufferedAndReturn(marker1, marker2); - - // The recombination applies. We have to transform the open side of marker1 and the close side - // of marker2 into an empty row with deletion time. - return processOtherSidesAndReturn(BTreeRow.emptyDeletedRow(clusteringPositionOf(marker1), Row.Deletion.regular(deletionTime)), - reversed, marker1, marker2, deletionTime); - } - - BTreeRow row2 = (BTreeRow) unfiltered2; - - if (!clusteringPositionsEqual(marker1, row2)) - return setNextAndBufferedAndReturn(marker1, row2); - - if (!wrapped.hasNext()) - return setNextAndBufferedAndReturn(marker1, row2); - - Unfiltered unfiltered3 = wrapped.next(); - if (unfiltered3.isRow()) - return setNextAndBufferedAndReturn(marker1, row2, unfiltered3); - - RangeTombstoneMarker marker3 = (RangeTombstoneMarker) unfiltered3; - assert marker3.isClose(reversed); - assert marker3.closeDeletionTime(reversed).equals(deletionTime); - if (!marker3.closeIsInclusive(reversed) || !clusteringPositionsEqual(marker1, marker3)) - return setNextAndBufferedAndReturn(marker1, row2, marker3); - - // The recombination applies. We have to transform the open side of marker1 and the close side - // of marker3 into a deletion time for row2. - return processOtherSidesAndReturn(BTreeRow.create(row2.clustering(), row2.primaryKeyLivenessInfo(), Row.Deletion.regular(deletionTime), row2.getBTree()), - reversed, marker1, marker3, deletionTime); - } - - private Unfiltered processOtherSidesAndReturn(Row row, - boolean reversed, - RangeTombstoneMarker markerLeft, - RangeTombstoneMarker markerRight, - DeletionTime deletionTime) - { - // Check if any of the markers is a boundary, and if so, report the other side. - if (!markerLeft.isClose(reversed)) - { - if (!markerRight.isOpen(reversed)) - return setNextAndReturn(row); - - return setNextAndBufferedAndReturn(row, - ((RangeTombstoneBoundaryMarker) markerRight).createCorrespondingOpenMarker(reversed)); - } - - if (!markerRight.isOpen(reversed)) - return setNextAndBufferedAndReturn(((RangeTombstoneBoundaryMarker) markerLeft).createCorrespondingCloseMarker(reversed), - row); - - // We have surviving markers on both sides. - final DeletionTime closeDeletionTime = markerLeft.closeDeletionTime(reversed); - if (markerRight.openDeletionTime(reversed).equals(closeDeletionTime) && !closeDeletionTime.supersedes(deletionTime)) - { - // The row interrupts a covering deletion, we can still drop both markers and report a deleted row. - return setNextAndReturn(row); - } - - return setNextAndBufferedAndReturn(((RangeTombstoneBoundaryMarker) markerLeft).createCorrespondingCloseMarker(reversed), - row, - ((RangeTombstoneBoundaryMarker) markerRight).createCorrespondingOpenMarker(reversed)); - } - - private Unfiltered setNextAndReturn(Unfiltered next) - { - this.next = next; - this.nextPrepared = true; - return next; - } - - private Unfiltered setNextAndBufferedAndReturn(Unfiltered next, Unfiltered bufferedOne) - { - this.bufferedOne = bufferedOne; - return setNextAndReturn(next); - } - - private Unfiltered setNextAndBufferedAndReturn(Unfiltered next, Row bufferedOne, Unfiltered bufferedTwo) - { - this.bufferedTwo = bufferedTwo; - return setNextAndBufferedAndReturn(next, bufferedOne); - } - - static boolean clusteringPositionsEqual(Unfiltered l, Unfiltered r) - { - return clusteringPositionsEqual(l.clustering(), r.clustering()); - } - - static boolean clusteringPositionsEqual(ClusteringPrefix cl, ClusteringPrefix cr) - { - if (cl.size() != cr.size()) - return false; - for (int i = cl.size() - 1; i >= 0; --i) - if (cl.accessor().compare(cl.get(i), cr.get(i), cr.accessor()) != 0) - return false; - return true; - } - - static Clustering clusteringPositionOf(Unfiltered unfiltered) - { - return clusteringPositionOf(unfiltered.clustering()); - } - - static Clustering clusteringPositionOf(ClusteringPrefix prefix) - { - return prefix.accessor().factory().clustering(prefix.getRawValues()); - } -} diff --git a/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java index e174bb0049e7..269c57c92b48 100644 --- a/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java +++ b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java @@ -269,21 +269,35 @@ public static TrieBackedPartition create(DecoratedKey partitionKey, : new WithEnsureOnHeap(partitionKey, columnMetadata, encodingStats, rowCountIncludingStatic, tombstoneCount, trie, metadata, ensureOnHeap); } - class RowIterator extends TrieEntriesIterator + class RowIterator extends TrieEntriesIterator.WithNullFiltering { public RowIterator(DeletionAwareTrie trie, Direction direction) { - super(trie.contentOnlyTrie(), direction, RowData.class::isInstance); + super(trie.mergedTrie(TrieBackedPartition::combineDataAndDeletion), direction); } @Override protected Row mapContent(Object content, byte[] bytes, int byteLength) { - var rd = (RowData) content; - return toRow(rd, - metadata.comparator.clusteringFromByteComparable( - ByteBufferAccessor.instance, - ByteComparable.preencoded(BYTE_COMPARABLE_VERSION, bytes, 0, byteLength))); + if (content instanceof RowData) + return toRow((RowData) content, + getClustering(bytes, byteLength)); + if (content instanceof Row) + { + BTreeRow row = (BTreeRow) content; + return BTreeRow.create(getClustering(bytes, byteLength), + row.primaryKeyLivenessInfo(), + row.deletion(), + row.getBTree(), + row.getMinLocalDeletionTime()); + } + + TrieTombstoneMarker marker = (TrieTombstoneMarker) content; + if (marker.hasPointData()) + return BTreeRow.emptyDeletedRow(getClustering(bytes, byteLength), + Row.Deletion.regular(marker.deletionTime())); + else + return null; } } @@ -318,16 +332,17 @@ protected static void putInTrie(ClusteringComparator comparator, InMemoryDeletio Clustering clustering = row.clustering(); DeletionTime deletionTime = row.deletion().time(); + ByteComparable comparableClustering = comparator.asByteComparable(clustering); if (!deletionTime.isLive()) { putDeletionInTrie(trie, - comparator.asByteComparable(clustering.asStartBound()), - comparator.asByteComparable(clustering.asEndBound()), + comparableClustering, + comparableClustering, deletionTime); } if (!row.isEmptyAfterDeletion()) { - trie.apply(DeletionAwareTrie.singleton(comparator.asByteComparable(clustering), + trie.apply(DeletionAwareTrie.singleton(comparableClustering, BYTE_COMPARABLE_VERSION, rowToData(row)), noConflictInData(), @@ -499,6 +514,27 @@ public UnfilteredRowIterator unfilteredIterator() return unfilteredIterator(ColumnFilter.selection(columns()), Slices.ALL, false); } + public static Object combineDataAndDeletion(Object data, TrieTombstoneMarker deletion) + { + if (data == null || data instanceof PartitionMarker) + return deletion; + + if (deletion == null || !deletion.hasPointData()) + return data; + + // This is a row combined with a point deletion. + RowData rowData = (RowData) data; + return rowData.toRow(Clustering.EMPTY, deletion.deletionTime()); + } + + private Clustering getClustering(byte[] bytes, int byteLength) + { + return metadata.comparator.clusteringFromByteComparable(ByteBufferAccessor.instance, + ByteComparable.preencoded(BYTE_COMPARABLE_VERSION, + bytes, 0, byteLength), + BYTE_COMPARABLE_VERSION); + } + /// Implementation of [UnfilteredRowIterator] for this partition. /// /// Currently, this implementation is pretty involved because it has to revert the transformations done to row and @@ -522,7 +558,8 @@ protected UnfilteredIterator(ColumnFilter selection, DeletionAwareTrie trie, boolean reversed, DeletionTime partitionLevelDeletion) { - super(trie.mergedTrieSwitchable((x, y) -> x instanceof RowData ? x : y), Direction.fromBoolean(reversed)); + super(trie.mergedTrieSwitchable(TrieBackedPartition::combineDataAndDeletion), + Direction.fromBoolean(reversed)); this.trie = trie; this.selection = selection; this.reversed = reversed; @@ -536,11 +573,23 @@ protected Unfiltered mapContent(Object content, byte[] bytes, int byteLength) { if (content instanceof RowData) return toRow((RowData) content, - metadata.comparator.clusteringFromByteComparable(ByteBufferAccessor.instance, - ByteComparable.preencoded(BYTE_COMPARABLE_VERSION, - bytes, 0, byteLength), - BYTE_COMPARABLE_VERSION)) // deletion is given as range tombstone - .filter(selection, metadata()); + getClustering(bytes, byteLength)) // deletion is given as range tombstone + .filter(selection, metadata()); + if (content instanceof Row) + { + BTreeRow row = (BTreeRow) content; + return BTreeRow.create(getClustering(bytes, byteLength), + row.primaryKeyLivenessInfo(), + row.deletion(), + row.getBTree(), + row.getMinLocalDeletionTime()) + .filter(selection, metadata()); + } + + TrieTombstoneMarker marker = (TrieTombstoneMarker) content; + if (marker.hasPointData()) + return BTreeRow.emptyDeletedRow(getClustering(bytes, byteLength), + Row.Deletion.regular(marker.deletionTime())); else return ((TrieTombstoneMarker) content).toRangeTombstoneMarker( ByteComparable.preencoded(BYTE_COMPARABLE_VERSION, bytes, 0, byteLength), @@ -576,7 +625,7 @@ public boolean isReverseOrder() @Override public RegularAndStaticColumns columns() { - return columns; + return selection.fetchedColumns(); } @Override @@ -615,7 +664,7 @@ public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, ByteComp return UnfilteredRowIterators.noRowsIterator(metadata, partitionKey, staticRow(), partitionLevelDeletion(), reversed); DeletionAwareTrie slicedTrie = trie.intersect(TrieSet.ranges(BYTE_COMPARABLE_VERSION, bounds)); - return new RecombiningUnfilteredRowIterator(new UnfilteredIterator(selection, slicedTrie, reversed)); + return new UnfilteredIterator(selection, slicedTrie, reversed); } public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, Slices slices, boolean reversed) diff --git a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdate.java b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdate.java index 58bb7f22ea0c..6cdf5060538c 100644 --- a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdate.java +++ b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdate.java @@ -440,15 +440,16 @@ public void add(Row row) Clustering clustering = row.clustering(); DeletionTime deletionTime = row.deletion().time(); + ByteComparable comparableClustering = metadata.comparator.asByteComparable(clustering); if (!deletionTime.isLive()) { - putDeletionInTrie(metadata.comparator.asByteComparable(clustering.asStartBound()), - metadata.comparator.asByteComparable(clustering.asEndBound()), + putDeletionInTrie(comparableClustering, + comparableClustering, deletionTime); } if (!row.isEmptyAfterDeletion()) { - trie.apply(DeletionAwareTrie.singleton(metadata.comparator.asByteComparable(clustering), + trie.apply(DeletionAwareTrie.singleton(comparableClustering, BYTE_COMPARABLE_VERSION, row), this::merge, diff --git a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdater.java b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdater.java index 8d2dff0b2b96..264b7fb6397a 100644 --- a/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdater.java +++ b/src/java/org/apache/cassandra/db/partitions/TriePartitionUpdater.java @@ -85,7 +85,19 @@ public TrieTombstoneMarker mergeMarkers(@Nullable TrieTombstoneMarker existing, { if (indexer != UpdateTransaction.NO_OP) { - if (update.isBoundary()) + if (update.hasPointData()) + { + Clustering clustering = metadata.comparator.clusteringFromByteComparable( + ByteArrayAccessor.instance, + ByteComparable.preencoded(TrieBackedPartition.BYTE_COMPARABLE_VERSION, + keyState.getBytes())); + if (existing != null) + indexer.onUpdated(BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(existing.deletionTime())), + BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(update.deletionTime()))); + else + indexer.onInserted(BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(update.deletionTime()))); + } + else if (update.isBoundary()) { if (rangeTombstoneOpenPosition != null) { diff --git a/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarker.java b/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarker.java index a07a3ab1cd92..544ef48aa1b8 100644 --- a/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarker.java +++ b/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarker.java @@ -54,6 +54,8 @@ RangeTombstoneMarker toRangeTombstoneMarker(ByteComparable clusteringPrefixAsByt /// equal) which is not stored or reported. TrieTombstoneMarker mergeWith(TrieTombstoneMarker existing); + boolean hasPointData(); + static TrieTombstoneMarker covering(DeletionTime deletionTime) { return TrieTombstoneMarkerImpl.covering(deletionTime); diff --git a/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarkerImpl.java b/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarkerImpl.java index 3dacd6786231..ef435dbb1b41 100644 --- a/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarkerImpl.java +++ b/src/java/org/apache/cassandra/db/rows/TrieTombstoneMarkerImpl.java @@ -100,11 +100,19 @@ public Covering rightDeletion() return this; } + @Override + public boolean hasPointData() + { + return false; + } + @Override public TrieTombstoneMarker mergeWith(TrieTombstoneMarker other) { if (other instanceof Boundary) return other.mergeWith(this); + if (other instanceof Point) + return other.mergeWith(this); return combine(this, (Covering) other); } @@ -139,6 +147,12 @@ public TrieTombstoneMarker asBoundary(Direction direction) return direction.isForward() ? new Boundary(null, this) : new Boundary(this, null); } + @Override + public TrieTombstoneMarker asPoint() + { + return new Point(this, null); + } + @Override public DeletionTime deletionTime() { @@ -185,6 +199,12 @@ public DeletionTime deletionTime() : leftDeletion; } + @Override + public boolean hasPointData() + { + return false; + } + @Override public RangeTombstoneMarker toRangeTombstoneMarker(ByteComparable clusteringPrefixAsByteComparable, ByteComparable.Version byteComparableVersion, @@ -221,6 +241,7 @@ public TrieTombstoneMarker mergeWith(TrieTombstoneMarker existing) if (existing == null) return this; + assert !existing.hasPointData() : "Boundary cannot be merged with point deletion"; TrieTombstoneMarkerImpl other = (TrieTombstoneMarkerImpl) existing; Covering otherLeft = other.leftDeletion(); Covering newLeft = combine(leftDeletion, otherLeft); @@ -296,4 +317,140 @@ public long unsharedHeapSize() return UNSHARED_HEAP_SIZE; } } + + static class Point implements TrieTombstoneMarkerImpl + { + // Every point deletion introduces a new deletion time. If it interrupts an existing deletion, it will reuse + // the Covering object provided by its end bounds. Thus, the unshared size is this object + the size of + // one Covering. + static final long UNSHARED_HEAP_SIZE = ObjectSizes.measure(new Point(new Covering(0, 0), + null)) + + Covering.HEAP_SIZE; + + final @Nullable Covering coveringDeletion; + final Covering pointDeletion; + + public Point(Covering pointDeletion, @Nullable Covering coveringDeletion) + { + assert pointDeletion != null; + this.coveringDeletion = coveringDeletion; + this.pointDeletion = pointDeletion; + } + + @Override + public Covering leftDeletion() + { + return coveringDeletion; + } + + @Override + public Covering rightDeletion() + { + return coveringDeletion; + } + + @Override + public DeletionTime deletionTime() + { + return pointDeletion; + } + + @Override + public RangeTombstoneMarker toRangeTombstoneMarker(ByteComparable clusteringPrefixAsByteComparable, + ByteComparable.Version byteComparableVersion, + ClusteringComparator comparator, + DeletionTime deletionToOmit) + { + return null; + } + + @Override + public TrieTombstoneMarker mergeWith(TrieTombstoneMarker existing) + { + if (existing == null) + return this; + + if (existing instanceof Covering) + { + Covering existingCovering = (Covering) existing; + if (!pointDeletion.supersedes(existingCovering)) + { + if (coveringDeletion == null || !coveringDeletion.supersedes(existingCovering)) + return null; + else + return coveringDeletion; + } + + Covering newCovering = combine(coveringDeletion, existingCovering); + if (newCovering == coveringDeletion) + return this; + else + return new Point(pointDeletion, newCovering); + } + else if (existing instanceof Point) + { + Point existingPoint = (Point) existing; + Covering newCovering = combine(coveringDeletion, existingPoint.coveringDeletion); + Covering newPoint = combine(pointDeletion, existingPoint.pointDeletion); + if (newCovering == coveringDeletion && newPoint == pointDeletion) + return this; + if (newCovering == existingPoint.coveringDeletion && newPoint == existingPoint.pointDeletion) + return existingPoint; + + return new Point(newPoint, newCovering); + } + else + throw new AssertionError("Boundaries cannot be positioned on row clusterings."); + } + + @Override + public boolean hasPointData() + { + return true; + } + + @Override + public TrieTombstoneMarker withUpdatedTimestamp(long l) + { + if (coveringDeletion != null) + return new Covering(l, coveringDeletion.localDeletionTime()); // subsumed by range deletion + return new Point(new Covering(l, pointDeletion.localDeletionTime()), null); + } + + @Override + public boolean isBoundary() + { + return true; + } + + @Override + public TrieTombstoneMarker precedingState(Direction direction) + { + return coveringDeletion; + } + + @Override + public TrieTombstoneMarker restrict(boolean applicableBefore, boolean applicableAfter) + { + throw new AssertionError("Cannot have a row clustering as slice bound."); + } + + @Override + public TrieTombstoneMarker asBoundary(Direction direction) + { + throw new AssertionError("Cannot have a row clustering as slice bound."); + } + + @Override + public String toString() + { + return pointDeletion + (coveringDeletion != null ? "(under " + (coveringDeletion != null ? coveringDeletion : "LIVE") + ")" : ""); + } + + @Override + public long unsharedHeapSize() + { + return UNSHARED_HEAP_SIZE; + } + } } diff --git a/src/java/org/apache/cassandra/db/tries/RangeState.java b/src/java/org/apache/cassandra/db/tries/RangeState.java index b94dc9a8eeda..65d11ea726e1 100644 --- a/src/java/org/apache/cassandra/db/tries/RangeState.java +++ b/src/java/org/apache/cassandra/db/tries/RangeState.java @@ -57,4 +57,10 @@ public interface RangeState> /// Assuming this is a covering state, promote it to a boundary active in the specified direction. S asBoundary(Direction direction); + + /// Assuming this is a covering state, convert it to a point deletion. + default S asPoint() + { + return null; + } } diff --git a/src/java/org/apache/cassandra/db/tries/TrieSetCursor.java b/src/java/org/apache/cassandra/db/tries/TrieSetCursor.java index b45c459202cf..cd5a5943ec89 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieSetCursor.java +++ b/src/java/org/apache/cassandra/db/tries/TrieSetCursor.java @@ -145,7 +145,7 @@ S applyToCoveringState(S srcState, Direction direction) switch (this) { case POINT: - return null; + return srcState.asPoint(); case COVERED: return srcState; case START: diff --git a/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java b/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java index 1e3b79fe715f..2a890fad7c94 100644 --- a/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java +++ b/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java @@ -72,6 +72,17 @@ public RowUpdateBuilder(TableMetadata metadata, int localDeletionTime, long time this.updateBuilder.nowInSec(localDeletionTime); } + public RowUpdateBuilder(TableMetadata metadata, DeletionTime partitionDeletion, int nowInSec, long timestamp, Object partitionKey) + { + this(PartitionUpdate.simpleBuilder(metadata, partitionKey)); + + this.updateBuilder.timestamp(partitionDeletion.markedForDeleteAt()); + this.updateBuilder.nowInSec(partitionDeletion.localDeletionTime()); + this.updateBuilder.delete(); + this.updateBuilder.timestamp(timestamp); + this.updateBuilder.nowInSec(nowInSec); + } + public RowUpdateBuilder timestamp(long ts) { updateBuilder.timestamp(ts); diff --git a/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java b/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java index 0858e50a40b5..dde7e90722ab 100644 --- a/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java +++ b/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java @@ -38,6 +38,7 @@ import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.Util; +import org.apache.cassandra.db.partitions.TrieBackedPartitionStage2; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.cql3.ColumnIdentifier; @@ -60,7 +61,8 @@ public class PartitionImplementationTest enum Implementation { BTREE(ImmutableBTreePartition::create), - TRIE(TrieBackedPartition::fromIterator); + TRIE(TrieBackedPartition::fromIterator), + TRIE_STAGE_2(TrieBackedPartitionStage2::fromIterator); final Function creator; @@ -454,7 +456,7 @@ private void testSlicingOfIterators(NavigableSet sortedContent, Par if (reversed) Collections.reverse(slicelist); - assertIteratorsEqual(Iterators.concat(slicelist.toArray(new Iterator[0])), slicedIter); + assertIteratorsEqual(maybeFilterInvalidCloseThenOpen(Iterators.concat(slicelist.toArray(new Iterator[0])), reversed), slicedIter); } } @@ -467,7 +469,47 @@ private Iterator invert(Iterator slice) private Iterator slice(NavigableSet sortedContent, Slices slices) { - return Iterators.concat(streamOf(slices).map(slice -> slice(sortedContent, slice)).iterator()); + Iterator result = Iterators.concat(streamOf(slices).map(slice -> slice(sortedContent, slice)).iterator()); + result = maybeFilterInvalidCloseThenOpen(result, false); + + return result; + } + + private static Iterator maybeFilterInvalidCloseThenOpen(Iterator result, boolean reversed) + { + // Older implementations concatenate the individual slices, which may create an invalid close+open sequence with the same clustering. + // Stage 3 tries fix this problem. + if (implementation != Implementation.TRIE || !result.hasNext()) + return result; + + List list = new ArrayList<>(); + Clusterable c1 = result.next(); + while (result.hasNext()) + { + Clusterable c2 = result.next(); + if (metadata.comparator.compare(c1.clustering(), c2.clustering()) == 0) + { + assertTrue(c1 instanceof RangeTombstoneBoundMarker); + assertTrue(c2 instanceof RangeTombstoneBoundMarker); + RangeTombstoneBoundMarker m1 = (RangeTombstoneBoundMarker) c1; + RangeTombstoneBoundMarker m2 = (RangeTombstoneBoundMarker) c2; + assertTrue(m1.isClose(reversed)); + assertTrue(m2.isOpen(reversed)); + if (m1.deletionTime().equals(m2.deletionTime())) + c1 = result.hasNext() ? result.next() : null; + else + c1 = RangeTombstoneBoundaryMarker.makeBoundary(reversed, m1.clustering(), m2.clustering(), m1.deletionTime(), m2.deletionTime()); + } + else + { + list.add(c1); + c1 = c2; + } + } + if (c1 != null) + list.add(c1); + result = list.iterator(); + return result; } private Iterator slice(NavigableSet sortedContent, Slice slice) diff --git a/test/unit/org/apache/cassandra/service/reads/DataResolverTest.java b/test/unit/org/apache/cassandra/service/reads/DataResolverTest.java index 38a3f8c19b8e..56addbcae1b2 100644 --- a/test/unit/org/apache/cassandra/service/reads/DataResolverTest.java +++ b/test/unit/org/apache/cassandra/service/reads/DataResolverTest.java @@ -658,10 +658,9 @@ public void testRepairRangeTombstoneWithPartitionDeletion2() // 1st "stream": a partition deletion and a range tombstone RangeTombstone rt1 = tombstone("0", true , "9", true, 11, nowInSec); - PartitionUpdate upd1 = new RowUpdateBuilder(cfm, nowInSec, 1L, dk) + PartitionUpdate upd1 = new RowUpdateBuilder(cfm, new DeletionTime(10, nowInSec), nowInSec, 1L, dk) .addRangeTombstone(rt1) .buildUpdate(); - ((MutableDeletionInfo)upd1.deletionInfo()).add(new DeletionTime(10, nowInSec)); UnfilteredPartitionIterator iter1 = iter(upd1); // 2nd "stream": a range tombstone that is covered by the other stream rt From 33b461e44b1c6b2f33ce16e967316cf87e3ca353 Mon Sep 17 00:00:00 2001 From: blambov Date: Thu, 2 Oct 2025 16:54:24 +0300 Subject: [PATCH 21/22] Generalize forEachValue/Entry --- .../org/apache/cassandra/db/tries/BaseTrie.java | 14 +++++++------- src/java/org/apache/cassandra/db/tries/Cursor.java | 2 +- .../cassandra/db/tries/DepthAdjustedCursor.java | 2 +- .../cassandra/db/tries/TrieEntriesWalker.java | 4 ++-- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/java/org/apache/cassandra/db/tries/BaseTrie.java b/src/java/org/apache/cassandra/db/tries/BaseTrie.java index 64dad35ad0c3..3cec51390168 100644 --- a/src/java/org/apache/cassandra/db/tries/BaseTrie.java +++ b/src/java/org/apache/cassandra/db/tries/BaseTrie.java @@ -78,25 +78,25 @@ default void addPathBytes(DirectBuffer buffer, int pos, int count) } /// Call the given consumer on all content values in the trie in order. - default void forEachValue(ValueConsumer consumer) + default void forEachValue(ValueConsumer consumer) { process(Direction.FORWARD, consumer); } /// Call the given consumer on all content values in the trie in order. - default void forEachValue(Direction direction, ValueConsumer consumer) + default void forEachValue(Direction direction, ValueConsumer consumer) { process(direction, consumer); } /// Call the given consumer on all (path, content) pairs with non-null content in the trie in order. - default void forEachEntry(BiConsumer consumer) + default void forEachEntry(BiConsumer consumer) { forEachEntry(Direction.FORWARD, consumer); } /// Call the given consumer on all (path, content) pairs with non-null content in the trie in order. - default void forEachEntry(Direction direction, BiConsumer consumer) + default void forEachEntry(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); cursor.process(new TrieEntriesWalker.WithConsumer<>(consumer, cursor.byteComparableVersion())); @@ -111,14 +111,14 @@ default R process(Direction direction, Cursor.Walker walker) } /// Process the trie using the given [ValueConsumer], skipping all branches below the top content-bearing node. - default void forEachValueSkippingBranches(Direction direction, ValueConsumer consumer) + default void forEachValueSkippingBranches(Direction direction, ValueConsumer consumer) { processSkippingBranches(direction, consumer); } /// Call the given consumer on all `(path, content)` pairs with non-null content in the trie in order, skipping all /// branches below the top content-bearing node. - default void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer) + default void forEachEntrySkippingBranches(Direction direction, BiConsumer consumer) { Cursor cursor = cursor(direction); cursor.processSkippingBranches(new TrieEntriesWalker.WithConsumer<>(consumer, cursor.byteComparableVersion())); @@ -127,7 +127,7 @@ default void forEachEntrySkippingBranches(Direction direction, BiConsumer R processSkippingBranches(Direction direction, Cursor.Walker walker) + default R processSkippingBranches(Direction direction, Cursor.Walker walker) { return cursor(direction).processSkippingBranches(walker); } diff --git a/src/java/org/apache/cassandra/db/tries/Cursor.java b/src/java/org/apache/cassandra/db/tries/Cursor.java index f442bbbae7b8..cbbfe7f781b7 100644 --- a/src/java/org/apache/cassandra/db/tries/Cursor.java +++ b/src/java/org/apache/cassandra/db/tries/Cursor.java @@ -289,7 +289,7 @@ default R process(Cursor.Walker walker) /// Process the trie using the given [Walker], skipping over branches where content was found. /// This method should only be called on a freshly constructed cursor. - default R processSkippingBranches(Cursor.Walker walker) + default R processSkippingBranches(Cursor.Walker walker) { assert depth() == 0 : "The provided cursor has already been advanced."; T content = content(); // handle content on the root node diff --git a/src/java/org/apache/cassandra/db/tries/DepthAdjustedCursor.java b/src/java/org/apache/cassandra/db/tries/DepthAdjustedCursor.java index 0c1803bf32b6..cf687a43c2be 100644 --- a/src/java/org/apache/cassandra/db/tries/DepthAdjustedCursor.java +++ b/src/java/org/apache/cassandra/db/tries/DepthAdjustedCursor.java @@ -121,7 +121,7 @@ public R process(Walker walker) } @Override - public R processSkippingBranches(Walker walker) + public R processSkippingBranches(Walker walker) { throw new AssertionError("Depth-adjusted cursors cannot be walked directly."); } diff --git a/src/java/org/apache/cassandra/db/tries/TrieEntriesWalker.java b/src/java/org/apache/cassandra/db/tries/TrieEntriesWalker.java index fc13751a99f8..d3b449eb932b 100644 --- a/src/java/org/apache/cassandra/db/tries/TrieEntriesWalker.java +++ b/src/java/org/apache/cassandra/db/tries/TrieEntriesWalker.java @@ -36,10 +36,10 @@ public void content(T content) /// Iterator representing the content of the trie a sequence of (path, content) pairs. static class WithConsumer extends TrieEntriesWalker { - private final BiConsumer consumer; + private final BiConsumer consumer; private final ByteComparable.Version byteComparableVersion; - public WithConsumer(BiConsumer consumer, ByteComparable.Version byteComparableVersion) + public WithConsumer(BiConsumer consumer, ByteComparable.Version byteComparableVersion) { this.consumer = consumer; this.byteComparableVersion = byteComparableVersion; From d59d5ec1fcad893e0a08126f94c279acf1d9224e Mon Sep 17 00:00:00 2001 From: blambov Date: Thu, 2 Oct 2025 16:56:16 +0300 Subject: [PATCH 22/22] Switch MemtableAverageRowSize to use trie directly and expand test --- .../org/apache/cassandra/db/IDataSize.java | 25 +++++++++++++ .../db/memtable/MemtableAverageRowSize.java | 33 +++++++++++++++++ .../cassandra/db/memtable/TrieMemtable.java | 2 +- .../db/memtable/TrieMemtableStage2.java | 2 +- .../db/partitions/TrieBackedPartition.java | 3 +- .../partitions/TrieBackedPartitionStage2.java | 3 +- .../org/apache/cassandra/db/rows/Row.java | 2 +- ...ingleRestrictionEstimatedRowCountTest.java | 35 +++++++++++++------ 8 files changed, 89 insertions(+), 16 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/IDataSize.java diff --git a/src/java/org/apache/cassandra/db/IDataSize.java b/src/java/org/apache/cassandra/db/IDataSize.java new file mode 100644 index 000000000000..8611a7b1e3e1 --- /dev/null +++ b/src/java/org/apache/cassandra/db/IDataSize.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db; + +/// Shared interface for providing data size information +public interface IDataSize +{ + int dataSize(); +} diff --git a/src/java/org/apache/cassandra/db/memtable/MemtableAverageRowSize.java b/src/java/org/apache/cassandra/db/memtable/MemtableAverageRowSize.java index 5200cd8a5e89..fe02472aacf0 100644 --- a/src/java/org/apache/cassandra/db/memtable/MemtableAverageRowSize.java +++ b/src/java/org/apache/cassandra/db/memtable/MemtableAverageRowSize.java @@ -18,11 +18,15 @@ package org.apache.cassandra.db.memtable; +import java.util.function.Consumer; + import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.IDataSize; import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.db.rows.Unfiltered; import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.tries.Trie; class MemtableAverageRowSize { @@ -31,6 +35,35 @@ class MemtableAverageRowSize public final long rowSize; public final long operations; + public MemtableAverageRowSize(Memtable memtable, Trie trie) + { + // If this is a trie-based memtable, get the row sizes from the trie elements. This achieves two things: + // - makes sure the size used is the size reflected in the memtable's dataSize + // (which e.g. excludes clustering keys) + // - avoids the conversion to Row, which has non-trivial cost + + class SizeCalculator implements Trie.ValueConsumer + { + long totalSize = 0; + long count = 0; + + @Override + public void accept(Object o) + { + if (o instanceof IDataSize) + { + totalSize += ((IDataSize) o).dataSize(); + ++count; + } + } + } + + SizeCalculator sizeCalculator = new SizeCalculator(); + trie.forEachValue(sizeCalculator); + + this.rowSize = sizeCalculator.count > 0 ? sizeCalculator.totalSize / sizeCalculator.count : 0; + this.operations = memtable.getOperations(); + } public MemtableAverageRowSize(Memtable memtable) { diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java index d5e91e66b197..acb934f0c6b8 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java @@ -310,7 +310,7 @@ public int getShardCount() public long getEstimatedAverageRowSize() { if (estimatedAverageRowSize == null || currentOperations.get() > estimatedAverageRowSize.operations * 1.5) - estimatedAverageRowSize = new MemtableAverageRowSize(this); + estimatedAverageRowSize = new MemtableAverageRowSize(this, mergedTrie.contentOnlyTrie()); return estimatedAverageRowSize.rowSize; } diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java index e81f43ee4321..ab0030f3aff8 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtableStage2.java @@ -315,7 +315,7 @@ public int getShardCount() public long getEstimatedAverageRowSize() { if (estimatedAverageRowSize == null || currentOperations.get() > estimatedAverageRowSize.operations * 1.5) - estimatedAverageRowSize = new MemtableAverageRowSize(this); + estimatedAverageRowSize = new MemtableAverageRowSize(this, mergedTrie); return estimatedAverageRowSize.rowSize; } diff --git a/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java index 269c57c92b48..f022789df31f 100644 --- a/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java +++ b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartition.java @@ -30,6 +30,7 @@ import org.apache.cassandra.db.ClusteringPrefix; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.IDataSize; import org.apache.cassandra.db.LivenessInfo; import org.apache.cassandra.db.RegularAndStaticColumns; import org.apache.cassandra.db.Slice; @@ -113,7 +114,7 @@ public static boolean isPartitionBoundary(Object content) /// The representation of a row stored at the leaf of a trie. Does not contain the row key. /// /// The method [#toRow] combines this with a clustering for the represented [Row]. - public static class RowData + public static class RowData implements IDataSize { final Object[] columnsBTree; final LivenessInfo livenessInfo; diff --git a/src/java/org/apache/cassandra/db/partitions/TrieBackedPartitionStage2.java b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartitionStage2.java index c7c2136f0be5..439d2633890e 100644 --- a/src/java/org/apache/cassandra/db/partitions/TrieBackedPartitionStage2.java +++ b/src/java/org/apache/cassandra/db/partitions/TrieBackedPartitionStage2.java @@ -31,6 +31,7 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.DeletionInfo; import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.IDataSize; import org.apache.cassandra.db.LivenessInfo; import org.apache.cassandra.db.MutableDeletionInfo; import org.apache.cassandra.db.RangeTombstone; @@ -98,7 +99,7 @@ public class TrieBackedPartitionStage2 implements Partition * * The methods toRow and copyToOnHeapRow combine this with a clustering for the represented Row. */ - public static class RowData + public static class RowData implements IDataSize { final Object[] columnsBTree; final LivenessInfo livenessInfo; diff --git a/src/java/org/apache/cassandra/db/rows/Row.java b/src/java/org/apache/cassandra/db/rows/Row.java index 4e1b6318b689..5aebc9ff9ccb 100644 --- a/src/java/org/apache/cassandra/db/rows/Row.java +++ b/src/java/org/apache/cassandra/db/rows/Row.java @@ -51,7 +51,7 @@ * it's own data. For instance, a {@code Row} cannot contains a cell that is deleted by its own * row deletion. */ -public interface Row extends Unfiltered, Iterable +public interface Row extends Unfiltered, Iterable, IDataSize { /** * The clustering values for this row. diff --git a/test/unit/org/apache/cassandra/index/sai/plan/SingleRestrictionEstimatedRowCountTest.java b/test/unit/org/apache/cassandra/index/sai/plan/SingleRestrictionEstimatedRowCountTest.java index 2aba5c000f80..3d7df712aed5 100644 --- a/test/unit/org/apache/cassandra/index/sai/plan/SingleRestrictionEstimatedRowCountTest.java +++ b/test/unit/org/apache/cassandra/index/sai/plan/SingleRestrictionEstimatedRowCountTest.java @@ -25,6 +25,8 @@ import java.util.Map; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.apache.cassandra.Util; import org.apache.cassandra.cql3.CQL3Type; @@ -43,8 +45,18 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; +@RunWith(Parameterized.class) public class SingleRestrictionEstimatedRowCountTest extends SAITester { + @Parameterized.Parameter + public String memtableClass; + + @Parameterized.Parameters(name = "{0}") + public static String[] memtableClasses() + { + return new String[]{ "TrieMemtable", "TrieMemtableStage2", "TrieMemtableStage1" }; + } + static protected Map, ColumnFamilyStore> tables = new HashMap<>(); static Version[] versions = new Version[]{ Version.DB, Version.EB }; static CQL3Type.Native[] types = new CQL3Type.Native[]{ INT, DECIMAL, VARINT }; @@ -75,12 +87,12 @@ public void testMemtablesSAI() createTables(); RowCountTest test = new RowCountTest(Operator.NEQ, 25); - test.doTest(Version.DB, INT, 97.0); - test.doTest(Version.EB, INT, 97.0); + test.doTest(Version.DB, INT, 100); + test.doTest(Version.EB, INT, 100); // Truncated numeric types planned differently - test.doTest(Version.DB, DECIMAL, 97.0); - test.doTest(Version.EB, DECIMAL, 97.0); - test.doTest(Version.EB, VARINT, 97.0); + test.doTest(Version.DB, DECIMAL, 100); + test.doTest(Version.EB, DECIMAL, 100); + test.doTest(Version.EB, VARINT, 100); test = new RowCountTest(Operator.LT, 50); test.doTest(Version.DB, INT, 48); @@ -89,10 +101,10 @@ public void testMemtablesSAI() test.doTest(Version.EB, DECIMAL, 48); test = new RowCountTest(Operator.LT, 150); - test.doTest(Version.DB, INT, 97); - test.doTest(Version.EB, INT, 97); - test.doTest(Version.DB, DECIMAL, 97); - test.doTest(Version.EB, DECIMAL, 97); + test.doTest(Version.DB, INT, 100); + test.doTest(Version.EB, INT, 100); + test.doTest(Version.DB, DECIMAL, 100); + test.doTest(Version.EB, DECIMAL, 100); test = new RowCountTest(Operator.EQ, 31); test.doTest(Version.DB, INT, 15); @@ -109,7 +121,8 @@ void createTables() SAIUtil.setCurrentVersion(version); for (CQL3Type.Native type : types) { - createTable("CREATE TABLE %s (pk text PRIMARY KEY, age " + type + ')'); + createTable("CREATE TABLE %s (pk text PRIMARY KEY, age " + type + ") " + + "WITH memtable = {'class': '" + memtableClass + "'}"); createIndex("CREATE CUSTOM INDEX ON %s(age) USING 'StorageAttachedIndex'"); tables.put(tablesEntryKey(version, type), getCurrentColumnFamilyStore()); } @@ -159,7 +172,7 @@ void doTest(Version version, CQL3Type.Native type, double expectedRows) long totalRows = controller.planFactory.tableMetrics.rows; assertEquals(0, cfs.metrics().liveSSTableCount.getValue().intValue()); - assertEquals(97, totalRows); + assertEquals(100, totalRows); Plan plan = controller.buildPlan(); assert plan instanceof Plan.RowsIteration;