You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by xe...@apache.org on 2016/01/24 04:36:09 UTC

[07/14] cassandra git commit: Integrate SASI index into Cassandra

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/trie/PatriciaTrie.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/trie/PatriciaTrie.java b/src/java/org/apache/cassandra/index/sasi/utils/trie/PatriciaTrie.java
new file mode 100644
index 0000000..3c672ec
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/trie/PatriciaTrie.java
@@ -0,0 +1,1261 @@
+/*
+ * Copyright 2005-2010 Roger Kapsi, Sam Berlin
+ *
+ *   Licensed 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.index.sasi.utils.trie;
+
+import java.io.Serializable;
+import java.util.*;
+
+/**
+ * This class is taken from https://github.com/rkapsi/patricia-trie (v0.6), and slightly modified
+ * to correspond to Cassandra code style, as the only Patricia Trie implementation,
+ * which supports pluggable key comparators (e.g. commons-collections PatriciaTrie (which is based
+ * on rkapsi/patricia-trie project) only supports String keys)
+ * but unfortunately is not deployed to the maven central as a downloadable artifact.
+ */
+
+/**
+ * <h3>PATRICIA {@link Trie}</h3>
+ *  
+ * <i>Practical Algorithm to Retrieve Information Coded in Alphanumeric</i>
+ * 
+ * <p>A PATRICIA {@link Trie} is a compressed {@link Trie}. Instead of storing 
+ * all data at the edges of the {@link Trie} (and having empty internal nodes), 
+ * PATRICIA stores data in every node. This allows for very efficient traversal, 
+ * insert, delete, predecessor, successor, prefix, range, and {@link #select(Object)} 
+ * operations. All operations are performed at worst in O(K) time, where K 
+ * is the number of bits in the largest item in the tree. In practice, 
+ * operations actually take O(A(K)) time, where A(K) is the average number of 
+ * bits of all items in the tree.
+ * 
+ * <p>Most importantly, PATRICIA requires very few comparisons to keys while
+ * doing any operation. While performing a lookup, each comparison (at most 
+ * K of them, described above) will perform a single bit comparison against 
+ * the given key, instead of comparing the entire key to another key.
+ * 
+ * <p>The {@link Trie} can return operations in lexicographical order using the 
+ * {@link #traverse(Cursor)}, 'prefix', 'submap', or 'iterator' methods. The 
+ * {@link Trie} can also scan for items that are 'bitwise' (using an XOR 
+ * metric) by the 'select' method. Bitwise closeness is determined by the 
+ * {@link KeyAnalyzer} returning true or false for a bit being set or not in 
+ * a given key.
+ * 
+ * <p>Any methods here that take an {@link Object} argument may throw a 
+ * {@link ClassCastException} if the method is expecting an instance of K 
+ * and it isn't K.
+ * 
+ * @see <a href="http://en.wikipedia.org/wiki/Radix_tree">Radix Tree</a>
+ * @see <a href="http://www.csse.monash.edu.au/~lloyd/tildeAlgDS/Tree/PATRICIA">PATRICIA</a>
+ * @see <a href="http://www.imperialviolet.org/binary/critbit.pdf">Crit-Bit Tree</a>
+ * 
+ * @author Roger Kapsi
+ * @author Sam Berlin
+ */
+public class PatriciaTrie<K, V> extends AbstractPatriciaTrie<K, V> implements Serializable
+{
+    private static final long serialVersionUID = -2246014692353432660L;
+    
+    public PatriciaTrie(KeyAnalyzer<? super K> keyAnalyzer)
+    {
+        super(keyAnalyzer);
+    }
+    
+    public PatriciaTrie(KeyAnalyzer<? super K> keyAnalyzer, Map<? extends K, ? extends V> m)
+    {
+        super(keyAnalyzer, m);
+    }
+    
+    @Override
+    public Comparator<? super K> comparator()
+    {
+        return keyAnalyzer;
+    }
+    
+    @Override
+    public SortedMap<K, V> prefixMap(K prefix)
+    {
+        return lengthInBits(prefix) == 0 ? this : new PrefixRangeMap(prefix);
+    }
+    
+    @Override
+    public K firstKey()
+    {
+        return firstEntry().getKey();
+    }
+    
+    @Override
+    public K lastKey()
+    {
+        TrieEntry<K, V> entry = lastEntry();
+        return entry != null ? entry.getKey() : null;
+    }
+    
+    @Override
+    public SortedMap<K, V> headMap(K toKey)
+    {
+        return new RangeEntryMap(null, toKey);
+    }
+    
+    @Override
+    public SortedMap<K, V> subMap(K fromKey, K toKey)
+    {
+        return new RangeEntryMap(fromKey, toKey);
+    }
+    
+    @Override
+    public SortedMap<K, V> tailMap(K fromKey)
+    {
+        return new RangeEntryMap(fromKey, null);
+    } 
+    
+    /**
+     * Returns an entry strictly higher than the given key,
+     * or null if no such entry exists.
+     */
+    private TrieEntry<K,V> higherEntry(K key)
+    {
+        // TODO: Cleanup so that we don't actually have to add/remove from the
+        //       tree.  (We do it here because there are other well-defined 
+        //       functions to perform the search.)
+        int lengthInBits = lengthInBits(key);
+        
+        if (lengthInBits == 0)
+        {
+            if (!root.isEmpty())
+            {
+                // If data in root, and more after -- return it.
+                return size() > 1 ? nextEntry(root) : null;
+            }
+            else
+            {
+                // Root is empty & we want something after empty, return first.
+                return firstEntry();
+            }
+        }
+        
+        TrieEntry<K, V> found = getNearestEntryForKey(key);
+        if (compareKeys(key, found.key))
+            return nextEntry(found);
+        
+        int bitIndex = bitIndex(key, found.key);
+        if (Tries.isValidBitIndex(bitIndex))
+        {
+            return replaceCeil(key, bitIndex);
+        }
+        else if (Tries.isNullBitKey(bitIndex))
+        {
+            if (!root.isEmpty())
+            {
+                return firstEntry();
+            }
+            else if (size() > 1)
+            {
+                return nextEntry(firstEntry());
+            }
+            else
+            {
+                return null;
+            }
+        }
+        else if (Tries.isEqualBitKey(bitIndex))
+        {
+            return nextEntry(found);
+        }
+
+        // we should have exited above.
+        throw new IllegalStateException("invalid lookup: " + key);
+    }
+    
+    /**
+     * Returns a key-value mapping associated with the least key greater
+     * than or equal to the given key, or null if there is no such key.
+     */
+    TrieEntry<K,V> ceilingEntry(K key)
+    {
+        // Basically:
+        // Follow the steps of adding an entry, but instead...
+        //
+        // - If we ever encounter a situation where we found an equal
+        //   key, we return it immediately.
+        //
+        // - If we hit an empty root, return the first iterable item.
+        //
+        // - If we have to add a new item, we temporarily add it,
+        //   find the successor to it, then remove the added item.
+        //
+        // These steps ensure that the returned value is either the
+        // entry for the key itself, or the first entry directly after
+        // the key.
+        
+        // TODO: Cleanup so that we don't actually have to add/remove from the
+        //       tree.  (We do it here because there are other well-defined 
+        //       functions to perform the search.)
+        int lengthInBits = lengthInBits(key);
+        
+        if (lengthInBits == 0)
+        {
+            if (!root.isEmpty())
+            {
+                return root;
+            }
+            else
+            {
+                return firstEntry();
+            }
+        }
+        
+        TrieEntry<K, V> found = getNearestEntryForKey(key);
+        if (compareKeys(key, found.key))
+            return found;
+        
+        int bitIndex = bitIndex(key, found.key);
+        if (Tries.isValidBitIndex(bitIndex))
+        {
+            return replaceCeil(key, bitIndex);
+        }
+        else if (Tries.isNullBitKey(bitIndex))
+        {
+            if (!root.isEmpty())
+            {
+                return root;
+            }
+            else
+            {
+                return firstEntry();
+            }
+        }
+        else if (Tries.isEqualBitKey(bitIndex))
+        {
+            return found;
+        }
+
+        // we should have exited above.
+        throw new IllegalStateException("invalid lookup: " + key);
+    }
+
+    private TrieEntry<K, V> replaceCeil(K key, int bitIndex)
+    {
+        TrieEntry<K, V> added = new TrieEntry<>(key, null, bitIndex);
+        addEntry(added);
+        incrementSize(); // must increment because remove will decrement
+        TrieEntry<K, V> ceil = nextEntry(added);
+        removeEntry(added);
+        modCount -= 2; // we didn't really modify it.
+        return ceil;
+    }
+
+    private TrieEntry<K, V> replaceLower(K key, int bitIndex)
+    {
+        TrieEntry<K, V> added = new TrieEntry<>(key, null, bitIndex);
+        addEntry(added);
+        incrementSize(); // must increment because remove will decrement
+        TrieEntry<K, V> prior = previousEntry(added);
+        removeEntry(added);
+        modCount -= 2; // we didn't really modify it.
+        return prior;
+    }
+    
+    /**
+     * Returns a key-value mapping associated with the greatest key
+     * strictly less than the given key, or null if there is no such key.
+     */
+    TrieEntry<K,V> lowerEntry(K key)
+    {
+        // Basically:
+        // Follow the steps of adding an entry, but instead...
+        //
+        // - If we ever encounter a situation where we found an equal
+        //   key, we return it's previousEntry immediately.
+        //
+        // - If we hit root (empty or not), return null.
+        //
+        // - If we have to add a new item, we temporarily add it,
+        //   find the previousEntry to it, then remove the added item.
+        //
+        // These steps ensure that the returned value is always just before
+        // the key or null (if there was nothing before it).
+        
+        // TODO: Cleanup so that we don't actually have to add/remove from the
+        //       tree.  (We do it here because there are other well-defined 
+        //       functions to perform the search.)
+        int lengthInBits = lengthInBits(key);
+        
+        if (lengthInBits == 0)
+            return null; // there can never be anything before root.
+        
+        TrieEntry<K, V> found = getNearestEntryForKey(key);
+        if (compareKeys(key, found.key))
+            return previousEntry(found);
+        
+        int bitIndex = bitIndex(key, found.key);
+        if (Tries.isValidBitIndex(bitIndex))
+        {
+            return replaceLower(key, bitIndex);
+        }
+        else if (Tries.isNullBitKey(bitIndex))
+        {
+            return null;
+        }
+        else if (Tries.isEqualBitKey(bitIndex))
+        {
+            return previousEntry(found);
+        }
+
+        // we should have exited above.
+        throw new IllegalStateException("invalid lookup: " + key);
+    }
+    
+    /**
+     * Returns a key-value mapping associated with the greatest key
+     * less than or equal to the given key, or null if there is no such key.
+     */
+    TrieEntry<K,V> floorEntry(K key) {        
+        // TODO: Cleanup so that we don't actually have to add/remove from the
+        //       tree.  (We do it here because there are other well-defined 
+        //       functions to perform the search.)
+        int lengthInBits = lengthInBits(key);
+        
+        if (lengthInBits == 0)
+        {
+            return !root.isEmpty() ? root : null;
+        }
+        
+        TrieEntry<K, V> found = getNearestEntryForKey(key);
+        if (compareKeys(key, found.key))
+            return found;
+        
+        int bitIndex = bitIndex(key, found.key);
+        if (Tries.isValidBitIndex(bitIndex))
+        {
+            return replaceLower(key, bitIndex);
+        }
+        else if (Tries.isNullBitKey(bitIndex))
+        {
+            if (!root.isEmpty())
+            {
+                return root;
+            }
+            else
+            {
+                return null;
+            }
+        }
+        else if (Tries.isEqualBitKey(bitIndex))
+        {
+            return found;
+        }
+
+        // we should have exited above.
+        throw new IllegalStateException("invalid lookup: " + key);
+    }
+    
+    /**
+     * Finds the subtree that contains the prefix.
+     * 
+     * This is very similar to getR but with the difference that
+     * we stop the lookup if h.bitIndex > lengthInBits.
+     */
+    private TrieEntry<K, V> subtree(K prefix)
+    {
+        int lengthInBits = lengthInBits(prefix);
+        
+        TrieEntry<K, V> current = root.left;
+        TrieEntry<K, V> path = root;
+        while(true)
+        {
+            if (current.bitIndex <= path.bitIndex || lengthInBits < current.bitIndex)
+                break;
+            
+            path = current;
+            current = !isBitSet(prefix, current.bitIndex)
+                    ? current.left : current.right;
+        }        
+
+        // Make sure the entry is valid for a subtree.
+        TrieEntry<K, V> entry = current.isEmpty() ? path : current;
+        
+        // If entry is root, it can't be empty.
+        if (entry.isEmpty())
+            return null;
+        
+        // if root && length of root is less than length of lookup,
+        // there's nothing.
+        // (this prevents returning the whole subtree if root has an empty
+        //  string and we want to lookup things with "\0")
+        if (entry == root && lengthInBits(entry.getKey()) < lengthInBits)
+            return null;
+        
+        // Found key's length-th bit differs from our key
+        // which means it cannot be the prefix...
+        if (isBitSet(prefix, lengthInBits) != isBitSet(entry.key, lengthInBits))
+            return null;
+        
+        // ... or there are less than 'length' equal bits
+        int bitIndex = bitIndex(prefix, entry.key);
+        return (bitIndex >= 0 && bitIndex < lengthInBits) ? null : entry;
+    }
+    
+    /**
+     * Returns the last entry the {@link Trie} is storing.
+     * 
+     * <p>This is implemented by going always to the right until
+     * we encounter a valid uplink. That uplink is the last key.
+     */
+    private TrieEntry<K, V> lastEntry()
+    {
+        return followRight(root.left);
+    }
+    
+    /**
+     * Traverses down the right path until it finds an uplink.
+     */
+    private TrieEntry<K, V> followRight(TrieEntry<K, V> node)
+    {
+        // if Trie is empty, no last entry.
+        if (node.right == null)
+            return null;
+        
+        // Go as far right as possible, until we encounter an uplink.
+        while (node.right.bitIndex > node.bitIndex)
+        {
+            node = node.right;
+        }
+        
+        return node.right;
+    }
+    
+    /**
+     * Returns the node lexicographically before the given node (or null if none).
+     * 
+     * This follows four simple branches:
+     *  - If the uplink that returned us was a right uplink:
+     *      - If predecessor's left is a valid uplink from predecessor, return it.
+     *      - Else, follow the right path from the predecessor's left.
+     *  - If the uplink that returned us was a left uplink:
+     *      - Loop back through parents until we encounter a node where 
+     *        node != node.parent.left.
+     *          - If node.parent.left is uplink from node.parent:
+     *              - If node.parent.left is not root, return it.
+     *              - If it is root & root isEmpty, return null.
+     *              - If it is root & root !isEmpty, return root.
+     *          - If node.parent.left is not uplink from node.parent:
+     *              - Follow right path for first right child from node.parent.left   
+     * 
+     * @param start the start entry
+     */
+    private TrieEntry<K, V> previousEntry(TrieEntry<K, V> start)
+    {
+        if (start.predecessor == null)
+            throw new IllegalArgumentException("must have come from somewhere!");
+        
+        if (start.predecessor.right == start)
+        {
+            return isValidUplink(start.predecessor.left, start.predecessor)
+                    ? start.predecessor.left
+                    : followRight(start.predecessor.left);
+        }
+
+        TrieEntry<K, V> node = start.predecessor;
+        while (node.parent != null && node == node.parent.left)
+        {
+            node = node.parent;
+        }
+
+        if (node.parent == null) // can be null if we're looking up root.
+            return null;
+
+        if (isValidUplink(node.parent.left, node.parent))
+        {
+            if (node.parent.left == root)
+            {
+                return root.isEmpty() ? null : root;
+            }
+            else
+            {
+                return node.parent.left;
+            }
+        }
+        else
+        {
+            return followRight(node.parent.left);
+        }
+    }
+    
+    /**
+     * Returns the entry lexicographically after the given entry.
+     * If the given entry is null, returns the first node.
+     * 
+     * This will traverse only within the subtree.  If the given node
+     * is not within the subtree, this will have undefined results.
+     */
+    private TrieEntry<K, V> nextEntryInSubtree(TrieEntry<K, V> node, TrieEntry<K, V> parentOfSubtree)
+    {
+        return (node == null) ? firstEntry() : nextEntryImpl(node.predecessor, node, parentOfSubtree);
+    }
+    
+    private boolean isPrefix(K key, K prefix)
+    {
+        return keyAnalyzer.isPrefix(key, prefix);
+    }
+    
+    /**
+     * A range view of the {@link Trie}
+     */
+    private abstract class RangeMap extends AbstractMap<K, V> implements SortedMap<K, V>
+    {
+        /**
+         * The {@link #entrySet()} view
+         */
+        private transient volatile Set<Map.Entry<K, V>> entrySet;
+
+        /**
+         * Creates and returns an {@link #entrySet()} 
+         * view of the {@link RangeMap}
+         */
+        protected abstract Set<Map.Entry<K, V>> createEntrySet();
+
+        /**
+         * Returns the FROM Key
+         */
+        protected abstract K getFromKey();
+        
+        /**
+         * Whether or not the {@link #getFromKey()} is in the range
+         */
+        protected abstract boolean isFromInclusive();
+        
+        /**
+         * Returns the TO Key
+         */
+        protected abstract K getToKey();
+        
+        /**
+         * Whether or not the {@link #getToKey()} is in the range
+         */
+        protected abstract boolean isToInclusive();
+        
+        
+        @Override
+        public Comparator<? super K> comparator()
+        {
+            return PatriciaTrie.this.comparator();
+        }
+        
+        @Override
+        public boolean containsKey(Object key)
+        {
+            return inRange(Tries.<K>cast(key)) && PatriciaTrie.this.containsKey(key);
+        }
+        
+        @Override
+        public V remove(Object key)
+        {
+            return (!inRange(Tries.<K>cast(key))) ? null : PatriciaTrie.this.remove(key);
+        }
+        
+        @Override
+        public V get(Object key)
+        {
+            return (!inRange(Tries.<K>cast(key))) ? null : PatriciaTrie.this.get(key);
+        }
+        
+        @Override
+        public V put(K key, V value)
+        {
+            if (!inRange(key))
+                throw new IllegalArgumentException("Key is out of range: " + key);
+
+            return PatriciaTrie.this.put(key, value);
+        }
+        
+        @Override
+        public Set<Map.Entry<K, V>> entrySet()
+        {
+            if (entrySet == null)
+                entrySet = createEntrySet();
+            return entrySet;
+        }
+        
+        @Override
+        public SortedMap<K, V> subMap(K fromKey, K toKey)
+        {
+            if (!inRange2(fromKey))
+                throw new IllegalArgumentException("FromKey is out of range: " + fromKey);
+
+            if (!inRange2(toKey))
+                throw new IllegalArgumentException("ToKey is out of range: " + toKey);
+
+            return createRangeMap(fromKey, isFromInclusive(), toKey, isToInclusive());
+        }
+        
+        @Override
+        public SortedMap<K, V> headMap(K toKey)
+        {
+            if (!inRange2(toKey))
+                throw new IllegalArgumentException("ToKey is out of range: " + toKey);
+
+            return createRangeMap(getFromKey(), isFromInclusive(), toKey, isToInclusive());
+        }
+        
+        @Override
+        public SortedMap<K, V> tailMap(K fromKey)
+        {
+            if (!inRange2(fromKey))
+                throw new IllegalArgumentException("FromKey is out of range: " + fromKey);
+
+            return createRangeMap(fromKey, isFromInclusive(), getToKey(), isToInclusive());
+        }
+
+        /**
+         * Returns true if the provided key is greater than TO and
+         * less than FROM
+         */
+        protected boolean inRange(K key)
+        {
+            K fromKey = getFromKey();
+            K toKey = getToKey();
+
+            return (fromKey == null || inFromRange(key, false))
+                    && (toKey == null || inToRange(key, false));
+        }
+
+        /**
+         * This form allows the high endpoint (as well as all legit keys)
+         */
+        protected boolean inRange2(K key)
+        {
+            K fromKey = getFromKey();
+            K toKey = getToKey();
+
+            return (fromKey == null || inFromRange(key, false))
+                    && (toKey == null || inToRange(key, true));
+        }
+
+        /**
+         * Returns true if the provided key is in the FROM range 
+         * of the {@link RangeMap}
+         */
+        protected boolean inFromRange(K key, boolean forceInclusive)
+        {
+            K fromKey = getFromKey();
+            boolean fromInclusive = isFromInclusive();
+
+            int ret = keyAnalyzer.compare(key, fromKey);
+            return (fromInclusive || forceInclusive) ? ret >= 0 : ret > 0;
+        }
+
+        /**
+         * Returns true if the provided key is in the TO range 
+         * of the {@link RangeMap}
+         */
+        protected boolean inToRange(K key, boolean forceInclusive)
+        {
+            K toKey = getToKey();
+            boolean toInclusive = isToInclusive();
+
+            int ret = keyAnalyzer.compare(key, toKey);
+            return (toInclusive || forceInclusive) ? ret <= 0 : ret < 0;
+        }
+
+        /**
+         * Creates and returns a sub-range view of the current {@link RangeMap}
+         */
+        protected abstract SortedMap<K, V> createRangeMap(K fromKey, boolean fromInclusive, K toKey, boolean toInclusive);
+    }
+   
+   /**
+    * A {@link RangeMap} that deals with {@link Entry}s
+    */
+   private class RangeEntryMap extends RangeMap
+   {
+       /** 
+        * The key to start from, null if the beginning. 
+        */
+       protected final K fromKey;
+       
+       /** 
+        * The key to end at, null if till the end. 
+        */
+       protected final K toKey;
+       
+       /** 
+        * Whether or not the 'from' is inclusive. 
+        */
+       protected final boolean fromInclusive;
+       
+       /** 
+        * Whether or not the 'to' is inclusive. 
+        */
+       protected final boolean toInclusive;
+       
+       /**
+        * Creates a {@link RangeEntryMap} with the fromKey included and
+        * the toKey excluded from the range
+        */
+       protected RangeEntryMap(K fromKey, K toKey)
+       {
+           this(fromKey, true, toKey, false);
+       }
+       
+       /**
+        * Creates a {@link RangeEntryMap}
+        */
+       protected RangeEntryMap(K fromKey, boolean fromInclusive, K toKey, boolean toInclusive)
+       {
+           if (fromKey == null && toKey == null)
+               throw new IllegalArgumentException("must have a from or to!");
+           
+           if (fromKey != null && toKey != null && keyAnalyzer.compare(fromKey, toKey) > 0)
+               throw new IllegalArgumentException("fromKey > toKey");
+           
+           this.fromKey = fromKey;
+           this.fromInclusive = fromInclusive;
+           this.toKey = toKey;
+           this.toInclusive = toInclusive;
+       }
+       
+       
+       @Override
+       public K firstKey()
+       {
+           Map.Entry<K,V> e  = fromKey == null
+                ? firstEntry()
+                : fromInclusive ? ceilingEntry(fromKey) : higherEntry(fromKey);
+           
+           K first = e != null ? e.getKey() : null;
+           if (e == null || toKey != null && !inToRange(first, false))
+               throw new NoSuchElementException();
+
+           return first;
+       }
+
+       
+       @Override
+       public K lastKey()
+       {
+           Map.Entry<K,V> e = toKey == null
+                ? lastEntry()
+                : toInclusive ? floorEntry(toKey) : lowerEntry(toKey);
+           
+           K last = e != null ? e.getKey() : null;
+           if (e == null || fromKey != null && !inFromRange(last, false))
+               throw new NoSuchElementException();
+
+           return last;
+       }
+       
+       @Override
+       protected Set<Entry<K, V>> createEntrySet()
+       {
+           return new RangeEntrySet(this);
+       }
+       
+       @Override
+       public K getFromKey()
+       {
+           return fromKey;
+       }
+       
+       @Override
+       public K getToKey()
+       {
+           return toKey;
+       }
+       
+       @Override
+       public boolean isFromInclusive()
+       {
+           return fromInclusive;
+       }
+       
+       @Override
+       public boolean isToInclusive()
+       {
+           return toInclusive;
+       }
+       
+       @Override
+       protected SortedMap<K, V> createRangeMap(K fromKey, boolean fromInclusive, K toKey, boolean toInclusive)
+       {
+           return new RangeEntryMap(fromKey, fromInclusive, toKey, toInclusive);
+       }
+   }
+   
+    /**
+     * A {@link Set} view of a {@link RangeMap}
+     */
+    private class RangeEntrySet extends AbstractSet<Map.Entry<K, V>>
+    {
+
+        private final RangeMap delegate;
+
+        private int size = -1;
+
+        private int expectedModCount = -1;
+
+        /**
+         * Creates a {@link RangeEntrySet}
+         */
+        public RangeEntrySet(RangeMap delegate)
+        {
+            if (delegate == null)
+                throw new NullPointerException("delegate");
+
+            this.delegate = delegate;
+        }
+        
+        @Override
+        public Iterator<Map.Entry<K, V>> iterator()
+        {
+            K fromKey = delegate.getFromKey();
+            K toKey = delegate.getToKey();
+
+            TrieEntry<K, V> first = fromKey == null ? firstEntry() : ceilingEntry(fromKey);
+            TrieEntry<K, V> last = null;
+            if (toKey != null)
+                last = ceilingEntry(toKey);
+
+            return new EntryIterator(first, last);
+        }
+        
+        @Override
+        public int size()
+        {
+            if (size == -1 || expectedModCount != PatriciaTrie.this.modCount)
+            {
+                size = 0;
+
+                for (Iterator<?> it = iterator(); it.hasNext(); it.next())
+                {
+                    ++size;
+                }
+
+                expectedModCount = PatriciaTrie.this.modCount;
+            }
+
+            return size;
+        }
+        
+        @Override
+        public boolean isEmpty()
+        {
+            return !iterator().hasNext();
+        }
+        
+        @Override
+        public boolean contains(Object o)
+        {
+            if (!(o instanceof Map.Entry<?, ?>))
+                return false;
+
+            @SuppressWarnings("unchecked")
+            Map.Entry<K, V> entry = (Map.Entry<K, V>) o;
+            K key = entry.getKey();
+            if (!delegate.inRange(key))
+                return false;
+
+            TrieEntry<K, V> node = getEntry(key);
+            return node != null && Tries.areEqual(node.getValue(), entry.getValue());
+        }
+        
+        @Override
+        public boolean remove(Object o)
+        {
+            if (!(o instanceof Map.Entry<?, ?>))
+                return false;
+
+            @SuppressWarnings("unchecked")
+            Map.Entry<K, V> entry = (Map.Entry<K, V>) o;
+            K key = entry.getKey();
+            if (!delegate.inRange(key))
+                return false;
+
+            TrieEntry<K, V> node = getEntry(key);
+            if (node != null && Tries.areEqual(node.getValue(), entry.getValue()))
+            {
+                removeEntry(node);
+                return true;
+            }
+
+            return false;
+        }
+        
+        /** 
+         * An {@link Iterator} for {@link RangeEntrySet}s. 
+         */
+        private final class EntryIterator extends TrieIterator<Map.Entry<K,V>>
+        {
+            private final K excludedKey;
+
+            /**
+             * Creates a {@link EntryIterator}
+             */
+            private EntryIterator(TrieEntry<K,V> first, TrieEntry<K,V> last)
+            {
+                super(first);
+                this.excludedKey = (last != null ? last.getKey() : null);
+            }
+            
+            @Override
+            public boolean hasNext()
+            {
+                return next != null && !Tries.areEqual(next.key, excludedKey);
+            }
+            
+            @Override
+            public Map.Entry<K,V> next()
+            {
+                if (next == null || Tries.areEqual(next.key, excludedKey))
+                    throw new NoSuchElementException();
+                
+                return nextEntry();
+            }
+        }
+    }   
+   
+    /** 
+     * A submap used for prefix views over the {@link Trie}. 
+     */
+    private class PrefixRangeMap extends RangeMap
+    {
+        
+        private final K prefix;
+        
+        private K fromKey = null;
+        
+        private K toKey = null;
+        
+        private int expectedModCount = -1;
+        
+        private int size = -1;
+        
+        /**
+         * Creates a {@link PrefixRangeMap}
+         */
+        private PrefixRangeMap(K prefix)
+        {
+            this.prefix = prefix;
+        }
+        
+        /**
+         * This method does two things. It determinates the FROM
+         * and TO range of the {@link PrefixRangeMap} and the number
+         * of elements in the range. This method must be called every 
+         * time the {@link Trie} has changed.
+         */
+        private int fixup()
+        {
+            // The trie has changed since we last
+            // found our toKey / fromKey
+            if (size == - 1 || PatriciaTrie.this.modCount != expectedModCount)
+            {
+                Iterator<Map.Entry<K, V>> it = entrySet().iterator();
+                size = 0;
+                
+                Map.Entry<K, V> entry = null;
+                if (it.hasNext())
+                {
+                    entry = it.next();
+                    size = 1;
+                }
+                
+                fromKey = entry == null ? null : entry.getKey();
+                if (fromKey != null)
+                {
+                    TrieEntry<K, V> prior = previousEntry((TrieEntry<K, V>)entry);
+                    fromKey = prior == null ? null : prior.getKey();
+                }
+                
+                toKey = fromKey;
+                
+                while (it.hasNext())
+                {
+                    ++size;
+                    entry = it.next();
+                }
+                
+                toKey = entry == null ? null : entry.getKey();
+                
+                if (toKey != null)
+                {
+                    entry = nextEntry((TrieEntry<K, V>)entry);
+                    toKey = entry == null ? null : entry.getKey();
+                }
+                
+                expectedModCount = PatriciaTrie.this.modCount;
+            }
+            
+            return size;
+        }
+        
+        @Override
+        public K firstKey()
+        {
+            fixup();
+            
+            Map.Entry<K,V> e = fromKey == null ? firstEntry() : higherEntry(fromKey);
+            K first = e != null ? e.getKey() : null;
+            if (e == null || !isPrefix(first, prefix))
+                throw new NoSuchElementException();
+            
+            return first;
+        }
+        
+        @Override
+        public K lastKey()
+        {
+            fixup();
+            
+            Map.Entry<K,V> e = toKey == null ? lastEntry() : lowerEntry(toKey);
+            K last = e != null ? e.getKey() : null;
+            if (e == null || !isPrefix(last, prefix))
+                throw new NoSuchElementException();
+            
+            return last;
+        }
+        
+        /**
+         * Returns true if this {@link PrefixRangeMap}'s key is a prefix
+         * of the provided key.
+         */
+        @Override
+        protected boolean inRange(K key)
+        {
+            return isPrefix(key, prefix);
+        }
+
+        /**
+         * Same as {@link #inRange(Object)}
+         */
+        @Override
+        protected boolean inRange2(K key)
+        {
+            return inRange(key);
+        }
+        
+        /**
+         * Returns true if the provided Key is in the FROM range
+         * of the {@link PrefixRangeMap}
+         */
+        @Override
+        protected boolean inFromRange(K key, boolean forceInclusive)
+        {
+            return isPrefix(key, prefix);
+        }
+        
+        /**
+         * Returns true if the provided Key is in the TO range
+         * of the {@link PrefixRangeMap}
+         */
+        @Override
+        protected boolean inToRange(K key, boolean forceInclusive)
+        {
+            return isPrefix(key, prefix);
+        }
+        
+        @Override
+        protected Set<Map.Entry<K, V>> createEntrySet()
+        {
+            return new PrefixRangeEntrySet(this);
+        }
+        
+        @Override
+        public K getFromKey()
+        {
+            return fromKey;
+        }
+        
+        @Override
+        public K getToKey()
+        {
+            return toKey;
+        }
+        
+        @Override
+        public boolean isFromInclusive()
+        {
+            return false;
+        }
+        
+        @Override
+        public boolean isToInclusive()
+        {
+            return false;
+        }
+        
+        @Override
+        protected SortedMap<K, V> createRangeMap(K fromKey, boolean fromInclusive,
+                                                 K toKey, boolean toInclusive)
+        {
+            return new RangeEntryMap(fromKey, fromInclusive, toKey, toInclusive);
+        }
+    }
+    
+    /**
+     * A prefix {@link RangeEntrySet} view of the {@link Trie}
+     */
+    private final class PrefixRangeEntrySet extends RangeEntrySet
+    {
+        private final PrefixRangeMap delegate;
+        
+        private TrieEntry<K, V> prefixStart;
+        
+        private int expectedModCount = -1;
+        
+        /**
+         * Creates a {@link PrefixRangeEntrySet}
+         */
+        public PrefixRangeEntrySet(PrefixRangeMap delegate)
+        {
+            super(delegate);
+            this.delegate = delegate;
+        }
+        
+        @Override
+        public int size()
+        {
+            return delegate.fixup();
+        }
+        
+        @Override
+        public Iterator<Map.Entry<K,V>> iterator()
+        {
+            if (PatriciaTrie.this.modCount != expectedModCount)
+            {
+                prefixStart = subtree(delegate.prefix);
+                expectedModCount = PatriciaTrie.this.modCount;
+            }
+            
+            if (prefixStart == null)
+            {
+                Set<Map.Entry<K,V>> empty = Collections.emptySet();
+                return empty.iterator();
+            }
+            else if (lengthInBits(delegate.prefix) >= prefixStart.bitIndex)
+            {
+                return new SingletonIterator(prefixStart);
+            }
+            else
+            {
+                return new EntryIterator(prefixStart, delegate.prefix);
+            }
+        }
+        
+        /** 
+         * An {@link Iterator} that holds a single {@link TrieEntry}. 
+         */
+        private final class SingletonIterator implements Iterator<Map.Entry<K, V>>
+        {
+            private final TrieEntry<K, V> entry;
+            
+            private int hit = 0;
+            
+            public SingletonIterator(TrieEntry<K, V> entry)
+            {
+                this.entry = entry;
+            }
+            
+            @Override
+            public boolean hasNext()
+            {
+                return hit == 0;
+            }
+            
+            @Override
+            public Map.Entry<K, V> next()
+            {
+                if (hit != 0)
+                    throw new NoSuchElementException();
+                
+                ++hit;
+                return entry;
+            }
+
+            
+            @Override
+            public void remove()
+            {
+                if (hit != 1)
+                    throw new IllegalStateException();
+                
+                ++hit;
+                PatriciaTrie.this.removeEntry(entry);
+            }
+        }
+        
+        /** 
+         * An {@link Iterator} for iterating over a prefix search. 
+         */
+        private final class EntryIterator extends TrieIterator<Map.Entry<K, V>>
+        {
+            // values to reset the subtree if we remove it.
+            protected final K prefix;
+            protected boolean lastOne;
+            
+            protected TrieEntry<K, V> subtree; // the subtree to search within
+            
+            /**
+             * Starts iteration at the given entry & search only 
+             * within the given subtree.
+             */
+            EntryIterator(TrieEntry<K, V> startScan, K prefix)
+            {
+                subtree = startScan;
+                next = PatriciaTrie.this.followLeft(startScan);
+                this.prefix = prefix;
+            }
+            
+            @Override
+            public Map.Entry<K,V> next()
+            {
+                Map.Entry<K, V> entry = nextEntry();
+                if (lastOne)
+                    next = null;
+                return entry;
+            }
+            
+            @Override
+            protected TrieEntry<K, V> findNext(TrieEntry<K, V> prior)
+            {
+                return PatriciaTrie.this.nextEntryInSubtree(prior, subtree);
+            }
+            
+            @Override
+            public void remove()
+            {
+                // If the current entry we're removing is the subtree
+                // then we need to find a new subtree parent.
+                boolean needsFixing = false;
+                int bitIdx = subtree.bitIndex;
+                if (current == subtree)
+                    needsFixing = true;
+                
+                super.remove();
+                
+                // If the subtree changed its bitIndex or we
+                // removed the old subtree, get a new one.
+                if (bitIdx != subtree.bitIndex || needsFixing)
+                    subtree = subtree(prefix);
+
+                // If the subtree's bitIndex is less than the
+                // length of our prefix, it's the last item
+                // in the prefix tree.
+                if (lengthInBits(prefix) >= subtree.bitIndex)
+                    lastOne = true;
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/trie/Trie.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/trie/Trie.java b/src/java/org/apache/cassandra/index/sasi/utils/trie/Trie.java
new file mode 100644
index 0000000..44809f3
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/trie/Trie.java
@@ -0,0 +1,152 @@
+/*
+ * Copyright 2005-2010 Roger Kapsi, Sam Berlin
+ *
+ *   Licensed 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.index.sasi.utils.trie;
+
+import java.util.Map;
+import java.util.SortedMap;
+
+import org.apache.cassandra.index.sasi.utils.trie.Cursor.Decision;
+
+/**
+ * This class is taken from https://github.com/rkapsi/patricia-trie (v0.6), and slightly modified
+ * to correspond to Cassandra code style, as the only Patricia Trie implementation,
+ * which supports pluggable key comparators (e.g. commons-collections PatriciaTrie (which is based
+ * on rkapsi/patricia-trie project) only supports String keys)
+ * but unfortunately is not deployed to the maven central as a downloadable artifact.
+ */
+
+/**
+ * Defines the interface for a prefix tree, an ordered tree data structure. For 
+ * more information, see <a href="http://en.wikipedia.org/wiki/Trie">Tries</a>.
+ * 
+ * @author Roger Kapsi
+ * @author Sam Berlin
+ */
+public interface Trie<K, V> extends SortedMap<K, V>
+{
+    /**
+     * Returns the {@link Map.Entry} whose key is closest in a bitwise XOR 
+     * metric to the given key. This is NOT lexicographic closeness.
+     * For example, given the keys:
+     *
+     * <ol>
+     * <li>D = 1000100
+     * <li>H = 1001000
+     * <li>L = 1001100
+     * </ol>
+     * 
+     * If the {@link Trie} contained 'H' and 'L', a lookup of 'D' would 
+     * return 'L', because the XOR distance between D &amp; L is smaller 
+     * than the XOR distance between D &amp; H. 
+     * 
+     * @return The {@link Map.Entry} whose key is closest in a bitwise XOR metric
+     * to the provided key.
+     */
+    Map.Entry<K, V> select(K key);
+    
+    /**
+     * Returns the key that is closest in a bitwise XOR metric to the 
+     * provided key. This is NOT lexicographic closeness!
+     * 
+     * For example, given the keys:
+     * 
+     * <ol>
+     * <li>D = 1000100
+     * <li>H = 1001000
+     * <li>L = 1001100
+     * </ol>
+     * 
+     * If the {@link Trie} contained 'H' and 'L', a lookup of 'D' would 
+     * return 'L', because the XOR distance between D &amp; L is smaller 
+     * than the XOR distance between D &amp; H. 
+     * 
+     * @return The key that is closest in a bitwise XOR metric to the provided key.
+     */
+    @SuppressWarnings("unused")
+    K selectKey(K key);
+    
+    /**
+     * Returns the value whose key is closest in a bitwise XOR metric to 
+     * the provided key. This is NOT lexicographic closeness!
+     * 
+     * For example, given the keys:
+     * 
+     * <ol>
+     * <li>D = 1000100
+     * <li>H = 1001000
+     * <li>L = 1001100
+     * </ol>
+     * 
+     * If the {@link Trie} contained 'H' and 'L', a lookup of 'D' would 
+     * return 'L', because the XOR distance between D &amp; L is smaller 
+     * than the XOR distance between D &amp; H. 
+     * 
+     * @return The value whose key is closest in a bitwise XOR metric
+     * to the provided key.
+     */
+    @SuppressWarnings("unused")
+    V selectValue(K key);
+    
+    /**
+     * Iterates through the {@link Trie}, starting with the entry whose bitwise
+     * value is closest in an XOR metric to the given key. After the closest
+     * entry is found, the {@link Trie} will call select on that entry and continue
+     * calling select for each entry (traversing in order of XOR closeness,
+     * NOT lexicographically) until the cursor returns {@link Decision#EXIT}.
+     * 
+     * <p>The cursor can return {@link Decision#CONTINUE} to continue traversing.
+     * 
+     * <p>{@link Decision#REMOVE_AND_EXIT} is used to remove the current element
+     * and stop traversing.
+     * 
+     * <p>Note: The {@link Decision#REMOVE} operation is not supported.
+     * 
+     * @return The entry the cursor returned {@link Decision#EXIT} on, or null 
+     * if it continued till the end.
+     */
+    Map.Entry<K,V> select(K key, Cursor<? super K, ? super V> cursor);
+    
+    /**
+     * Traverses the {@link Trie} in lexicographical order. 
+     * {@link Cursor#select(java.util.Map.Entry)} will be called on each entry.
+     * 
+     * <p>The traversal will stop when the cursor returns {@link Decision#EXIT}, 
+     * {@link Decision#CONTINUE} is used to continue traversing and 
+     * {@link Decision#REMOVE} is used to remove the element that was selected 
+     * and continue traversing.
+     * 
+     * <p>{@link Decision#REMOVE_AND_EXIT} is used to remove the current element
+     * and stop traversing.
+     *   
+     * @return The entry the cursor returned {@link Decision#EXIT} on, or null 
+     * if it continued till the end.
+     */
+    Map.Entry<K,V> traverse(Cursor<? super K, ? super V> cursor);
+    
+    /**
+     * Returns a view of this {@link Trie} of all elements that are prefixed 
+     * by the given key.
+     * 
+     * <p>In a {@link Trie} with fixed size keys, this is essentially a 
+     * {@link #get(Object)} operation.
+     * 
+     * <p>For example, if the {@link Trie} contains 'Anna', 'Anael', 
+     * 'Analu', 'Andreas', 'Andrea', 'Andres', and 'Anatole', then
+     * a lookup of 'And' would return 'Andreas', 'Andrea', and 'Andres'.
+     */
+    SortedMap<K, V> prefixMap(K prefix);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/trie/Tries.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/trie/Tries.java b/src/java/org/apache/cassandra/index/sasi/utils/trie/Tries.java
new file mode 100644
index 0000000..c258dd2
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/trie/Tries.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2005-2010 Roger Kapsi
+ *
+ *   Licensed 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.
+ */
+
+/**
+ * This class is taken from https://github.com/rkapsi/patricia-trie (v0.6), and slightly modified
+ * to correspond to Cassandra code style, as the only Patricia Trie implementation,
+ * which supports pluggable key comparators (e.g. commons-collections PatriciaTrie (which is based
+ * on rkapsi/patricia-trie project) only supports String keys)
+ * but unfortunately is not deployed to the maven central as a downloadable artifact.
+ */
+
+package org.apache.cassandra.index.sasi.utils.trie;
+
+/**
+ * A collection of {@link Trie} utilities
+ */
+public class Tries
+{
+    /** 
+     * Returns true if bitIndex is a {@link KeyAnalyzer#OUT_OF_BOUNDS_BIT_KEY}
+     */
+    static boolean isOutOfBoundsIndex(int bitIndex)
+    {
+        return bitIndex == KeyAnalyzer.OUT_OF_BOUNDS_BIT_KEY;
+    }
+
+    /** 
+     * Returns true if bitIndex is a {@link KeyAnalyzer#EQUAL_BIT_KEY}
+     */
+    static boolean isEqualBitKey(int bitIndex)
+    {
+        return bitIndex == KeyAnalyzer.EQUAL_BIT_KEY;
+    }
+
+    /** 
+     * Returns true if bitIndex is a {@link KeyAnalyzer#NULL_BIT_KEY} 
+     */
+    static boolean isNullBitKey(int bitIndex)
+    {
+        return bitIndex == KeyAnalyzer.NULL_BIT_KEY;
+    }
+
+    /** 
+     * Returns true if the given bitIndex is valid. Indices 
+     * are considered valid if they're between 0 and 
+     * {@link Integer#MAX_VALUE}
+     */
+    static boolean isValidBitIndex(int bitIndex)
+    {
+        return 0 <= bitIndex;
+    }
+
+    /**
+     * Returns true if both values are either null or equal
+     */
+    static boolean areEqual(Object a, Object b)
+    {
+        return (a == null ? b == null : a.equals(b));
+    }
+
+    /**
+     * Throws a {@link NullPointerException} with the given message if 
+     * the argument is null.
+     */
+    static <T> T notNull(T o, String message)
+    {
+        if (o == null)
+            throw new NullPointerException(message);
+
+        return o;
+    }
+
+    /**
+     * A utility method to cast keys. It actually doesn't
+     * cast anything. It's just fooling the compiler!
+     */
+    @SuppressWarnings("unchecked")
+    static <K> K cast(Object key)
+    {
+        return (K)key;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/io/sstable/Component.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/Component.java b/src/java/org/apache/cassandra/io/sstable/Component.java
index 9454882..38152af 100644
--- a/src/java/org/apache/cassandra/io/sstable/Component.java
+++ b/src/java/org/apache/cassandra/io/sstable/Component.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.io.sstable;
 
 import java.io.File;
 import java.util.EnumSet;
+import java.util.regex.Pattern;
 
 import com.google.common.base.Objects;
 
@@ -57,6 +58,8 @@ public class Component
         SUMMARY("Summary.db"),
         // table of contents, stores the list of all components for the sstable
         TOC("TOC.txt"),
+        // built-in secondary index (may be multiple per sstable)
+        SECONDARY_INDEX("SI_.*.db"),
         // custom component, used by e.g. custom compaction strategy
         CUSTOM(new String[] { null });
         
@@ -74,9 +77,12 @@ public class Component
         static Type fromRepresentation(String repr)
         {
             for (Type type : TYPES)
-                for (String representation : type.repr)
-                    if (repr.equals(representation))
-                        return type;
+            {
+                if (type.repr == null || type.repr.length == 0 || type.repr[0] == null)
+                    continue;
+                if (Pattern.matches(type.repr[0], repr))
+                    return type;
+            }
             return CUSTOM;
         }
     }
@@ -169,6 +175,7 @@ public class Component
             case CRC:               component = Component.CRC;                          break;
             case SUMMARY:           component = Component.SUMMARY;                      break;
             case TOC:               component = Component.TOC;                          break;
+            case SECONDARY_INDEX:   component = new Component(Type.SECONDARY_INDEX, path.right); break;
             case CUSTOM:            component = new Component(Type.CUSTOM, path.right); break;
             default:
                  throw new IllegalStateException();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/KeyIterator.java b/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
index f02b9d1..d51e97b 100644
--- a/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
@@ -84,6 +84,7 @@ public class KeyIterator extends AbstractIterator<DecoratedKey> implements Close
     private final In in;
     private final IPartitioner partitioner;
 
+    private long keyPosition;
 
     public KeyIterator(Descriptor desc, CFMetaData metadata)
     {
@@ -99,6 +100,7 @@ public class KeyIterator extends AbstractIterator<DecoratedKey> implements Close
             if (in.isEOF())
                 return endOfData();
 
+            keyPosition = in.getFilePointer();
             DecoratedKey key = partitioner.decorateKey(ByteBufferUtil.readWithShortLength(in.get()));
             RowIndexEntry.Serializer.skip(in.get(), desc.version); // skip remainder of the entry
             return key;
@@ -123,4 +125,9 @@ public class KeyIterator extends AbstractIterator<DecoratedKey> implements Close
     {
         return in.length();
     }
+
+    public long getKeyPosition()
+    {
+        return keyPosition;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java
index d6f54e2..f0b6bac 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java
@@ -18,7 +18,7 @@
 package org.apache.cassandra.io.sstable.format;
 
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.rows.ColumnData;
+import org.apache.cassandra.db.rows.Unfiltered;
 
 /**
  * Observer for events in the lifecycle of writing out an sstable.
@@ -32,7 +32,7 @@ public interface SSTableFlushObserver
 
     /**
      * Called when a new partition in being written to the sstable,
-     * but before any cells are processed (see {@link #nextCell(ColumnData)}).
+     * but before any cells are processed (see {@link #nextUnfilteredCluster(Unfiltered)}).
      *
      * @param key The key being appended to SSTable.
      * @param indexPosition The position of the key in the SSTable PRIMARY_INDEX file.
@@ -40,13 +40,13 @@ public interface SSTableFlushObserver
     void startPartition(DecoratedKey key, long indexPosition);
 
     /**
-     * Called after the cell is written to the sstable.
+     * Called after the unfiltered cluster is written to the sstable.
      * Will be preceded by a call to {@code startPartition(DecoratedKey, long)},
-     * and the cell should be assumed to belong to that row.
+     * and the cluster should be assumed to belong to that partition.
      *
-     * @param cell The cell being added to the row.
+     * @param unfilteredCluster The unfiltered cluster being added to SSTable.
      */
-    void nextCell(ColumnData cell);
+    void nextUnfilteredCluster(Unfiltered unfilteredCluster);
 
     /**
      * Called when all data is written to the file and it's ready to be finished up.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 8206092..9e9b98a 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -1795,6 +1795,26 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         return sstableMetadata.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE;
     }
 
+    public DecoratedKey keyAt(long indexPosition) throws IOException
+    {
+        DecoratedKey key;
+        try (FileDataInput in = ifile.createReader(indexPosition))
+        {
+            if (in.isEOF())
+                return null;
+
+            key = decorateKey(ByteBufferUtil.readWithShortLength(in));
+
+            // hint read path about key location if caching is enabled
+            // this saves index summary lookup and index file iteration which whould be pretty costly
+            // especially in presence of promoted column indexes
+            if (isKeyCacheSetup())
+                cacheKey(key, rowIndexEntrySerializer.deserialize(in));
+        }
+
+        return key;
+    }
+
     /**
      * TODO: Move someplace reusable
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
index 3203964..ab38ba9 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@ -277,7 +277,14 @@ public abstract class SSTableWriter extends SSTable implements Transactional
 
     public final Throwable commit(Throwable accumulate)
     {
-        return txnProxy.commit(accumulate);
+        try
+        {
+            return txnProxy.commit(accumulate);
+        }
+        finally
+        {
+            observers.forEach(SSTableFlushObserver::complete);
+        }
     }
 
     public final Throwable abort(Throwable accumulate)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java b/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java
index c815c9e..c2cc549 100644
--- a/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java
+++ b/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java
@@ -62,4 +62,9 @@ public class DataOutputBufferFixed extends DataOutputBuffer
     {
         throw new BufferOverflowException();
     }
+
+    public void clear()
+    {
+        buffer.clear();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/io/util/SequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
index dd49868..5120e3c 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -169,6 +169,13 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
         return this;
     }
 
+    public void skipBytes(int numBytes) throws IOException
+    {
+        flush();
+        fchannel.position(fchannel.position() + numBytes);
+        bufferOffset = fchannel.position();
+    }
+
     /**
      * Synchronize file contents with disk.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
index 4712dff..fcda9ba 100644
--- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
+++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
@@ -669,4 +669,45 @@ public class ByteBufferUtil
         return buf;
     }
 
+    /**
+     * Check is the given buffer contains a given sub-buffer.
+     *
+     * @param buffer The buffer to search for sequence of bytes in.
+     * @param subBuffer The buffer to match.
+     *
+     * @return true if buffer contains sub-buffer, false otherwise.
+     */
+    public static boolean contains(ByteBuffer buffer, ByteBuffer subBuffer)
+    {
+        int len = subBuffer.remaining();
+        if (buffer.remaining() - len < 0)
+            return false;
+
+        // adapted form the JDK's String.indexOf()
+        byte first = subBuffer.get(subBuffer.position());
+        int max = buffer.position() + (buffer.remaining() - len);
+
+        for (int i = buffer.position(); i <= max; i++)
+        {
+            /* Look for first character. */
+            if (buffer.get(i) != first)
+            {
+                while (++i <= max && buffer.get(i) != first)
+                {}
+            }
+
+            /* (maybe) Found first character, now look at the rest of v2 */
+            if (i <= max)
+            {
+                int j = i + 1;
+                int end = j + len - 1;
+                for (int k = 1 + subBuffer.position(); j < end && buffer.get(j) == subBuffer.get(k); j++, k++)
+                {}
+
+                if (j == end)
+                    return true;
+            }
+        }
+        return false;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/utils/FBUtilities.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index 0bd2a51..ea41ebf 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -844,4 +844,9 @@ public class FBUtilities
             throw new RuntimeException(e);
         }
     }
+
+    public static long align(long val, int boundary)
+    {
+        return (val + boundary) & ~(boundary - 1);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
index 9a8f7e0..7fa01d2 100644
--- a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
@@ -31,7 +31,7 @@ public abstract class MemoryUtil
     private static final long UNSAFE_COPY_THRESHOLD = 1024 * 1024L; // copied from java.nio.Bits
 
     private static final Unsafe unsafe;
-    private static final Class<?> DIRECT_BYTE_BUFFER_CLASS;
+    private static final Class<?> DIRECT_BYTE_BUFFER_CLASS, RO_DIRECT_BYTE_BUFFER_CLASS;
     private static final long DIRECT_BYTE_BUFFER_ADDRESS_OFFSET;
     private static final long DIRECT_BYTE_BUFFER_CAPACITY_OFFSET;
     private static final long DIRECT_BYTE_BUFFER_LIMIT_OFFSET;
@@ -65,6 +65,7 @@ public abstract class MemoryUtil
             DIRECT_BYTE_BUFFER_POSITION_OFFSET = unsafe.objectFieldOffset(Buffer.class.getDeclaredField("position"));
             DIRECT_BYTE_BUFFER_ATTACHMENT_OFFSET = unsafe.objectFieldOffset(clazz.getDeclaredField("att"));
             DIRECT_BYTE_BUFFER_CLASS = clazz;
+            RO_DIRECT_BYTE_BUFFER_CLASS = ByteBuffer.allocateDirect(0).asReadOnlyBuffer().getClass();
 
             clazz = ByteBuffer.allocate(0).getClass();
             BYTE_BUFFER_OFFSET_OFFSET = unsafe.objectFieldOffset(ByteBuffer.class.getDeclaredField("offset"));
@@ -204,7 +205,7 @@ public abstract class MemoryUtil
 
     public static ByteBuffer duplicateDirectByteBuffer(ByteBuffer source, ByteBuffer hollowBuffer)
     {
-        assert source.getClass() == DIRECT_BYTE_BUFFER_CLASS;
+        assert source.getClass() == DIRECT_BYTE_BUFFER_CLASS || source.getClass() == RO_DIRECT_BYTE_BUFFER_CLASS;
         unsafe.putLong(hollowBuffer, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET, unsafe.getLong(source, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET));
         unsafe.putInt(hollowBuffer, DIRECT_BYTE_BUFFER_POSITION_OFFSET, unsafe.getInt(source, DIRECT_BYTE_BUFFER_POSITION_OFFSET));
         unsafe.putInt(hollowBuffer, DIRECT_BYTE_BUFFER_LIMIT_OFFSET, unsafe.getInt(source, DIRECT_BYTE_BUFFER_LIMIT_OFFSET));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/ar_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/ar_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/ar_ST.txt
new file mode 100644
index 0000000..97bedb6
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/ar_ST.txt
@@ -0,0 +1,163 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+ب
+ا
+أ
+،
+عشر
+عدد
+عدة
+عشرة
+عدم
+عام
+عاما
+عن
+عند
+عندما
+على
+عليه
+عليها
+زيارة
+سنة
+سنوات
+تم
+ضد
+بعد
+بعض
+اعادة
+اعلنت
+بسبب
+حتى
+اذا
+احد
+اثر
+برس
+باسم
+غدا
+شخصا
+صباح
+اطار
+اربعة
+اخرى
+بان
+اجل
+غير
+بشكل
+حاليا
+بن
+به
+ثم
+اف
+ان
+او
+اي
+بها
+صفر
+حيث
+اكد
+الا
+اما
+امس
+السابق
+التى
+التي
+اكثر
+ايار
+ايضا
+ثلاثة
+الذاتي
+الاخيرة
+الثاني
+الثانية
+الذى
+الذي
+الان
+امام
+ايام
+خلال
+حوالى
+الذين
+الاول
+الاولى
+بين
+ذلك
+دون
+حول
+حين
+الف
+الى
+انه
+اول
+ضمن
+انها
+جميع
+الماضي
+الوقت
+المقبل
+اليوم
+ـ
+ف
+و
+و6
+قد
+لا
+ما
+مع
+مساء
+هذا
+واحد
+واضاف
+واضافت
+فان
+قبل
+قال
+كان
+لدى
+نحو
+هذه
+وان
+واكد
+كانت
+واوضح
+مايو
+فى
+في
+كل
+لم
+لن
+له
+من
+هو
+هي
+قوة
+كما
+لها
+منذ
+وقد
+ولا
+نفسه
+لقاء
+مقابل
+هناك
+وقال
+وكان
+نهاية
+وقالت
+وكانت
+للامم
+فيه
+كلم
+لكن
+وفي
+وقف
+ولم
+ومن
+وهو
+وهي
+يوم
+فيها
+منها
+مليار
+لوكالة
+يكون
+يمكن
+مليون

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/bg_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/bg_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/bg_ST.txt
new file mode 100644
index 0000000..ed6049d
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/bg_ST.txt
@@ -0,0 +1,260 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+а
+автентичен
+аз
+ако
+ала
+бе
+без
+беше
+би
+бивш
+бивша
+бившо
+бил
+била
+били
+било
+благодаря
+близо
+бъдат
+бъде
+бяха
+в
+вас
+ваш
+ваша
+вероятно
+вече
+взема
+ви
+вие
+винаги
+внимава
+време
+все
+всеки
+всички
+всичко
+всяка
+във
+въпреки
+върху
+г
+ги
+главен
+главна
+главно
+глас
+го
+година
+години
+годишен
+д
+да
+дали
+два
+двама
+двамата
+две
+двете
+ден
+днес
+дни
+до
+добра
+добре
+добро
+добър
+докато
+докога
+дори
+досега
+доста
+друг
+друга
+други
+е
+евтин
+едва
+един
+една
+еднаква
+еднакви
+еднакъв
+едно
+екип
+ето
+живот
+за
+забавям
+зад
+заедно
+заради
+засега
+заспал
+затова
+защо
+защото
+и
+из
+или
+им
+има
+имат
+иска
+й
+каза
+как
+каква
+какво
+както
+какъв
+като
+кога
+когато
+което
+които
+кой
+който
+колко
+която
+къде
+където
+към
+лесен
+лесно
+ли
+лош
+м
+май
+малко
+ме
+между
+мек
+мен
+месец
+ми
+много
+мнозина
+мога
+могат
+може
+мокър
+моля
+момента
+му
+н
+на
+над
+назад
+най
+направи
+напред
+например
+нас
+не
+него
+нещо
+нея
+ни
+ние
+никой
+нито
+нищо
+но
+нов
+нова
+нови
+новина
+някои
+някой
+няколко
+няма
+обаче
+около
+освен
+особено
+от
+отгоре
+отново
+още
+пак
+по
+повече
+повечето
+под
+поне
+поради
+после
+почти
+прави
+пред
+преди
+през
+при
+пък
+първата
+първи
+първо
+пъти
+равен
+равна
+с
+са
+сам
+само
+се
+сега
+си
+син
+скоро
+след
+следващ
+сме
+смях
+според
+сред
+срещу
+сте
+съм
+със
+също
+т
+тази
+така
+такива
+такъв
+там
+твой
+те
+тези
+ти
+т.н.
+то
+това
+тогава
+този
+той
+толкова
+точно
+три
+трябва
+тук
+тъй
+тя
+тях
+у
+утре
+харесва
+хиляди
+ч
+часа
+че
+често
+чрез
+ще
+щом
+юмрук
+я
+як
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/cs_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/cs_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/cs_ST.txt
new file mode 100644
index 0000000..49b52e1
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/cs_ST.txt
@@ -0,0 +1,257 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+ačkoli
+ahoj
+ale
+anebo
+ano
+asi
+aspoň
+během
+bez
+beze
+blízko
+bohužel
+brzo
+bude
+budeme
+budeš
+budete
+budou
+budu
+byl
+byla
+byli
+bylo
+byly
+bys
+čau
+chce
+chceme
+chceš
+chcete
+chci
+chtějí
+chtít
+chut'
+chuti
+co
+čtrnáct
+čtyři
+dál
+dále
+daleko
+děkovat
+děkujeme
+děkuji
+den
+deset
+devatenáct
+devět
+do
+dobrý
+docela
+dva
+dvacet
+dvanáct
+dvě
+hodně
+já
+jak
+jde
+je
+jeden
+jedenáct
+jedna
+jedno
+jednou
+jedou
+jeho
+její
+jejich
+jemu
+jen
+jenom
+ještě
+jestli
+jestliže
+jí
+jich
+jím
+jimi
+jinak
+jsem
+jsi
+jsme
+jsou
+jste
+kam
+kde
+kdo
+kdy
+když
+ke
+kolik
+kromě
+která
+které
+kteří
+který
+kvůli
+má
+mají
+málo
+mám
+máme
+máš
+máte
+mé
+mě
+mezi
+mí
+mít
+mně
+mnou
+moc
+mohl
+mohou
+moje
+moji
+možná
+můj
+musí
+může
+my
+na
+nad
+nade
+nám
+námi
+naproti
+nás
+náš
+naše
+naši
+ne
+ně
+nebo
+nebyl
+nebyla
+nebyli
+nebyly
+něco
+nedělá
+nedělají
+nedělám
+neděláme
+neděláš
+neděláte
+nějak
+nejsi
+někde
+někdo
+nemají
+nemáme
+nemáte
+neměl
+němu
+není
+nestačí
+nevadí
+než
+nic
+nich
+ním
+nimi
+nula
+od
+ode
+on
+ona
+oni
+ono
+ony
+osm
+osmnáct
+pak
+patnáct
+pět
+po
+pořád
+potom
+pozdě
+před
+přes
+přese
+pro
+proč
+prosím
+prostě
+proti
+protože
+rovně
+se
+sedm
+sedmnáct
+šest
+šestnáct
+skoro
+smějí
+smí
+snad
+spolu
+sta
+sté
+sto
+ta
+tady
+tak
+takhle
+taky
+tam
+tamhle
+tamhleto
+tamto
+tě
+tebe
+tebou
+ted'
+tedy
+ten
+ti
+tisíc
+tisíce
+to
+tobě
+tohle
+toto
+třeba
+tři
+třináct
+trošku
+tvá
+tvé
+tvoje
+tvůj
+ty
+určitě
+už
+vám
+vámi
+vás
+váš
+vaše
+vaši
+ve
+večer
+vedle
+vlastně
+všechno
+všichni
+vůbec
+vy
+vždy
+za
+zač
+zatímco
+ze
+že

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/de_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/de_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/de_ST.txt
new file mode 100644
index 0000000..747e682
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/de_ST.txt
@@ -0,0 +1,604 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+a
+ab
+aber
+aber
+ach
+acht
+achte
+achten
+achter
+achtes
+ag
+alle
+allein
+allem
+allen
+aller
+allerdings
+alles
+allgemeinen
+als
+als
+also
+am
+an
+andere
+anderen
+andern
+anders
+au
+auch
+auch
+auf
+aus
+ausser
+au�er
+ausserdem
+au�erdem
+b
+bald
+bei
+beide
+beiden
+beim
+beispiel
+bekannt
+bereits
+besonders
+besser
+besten
+bin
+bis
+bisher
+bist
+c
+d
+da
+dabei
+dadurch
+daf�r
+dagegen
+daher
+dahin
+dahinter
+damals
+damit
+danach
+daneben
+dank
+dann
+daran
+darauf
+daraus
+darf
+darfst
+darin
+dar�ber
+darum
+darunter
+das
+das
+dasein
+daselbst
+dass
+da�
+dasselbe
+davon
+davor
+dazu
+dazwischen
+dein
+deine
+deinem
+deiner
+dem
+dementsprechend
+demgegen�ber
+demgem�ss
+demgem��
+demselben
+demzufolge
+den
+denen
+denn
+denn
+denselben
+der
+deren
+derjenige
+derjenigen
+dermassen
+derma�en
+derselbe
+derselben
+des
+deshalb
+desselben
+dessen
+deswegen
+d.h
+dich
+die
+diejenige
+diejenigen
+dies
+diese
+dieselbe
+dieselben
+diesem
+diesen
+dieser
+dieses
+dir
+doch
+dort
+drei
+drin
+dritte
+dritten
+dritter
+drittes
+du
+durch
+durchaus
+d�rfen
+d�rft
+durfte
+durften
+e
+eben
+ebenso
+ehrlich
+ei
+ei,
+ei,
+eigen
+eigene
+eigenen
+eigener
+eigenes
+ein
+einander
+eine
+einem
+einen
+einer
+eines
+einige
+einigen
+einiger
+einiges
+einmal
+einmal
+eins
+elf
+en
+ende
+endlich
+entweder
+entweder
+er
+Ernst
+erst
+erste
+ersten
+erster
+erstes
+es
+etwa
+etwas
+euch
+f
+fr�her
+f�nf
+f�nfte
+f�nften
+f�nfter
+f�nftes
+f�r
+g
+gab
+ganz
+ganze
+ganzen
+ganzer
+ganzes
+gar
+gedurft
+gegen
+gegen�ber
+gehabt
+gehen
+geht
+gekannt
+gekonnt
+gemacht
+gemocht
+gemusst
+genug
+gerade
+gern
+gesagt
+gesagt
+geschweige
+gewesen
+gewollt
+geworden
+gibt
+ging
+gleich
+gott
+gross
+gro�
+grosse
+gro�e
+grossen
+gro�en
+grosser
+gro�er
+grosses
+gro�es
+gut
+gute
+guter
+gutes
+h
+habe
+haben
+habt
+hast
+hat
+hatte
+h�tte
+hatten
+h�tten
+heisst
+her
+heute
+hier
+hin
+hinter
+hoch
+i
+ich
+ihm
+ihn
+ihnen
+ihr
+ihre
+ihrem
+ihren
+ihrer
+ihres
+im
+im
+immer
+in
+in
+indem
+infolgedessen
+ins
+irgend
+ist
+j
+ja
+ja
+jahr
+jahre
+jahren
+je
+jede
+jedem
+jeden
+jeder
+jedermann
+jedermanns
+jedoch
+jemand
+jemandem
+jemanden
+jene
+jenem
+jenen
+jener
+jenes
+jetzt
+k
+kam
+kann
+kannst
+kaum
+kein
+keine
+keinem
+keinen
+keiner
+kleine
+kleinen
+kleiner
+kleines
+kommen
+kommt
+k�nnen
+k�nnt
+konnte
+k�nnte
+konnten
+kurz
+l
+lang
+lange
+lange
+leicht
+leide
+lieber
+los
+m
+machen
+macht
+machte
+mag
+magst
+mahn
+man
+manche
+manchem
+manchen
+mancher
+manches
+mann
+mehr
+mein
+meine
+meinem
+meinen
+meiner
+meines
+mensch
+menschen
+mich
+mir
+mit
+mittel
+mochte
+m�chte
+mochten
+m�gen
+m�glich
+m�gt
+morgen
+muss
+mu�
+m�ssen
+musst
+m�sst
+musste
+mussten
+n
+na
+nach
+nachdem
+nahm
+nat�rlich
+neben
+nein
+neue
+neuen
+neun
+neunte
+neunten
+neunter
+neuntes
+nicht
+nicht
+nichts
+nie
+niemand
+niemandem
+niemanden
+noch
+nun
+nun
+nur
+o
+ob
+ob
+oben
+oder
+oder
+offen
+oft
+oft
+ohne
+Ordnung
+p
+q
+r
+recht
+rechte
+rechten
+rechter
+rechtes
+richtig
+rund
+s
+sa
+sache
+sagt
+sagte
+sah
+satt
+schlecht
+Schluss
+schon
+sechs
+sechste
+sechsten
+sechster
+sechstes
+sehr
+sei
+sei
+seid
+seien
+sein
+seine
+seinem
+seinen
+seiner
+seines
+seit
+seitdem
+selbst
+selbst
+sich
+sie
+sieben
+siebente
+siebenten
+siebenter
+siebentes
+sind
+so
+solang
+solche
+solchem
+solchen
+solcher
+solches
+soll
+sollen
+sollte
+sollten
+sondern
+sonst
+sowie
+sp�ter
+statt
+t
+tag
+tage
+tagen
+tat
+teil
+tel
+tritt
+trotzdem
+tun
+u
+�ber
+�berhaupt
+�brigens
+uhr
+um
+und
+und?
+uns
+unser
+unsere
+unserer
+unter
+v
+vergangenen
+viel
+viele
+vielem
+vielen
+vielleicht
+vier
+vierte
+vierten
+vierter
+viertes
+vom
+von
+vor
+w
+wahr?
+w�hrend
+w�hrenddem
+w�hrenddessen
+wann
+war
+w�re
+waren
+wart
+warum
+was
+wegen
+weil
+weit
+weiter
+weitere
+weiteren
+weiteres
+welche
+welchem
+welchen
+welcher
+welches
+wem
+wen
+wenig
+wenig
+wenige
+weniger
+weniges
+wenigstens
+wenn
+wenn
+wer
+werde
+werden
+werdet
+wessen
+wie
+wie
+wieder
+will
+willst
+wir
+wird
+wirklich
+wirst
+wo
+wohl
+wollen
+wollt
+wollte
+wollten
+worden
+wurde
+w�rde
+wurden
+w�rden
+x
+y
+z
+z.b
+zehn
+zehnte
+zehnten
+zehnter
+zehntes
+zeit
+zu
+zuerst
+zugleich
+zum
+zum
+zun�chst
+zur
+zur�ck
+zusammen
+zwanzig
+zwar
+zwar
+zwei
+zweite
+zweiten
+zweiter
+zweites
+zwischen
+zw�lf

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/en_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/en_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/en_ST.txt
new file mode 100644
index 0000000..d30da31
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/en_ST.txt
@@ -0,0 +1,572 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+a
+a's
+able
+about
+above
+according
+accordingly
+across
+actually
+after
+afterwards
+again
+against
+ain't
+all
+allow
+allows
+almost
+alone
+along
+already
+also
+although
+always
+am
+among
+amongst
+an
+and
+another
+any
+anybody
+anyhow
+anyone
+anything
+anyway
+anyways
+anywhere
+apart
+appear
+appreciate
+appropriate
+are
+aren't
+around
+as
+aside
+ask
+asking
+associated
+at
+available
+away
+awfully
+b
+be
+became
+because
+become
+becomes
+becoming
+been
+before
+beforehand
+behind
+being
+believe
+below
+beside
+besides
+best
+better
+between
+beyond
+both
+brief
+but
+by
+c
+c'mon
+c's
+came
+can
+can't
+cannot
+cant
+cause
+causes
+certain
+certainly
+changes
+clearly
+co
+com
+come
+comes
+concerning
+consequently
+consider
+considering
+contain
+containing
+contains
+corresponding
+could
+couldn't
+course
+currently
+d
+definitely
+described
+despite
+did
+didn't
+different
+do
+does
+doesn't
+doing
+don't
+done
+down
+downwards
+during
+e
+each
+edu
+eg
+eight
+either
+else
+elsewhere
+enough
+entirely
+especially
+et
+etc
+even
+ever
+every
+everybody
+everyone
+everything
+everywhere
+ex
+exactly
+example
+except
+f
+far
+few
+fifth
+first
+five
+followed
+following
+follows
+for
+former
+formerly
+forth
+four
+from
+further
+furthermore
+g
+get
+gets
+getting
+given
+gives
+go
+goes
+going
+gone
+got
+gotten
+greetings
+h
+had
+hadn't
+happens
+hardly
+has
+hasn't
+have
+haven't
+having
+he
+he's
+hello
+help
+hence
+her
+here
+here's
+hereafter
+hereby
+herein
+hereupon
+hers
+herself
+hi
+him
+himself
+his
+hither
+hopefully
+how
+howbeit
+however
+i
+i'd
+i'll
+i'm
+i've
+ie
+if
+ignored
+immediate
+in
+inasmuch
+inc
+indeed
+indicate
+indicated
+indicates
+inner
+insofar
+instead
+into
+inward
+is
+isn't
+it
+it'd
+it'll
+it's
+its
+itself
+j
+just
+k
+keep
+keeps
+kept
+know
+knows
+known
+l
+last
+lately
+later
+latter
+latterly
+least
+less
+lest
+let
+let's
+like
+liked
+likely
+little
+look
+looking
+looks
+ltd
+m
+mainly
+many
+may
+maybe
+me
+mean
+meanwhile
+merely
+might
+more
+moreover
+most
+mostly
+much
+must
+my
+myself
+n
+name
+namely
+nd
+near
+nearly
+necessary
+need
+needs
+neither
+never
+nevertheless
+new
+next
+nine
+no
+nobody
+non
+none
+noone
+nor
+normally
+not
+nothing
+novel
+now
+nowhere
+o
+obviously
+of
+off
+often
+oh
+ok
+okay
+old
+on
+once
+one
+ones
+only
+onto
+or
+other
+others
+otherwise
+ought
+our
+ours
+ourselves
+out
+outside
+over
+overall
+own
+p
+particular
+particularly
+per
+perhaps
+placed
+please
+plus
+possible
+presumably
+probably
+provides
+q
+que
+quite
+qv
+r
+rather
+rd
+re
+really
+reasonably
+regarding
+regardless
+regards
+relatively
+respectively
+right
+s
+said
+same
+saw
+say
+saying
+says
+second
+secondly
+see
+seeing
+seem
+seemed
+seeming
+seems
+seen
+self
+selves
+sensible
+sent
+serious
+seriously
+seven
+several
+shall
+she
+should
+shouldn't
+since
+six
+so
+some
+somebody
+somehow
+someone
+something
+sometime
+sometimes
+somewhat
+somewhere
+soon
+sorry
+specified
+specify
+specifying
+still
+sub
+such
+sup
+sure
+t
+t's
+take
+taken
+tell
+tends
+th
+than
+thank
+thanks
+thanx
+that
+that's
+thats
+the
+their
+theirs
+them
+themselves
+then
+thence
+there
+there's
+thereafter
+thereby
+therefore
+therein
+theres
+thereupon
+these
+they
+they'd
+they'll
+they're
+they've
+think
+third
+this
+thorough
+thoroughly
+those
+though
+three
+through
+throughout
+thru
+thus
+to
+together
+too
+took
+toward
+towards
+tried
+tries
+truly
+try
+trying
+twice
+two
+u
+un
+under
+unfortunately
+unless
+unlikely
+until
+unto
+up
+upon
+us
+use
+used
+useful
+uses
+using
+usually
+uucp
+v
+value
+various
+very
+via
+viz
+vs
+w
+want
+wants
+was
+wasn't
+way
+we
+we'd
+we'll
+we're
+we've
+welcome
+well
+went
+were
+weren't
+what
+what's
+whatever
+when
+whence
+whenever
+where
+where's
+whereafter
+whereas
+whereby
+wherein
+whereupon
+wherever
+whether
+which
+while
+whither
+who
+who's
+whoever
+whole
+whom
+whose
+why
+will
+willing
+wish
+with
+within
+without
+won't
+wonder
+would
+would
+wouldn't
+x
+y
+yes
+yet
+you
+you'd
+you'll
+you're
+you've
+your
+yours
+yourself
+yourselves
+z
+zero

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/es_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/es_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/es_ST.txt
new file mode 100644
index 0000000..75e2086
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/es_ST.txt
@@ -0,0 +1,308 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+a
+acuerdo
+adelante
+ademas
+adem�s
+adrede
+ahi
+ah�
+ahora
+al
+alli
+all�
+alrededor
+antano
+anta�o
+ante
+antes
+apenas
+aproximadamente
+aquel
+aqu�l
+aquella
+aqu�lla
+aquellas
+aqu�llas
+aquello
+aquellos
+aqu�llos
+aqui
+aqu�
+arribaabajo
+asi
+as�
+aun
+a�n
+aunque
+b
+bajo
+bastante
+bien
+breve
+c
+casi
+cerca
+claro
+como
+c�mo
+con
+conmigo
+contigo
+contra
+cual
+cu�l
+cuales
+cu�les
+cuando
+cu�ndo
+cuanta
+cu�nta
+cuantas
+cu�ntas
+cuanto
+cu�nto
+cuantos
+cu�ntos
+d
+de
+debajo
+del
+delante
+demasiado
+dentro
+deprisa
+desde
+despacio
+despues
+despu�s
+detras
+detr�s
+dia
+d�a
+dias
+d�as
+donde
+d�nde
+dos
+durante
+e
+el
+�l
+ella
+ellas
+ellos
+en
+encima
+enfrente
+enseguida
+entre
+es
+esa
+�sa
+esas
+�sas
+ese
+�se
+eso
+esos
+�sos
+esta
+est�
+�sta
+estado
+estados
+estan
+est�n
+estar
+estas
+�stas
+este
+�ste
+esto
+estos
+�stos
+ex
+excepto
+f
+final
+fue
+fuera
+fueron
+g
+general
+gran
+h
+ha
+habia
+hab�a
+habla
+hablan
+hace
+hacia
+han
+hasta
+hay
+horas
+hoy
+i
+incluso
+informo
+inform�
+j
+junto
+k
+l
+la
+lado
+las
+le
+lejos
+lo
+los
+luego
+m
+mal
+mas
+m�s
+mayor
+me
+medio
+mejor
+menos
+menudo
+mi
+m�
+mia
+m�a
+mias
+m�as
+mientras
+mio
+m�o
+mios
+m�os
+mis
+mismo
+mucho
+muy
+n
+nada
+nadie
+ninguna
+no
+nos
+nosotras
+nosotros
+nuestra
+nuestras
+nuestro
+nuestros
+nueva
+nuevo
+nunca
+o
+os
+otra
+otros
+p
+pais
+pa�s
+para
+parte
+pasado
+peor
+pero
+poco
+por
+porque
+pronto
+proximo
+pr�ximo
+puede
+q
+qeu
+que
+qu�
+quien
+qui�n
+quienes
+qui�nes
+quiza
+quiz�
+quizas
+quiz�s
+r
+raras
+repente
+s
+salvo
+se
+s�
+segun
+seg�n
+ser
+sera
+ser�
+si
+s�
+sido
+siempre
+sin
+sobre
+solamente
+solo
+s�lo
+son
+soyos
+su
+supuesto
+sus
+suya
+suyas
+suyo
+t
+tal
+tambien
+tambi�n
+tampoco
+tarde
+te
+temprano
+ti
+tiene
+todavia
+todav�a
+todo
+todos
+tras
+tu
+t�
+tus
+tuya
+tuyas
+tuyo
+tuyos
+u
+un
+una
+unas
+uno
+unos
+usted
+ustedes
+v
+veces
+vez
+vosotras
+vosotros
+vuestra
+vuestras
+vuestro
+vuestros
+w
+x
+y
+ya
+yo
+z