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:03 UTC

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

Repository: cassandra
Updated Branches:
  refs/heads/trunk 11c8ca6b5 -> 72790dc8e


http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java b/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java
new file mode 100644
index 0000000..92fbf69
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java
@@ -0,0 +1,645 @@
+/*
+ * 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.index.sasi.plan;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.RowFilter;
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.index.sasi.plan.Operation.OperationType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Tables;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.utils.FBUtilities;
+
+import org.junit.*;
+
+public class OperationTest extends SchemaLoader
+{
+    private static final String KS_NAME = "sasi";
+    private static final String CF_NAME = "test_cf";
+    private static final String CLUSTERING_CF_NAME = "clustering_test_cf";
+
+    private static ColumnFamilyStore BACKEND;
+    private static ColumnFamilyStore CLUSTERING_BACKEND;
+
+    @BeforeClass
+    public static void loadSchema() throws ConfigurationException
+    {
+        System.setProperty("cassandra.config", "cassandra-murmur.yaml");
+        SchemaLoader.loadSchema();
+        MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(KS_NAME,
+                                                                     KeyspaceParams.simpleTransient(1),
+                                                                     Tables.of(SchemaLoader.sasiCFMD(KS_NAME, CF_NAME),
+                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME))));
+
+        BACKEND = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+        CLUSTERING_BACKEND = Keyspace.open(KS_NAME).getColumnFamilyStore(CLUSTERING_CF_NAME);
+    }
+
+    private QueryController controller;
+
+    @Before
+    public void beforeTest()
+    {
+        controller = new QueryController(BACKEND,
+                                         PartitionRangeReadCommand.allDataRead(BACKEND.metadata, FBUtilities.nowInSeconds()),
+                                         TimeUnit.SECONDS.toMillis(10));
+    }
+
+    @After
+    public void afterTest()
+    {
+        controller.finish();
+    }
+
+    @Test
+    public void testAnalyze() throws Exception
+    {
+        final ColumnDefinition firstName = getColumn(UTF8Type.instance.decompose("first_name"));
+        final ColumnDefinition age = getColumn(UTF8Type.instance.decompose("age"));
+        final ColumnDefinition comment = getColumn(UTF8Type.instance.decompose("comment"));
+
+        // age != 5 AND age > 1 AND age != 6 AND age <= 10
+        Map<Expression.Op, Expression> expressions = convert(Operation.analyzeGroup(controller, OperationType.AND,
+                                                                                Arrays.asList(new SimpleExpression(age, Operator.NEQ, Int32Type.instance.decompose(5)),
+                                                                                              new SimpleExpression(age, Operator.GT, Int32Type.instance.decompose(1)),
+                                                                                              new SimpleExpression(age, Operator.NEQ, Int32Type.instance.decompose(6)),
+                                                                                              new SimpleExpression(age, Operator.LTE, Int32Type.instance.decompose(10)))));
+
+        Expression expected = new Expression("age", Int32Type.instance)
+        {{
+            operation = Op.RANGE;
+            lower = new Bound(Int32Type.instance.decompose(1), false);
+            upper = new Bound(Int32Type.instance.decompose(10), true);
+
+            exclusions.add(Int32Type.instance.decompose(5));
+            exclusions.add(Int32Type.instance.decompose(6));
+        }};
+
+        Assert.assertEquals(1, expressions.size());
+        Assert.assertEquals(expected, expressions.get(Expression.Op.RANGE));
+
+        // age != 5 OR age >= 7
+        expressions = convert(Operation.analyzeGroup(controller, OperationType.OR,
+                                                    Arrays.asList(new SimpleExpression(age, Operator.NEQ, Int32Type.instance.decompose(5)),
+                                                                  new SimpleExpression(age, Operator.GTE, Int32Type.instance.decompose(7)))));
+        Assert.assertEquals(2, expressions.size());
+
+        Assert.assertEquals(new Expression("age", Int32Type.instance)
+                            {{
+                                    operation = Op.NOT_EQ;
+                                    lower = new Bound(Int32Type.instance.decompose(5), true);
+                                    upper = lower;
+                            }}, expressions.get(Expression.Op.NOT_EQ));
+
+        Assert.assertEquals(new Expression("age", Int32Type.instance)
+                            {{
+                                    operation = Op.RANGE;
+                                    lower = new Bound(Int32Type.instance.decompose(7), true);
+                            }}, expressions.get(Expression.Op.RANGE));
+
+        // age != 5 OR age < 7
+        expressions = convert(Operation.analyzeGroup(controller, OperationType.OR,
+                                                    Arrays.asList(new SimpleExpression(age, Operator.NEQ, Int32Type.instance.decompose(5)),
+                                                                  new SimpleExpression(age, Operator.LT, Int32Type.instance.decompose(7)))));
+
+        Assert.assertEquals(2, expressions.size());
+        Assert.assertEquals(new Expression("age", Int32Type.instance)
+                            {{
+                                    operation = Op.RANGE;
+                                    upper = new Bound(Int32Type.instance.decompose(7), false);
+                            }}, expressions.get(Expression.Op.RANGE));
+        Assert.assertEquals(new Expression("age", Int32Type.instance)
+                            {{
+                                    operation = Op.NOT_EQ;
+                                    lower = new Bound(Int32Type.instance.decompose(5), true);
+                                    upper = lower;
+                            }}, expressions.get(Expression.Op.NOT_EQ));
+
+        // age > 1 AND age < 7
+        expressions = convert(Operation.analyzeGroup(controller, OperationType.AND,
+                                                    Arrays.asList(new SimpleExpression(age, Operator.GT, Int32Type.instance.decompose(1)),
+                                                                  new SimpleExpression(age, Operator.LT, Int32Type.instance.decompose(7)))));
+
+        Assert.assertEquals(1, expressions.size());
+        Assert.assertEquals(new Expression("age", Int32Type.instance)
+                            {{
+                                    operation = Op.RANGE;
+                                    lower = new Bound(Int32Type.instance.decompose(1), false);
+                                    upper = new Bound(Int32Type.instance.decompose(7), false);
+                            }}, expressions.get(Expression.Op.RANGE));
+
+        // first_name = 'a' OR first_name != 'b'
+        expressions = convert(Operation.analyzeGroup(controller, OperationType.OR,
+                                                    Arrays.asList(new SimpleExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                                                                  new SimpleExpression(firstName, Operator.NEQ, UTF8Type.instance.decompose("b")))));
+
+        Assert.assertEquals(2, expressions.size());
+        Assert.assertEquals(new Expression("first_name", UTF8Type.instance)
+                            {{
+                                    operation = Op.NOT_EQ;
+                                    lower = new Bound(UTF8Type.instance.decompose("b"), true);
+                                    upper = lower;
+                            }}, expressions.get(Expression.Op.NOT_EQ));
+        Assert.assertEquals(new Expression("first_name", UTF8Type.instance)
+                            {{
+                                    operation = Op.EQ;
+                                    lower = upper = new Bound(UTF8Type.instance.decompose("a"), true);
+                            }}, expressions.get(Expression.Op.EQ));
+
+        // comment = 'soft eng' and comment != 'likes do'
+        ListMultimap<ColumnDefinition, Expression> e = Operation.analyzeGroup(controller, OperationType.OR,
+                                                    Arrays.asList(new SimpleExpression(comment, Operator.EQ, UTF8Type.instance.decompose("soft eng")),
+                                                                  new SimpleExpression(comment, Operator.NEQ, UTF8Type.instance.decompose("likes do"))));
+
+        List<Expression> expectedExpressions = new ArrayList<Expression>(2)
+        {{
+                add(new Expression("comment", UTF8Type.instance)
+                {{
+                        operation = Op.EQ;
+                        lower = new Bound(UTF8Type.instance.decompose("soft"), true);
+                        upper = lower;
+                }});
+
+                add(new Expression("comment", UTF8Type.instance)
+                {{
+                        operation = Op.EQ;
+                        lower = new Bound(UTF8Type.instance.decompose("eng"), true);
+                        upper = lower;
+                }});
+
+                add(new Expression("comment", UTF8Type.instance)
+                {{
+                        operation = Op.NOT_EQ;
+                        lower = new Bound(UTF8Type.instance.decompose("likes"), true);
+                        upper = lower;
+                }});
+
+                add(new Expression("comment", UTF8Type.instance)
+                {{
+                        operation = Op.NOT_EQ;
+                        lower = new Bound(UTF8Type.instance.decompose("do"), true);
+                        upper = lower;
+                }});
+        }};
+
+        Assert.assertEquals(expectedExpressions, e.get(comment));
+
+        // first_name = 'j' and comment != 'likes do'
+        e = Operation.analyzeGroup(controller, OperationType.OR,
+                        Arrays.asList(new SimpleExpression(comment, Operator.NEQ, UTF8Type.instance.decompose("likes do")),
+                                      new SimpleExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("j"))));
+
+        expectedExpressions = new ArrayList<Expression>(2)
+        {{
+                add(new Expression("comment", UTF8Type.instance)
+                {{
+                        operation = Op.NOT_EQ;
+                        lower = new Bound(UTF8Type.instance.decompose("likes"), true);
+                        upper = lower;
+                }});
+
+                add(new Expression("comment", UTF8Type.instance)
+                {{
+                        operation = Op.NOT_EQ;
+                        lower = new Bound(UTF8Type.instance.decompose("do"), true);
+                        upper = lower;
+                }});
+        }};
+
+        Assert.assertEquals(expectedExpressions, e.get(comment));
+
+        // age != 27 first_name = 'j' and age != 25
+        e = Operation.analyzeGroup(controller, OperationType.OR,
+                        Arrays.asList(new SimpleExpression(age, Operator.NEQ, Int32Type.instance.decompose(27)),
+                                      new SimpleExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("j")),
+                                      new SimpleExpression(age, Operator.NEQ, Int32Type.instance.decompose(25))));
+
+        expectedExpressions = new ArrayList<Expression>(2)
+        {{
+                add(new Expression("age", Int32Type.instance)
+                {{
+                        operation = Op.NOT_EQ;
+                        lower = new Bound(Int32Type.instance.decompose(27), true);
+                        upper = lower;
+                }});
+
+                add(new Expression("age", Int32Type.instance)
+                {{
+                        operation = Op.NOT_EQ;
+                        lower = new Bound(Int32Type.instance.decompose(25), true);
+                        upper = lower;
+                }});
+        }};
+
+        Assert.assertEquals(expectedExpressions, e.get(age));
+    }
+
+    @Test
+    public void testSatisfiedBy() throws Exception
+    {
+        final ColumnDefinition timestamp = getColumn(UTF8Type.instance.decompose("timestamp"));
+        final ColumnDefinition age = getColumn(UTF8Type.instance.decompose("age"));
+
+        Operation.Builder builder = new Operation.Builder(OperationType.AND, controller, new SimpleExpression(age, Operator.NEQ, Int32Type.instance.decompose(5)));
+        Operation op = builder.complete();
+
+        Unfiltered row = buildRow(buildCell(age, Int32Type.instance.decompose(6), System.currentTimeMillis()));
+
+        Assert.assertTrue(op.satisfiedBy(row, false));
+
+        row = buildRow(buildCell(age, Int32Type.instance.decompose(5), System.currentTimeMillis()));
+
+        // and reject incorrect value
+        Assert.assertFalse(op.satisfiedBy(row, false));
+
+        row = buildRow(buildCell(age, Int32Type.instance.decompose(6), System.currentTimeMillis()));
+
+        Assert.assertTrue(op.satisfiedBy(row, false));
+
+        // range with exclusions - age != 5 AND age > 1 AND age != 6 AND age <= 10
+        builder = new Operation.Builder(OperationType.AND, controller,
+                                        new SimpleExpression(age, Operator.NEQ, Int32Type.instance.decompose(5)),
+                                        new SimpleExpression(age, Operator.GT, Int32Type.instance.decompose(1)),
+                                        new SimpleExpression(age, Operator.NEQ, Int32Type.instance.decompose(6)),
+                                        new SimpleExpression(age, Operator.LTE, Int32Type.instance.decompose(10)));
+        op = builder.complete();
+
+        Set<Integer> exclusions = Sets.newHashSet(0, 1, 5, 6, 11);
+        for (int i = 0; i <= 11; i++)
+        {
+            row = buildRow(buildCell(age, Int32Type.instance.decompose(i), System.currentTimeMillis()));
+
+            boolean result = op.satisfiedBy(row, false);
+            Assert.assertTrue(exclusions.contains(i) != result);
+        }
+
+        // now let's do something more complex - age = 5 OR age = 6
+        builder = new Operation.Builder(OperationType.OR, controller,
+                                        new SimpleExpression(age, Operator.EQ, Int32Type.instance.decompose(5)),
+                                        new SimpleExpression(age, Operator.EQ, Int32Type.instance.decompose(6)));
+
+        op = builder.complete();
+
+        exclusions = Sets.newHashSet(0, 1, 2, 3, 4, 7, 8, 9, 10);
+        for (int i = 0; i <= 10; i++)
+        {
+            row = buildRow(buildCell(age, Int32Type.instance.decompose(i), System.currentTimeMillis()));
+
+            boolean result = op.satisfiedBy(row, false);
+            Assert.assertTrue(exclusions.contains(i) != result);
+        }
+
+        // now let's test aggregated AND commands
+        builder = new Operation.Builder(OperationType.AND, controller);
+
+        // logical should be ignored by analyzer, but we still what to make sure that it is
+        //IndexExpression logical = new IndexExpression(ByteBufferUtil.EMPTY_BYTE_BUFFER, IndexOperator.EQ, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+        //logical.setLogicalOp(LogicalIndexOperator.AND);
+
+        //builder.add(logical);
+        builder.add(new SimpleExpression(age, Operator.GTE, Int32Type.instance.decompose(0)));
+        builder.add(new SimpleExpression(age, Operator.LT, Int32Type.instance.decompose(10)));
+        builder.add(new SimpleExpression(age, Operator.NEQ, Int32Type.instance.decompose(7)));
+
+        op = builder.complete();
+
+        exclusions = Sets.newHashSet(7);
+        for (int i = 0; i < 10; i++)
+        {
+            row = buildRow(buildCell(age, Int32Type.instance.decompose(i), System.currentTimeMillis()));
+
+            boolean result = op.satisfiedBy(row, false);
+            Assert.assertTrue(exclusions.contains(i) != result);
+        }
+
+        // multiple analyzed expressions in the Operation timestamp >= 10 AND age = 5
+        builder = new Operation.Builder(OperationType.AND, controller);
+        builder.add(new SimpleExpression(timestamp, Operator.GTE, LongType.instance.decompose(10L)));
+        builder.add(new SimpleExpression(age, Operator.EQ, Int32Type.instance.decompose(5)));
+
+        op = builder.complete();
+
+        row = buildRow(buildCell(age, Int32Type.instance.decompose(6), System.currentTimeMillis()),
+                                  buildCell(timestamp, LongType.instance.decompose(11L), System.currentTimeMillis()));
+
+        Assert.assertFalse(op.satisfiedBy(row, false));
+
+        row = buildRow(buildCell(age, Int32Type.instance.decompose(5), System.currentTimeMillis()),
+                                  buildCell(timestamp, LongType.instance.decompose(22L), System.currentTimeMillis()));
+
+        Assert.assertTrue(op.satisfiedBy(row, false));
+
+        row = buildRow(buildCell(age, Int32Type.instance.decompose(5), System.currentTimeMillis()),
+                                  buildCell(timestamp, LongType.instance.decompose(9L), System.currentTimeMillis()));
+
+        Assert.assertFalse(op.satisfiedBy(row, false));
+
+        // operation with internal expressions and right child
+        builder = new Operation.Builder(OperationType.OR, controller,
+                                        new SimpleExpression(timestamp, Operator.GT, LongType.instance.decompose(10L)));
+        builder.setRight(new Operation.Builder(OperationType.AND, controller,
+                                               new SimpleExpression(age, Operator.GT, Int32Type.instance.decompose(0)),
+                                               new SimpleExpression(age, Operator.LT, Int32Type.instance.decompose(10))));
+        op = builder.complete();
+
+        row = buildRow(buildCell(age, Int32Type.instance.decompose(5), System.currentTimeMillis()),
+                                  buildCell(timestamp, LongType.instance.decompose(9L), System.currentTimeMillis()));
+
+        Assert.assertTrue(op.satisfiedBy(row, false));
+
+        row = buildRow(buildCell(age, Int32Type.instance.decompose(20), System.currentTimeMillis()),
+                                  buildCell(timestamp, LongType.instance.decompose(11L), System.currentTimeMillis()));
+
+        Assert.assertTrue(op.satisfiedBy(row, false));
+
+        row = buildRow(buildCell(age, Int32Type.instance.decompose(0), System.currentTimeMillis()),
+                                  buildCell(timestamp, LongType.instance.decompose(9L), System.currentTimeMillis()));
+
+        Assert.assertFalse(op.satisfiedBy(row, false));
+
+        // and for desert let's try out null and deleted rows etc.
+        builder = new Operation.Builder(OperationType.AND, controller);
+        builder.add(new SimpleExpression(age, Operator.EQ, Int32Type.instance.decompose(30)));
+        op = builder.complete();
+
+        Assert.assertFalse(op.satisfiedBy(null, false));
+        Assert.assertFalse(op.satisfiedBy(row, false));
+
+        long now = System.currentTimeMillis();
+
+        row = OperationTest.buildRow(
+                Row.Deletion.regular(new DeletionTime(now - 10, (int) (now / 1000))),
+                          buildCell(age, Int32Type.instance.decompose(6), System.currentTimeMillis()));
+
+        Assert.assertFalse(op.satisfiedBy(row, false));
+
+        row = buildRow(deletedCell(age, System.currentTimeMillis(), FBUtilities.nowInSeconds()));
+
+        Assert.assertFalse(op.satisfiedBy(row, true));
+
+        try
+        {
+            Assert.assertFalse(op.satisfiedBy(buildRow(), false));
+        }
+        catch (IllegalStateException e)
+        {
+            // expected
+        }
+
+        try
+        {
+            Assert.assertFalse(op.satisfiedBy(buildRow(), true));
+        }
+        catch (IllegalStateException e)
+        {
+            Assert.fail("IllegalStateException should not be thrown when missing column and allowMissingColumns=true");
+        }
+    }
+
+    @Test
+    public void testAnalyzeNotIndexedButDefinedColumn() throws Exception
+    {
+        final ColumnDefinition firstName = getColumn(UTF8Type.instance.decompose("first_name"));
+        final ColumnDefinition height = getColumn(UTF8Type.instance.decompose("height"));
+
+        // first_name = 'a' AND height != 10
+        Map<Expression.Op, Expression> expressions;
+        expressions = convert(Operation.analyzeGroup(controller, OperationType.AND,
+                Arrays.asList(new SimpleExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                              new SimpleExpression(height, Operator.NEQ, Int32Type.instance.decompose(5)))));
+
+        Assert.assertEquals(2, expressions.size());
+
+        Assert.assertEquals(new Expression("height", Int32Type.instance)
+        {{
+                operation = Op.NOT_EQ;
+                lower = new Bound(Int32Type.instance.decompose(5), true);
+                upper = lower;
+        }}, expressions.get(Expression.Op.NOT_EQ));
+
+        expressions = convert(Operation.analyzeGroup(controller, OperationType.AND,
+                Arrays.asList(new SimpleExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                              new SimpleExpression(height, Operator.GT, Int32Type.instance.decompose(0)),
+                              new SimpleExpression(height, Operator.NEQ, Int32Type.instance.decompose(5)))));
+
+        Assert.assertEquals(2, expressions.size());
+
+        Assert.assertEquals(new Expression("height", Int32Type.instance)
+        {{
+            operation = Op.RANGE;
+            lower = new Bound(Int32Type.instance.decompose(0), false);
+            exclusions.add(Int32Type.instance.decompose(5));
+        }}, expressions.get(Expression.Op.RANGE));
+
+        expressions = convert(Operation.analyzeGroup(controller, OperationType.AND,
+                Arrays.asList(new SimpleExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                              new SimpleExpression(height, Operator.NEQ, Int32Type.instance.decompose(5)),
+                              new SimpleExpression(height, Operator.GTE, Int32Type.instance.decompose(0)),
+                              new SimpleExpression(height, Operator.LT, Int32Type.instance.decompose(10)))));
+
+        Assert.assertEquals(2, expressions.size());
+
+        Assert.assertEquals(new Expression("height", Int32Type.instance)
+        {{
+                operation = Op.RANGE;
+                lower = new Bound(Int32Type.instance.decompose(0), true);
+                upper = new Bound(Int32Type.instance.decompose(10), false);
+                exclusions.add(Int32Type.instance.decompose(5));
+        }}, expressions.get(Expression.Op.RANGE));
+    }
+
+    @Test
+    public void testSatisfiedByWithMultipleTerms()
+    {
+        final ColumnDefinition comment = getColumn(UTF8Type.instance.decompose("comment"));
+
+        Unfiltered row = buildRow(
+                buildCell(comment,
+                                                          UTF8Type.instance.decompose("software engineer is working on a project"),
+                                                          System.currentTimeMillis()));
+
+        Operation.Builder builder = new Operation.Builder(OperationType.AND, controller,
+                                            new SimpleExpression(comment, Operator.EQ, UTF8Type.instance.decompose("eng is a work")));
+        Operation op = builder.complete();
+
+        Assert.assertTrue(op.satisfiedBy(row, false));
+
+        builder = new Operation.Builder(OperationType.AND, controller,
+                                            new SimpleExpression(comment, Operator.EQ, UTF8Type.instance.decompose("soft works fine")));
+        op = builder.complete();
+
+        Assert.assertTrue(op.satisfiedBy(row, false));
+    }
+
+    @Test
+    public void testSatisfiedByWithClustering()
+    {
+        ColumnDefinition location = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("location"));
+        ColumnDefinition age = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("age"));
+        ColumnDefinition height = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("height"));
+        ColumnDefinition score = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("score"));
+
+        Unfiltered row = buildRow(new Clustering(UTF8Type.instance.fromString("US"), Int32Type.instance.decompose(27)),
+                                  buildCell(height, Int32Type.instance.decompose(182), System.currentTimeMillis()),
+                                  buildCell(score, DoubleType.instance.decompose(1.0d), System.currentTimeMillis()));
+
+        Operation.Builder builder = new Operation.Builder(OperationType.AND, controller);
+        builder.add(new SimpleExpression(age, Operator.EQ, Int32Type.instance.decompose(27)));
+        builder.add(new SimpleExpression(height, Operator.EQ, Int32Type.instance.decompose(182)));
+
+        Assert.assertTrue(builder.complete().satisfiedBy(row, false));
+
+        builder = new Operation.Builder(OperationType.AND, controller);
+
+        builder.add(new SimpleExpression(age, Operator.EQ, Int32Type.instance.decompose(28)));
+        builder.add(new SimpleExpression(height, Operator.EQ, Int32Type.instance.decompose(182)));
+
+        Assert.assertFalse(builder.complete().satisfiedBy(row, false));
+
+        builder = new Operation.Builder(OperationType.AND, controller);
+        builder.add(new SimpleExpression(location, Operator.EQ, UTF8Type.instance.decompose("US")));
+        builder.add(new SimpleExpression(age, Operator.GTE, Int32Type.instance.decompose(27)));
+
+        Assert.assertTrue(builder.complete().satisfiedBy(row, false));
+
+        builder = new Operation.Builder(OperationType.AND, controller);
+        builder.add(new SimpleExpression(location, Operator.EQ, UTF8Type.instance.decompose("BY")));
+        builder.add(new SimpleExpression(age, Operator.GTE, Int32Type.instance.decompose(28)));
+
+        Assert.assertFalse(builder.complete().satisfiedBy(row, false));
+
+        builder = new Operation.Builder(OperationType.AND, controller);
+        builder.add(new SimpleExpression(location, Operator.EQ, UTF8Type.instance.decompose("US")));
+        builder.add(new SimpleExpression(age, Operator.LTE, Int32Type.instance.decompose(27)));
+        builder.add(new SimpleExpression(height, Operator.GTE, Int32Type.instance.decompose(182)));
+
+        Assert.assertTrue(builder.complete().satisfiedBy(row, false));
+
+        builder = new Operation.Builder(OperationType.AND, controller);
+        builder.add(new SimpleExpression(location, Operator.EQ, UTF8Type.instance.decompose("US")));
+        builder.add(new SimpleExpression(height, Operator.GTE, Int32Type.instance.decompose(182)));
+        builder.add(new SimpleExpression(score, Operator.EQ, DoubleType.instance.decompose(1.0d)));
+
+        Assert.assertTrue(builder.complete().satisfiedBy(row, false));
+
+        builder = new Operation.Builder(OperationType.AND, controller);
+        builder.add(new SimpleExpression(height, Operator.GTE, Int32Type.instance.decompose(182)));
+        builder.add(new SimpleExpression(score, Operator.EQ, DoubleType.instance.decompose(1.0d)));
+
+        Assert.assertTrue(builder.complete().satisfiedBy(row, false));
+    }
+
+    private Map<Expression.Op, Expression> convert(Multimap<ColumnDefinition, Expression> expressions)
+    {
+        Map<Expression.Op, Expression> converted = new HashMap<>();
+        for (Expression expression : expressions.values())
+        {
+            Expression column = converted.get(expression.getOp());
+            assert column == null; // sanity check
+            converted.put(expression.getOp(), expression);
+        }
+
+        return converted;
+    }
+
+    private static class SimpleExpression extends RowFilter.Expression
+    {
+        protected SimpleExpression(ColumnDefinition column, Operator operator, ByteBuffer value)
+        {
+            super(column, operator, value);
+        }
+
+        @Override
+        protected Kind kind()
+        {
+            return Kind.SIMPLE;
+        }
+
+        @Override
+        public boolean isSatisfiedBy(CFMetaData metadata, DecoratedKey partitionKey, Row row)
+        {
+            throw new UnsupportedOperationException();
+        }
+    }
+
+    private static Unfiltered buildRow(Cell... cells)
+    {
+        return buildRow(Clustering.EMPTY, null, cells);
+    }
+
+    private static Row buildRow(Row.Deletion deletion, Cell... cells)
+    {
+        return buildRow(Clustering.EMPTY, deletion, cells);
+    }
+
+    private static Row buildRow(Clustering clustering, Cell... cells)
+    {
+        return buildRow(clustering, null, cells);
+    }
+
+    private static Row buildRow(Clustering clustering, Row.Deletion deletion, Cell... cells)
+    {
+        Row.Builder rowBuilder = BTreeRow.sortedBuilder();
+        rowBuilder.newRow(clustering);
+        for (Cell c : cells)
+            rowBuilder.addCell(c);
+
+        if (deletion != null)
+            rowBuilder.addRowDeletion(deletion);
+
+        return rowBuilder.build();
+    }
+
+    private static Cell buildCell(ColumnDefinition column, ByteBuffer value, long timestamp)
+    {
+        return BufferCell.live(BACKEND.metadata, column, timestamp, value);
+    }
+
+    private static Cell deletedCell(ColumnDefinition column, long timestamp, int nowInSeconds)
+    {
+        return BufferCell.tombstone(column, timestamp, nowInSeconds);
+    }
+
+    private static ColumnDefinition getColumn(ByteBuffer name)
+    {
+        return getColumn(BACKEND, name);
+    }
+
+    private static ColumnDefinition getColumn(ColumnFamilyStore cfs, ByteBuffer name)
+    {
+        return cfs.metadata.getColumnDefinition(name);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/index/sasi/utils/LongIterator.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/utils/LongIterator.java b/test/unit/org/apache/cassandra/index/sasi/utils/LongIterator.java
new file mode 100644
index 0000000..96e7610
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/utils/LongIterator.java
@@ -0,0 +1,106 @@
+/*
+ * 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.index.sasi.utils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.index.sasi.disk.Token;
+
+public class LongIterator extends RangeIterator<Long, Token>
+{
+    private final List<LongToken> tokens;
+    private int currentIdx = 0;
+
+    public LongIterator(long[] tokens)
+    {
+        super(tokens.length == 0 ? null : tokens[0], tokens.length == 0 ? null : tokens[tokens.length - 1], tokens.length);
+        this.tokens = new ArrayList<>(tokens.length);
+        for (long token : tokens)
+            this.tokens.add(new LongToken(token));
+    }
+
+    @Override
+    protected Token computeNext()
+    {
+        if (currentIdx >= tokens.size())
+            return endOfData();
+
+        return tokens.get(currentIdx++);
+    }
+
+    @Override
+    protected void performSkipTo(Long nextToken)
+    {
+        for (int i = currentIdx == 0 ? 0 : currentIdx - 1; i < tokens.size(); i++)
+        {
+            LongToken token = tokens.get(i);
+            if (token.get().compareTo(nextToken) >= 0)
+            {
+                currentIdx = i;
+                break;
+            }
+        }
+    }
+
+    @Override
+    public void close() throws IOException
+    {}
+
+    public static class LongToken extends Token
+    {
+        public LongToken(long token)
+        {
+            super(token);
+        }
+
+        @Override
+        public void merge(CombinedValue<Long> other)
+        {
+            // no-op
+        }
+
+        @Override
+        public Iterator<DecoratedKey> iterator()
+        {
+            return Collections.emptyIterator();
+        }
+    }
+
+    public static List<Long> convert(RangeIterator<Long, Token> tokens)
+    {
+        List<Long> results = new ArrayList<>();
+        while (tokens.hasNext())
+            results.add(tokens.next().get());
+
+        return results;
+    }
+
+    public static List<Long> convert(final long... nums)
+    {
+        return new ArrayList<Long>(nums.length)
+        {{
+                for (long n : nums)
+                    add(n);
+        }};
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/index/sasi/utils/MappedBufferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/utils/MappedBufferTest.java b/test/unit/org/apache/cassandra/index/sasi/utils/MappedBufferTest.java
new file mode 100644
index 0000000..7ffebf1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/utils/MappedBufferTest.java
@@ -0,0 +1,540 @@
+/*
+ * 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.index.sasi.utils;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.FileUtils;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class MappedBufferTest
+{
+    @Test
+    public void testBasicWriteThenRead() throws Exception
+    {
+        long numLongs = 10000;
+        final MappedBuffer buffer = createTestFile(numLongs);
+
+        Assert.assertEquals(0, buffer.position());
+        for (long i = 0; i < numLongs; i++)
+        {
+            Assert.assertEquals(i * 8, buffer.position());
+            Assert.assertEquals(i, buffer.getLong());
+        }
+
+        buffer.position(0);
+        for (long i = 0; i < numLongs; i++)
+        {
+            Assert.assertEquals(i, buffer.getLong(i * 8));
+            Assert.assertEquals(0, buffer.position());
+        }
+
+        // read all the numbers as shorts (all numbers fit into four bytes)
+        for (long i = 0; i < Math.min(Integer.MAX_VALUE, numLongs); i++)
+            Assert.assertEquals(i, buffer.getInt((i * 8) + 4));
+
+        // read all the numbers as shorts (all numbers fit into two bytes)
+        for (long i = 0; i < Math.min(Short.MAX_VALUE, numLongs); i++) {
+            Assert.assertEquals(i, buffer.getShort((i * 8) + 6));
+        }
+
+        // read all the numbers that can be represented as a single byte
+        for (long i = 0; i < 128; i++)
+            Assert.assertEquals(i, buffer.get((i * 8) + 7));
+
+        buffer.close();
+    }
+
+    @Test
+    public void testDuplicate() throws Exception
+    {
+        long numLongs = 10;
+        final MappedBuffer buffer1 = createTestFile(numLongs);
+
+        Assert.assertEquals(0, buffer1.getLong());
+        Assert.assertEquals(1, buffer1.getLong());
+
+        final MappedBuffer buffer2 = buffer1.duplicate();
+
+        Assert.assertEquals(2, buffer1.getLong());
+        Assert.assertEquals(2, buffer2.getLong());
+
+        buffer2.position(0);
+        Assert.assertEquals(3, buffer1.getLong());
+        Assert.assertEquals(0, buffer2.getLong());
+    }
+
+    @Test
+    public void testLimit() throws Exception
+    {
+        long numLongs =  10;
+        final MappedBuffer buffer1 = createTestFile(numLongs);
+
+        MappedBuffer buffer2 = buffer1.duplicate().position(16).limit(32);
+        buffer1.position(0).limit(16);
+        List<Long> longs = new ArrayList<>(4);
+
+        while (buffer1.hasRemaining())
+            longs.add(buffer1.getLong());
+
+        while (buffer2.hasRemaining())
+            longs.add(buffer2.getLong());
+
+        Assert.assertArrayEquals(new Long[]{0L, 1L, 2L, 3L}, longs.toArray());
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testPositionGreaterThanLimit() throws Exception
+    {
+        final MappedBuffer buffer = createTestFile(1);
+
+        buffer.limit(4);
+
+        try
+        {
+            buffer.position(buffer.limit() + 1);
+        }
+        finally
+        {
+            buffer.close();
+        }
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testNegativePosition() throws Exception
+    {
+        try (MappedBuffer buffer = createTestFile(1))
+        {
+            buffer.position(-1);
+        }
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testLimitGreaterThanCapacity() throws Exception
+    {
+        try (MappedBuffer buffer = createTestFile(1))
+        {
+            buffer.limit(buffer.capacity() + 1);
+        }
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testLimitLessThanPosition() throws Exception
+    {
+        final MappedBuffer buffer = createTestFile(1);
+
+        buffer.position(1);
+
+        try
+        {
+            buffer.limit(0);
+        }
+        finally
+        {
+            buffer.close();
+        }
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetRelativeUnderflow() throws Exception
+    {
+        final MappedBuffer buffer = createTestFile(1);
+
+        buffer.position(buffer.limit());
+        try
+        {
+            buffer.get();
+        }
+        finally
+        {
+            buffer.close();
+        }
+
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetAbsoluteGreaterThanCapacity() throws Exception
+    {
+        try (MappedBuffer buffer = createTestFile(1))
+        {
+            buffer.get(buffer.limit());
+        }
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetAbsoluteNegativePosition() throws Exception
+    {
+        try (MappedBuffer buffer = createTestFile(1))
+        {
+            buffer.get(-1);
+        }
+    }
+
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetShortRelativeUnderflow() throws Exception
+    {
+        final MappedBuffer buffer = createTestFile(1);
+
+        buffer.position(buffer.capacity() - 1);
+        try
+        {
+            buffer.getShort();
+        }
+        finally
+        {
+            buffer.close();
+        }
+
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetShortAbsoluteGreaterThanCapacity() throws Exception
+    {
+        final MappedBuffer buffer = createTestFile(1);
+
+        Assert.assertEquals(8, buffer.capacity());
+        try
+        {
+            buffer.getShort(buffer.capacity() - 1);
+        }
+        finally
+        {
+            buffer.close();
+        }
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetShortAbsoluteNegativePosition() throws Exception
+    {
+        try (MappedBuffer buffer = createTestFile(1))
+        {
+            buffer.getShort(-1);
+        }
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetIntRelativeUnderflow() throws Exception
+    {
+        final MappedBuffer buffer = createTestFile(1);
+
+        buffer.position(buffer.capacity() - 3);
+        try
+        {
+            buffer.getInt();
+        }
+        finally
+        {
+            buffer.close();
+        }
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetIntAbsoluteGreaterThanCapacity() throws Exception
+    {
+        final MappedBuffer buffer = createTestFile(1);
+
+        Assert.assertEquals(8, buffer.capacity());
+        try
+        {
+            buffer.getInt(buffer.capacity() - 3);
+        }
+        finally
+        {
+            buffer.close();
+        }
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetIntAbsoluteNegativePosition() throws Exception
+    {
+        try (MappedBuffer buffer = createTestFile(1))
+        {
+            buffer.getInt(-1);
+        }
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetLongRelativeUnderflow() throws Exception
+    {
+        final MappedBuffer buffer = createTestFile(1);
+
+        buffer.position(buffer.capacity() - 7);
+        try
+        {
+            buffer.getLong();
+        }
+        finally
+        {
+            buffer.close();
+        }
+
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetLongAbsoluteGreaterThanCapacity() throws Exception
+    {
+        final MappedBuffer buffer = createTestFile(1);
+
+        Assert.assertEquals(8, buffer.capacity());
+        try
+        {
+            buffer.getLong(buffer.capacity() - 7);
+        }
+        finally
+        {
+            buffer.close();
+        }
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testGetLongAbsoluteNegativePosition() throws Exception
+    {
+        try (MappedBuffer buffer = createTestFile(1))
+        {
+            buffer.getLong(-1);
+        }
+    }
+
+    @Test
+    public void testGetPageRegion() throws Exception
+    {
+        ThreadLocalRandom random = ThreadLocalRandom.current();
+
+        int numLongs = 1000;
+        int byteSize = 8;
+        int capacity = numLongs * byteSize;
+        try (MappedBuffer buffer = createTestFile(numLongs))
+        {
+            for (int i = 0; i < 1000; i++)
+            {
+                // offset, length are always aligned on sizeof(long)
+                int offset = random.nextInt(0, 1000 * byteSize - byteSize) & ~(byteSize - 1);
+                int length = Math.min(capacity, random.nextInt(byteSize, capacity - offset) & ~(byteSize - 1));
+
+                ByteBuffer region = buffer.getPageRegion(offset, length);
+                for (int j = offset; j < (offset + length); j += 8)
+                    Assert.assertEquals(j / 8, region.getLong(j));
+            }
+        }
+    }
+
+    @Test (expected = IllegalArgumentException.class)
+    public void testMisalignedRegionAccess() throws Exception
+    {
+        try (MappedBuffer buffer = createTestFile(100, 8, 4, 0))
+        {
+            buffer.getPageRegion(13, 27);
+        }
+    }
+
+    @Test
+    public void testSequentialIterationWithPadding() throws Exception
+    {
+        long numValues = 1000;
+        int maxPageBits = 6; // 64 bytes page
+        int[] paddings = new int[] { 0, 3, 5, 7, 9, 11, 13 };
+
+        // test different page sizes, with different padding and types
+        for (int numPageBits = 3; numPageBits <= maxPageBits; numPageBits++)
+        {
+            for (int typeSize = 2; typeSize <= 8; typeSize *= 2)
+            {
+                for (int padding : paddings)
+                {
+                    try (MappedBuffer buffer = createTestFile(numValues, typeSize, numPageBits, padding))
+                    {
+                        long offset = 0;
+                        for (long j = 0; j < numValues; j++)
+                        {
+                            switch (typeSize)
+                            {
+                                case 2:
+                                    Assert.assertEquals(j, buffer.getShort(offset));
+                                    break;
+
+                                case 4:
+                                    Assert.assertEquals(j, buffer.getInt(offset));
+                                    break;
+
+                                case 8:
+                                    Assert.assertEquals(j, buffer.getLong(offset));
+                                    break;
+
+                                default:
+                                    throw new AssertionError();
+                            }
+
+                            offset += typeSize + padding;
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testSequentialIteration() throws IOException
+    {
+        long numValues = 1000;
+        for (int typeSize = 2; typeSize <= 8; typeSize *= 2)
+        {
+            try (MappedBuffer buffer = createTestFile(numValues, typeSize, 16, 0))
+            {
+                for (int j = 0; j < numValues; j++)
+                {
+                    Assert.assertEquals(j * typeSize, buffer.position());
+
+                    switch (typeSize)
+                    {
+                        case 2:
+                            Assert.assertEquals(j, buffer.getShort());
+                            break;
+
+                        case 4:
+                            Assert.assertEquals(j, buffer.getInt());
+                            break;
+
+                        case 8:
+                            Assert.assertEquals(j, buffer.getLong());
+                            break;
+
+                        default:
+                            throw new AssertionError();
+                    }
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testCompareToPage() throws IOException
+    {
+        long numValues = 100;
+        int typeSize = 8;
+
+        try (MappedBuffer buffer = createTestFile(numValues))
+        {
+            for (long i = 0; i < numValues * typeSize; i += typeSize)
+            {
+                long value = i / typeSize;
+                Assert.assertEquals(0, buffer.comparePageTo(i, typeSize, LongType.instance, LongType.instance.decompose(value)));
+            }
+        }
+    }
+
+    @Test
+    public void testOpenWithoutPageBits() throws IOException
+    {
+        File tmp = File.createTempFile("mapped-buffer", "tmp");
+        tmp.deleteOnExit();
+
+        RandomAccessFile file = new RandomAccessFile(tmp, "rw");
+
+        long numValues = 1000;
+        for (long i = 0; i < numValues; i++)
+            file.writeLong(i);
+
+        file.getFD().sync();
+
+        try (MappedBuffer buffer = new MappedBuffer(new ChannelProxy(tmp.getAbsolutePath(), file.getChannel())))
+        {
+            Assert.assertEquals(numValues * 8, buffer.limit());
+            Assert.assertEquals(numValues * 8, buffer.capacity());
+
+            for (long i = 0; i < numValues; i++)
+            {
+                Assert.assertEquals(i * 8, buffer.position());
+                Assert.assertEquals(i, buffer.getLong());
+            }
+        }
+        finally
+        {
+            FileUtils.closeQuietly(file);
+        }
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testIncorrectPageSize() throws Exception
+    {
+        new MappedBuffer(null, 33);
+    }
+
+    private MappedBuffer createTestFile(long numCount) throws IOException
+    {
+        return createTestFile(numCount, 8, 16, 0);
+    }
+
+    private MappedBuffer createTestFile(long numCount, int typeSize, int numPageBits, int padding) throws IOException
+    {
+        final File testFile = File.createTempFile("mapped-buffer-test", "db");
+        testFile.deleteOnExit();
+
+        RandomAccessFile file = new RandomAccessFile(testFile, "rw");
+
+        for (long i = 0; i < numCount; i++)
+        {
+
+            switch (typeSize)
+            {
+                case 1:
+                    file.write((byte) i);
+                    break;
+
+                case 2:
+                    file.writeShort((short) i);
+                    break;
+
+                case 4:
+                    file.writeInt((int) i);
+                    break;
+
+                case 8:
+                    // bunch of longs
+                    file.writeLong(i);
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("unknown byte size: " + typeSize);
+            }
+
+            for (int j = 0; j < padding; j++)
+                file.write(0);
+        }
+
+        file.getFD().sync();
+
+        try
+        {
+            return new MappedBuffer(new ChannelProxy(testFile.getAbsolutePath(), file.getChannel()), numPageBits);
+        }
+        finally
+        {
+            FileUtils.closeQuietly(file);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/index/sasi/utils/RangeIntersectionIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/utils/RangeIntersectionIteratorTest.java b/test/unit/org/apache/cassandra/index/sasi/utils/RangeIntersectionIteratorTest.java
new file mode 100644
index 0000000..18b9dd7
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/utils/RangeIntersectionIteratorTest.java
@@ -0,0 +1,387 @@
+/*
+ * 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.index.sasi.utils;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.utils.RangeIntersectionIterator.Strategy;
+import org.apache.cassandra.index.sasi.utils.RangeIntersectionIterator.LookupIntersectionIterator;
+import org.apache.cassandra.index.sasi.utils.RangeIntersectionIterator.BounceIntersectionIterator;
+import org.apache.cassandra.io.util.FileUtils;
+
+import com.carrotsearch.hppc.LongOpenHashSet;
+import com.carrotsearch.hppc.LongSet;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.cassandra.index.sasi.utils.LongIterator.convert;
+
+public class RangeIntersectionIteratorTest
+{
+    @Test
+    public void testNoOverlappingValues()
+    {
+        for (Strategy strategy : Strategy.values())
+            testNoOverlappingValues(strategy);
+    }
+
+    private void testNoOverlappingValues(Strategy strategy)
+    {
+        RangeIterator.Builder<Long, Token> builder = RangeIntersectionIterator.builder(strategy);
+
+        builder.add(new LongIterator(new long[] { 2L, 3L, 5L, 6L }));
+        builder.add(new LongIterator(new long[] { 1L, 7L }));
+        builder.add(new LongIterator(new long[] { 4L, 8L, 9L, 10L }));
+
+        Assert.assertEquals(convert(), convert(builder.build()));
+
+        builder = RangeIntersectionIterator.builder(strategy);
+        // both ranges overlap by min/max but not by value
+        builder.add(new LongIterator(new long[] { 1L, 5L, 7L, 9L }));
+        builder.add(new LongIterator(new long[] { 6L }));
+
+        RangeIterator<Long, Token> range = builder.build();
+
+        Assert.assertNotNull(range);
+        Assert.assertFalse(range.hasNext());
+
+        builder = RangeIntersectionIterator.builder(strategy);
+        // both ranges overlap by min/max but not by value
+        builder.add(new LongIterator(new long[] { 1L, 5L, 7L, 9L }));
+        builder.add(new LongIterator(new long[] { 0L, 10L, 12L }));
+
+        range = builder.build();
+
+        Assert.assertNotNull(range);
+        Assert.assertFalse(range.hasNext());
+    }
+
+    @Test
+    public void testOverlappingValues()
+    {
+        for (Strategy strategy : Strategy.values())
+            testOverlappingValues(strategy);
+    }
+
+    private void testOverlappingValues(Strategy strategy)
+    {
+        RangeIterator.Builder<Long, Token> builder = RangeIntersectionIterator.builder(strategy);
+
+        builder.add(new LongIterator(new long[] { 1L, 4L, 6L, 7L }));
+        builder.add(new LongIterator(new long[] { 2L, 4L, 5L, 6L }));
+        builder.add(new LongIterator(new long[] { 4L, 6L, 8L, 9L, 10L }));
+
+        Assert.assertEquals(convert(4L, 6L), convert(builder.build()));
+    }
+
+    @Test
+    public void testSingleIterator()
+    {
+        for (Strategy strategy : Strategy.values())
+            testSingleIterator(strategy);
+    }
+
+    private void testSingleIterator(Strategy strategy)
+    {
+        RangeIntersectionIterator.Builder<Long, Token> builder = RangeIntersectionIterator.builder(strategy);
+
+        builder.add(new LongIterator(new long[] { 1L, 2L, 4L, 9L }));
+
+        Assert.assertEquals(convert(1L, 2L, 4L, 9L), convert(builder.build()));
+    }
+
+    @Test
+    public void testSkipTo()
+    {
+        for (Strategy strategy : Strategy.values())
+            testSkipTo(strategy);
+    }
+
+    private void testSkipTo(Strategy strategy)
+    {
+        RangeIterator.Builder<Long, Token> builder = RangeIntersectionIterator.builder(strategy);
+
+        builder.add(new LongIterator(new long[] { 1L, 4L, 6L, 7L, 9L, 10L }));
+        builder.add(new LongIterator(new long[] { 2L, 4L, 5L, 6L, 7L, 10L, 12L }));
+        builder.add(new LongIterator(new long[] { 4L, 6L, 7L, 9L, 10L }));
+
+        RangeIterator<Long, Token> range = builder.build();
+        Assert.assertNotNull(range);
+
+        // first let's skipTo something before range
+        Assert.assertEquals(4L, (long) range.skipTo(3L).get());
+        Assert.assertEquals(4L, (long) range.getCurrent());
+
+        // now let's skip right to the send value
+        Assert.assertEquals(6L, (long) range.skipTo(5L).get());
+        Assert.assertEquals(6L, (long) range.getCurrent());
+
+        // now right to the element
+        Assert.assertEquals(7L, (long) range.skipTo(7L).get());
+        Assert.assertEquals(7L, (long) range.getCurrent());
+        Assert.assertEquals(7L, (long) range.next().get());
+
+        Assert.assertTrue(range.hasNext());
+        Assert.assertEquals(10L, (long) range.getCurrent());
+
+        // now right after the last element
+        Assert.assertNull(range.skipTo(11L));
+        Assert.assertFalse(range.hasNext());
+    }
+
+    @Test
+    public void testMinMaxAndCount()
+    {
+        for (Strategy strategy : Strategy.values())
+            testMinMaxAndCount(strategy);
+    }
+
+    private void testMinMaxAndCount(Strategy strategy)
+    {
+        RangeIterator.Builder<Long, Token> builder = RangeIntersectionIterator.builder(strategy);
+
+        builder.add(new LongIterator(new long[]{1L, 2L, 9L}));
+        builder.add(new LongIterator(new long[]{4L, 5L, 9L}));
+        builder.add(new LongIterator(new long[]{7L, 8L, 9L}));
+
+        Assert.assertEquals(9L, (long) builder.getMaximum());
+        Assert.assertEquals(9L, builder.getTokenCount());
+
+        RangeIterator<Long, Token> tokens = builder.build();
+
+        Assert.assertNotNull(tokens);
+        Assert.assertEquals(7L, (long) tokens.getMinimum());
+        Assert.assertEquals(9L, (long) tokens.getMaximum());
+        Assert.assertEquals(9L, tokens.getCount());
+
+        Assert.assertEquals(convert(9L), convert(builder.build()));
+    }
+
+    @Test
+    public void testBuilder()
+    {
+        for (Strategy strategy : Strategy.values())
+            testBuilder(strategy);
+    }
+
+    private void testBuilder(Strategy strategy)
+    {
+        RangeIterator.Builder<Long, Token> builder = RangeIntersectionIterator.builder(strategy);
+
+        Assert.assertNull(builder.getMinimum());
+        Assert.assertNull(builder.getMaximum());
+        Assert.assertEquals(0L, builder.getTokenCount());
+        Assert.assertEquals(0L, builder.rangeCount());
+
+        builder.add(new LongIterator(new long[] { 1L, 2L, 6L }));
+        builder.add(new LongIterator(new long[] { 4L, 5L, 6L }));
+        builder.add(new LongIterator(new long[] { 6L, 8L, 9L }));
+
+        Assert.assertEquals(6L, (long) builder.getMinimum());
+        Assert.assertEquals(6L, (long) builder.getMaximum());
+        Assert.assertEquals(9L, builder.getTokenCount());
+        Assert.assertEquals(3L, builder.rangeCount());
+        Assert.assertFalse(builder.statistics.isDisjoint());
+
+        Assert.assertEquals(1L, (long) builder.ranges.poll().getMinimum());
+        Assert.assertEquals(4L, (long) builder.ranges.poll().getMinimum());
+        Assert.assertEquals(6L, (long) builder.ranges.poll().getMinimum());
+
+        builder.add(new LongIterator(new long[] { 1L, 2L, 6L }));
+        builder.add(new LongIterator(new long[] { 4L, 5L, 6L }));
+        builder.add(new LongIterator(new long[] { 6L, 8L, 9L }));
+
+        Assert.assertEquals(convert(6L), convert(builder.build()));
+
+        builder = RangeIntersectionIterator.builder(strategy);
+        builder.add(new LongIterator(new long[]{ 1L, 5L, 6L }));
+        builder.add(new LongIterator(new long[]{ 3L, 5L, 6L }));
+
+        RangeIterator<Long, Token> tokens = builder.build();
+
+        Assert.assertEquals(convert(5L, 6L), convert(tokens));
+
+        FileUtils.closeQuietly(tokens);
+
+        RangeIterator emptyTokens = RangeIntersectionIterator.builder(strategy).build();
+        Assert.assertNull(emptyTokens);
+
+        builder = RangeIntersectionIterator.builder(strategy);
+        Assert.assertEquals(0L, builder.add((RangeIterator<Long, Token>) null).rangeCount());
+        Assert.assertEquals(0L, builder.add((List<RangeIterator<Long, Token>>) null).getTokenCount());
+        Assert.assertEquals(0L, builder.add(new LongIterator(new long[] {})).rangeCount());
+
+        RangeIterator<Long, Token> single = new LongIterator(new long[] { 1L, 2L, 3L });
+        RangeIterator<Long, Token> range = RangeIntersectionIterator.<Long, Token>builder().add(single).build();
+
+        // because build should return first element if it's only one instead of building yet another iterator
+        Assert.assertEquals(range, single);
+
+        // disjoint case
+        builder = RangeIntersectionIterator.builder();
+        builder.add(new LongIterator(new long[] { 1L, 2L, 3L }));
+        builder.add(new LongIterator(new long[] { 4L, 5L, 6L }));
+
+        Assert.assertTrue(builder.statistics.isDisjoint());
+
+        RangeIterator<Long, Token> disjointIntersection = builder.build();
+        Assert.assertNotNull(disjointIntersection);
+        Assert.assertFalse(disjointIntersection.hasNext());
+
+    }
+
+    @Test
+    public void testClose() throws IOException
+    {
+        for (Strategy strategy : Strategy.values())
+            testClose(strategy);
+    }
+
+    private void testClose(Strategy strategy) throws IOException
+    {
+        RangeIterator<Long, Token> tokens = RangeIntersectionIterator.<Long, Token>builder(strategy)
+                                            .add(new LongIterator(new long[] { 1L, 2L, 3L }))
+                                            .build();
+
+        Assert.assertNotNull(tokens);
+        tokens.close();
+    }
+
+    @Test
+    public void testIsOverlapping()
+    {
+        RangeIterator<Long, Token> rangeA, rangeB;
+
+        rangeA = new LongIterator(new long[] { 1L, 5L });
+        rangeB = new LongIterator(new long[] { 5L, 9L });
+        Assert.assertTrue(RangeIterator.isOverlapping(rangeA, rangeB));
+
+        rangeA = new LongIterator(new long[] { 5L, 9L });
+        rangeB = new LongIterator(new long[] { 1L, 6L });
+        Assert.assertTrue(RangeIterator.isOverlapping(rangeA, rangeB));
+
+        rangeA = new LongIterator(new long[] { 5L, 9L });
+        rangeB = new LongIterator(new long[] { 5L, 9L });
+        Assert.assertTrue(RangeIterator.isOverlapping(rangeA, rangeB));
+
+        rangeA = new LongIterator(new long[] { 1L, 4L });
+        rangeB = new LongIterator(new long[] { 5L, 9L });
+        Assert.assertFalse(RangeIterator.isOverlapping(rangeA, rangeB));
+
+        rangeA = new LongIterator(new long[] { 6L, 9L });
+        rangeB = new LongIterator(new long[] { 1L, 4L });
+        Assert.assertFalse(RangeIterator.isOverlapping(rangeA, rangeB));
+    }
+
+    @Test
+    public void testIntersectionOfRandomRanges()
+    {
+        for (Strategy strategy : Strategy.values())
+            testIntersectionOfRandomRanges(strategy);
+    }
+
+    private void testIntersectionOfRandomRanges(Strategy strategy)
+    {
+        for (int attempt = 0; attempt < 16; attempt++)
+        {
+            final ThreadLocalRandom random = ThreadLocalRandom.current();
+            final int maxRanges = random.nextInt(2, 16);
+
+            // generate randomize ranges
+            long[][] ranges = new long[maxRanges][];
+            for (int i = 0; i < ranges.length; i++)
+            {
+                int rangeSize = random.nextInt(16, 512);
+                LongSet range = new LongOpenHashSet(rangeSize);
+
+                for (int j = 0; j < rangeSize; j++)
+                    range.add(random.nextLong(0, 100));
+
+                ranges[i] = range.toArray();
+                Arrays.sort(ranges[i]);
+            }
+
+            List<Long> expected = new ArrayList<>();
+            // determine unique tokens which intersect every range
+            for (long token : ranges[0])
+            {
+                boolean intersectsAll = true;
+                for (int i = 1; i < ranges.length; i++)
+                {
+                    if (Arrays.binarySearch(ranges[i], token) < 0)
+                    {
+                        intersectsAll = false;
+                        break;
+                    }
+                }
+
+                if (intersectsAll)
+                    expected.add(token);
+            }
+
+            RangeIterator.Builder<Long, Token> builder = RangeIntersectionIterator.builder(strategy);
+            for (long[] range : ranges)
+                builder.add(new LongIterator(range));
+
+            Assert.assertEquals(expected, convert(builder.build()));
+        }
+    }
+
+    @Test
+    public void testIteratorPeeking()
+    {
+        RangeIterator.Builder<Long, Token> builder = RangeIntersectionIterator.builder();
+
+        // iterator with only one element
+        builder.add(new LongIterator(new long[] { 10L }));
+
+        // iterator with 150 elements (lookup is going to be advantageous over bound in this case)
+        long[] tokens = new long[150];
+        for (int i = 0; i < tokens.length; i++)
+            tokens[i] = i;
+
+        builder.add(new LongIterator(tokens));
+
+        RangeIterator<Long, Token> intersection = builder.build();
+
+        Assert.assertNotNull(intersection);
+        Assert.assertEquals(LookupIntersectionIterator.class, intersection.getClass());
+
+        Assert.assertTrue(intersection.hasNext());
+        Assert.assertEquals(convert(10L), convert(intersection));
+
+        builder = RangeIntersectionIterator.builder();
+
+        builder.add(new LongIterator(new long[] { 1L, 3L, 5L, 7L, 9L }));
+        builder.add(new LongIterator(new long[] { 1L, 2L, 5L, 6L }));
+
+        intersection = builder.build();
+
+        // in the situation when there is a similar number of elements inside ranges
+        // ping-pong (bounce) intersection is preferred as it covers gaps quicker then linear scan + lookup.
+        Assert.assertNotNull(intersection);
+        Assert.assertEquals(BounceIntersectionIterator.class, intersection.getClass());
+
+        Assert.assertTrue(intersection.hasNext());
+        Assert.assertEquals(convert(1L, 5L), convert(intersection));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/index/sasi/utils/RangeUnionIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/utils/RangeUnionIteratorTest.java b/test/unit/org/apache/cassandra/index/sasi/utils/RangeUnionIteratorTest.java
new file mode 100644
index 0000000..f69086b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/utils/RangeUnionIteratorTest.java
@@ -0,0 +1,306 @@
+/*
+ * 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.index.sasi.utils;
+
+import java.util.*;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.io.util.FileUtils;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.cassandra.index.sasi.utils.LongIterator.convert;
+
+public class RangeUnionIteratorTest
+{
+    @Test
+    public void testNoOverlappingValues()
+    {
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+
+        builder.add(new LongIterator(new long[] { 2L, 3L, 5L, 6L }));
+        builder.add(new LongIterator(new long[] { 1L, 7L }));
+        builder.add(new LongIterator(new long[] { 4L, 8L, 9L, 10L }));
+
+        Assert.assertEquals(convert(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L), convert(builder.build()));
+    }
+
+    @Test
+    public void testSingleIterator()
+    {
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+
+        builder.add(new LongIterator(new long[] { 1L, 2L, 4L, 9L }));
+
+        Assert.assertEquals(convert(1L, 2L, 4L, 9L), convert(builder.build()));
+    }
+
+    @Test
+    public void testOverlappingValues()
+    {
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+
+        builder.add(new LongIterator(new long[] { 1L, 4L, 6L, 7L }));
+        builder.add(new LongIterator(new long[] { 2L, 3L, 5L, 6L }));
+        builder.add(new LongIterator(new long[] { 4L, 6L, 8L, 9L, 10L }));
+
+        List<Long> values = convert(builder.build());
+
+        Assert.assertEquals(values.toString(), convert(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L), values);
+    }
+
+    @Test
+    public void testNoOverlappingRanges()
+    {
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+
+        builder.add(new LongIterator(new long[] { 1L, 2L, 3L }));
+        builder.add(new LongIterator(new long[] { 4L, 5L, 6L }));
+        builder.add(new LongIterator(new long[] { 7L, 8L, 9L }));
+
+        Assert.assertEquals(convert(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L), convert(builder.build()));
+    }
+
+    @Test
+    public void testTwoIteratorsWithSingleValues()
+    {
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+
+        builder.add(new LongIterator(new long[] { 1L }));
+        builder.add(new LongIterator(new long[] { 1L }));
+
+        Assert.assertEquals(convert(1L), convert(builder.build()));
+    }
+
+    @Test
+    public void testDifferentSizeIterators()
+    {
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+
+        builder.add(new LongIterator(new long[] { 2L, 3L, 5L, 6L, 12L, 13L }));
+        builder.add(new LongIterator(new long[] { 1L, 7L, 14L, 15 }));
+        builder.add(new LongIterator(new long[] { 4L, 5L, 8L, 9L, 10L }));
+
+        Assert.assertEquals(convert(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 12L, 13L, 14L, 15L), convert(builder.build()));
+    }
+
+    @Test
+    public void testRandomSequences()
+    {
+        ThreadLocalRandom random = ThreadLocalRandom.current();
+
+        long[][] values = new long[random.nextInt(1, 20)][];
+        int numTests = random.nextInt(10, 20);
+
+        for (int tests = 0; tests < numTests; tests++)
+        {
+            RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+            int totalCount = 0;
+
+            for (int i = 0; i < values.length; i++)
+            {
+                long[] part = new long[random.nextInt(1, 500)];
+                for (int j = 0; j < part.length; j++)
+                    part[j] = random.nextLong();
+
+                // all of the parts have to be sorted to mimic SSTable
+                Arrays.sort(part);
+
+                values[i] = part;
+                builder.add(new LongIterator(part));
+                totalCount += part.length;
+            }
+
+            long[] totalOrdering = new long[totalCount];
+            int index = 0;
+
+            for (long[] part : values)
+            {
+                for (long value : part)
+                    totalOrdering[index++] = value;
+            }
+
+            Arrays.sort(totalOrdering);
+
+            int count = 0;
+            RangeIterator<Long, Token> tokens = builder.build();
+
+            Assert.assertNotNull(tokens);
+            while (tokens.hasNext())
+                Assert.assertEquals(totalOrdering[count++], (long) tokens.next().get());
+
+            Assert.assertEquals(totalCount, count);
+        }
+    }
+
+    @Test
+    public void testMinMaxAndCount()
+    {
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+
+        builder.add(new LongIterator(new long[] { 1L, 2L, 3L }));
+        builder.add(new LongIterator(new long[] { 4L, 5L, 6L }));
+        builder.add(new LongIterator(new long[] { 7L, 8L, 9L }));
+
+        Assert.assertEquals(9L, (long) builder.getMaximum());
+        Assert.assertEquals(9L, builder.getTokenCount());
+
+        RangeIterator<Long, Token> tokens = builder.build();
+
+        Assert.assertNotNull(tokens);
+        Assert.assertEquals(1L, (long) tokens.getMinimum());
+        Assert.assertEquals(9L, (long) tokens.getMaximum());
+        Assert.assertEquals(9L, tokens.getCount());
+
+        for (long i = 1; i < 10; i++)
+        {
+            Assert.assertTrue(tokens.hasNext());
+            Assert.assertEquals(i, (long) tokens.next().get());
+        }
+
+        Assert.assertFalse(tokens.hasNext());
+        Assert.assertEquals(1L, (long) tokens.getMinimum());
+    }
+
+    @Test
+    public void testBuilder()
+    {
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+
+        Assert.assertNull(builder.getMinimum());
+        Assert.assertNull(builder.getMaximum());
+        Assert.assertEquals(0L, builder.getTokenCount());
+        Assert.assertEquals(0L, builder.rangeCount());
+
+        builder.add(new LongIterator(new long[] { 1L, 2L, 3L }));
+        builder.add(new LongIterator(new long[] { 4L, 5L, 6L }));
+        builder.add(new LongIterator(new long[] { 7L, 8L, 9L }));
+
+        Assert.assertEquals(1L, (long) builder.getMinimum());
+        Assert.assertEquals(9L, (long) builder.getMaximum());
+        Assert.assertEquals(9L, builder.getTokenCount());
+        Assert.assertEquals(3L, builder.rangeCount());
+        Assert.assertFalse(builder.statistics.isDisjoint());
+
+        Assert.assertEquals(1L, (long) builder.ranges.poll().getMinimum());
+        Assert.assertEquals(4L, (long) builder.ranges.poll().getMinimum());
+        Assert.assertEquals(7L, (long) builder.ranges.poll().getMinimum());
+
+        RangeIterator<Long, Token> tokens = RangeUnionIterator.build(new ArrayList<RangeIterator<Long, Token>>()
+        {{
+            add(new LongIterator(new long[]{1L, 2L, 4L}));
+            add(new LongIterator(new long[]{3L, 5L, 6L}));
+        }});
+
+        Assert.assertEquals(convert(1L, 2L, 3L, 4L, 5L, 6L), convert(tokens));
+
+        FileUtils.closeQuietly(tokens);
+
+        RangeIterator emptyTokens = RangeUnionIterator.builder().build();
+        Assert.assertNull(emptyTokens);
+
+        builder = RangeUnionIterator.builder();
+        Assert.assertEquals(0L, builder.add((RangeIterator<Long, Token>) null).rangeCount());
+        Assert.assertEquals(0L, builder.add((List<RangeIterator<Long, Token>>) null).getTokenCount());
+        Assert.assertEquals(0L, builder.add(new LongIterator(new long[] {})).rangeCount());
+
+        RangeIterator<Long, Token> single = new LongIterator(new long[] { 1L, 2L, 3L });
+        RangeIterator<Long, Token> range = RangeIntersectionIterator.<Long, Token>builder().add(single).build();
+
+        // because build should return first element if it's only one instead of building yet another iterator
+        Assert.assertEquals(range, single);
+    }
+
+    @Test
+    public void testSkipTo()
+    {
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+
+        builder.add(new LongIterator(new long[]{1L, 2L, 3L}));
+        builder.add(new LongIterator(new long[]{4L, 5L, 6L}));
+        builder.add(new LongIterator(new long[]{7L, 8L, 9L}));
+
+        RangeIterator<Long, Token> tokens = builder.build();
+        Assert.assertNotNull(tokens);
+
+        tokens.skipTo(5L);
+        Assert.assertTrue(tokens.hasNext());
+        Assert.assertEquals(5L, (long) tokens.next().get());
+
+        tokens.skipTo(7L);
+        Assert.assertTrue(tokens.hasNext());
+        Assert.assertEquals(7L, (long) tokens.next().get());
+
+        tokens.skipTo(10L);
+        Assert.assertFalse(tokens.hasNext());
+        Assert.assertEquals(1L, (long) tokens.getMinimum());
+        Assert.assertEquals(9L, (long) tokens.getMaximum());
+    }
+
+    @Test
+    public void testMergingMultipleIterators()
+    {
+        RangeUnionIterator.Builder<Long, Token> builderA = RangeUnionIterator.builder();
+
+        builderA.add(new LongIterator(new long[] { 1L, 3L, 5L }));
+        builderA.add(new LongIterator(new long[] { 8L, 10L, 12L }));
+
+        RangeUnionIterator.Builder<Long, Token> builderB = RangeUnionIterator.builder();
+
+        builderB.add(new LongIterator(new long[] { 7L, 9L, 11L }));
+        builderB.add(new LongIterator(new long[] { 2L, 4L, 6L }));
+
+        RangeIterator<Long, Token> union = RangeUnionIterator.build(Arrays.asList(builderA.build(), builderB.build()));
+        Assert.assertEquals(convert(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L, 12L), convert(union));
+    }
+
+    @Test
+    public void testRangeIterator()
+    {
+        LongIterator tokens = new LongIterator(new long[] { 0L, 1L, 2L, 3L });
+
+        Assert.assertEquals(0L, (long) tokens.getMinimum());
+        Assert.assertEquals(3L, (long) tokens.getMaximum());
+
+        for (int i = 0; i <= 3; i++)
+        {
+            Assert.assertTrue(tokens.hasNext());
+            Assert.assertEquals(i, (long) tokens.getCurrent());
+            Assert.assertEquals(i, (long) tokens.next().get());
+        }
+
+        tokens = new LongIterator(new long[] { 0L, 1L, 3L, 5L });
+
+        Assert.assertEquals(3L, (long) tokens.skipTo(2L).get());
+        Assert.assertTrue(tokens.hasNext());
+        Assert.assertEquals(3L, (long) tokens.getCurrent());
+        Assert.assertEquals(3L, (long) tokens.next().get());
+
+        Assert.assertEquals(5L, (long) tokens.skipTo(5L).get());
+        Assert.assertTrue(tokens.hasNext());
+        Assert.assertEquals(5L, (long) tokens.getCurrent());
+        Assert.assertEquals(5L, (long) tokens.next().get());
+
+        LongIterator empty = new LongIterator(new long[0]);
+
+        Assert.assertNull(empty.skipTo(3L));
+        Assert.assertFalse(empty.hasNext());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java b/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
index 29ad387..9cac007 100644
--- a/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
@@ -190,9 +190,10 @@ public class SSTableFlushObserverTest
         }
 
         @Override
-        public void nextCell(ColumnData cell)
+        public void nextUnfilteredCluster(Unfiltered row)
         {
-            rows.put(currentKey, (Cell) cell);
+            if (row.isRow())
+                ((Row) row).forEach((c) -> rows.put(currentKey, (Cell) c));
         }
 
         @Override


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

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/resources/tokenization/apache_license_header.txt
----------------------------------------------------------------------
diff --git a/test/resources/tokenization/apache_license_header.txt b/test/resources/tokenization/apache_license_header.txt
new file mode 100644
index 0000000..d973dce
--- /dev/null
+++ b/test/resources/tokenization/apache_license_header.txt
@@ -0,0 +1,16 @@
+/*
+ * 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.
+ */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/resources/tokenization/ja_jp_1.txt
----------------------------------------------------------------------
diff --git a/test/resources/tokenization/ja_jp_1.txt b/test/resources/tokenization/ja_jp_1.txt
new file mode 100644
index 0000000..1a0a198
--- /dev/null
+++ b/test/resources/tokenization/ja_jp_1.txt
@@ -0,0 +1 @@
+古写本は題名の記されていないものも多く、記されている場合であっても内容はさまざまである。『源氏物語』の場合は冊子の標題として「源氏物語」ないしそれに相当する物語全体の標題が記されている場合よりも、それぞれの帖名が記されていることが少なくない。こうした経緯から、現在において一般に『源氏物語』と呼ばれているこの物語が書かれた当時の題名が何であったのかは明らかではない。古い時代の写本や注釈書などの文献に記されている名称は大きく以下の系統に分かれる。
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/resources/tokenization/ja_jp_2.txt
----------------------------------------------------------------------
diff --git a/test/resources/tokenization/ja_jp_2.txt b/test/resources/tokenization/ja_jp_2.txt
new file mode 100644
index 0000000..278b4fd
--- /dev/null
+++ b/test/resources/tokenization/ja_jp_2.txt
@@ -0,0 +1,2 @@
+中野幸一編『常用 源氏物語要覧』武蔵野書院、1997年(平成9年)。 ISBN 4-8386-0383-5
+その他にCD-ROM化された本文検索システムとして次のようなものがある。
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/resources/tokenization/lorem_ipsum.txt
----------------------------------------------------------------------
diff --git a/test/resources/tokenization/lorem_ipsum.txt b/test/resources/tokenization/lorem_ipsum.txt
new file mode 100644
index 0000000..14a4477
--- /dev/null
+++ b/test/resources/tokenization/lorem_ipsum.txt
@@ -0,0 +1 @@
+"Lorem ipsum dolor sit amet, consectetur adipiscing elit, sed do eiusmod tempor incididunt ut labore et dolore magna aliqua. Ut enim ad minim veniam, quis nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo consequat. Duis aute irure dolor in reprehenderit in voluptate velit esse cillum dolore eu fugiat nulla pariatur. Excepteur sint occaecat cupidatat non proident, sunt in culpa qui officia deserunt mollit anim id est laborum."
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/resources/tokenization/ru_ru_1.txt
----------------------------------------------------------------------
diff --git a/test/resources/tokenization/ru_ru_1.txt b/test/resources/tokenization/ru_ru_1.txt
new file mode 100644
index 0000000..c19a9be
--- /dev/null
+++ b/test/resources/tokenization/ru_ru_1.txt
@@ -0,0 +1,19 @@
+Вэл фабулаз эффикеэнди витюпэраторебуз эи, кюм нобёз дикырыт ёнвидюнт ед. Ючю золэт ийжквюы эа, нык но элитр волуптюа пэркёпитюр. Ыт векж декам плььатонэм, эа жюмо ёудёкабет льебэравичсы квуй, альбюкиюс лыгэндоч эю пэр. Еюж ед аутым нюмквуам тебиквюэ, эи амэт дэбыт нюлльам квюо. Ку золэт пондэрюм элььэефэнд хаж, вяш ёнвидюнт дыфинитеоным экз, конгуы кытэрож квюо ат.
+
+Ад фиэрэнт ыкжплььикари нык, ут дольорэ емпэтюсъ зыд. Зыд ажжюм пэржыкюти жкряпшэрит эю, ыюм ан витаэ аляквюид дяшзынтиыт. Вэл квюандо ридэнж эю. Еюж жэмпэр конклььюжионэмквуэ нэ.
+
+Ку хёз порро тамквюам плььатонэм, льаборэ ыпикурэи вэл ты. Но ентэгры компльыктётюр мэя, дуо жанктюч дэльэнйт льебэравичсы нэ. Эжт фалля пропрёаы эю, эож вэрыар ёнэрмйщ ан. Мюндй контынтёонэж прё ат. Эрож копиожаы пытынтёюм шэа эи.
+
+Но мэль омниюм рэпудёандаэ. Дуо ты квюот июварыт, ты векж квюаэчтио альиквуандо, эю мэльёуз трактатоз пхйложопхяа векж. Ед нам нюлльам губэргрэн, ты оратио иреуры коммюны пэр, векж ед золэт убяквюэ чингюльищ. Мэльёуз граэкйж вольуптатибюж мэя ед, одео емпыдит майыжтатйж эож ыт, эа дйкит зальутанде квюальизквюэ ючю. Йн еюж анкилльаы аккоммодары, ан выльёт оптёон ывыртятюр вэл.
+
+Йн граэко дычэрунт мандамюч мыа. Но про щольыат примич, нык ан этёам дольорэ элььэефэнд. Ыам нэ квюандо нощтыр. Ныморэ дикунт ад хаж, хаж квюод дёко эуежмод ты, амэт дыфинитеоным еюж ыт. Эжт ад апэряря апыирёан, кюм зальы рэктэквуэ нэ.
+
+Эи эрюдитя факёльиси еюж, ыам дольорэ фабулаз вокябюч ат. Про опортэры азжюывырит йн. Мовэт аюдиам ючю эю, нэ едквюэ пэркйпет квюальизквюэ хёз. Эа кюм этёам ырант граэкы. Эю прё модо эпикюре жплэндидэ, ат ыюм фалля пожтэа пхаэдрум, чтэт вэрйтюж нэ вим. Конгуы оратио лобортис ут кюм. Нобёз опортэат но жят, вэрйтюж пэркйпет мальюизчыт квуй ан.
+
+Мацим ютенам рыфэррэнтур вим ат. Ку квюо квюач дигнижжим, одео жольюта тебиквюэ мыа ыт. Ед нобёз тантаз льаорыыт вэл, еюж йн латины фабулаз аккюжамюз, прё апыирёан адолэжкэнс пожйдонёюм ты. Консэквюат котёдиэквюэ ыюм ан, хёз ут хабымуч ыпикурэи чэнзэрет. Ат квуй дэбыт вирйз, нам эю ыльит фабыллас дэлььякатезшимя. Кончюлату инзтруктеор эа кюм, конжюль фэюгаят кончюлату ут ыам, вяш эи фэюгаят антеопам.
+
+Юллюм оратио консэквюат ут вэл, выльёт рыпудяары хэндрэрет эю прё. Унюм ыкчпэтында торквюатоз ад векж. Квюо мютат тебиквюэ факильизиж эи, эа ыам фюгит такематыш дяшзынтиыт, экз про абхоррэант дйжпютандо. Ку хаж льабятюр эрепюят, нолюёжжэ ёудёкабет пэр эю. Тота долорюм азжюывырит прё ут, нык зальы элитр дикырыт эю. Ед дуо ыкжплььикари мныжаркхюм конклььюжионэмквуэ.
+
+Кончюлату азжюывырит нэ зыд. Вэл но квуым граэкйж юрбанйтаж. Про эффякиантур дэфянятйоныс ут, зюаз эрат конкыптам векж эю. Юллюм зюжкепиантюр экз прё, оратио нонумй орнатюс эи эож. Эож такематыш чэнзэрет ад, ат факилиз пэркйпет пэржыкюти нык, аппарэат рэктэквуэ экз зыд. Кюм йн вёвындо дэтракто окюррырэт.
+
+Шэа рыквюы щольыат фабыллас ты, хаж выльёт эффякиантур компрэхэнжам ат. Ты мэя эзшэ ажжюм апыирёан, ат докэндё конкыптам еюж. Ножтрюд жанктюч ывыртятюр ты вяш, но примич промпта пэрчыквюэрёж дуо. Выро мютат омнэжквюы ыам эю.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/resources/tokenization/top_visited_domains.txt
----------------------------------------------------------------------
diff --git a/test/resources/tokenization/top_visited_domains.txt b/test/resources/tokenization/top_visited_domains.txt
new file mode 100644
index 0000000..0238c36
--- /dev/null
+++ b/test/resources/tokenization/top_visited_domains.txt
@@ -0,0 +1,3 @@
+google.com facebook.com youtube.com yahoo.com baidu.com amazon.com wikipedia.org taobao.com twitter.com Qq.com google.co.in apple.com
+
+http://www.alexa.com/topsites
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/resources/tokenization/zn_tw_1.txt
----------------------------------------------------------------------
diff --git a/test/resources/tokenization/zn_tw_1.txt b/test/resources/tokenization/zn_tw_1.txt
new file mode 100644
index 0000000..7e1e545
--- /dev/null
+++ b/test/resources/tokenization/zn_tw_1.txt
@@ -0,0 +1,19 @@
+銌鞁鬿 蝑蝞蝢 砫粍紞 顃餭 蜭, 趍跠跬 鏀顝饇 轞騹鼚 曨曣 慔 巆忁 嫷 惝掭掝 鑳鱨鱮, 滆 浶洯浽 牣犿玒 嶕憱撏 駇僾 憱撏 硾 湹渵焲 鋧鋓頠 匊呥 犌犐瑆 翣聜蒢 蜙, 毼 噳墺 耇胇赲 稨窨箌 埱娵徖, 鍆錌雔 熩熝犚 庲悊 槄 銌鞁鬿 烍烚珜 疿疶砳 魆 糑縒 魦 萴葂 貄趎跰 萰葍萯, 嗛嗕塨 礂簅縭 婜孲 跣, 楋 澭濇 嗢嗂塝 姌弣抶 曋橪橤
+
+崺崸 獧瞝瞣 牣犿玒 嫷, 墆 齴讘麡 毊灚襳 毚丮厹 甿虮 箯 埱娵 脀蚅蚡 礯籔羻 鈁陾靰, 垼娕 螏螉褩 竀篴臌 槶, 鵳齖齘 驐鷑鷩 絒翗腏 輗 嬦憼 耜僇鄗 訬軗郲 舿萐菿 頠餈 槶, 抰枅 嬃 軹軦軵 鸙讟钃 椵楘溍 渳湥牋 蔝蓶蓨 跜軥 嫀, 砯砨 嗢 鄨鎷闒 縓罃蔾, 鍹餳駷 玝甿虮 熩熝犚 碡碙 銇
+
+瞝瞣 莃荶衒 碄碆碃 樆樦潏 穊, 枲柊氠 婰婜孲 踣 繗繓 犈犆犅 溗煂獂 儋圚 餀, 蟷蠉蟼 禒箈箑 牬玾 槶 玾珆玸 錖霒馞 撖 姴怤 犆犅 鳻嶬幧 誁趏跮 墐
+
+墐 鬵鵛嚪 圩芰敔 蒝蒧蓏 餳駷, 葮 廘榙 斶檎檦 謺貙蹖 澉 駍駔鳿 蒝蒧蓏 蔊蓴蔖 垌壴, 煻 垺垼娕 簻臗藱 笓粊 絼 騉鬵 樛槷殦 縸縩薋 巕氍爟, 璸瓁穟 鯦鯢鯡 罫蓱蒆 齥廲 滘 鯠鯦 噮噦噞 忁曨曣 釂鱞鸄 鉌 寔嵒 葮 瀿犨皫 顤鰩鷎, 憢 蔏蔍蓪 柦柋牬 玝甿虮 偢偣
+
+嗂 蒏葝葮 鋱鋟鋈 鬄鵊鵙 繖藒 毚丮 匢奾灱 枲柊氠 椵楘溍 斠, 鬄鵊 鼏噳墺 巕氍爟 鋟, 鳱 鵳齖齘 雥齆犪 騧騜 轞騹鼚 溗煂獂 諙 岵帔, 煻 廦廥彋 繠繗繓 馦騧騜 齖齘 煘煓 喥喓堮 軹軦軵 壿, 斖蘱 酳 圛嬖嬨 姛帡恦, 摿斠榱 櫧櫋瀩 廅愮揫 驧鬤 跾
+
+綒 鞻饙騴 萆覕貹 稘稒稕 稢綌 笢笣紽 磃箹糈 瑽 氕厊, 剆坲 禖 鶀嚵巆 枅杺枙, 郔镺陯 烗猀珖 伒匢 殟 憢 箛箙 馺骱魡 潧潣瑽 觶譈譀 塝 豥趍 捘栒毤 幨懅憴 稘稒稕, 撖撱暲 駓駗鴀 鄻鎟霣 蝯 訑紱 縢羱 槏殟殠 浘涀缹 鄻鎟霣 輘, 籺籿 媝寔嵒 樧槧樈 焟硱筎 瞂
+
+蚔趵郚 碄碆碃 幋 璻甔 輘 裧頖 簎艜薤 鑤仜伒 誽賚賧 淠淉 鄜酳銪 炾笀耔 椵楘溍 魡 疿疶砳 趡趛踠 躨钀钁 馺 哤垽 庌弝彶 譋轐鏕 毄滱漮 踣 墡嬇, 賗 鯦鯢鯡 齈龘墻 輘輠 蕡蕇蕱 襛襡襙 隒雸頍 紒翀 楋, 殠漊 皾籈譧 磩磟窱 狅妵妶 榎
+
+釂鱞 禠 袟袘觕 餈餖駜 椵楘溍 銈 欿殽 鬵鵛嚪 鬎鯪鯠 礂簅縭, 彃 嶝仉圠 裍裚詷 莃荶 茺苶 趍跠跬 燚璒瘭 廲籗糴 殠 魦魵 姛帡恦 賌輈鄍 沀皯竻 墏, 橁橖澭 牣犿玒 捃挸栚 酳 劻穋 噮噦噞 獧瞝瞣 釂鱞 暕, 蝺 葝葮 壾嵷幓 褣諝趥
+
+跿 鮛鮥鴮 燲獯璯 鵵鵹鵿 唗哱 蓪 塛嫆嫊 邆錉霋 哤垽, 瀁瀎 馺骱魡 鏾鐇闠 闟顣飁 墆, 壾嵷幓 摬摙敳 鵳齖齘 歅 鋄銶 澂 櫞氌瀙 忕汌卣 蠁襏 斶檎檦 觶譈譀 釪傛 瑽, 觾韄鷡 輐銛靾 廞 袚觙 剆坲姏 鼏噳墺 榯槄 觢, 榎 鷃黫鼱 蛚袲褁 闟顣飁 饙騴, 諙踣踙 齸圞趲 鄜 鶾鷃 驐鷑鷩 禒箈箑 痵 娭屔, 蓨蝪 譋轐鏕 蔪蓩蔮 楋
+
+褅褌諃 蛃袚觙 傎圌媔 侹厗 榃, 緦 恦拻 杍肜阰 軥軱逴 緷 摲摓 郔镺陯 揈敜敥, 誙賗跿 彔抳抰 袀豇貣 蜬蝁 榎 傎圌 圛嬖嬨 鑴鱱爧 潣, 枲柊 誙賗跿 貵趀跅 鮂鮐嚃 溿 禖 笓粊 齴讘麡 漻漍犕 趡趛踠, 廞 騩鰒鰔 峷敊浭 烒珛
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index e375990..51db4cd 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -21,6 +21,9 @@ import java.io.File;
 import java.io.IOException;
 import java.util.*;
 
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.index.sasi.SASIIndex;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
 import org.junit.After;
 import org.junit.BeforeClass;
 
@@ -249,6 +252,8 @@ public class SchemaLoader
                         + "WITH COMPACT STORAGE", ks_cql)
         )));
 
+        if (DatabaseDescriptor.getPartitioner() instanceof Murmur3Partitioner)
+            schema.add(KeyspaceMetadata.create("sasi", KeyspaceParams.simpleTransient(1), Tables.of(sasiCFMD("sasi", "test_cf"), clusteringSASICFMD("sasi", "clustering_test_cf"))));
 
         if (Boolean.parseBoolean(System.getProperty("cassandra.test.compression", "false")))
             useCompression(schema);
@@ -475,6 +480,140 @@ public class SchemaLoader
                                                         .compression(getCompressionParameters());
     }
 
+    public static CFMetaData sasiCFMD(String ksName, String cfName)
+    {
+        CFMetaData cfm = CFMetaData.Builder.create(ksName, cfName)
+                                           .addPartitionKey("id", UTF8Type.instance)
+                                           .addRegularColumn("first_name", UTF8Type.instance)
+                                           .addRegularColumn("last_name", UTF8Type.instance)
+                                           .addRegularColumn("age", Int32Type.instance)
+                                           .addRegularColumn("height", Int32Type.instance)
+                                           .addRegularColumn("timestamp", LongType.instance)
+                                           .addRegularColumn("address", UTF8Type.instance)
+                                           .addRegularColumn("score", DoubleType.instance)
+                                           .addRegularColumn("comment", UTF8Type.instance)
+                                           .addRegularColumn("comment_suffix_split", UTF8Type.instance)
+                                           .addRegularColumn("/output/full-name/", UTF8Type.instance)
+                                           .addRegularColumn("/data/output/id", UTF8Type.instance)
+                                           .addRegularColumn("first_name_prefix", UTF8Type.instance)
+                                           .build();
+
+        cfm.indexes(cfm.getIndexes()
+                        .with(IndexMetadata.fromSchemaMetadata("first_name", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                        {{
+                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                            put(IndexTarget.TARGET_OPTION_NAME, "first_name");
+                            put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
+                        }}))
+                        .with(IndexMetadata.fromSchemaMetadata("last_name", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                        {{
+                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                            put(IndexTarget.TARGET_OPTION_NAME, "last_name");
+                            put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
+                        }}))
+                        .with(IndexMetadata.fromSchemaMetadata("age", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                        {{
+                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                            put(IndexTarget.TARGET_OPTION_NAME, "age");
+
+                        }}))
+                        .with(IndexMetadata.fromSchemaMetadata("timestamp", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                        {{
+                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                            put(IndexTarget.TARGET_OPTION_NAME, "timestamp");
+                            put("mode", OnDiskIndexBuilder.Mode.SPARSE.toString());
+
+                        }}))
+                        .with(IndexMetadata.fromSchemaMetadata("address", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                        {{
+                            put("analyzer_class", "org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer");
+                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                            put(IndexTarget.TARGET_OPTION_NAME, "address");
+                            put("mode", OnDiskIndexBuilder.Mode.PREFIX.toString());
+                            put("case_sensitive", "false");
+                        }}))
+                        .with(IndexMetadata.fromSchemaMetadata("score", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                        {{
+                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                            put(IndexTarget.TARGET_OPTION_NAME, "score");
+                        }}))
+                        .with(IndexMetadata.fromSchemaMetadata("comment", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                        {{
+                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                            put(IndexTarget.TARGET_OPTION_NAME, "comment");
+                            put("analyzed", "true");
+                        }}))
+                        .with(IndexMetadata.fromSchemaMetadata("comment_suffix_split", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                        {{
+                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                            put(IndexTarget.TARGET_OPTION_NAME, "comment_suffix_split");
+                            put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
+                            put("analyzed", "false");
+                        }}))
+                        .with(IndexMetadata.fromSchemaMetadata("output_full_name", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                        {{
+                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                            put(IndexTarget.TARGET_OPTION_NAME, "/output/full-name/");
+                            put("analyzed", "true");
+                            put("analyzer_class", "org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer");
+                            put("case_sensitive", "false");
+                        }}))
+                        .with(IndexMetadata.fromSchemaMetadata("data_output_id", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                        {{
+                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                            put(IndexTarget.TARGET_OPTION_NAME, "/data/output/id");
+                            put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
+                        }}))
+                        .with(IndexMetadata.fromSchemaMetadata("first_name_prefix", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                        {{
+                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                            put(IndexTarget.TARGET_OPTION_NAME, "first_name_prefix");
+                            put("analyzed", "true");
+                            put("tokenization_normalize_lowercase", "true");
+                        }})));
+
+        return cfm;
+    }
+
+    public static CFMetaData clusteringSASICFMD(String ksName, String cfName)
+    {
+        CFMetaData cfm = CFMetaData.Builder.create(ksName, cfName)
+                                           .addPartitionKey("name", UTF8Type.instance)
+                                           .addClusteringColumn("location", UTF8Type.instance)
+                                           .addClusteringColumn("age", Int32Type.instance)
+                                           .addRegularColumn("height", Int32Type.instance)
+                                           .addRegularColumn("score", DoubleType.instance)
+                                           .build();
+
+        cfm.indexes(cfm.getIndexes()
+                       .with(IndexMetadata.fromSchemaMetadata("location", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                       {{
+                           put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                           put(IndexTarget.TARGET_OPTION_NAME, "location");
+                           put("mode", OnDiskIndexBuilder.Mode.PREFIX.toString());
+                       }}))
+                       .with(IndexMetadata.fromSchemaMetadata("age", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                       {{
+                           put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                           put(IndexTarget.TARGET_OPTION_NAME, "age");
+                           put("mode", OnDiskIndexBuilder.Mode.PREFIX.toString());
+                       }}))
+                       .with(IndexMetadata.fromSchemaMetadata("height", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                       {{
+                           put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                           put(IndexTarget.TARGET_OPTION_NAME, "height");
+                           put("mode", OnDiskIndexBuilder.Mode.PREFIX.toString());
+                       }}))
+                       .with(IndexMetadata.fromSchemaMetadata("score", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                       {{
+                           put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                           put(IndexTarget.TARGET_OPTION_NAME, "score");
+                           put("mode", OnDiskIndexBuilder.Mode.PREFIX.toString());
+                       }})));
+
+        return cfm;
+    }
+
     public static CompressionParams getCompressionParameters()
     {
         return getCompressionParameters(null);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java b/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java
index 54d39b1..2529de1 100644
--- a/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java
+++ b/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.concurrent.Callable;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -29,6 +30,7 @@ import org.junit.Test;
 import org.apache.cassandra.cache.KeyCacheKey;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.index.Index;
 import org.apache.cassandra.metrics.CacheMetrics;
 import org.apache.cassandra.metrics.CassandraMetricsRegistry;
 import org.apache.cassandra.service.CacheService;
@@ -407,7 +409,7 @@ public class KeyCacheCqlTest extends CQLTester
         if (index != null)
         {
             StorageService.instance.disableAutoCompaction(KEYSPACE, table + '.' + index);
-            Keyspace.open(KEYSPACE).getColumnFamilyStore(table).indexManager.getIndexByName(index).getBlockingFlushTask().call();
+            triggerBlockingFlush(Keyspace.open(KEYSPACE).getColumnFamilyStore(table).indexManager.getIndexByName(index));
         }
 
         for (int i = 0; i < 100; i++)
@@ -432,7 +434,7 @@ public class KeyCacheCqlTest extends CQLTester
             {
                 Keyspace.open(KEYSPACE).getColumnFamilyStore(table).forceFlush().get();
                 if (index != null)
-                    Keyspace.open(KEYSPACE).getColumnFamilyStore(table).indexManager.getIndexByName(index).getBlockingFlushTask().call();
+                    triggerBlockingFlush(Keyspace.open(KEYSPACE).getColumnFamilyStore(table).indexManager.getIndexByName(index));
             }
         }
     }
@@ -464,4 +466,12 @@ public class KeyCacheCqlTest extends CQLTester
         Assert.assertEquals(0L, metrics.requests.getCount());
         Assert.assertEquals(0L, metrics.size.getValue().longValue());
     }
+
+    private static void triggerBlockingFlush(Index index) throws Exception
+    {
+        assert index != null;
+        Callable<?> flushTask = index.getBlockingFlushTask();
+        if (flushTask != null)
+            flushTask.call();
+    }
 }


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

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/lib/licenses/jflex-1.6.0.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/jflex-1.6.0.txt b/lib/licenses/jflex-1.6.0.txt
new file mode 100644
index 0000000..50086f8
--- /dev/null
+++ b/lib/licenses/jflex-1.6.0.txt
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!) The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   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.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/lib/licenses/primitive-1.0.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/primitive-1.0.txt b/lib/licenses/primitive-1.0.txt
new file mode 100644
index 0000000..50086f8
--- /dev/null
+++ b/lib/licenses/primitive-1.0.txt
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!) The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   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.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/lib/licenses/snowball-stemmer-1.3.0.581.1.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/snowball-stemmer-1.3.0.581.1.txt b/lib/licenses/snowball-stemmer-1.3.0.581.1.txt
new file mode 100644
index 0000000..50086f8
--- /dev/null
+++ b/lib/licenses/snowball-stemmer-1.3.0.581.1.txt
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!) The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   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.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/lib/primitive-1.0.jar
----------------------------------------------------------------------
diff --git a/lib/primitive-1.0.jar b/lib/primitive-1.0.jar
new file mode 100644
index 0000000..288daa0
Binary files /dev/null and b/lib/primitive-1.0.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/lib/snowball-stemmer-1.3.0.581.1.jar
----------------------------------------------------------------------
diff --git a/lib/snowball-stemmer-1.3.0.581.1.jar b/lib/snowball-stemmer-1.3.0.581.1.jar
new file mode 100644
index 0000000..92189b9
Binary files /dev/null and b/lib/snowball-stemmer-1.3.0.581.1.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 116d92e..2a2719a 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -1964,5 +1964,10 @@ public class DatabaseDescriptor
     public static void setEncryptionContext(EncryptionContext ec)
     {
         encryptionContext = ec;
-    } 
+    }
+
+    public static int searchConcurrencyFactor()
+    {
+        return Integer.valueOf(System.getProperty("cassandra.search_concurrency_factor", "1"));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/db/ColumnIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnIndex.java b/src/java/org/apache/cassandra/db/ColumnIndex.java
index 749c155..930fc05 100644
--- a/src/java/org/apache/cassandra/db/ColumnIndex.java
+++ b/src/java/org/apache/cassandra/db/ColumnIndex.java
@@ -152,9 +152,9 @@ public class ColumnIndex
 
             UnfilteredSerializer.serializer.serialize(unfiltered, header, writer, pos - previousRowStart, version);
 
-            // notify observers about each new cell added to the row
-            if (!observers.isEmpty() && unfiltered.isRow())
-                ((Row) unfiltered).stream().forEach(cell -> observers.forEach((o) -> o.nextCell(cell)));
+            // notify observers about each new row
+            if (!observers.isEmpty())
+                observers.forEach((o) -> o.nextUnfilteredCluster(unfiltered));
 
             lastClustering = unfiltered.clustering();
             previousRowStart = pos;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/db/filter/RowFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/RowFilter.java b/src/java/org/apache/cassandra/db/filter/RowFilter.java
index 4cd2f64..c234fc9 100644
--- a/src/java/org/apache/cassandra/db/filter/RowFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java
@@ -171,6 +171,11 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         return withNewExpressions(newExpressions);
     }
 
+    public RowFilter withoutExpressions()
+    {
+        return withNewExpressions(Collections.emptyList());
+    }
+
     protected abstract RowFilter withNewExpressions(List<Expression> expressions);
 
     public boolean isEmpty()
@@ -312,7 +317,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         // Note: the order of this enum matter, it's used for serialization
         protected enum Kind { SIMPLE, MAP_EQUALITY, THRIFT_DYN_EXPR, CUSTOM }
 
-        abstract Kind kind();
+        protected abstract Kind kind();
         protected final ColumnDefinition column;
         protected final Operator operator;
         protected final ByteBuffer value;
@@ -689,7 +694,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         }
 
         @Override
-        Kind kind()
+        protected Kind kind()
         {
             return Kind.SIMPLE;
         }
@@ -782,7 +787,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         }
 
         @Override
-        Kind kind()
+        protected Kind kind()
         {
             return Kind.MAP_EQUALITY;
         }
@@ -833,7 +838,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         }
 
         @Override
-        Kind kind()
+        protected Kind kind()
         {
             return Kind.THRIFT_DYN_EXPR;
         }
@@ -883,7 +888,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
                                          .customExpressionValueType());
         }
 
-        Kind kind()
+        protected Kind kind()
         {
             return Kind.CUSTOM;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index 7349167..e4a03de 100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@ -665,6 +665,11 @@ public class SecondaryIndexManager implements IndexRegistry
         return selected;
     }
 
+    public Optional<Index> getBestIndexFor(RowFilter.Expression expression)
+    {
+        return indexes.values().stream().filter((i) -> i.supportsExpression(expression.column(), expression.operator())).findFirst();
+    }
+
     /**
      * Called at write time to ensure that values present in the update
      * are valid according to the rules of all registered indexes which
@@ -1040,6 +1045,12 @@ public class SecondaryIndexManager implements IndexRegistry
 
     private static void executeAllBlocking(Stream<Index> indexers, Function<Index, Callable<?>> function)
     {
+        if (function == null)
+        {
+            logger.error("failed to flush indexes: {} because flush task is missing.", indexers);
+            return;
+        }
+
         List<Future<?>> waitFor = new ArrayList<>();
         indexers.forEach(indexer -> {
             Callable<?> task = function.apply(indexer);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/SASIIndex.java b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
new file mode 100644
index 0000000..d69b440
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
@@ -0,0 +1,288 @@
+/*
+ * 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.index.sasi;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+import java.util.function.BiFunction;
+
+import com.googlecode.concurrenttrees.common.Iterables;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.filter.RowFilter;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.index.Index;
+import org.apache.cassandra.index.IndexRegistry;
+import org.apache.cassandra.index.SecondaryIndexBuilder;
+import org.apache.cassandra.index.internal.CassandraIndex;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.disk.PerSSTableIndexWriter;
+import org.apache.cassandra.index.sasi.plan.QueryPlan;
+import org.apache.cassandra.index.transactions.IndexTransaction;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableFlushObserver;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.notifications.*;
+import org.apache.cassandra.schema.IndexMetadata;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+public class SASIIndex implements Index, INotificationConsumer
+{
+    private static class SASIIndexBuildingSupport implements IndexBuildingSupport
+    {
+        public SecondaryIndexBuilder getIndexBuildTask(ColumnFamilyStore cfs,
+                                                       Set<Index> indexes,
+                                                       Collection<SSTableReader> sstablesToRebuild)
+        {
+            NavigableMap<SSTableReader, Map<ColumnDefinition, ColumnIndex>> sstables = new TreeMap<>((a, b) -> {
+                return Integer.compare(a.descriptor.generation, b.descriptor.generation);
+            });
+
+            indexes.stream()
+                   .filter((i) -> i instanceof SASIIndex)
+                   .forEach((i) -> {
+                       SASIIndex sasi = (SASIIndex) i;
+                       sstablesToRebuild.stream()
+                                        .filter((sstable) -> !sasi.index.hasSSTable(sstable))
+                                        .forEach((sstable) -> {
+                                            Map<ColumnDefinition, ColumnIndex> toBuild = sstables.get(sstable);
+                                            if (toBuild == null)
+                                                sstables.put(sstable, (toBuild = new HashMap<>()));
+
+                                            toBuild.put(sasi.index.getDefinition(), sasi.index);
+                                        });
+                   });
+
+            return new SASIIndexBuilder(cfs, sstables);
+        }
+    }
+
+    private static final SASIIndexBuildingSupport INDEX_BUILDER_SUPPORT = new SASIIndexBuildingSupport();
+
+    private final ColumnFamilyStore baseCfs;
+    private final IndexMetadata config;
+    private final ColumnIndex index;
+
+    public SASIIndex(ColumnFamilyStore baseCfs, IndexMetadata config)
+    {
+        this.baseCfs = baseCfs;
+        this.config = config;
+
+        ColumnDefinition column = CassandraIndex.parseTarget(baseCfs.metadata, config).left;
+        this.index = new ColumnIndex(baseCfs.metadata.getKeyValidator(), column, config);
+
+        baseCfs.getTracker().subscribe(this);
+    }
+
+    public static Map<String, String> validateOptions(Map<String, String> options)
+    {
+        return Collections.emptyMap();
+    }
+
+    public void register(IndexRegistry registry)
+    {
+        registry.registerIndex(this);
+    }
+
+    public IndexMetadata getIndexMetadata()
+    {
+        return config;
+    }
+
+    public Callable<?> getInitializationTask()
+    {
+        return null;
+    }
+
+    public Callable<?> getMetadataReloadTask(IndexMetadata indexMetadata)
+    {
+        return null;
+    }
+
+    public Callable<?> getBlockingFlushTask()
+    {
+        return null; // SASI indexes are flushed along side memtable
+    }
+
+    public Callable<?> getInvalidateTask()
+    {
+        return getTruncateTask(FBUtilities.timestampMicros());
+    }
+
+    public Callable<?> getTruncateTask(long truncatedAt)
+    {
+        return () -> {
+            index.dropData(truncatedAt);
+            return null;
+        };
+    }
+
+    public boolean shouldBuildBlocking()
+    {
+        return true;
+    }
+
+    public Optional<ColumnFamilyStore> getBackingTable()
+    {
+        return Optional.empty();
+    }
+
+    public boolean indexes(PartitionColumns columns)
+    {
+        return columns.contains(index.getDefinition());
+    }
+
+    public boolean dependsOn(ColumnDefinition column)
+    {
+        return index.getDefinition().compareTo(column) == 0;
+    }
+
+    public boolean supportsExpression(ColumnDefinition column, Operator operator)
+    {
+        return dependsOn(column);
+    }
+
+    public AbstractType<?> customExpressionValueType()
+    {
+        return null;
+    }
+
+    public RowFilter getPostIndexQueryFilter(RowFilter filter)
+    {
+        return filter.withoutExpressions();
+    }
+
+    public long getEstimatedResultRows()
+    {
+        // this is temporary (until proper QueryPlan is integrated into Cassandra)
+        // and allows us to priority SASI indexes if any in the query since they
+        // are going to be more efficient, to query and intersect, than built-in indexes.
+        return Long.MIN_VALUE;
+    }
+
+    public void validate(PartitionUpdate update) throws InvalidRequestException
+    {}
+
+    public Indexer indexerFor(DecoratedKey key, PartitionColumns columns, int nowInSec, OpOrder.Group opGroup, IndexTransaction.Type transactionType)
+    {
+        return new Indexer()
+        {
+            public void begin()
+            {}
+
+            public void partitionDelete(DeletionTime deletionTime)
+            {}
+
+            public void rangeTombstone(RangeTombstone tombstone)
+            {}
+
+            public void insertRow(Row row)
+            {
+                if (isNewData())
+                    adjustMemtableSize(index.index(key, row), opGroup);
+            }
+
+            public void updateRow(Row oldRow, Row newRow)
+            {
+                insertRow(newRow);
+            }
+
+            public void removeRow(Row row)
+            {}
+
+            public void finish()
+            {}
+
+            // we are only interested in the data from Memtable
+            // everything else is going to be handled by SSTableWriter observers
+            private boolean isNewData()
+            {
+                return transactionType == IndexTransaction.Type.UPDATE;
+            }
+
+            public void adjustMemtableSize(long additionalSpace, OpOrder.Group opGroup)
+            {
+                baseCfs.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().allocate(additionalSpace, opGroup);
+            }
+        };
+    }
+
+    public Searcher searcherFor(ReadCommand command) throws InvalidRequestException
+    {
+        CFMetaData config = command.metadata();
+        ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(config.cfId);
+        return controller -> new QueryPlan(cfs, command, DatabaseDescriptor.getRangeRpcTimeout()).execute(controller);
+    }
+
+    public SSTableFlushObserver getFlushObserver(Descriptor descriptor, OperationType opType)
+    {
+        return newWriter(baseCfs.metadata.getKeyValidator(), descriptor, Collections.singletonMap(index.getDefinition(), index), opType);
+    }
+
+    public BiFunction<PartitionIterator, ReadCommand, PartitionIterator> postProcessorFor(ReadCommand command)
+    {
+        return (partitionIterator, readCommand) -> partitionIterator;
+    }
+
+    public IndexBuildingSupport getBuildTaskSupport()
+    {
+        return INDEX_BUILDER_SUPPORT;
+    }
+
+    public void handleNotification(INotification notification, Object sender)
+    {
+        // unfortunately, we can only check the type of notification via instanceof :(
+        if (notification instanceof SSTableAddedNotification)
+        {
+            SSTableAddedNotification notice = (SSTableAddedNotification) notification;
+            index.update(Collections.<SSTableReader>emptyList(), Iterables.toList(notice.added));
+        }
+        else if (notification instanceof SSTableListChangedNotification)
+        {
+            SSTableListChangedNotification notice = (SSTableListChangedNotification) notification;
+            index.update(notice.removed, notice.added);
+        }
+        else if (notification instanceof MemtableRenewedNotification)
+        {
+            index.switchMemtable();
+        }
+    }
+
+    public ColumnIndex getIndex()
+    {
+        return index;
+    }
+
+    protected static PerSSTableIndexWriter newWriter(AbstractType<?> keyValidator,
+                                                     Descriptor descriptor,
+                                                     Map<ColumnDefinition, ColumnIndex> indexes,
+                                                     OperationType opType)
+    {
+        return new PerSSTableIndexWriter(keyValidator, descriptor, opType, indexes);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java b/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java
new file mode 100644
index 0000000..fc5b675
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java
@@ -0,0 +1,128 @@
+package org.apache.cassandra.index.sasi;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.compaction.CompactionInfo;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.index.SecondaryIndexBuilder;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.disk.PerSSTableIndexWriter;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.sstable.KeyIterator;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
+
+class SASIIndexBuilder extends SecondaryIndexBuilder
+{
+    private final ColumnFamilyStore cfs;
+    private final UUID compactionId = UUIDGen.getTimeUUID();
+
+    private final SortedMap<SSTableReader, Map<ColumnDefinition, ColumnIndex>> sstables;
+
+    private long bytesProcessed = 0;
+    private final long totalSizeInBytes;
+
+    public SASIIndexBuilder(ColumnFamilyStore cfs, SortedMap<SSTableReader, Map<ColumnDefinition, ColumnIndex>> sstables)
+    {
+        long totalIndexBytes = 0;
+        for (SSTableReader sstable : sstables.keySet())
+            totalIndexBytes += getPrimaryIndexLength(sstable);
+
+        this.cfs = cfs;
+        this.sstables = sstables;
+        this.totalSizeInBytes = totalIndexBytes;
+    }
+
+    public void build()
+    {
+        AbstractType<?> keyValidator = cfs.metadata.getKeyValidator();
+        for (Map.Entry<SSTableReader, Map<ColumnDefinition, ColumnIndex>> e : sstables.entrySet())
+        {
+            SSTableReader sstable = e.getKey();
+            Map<ColumnDefinition, ColumnIndex> indexes = e.getValue();
+
+            try (RandomAccessReader dataFile = sstable.openDataReader())
+            {
+                PerSSTableIndexWriter indexWriter = SASIIndex.newWriter(keyValidator, sstable.descriptor, indexes, OperationType.COMPACTION);
+
+                long previousKeyPosition = 0;
+                try (KeyIterator keys = new KeyIterator(sstable.descriptor, cfs.metadata))
+                {
+                    while (keys.hasNext())
+                    {
+                        if (isStopRequested())
+                            throw new CompactionInterruptedException(getCompactionInfo());
+
+                        final DecoratedKey key = keys.next();
+                        final long keyPosition = keys.getKeyPosition();
+
+                        indexWriter.startPartition(key, keyPosition);
+
+                        try
+                        {
+                            RowIndexEntry indexEntry = sstable.getPosition(key, SSTableReader.Operator.EQ);
+                            dataFile.seek(indexEntry.position + indexEntry.headerOffset());
+                            ByteBufferUtil.readWithShortLength(dataFile); // key
+
+                            try (SSTableIdentityIterator partition = new SSTableIdentityIterator(sstable, dataFile, key))
+                            {
+                                while (partition.hasNext())
+                                    indexWriter.nextUnfilteredCluster(partition.next());
+                            }
+                        }
+                        catch (IOException ex)
+                        {
+                            throw new FSReadError(ex, sstable.getFilename());
+                        }
+
+                        bytesProcessed += keyPosition - previousKeyPosition;
+                        previousKeyPosition = keyPosition;
+                    }
+
+                    completeSSTable(indexWriter, sstable, indexes.values());
+                }
+            }
+        }
+    }
+
+    public CompactionInfo getCompactionInfo()
+    {
+        return new CompactionInfo(cfs.metadata,
+                                  OperationType.INDEX_BUILD,
+                                  bytesProcessed,
+                                  totalSizeInBytes,
+                                  compactionId);
+    }
+
+    private long getPrimaryIndexLength(SSTable sstable)
+    {
+        File primaryIndex = new File(sstable.getIndexFilename());
+        return primaryIndex.exists() ? primaryIndex.length() : 0;
+    }
+
+    private void completeSSTable(PerSSTableIndexWriter indexWriter, SSTableReader sstable, Collection<ColumnIndex> indexes)
+    {
+        indexWriter.complete();
+
+        for (ColumnIndex index : indexes)
+        {
+            File tmpIndex = new File(sstable.descriptor.filenameFor(index.getComponent()));
+            if (!tmpIndex.exists()) // no data was inserted into the index for given sstable
+                continue;
+
+            index.update(Collections.<SSTableReader>emptyList(), Collections.singletonList(sstable));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java b/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java
new file mode 100644
index 0000000..7b65232
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java
@@ -0,0 +1,187 @@
+/*
+ * 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.index.sasi;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndex;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.concurrent.Ref;
+
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+import com.google.common.base.Function;
+
+public class SSTableIndex
+{
+    private final ColumnIndex columnIndex;
+    private final Ref<SSTableReader> sstableRef;
+    private final SSTableReader sstable;
+    private final OnDiskIndex index;
+    private final AtomicInteger references = new AtomicInteger(1);
+    private final AtomicBoolean obsolete = new AtomicBoolean(false);
+
+    public SSTableIndex(ColumnIndex index, File indexFile, SSTableReader referent)
+    {
+        this.columnIndex = index;
+        this.sstableRef = referent.tryRef();
+        this.sstable = sstableRef.get();
+
+        if (sstable == null)
+            throw new IllegalStateException("Couldn't acquire reference to the sstable: " + referent);
+
+        AbstractType<?> validator = columnIndex.getValidator();
+
+        assert validator != null;
+        assert indexFile.exists() : String.format("SSTable %s should have index %s.",
+                sstable.getFilename(),
+                columnIndex.getIndexName());
+
+        this.index = new OnDiskIndex(indexFile, validator, new DecoratedKeyFetcher(sstable));
+    }
+
+    public ByteBuffer minTerm()
+    {
+        return index.minTerm();
+    }
+
+    public ByteBuffer maxTerm()
+    {
+        return index.maxTerm();
+    }
+
+    public ByteBuffer minKey()
+    {
+        return index.minKey();
+    }
+
+    public ByteBuffer maxKey()
+    {
+        return index.maxKey();
+    }
+
+    public RangeIterator<Long, Token> search(Expression expression)
+    {
+        return index.search(expression);
+    }
+
+    public SSTableReader getSSTable()
+    {
+        return sstable;
+    }
+
+    public String getPath()
+    {
+        return index.getIndexPath();
+    }
+
+    public boolean reference()
+    {
+        while (true)
+        {
+            int n = references.get();
+            if (n <= 0)
+                return false;
+            if (references.compareAndSet(n, n + 1))
+                return true;
+        }
+    }
+
+    public void release()
+    {
+        int n = references.decrementAndGet();
+        if (n == 0)
+        {
+            FileUtils.closeQuietly(index);
+            sstableRef.release();
+            if (obsolete.get() || sstableRef.globalCount() == 0)
+                FileUtils.delete(index.getIndexPath());
+        }
+    }
+
+    public void markObsolete()
+    {
+        obsolete.getAndSet(true);
+        release();
+    }
+
+    public boolean isObsolete()
+    {
+        return obsolete.get();
+    }
+
+    public boolean equals(Object o)
+    {
+        return o instanceof SSTableIndex && index.getIndexPath().equals(((SSTableIndex) o).index.getIndexPath());
+    }
+
+    public int hashCode()
+    {
+        return new HashCodeBuilder().append(index.getIndexPath()).build();
+    }
+
+    public String toString()
+    {
+        return String.format("SSTableIndex(column: %s, SSTable: %s)", columnIndex.getColumnName(), sstable.descriptor);
+    }
+
+    private static class DecoratedKeyFetcher implements Function<Long, DecoratedKey>
+    {
+        private final SSTableReader sstable;
+
+        DecoratedKeyFetcher(SSTableReader reader)
+        {
+            sstable = reader;
+        }
+
+        public DecoratedKey apply(Long offset)
+        {
+            try
+            {
+                return sstable.keyAt(offset);
+            }
+            catch (IOException e)
+            {
+                throw new FSReadError(new IOException("Failed to read key from " + sstable.descriptor, e), sstable.getFilename());
+            }
+        }
+
+        public int hashCode()
+        {
+            return sstable.descriptor.hashCode();
+        }
+
+        public boolean equals(Object other)
+        {
+            return other instanceof DecoratedKeyFetcher
+                    && sstable.descriptor.equals(((DecoratedKeyFetcher) other).sstable.descriptor);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/Term.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/Term.java b/src/java/org/apache/cassandra/index/sasi/Term.java
new file mode 100644
index 0000000..8e8ceb2
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/Term.java
@@ -0,0 +1,65 @@
+/*
+ * 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.index.sasi;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder.TermSize;
+import org.apache.cassandra.index.sasi.utils.MappedBuffer;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public class Term
+{
+    protected final MappedBuffer content;
+    protected final TermSize termSize;
+
+
+    public Term(MappedBuffer content, TermSize size)
+    {
+        this.content = content;
+        this.termSize = size;
+    }
+
+    public ByteBuffer getTerm()
+    {
+        long offset = termSize.isConstant() ? content.position() : content.position() + 2;
+        int  length = termSize.isConstant() ? termSize.size : content.getShort(content.position());
+
+        return content.getPageRegion(offset, length);
+    }
+
+    public long getDataOffset()
+    {
+        long position = content.position();
+        return position + (termSize.isConstant() ? termSize.size : 2 + content.getShort(position));
+    }
+
+    public int compareTo(AbstractType<?> comparator, ByteBuffer query)
+    {
+        return compareTo(comparator, query, true);
+    }
+
+    public int compareTo(AbstractType<?> comparator, ByteBuffer query, boolean checkFully)
+    {
+        long position = content.position();
+        int padding = termSize.isConstant() ? 0 : 2;
+        int len = termSize.isConstant() ? termSize.size : content.getShort(position);
+
+        return content.comparePageTo(position + padding, checkFully ? len : Math.min(len, query.remaining()), comparator, query);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/TermIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/TermIterator.java b/src/java/org/apache/cassandra/index/sasi/TermIterator.java
new file mode 100644
index 0000000..cfa87c0
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/TermIterator.java
@@ -0,0 +1,208 @@
+/*
+ * 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.index.sasi;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.index.sasi.utils.RangeUnionIterator;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.io.util.FileUtils;
+
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.Uninterruptibles;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TermIterator extends RangeIterator<Long, Token>
+{
+    private static final Logger logger = LoggerFactory.getLogger(TermIterator.class);
+
+    private static final ThreadLocal<ExecutorService> SEARCH_EXECUTOR = new ThreadLocal<ExecutorService>()
+    {
+        public ExecutorService initialValue()
+        {
+            final String currentThread = Thread.currentThread().getName();
+            final int concurrencyFactor = DatabaseDescriptor.searchConcurrencyFactor();
+
+            logger.info("Search Concurrency Factor is set to {} for {}", concurrencyFactor, currentThread);
+
+            return (concurrencyFactor <= 1)
+                    ? MoreExecutors.newDirectExecutorService()
+                    : Executors.newFixedThreadPool(concurrencyFactor, new ThreadFactory()
+            {
+                public final AtomicInteger count = new AtomicInteger();
+
+                public Thread newThread(Runnable task)
+                {
+                    return new Thread(task, currentThread + "-SEARCH-" + count.incrementAndGet()) {{ setDaemon(true); }};
+                }
+            });
+        }
+    };
+
+    private final Expression expression;
+
+    private final RangeIterator<Long, Token> union;
+    private final Set<SSTableIndex> referencedIndexes;
+
+    private TermIterator(Expression e,
+                         RangeIterator<Long, Token> union,
+                         Set<SSTableIndex> referencedIndexes)
+    {
+        super(union.getMinimum(), union.getMaximum(), union.getCount());
+
+        this.expression = e;
+        this.union = union;
+        this.referencedIndexes = referencedIndexes;
+    }
+
+    public static TermIterator build(final Expression e, Set<SSTableIndex> perSSTableIndexes)
+    {
+        final List<RangeIterator<Long, Token>> tokens = new CopyOnWriteArrayList<>();
+        final AtomicLong tokenCount = new AtomicLong(0);
+
+        RangeIterator<Long, Token> memtableIterator = e.index.searchMemtable(e);
+        if (memtableIterator != null)
+        {
+            tokens.add(memtableIterator);
+            tokenCount.addAndGet(memtableIterator.getCount());
+        }
+
+        final Set<SSTableIndex> referencedIndexes = new CopyOnWriteArraySet<>();
+
+        try
+        {
+            final CountDownLatch latch = new CountDownLatch(perSSTableIndexes.size());
+            final ExecutorService searchExecutor = SEARCH_EXECUTOR.get();
+
+            for (final SSTableIndex index : perSSTableIndexes)
+            {
+                if (!index.reference())
+                {
+                    latch.countDown();
+                    continue;
+                }
+
+                // add to referenced right after the reference was acquired,
+                // that helps to release index if something goes bad inside of the search
+                referencedIndexes.add(index);
+
+                searchExecutor.submit((Runnable) () -> {
+                    try
+                    {
+                        e.checkpoint();
+
+                        RangeIterator<Long, Token> keyIterator = index.search(e);
+                        if (keyIterator == null)
+                        {
+                            releaseIndex(referencedIndexes, index);
+                            return;
+                        }
+
+                        tokens.add(keyIterator);
+                        tokenCount.getAndAdd(keyIterator.getCount());
+                    }
+                    catch (Throwable e1)
+                    {
+                        releaseIndex(referencedIndexes, index);
+
+                        if (logger.isDebugEnabled())
+                            logger.debug(String.format("Failed search an index %s, skipping.", index.getPath()), e1);
+                    }
+                    finally
+                    {
+                        latch.countDown();
+                    }
+                });
+            }
+
+            Uninterruptibles.awaitUninterruptibly(latch);
+
+            // checkpoint right away after all indexes complete search because we might have crossed the quota
+            e.checkpoint();
+
+            RangeIterator<Long, Token> ranges = RangeUnionIterator.build(tokens);
+            return ranges == null ? null : new TermIterator(e, ranges, referencedIndexes);
+        }
+        catch (Throwable ex)
+        {
+            // if execution quota was exceeded while opening indexes or something else happened
+            // local (yet to be tracked) indexes should be released first before re-throwing exception
+            referencedIndexes.forEach(TermIterator::releaseQuietly);
+
+            throw ex;
+        }
+    }
+
+    protected Token computeNext()
+    {
+        try
+        {
+            return union.hasNext() ? union.next() : endOfData();
+        }
+        finally
+        {
+            expression.checkpoint();
+        }
+    }
+
+    protected void performSkipTo(Long nextToken)
+    {
+        try
+        {
+            union.skipTo(nextToken);
+        }
+        finally
+        {
+            expression.checkpoint();
+        }
+    }
+
+    public void close()
+    {
+        FileUtils.closeQuietly(union);
+        referencedIndexes.forEach(TermIterator::releaseQuietly);
+        referencedIndexes.clear();
+    }
+
+    private static void releaseIndex(Set<SSTableIndex> indexes, SSTableIndex index)
+    {
+        indexes.remove(index);
+        releaseQuietly(index);
+    }
+
+    private static void releaseQuietly(SSTableIndex index)
+    {
+        try
+        {
+            index.release();
+        }
+        catch (Throwable e)
+        {
+            logger.error(String.format("Failed to release index %s", index.getPath()), e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/AbstractAnalyzer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/AbstractAnalyzer.java b/src/java/org/apache/cassandra/index/sasi/analyzer/AbstractAnalyzer.java
new file mode 100644
index 0000000..b3fdd8c
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/AbstractAnalyzer.java
@@ -0,0 +1,51 @@
+/*
+ * 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.index.sasi.analyzer;
+
+import java.nio.ByteBuffer;
+import java.text.Normalizer;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public abstract class AbstractAnalyzer implements Iterator<ByteBuffer>
+{
+    protected ByteBuffer next = null;
+
+    public ByteBuffer next()
+    {
+        return next;
+    }
+
+    public void remove()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public abstract void init(Map<String, String> options, AbstractType validator);
+
+    public abstract void reset(ByteBuffer input);
+
+    public static String normalize(String original)
+    {
+        return Normalizer.isNormalized(original, Normalizer.Form.NFC)
+                ? original
+                : Normalizer.normalize(original, Normalizer.Form.NFC);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/NoOpAnalyzer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/NoOpAnalyzer.java b/src/java/org/apache/cassandra/index/sasi/analyzer/NoOpAnalyzer.java
new file mode 100644
index 0000000..9939a13
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/NoOpAnalyzer.java
@@ -0,0 +1,54 @@
+/*
+ * 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.index.sasi.analyzer;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+/**
+ * Default noOp tokenizer. The iterator will iterate only once
+ * returning the unmodified input
+ */
+public class NoOpAnalyzer extends AbstractAnalyzer
+{
+    private ByteBuffer input;
+    private boolean hasNext = false;
+
+    public void init(Map<String, String> options, AbstractType validator)
+    {}
+
+    public boolean hasNext()
+    {
+        if (hasNext)
+        {
+            this.next = input;
+            this.hasNext = false;
+            return true;
+        }
+        return false;
+    }
+
+    public void reset(ByteBuffer input)
+    {
+        this.next = null;
+        this.input = input;
+        this.hasNext = true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzer.java b/src/java/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzer.java
new file mode 100644
index 0000000..676b304
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzer.java
@@ -0,0 +1,126 @@
+/*
+ * 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.index.sasi.analyzer;
+
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.cassandra.index.sasi.analyzer.filter.BasicResultFilters;
+import org.apache.cassandra.index.sasi.analyzer.filter.FilterPipelineBuilder;
+import org.apache.cassandra.index.sasi.analyzer.filter.FilterPipelineExecutor;
+import org.apache.cassandra.index.sasi.analyzer.filter.FilterPipelineTask;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Analyzer that does *not* tokenize the input. Optionally will
+ * apply filters for the input output as defined in analyzers options
+ */
+public class NonTokenizingAnalyzer extends AbstractAnalyzer
+{
+    private static final Logger logger = LoggerFactory.getLogger(NonTokenizingAnalyzer.class);
+
+    private static final Set<AbstractType<?>> VALID_ANALYZABLE_TYPES = new HashSet<AbstractType<?>>()
+    {{
+            add(UTF8Type.instance);
+            add(AsciiType.instance);
+    }};
+
+    private AbstractType validator;
+    private NonTokenizingOptions options;
+    private FilterPipelineTask filterPipeline;
+
+    private ByteBuffer input;
+    private boolean hasNext = false;
+
+    public void init(Map<String, String> options, AbstractType validator)
+    {
+        init(NonTokenizingOptions.buildFromMap(options), validator);
+    }
+
+    public void init(NonTokenizingOptions tokenizerOptions, AbstractType validator)
+    {
+        this.validator = validator;
+        this.options = tokenizerOptions;
+        this.filterPipeline = getFilterPipeline();
+    }
+
+    public boolean hasNext()
+    {
+        // check that we know how to handle the input, otherwise bail
+        if (!VALID_ANALYZABLE_TYPES.contains(validator))
+            return false;
+
+        if (hasNext)
+        {
+            String inputStr;
+
+            try
+            {
+                inputStr = validator.getString(input);
+                if (inputStr == null)
+                    throw new MarshalException(String.format("'null' deserialized value for %s with %s", ByteBufferUtil.bytesToHex(input), validator));
+
+                Object pipelineRes = FilterPipelineExecutor.execute(filterPipeline, inputStr);
+                if (pipelineRes == null)
+                    return false;
+
+                next = validator.fromString(normalize((String) pipelineRes));
+                return true;
+            }
+            catch (MarshalException e)
+            {
+                logger.error("Failed to deserialize value with " + validator, e);
+                return false;
+            }
+            finally
+            {
+                hasNext = false;
+            }
+        }
+
+        return false;
+    }
+
+    public void reset(ByteBuffer input)
+    {
+        this.next = null;
+        this.input = input;
+        this.hasNext = true;
+    }
+
+    private FilterPipelineTask getFilterPipeline()
+    {
+        FilterPipelineBuilder builder = new FilterPipelineBuilder(new BasicResultFilters.NoOperation());
+        if (options.isCaseSensitive() && options.shouldLowerCaseOutput())
+            builder = builder.add("to_lower", new BasicResultFilters.LowerCase());
+        if (options.isCaseSensitive() && options.shouldUpperCaseOutput())
+            builder = builder.add("to_upper", new BasicResultFilters.UpperCase());
+        if (!options.isCaseSensitive())
+            builder = builder.add("to_lower", new BasicResultFilters.LowerCase());
+        return builder.build();
+    }
+}


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

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/resources/tokenization/adventures_of_huckleberry_finn_mark_twain.txt
----------------------------------------------------------------------
diff --git a/test/resources/tokenization/adventures_of_huckleberry_finn_mark_twain.txt b/test/resources/tokenization/adventures_of_huckleberry_finn_mark_twain.txt
new file mode 100644
index 0000000..27cadc3
--- /dev/null
+++ b/test/resources/tokenization/adventures_of_huckleberry_finn_mark_twain.txt
@@ -0,0 +1,12361 @@
+
+
+The Project Gutenberg EBook of Adventures of Huckleberry Finn, Complete
+by Mark Twain (Samuel Clemens)
+
+This eBook is for the use of anyone anywhere at no cost and with almost
+no restrictions whatsoever. You may copy it, give it away or re-use
+it under the terms of the Project Gutenberg License included with this
+eBook or online at www.gutenberg.net
+
+Title: Adventures of Huckleberry Finn, Complete
+
+Author: Mark Twain (Samuel Clemens)
+
+Release Date: August 20, 2006 [EBook #76]
+
+Last Updated: October 20, 2012]
+
+Language: English
+
+
+*** START OF THIS PROJECT GUTENBERG EBOOK HUCKLEBERRY FINN ***
+
+Produced by David Widger
+
+
+
+
+
+ADVENTURES
+
+OF
+
+HUCKLEBERRY FINN
+
+(Tom Sawyer's Comrade)
+
+By Mark Twain
+
+Complete
+
+
+
+
+CONTENTS.
+
+CHAPTER I. Civilizing Huck.—Miss Watson.—Tom Sawyer Waits.
+
+CHAPTER II. The Boys Escape Jim.—Torn Sawyer's Gang.—Deep-laid Plans.
+
+CHAPTER III. A Good Going-over.—Grace Triumphant.—"One of Tom Sawyers's
+Lies".
+
+CHAPTER IV. Huck and the Judge.—Superstition.
+
+CHAPTER V. Huck's Father.—The Fond Parent.—Reform.
+
+CHAPTER VI. He Went for Judge Thatcher.—Huck Decided to Leave.—Political
+Economy.—Thrashing Around.
+
+CHAPTER VII. Laying for Him.—Locked in the Cabin.—Sinking the
+Body.—Resting.
+
+CHAPTER VIII. Sleeping in the Woods.—Raising the Dead.—Exploring the
+Island.—Finding Jim.—Jim's Escape.—Signs.—Balum.
+
+CHAPTER IX. The Cave.—The Floating House.
+
+CHAPTER X. The Find.—Old Hank Bunker.—In Disguise.
+
+CHAPTER XI. Huck and the Woman.—The Search.—Prevarication.—Going to
+Goshen.
+
+CHAPTER XII. Slow Navigation.—Borrowing Things.—Boarding the Wreck.—The
+Plotters.—Hunting for the Boat.
+
+CHAPTER XIII. Escaping from the Wreck.—The Watchman.—Sinking.
+
+CHAPTER XIV. A General Good Time.—The Harem.—French.
+
+CHAPTER XV. Huck Loses the Raft.—In the Fog.—Huck Finds the Raft.—Trash.
+
+CHAPTER XVI. Expectation.—A White Lie.—Floating Currency.—Running by
+Cairo.—Swimming Ashore.
+
+CHAPTER XVII. An Evening Call.—The Farm in Arkansaw.—Interior
+Decorations.—Stephen Dowling Bots.—Poetical Effusions.
+
+CHAPTER XVIII. Col. Grangerford.—Aristocracy.—Feuds.—The
+Testament.—Recovering the Raft.—The Wood—pile.—Pork and Cabbage.
+
+CHAPTER XIX. Tying Up Day—times.—An Astronomical Theory.—Running a
+Temperance Revival.—The Duke of Bridgewater.—The Troubles of Royalty.
+
+CHAPTER XX. Huck Explains.—Laying Out a Campaign.—Working the
+Camp—meeting.—A Pirate at the Camp—meeting.—The Duke as a Printer.
+
+CHAPTER XXI. Sword Exercise.—Hamlet's Soliloquy.—They Loafed Around
+Town.—A Lazy Town.—Old Boggs.—Dead.
+
+CHAPTER XXII. Sherburn.—Attending the Circus.—Intoxication in the
+Ring.—The Thrilling Tragedy.
+
+CHAPTER XXIII. Sold.—Royal Comparisons.—Jim Gets Home-sick.
+
+CHAPTER XXIV. Jim in Royal Robes.—They Take a Passenger.—Getting
+Information.—Family Grief.
+
+CHAPTER XXV. Is It Them?—Singing the "Doxologer."—Awful Square—Funeral
+Orgies.—A Bad Investment .
+
+CHAPTER XXVI. A Pious King.—The King's Clergy.—She Asked His
+Pardon.—Hiding in the Room.—Huck Takes the Money.
+
+CHAPTER XXVII. The Funeral.—Satisfying Curiosity.—Suspicious of
+Huck,—Quick Sales and Small.
+
+CHAPTER XXVIII. The Trip to England.—"The Brute!"—Mary Jane Decides to
+Leave.—Huck Parting with Mary Jane.—Mumps.—The Opposition Line.
+
+CHAPTER XXIX. Contested Relationship.—The King Explains the Loss.—A
+Question of Handwriting.—Digging up the Corpse.—Huck Escapes.
+
+CHAPTER XXX. The King Went for Him.—A Royal Row.—Powerful Mellow.
+
+CHAPTER XXXI. Ominous Plans.—News from Jim.—Old Recollections.—A Sheep
+Story.—Valuable Information.
+
+CHAPTER XXXII. Still and Sunday—like.—Mistaken Identity.—Up a Stump.—In
+a Dilemma.
+
+CHAPTER XXXIII. A Nigger Stealer.—Southern Hospitality.—A Pretty Long
+Blessing.—Tar and Feathers.
+
+CHAPTER XXXIV. The Hut by the Ash Hopper.—Outrageous.—Climbing the
+Lightning Rod.—Troubled with Witches.
+
+CHAPTER XXXV. Escaping Properly.—Dark Schemes.—Discrimination in
+Stealing.—A Deep Hole.
+
+CHAPTER XXXVI. The Lightning Rod.—His Level Best.—A Bequest to
+Posterity.—A High Figure.
+
+CHAPTER XXXVII. The Last Shirt.—Mooning Around.—Sailing Orders.—The
+Witch Pie.
+
+CHAPTER XXXVIII. The Coat of Arms.—A Skilled Superintendent.—Unpleasant
+Glory.—A Tearful Subject.
+
+CHAPTER XXXIX. Rats.—Lively Bed—fellows.—The Straw Dummy.
+
+CHAPTER XL. Fishing.—The Vigilance Committee.—A Lively Run.—Jim Advises
+a Doctor.
+
+CHAPTER XLI. The Doctor.—Uncle Silas.—Sister Hotchkiss.—Aunt Sally in
+Trouble.
+
+CHAPTER XLII. Tom Sawyer Wounded.—The Doctor's Story.—Tom
+Confesses.—Aunt Polly Arrives.—Hand Out Them Letters    .
+
+CHAPTER THE LAST. Out of Bondage.—Paying the Captive.—Yours Truly, Huck
+Finn.
+
+
+
+
+ILLUSTRATIONS.
+
+The Widows
+
+Moses and the "Bulrushers"
+
+Miss Watson
+
+Huck Stealing Away
+
+They Tip-toed Along
+
+Jim
+
+Tom Sawyer's Band of Robbers  
+
+Huck Creeps into his Window
+
+Miss Watson's Lecture
+
+The Robbers Dispersed
+
+Rubbing the Lamp
+
+! ! ! !
+
+Judge Thatcher surprised
+
+Jim Listening
+
+"Pap"
+
+Huck and his Father
+
+Reforming the Drunkard
+
+Falling from Grace
+
+The Widows
+
+Moses and the "Bulrushers"
+
+Miss Watson
+
+Huck Stealing Away
+
+They Tip-toed Along
+
+Jim
+
+Tom Sawyer's Band of Robbers  
+
+Huck Creeps into his Window
+
+Miss Watson's Lecture
+
+The Robbers Dispersed
+
+Rubbing the Lamp
+
+! ! ! !
+
+Judge Thatcher surprised
+
+Jim Listening
+
+"Pap"
+
+Huck and his Father
+
+Reforming the Drunkard
+
+Falling from Grace
+
+Getting out of the Way
+
+Solid Comfort
+
+Thinking it Over
+
+Raising a Howl
+
+"Git Up"
+
+The Shanty
+
+Shooting the Pig
+
+Taking a Rest
+
+In the Woods
+
+Watching the Boat
+
+Discovering the Camp Fire
+
+Jim and the Ghost
+
+Misto Bradish's Nigger
+
+Exploring the Cave
+
+In the Cave
+
+Jim sees a Dead Man
+
+They Found Eight Dollars
+
+Jim and the Snake
+
+Old Hank Bunker
+
+"A Fair Fit"
+
+"Come In"
+
+"Him and another Man"
+
+She puts up a Snack
+
+"Hump Yourself"
+
+On the Raft
+
+He sometimes Lifted a Chicken
+
+"Please don't, Bill"
+
+"It ain't Good Morals"
+
+"Oh! Lordy, Lordy!"
+
+In a Fix
+
+"Hello, What's Up?"
+
+The Wreck
+
+We turned in and Slept
+
+Turning over the Truck
+
+Solomon and his Million Wives
+
+The story of "Sollermun"
+
+"We Would Sell the Raft"
+
+Among the Snags
+
+Asleep on the Raft
+
+"Something being Raftsman"
+
+"Boy, that's a Lie"
+
+"Here I is, Huck"
+
+Climbing up the Bank
+
+"Who's There?"
+
+"Buck"
+
+"It made Her look Spidery"
+
+"They got him out and emptied Him"  
+
+The House
+
+Col. Grangerford
+
+Young Harney Shepherdson
+
+Miss Charlotte
+
+"And asked me if I Liked Her"
+
+"Behind the Wood-pile"
+
+Hiding Day-times
+
+"And Dogs a-Coming"
+
+"By rights I am a Duke!"
+
+"I am the Late Dauphin"
+
+Tail Piece
+
+On the Raft
+
+The King as Juliet
+
+"Courting on the Sly"
+
+"A Pirate for Thirty Years"
+
+Another little Job
+
+Practizing
+
+Hamlet's Soliloquy
+
+"Gimme a Chaw"
+
+A Little Monthly Drunk
+
+The Death of Boggs
+
+Sherburn steps out
+
+A Dead Head
+
+He shed Seventeen Suits
+
+Tragedy
+
+Their Pockets Bulged
+
+Henry the Eighth in Boston Harbor
+
+Harmless
+
+Adolphus
+
+He fairly emptied that Young Fellow
+
+"Alas, our Poor Brother"
+
+"You Bet it is"
+
+Leaking
+
+Making up the "Deffisit"
+
+Going for him
+
+The Doctor
+
+The Bag of Money
+
+The Cubby
+
+Supper with the Hare-Lip
+
+Honest Injun
+
+The Duke looks under the Bed
+
+Huck takes the Money
+
+A Crack in the Dining-room Door
+
+The Undertaker
+
+"He had a Rat!"
+
+"Was you in my Room?"
+
+Jawing
+
+In Trouble
+
+Indignation
+
+How to Find Them
+
+He Wrote
+
+Hannah with the Mumps
+
+The Auction
+
+The True Brothers
+
+The Doctor leads Huck
+
+The Duke Wrote
+
+"Gentlemen, Gentlemen!"
+
+"Jim Lit Out"
+
+The King shakes Huck
+
+The Duke went for Him
+
+Spanish Moss
+
+"Who Nailed Him?"
+
+Thinking
+
+He gave him Ten Cents
+
+Striking for the Back Country
+
+Still and Sunday-like
+
+She hugged him tight
+
+"Who do you reckon it is?"
+
+"It was Tom Sawyer"
+
+"Mr. Archibald Nichols, I presume?"
+
+A pretty long Blessing
+
+Traveling By Rail
+
+Vittles
+
+A Simple Job
+
+Witches
+
+Getting Wood
+
+One of the Best Authorities
+
+The Breakfast-Horn
+
+Smouching the Knives
+
+Going down the Lightning-Rod
+
+Stealing spoons
+
+Tom advises a Witch Pie
+
+The Rubbage-Pile
+
+"Missus, dey's a Sheet Gone"
+
+In a Tearing Way
+
+One of his Ancestors
+
+Jim's Coat of Arms
+
+A Tough Job
+
+Buttons on their Tails
+
+Irrigation
+
+Keeping off Dull Times
+
+Sawdust Diet
+
+Trouble is Brewing
+
+Fishing
+
+Every one had a Gun
+
+Tom caught on a Splinter
+
+Jim advises a Doctor
+
+The Doctor
+
+Uncle Silas in Danger
+
+Old Mrs. Hotchkiss
+
+Aunt Sally talks to Huck
+
+Tom Sawyer wounded
+
+The Doctor speaks for Jim
+
+Tom rose square up in Bed
+
+"Hand out them Letters"
+
+Out of Bondage
+
+Tom's Liberality
+
+Yours Truly
+
+
+
+
+EXPLANATORY
+
+IN this book a number of dialects are used, to wit:  the Missouri negro
+dialect; the extremest form of the backwoods Southwestern dialect; the
+ordinary "Pike County" dialect; and four modified varieties of this
+last. The shadings have not been done in a haphazard fashion, or by
+guesswork; but painstakingly, and with the trustworthy guidance and
+support of personal familiarity with these several forms of speech.
+
+I make this explanation for the reason that without it many readers
+would suppose that all these characters were trying to talk alike and
+not succeeding.
+
+THE AUTHOR.
+
+
+
+
+HUCKLEBERRY FINN
+
+Scene:  The Mississippi Valley Time:  Forty to fifty years ago
+
+
+
+
+CHAPTER I.
+
+YOU don't know about me without you have read a book by the name of The
+Adventures of Tom Sawyer; but that ain't no matter.  That book was made
+by Mr. Mark Twain, and he told the truth, mainly.  There was things
+which he stretched, but mainly he told the truth.  That is nothing.  I
+never seen anybody but lied one time or another, without it was Aunt
+Polly, or the widow, or maybe Mary.  Aunt Polly—Tom's Aunt Polly, she
+is—and Mary, and the Widow Douglas is all told about in that book, which
+is mostly a true book, with some stretchers, as I said before.
+
+Now the way that the book winds up is this:  Tom and me found the money
+that the robbers hid in the cave, and it made us rich.  We got six
+thousand dollars apiece—all gold.  It was an awful sight of money when
+it was piled up.  Well, Judge Thatcher he took it and put it out
+at interest, and it fetched us a dollar a day apiece all the year
+round—more than a body could tell what to do with.  The Widow Douglas
+she took me for her son, and allowed she would sivilize me; but it was
+rough living in the house all the time, considering how dismal regular
+and decent the widow was in all her ways; and so when I couldn't stand
+it no longer I lit out.  I got into my old rags and my sugar-hogshead
+again, and was free and satisfied.  But Tom Sawyer he hunted me up and
+said he was going to start a band of robbers, and I might join if I
+would go back to the widow and be respectable.  So I went back.
+
+The widow she cried over me, and called me a poor lost lamb, and she
+called me a lot of other names, too, but she never meant no harm by
+it. She put me in them new clothes again, and I couldn't do nothing but
+sweat and sweat, and feel all cramped up.  Well, then, the old thing
+commenced again.  The widow rung a bell for supper, and you had to come
+to time. When you got to the table you couldn't go right to eating, but
+you had to wait for the widow to tuck down her head and grumble a little
+over the victuals, though there warn't really anything the matter with
+them,—that is, nothing only everything was cooked by itself.  In a
+barrel of odds and ends it is different; things get mixed up, and the
+juice kind of swaps around, and the things go better.
+
+After supper she got out her book and learned me about Moses and the
+Bulrushers, and I was in a sweat to find out all about him; but by and
+by she let it out that Moses had been dead a considerable long time; so
+then I didn't care no more about him, because I don't take no stock in
+dead people.
+
+Pretty soon I wanted to smoke, and asked the widow to let me.  But she
+wouldn't.  She said it was a mean practice and wasn't clean, and I must
+try to not do it any more.  That is just the way with some people.  They
+get down on a thing when they don't know nothing about it.  Here she was
+a-bothering about Moses, which was no kin to her, and no use to anybody,
+being gone, you see, yet finding a power of fault with me for doing a
+thing that had some good in it.  And she took snuff, too; of course that
+was all right, because she done it herself.
+
+Her sister, Miss Watson, a tolerable slim old maid, with goggles on,
+had just come to live with her, and took a set at me now with a
+spelling-book. She worked me middling hard for about an hour, and then
+the widow made her ease up.  I couldn't stood it much longer.  Then for
+an hour it was deadly dull, and I was fidgety.  Miss Watson would say,
+"Don't put your feet up there, Huckleberry;" and "Don't scrunch up
+like that, Huckleberry—set up straight;" and pretty soon she would
+say, "Don't gap and stretch like that, Huckleberry—why don't you try to
+behave?"  Then she told me all about the bad place, and I said I wished
+I was there. She got mad then, but I didn't mean no harm.  All I wanted
+was to go somewheres; all I wanted was a change, I warn't particular.
+ She said it was wicked to say what I said; said she wouldn't say it for
+the whole world; she was going to live so as to go to the good place.
+ Well, I couldn't see no advantage in going where she was going, so I
+made up my mind I wouldn't try for it.  But I never said so, because it
+would only make trouble, and wouldn't do no good.
+
+Now she had got a start, and she went on and told me all about the good
+place.  She said all a body would have to do there was to go around all
+day long with a harp and sing, forever and ever.  So I didn't think
+much of it. But I never said so.  I asked her if she reckoned Tom Sawyer
+would go there, and she said not by a considerable sight.  I was glad
+about that, because I wanted him and me to be together.
+
+Miss Watson she kept pecking at me, and it got tiresome and lonesome.
+ By and by they fetched the niggers in and had prayers, and then
+everybody was off to bed.  I went up to my room with a piece of candle,
+and put it on the table.  Then I set down in a chair by the window and
+tried to think of something cheerful, but it warn't no use.  I felt
+so lonesome I most wished I was dead.  The stars were shining, and the
+leaves rustled in the woods ever so mournful; and I heard an owl, away
+off, who-whooing about somebody that was dead, and a whippowill and a
+dog crying about somebody that was going to die; and the wind was trying
+to whisper something to me, and I couldn't make out what it was, and so
+it made the cold shivers run over me. Then away out in the woods I heard
+that kind of a sound that a ghost makes when it wants to tell about
+something that's on its mind and can't make itself understood, and so
+can't rest easy in its grave, and has to go about that way every night
+grieving.  I got so down-hearted and scared I did wish I had some
+company.  Pretty soon a spider went crawling up my shoulder, and I
+flipped it off and it lit in the candle; and before I could budge it
+was all shriveled up.  I didn't need anybody to tell me that that was
+an awful bad sign and would fetch me some bad luck, so I was scared
+and most shook the clothes off of me. I got up and turned around in my
+tracks three times and crossed my breast every time; and then I tied
+up a little lock of my hair with a thread to keep witches away.  But
+I hadn't no confidence.  You do that when you've lost a horseshoe that
+you've found, instead of nailing it up over the door, but I hadn't ever
+heard anybody say it was any way to keep off bad luck when you'd killed
+a spider.
+
+I set down again, a-shaking all over, and got out my pipe for a smoke;
+for the house was all as still as death now, and so the widow wouldn't
+know. Well, after a long time I heard the clock away off in the town
+go boom—boom—boom—twelve licks; and all still again—stiller than
+ever. Pretty soon I heard a twig snap down in the dark amongst the
+trees—something was a stirring.  I set still and listened.  Directly I
+could just barely hear a "me-yow! me-yow!" down there.  That was good!
+ Says I, "me-yow! me-yow!" as soft as I could, and then I put out the
+light and scrambled out of the window on to the shed.  Then I slipped
+down to the ground and crawled in among the trees, and, sure enough,
+there was Tom Sawyer waiting for me.
+
+
+
+
+CHAPTER II.
+
+WE went tiptoeing along a path amongst the trees back towards the end of
+the widow's garden, stooping down so as the branches wouldn't scrape our
+heads. When we was passing by the kitchen I fell over a root and made
+a noise.  We scrouched down and laid still.  Miss Watson's big nigger,
+named Jim, was setting in the kitchen door; we could see him pretty
+clear, because there was a light behind him.  He got up and stretched
+his neck out about a minute, listening.  Then he says:
+
+"Who dah?"
+
+He listened some more; then he come tiptoeing down and stood right
+between us; we could a touched him, nearly.  Well, likely it was
+minutes and minutes that there warn't a sound, and we all there so close
+together.  There was a place on my ankle that got to itching, but I
+dasn't scratch it; and then my ear begun to itch; and next my back,
+right between my shoulders.  Seemed like I'd die if I couldn't scratch.
+ Well, I've noticed that thing plenty times since.  If you are with
+the quality, or at a funeral, or trying to go to sleep when you ain't
+sleepy—if you are anywheres where it won't do for you to scratch, why
+you will itch all over in upwards of a thousand places. Pretty soon Jim
+says:
+
+"Say, who is you?  Whar is you?  Dog my cats ef I didn' hear sumf'n.
+Well, I know what I's gwyne to do:  I's gwyne to set down here and
+listen tell I hears it agin."
+
+So he set down on the ground betwixt me and Tom.  He leaned his back up
+against a tree, and stretched his legs out till one of them most touched
+one of mine.  My nose begun to itch.  It itched till the tears come into
+my eyes.  But I dasn't scratch.  Then it begun to itch on the inside.
+Next I got to itching underneath.  I didn't know how I was going to set
+still. This miserableness went on as much as six or seven minutes; but
+it seemed a sight longer than that.  I was itching in eleven different
+places now.  I reckoned I couldn't stand it more'n a minute longer,
+but I set my teeth hard and got ready to try.  Just then Jim begun
+to breathe heavy; next he begun to snore—and then I was pretty soon
+comfortable again.
+
+Tom he made a sign to me—kind of a little noise with his mouth—and we
+went creeping away on our hands and knees.  When we was ten foot off Tom
+whispered to me, and wanted to tie Jim to the tree for fun.  But I said
+no; he might wake and make a disturbance, and then they'd find out I
+warn't in. Then Tom said he hadn't got candles enough, and he would slip
+in the kitchen and get some more.  I didn't want him to try.  I said Jim
+might wake up and come.  But Tom wanted to resk it; so we slid in there
+and got three candles, and Tom laid five cents on the table for pay.
+Then we got out, and I was in a sweat to get away; but nothing would do
+Tom but he must crawl to where Jim was, on his hands and knees, and play
+something on him.  I waited, and it seemed a good while, everything was
+so still and lonesome.
+
+As soon as Tom was back we cut along the path, around the garden fence,
+and by and by fetched up on the steep top of the hill the other side of
+the house.  Tom said he slipped Jim's hat off of his head and hung it
+on a limb right over him, and Jim stirred a little, but he didn't wake.
+Afterwards Jim said the witches be witched him and put him in a trance,
+and rode him all over the State, and then set him under the trees again,
+and hung his hat on a limb to show who done it.  And next time Jim told
+it he said they rode him down to New Orleans; and, after that, every
+time he told it he spread it more and more, till by and by he said they
+rode him all over the world, and tired him most to death, and his back
+was all over saddle-boils.  Jim was monstrous proud about it, and he
+got so he wouldn't hardly notice the other niggers.  Niggers would come
+miles to hear Jim tell about it, and he was more looked up to than any
+nigger in that country.  Strange niggers would stand with their mouths
+open and look him all over, same as if he was a wonder.  Niggers is
+always talking about witches in the dark by the kitchen fire; but
+whenever one was talking and letting on to know all about such things,
+Jim would happen in and say, "Hm!  What you know 'bout witches?" and
+that nigger was corked up and had to take a back seat.  Jim always kept
+that five-center piece round his neck with a string, and said it was a
+charm the devil give to him with his own hands, and told him he could
+cure anybody with it and fetch witches whenever he wanted to just by
+saying something to it; but he never told what it was he said to it.
+ Niggers would come from all around there and give Jim anything they
+had, just for a sight of that five-center piece; but they wouldn't touch
+it, because the devil had had his hands on it.  Jim was most ruined for
+a servant, because he got stuck up on account of having seen the devil
+and been rode by witches.
+
+Well, when Tom and me got to the edge of the hilltop we looked away down
+into the village and could see three or four lights twinkling, where
+there was sick folks, maybe; and the stars over us was sparkling ever
+so fine; and down by the village was the river, a whole mile broad, and
+awful still and grand.  We went down the hill and found Jo Harper and
+Ben Rogers, and two or three more of the boys, hid in the old tanyard.
+ So we unhitched a skiff and pulled down the river two mile and a half,
+to the big scar on the hillside, and went ashore.
+
+We went to a clump of bushes, and Tom made everybody swear to keep the
+secret, and then showed them a hole in the hill, right in the thickest
+part of the bushes.  Then we lit the candles, and crawled in on our
+hands and knees.  We went about two hundred yards, and then the cave
+opened up. Tom poked about amongst the passages, and pretty soon ducked
+under a wall where you wouldn't a noticed that there was a hole.  We
+went along a narrow place and got into a kind of room, all damp and
+sweaty and cold, and there we stopped.  Tom says:
+
+"Now, we'll start this band of robbers and call it Tom Sawyer's Gang.
+Everybody that wants to join has got to take an oath, and write his name
+in blood."
+
+Everybody was willing.  So Tom got out a sheet of paper that he had
+wrote the oath on, and read it.  It swore every boy to stick to the
+band, and never tell any of the secrets; and if anybody done anything to
+any boy in the band, whichever boy was ordered to kill that person and
+his family must do it, and he mustn't eat and he mustn't sleep till he
+had killed them and hacked a cross in their breasts, which was the sign
+of the band. And nobody that didn't belong to the band could use that
+mark, and if he did he must be sued; and if he done it again he must be
+killed.  And if anybody that belonged to the band told the secrets, he
+must have his throat cut, and then have his carcass burnt up and the
+ashes scattered all around, and his name blotted off of the list with
+blood and never mentioned again by the gang, but have a curse put on it
+and be forgot forever.
+
+Everybody said it was a real beautiful oath, and asked Tom if he got
+it out of his own head.  He said, some of it, but the rest was out of
+pirate-books and robber-books, and every gang that was high-toned had
+it.
+
+Some thought it would be good to kill the families of boys that told
+the secrets.  Tom said it was a good idea, so he took a pencil and wrote
+it in. Then Ben Rogers says:
+
+"Here's Huck Finn, he hain't got no family; what you going to do 'bout
+him?"
+
+"Well, hain't he got a father?" says Tom Sawyer.
+
+"Yes, he's got a father, but you can't never find him these days.  He
+used to lay drunk with the hogs in the tanyard, but he hain't been seen
+in these parts for a year or more."
+
+They talked it over, and they was going to rule me out, because they
+said every boy must have a family or somebody to kill, or else it
+wouldn't be fair and square for the others.  Well, nobody could think of
+anything to do—everybody was stumped, and set still.  I was most ready
+to cry; but all at once I thought of a way, and so I offered them Miss
+Watson—they could kill her.  Everybody said:
+
+"Oh, she'll do.  That's all right.  Huck can come in."
+
+Then they all stuck a pin in their fingers to get blood to sign with,
+and I made my mark on the paper.
+
+"Now," says Ben Rogers, "what's the line of business of this Gang?"
+
+"Nothing only robbery and murder," Tom said.
+
+"But who are we going to rob?—houses, or cattle, or—"
+
+"Stuff! stealing cattle and such things ain't robbery; it's burglary,"
+says Tom Sawyer.  "We ain't burglars.  That ain't no sort of style.  We
+are highwaymen.  We stop stages and carriages on the road, with masks
+on, and kill the people and take their watches and money."
+
+"Must we always kill the people?"
+
+"Oh, certainly.  It's best.  Some authorities think different, but
+mostly it's considered best to kill them—except some that you bring to
+the cave here, and keep them till they're ransomed."
+
+"Ransomed?  What's that?"
+
+"I don't know.  But that's what they do.  I've seen it in books; and so
+of course that's what we've got to do."
+
+"But how can we do it if we don't know what it is?"
+
+"Why, blame it all, we've got to do it.  Don't I tell you it's in the
+books?  Do you want to go to doing different from what's in the books,
+and get things all muddled up?"
+
+"Oh, that's all very fine to say, Tom Sawyer, but how in the nation
+are these fellows going to be ransomed if we don't know how to do it
+to them?—that's the thing I want to get at.  Now, what do you reckon it
+is?"
+
+"Well, I don't know.  But per'aps if we keep them till they're ransomed,
+it means that we keep them till they're dead."
+
+"Now, that's something like.  That'll answer.  Why couldn't you said
+that before?  We'll keep them till they're ransomed to death; and a
+bothersome lot they'll be, too—eating up everything, and always trying
+to get loose."
+
+"How you talk, Ben Rogers.  How can they get loose when there's a guard
+over them, ready to shoot them down if they move a peg?"
+
+"A guard!  Well, that is good.  So somebody's got to set up all night
+and never get any sleep, just so as to watch them.  I think that's
+foolishness. Why can't a body take a club and ransom them as soon as
+they get here?"
+
+"Because it ain't in the books so—that's why.  Now, Ben Rogers, do you
+want to do things regular, or don't you?—that's the idea.  Don't you
+reckon that the people that made the books knows what's the correct
+thing to do?  Do you reckon you can learn 'em anything?  Not by a good
+deal. No, sir, we'll just go on and ransom them in the regular way."
+
+"All right.  I don't mind; but I say it's a fool way, anyhow.  Say, do
+we kill the women, too?"
+
+"Well, Ben Rogers, if I was as ignorant as you I wouldn't let on.  Kill
+the women?  No; nobody ever saw anything in the books like that.  You
+fetch them to the cave, and you're always as polite as pie to them;
+and by and by they fall in love with you, and never want to go home any
+more."
+
+"Well, if that's the way I'm agreed, but I don't take no stock in it.
+Mighty soon we'll have the cave so cluttered up with women, and fellows
+waiting to be ransomed, that there won't be no place for the robbers.
+But go ahead, I ain't got nothing to say."
+
+Little Tommy Barnes was asleep now, and when they waked him up he was
+scared, and cried, and said he wanted to go home to his ma, and didn't
+want to be a robber any more.
+
+So they all made fun of him, and called him cry-baby, and that made him
+mad, and he said he would go straight and tell all the secrets.  But
+Tom give him five cents to keep quiet, and said we would all go home and
+meet next week, and rob somebody and kill some people.
+
+Ben Rogers said he couldn't get out much, only Sundays, and so he wanted
+to begin next Sunday; but all the boys said it would be wicked to do it
+on Sunday, and that settled the thing.  They agreed to get together and
+fix a day as soon as they could, and then we elected Tom Sawyer first
+captain and Jo Harper second captain of the Gang, and so started home.
+
+I clumb up the shed and crept into my window just before day was
+breaking. My new clothes was all greased up and clayey, and I was
+dog-tired.
+
+
+
+
+CHAPTER III.
+
+WELL, I got a good going-over in the morning from old Miss Watson on
+account of my clothes; but the widow she didn't scold, but only cleaned
+off the grease and clay, and looked so sorry that I thought I would
+behave awhile if I could.  Then Miss Watson she took me in the closet
+and prayed, but nothing come of it.  She told me to pray every day, and
+whatever I asked for I would get it.  But it warn't so.  I tried it.
+Once I got a fish-line, but no hooks.  It warn't any good to me without
+hooks.  I tried for the hooks three or four times, but somehow I
+couldn't make it work.  By and by, one day, I asked Miss Watson to
+try for me, but she said I was a fool.  She never told me why, and I
+couldn't make it out no way.
+
+I set down one time back in the woods, and had a long think about it.
+ I says to myself, if a body can get anything they pray for, why don't
+Deacon Winn get back the money he lost on pork?  Why can't the widow get
+back her silver snuffbox that was stole?  Why can't Miss Watson fat up?
+No, says I to my self, there ain't nothing in it.  I went and told the
+widow about it, and she said the thing a body could get by praying for
+it was "spiritual gifts."  This was too many for me, but she told me
+what she meant—I must help other people, and do everything I could for
+other people, and look out for them all the time, and never think about
+myself. This was including Miss Watson, as I took it.  I went out in the
+woods and turned it over in my mind a long time, but I couldn't see no
+advantage about it—except for the other people; so at last I reckoned
+I wouldn't worry about it any more, but just let it go.  Sometimes the
+widow would take me one side and talk about Providence in a way to make
+a body's mouth water; but maybe next day Miss Watson would take hold
+and knock it all down again.  I judged I could see that there was two
+Providences, and a poor chap would stand considerable show with the
+widow's Providence, but if Miss Watson's got him there warn't no help
+for him any more.  I thought it all out, and reckoned I would belong
+to the widow's if he wanted me, though I couldn't make out how he was
+a-going to be any better off then than what he was before, seeing I was
+so ignorant, and so kind of low-down and ornery.
+
+Pap he hadn't been seen for more than a year, and that was comfortable
+for me; I didn't want to see him no more.  He used to always whale me
+when he was sober and could get his hands on me; though I used to take
+to the woods most of the time when he was around.  Well, about this time
+he was found in the river drownded, about twelve mile above town, so
+people said.  They judged it was him, anyway; said this drownded man was
+just his size, and was ragged, and had uncommon long hair, which was all
+like pap; but they couldn't make nothing out of the face, because it had
+been in the water so long it warn't much like a face at all.  They said
+he was floating on his back in the water.  They took him and buried him
+on the bank.  But I warn't comfortable long, because I happened to think
+of something.  I knowed mighty well that a drownded man don't float on
+his back, but on his face.  So I knowed, then, that this warn't pap, but
+a woman dressed up in a man's clothes.  So I was uncomfortable again.
+ I judged the old man would turn up again by and by, though I wished he
+wouldn't.
+
+We played robber now and then about a month, and then I resigned.  All
+the boys did.  We hadn't robbed nobody, hadn't killed any people, but
+only just pretended.  We used to hop out of the woods and go charging
+down on hog-drivers and women in carts taking garden stuff to market,
+but we never hived any of them.  Tom Sawyer called the hogs "ingots,"
+and he called the turnips and stuff "julery," and we would go to the
+cave and powwow over what we had done, and how many people we had killed
+and marked.  But I couldn't see no profit in it.  One time Tom sent a
+boy to run about town with a blazing stick, which he called a slogan
+(which was the sign for the Gang to get together), and then he said he
+had got secret news by his spies that next day a whole parcel of Spanish
+merchants and rich A-rabs was going to camp in Cave Hollow with two
+hundred elephants, and six hundred camels, and over a thousand "sumter"
+mules, all loaded down with di'monds, and they didn't have only a guard
+of four hundred soldiers, and so we would lay in ambuscade, as he called
+it, and kill the lot and scoop the things.  He said we must slick up
+our swords and guns, and get ready.  He never could go after even a
+turnip-cart but he must have the swords and guns all scoured up for it,
+though they was only lath and broomsticks, and you might scour at them
+till you rotted, and then they warn't worth a mouthful of ashes more
+than what they was before.  I didn't believe we could lick such a crowd
+of Spaniards and A-rabs, but I wanted to see the camels and elephants,
+so I was on hand next day, Saturday, in the ambuscade; and when we got
+the word we rushed out of the woods and down the hill.  But there warn't
+no Spaniards and A-rabs, and there warn't no camels nor no elephants.
+ It warn't anything but a Sunday-school picnic, and only a primer-class
+at that.  We busted it up, and chased the children up the hollow; but we
+never got anything but some doughnuts and jam, though Ben Rogers got
+a rag doll, and Jo Harper got a hymn-book and a tract; and then the
+teacher charged in, and made us drop everything and cut.
+
+ I didn't see no di'monds, and I told Tom Sawyer so.  He said there was
+loads of them there, anyway; and he said there was A-rabs there, too,
+and elephants and things.  I said, why couldn't we see them, then?  He
+said if I warn't so ignorant, but had read a book called Don Quixote, I
+would know without asking.  He said it was all done by enchantment.  He
+said there was hundreds of soldiers there, and elephants and treasure,
+and so on, but we had enemies which he called magicians; and they had
+turned the whole thing into an infant Sunday-school, just out of spite.
+ I said, all right; then the thing for us to do was to go for the
+magicians.  Tom Sawyer said I was a numskull.
+
+"Why," said he, "a magician could call up a lot of genies, and they
+would hash you up like nothing before you could say Jack Robinson.  They
+are as tall as a tree and as big around as a church."
+
+"Well," I says, "s'pose we got some genies to help us—can't we lick
+the other crowd then?"
+
+"How you going to get them?"
+
+"I don't know.  How do they get them?"
+
+"Why, they rub an old tin lamp or an iron ring, and then the genies
+come tearing in, with the thunder and lightning a-ripping around and the
+smoke a-rolling, and everything they're told to do they up and do it.
+ They don't think nothing of pulling a shot-tower up by the roots, and
+belting a Sunday-school superintendent over the head with it—or any
+other man."
+
+"Who makes them tear around so?"
+
+"Why, whoever rubs the lamp or the ring.  They belong to whoever rubs
+the lamp or the ring, and they've got to do whatever he says.  If he
+tells them to build a palace forty miles long out of di'monds, and fill
+it full of chewing-gum, or whatever you want, and fetch an emperor's
+daughter from China for you to marry, they've got to do it—and they've
+got to do it before sun-up next morning, too.  And more:  they've got
+to waltz that palace around over the country wherever you want it, you
+understand."
+
+"Well," says I, "I think they are a pack of flat-heads for not keeping
+the palace themselves 'stead of fooling them away like that.  And what's
+more—if I was one of them I would see a man in Jericho before I would
+drop my business and come to him for the rubbing of an old tin lamp."
+
+"How you talk, Huck Finn.  Why, you'd have to come when he rubbed it,
+whether you wanted to or not."
+
+"What! and I as high as a tree and as big as a church?  All right, then;
+I would come; but I lay I'd make that man climb the highest tree there
+was in the country."
+
+"Shucks, it ain't no use to talk to you, Huck Finn.  You don't seem to
+know anything, somehow—perfect saphead."
+
+I thought all this over for two or three days, and then I reckoned I
+would see if there was anything in it.  I got an old tin lamp and an
+iron ring, and went out in the woods and rubbed and rubbed till I sweat
+like an Injun, calculating to build a palace and sell it; but it warn't
+no use, none of the genies come.  So then I judged that all that stuff
+was only just one of Tom Sawyer's lies.  I reckoned he believed in the
+A-rabs and the elephants, but as for me I think different.  It had all
+the marks of a Sunday-school.
+
+
+
+
+CHAPTER IV.
+
+WELL, three or four months run along, and it was well into the winter
+now. I had been to school most all the time and could spell and read and
+write just a little, and could say the multiplication table up to six
+times seven is thirty-five, and I don't reckon I could ever get any
+further than that if I was to live forever.  I don't take no stock in
+mathematics, anyway.
+
+At first I hated the school, but by and by I got so I could stand it.
+Whenever I got uncommon tired I played hookey, and the hiding I got next
+day done me good and cheered me up.  So the longer I went to school the
+easier it got to be.  I was getting sort of used to the widow's ways,
+too, and they warn't so raspy on me.  Living in a house and sleeping in
+a bed pulled on me pretty tight mostly, but before the cold weather I
+used to slide out and sleep in the woods sometimes, and so that was a
+rest to me.  I liked the old ways best, but I was getting so I liked the
+new ones, too, a little bit. The widow said I was coming along slow but
+sure, and doing very satisfactory.  She said she warn't ashamed of me.
+
+One morning I happened to turn over the salt-cellar at breakfast.
+ I reached for some of it as quick as I could to throw over my left
+shoulder and keep off the bad luck, but Miss Watson was in ahead of me,
+and crossed me off. She says, "Take your hands away, Huckleberry; what
+a mess you are always making!"  The widow put in a good word for me, but
+that warn't going to keep off the bad luck, I knowed that well enough.
+ I started out, after breakfast, feeling worried and shaky, and
+wondering where it was going to fall on me, and what it was going to be.
+ There is ways to keep off some kinds of bad luck, but this wasn't one
+of them kind; so I never tried to do anything, but just poked along
+low-spirited and on the watch-out.
+
+I went down to the front garden and clumb over the stile where you go
+through the high board fence.  There was an inch of new snow on the
+ground, and I seen somebody's tracks.  They had come up from the quarry
+and stood around the stile a while, and then went on around the garden
+fence.  It was funny they hadn't come in, after standing around so.  I
+couldn't make it out.  It was very curious, somehow.  I was going to
+follow around, but I stooped down to look at the tracks first.  I didn't
+notice anything at first, but next I did.  There was a cross in the left
+boot-heel made with big nails, to keep off the devil.
+
+I was up in a second and shinning down the hill.  I looked over my
+shoulder every now and then, but I didn't see nobody.  I was at Judge
+Thatcher's as quick as I could get there.  He said:
+
+"Why, my boy, you are all out of breath.  Did you come for your
+interest?"
+
+"No, sir," I says; "is there some for me?"
+
+"Oh, yes, a half-yearly is in last night—over a hundred and fifty
+dollars.  Quite a fortune for you.  You had better let me invest it
+along with your six thousand, because if you take it you'll spend it."
+
+"No, sir," I says, "I don't want to spend it.  I don't want it at
+all—nor the six thousand, nuther.  I want you to take it; I want to give
+it to you—the six thousand and all."
+
+He looked surprised.  He couldn't seem to make it out.  He says:
+
+"Why, what can you mean, my boy?"
+
+I says, "Don't you ask me no questions about it, please.  You'll take
+it—won't you?"
+
+He says:
+
+"Well, I'm puzzled.  Is something the matter?"
+
+"Please take it," says I, "and don't ask me nothing—then I won't have to
+tell no lies."
+
+He studied a while, and then he says:
+
+"Oho-o!  I think I see.  You want to sell all your property to me—not
+give it.  That's the correct idea."
+
+Then he wrote something on a paper and read it over, and says:
+
+"There; you see it says 'for a consideration.'  That means I have bought
+it of you and paid you for it.  Here's a dollar for you.  Now you sign
+it."
+
+So I signed it, and left.
+
+Miss Watson's nigger, Jim, had a hair-ball as big as your fist, which
+had been took out of the fourth stomach of an ox, and he used to do
+magic with it.  He said there was a spirit inside of it, and it knowed
+everything.  So I went to him that night and told him pap was here
+again, for I found his tracks in the snow.  What I wanted to know was,
+what he was going to do, and was he going to stay?  Jim got out his
+hair-ball and said something over it, and then he held it up and dropped
+it on the floor.  It fell pretty solid, and only rolled about an inch.
+ Jim tried it again, and then another time, and it acted just the same.
+ Jim got down on his knees, and put his ear against it and listened.
+ But it warn't no use; he said it wouldn't talk. He said sometimes it
+wouldn't talk without money.  I told him I had an old slick counterfeit
+quarter that warn't no good because the brass showed through the silver
+a little, and it wouldn't pass nohow, even if the brass didn't show,
+because it was so slick it felt greasy, and so that would tell on it
+every time.  (I reckoned I wouldn't say nothing about the dollar I got
+from the judge.) I said it was pretty bad money, but maybe the hair-ball
+would take it, because maybe it wouldn't know the difference.  Jim smelt
+it and bit it and rubbed it, and said he would manage so the hair-ball
+would think it was good.  He said he would split open a raw Irish potato
+and stick the quarter in between and keep it there all night, and next
+morning you couldn't see no brass, and it wouldn't feel greasy no more,
+and so anybody in town would take it in a minute, let alone a hair-ball.
+ Well, I knowed a potato would do that before, but I had forgot it.
+
+Jim put the quarter under the hair-ball, and got down and listened
+again. This time he said the hair-ball was all right.  He said it
+would tell my whole fortune if I wanted it to.  I says, go on.  So the
+hair-ball talked to Jim, and Jim told it to me.  He says:
+
+"Yo' ole father doan' know yit what he's a-gwyne to do.  Sometimes he
+spec he'll go 'way, en den agin he spec he'll stay.  De bes' way is to
+res' easy en let de ole man take his own way.  Dey's two angels hoverin'
+roun' 'bout him.  One uv 'em is white en shiny, en t'other one is black.
+De white one gits him to go right a little while, den de black one sail
+in en bust it all up.  A body can't tell yit which one gwyne to fetch
+him at de las'.  But you is all right.  You gwyne to have considable
+trouble in yo' life, en considable joy.  Sometimes you gwyne to git
+hurt, en sometimes you gwyne to git sick; but every time you's gwyne
+to git well agin.  Dey's two gals flyin' 'bout you in yo' life.  One
+uv 'em's light en t'other one is dark. One is rich en t'other is po'.
+ You's gwyne to marry de po' one fust en de rich one by en by.  You
+wants to keep 'way fum de water as much as you kin, en don't run no
+resk, 'kase it's down in de bills dat you's gwyne to git hung."
+
+When I lit my candle and went up to my room that night there sat pap his
+own self!
+
+
+
+
+CHAPTER V.
+
+I had shut the door to.  Then I turned around and there he was.  I used
+to be scared of him all the time, he tanned me so much.  I reckoned I
+was scared now, too; but in a minute I see I was mistaken—that is, after
+the first jolt, as you may say, when my breath sort of hitched, he being
+so unexpected; but right away after I see I warn't scared of him worth
+bothring about.
+
+He was most fifty, and he looked it.  His hair was long and tangled and
+greasy, and hung down, and you could see his eyes shining through
+like he was behind vines.  It was all black, no gray; so was his long,
+mixed-up whiskers.  There warn't no color in his face, where his face
+showed; it was white; not like another man's white, but a white to make
+a body sick, a white to make a body's flesh crawl—a tree-toad white, a
+fish-belly white.  As for his clothes—just rags, that was all.  He had
+one ankle resting on t'other knee; the boot on that foot was busted, and
+two of his toes stuck through, and he worked them now and then.  His hat
+was laying on the floor—an old black slouch with the top caved in, like
+a lid.
+
+I stood a-looking at him; he set there a-looking at me, with his chair
+tilted back a little.  I set the candle down.  I noticed the window was
+up; so he had clumb in by the shed.  He kept a-looking me all over.  By
+and by he says:
+
+"Starchy clothes—very.  You think you're a good deal of a big-bug,
+don't you?"
+
+"Maybe I am, maybe I ain't," I says.
+
+"Don't you give me none o' your lip," says he.  "You've put on
+considerable many frills since I been away.  I'll take you down a peg
+before I get done with you.  You're educated, too, they say—can read and
+write.  You think you're better'n your father, now, don't you, because
+he can't?  I'll take it out of you.  Who told you you might meddle
+with such hifalut'n foolishness, hey?—who told you you could?"
+
+"The widow.  She told me."
+
+"The widow, hey?—and who told the widow she could put in her shovel
+about a thing that ain't none of her business?"
+
+"Nobody never told her."
+
+"Well, I'll learn her how to meddle.  And looky here—you drop that
+school, you hear?  I'll learn people to bring up a boy to put on airs
+over his own father and let on to be better'n what he is.  You lemme
+catch you fooling around that school again, you hear?  Your mother
+couldn't read, and she couldn't write, nuther, before she died.  None
+of the family couldn't before they died.  I can't; and here you're
+a-swelling yourself up like this.  I ain't the man to stand it—you hear?
+Say, lemme hear you read."
+
+I took up a book and begun something about General Washington and the
+wars. When I'd read about a half a minute, he fetched the book a whack
+with his hand and knocked it across the house.  He says:
+
+"It's so.  You can do it.  I had my doubts when you told me.  Now looky
+here; you stop that putting on frills.  I won't have it.  I'll lay for
+you, my smarty; and if I catch you about that school I'll tan you good.
+First you know you'll get religion, too.  I never see such a son."
+
+He took up a little blue and yaller picture of some cows and a boy, and
+says:
+
+"What's this?"
+
+"It's something they give me for learning my lessons good."
+
+He tore it up, and says:
+
+"I'll give you something better—I'll give you a cowhide."
+
+He set there a-mumbling and a-growling a minute, and then he says:
+
+"Ain't you a sweet-scented dandy, though?  A bed; and bedclothes; and
+a look'n'-glass; and a piece of carpet on the floor—and your own father
+got to sleep with the hogs in the tanyard.  I never see such a son.  I
+bet I'll take some o' these frills out o' you before I'm done with you.
+Why, there ain't no end to your airs—they say you're rich.  Hey?—how's
+that?"
+
+"They lie—that's how."
+
+"Looky here—mind how you talk to me; I'm a-standing about all I can
+stand now—so don't gimme no sass.  I've been in town two days, and I
+hain't heard nothing but about you bein' rich.  I heard about it
+away down the river, too.  That's why I come.  You git me that money
+to-morrow—I want it."
+
+"I hain't got no money."
+
+"It's a lie.  Judge Thatcher's got it.  You git it.  I want it."
+
+"I hain't got no money, I tell you.  You ask Judge Thatcher; he'll tell
+you the same."
+
+"All right.  I'll ask him; and I'll make him pungle, too, or I'll know
+the reason why.  Say, how much you got in your pocket?  I want it."
+
+"I hain't got only a dollar, and I want that to—"
+
+"It don't make no difference what you want it for—you just shell it
+out."
+
+He took it and bit it to see if it was good, and then he said he was
+going down town to get some whisky; said he hadn't had a drink all day.
+When he had got out on the shed he put his head in again, and cussed
+me for putting on frills and trying to be better than him; and when I
+reckoned he was gone he come back and put his head in again, and told me
+to mind about that school, because he was going to lay for me and lick
+me if I didn't drop that.
+
+Next day he was drunk, and he went to Judge Thatcher's and bullyragged
+him, and tried to make him give up the money; but he couldn't, and then
+he swore he'd make the law force him.
+
+The judge and the widow went to law to get the court to take me away
+from him and let one of them be my guardian; but it was a new judge that
+had just come, and he didn't know the old man; so he said courts mustn't
+interfere and separate families if they could help it; said he'd druther
+not take a child away from its father.  So Judge Thatcher and the widow
+had to quit on the business.
+
+That pleased the old man till he couldn't rest.  He said he'd cowhide
+me till I was black and blue if I didn't raise some money for him.  I
+borrowed three dollars from Judge Thatcher, and pap took it and got
+drunk, and went a-blowing around and cussing and whooping and carrying
+on; and he kept it up all over town, with a tin pan, till most midnight;
+then they jailed him, and next day they had him before court, and jailed
+him again for a week.  But he said he was satisfied; said he was boss
+of his son, and he'd make it warm for him.
+
+When he got out the new judge said he was a-going to make a man of him.
+So he took him to his own house, and dressed him up clean and nice, and
+had him to breakfast and dinner and supper with the family, and was just
+old pie to him, so to speak.  And after supper he talked to him about
+temperance and such things till the old man cried, and said he'd been
+a fool, and fooled away his life; but now he was a-going to turn over
+a new leaf and be a man nobody wouldn't be ashamed of, and he hoped the
+judge would help him and not look down on him.  The judge said he could
+hug him for them words; so he cried, and his wife she cried again; pap
+said he'd been a man that had always been misunderstood before, and the
+judge said he believed it.  The old man said that what a man wanted
+that was down was sympathy, and the judge said it was so; so they cried
+again.  And when it was bedtime the old man rose up and held out his
+hand, and says:
+
+"Look at it, gentlemen and ladies all; take a-hold of it; shake it.
+There's a hand that was the hand of a hog; but it ain't so no more; it's
+the hand of a man that's started in on a new life, and'll die before
+he'll go back.  You mark them words—don't forget I said them.  It's a
+clean hand now; shake it—don't be afeard."
+
+So they shook it, one after the other, all around, and cried.  The
+judge's wife she kissed it.  Then the old man he signed a pledge—made
+his mark. The judge said it was the holiest time on record, or something
+like that. Then they tucked the old man into a beautiful room, which was
+the spare room, and in the night some time he got powerful thirsty and
+clumb out on to the porch-roof and slid down a stanchion and traded his
+new coat for a jug of forty-rod, and clumb back again and had a good old
+time; and towards daylight he crawled out again, drunk as a fiddler, and
+rolled off the porch and broke his left arm in two places, and was most
+froze to death when somebody found him after sun-up.  And when they come
+to look at that spare room they had to take soundings before they could
+navigate it.
+
+The judge he felt kind of sore.  He said he reckoned a body could reform
+the old man with a shotgun, maybe, but he didn't know no other way.
+
+
+
+
+CHAPTER VI.
+
+WELL, pretty soon the old man was up and around again, and then he went
+for Judge Thatcher in the courts to make him give up that money, and he
+went for me, too, for not stopping school.  He catched me a couple of
+times and thrashed me, but I went to school just the same, and dodged
+him or outrun him most of the time.  I didn't want to go to school much
+before, but I reckoned I'd go now to spite pap.  That law trial was a
+slow business—appeared like they warn't ever going to get started on it;
+so every now and then I'd borrow two or three dollars off of the judge
+for him, to keep from getting a cowhiding.  Every time he got money he
+got drunk; and every time he got drunk he raised Cain around town; and
+every time he raised Cain he got jailed.  He was just suited—this kind
+of thing was right in his line.
+
+He got to hanging around the widow's too much and so she told him at
+last that if he didn't quit using around there she would make trouble
+for him. Well, wasn't he mad?  He said he would show who was Huck
+Finn's boss.  So he watched out for me one day in the spring, and
+catched me, and took me up the river about three mile in a skiff, and
+crossed over to the Illinois shore where it was woody and there warn't
+no houses but an old log hut in a place where the timber was so thick
+you couldn't find it if you didn't know where it was.
+
+He kept me with him all the time, and I never got a chance to run off.
+We lived in that old cabin, and he always locked the door and put the
+key under his head nights.  He had a gun which he had stole, I reckon,
+and we fished and hunted, and that was what we lived on.  Every little
+while he locked me in and went down to the store, three miles, to the
+ferry, and traded fish and game for whisky, and fetched it home and got
+drunk and had a good time, and licked me.  The widow she found out where
+I was by and by, and she sent a man over to try to get hold of me; but
+pap drove him off with the gun, and it warn't long after that till I was
+used to being where I was, and liked it—all but the cowhide part.
+
+It was kind of lazy and jolly, laying off comfortable all day, smoking
+and fishing, and no books nor study.  Two months or more run along, and
+my clothes got to be all rags and dirt, and I didn't see how I'd ever
+got to like it so well at the widow's, where you had to wash, and eat on
+a plate, and comb up, and go to bed and get up regular, and be forever
+bothering over a book, and have old Miss Watson pecking at you all the
+time.  I didn't want to go back no more.  I had stopped cussing, because
+the widow didn't like it; but now I took to it again because pap hadn't
+no objections.  It was pretty good times up in the woods there, take it
+all around.
+
+But by and by pap got too handy with his hick'ry, and I couldn't stand
+it. I was all over welts.  He got to going away so much, too, and
+locking me in.  Once he locked me in and was gone three days.  It was
+dreadful lonesome.  I judged he had got drownded, and I wasn't ever
+going to get out any more.  I was scared.  I made up my mind I would fix
+up some way to leave there.  I had tried to get out of that cabin many
+a time, but I couldn't find no way.  There warn't a window to it big
+enough for a dog to get through.  I couldn't get up the chimbly; it
+was too narrow.  The door was thick, solid oak slabs.  Pap was pretty
+careful not to leave a knife or anything in the cabin when he was away;
+I reckon I had hunted the place over as much as a hundred times; well, I
+was most all the time at it, because it was about the only way to put in
+the time.  But this time I found something at last; I found an old rusty
+wood-saw without any handle; it was laid in between a rafter and the
+clapboards of the roof. I greased it up and went to work.  There was an
+old horse-blanket nailed against the logs at the far end of the cabin
+behind the table, to keep the wind from blowing through the chinks and
+putting the candle out.  I got under the table and raised the blanket,
+and went to work to saw a section of the big bottom log out—big enough
+to let me through.  Well, it was a good long job, but I was getting
+towards the end of it when I heard pap's gun in the woods.  I got rid of
+the signs of my work, and dropped the blanket and hid my saw, and pretty
+soon pap come in.
+
+Pap warn't in a good humor—so he was his natural self.  He said he was
+down town, and everything was going wrong.  His lawyer said he reckoned
+he would win his lawsuit and get the money if they ever got started on
+the trial; but then there was ways to put it off a long time, and Judge
+Thatcher knowed how to do it. And he said people allowed there'd be
+another trial to get me away from him and give me to the widow for my
+guardian, and they guessed it would win this time.  This shook me up
+considerable, because I didn't want to go back to the widow's any more
+and be so cramped up and sivilized, as they called it.  Then the old man
+got to cussing, and cussed everything and everybody he could think of,
+and then cussed them all over again to make sure he hadn't skipped any,
+and after that he polished off with a kind of a general cuss all round,
+including a considerable parcel of people which he didn't know the names
+of, and so called them what's-his-name when he got to them, and went
+right along with his cussing.
+
+He said he would like to see the widow get me.  He said he would watch
+out, and if they tried to come any such game on him he knowed of a place
+six or seven mile off to stow me in, where they might hunt till they
+dropped and they couldn't find me.  That made me pretty uneasy again,
+but only for a minute; I reckoned I wouldn't stay on hand till he got
+that chance.
+
+The old man made me go to the skiff and fetch the things he had
+got. There was a fifty-pound sack of corn meal, and a side of bacon,
+ammunition, and a four-gallon jug of whisky, and an old book and two
+newspapers for wadding, besides some tow.  I toted up a load, and went
+back and set down on the bow of the skiff to rest.  I thought it all
+over, and I reckoned I would walk off with the gun and some lines, and
+take to the woods when I run away.  I guessed I wouldn't stay in one
+place, but just tramp right across the country, mostly night times, and
+hunt and fish to keep alive, and so get so far away that the old man nor
+the widow couldn't ever find me any more.  I judged I would saw out and
+leave that night if pap got drunk enough, and I reckoned he would.  I
+got so full of it I didn't notice how long I was staying till the old
+man hollered and asked me whether I was asleep or drownded.
+
+I got the things all up to the cabin, and then it was about dark.  While
+I was cooking supper the old man took a swig or two and got sort of
+warmed up, and went to ripping again.  He had been drunk over in town,
+and laid in the gutter all night, and he was a sight to look at.  A body
+would a thought he was Adam—he was just all mud.  Whenever his liquor
+begun to work he most always went for the govment, this time he says:
+
+"Call this a govment! why, just look at it and see what it's like.
+Here's the law a-standing ready to take a man's son away from him—a
+man's own son, which he has had all the trouble and all the anxiety
+and all the expense of raising.  Yes, just as that man has got that
+son raised at last, and ready to go to work and begin to do suthin' for
+him and give him a rest, the law up and goes for him.  And they call
+that govment!  That ain't all, nuther.  The law backs that old Judge
+Thatcher up and helps him to keep me out o' my property.  Here's what
+the law does:  The law takes a man worth six thousand dollars and
+up'ards, and jams him into an old trap of a cabin like this, and lets
+him go round in clothes that ain't fitten for a hog. They call that
+govment!  A man can't get his rights in a govment like this. Sometimes
+I've a mighty notion to just leave the country for good and all. Yes,
+and I told 'em so; I told old Thatcher so to his face.  Lots of 'em
+heard me, and can tell what I said.  Says I, for two cents I'd leave the
+blamed country and never come a-near it agin.  Them's the very words.  I
+says look at my hat—if you call it a hat—but the lid raises up and the
+rest of it goes down till it's below my chin, and then it ain't rightly
+a hat at all, but more like my head was shoved up through a jint o'
+stove-pipe.  Look at it, says I—such a hat for me to wear—one of the
+wealthiest men in this town if I could git my rights.
+
+"Oh, yes, this is a wonderful govment, wonderful.  Why, looky here.
+There was a free nigger there from Ohio—a mulatter, most as white as
+a white man.  He had the whitest shirt on you ever see, too, and the
+shiniest hat; and there ain't a man in that town that's got as fine
+clothes as what he had; and he had a gold watch and chain, and a
+silver-headed cane—the awfulest old gray-headed nabob in the State.  And
+what do you think?  They said he was a p'fessor in a college, and could
+talk all kinds of languages, and knowed everything.  And that ain't the
+wust. They said he could vote when he was at home.  Well, that let me
+out. Thinks I, what is the country a-coming to?  It was 'lection day,
+and I was just about to go and vote myself if I warn't too drunk to get
+there; but when they told me there was a State in this country where
+they'd let that nigger vote, I drawed out.  I says I'll never vote agin.
+ Them's the very words I said; they all heard me; and the country may
+rot for all me—I'll never vote agin as long as I live.  And to see the
+cool way of that nigger—why, he wouldn't a give me the road if I hadn't
+shoved him out o' the way.  I says to the people, why ain't this nigger
+put up at auction and sold?—that's what I want to know.  And what do you
+reckon they said? Why, they said he couldn't be sold till he'd been in
+the State six months, and he hadn't been there that long yet.  There,
+now—that's a specimen.  They call that a govment that can't sell a free
+nigger till he's been in the State six months.  Here's a govment that
+calls itself a govment, and lets on to be a govment, and thinks it is a
+govment, and yet's got to set stock-still for six whole months before
+it can take a hold of a prowling, thieving, infernal, white-shirted free
+nigger, and—"
+
+Pap was agoing on so he never noticed where his old limber legs was
+taking him to, so he went head over heels over the tub of salt pork and
+barked both shins, and the rest of his speech was all the hottest kind
+of language—mostly hove at the nigger and the govment, though he give
+the tub some, too, all along, here and there.  He hopped around the
+cabin considerable, first on one leg and then on the other, holding
+first one shin and then the other one, and at last he let out with his
+left foot all of a sudden and fetched the tub a rattling kick.  But it
+warn't good judgment, because that was the boot that had a couple of his
+toes leaking out of the front end of it; so now he raised a howl that
+fairly made a body's hair raise, and down he went in the dirt, and
+rolled there, and held his toes; and the cussing he done then laid over
+anything he had ever done previous.  He said so his own self afterwards.
+ He had heard old Sowberry Hagan in his best days, and he said it laid
+over him, too; but I reckon that was sort of piling it on, maybe.
+
+After supper pap took the jug, and said he had enough whisky there
+for two drunks and one delirium tremens.  That was always his word.  I
+judged he would be blind drunk in about an hour, and then I would steal
+the key, or saw myself out, one or t'other.  He drank and drank, and
+tumbled down on his blankets by and by; but luck didn't run my way.
+ He didn't go sound asleep, but was uneasy.  He groaned and moaned and
+thrashed around this way and that for a long time.  At last I got so
+sleepy I couldn't keep my eyes open all I could do, and so before I
+knowed what I was about I was sound asleep, and the candle burning.
+
+I don't know how long I was asleep, but all of a sudden there was an
+awful scream and I was up.  There was pap looking wild, and skipping
+around every which way and yelling about snakes.  He said they was
+crawling up his legs; and then he would give a jump and scream, and say
+one had bit him on the cheek—but I couldn't see no snakes.  He started
+and run round and round the cabin, hollering "Take him off! take him
+off! he's biting me on the neck!"  I never see a man look so wild in the
+eyes. Pretty soon he was all fagged out, and fell down panting; then he
+rolled over and over wonderful fast, kicking things every which way,
+and striking and grabbing at the air with his hands, and screaming and
+saying there was devils a-hold of him.  He wore out by and by, and laid
+still a while, moaning.  Then he laid stiller, and didn't make a sound.
+ I could hear the owls and the wolves away off in the woods, and it
+seemed terrible still.  He was laying over by the corner. By and by he
+raised up part way and listened, with his head to one side.  He says,
+very low:
+
+"Tramp—tramp—tramp; that's the dead; tramp—tramp—tramp; they're coming
+after me; but I won't go.  Oh, they're here! don't touch me—don't! hands
+off—they're cold; let go.  Oh, let a poor devil alone!"
+
+Then he went down on all fours and crawled off, begging them to let him
+alone, and he rolled himself up in his blanket and wallowed in under the
+old pine table, still a-begging; and then he went to crying.  I could
+hear him through the blanket.
+
+By and by he rolled out and jumped up on his feet looking wild, and he
+see me and went for me.  He chased me round and round the place with a
+clasp-knife, calling me the Angel of Death, and saying he would kill me,
+and then I couldn't come for him no more.  I begged, and told him I
+was only Huck; but he laughed such a screechy laugh, and roared and
+cussed, and kept on chasing me up.  Once when I turned short and
+dodged under his arm he made a grab and got me by the jacket between my
+shoulders, and I thought I was gone; but I slid out of the jacket quick
+as lightning, and saved myself. Pretty soon he was all tired out, and
+dropped down with his back against the door, and said he would rest a
+minute and then kill me. He put his knife under him, and said he would
+sleep and get strong, and then he would see who was who.
+
+So he dozed off pretty soon.  By and by I got the old split-bottom chair
+and clumb up as easy as I could, not to make any noise, and got down the
+gun.  I slipped the ramrod down it to make sure it was loaded, then I
+laid it across the turnip barrel, pointing towards pap, and set down
+behind it to wait for him to stir.  And how slow and still the time did
+drag along.
+
+
+
+
+CHAPTER VII.
+
+"GIT up!  What you 'bout?"
+
+I opened my eyes and looked around, trying to make out where I was.  It
+was after sun-up, and I had been sound asleep.  Pap was standing over me
+looking sour and sick, too.  He says:
+
+"What you doin' with this gun?"
+
+I judged he didn't know nothing about what he had been doing, so I says:
+
+"Somebody tried to get in, so I was laying for him."
+
+"Why didn't you roust me out?"
+
+"Well, I tried to, but I couldn't; I couldn't budge you."
+
+"Well, all right.  Don't stand there palavering all day, but out with
+you and see if there's a fish on the lines for breakfast.  I'll be along
+in a minute."
+
+He unlocked the door, and I cleared out up the river-bank.  I noticed
+some pieces of limbs and such things floating down, and a sprinkling of
+bark; so I knowed the river had begun to rise.  I reckoned I would have
+great times now if I was over at the town.  The June rise used to be
+always luck for me; because as soon as that rise begins here comes
+cordwood floating down, and pieces of log rafts—sometimes a dozen logs
+together; so all you have to do is to catch them and sell them to the
+wood-yards and the sawmill.
+
+I went along up the bank with one eye out for pap and t'other one out
+for what the rise might fetch along.  Well, all at once here comes a
+canoe; just a beauty, too, about thirteen or fourteen foot long, riding
+high like a duck.  I shot head-first off of the bank like a frog,
+clothes and all on, and struck out for the canoe.  I just expected
+there'd be somebody laying down in it, because people often done that
+to fool folks, and when a chap had pulled a skiff out most to it they'd
+raise up and laugh at him.  But it warn't so this time.  It was a
+drift-canoe sure enough, and I clumb in and paddled her ashore.  Thinks
+I, the old man will be glad when he sees this—she's worth ten dollars.
+ But when I got to shore pap wasn't in sight yet, and as I was running
+her into a little creek like a gully, all hung over with vines and
+willows, I struck another idea:  I judged I'd hide her good, and then,
+'stead of taking to the woods when I run off, I'd go down the river
+about fifty mile and camp in one place for good, and not have such a
+rough time tramping on foot.
+
+It was pretty close to the shanty, and I thought I heard the old man
+coming all the time; but I got her hid; and then I out and looked around
+a bunch of willows, and there was the old man down the path a piece just
+drawing a bead on a bird with his gun.  So he hadn't seen anything.
+
+When he got along I was hard at it taking up a "trot" line.  He abused
+me a little for being so slow; but I told him I fell in the river, and
+that was what made me so long.  I knowed he would see I was wet, and
+then he would be asking questions.  We got five catfish off the lines
+and went home.
+
+While we laid off after breakfast to sleep up, both of us being about
+wore out, I got to thinking that if I could fix up some way to keep pap
+and the widow from trying to follow me, it would be a certainer thing
+than trusting to luck to get far enough off before they missed me; you
+see, all kinds of things might happen.  Well, I didn't see no way for a
+while, but by and by pap raised up a minute to drink another barrel of
+water, and he says:
+
+"Another time a man comes a-prowling round here you roust me out, you
+hear? That man warn't here for no good.  I'd a shot him.  Next time you
+roust me out, you hear?"
+
+Then he dropped down and went to sleep again; but what he had been
+saying give me the very idea I wanted.  I says to myself, I can fix it
+now so nobody won't think of following me.
+
+About twelve o'clock we turned out and went along up the bank.  The
+river was coming up pretty fast, and lots of driftwood going by on the
+rise. By and by along comes part of a log raft—nine logs fast together.
+ We went out with the skiff and towed it ashore.  Then we had dinner.
+Anybody but pap would a waited and seen the day through, so as to catch
+more stuff; but that warn't pap's style.  Nine logs was enough for one
+time; he must shove right over to town and sell.  So he locked me in and
+took the skiff, and started off towing the raft about half-past three.
+ I judged he wouldn't come back that night.  I waited till I reckoned he
+had got a good start; then I out with my saw, and went to work on that
+log again.  Before he was t'other side of the river I was out of the
+hole; him and his raft was just a speck on the water away off yonder.
+
+I took the sack of corn meal and took it to where the canoe was hid, and
+shoved the vines and branches apart and put it in; then I done the same
+with the side of bacon; then the whisky-jug.  I took all the coffee and
+sugar there was, and all the ammunition; I took the wadding; I took the
+bucket and gourd; I took a dipper and a tin cup, and my old saw and two
+blankets, and the skillet and the coffee-pot.  I took fish-lines and
+matches and other things—everything that was worth a cent.  I cleaned
+out the place.  I wanted an axe, but there wasn't any, only the one out
+at the woodpile, and I knowed why I was going to leave that.  I fetched
+out the gun, and now I was done.
+
+I had wore the ground a good deal crawling out of the hole and dragging
+out so many things.  So I fixed that as good as I could from the outside
+by scattering dust on the place, which covered up the smoothness and the
+sawdust.  Then I fixed the piece of log back into its place, and put two
+rocks under it and one against it to hold it there, for it was bent up
+at that place and didn't quite touch ground.  If you stood four or five
+foot away and didn't know it was sawed, you wouldn't never notice
+it; and besides, this was the back of the cabin, and it warn't likely
+anybody would go fooling around there.
+
+It was all grass clear to the canoe, so I hadn't left a track.  I
+followed around to see.  I stood on the bank and looked out over the
+river.  All safe.  So I took the gun and went up a piece into the woods,
+and was hunting around for some birds when I see a wild pig; hogs soon
+went wild in them bottoms after they had got away from the prairie
+farms. I shot this fellow and took him into camp.
+
+I took the axe and smashed in the door.  I beat it and hacked it
+considerable a-doing it.  I fetched the pig in, and took him back nearly
+to the table and hacked into his throat with the axe, and laid him down
+on the ground to bleed; I say ground because it was ground—hard packed,
+and no boards.  Well, next I took an old sack and put a lot of big rocks
+in it—all I could drag—and I started it from the pig, and dragged it to
+the door and through the woods down to the river and dumped it in, and
+down it sunk, out of sight.  You could easy see that something had been
+dragged over the ground.  I did wish Tom Sawyer was there; I knowed he
+would take an interest in this kind of business, and throw in the fancy
+touches.  Nobody could spread himself like Tom Sawyer in such a thing as
+that.
+
+Well, last I pulled out some of my hair, and blooded the axe good, and
+stuck it on the back side, and slung the axe in the corner.  Then I
+took up the pig and held him to my breast with my jacket (so he couldn't
+drip) till I got a good piece below the house and then dumped him into
+the river.  Now I thought of something else.  So I went and got the bag
+of meal and my old saw out of the canoe, and fetched them to the house.
+ I took the bag to where it used to stand, and ripped a hole in the
+bottom of it with the saw, for there warn't no knives and forks on the
+place—pap done everything with his clasp-knife about the cooking.  Then
+I carried the sack about a hundred yards across the grass and through
+the willows east of the house, to a shallow lake that was five mile wide
+and full of rushes—and ducks too, you might say, in the season.  There
+was a slough or a creek leading out of it on the other side that went
+miles away, I don't know where, but it didn't go to the river.  The meal
+sifted out and made a little track all the way to the lake.  I dropped
+pap's whetstone there too, so as to look like it had been done by
+accident. Then I tied up the rip in the meal sack with a string, so it
+wouldn't leak no more, and took it and my saw to the canoe again.
+
+It was about dark now; so I dropped the canoe down the river under some
+willows that hung over the bank, and waited for the moon to rise.  I
+made fast to a willow; then I took a bite to eat, and by and by laid
+down in the canoe to smoke a pipe and lay out a plan.  I says to myself,
+they'll follow the track of that sackful of rocks to the shore and then
+drag the river for me.  And they'll follow that meal track to the lake
+and go browsing down the creek that leads out of it to find the robbers
+that killed me and took the things.  They won't ever hunt the river for
+anything but my dead carcass. They'll soon get tired of that, and won't
+bother no more about me.  All right; I can stop anywhere I want to.
+Jackson's Island is good enough for me; I know that island pretty well,
+and nobody ever comes there.  And then I can paddle over to town nights,
+and slink around and pick up things I want. Jackson's Island's the
+place.
+
+I was pretty tired, and the first thing I knowed I was asleep.  When
+I woke up I didn't know where I was for a minute.  I set up and looked
+around, a little scared.  Then I remembered.  The river looked miles and
+miles across.  The moon was so bright I could a counted the drift logs
+that went a-slipping along, black and still, hundreds of yards out from
+shore. Everything was dead quiet, and it looked late, and smelt late.
+You know what I mean—I don't know the words to put it in.
+
+I took a good gap and a stretch, and was just going to unhitch and start
+when I heard a sound away over the water.  I listened.  Pretty soon I
+made it out.  It was that dull kind of a regular sound that comes from
+oars working in rowlocks when it's a still night.  I peeped out through
+the willow branches, and there it was—a skiff, away across the water.
+ I couldn't tell how many was in it.  It kept a-coming, and when it was
+abreast of me I see there warn't but one man in it.  Think's I, maybe
+it's pap, though I warn't expecting him.  He dropped below me with the
+current, and by and by he came a-swinging up shore in the easy water,
+and he went by so close I could a reached out the gun and touched him.
+ Well, it was pap, sure enough—and sober, too, by the way he laid his
+oars.
+
+I didn't lose no time.  The next minute I was a-spinning down stream
+soft but quick in the shade of the bank.  I made two mile and a half,
+and then struck out a quarter of a mile or more towards the middle of
+the river, because pretty soon I would be passing the ferry landing, and
+people might see me and hail me.  I got out amongst the driftwood, and
+then laid down in the bottom of the canoe and let her float.
+
+ I laid there, and had a good rest and a smoke out of my pipe, looking
+away into the sky; not a cloud in it.  The sky looks ever so deep when
+you lay down on your back in the moonshine; I never knowed it before.
+ And how far a body can hear on the water such nights!  I heard people
+talking at the ferry landing. I heard what they said, too—every word
+of it.  One man said it was getting towards the long days and the short
+nights now.  T'other one said this warn't one of the short ones, he
+reckoned—and then they laughed, and he said it over again, and they
+laughed again; then they waked up another fellow and told him, and
+laughed, but he didn't laugh; he ripped out something brisk, and said
+let him alone.  The first fellow said he 'lowed to tell it to his
+old woman—she would think it was pretty good; but he said that warn't
+nothing to some things he had said in his time. I heard one man say it
+was nearly three o'clock, and he hoped daylight wouldn't wait more than
+about a week longer.  After that the talk got further and further away,
+and I couldn't make out the words any more; but I could hear the mumble,
+and now and then a laugh, too, but it seemed a long ways off.
+
+I was away below the ferry now.  I rose up, and there was Jackson's
+Island, about two mile and a half down stream, heavy timbered and
+standing up out of the middle of the river, big and dark and solid, like
+a steamboat without any lights.  There warn't any signs of the bar at
+the head—it was all under water now.
+
+It didn't take me long to get there.  I shot past the head at a ripping
+rate, the current was so swift, and then I got into the dead water and
+landed on the side towards the Illinois shore.  I run the canoe into
+a deep dent in the bank that I knowed about; I had to part the willow
+branches to get in; and when I made fast nobody could a seen the canoe
+from the outside.
+
+I went up and set down on a log at the head of the island, and looked
+out on the big river and the black driftwood and away over to the town,
+three mile away, where there was three or four lights twinkling.  A
+monstrous big lumber-raft was about a mile up stream, coming along down,
+with a lantern in the middle of it.  I watched it come creeping down,
+and when it was most abreast of where I stood I heard a man say, "Stern
+oars, there! heave her head to stabboard!"  I heard that just as plain
+as if the man was by my side.
+
+There was a little gray in the sky now; so I stepped into the woods, and
+laid down for a nap before breakfast.
+
+
+
+
+CHAPTER VIII.
+
+THE sun was up so high when I waked that I judged it was after eight
+o'clock.  I laid there in the grass and the cool shade thinking about
+things, and feeling rested and ruther comfortable and satisfied.  I
+could see the sun out at one or two holes, but mostly it was big trees
+all about, and gloomy in there amongst them.  There was freckled places
+on the ground where the light sifted down through the leaves, and the
+freckled places swapped about a little, showing there was a little
+breeze up there.  A couple of squirrels set on a limb and jabbered at me
+very friendly.
+
+I was powerful lazy and comfortable—didn't want to get up and cook
+breakfast.  Well, I was dozing off again when I thinks I hears a deep
+sound of "boom!" away up the river.  I rouses up, and rests on my elbow
+and listens; pretty soon I hears it again.  I hopped up, and went and
+looked out at a hole in the leaves, and I see a bunch of smoke laying
+on the water a long ways up—about abreast the ferry.  And there was the
+ferryboat full of people floating along down.  I knowed what was the
+matter now.  "Boom!" I see the white smoke squirt out of the ferryboat's
+side.  You see, they was firing cannon over the water, trying to make my
+carcass come to the top.
+
+I was pretty hungry, but it warn't going to do for me to start a fire,
+because they might see the smoke.  So I set there and watched the
+cannon-smoke and listened to the boom.  The river was a mile wide there,
+and it always looks pretty on a summer morning—so I was having a good
+enough time seeing them hunt for my remainders if I only had a bite to
+eat. Well, then I happened to think how they always put quicksilver in
+loaves of bread and float them off, because they always go right to the
+drownded carcass and stop there.  So, says I, I'll keep a lookout, and
+if any of them's floating around after me I'll give them a show.  I
+changed to the Illinois edge of the island to see what luck I could
+have, and I warn't disappointed.  A big double loaf come along, and I
+most got it with a long stick, but my foot slipped and she floated out
+further.  Of course I was where the current set in the closest to the
+shore—I knowed enough for that.  But by and by along comes another one,
+and this time I won.  I took out the plug and shook out the little dab
+of quicksilver, and set my teeth in.  It was "baker's bread"—what the
+quality eat; none of your low-down corn-pone.
+
+I got a good place amongst the leaves, and set there on a log, munching
+the bread and watching the ferry-boat, and very well satisfied.  And
+then something struck me.  I says, now I reckon the widow or the parson
+or somebody prayed that this bread would find me, and here it has gone
+and done it.  So there ain't no doubt but there is something in that
+thing—that is, there's something in it when a body like the widow or the
+parson prays, but it don't work for me, and I reckon it don't work for
+only just the right kind.
+
+I lit a pipe and had a good long smoke, and went on watching.  The
+ferryboat was floating with the current, and I allowed I'd have a chance
+to see who was aboard when she come along, because she would come in
+close, where the bread did.  When she'd got pretty well along down
+towards me, I put out my pipe and went to where I fished out the bread,
+and laid down behind a log on the bank in a little open place.  Where
+the log forked I could peep through.
+
+By and by she come along, and she drifted in so close that they could
+a run out a plank and walked ashore.  Most everybody was on the boat.
+ Pap, and Judge Thatcher, and Bessie Thatcher, and Jo Harper, and Tom
+Sawyer, and his old Aunt Polly, and Sid and Mary, and plenty more.
+ Everybody was talking about the murder, but the captain broke in and
+says:
+
+"Look sharp, now; the current sets in the closest here, and maybe he's
+washed ashore and got tangled amongst the brush at the water's edge.  I
+hope so, anyway."
+
+I didn't hope so.  They all crowded up and leaned over the rails, nearly
+in my face, and kept still, watching with all their might.  I could see
+them first-rate, but they couldn't see me.  Then the captain sung out:
+
+"Stand away!" and the cannon let off such a blast right before me that
+it made me deef with the noise and pretty near blind with the smoke, and
+I judged I was gone.  If they'd a had some bullets in, I reckon they'd
+a got the corpse they was after.  Well, I see I warn't hurt, thanks to
+goodness. The boat floated on and went out of sight around the shoulder
+of the island.  I could hear the booming now and then, further and
+further off, and by and by, after an hour, I didn't hear it no more.
+ The island was three mile long.  I judged they had got to the foot, and
+was giving it up.  But they didn't yet a while.  They turned around
+the foot of the island and started up the channel on the Missouri side,
+under steam, and booming once in a while as they went.  I crossed over
+to that side and watched them. When they got abreast the head of the
+island they quit shooting and dropped over to the Missouri shore and
+went home to the town.
+
+I knowed I was all rig

<TRUNCATED>

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

Posted by xe...@apache.org.
Integrate SASI index into Cassandra

patch by xedin; reviewed by beobal for CASSANDRA-10661


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/72790dc8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/72790dc8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/72790dc8

Branch: refs/heads/trunk
Commit: 72790dc8e34826b39ac696b03025ae6b7b6beb2b
Parents: 11c8ca6
Author: Pavel Yaskevich <xe...@apache.org>
Authored: Wed Dec 2 19:23:54 2015 -0800
Committer: Pavel Yaskevich <xe...@apache.org>
Committed: Sat Jan 23 19:35:29 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |     1 +
 build.xml                                       |    22 +-
 doc/SASI.md                                     |   768 +
 lib/concurrent-trees-2.4.0.jar                  |   Bin 0 -> 118696 bytes
 lib/hppc-0.5.4.jar                              |   Bin 0 -> 1305173 bytes
 lib/jflex-1.6.0.jar                             |   Bin 0 -> 1048690 bytes
 lib/licenses/concurrent-trees-2.4.0.txt         |   201 +
 lib/licenses/hppc-0.5.4.txt                     |   202 +
 lib/licenses/jflex-1.6.0.txt                    |   201 +
 lib/licenses/primitive-1.0.txt                  |   201 +
 lib/licenses/snowball-stemmer-1.3.0.581.1.txt   |   201 +
 lib/primitive-1.0.jar                           |   Bin 0 -> 52589 bytes
 lib/snowball-stemmer-1.3.0.581.1.jar            |   Bin 0 -> 93019 bytes
 .../cassandra/config/DatabaseDescriptor.java    |     7 +-
 .../org/apache/cassandra/db/ColumnIndex.java    |     6 +-
 .../apache/cassandra/db/filter/RowFilter.java   |    15 +-
 .../cassandra/index/SecondaryIndexManager.java  |    11 +
 .../apache/cassandra/index/sasi/SASIIndex.java  |   288 +
 .../cassandra/index/sasi/SASIIndexBuilder.java  |   128 +
 .../cassandra/index/sasi/SSTableIndex.java      |   187 +
 .../org/apache/cassandra/index/sasi/Term.java   |    65 +
 .../cassandra/index/sasi/TermIterator.java      |   208 +
 .../index/sasi/analyzer/AbstractAnalyzer.java   |    51 +
 .../index/sasi/analyzer/NoOpAnalyzer.java       |    54 +
 .../sasi/analyzer/NonTokenizingAnalyzer.java    |   126 +
 .../sasi/analyzer/NonTokenizingOptions.java     |   147 +
 .../sasi/analyzer/SUPPLEMENTARY.jflex-macro     |   143 +
 .../index/sasi/analyzer/StandardAnalyzer.java   |   194 +
 .../sasi/analyzer/StandardTokenizerImpl.jflex   |   220 +
 .../analyzer/StandardTokenizerInterface.java    |    65 +
 .../sasi/analyzer/StandardTokenizerOptions.java |   272 +
 .../analyzer/filter/BasicResultFilters.java     |    76 +
 .../analyzer/filter/FilterPipelineBuilder.java  |    51 +
 .../analyzer/filter/FilterPipelineExecutor.java |    53 +
 .../analyzer/filter/FilterPipelineTask.java     |    52 +
 .../sasi/analyzer/filter/StemmerFactory.java    |   101 +
 .../sasi/analyzer/filter/StemmingFilters.java   |    46 +
 .../sasi/analyzer/filter/StopWordFactory.java   |   100 +
 .../sasi/analyzer/filter/StopWordFilters.java   |    42 +
 .../cassandra/index/sasi/conf/ColumnIndex.java  |   193 +
 .../cassandra/index/sasi/conf/DataTracker.java  |   162 +
 .../cassandra/index/sasi/conf/IndexMode.java    |   169 +
 .../index/sasi/conf/view/PrefixTermTree.java    |   194 +
 .../index/sasi/conf/view/RangeTermTree.java     |    77 +
 .../index/sasi/conf/view/TermTree.java          |    58 +
 .../cassandra/index/sasi/conf/view/View.java    |   104 +
 .../cassandra/index/sasi/disk/Descriptor.java   |    51 +
 .../cassandra/index/sasi/disk/OnDiskBlock.java  |   142 +
 .../cassandra/index/sasi/disk/OnDiskIndex.java  |   773 ++
 .../index/sasi/disk/OnDiskIndexBuilder.java     |   627 +
 .../index/sasi/disk/PerSSTableIndexWriter.java  |   361 +
 .../apache/cassandra/index/sasi/disk/Token.java |    42 +
 .../cassandra/index/sasi/disk/TokenTree.java    |   519 +
 .../index/sasi/disk/TokenTreeBuilder.java       |   839 ++
 .../exceptions/TimeQuotaExceededException.java  |    21 +
 .../index/sasi/memory/IndexMemtable.java        |    71 +
 .../index/sasi/memory/KeyRangeIterator.java     |   118 +
 .../cassandra/index/sasi/memory/MemIndex.java   |    51 +
 .../index/sasi/memory/SkipListMemIndex.java     |    97 +
 .../index/sasi/memory/TrieMemIndex.java         |   254 +
 .../cassandra/index/sasi/plan/Expression.java   |   340 +
 .../cassandra/index/sasi/plan/Operation.java    |   477 +
 .../index/sasi/plan/QueryController.java        |   261 +
 .../cassandra/index/sasi/plan/QueryPlan.java    |   170 +
 .../cassandra/index/sasi/sa/ByteTerm.java       |    51 +
 .../cassandra/index/sasi/sa/CharTerm.java       |    54 +
 .../cassandra/index/sasi/sa/IntegralSA.java     |    84 +
 .../org/apache/cassandra/index/sasi/sa/SA.java  |    58 +
 .../cassandra/index/sasi/sa/SuffixSA.java       |   143 +
 .../apache/cassandra/index/sasi/sa/Term.java    |    58 +
 .../cassandra/index/sasi/sa/TermIterator.java   |    31 +
 .../index/sasi/utils/AbstractIterator.java      |   155 +
 .../index/sasi/utils/CombinedTerm.java          |   103 +
 .../index/sasi/utils/CombinedTermIterator.java  |    87 +
 .../index/sasi/utils/CombinedValue.java         |    25 +
 .../index/sasi/utils/MappedBuffer.java          |   253 +
 .../index/sasi/utils/OnDiskIndexIterator.java   |    64 +
 .../sasi/utils/RangeIntersectionIterator.java   |   281 +
 .../index/sasi/utils/RangeIterator.java         |   279 +
 .../index/sasi/utils/RangeUnionIterator.java    |   158 +
 .../cassandra/index/sasi/utils/TypeUtil.java    |    84 +
 .../sasi/utils/trie/AbstractPatriciaTrie.java   |  1151 ++
 .../index/sasi/utils/trie/AbstractTrie.java     |   230 +
 .../cassandra/index/sasi/utils/trie/Cursor.java |    83 +
 .../index/sasi/utils/trie/KeyAnalyzer.java      |    73 +
 .../index/sasi/utils/trie/PatriciaTrie.java     |  1261 ++
 .../cassandra/index/sasi/utils/trie/Trie.java   |   152 +
 .../cassandra/index/sasi/utils/trie/Tries.java  |    95 +
 .../apache/cassandra/io/sstable/Component.java  |    13 +-
 .../cassandra/io/sstable/KeyIterator.java       |     7 +
 .../io/sstable/format/SSTableFlushObserver.java |    12 +-
 .../io/sstable/format/SSTableReader.java        |    20 +
 .../io/sstable/format/SSTableWriter.java        |     9 +-
 .../io/util/DataOutputBufferFixed.java          |     5 +
 .../cassandra/io/util/SequentialWriter.java     |     7 +
 .../apache/cassandra/utils/ByteBufferUtil.java  |    41 +
 .../org/apache/cassandra/utils/FBUtilities.java |     5 +
 .../cassandra/utils/memory/MemoryUtil.java      |     5 +-
 .../index/sasi/analyzer/filter/ar_ST.txt        |   163 +
 .../index/sasi/analyzer/filter/bg_ST.txt        |   260 +
 .../index/sasi/analyzer/filter/cs_ST.txt        |   257 +
 .../index/sasi/analyzer/filter/de_ST.txt        |   604 +
 .../index/sasi/analyzer/filter/en_ST.txt        |   572 +
 .../index/sasi/analyzer/filter/es_ST.txt        |   308 +
 .../index/sasi/analyzer/filter/fi_ST.txt        |   748 +
 .../index/sasi/analyzer/filter/fr_ST.txt        |   464 +
 .../index/sasi/analyzer/filter/hi_ST.txt        |   164 +
 .../index/sasi/analyzer/filter/hu_ST.txt        |   738 +
 .../index/sasi/analyzer/filter/it_ST.txt        |   400 +
 .../index/sasi/analyzer/filter/pl_ST.txt        |   139 +
 .../index/sasi/analyzer/filter/pt_ST.txt        |   357 +
 .../index/sasi/analyzer/filter/ro_ST.txt        |   283 +
 .../index/sasi/analyzer/filter/ru_ST.txt        |   423 +
 .../index/sasi/analyzer/filter/sv_ST.txt        |   387 +
 test/conf/cassandra-murmur.yaml                 |    43 +
 ...dventures_of_huckleberry_finn_mark_twain.txt | 12361 +++++++++++++++++
 .../tokenization/apache_license_header.txt      |    16 +
 test/resources/tokenization/ja_jp_1.txt         |     1 +
 test/resources/tokenization/ja_jp_2.txt         |     2 +
 test/resources/tokenization/lorem_ipsum.txt     |     1 +
 test/resources/tokenization/ru_ru_1.txt         |    19 +
 .../tokenization/top_visited_domains.txt        |     3 +
 test/resources/tokenization/zn_tw_1.txt         |    19 +
 .../unit/org/apache/cassandra/SchemaLoader.java |   139 +
 .../apache/cassandra/cql3/KeyCacheCqlTest.java  |    14 +-
 .../cassandra/index/sasi/SASIIndexTest.java     |  1852 +++
 .../analyzer/NonTokenizingAnalyzerTest.java     |    78 +
 .../sasi/analyzer/StandardAnalyzerTest.java     |   196 +
 .../index/sasi/disk/OnDiskIndexTest.java        |   856 ++
 .../sasi/disk/PerSSTableIndexWriterTest.java    |   161 +
 .../index/sasi/disk/TokenTreeTest.java          |   535 +
 .../index/sasi/plan/OperationTest.java          |   645 +
 .../index/sasi/utils/LongIterator.java          |   106 +
 .../index/sasi/utils/MappedBufferTest.java      |   540 +
 .../utils/RangeIntersectionIteratorTest.java    |   387 +
 .../sasi/utils/RangeUnionIteratorTest.java      |   306 +
 .../format/SSTableFlushObserverTest.java        |     5 +-
 137 files changed, 40339 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 650bf5f..9309ef4 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.4
+ * Integrate SASI index into Cassandra (CASSANDRA-10661)
  * Add --skip-flush option to nodetool snapshot
  * Skip values for non-queried columns (CASSANDRA-10657)
  * Add support for secondary indexes on static columns (CASSANDRA-8103)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index f9f42f7..035628a 100644
--- a/build.xml
+++ b/build.xml
@@ -244,6 +244,15 @@
     </target>
 
     <!--
+        Generates Java sources for tokenization support from jflex
+        grammar files
+    -->
+    <target name="generate-jflex-java" description="Generate Java from jflex grammar">
+        <taskdef classname="jflex.anttask.JFlexTask" classpath="${build.lib}/jflex-1.6.0.jar" name="jflex" />
+        <jflex file="${build.src.java}/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerImpl.jflex" destdir="${build.src.gen-java}/" />
+    </target>
+
+    <!--
        Fetch Maven Ant Tasks and Cassandra's dependencies
        These targets are intentionally free of dependencies so that they
        can be run stand-alone from a binary release artifact.
@@ -405,6 +414,11 @@
           	<exclusion groupId="log4j" artifactId="log4j"/>
           </dependency>
           <dependency groupId="joda-time" artifactId="joda-time" version="2.4" />
+          <dependency groupId="com.carrotsearch" artifactId="hppc" version="0.5.4" />
+          <dependency groupId="de.jflex" artifactId="jflex" version="1.6.0" />
+          <dependency groupId="net.mintern" artifactId="primitive" version="1.0" />
+          <dependency groupId="com.github.rholder" artifactId="snowball-stemmer" version="1.3.0.581.1" />
+          <dependency groupId="com.googlecode.concurrent-trees" artifactId="concurrent-trees" version="2.4.0" />
 
         </dependencyManagement>
         <developer id="alakshman" name="Avinash Lakshman"/>
@@ -568,6 +582,12 @@
         <dependency groupId="org.slf4j" artifactId="log4j-over-slf4j"/>
         <dependency groupId="org.slf4j" artifactId="jcl-over-slf4j"/>
         <dependency groupId="org.apache.thrift" artifactId="libthrift"/>
+        <dependency groupId="com.carrotsearch" artifactId="hppc" version="0.5.4" />
+        <dependency groupId="de.jflex" artifactId="jflex" version="1.6.0" />
+        <dependency groupId="net.mintern" artifactId="primitive" version="1.0" />
+        <dependency groupId="com.github.rholder" artifactId="snowball-stemmer" version="1.3.0.581.1" />
+        <dependency groupId="com.googlecode.concurrent-trees" artifactId="concurrent-trees" version="2.4.0" />
+
       </artifact:pom>
       <artifact:pom id="clientutil-pom"
                     artifactId="cassandra-clientutil"
@@ -734,7 +754,7 @@
         depends="maven-ant-tasks-retrieve-build,build-project" description="Compile Cassandra classes"/>
     <target name="codecoverage" depends="jacoco-run,jacoco-report" description="Create code coverage report"/>
 
-    <target depends="init,gen-cql3-grammar,generate-cql-html"
+    <target depends="init,gen-cql3-grammar,generate-cql-html,generate-jflex-java"
             name="build-project">
         <echo message="${ant.project.name}: ${ant.file}"/>
         <!-- Order matters! -->

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/doc/SASI.md
----------------------------------------------------------------------
diff --git a/doc/SASI.md b/doc/SASI.md
new file mode 100644
index 0000000..64573b8
--- /dev/null
+++ b/doc/SASI.md
@@ -0,0 +1,768 @@
+# SASIIndex
+
+[`SASIIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/SASIIndex.java),
+or "SASI" for short, is an implementation of Cassandra's
+`Index` interface that can be used as an alternative to the
+existing implementations. SASI's indexing and querying improves on
+existing implementations by tailoring it specifically to Cassandra's
+needs. SASI has superior performance in cases where queries would
+previously require filtering. In achieving this performance, SASI aims
+to be significantly less resource intensive than existing
+implementations, in memory, disk, and CPU usage. In addition, SASI
+supports prefix and contains queries on strings (similar to SQL's
+`LIKE = "foo*"` or `LIKE = "*foo*"'`).
+
+The following goes on describe how to get up and running with SASI,
+demonstrates usage with examples, and provides some details on its
+implementation.
+
+## Using SASI
+
+The examples below walk through creating a table and indexes on its
+columns, and performing queries on some inserted data. The patchset in
+this repository includes support for the Thrift and CQL3 interfaces.
+
+The examples below assume the `demo` keyspace has been created and is
+in use.
+
+```
+cqlsh> CREATE KEYSPACE demo WITH replication = {
+   ... 'class': 'SimpleStrategy',
+   ... 'replication_factor': '1'
+   ... };
+cqlsh> USE demo;
+```
+
+All examples are performed on the `sasi` table:
+
+```
+cqlsh:demo> CREATE TABLE sasi (id uuid, first_name text, last_name text,
+        ... age int, height int, created_at bigint, primary key (id));
+```
+
+#### Creating Indexes
+
+To create SASI indexes use CQLs `CREATE CUSTOM INDEX` statement:
+
+```
+cqlsh:demo> CREATE CUSTOM INDEX ON sasi (first_name) USING 'org.apache.cassandra.index.sasi.SASIIndex'
+        ... WITH OPTIONS = {
+        ... 'analyzer_class':
+        ...   'org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer',
+        ... 'case_sensitive': 'false'
+        ... };
+
+cqlsh:demo> CREATE CUSTOM INDEX ON sasi (last_name) USING 'org.apache.cassandra.index.sasi.SASIIndex'
+        ... WITH OPTIONS = {'mode': 'CONTAINS'};
+
+cqlsh:demo> CREATE CUSTOM INDEX ON sasi (age) USING 'org.apache.cassandra.index.sasi.SASIIndex';
+
+cqlsh:demo> CREATE CUSTOM INDEX ON sasi (created_at) USING 'org.apache.cassandra.index.sasi.SASIIndex'
+        ...  WITH OPTIONS = {'mode': 'SPARSE'};
+```
+
+The indexes created have some options specified that customize their
+behaviour and potentially performance. The index on `first_name` is
+case-insensitive. The analyzers are discussed more in a subsequent
+example. The `NonTokenizingAnalyzer` performs no analysis on the
+text. Each index has a mode: `PREFIX`, `CONTAINS`, or `SPARSE`, the
+first being the default. The `last_name` index is created with the
+mode `CONTAINS` which matches terms on suffixes instead of prefix
+only. Examples of this are available below and more detail can be
+found in the section on
+[OnDiskIndex](https://github.com/xedin/sasi#ondiskindexbuilder).The
+`created_at` column is created with its mode set to `SPARSE`, which is
+meant to improve performance of querying large, dense number ranges
+like timestamps for data inserted every millisecond. Details of the
+`SPARSE` implementation can also be found in the section on the
+[OnDiskIndex](https://github.com/xedin/sasi#ondiskindexbuilder). The `age`
+index is created with the default `PREFIX` mode and no
+case-sensitivity or text analysis options are specified since the
+field is numeric.
+
+After inserting the following data and performing a `nodetool flush`,
+SASI performing index flushes to disk can be seen in Cassandra's logs
+-- although the direct call to flush is not required (see
+[IndexMemtable](#indexmemtable) for more details).
+
+```
+cqlsh:demo> INSERT INTO sasi (id, first_name, last_name, age, height, created_at)
+        ... VALUES (556ebd54-cbe5-4b75-9aae-bf2a31a24500, 'Pavel', 'Yaskevich', 27, 181, 1442959315018);
+
+cqlsh:demo> INSERT INTO sasi (id, first_name, last_name, age, height, created_at)
+        ... VALUES (5770382a-c56f-4f3f-b755-450e24d55217, 'Jordan', 'West', 26, 173, 1442959315019);
+
+cqlsh:demo> INSERT INTO sasi (id, first_name, last_name, age, height, created_at)
+        ... VALUES (96053844-45c3-4f15-b1b7-b02c441d3ee1, 'Mikhail', 'Stepura', 36, 173, 1442959315020);
+
+cqlsh:demo> INSERT INTO sasi (id, first_name, last_name, age, height, created_at)
+        ... VALUES (f5dfcabe-de96-4148-9b80-a1c41ed276b4, 'Michael', 'Kjellman', 26, 180, 1442959315021);
+
+cqlsh:demo> INSERT INTO sasi (id, first_name, last_name, age, height, created_at)
+        ... VALUES (2970da43-e070-41a8-8bcb-35df7a0e608a, 'Johnny', 'Zhang', 32, 175, 1442959315022);
+
+cqlsh:demo> INSERT INTO sasi (id, first_name, last_name, age, height, created_at)
+        ... VALUES (6b757016-631d-4fdb-ac62-40b127ccfbc7, 'Jason', 'Brown', 40, 182, 1442959315023);
+
+cqlsh:demo> INSERT INTO sasi (id, first_name, last_name, age, height, created_at)
+        ... VALUES (8f909e8a-008e-49dd-8d43-1b0df348ed44, 'Vijay', 'Parthasarathy', 34, 183, 1442959315024);
+
+cqlsh:demo> SELECT first_name, last_name, age, height, created_at FROM sasi;
+
+ first_name | last_name     | age | height | created_at
+------------+---------------+-----+--------+---------------
+    Michael |      Kjellman |  26 |    180 | 1442959315021
+    Mikhail |       Stepura |  36 |    173 | 1442959315020
+      Jason |         Brown |  40 |    182 | 1442959315023
+      Pavel |     Yaskevich |  27 |    181 | 1442959315018
+      Vijay | Parthasarathy |  34 |    183 | 1442959315024
+     Jordan |          West |  26 |    173 | 1442959315019
+     Johnny |         Zhang |  32 |    175 | 1442959315022
+
+(7 rows)
+```
+
+#### Equality & Prefix Queries
+
+SASI supports simple queries already supported by CQL, however, for
+text fields like `first_name` equals queries perform prefix searches
+-- this is similar to `first_name LIKE 'M*'` in SQL (excluding case
+sensitivity, which is dependent on the index configuration). The
+semantics of CQL's `=` were modified instead of making further
+modifications of the grammar with the introduction of a `LIKE`
+operator. Ideally, CQL would be modified to include such an operator,
+supporting both prefix and suffix searches.
+
+```
+cqlsh:demo> SELECT first_name, last_name, age, height, created_at FROM sasi
+        ... WHERE first_name = 'M';
+
+ first_name | last_name | age | height | created_at
+------------+-----------+-----+--------+---------------
+    Michael |  Kjellman |  26 |    180 | 1442959315021
+    Mikhail |   Stepura |  36 |    173 | 1442959315020
+
+(2 rows)
+```
+
+Of course, the case of the query does not matter for the `first_name`
+column because of the options provided at index creation time.
+
+```
+cqlsh:demo> SELECT first_name, last_name, age, height, created_at FROM sasi
+        ... WHERE first_name = 'm';
+
+ first_name | last_name | age | height | created_at
+------------+-----------+-----+--------+---------------
+    Michael |  Kjellman |  26 |    180 | 1442959315021
+    Mikhail |   Stepura |  36 |    173 | 1442959315020
+
+(2 rows)
+```
+
+#### Compound Queries
+
+SASI supports queries with multiple predicates, however, due to the
+nature of the default indexing implementation, CQL requires the user
+to specify `ALLOW FILTERING` to opt-in to the potential performance
+pitfalls of such a query. With SASI, while the requirement to include
+`ALLOW FILTERING` remains, to reduce modifications to the grammar, the
+performance pitfalls do not exist because filtering is not
+performed. Details on how SASI joins data from multiple predicates is
+available below in the
+[Implementation Details](https://github.com/xedin/sasi#implementation-details)
+section.
+
+```
+cqlsh:demo> SELECT first_name, last_name, age, height, created_at FROM sasi
+        ... WHERE first_name = 'M' and age < 30 ALLOW FILTERING;
+
+ first_name | last_name | age | height | created_at
+------------+-----------+-----+--------+---------------
+    Michael |  Kjellman |  26 |    180 | 1442959315021
+
+(1 rows)
+```
+
+#### Suffix Queries
+
+The next example demonstrates `CONTAINS` mode on the `last_name`
+column. By using this mode predicates can search for any strings
+containing the search string as a sub-string. In this case the strings
+containing "a" or "an".
+
+```
+cqlsh:demo> SELECT * FROM sasi WHERE last_name = 'a';
+
+ id                                   | age | created_at    | first_name | height | last_name
+--------------------------------------+-----+---------------+------------+--------+---------------
+ f5dfcabe-de96-4148-9b80-a1c41ed276b4 |  26 | 1442959315021 |    Michael |    180 |      Kjellman
+ 96053844-45c3-4f15-b1b7-b02c441d3ee1 |  36 | 1442959315020 |    Mikhail |    173 |       Stepura
+ 556ebd54-cbe5-4b75-9aae-bf2a31a24500 |  27 | 1442959315018 |      Pavel |    181 |     Yaskevich
+ 8f909e8a-008e-49dd-8d43-1b0df348ed44 |  34 | 1442959315024 |      Vijay |    183 | Parthasarathy
+ 2970da43-e070-41a8-8bcb-35df7a0e608a |  32 | 1442959315022 |     Johnny |    175 |         Zhang
+
+(5 rows)
+
+cqlsh:demo> SELECT * FROM sasi WHERE last_name = 'an';
+
+ id                                   | age | created_at    | first_name | height | last_name
+--------------------------------------+-----+---------------+------------+--------+-----------
+ f5dfcabe-de96-4148-9b80-a1c41ed276b4 |  26 | 1442959315021 |    Michael |    180 |  Kjellman
+ 2970da43-e070-41a8-8bcb-35df7a0e608a |  32 | 1442959315022 |     Johnny |    175 |     Zhang
+
+(2 rows)
+```
+
+#### Expressions on Non-Indexed Columns
+
+SASI also supports filtering on non-indexed columns like `height`. The
+expression can only narrow down an existing query using `AND`.
+
+```
+cqlsh:demo> SELECT * FROM sasi WHERE last_name = 'a' AND height >= 175 ALLOW FILTERING;
+
+ id                                   | age | created_at    | first_name | height | last_name
+--------------------------------------+-----+---------------+------------+--------+---------------
+ f5dfcabe-de96-4148-9b80-a1c41ed276b4 |  26 | 1442959315021 |    Michael |    180 |      Kjellman
+ 556ebd54-cbe5-4b75-9aae-bf2a31a24500 |  27 | 1442959315018 |      Pavel |    181 |     Yaskevich
+ 8f909e8a-008e-49dd-8d43-1b0df348ed44 |  34 | 1442959315024 |      Vijay |    183 | Parthasarathy
+ 2970da43-e070-41a8-8bcb-35df7a0e608a |  32 | 1442959315022 |     Johnny |    175 |         Zhang
+
+(4 rows)
+```
+
+#### Text Analysis (Tokenization and Stemming)
+
+Lastly, to demonstrate text analysis an additional column is needed on
+the table. Its definition, index, and statements to update rows are shown below.
+
+```
+cqlsh:demo> ALTER TABLE sasi ADD bio text;
+cqlsh:demo> CREATE CUSTOM INDEX ON sasi (bio) USING 'org.apache.cassandra.index.sasi.SASIIndex'
+        ... WITH OPTIONS = {
+        ... 'analyzer_class': 'org.apache.cassandra.index.sasi.analyzer.StandardAnalyzer',
+        ... 'tokenization_enable_stemming': 'true',
+        ... 'analyzed': 'true',
+        ... 'tokenization_normalize_lowercase': 'true',
+        ... 'tokenization_locale': 'en'
+        ... };
+cqlsh:demo> UPDATE sasi SET bio = 'Software Engineer, who likes distributed systems, doesnt like to argue.' WHERE id = 5770382a-c56f-4f3f-b755-450e24d55217;
+cqlsh:demo> UPDATE sasi SET bio = 'Software Engineer, works on the freight distribution at nights and likes arguing' WHERE id = 556ebd54-cbe5-4b75-9aae-bf2a31a24500;
+cqlsh:demo> SELECT * FROM sasi;
+
+ id                                   | age | bio                                                                              | created_at    | first_name | height | last_name
+--------------------------------------+-----+----------------------------------------------------------------------------------+---------------+------------+--------+---------------
+ f5dfcabe-de96-4148-9b80-a1c41ed276b4 |  26 |                                                                             null | 1442959315021 |    Michael |    180 |      Kjellman
+ 96053844-45c3-4f15-b1b7-b02c441d3ee1 |  36 |                                                                             null | 1442959315020 |    Mikhail |    173 |       Stepura
+ 6b757016-631d-4fdb-ac62-40b127ccfbc7 |  40 |                                                                             null | 1442959315023 |      Jason |    182 |         Brown
+ 556ebd54-cbe5-4b75-9aae-bf2a31a24500 |  27 | Software Engineer, works on the freight distribution at nights and likes arguing | 1442959315018 |      Pavel |    181 |     Yaskevich
+ 8f909e8a-008e-49dd-8d43-1b0df348ed44 |  34 |                                                                             null | 1442959315024 |      Vijay |    183 | Parthasarathy
+ 5770382a-c56f-4f3f-b755-450e24d55217 |  26 |          Software Engineer, who likes distributed systems, doesnt like to argue. | 1442959315019 |     Jordan |    173 |          West
+ 2970da43-e070-41a8-8bcb-35df7a0e608a |  32 |                                                                             null | 1442959315022 |     Johnny |    175 |         Zhang
+
+(7 rows)
+```
+
+Index terms and query search strings are stemmed for the `bio` column
+because it was configured to use the
+[`StandardAnalyzer`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzer.java)
+and `analyzed` is set to `true`. The
+`tokenization_normalize_lowercase` is similar to the `case_sensitive`
+property but for the
+[`StandardAnalyzer`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzer.java). These
+query demonstrates the stemming applied by [`StandardAnalyzer`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzer.java).
+
+```
+cqlsh:demo> SELECT * FROM sasi WHERE bio = 'distributing';
+
+ id                                   | age | bio                                                                              | created_at    | first_name | height | last_name
+--------------------------------------+-----+----------------------------------------------------------------------------------+---------------+------------+--------+-----------
+ 556ebd54-cbe5-4b75-9aae-bf2a31a24500 |  27 | Software Engineer, works on the freight distribution at nights and likes arguing | 1442959315018 |      Pavel |    181 | Yaskevich
+ 5770382a-c56f-4f3f-b755-450e24d55217 |  26 |          Software Engineer, who likes distributed systems, doesnt like to argue. | 1442959315019 |     Jordan |    173 |      West
+
+(2 rows)
+
+cqlsh:demo> SELECT * FROM sasi WHERE bio = 'they argued';
+
+ id                                   | age | bio                                                                              | created_at    | first_name | height | last_name
+--------------------------------------+-----+----------------------------------------------------------------------------------+---------------+------------+--------+-----------
+ 556ebd54-cbe5-4b75-9aae-bf2a31a24500 |  27 | Software Engineer, works on the freight distribution at nights and likes arguing | 1442959315018 |      Pavel |    181 | Yaskevich
+ 5770382a-c56f-4f3f-b755-450e24d55217 |  26 |          Software Engineer, who likes distributed systems, doesnt like to argue. | 1442959315019 |     Jordan |    173 |      West
+
+(2 rows)
+
+cqlsh:demo> SELECT * FROM sasi WHERE bio = 'working at the company';
+
+ id                                   | age | bio                                                                              | created_at    | first_name | height | last_name
+--------------------------------------+-----+----------------------------------------------------------------------------------+---------------+------------+--------+-----------
+ 556ebd54-cbe5-4b75-9aae-bf2a31a24500 |  27 | Software Engineer, works on the freight distribution at nights and likes arguing | 1442959315018 |      Pavel |    181 | Yaskevich
+
+(1 rows)
+
+cqlsh:demo> SELECT * FROM sasi WHERE bio = 'soft eng';
+
+ id                                   | age | bio                                                                              | created_at    | first_name | height | last_name
+--------------------------------------+-----+----------------------------------------------------------------------------------+---------------+------------+--------+-----------
+ 556ebd54-cbe5-4b75-9aae-bf2a31a24500 |  27 | Software Engineer, works on the freight distribution at nights and likes arguing | 1442959315018 |      Pavel |    181 | Yaskevich
+ 5770382a-c56f-4f3f-b755-450e24d55217 |  26 |          Software Engineer, who likes distributed systems, doesnt like to argue. | 1442959315019 |     Jordan |    173 |      West
+
+(2 rows)
+```
+
+## Implementation Details
+
+While SASI, at the surface, is simply an implementation of the
+`Index` interface, at its core there are several data
+structures and algorithms used to satisfy it. These are described
+here. Additionally, the changes internal to Cassandra to support SASIs
+integration are described.
+
+The `Index` interface divides responsibility of the
+implementer into two parts: Indexing and Querying. Further, Cassandra
+makes it possible to divide those responsibilities into the memory and
+disk components. SASI takes advantage of Cassandra's write-once,
+immutable, ordered data model to build indexes along with the flushing
+of the memtable to disk -- this is the origin of the name "SSTable
+Attached Secondary Index".
+
+The SASI index data structures are built in memory as the SSTable is
+being written and they are flushed to disk before the writing of the
+SSTable completes. The writing of each index file only requires
+sequential writes to disk. In some cases, partial flushes are
+performed, and later stitched back together, to reduce memory
+usage. These data structures are optimized for this use case.
+
+Taking advantage of Cassandra's ordered data model, at query time,
+candidate indexes are narrowed down for searching minimize the amount
+of work done. Searching is then performed using an efficient method
+that streams data off disk as needed.
+
+### Indexing
+
+Per SSTable, SASI writes an index file for each indexed column. The
+data for these files is built in memory using the
+[`OnDiskIndexBuilder`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java). Once
+flushed to disk, the data is read using the
+[`OnDiskIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java)
+class. These are composed of bytes representing indexed terms,
+organized for efficient writing or searching respectively. The keys
+and values they hold represent tokens and positions in an SSTable and
+these are stored per-indexed term in
+[`TokenTreeBuilder`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/TokenTreeBuilder.java)s
+for writing, and
+[`TokenTree`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java)s
+for querying. These index files are memory mapped after being written
+to disk, for quicker access. For indexing data in the memtable SASI
+uses its
+[`IndexMemtable`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/memory/IndexMemtable.java)
+class.
+
+#### OnDiskIndex(Builder)
+
+Each
+[`OnDiskIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java)
+is an instance of a modified
+[Suffix Array](https://en.wikipedia.org/wiki/Suffix_array) data
+structure. The
+[`OnDiskIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java)
+is comprised of page-size blocks of sorted terms and pointers to the
+terms' associated data, as well as the data itself, stored also in one
+or more page-sized blocks. The
+[`OnDiskIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java)
+is structured as a tree of arrays, where each level describes the
+terms in the level below, the final level being the terms
+themselves. The `PointerLevel`s and their `PointerBlock`s contain
+terms and pointers to other blocks that *end* with those terms. The
+`DataLevel`, the final level, and its `DataBlock`s contain terms and
+point to the data itself, contained in [`TokenTree`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java)s.
+
+The terms written to the
+[`OnDiskIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java)
+vary depending on its "mode": either `PREFIX`, `CONTAINS`, or
+`SPARSE`. In the `PREFIX` and `SPARSE` cases terms exact values are
+written exactly once per `OnDiskIndex`. For example, a `PREFIX` index
+with terms `Jason`, `Jordan`, `Pavel`, all three will be included in
+the index. A `CONTAINS` index writes additional terms for each suffix of
+each term recursively. Continuing with the example, a `CONTAINS` index
+storing the previous terms would also store `ason`, `ordan`, `avel`,
+`son`, `rdan`, `vel`, etc. This allows for queries on the suffix of
+strings. The `SPARSE` mode differs from `PREFIX` in that for every 64
+blocks of terms a
+[`TokenTree`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java)
+is built merging all the
+[`TokenTree`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java)s
+for each term into a single one. This copy of the data is used for
+efficient iteration of large ranges of e.g. timestamps. The index
+"mode" is configurable per column at index creation time.
+
+#### TokenTree(Builder)
+
+The
+[`TokenTree`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java)
+is an implementation of the well-known
+[B+-tree](https://en.wikipedia.org/wiki/B%2B_tree) that has been
+modified to optimize for its use-case. In particular, it has been
+optimized to associate tokens, longs, with a set of positions in an
+SSTable, also longs. Allowing the set of long values accommodates
+the possibility of a hash collision in the token, but the data
+structure is optimized for the unlikely possibility of such a
+collision.
+
+To optimize for its write-once environment the
+[`TokenTreeBuilder`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/TokenTreeBuilder.java)
+completely loads its interior nodes as the tree is built and it uses
+the well-known algorithm optimized for bulk-loading the data
+structure.
+
+[`TokenTree`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java)s provide the means to iterate a tokens, and file
+positions, that match a given term, and to skip forward in that
+iteration, an operation used heavily at query time.
+
+#### IndexMemtable
+
+The
+[`IndexMemtable`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/memory/IndexMemtable.java)
+handles indexing the in-memory data held in the memtable. The
+[`IndexMemtable`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/memory/IndexMemtable.java)
+in turn manages either a
+[`TrieMemIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java)
+or a
+[`SkipListMemIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/memory/SkipListMemIndex.java)
+per-column. The choice of which index type is used is data
+dependent. The
+[`TrieMemIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java)
+is used for literal types. `AsciiType` and `UTF8Type` are literal
+types by defualt but any column can be configured as a literal type
+using the `is_literal` option at index creation time. For non-literal
+types the
+[`SkipListMemIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/memory/SkipListMemIndex.java)
+is used. The
+[`TrieMemIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java)
+is an implementation that can efficiently support prefix queries on
+character-like data. The
+[`SkipListMemIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/memory/SkipListMemIndex.java),
+conversely, is better suited for Cassandra other data types like
+numbers.
+
+The
+[`TrieMemIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java)
+is built using either the `ConcurrentRadixTree` or
+`ConcurrentSuffixTree` from the `com.goooglecode.concurrenttrees`
+package. The choice between the two is made based on the indexing
+mode, `PREFIX` or other modes, and `CONTAINS` mode, respectively.
+
+The
+[`SkipListMemIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/memory/SkipListMemIndex.java)
+is built on top of `java.util.concurrent.ConcurrentSkipListSet`.
+
+### Querying
+
+Responsible for converting the internal `IndexExpression`
+representation into SASI's
+[`Operation`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java)
+and
+[`Expression`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Expression.java)
+tree, optimizing the tree to reduce the amount of work done, and
+driving the query itself the
+[`QueryPlan`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java)
+is the work horse of SASI's querying implementation. To efficiently
+perform union and intersection operations SASI provides several
+iterators similar to Cassandra's `MergeIterator` but tailored
+specifically for SASIs use, and with more features. The
+[`RangeUnionIterator`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/utils/RangeUnionIterator.java),
+like its name suggests, performs set union over sets of tokens/keys
+matching the query, only reading as much data as it needs from each
+set to satisfy the query. The
+[`RangeIntersectionIterator`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/utils/RangeIntersectionIterator.java),
+similar to its counterpart, performs set intersection over its data.
+
+#### QueryPlan
+
+The
+[`QueryPlan`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java)
+instantiated per search query is at the core of SASIs querying
+implementation. Its work can be divided in two stages: analysis and
+execution.
+
+During the analysis phase,
+[`QueryPlan`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java)
+converts from Cassandra's internal representation of
+`IndexExpression`s, which has also been modified to support encoding
+queries that contain ORs and groupings of expressions using
+parentheses (see the
+[Cassandra Internal Changes](https://github.com/xedin/sasi#cassandra-internal-changes)
+section below for more details). This process produces a tree of
+[`Operation`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java)s, which in turn may contain [`Expression`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Expression.java)s, all of which
+provide an alternative, more efficient, representation of the query.
+
+During execution the
+[`QueryPlan`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java)
+uses the `DecoratedKey`-generating iterator created from the
+[`Operation`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java) tree. These keys are read from disk and a final check to
+ensure they satisfy the query is made, once again using the
+[`Operation`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java) tree. At the point the desired amount of matching data has
+been found, or there is no more matching data, the result set is
+returned to the coordinator through the existing internal components.
+
+The number of queries (total/failed/timed-out), and their latencies,
+are maintined per-table/column family.
+
+SASI also supports concurrently iterating terms for the same index
+accross SSTables. The concurrency factor is controlled by the
+`cassandra.search_concurrency_factor` system property. The default is
+`1`.
+
+##### QueryController
+
+Each
+[`QueryPlan`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java)
+references a
+[`QueryController`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java)
+used throughout the execution phase. The
+[`QueryController`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java)
+has two responsibilities: to manage and ensure the proper cleanup of
+resources (indexes), and to strictly enforce the time bound for query,
+specified by the user via the range slice timeout. All indexes are
+accessed via the
+[`QueryController`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java)
+so that they can be safely released by it later. The
+[`QueryController`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java)'s
+`checkpoint` function is called in specific places in the execution
+path to ensure the time-bound is enforced.
+
+##### QueryPlan Optimizations
+
+While in the analysis phase, the
+[`QueryPlan`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java)
+performs several potential optimizations to the query. The goal of
+these optimizations is to reduce the amount of work performed during
+the execution phase.
+
+The simplest optimization performed is compacting multiple expressions
+joined by logical intersection (`AND`) into a single [`Operation`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java) with
+three or more [`Expression`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Expression.java)s. For example, the query `WHERE age < 100 AND
+fname = 'p*' AND first_name != 'pa*' AND age > 21` would,
+without modification, have the following tree:
+
+                          ┌───────┐
+                 ┌────────│  AND  │──────┐
+                 │        └───────┘      │
+                 ▼                       ▼
+              ┌───────┐             ┌──────────┐
+        ┌─────│  AND  │─────┐       │age < 100 │
+        │     └───────┘     │       └──────────┘
+        ▼                   ▼
+    ┌──────────┐          ┌───────┐
+    │ fname=p* │        ┌─│  AND  │───┐
+    └──────────┘        │ └───────┘   │
+                        ▼             ▼
+                    ┌──────────┐  ┌──────────┐
+                    │fname!=pa*│  │ age > 21 │
+                    └──────────┘  └──────────┘
+
+[`QueryPlan`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java)
+will remove the redundant right branch whose root is the final `AND`
+and has leaves `fname != pa*` and `age > 21`. These [`Expression`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Expression.java)s will
+be compacted into the parent `AND`, a safe operation due to `AND`
+being associative and commutative. The resulting tree looks like the
+following:
+
+                                  ┌───────┐
+                         ┌────────│  AND  │──────┐
+                         │        └───────┘      │
+                         ▼                       ▼
+                      ┌───────┐             ┌──────────┐
+          ┌───────────│  AND  │────────┐    │age < 100 │
+          │           └───────┘        │    └──────────┘
+          ▼               │            ▼
+    ┌──────────┐          │      ┌──────────┐
+    │ fname=p* │          ▼      │ age > 21 │
+    └──────────┘    ┌──────────┐ └──────────┘
+                    │fname!=pa*│
+                    └──────────┘
+
+When excluding results from the result set, using `!=`, the
+[`QueryPlan`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java)
+determines the best method for handling it. For range queries, for
+example, it may be optimal to divide the range into multiple parts
+with a hole for the exclusion. For string queries, such as this one,
+it is more optimal, however, to simply note which data to skip, or
+exclude, while scanning the index. Following this optimization the
+tree looks like this:
+
+                                   ┌───────┐
+                          ┌────────│  AND  │──────┐
+                          │        └───────┘      │
+                          ▼                       ▼
+                       ┌───────┐             ┌──────────┐
+               ┌───────│  AND  │────────┐    │age < 100 │
+               │       └───────┘        │    └──────────┘
+               ▼                        ▼
+        ┌──────────────────┐         ┌──────────┐
+        │     fname=p*     │         │ age > 21 │
+        │ exclusions=[pa*] │         └──────────┘
+        └──────────────────┘
+
+The last type of optimization applied, for this query, is to merge
+range expressions across branches of the tree -- without modifying the
+meaning of the query, of course. In this case, because the query
+contains all `AND`s the `age` expressions can be collapsed. Along with
+this optimization, the initial collapsing of unneeded `AND`s can also
+be applied once more to result in this final tree using to execute the
+query:
+
+                            ┌───────┐
+                     ┌──────│  AND  │───────┐
+                     │      └───────┘       │
+                     ▼                      ▼
+           ┌──────────────────┐    ┌────────────────┐
+           │     fname=p*     │    │ 21 < age < 100 │
+           │ exclusions=[pa*] │    └────────────────┘
+           └──────────────────┘
+
+#### Operations and Expressions
+
+As discussed, the
+[`QueryPlan`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java)
+optimizes a tree represented by
+[`Operation`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java)s
+as interior nodes, and
+[`Expression`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Expression.java)s
+as leaves. The
+[`Operation`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java)
+class, more specifically, can have zero, one, or two
+[`Operation`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java)s
+as children and an unlimited number of expressions. The iterators used
+to perform the queries, discussed below in the
+"Range(Union|Intersection)Iterator" section, implement the necessary
+logic to merge results transparently regardless of the
+[`Operation`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java)s
+children.
+
+Besides participating in the optimizations performed by the
+[`QueryPlan`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java),
+[`Operation`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java)
+is also responsible for taking a row that has been returned by the
+query and making a final validation that it in fact does match. This
+`satisfiesBy` operation is performed recursively from the root of the
+[`Operation`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java)
+tree for a given query. These checks are performed directly on the
+data in a given row. For more details on how `satisfiesBy` works see
+the documentation
+[in the code](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/Operation.java#L87-L123).
+
+#### Range(Union|Intersection)Iterator
+
+The abstract `RangeIterator` class provides a unified interface over
+the two main operations performed by SASI at various layers in the
+execution path: set intersection and union. These operations are
+performed in a iterated, or "streaming", fashion to prevent unneeded
+reads of elements from either set. In both the intersection and union
+cases the algorithms take advantage of the data being pre-sorted using
+the same sort order, e.g. term or token order.
+
+The
+[`RangeUnionIterator`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/utils/RangeUnionIterator.java)
+performs the "Merge-Join" portion of the
+[Sort-Merge-Join](https://en.wikipedia.org/wiki/Sort-merge_join)
+algorithm, with the properties of an outer-join, or union. It is
+implemented with several optimizations to improve its performance over
+a large number of iterators -- sets to union. Specifically, the
+iterator exploits the likely case of the data having many sub-groups
+of overlapping ranges and the unlikely case that all ranges will
+overlap each other. For more details see the
+[javadoc](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/utils/RangeUnionIterator.java#L9-L21).
+
+The
+[`RangeIntersectionIterator`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/utils/RangeIntersectionIterator.java)
+itself is not a subclass of `RangeIterator`. It is a container for
+several classes, one of which, `AbstractIntersectionIterator`,
+sub-classes `RangeIterator`. SASI supports two methods of performing
+the intersection operation, and the ability to be adaptive in choosing
+between them based on some properties of the data.
+
+`BounceIntersectionIterator`, and the `BOUNCE` strategy, works like
+the
+[`RangeUnionIterator`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/utils/RangeUnionIterator.java)
+in that it performs a "Merge-Join", however, its nature is similar to
+a inner-join, where like values are merged by a data-specific merge
+function (e.g. merging two tokens in a list to lookup in a SSTable
+later). See the
+[javadoc](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/utils/RangeIntersectionIterator.java#L88-L101)
+for more details on its implementation.
+
+`LookupIntersectionIterator`, and the `LOOKUP` strategy, performs a
+different operation, more similar to a lookup in an associative data
+structure, or "hash lookup" in database terminology. Once again,
+details on the implementation can be found in the
+[javadoc](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/utils/RangeIntersectionIterator.java#L199-L208).
+
+The choice between the two iterators, or the `ADAPTIVE` strategy, is
+based upon the ratio of data set sizes of the minimum and maximum
+range of the sets being intersected. If the number of the elements in
+minimum range divided by the number of elements is the maximum range
+is less than or equal to `0.01`, then the `ADAPTIVE` strategy chooses
+the `LookupIntersectionIterator`, otherwise the
+`BounceIntersectionIterator` is chosen.
+
+### The SASIIndex Class
+
+The above components are glued together by the
+[`SASIIndex`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasiIndex.java)
+class which implements `Index`, and is instantiated
+per-table containing SASI indexes. It manages all indexes for a table
+via the
+[`sasi.conf.DataTracker`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/conf/DataTracker.java)
+and
+[`sasi.conf.view.View`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/conf/view/View.java)
+components, controls writing of all indexes for an SSTable via its
+[`PerSSTableIndexWriter`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java), and initiates searches with
+`Indexer`. These classes glue the previously
+mentioned indexing components together with Cassandra's SSTable
+life-cycle ensuring indexes are not only written when Memtable's flush
+but also as SSTable's are compacted. For querying, the
+`Indexer` does little but defer to
+[`QueryPlan`](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java)
+and update e.g. latency metrics exposed by SASI.
+
+### Cassandra Internal Changes
+
+To support the above changes and integrate them into Cassandra a few
+minor internal changes were made to Cassandra itself. These are
+described here.
+
+#### SSTable Write Life-cycle Notifications
+
+The `SSTableFlushObserver` is an observer pattern-like interface,
+whose sub-classes can register to be notified about events in the
+life-cycle of writing out a SSTable. Sub-classes can be notified when a
+flush begins and ends, as well as when each next row is about to be
+written, and each next column. SASI's `PerSSTableIndexWriter`,
+discussed above, is the only current subclass.
+
+### Limitations and Caveats
+
+The following are items that can be addressed in future updates but are not
+available in this repository or are not currently implemented.
+
+* The cluster must be configured to use a partitioner that produces
+  `LongToken`s, e.g. `Murmur3Partitioner`. Other existing partitioners which
+  don't produce LongToken e.g. `ByteOrderedPartitioner` and `RandomPartitioner`
+  will not work with SASI.
+* `ALLOW FILTERING`, the requirement of at least one indexes `=`
+  expression, and lack of `LIKE` limit SASIs
+  feature-set. Modifications to the grammar to allow `Index`
+  implementations to enumerate its supported features would allow SASI
+  to expose more features without need to support them in other
+  implementations.
+* Not Equals and OR support have been removed in this release while
+  changes are made to Cassandra itself to support them.
+
+### Contributors
+
+* [Pavel Yaskevich](https://github.com/xedin)
+* [Jordan West](https://github.com/jrwest)
+* [Michael Kjellman](https://github.com/mkjellman)
+* [Jason Brown](https://github.com/jasobrown)
+* [Mikhail Stepura](https://github.com/mishail)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/lib/concurrent-trees-2.4.0.jar
----------------------------------------------------------------------
diff --git a/lib/concurrent-trees-2.4.0.jar b/lib/concurrent-trees-2.4.0.jar
new file mode 100644
index 0000000..9c488fe
Binary files /dev/null and b/lib/concurrent-trees-2.4.0.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/lib/hppc-0.5.4.jar
----------------------------------------------------------------------
diff --git a/lib/hppc-0.5.4.jar b/lib/hppc-0.5.4.jar
new file mode 100644
index 0000000..d84b83b
Binary files /dev/null and b/lib/hppc-0.5.4.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/lib/jflex-1.6.0.jar
----------------------------------------------------------------------
diff --git a/lib/jflex-1.6.0.jar b/lib/jflex-1.6.0.jar
new file mode 100644
index 0000000..550e446
Binary files /dev/null and b/lib/jflex-1.6.0.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/lib/licenses/concurrent-trees-2.4.0.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/concurrent-trees-2.4.0.txt b/lib/licenses/concurrent-trees-2.4.0.txt
new file mode 100644
index 0000000..50086f8
--- /dev/null
+++ b/lib/licenses/concurrent-trees-2.4.0.txt
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!) The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   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.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/lib/licenses/hppc-0.5.4.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/hppc-0.5.4.txt b/lib/licenses/hppc-0.5.4.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/lib/licenses/hppc-0.5.4.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   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.


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

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
new file mode 100644
index 0000000..cb5ec73
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
@@ -0,0 +1,1852 @@
+/*
+ * 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.index.sasi;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.filter.RowFilter;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
+import org.apache.cassandra.index.sasi.exceptions.TimeQuotaExceededException;
+import org.apache.cassandra.index.sasi.plan.QueryPlan;
+import org.apache.cassandra.schema.IndexMetadata;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Tables;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.thrift.CqlRow;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.Uninterruptibles;
+
+import junit.framework.Assert;
+
+import org.junit.*;
+
+public class SASIIndexTest
+{
+    private static final IPartitioner PARTITIONER = new Murmur3Partitioner();
+
+    private static final String KS_NAME = "sasi";
+    private static final String CF_NAME = "test_cf";
+    private static final String CLUSTRING_CF_NAME = "clustering_test_cf";
+
+    @BeforeClass
+    public static void loadSchema() throws ConfigurationException
+    {
+        System.setProperty("cassandra.config", "cassandra-murmur.yaml");
+        SchemaLoader.loadSchema();
+        MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(KS_NAME,
+                                                                     KeyspaceParams.simpleTransient(1),
+                                                                     Tables.of(SchemaLoader.sasiCFMD(KS_NAME, CF_NAME),
+                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTRING_CF_NAME))));
+    }
+
+    @After
+    public void cleanUp()
+    {
+        Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).truncateBlocking();
+    }
+
+    @Test
+    public void testSingleExpressionQueries() throws Exception
+    {
+        testSingleExpressionQueries(false);
+        cleanupData();
+        testSingleExpressionQueries(true);
+    }
+
+    private void testSingleExpressionQueries(boolean forceFlush) throws Exception
+    {
+        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
+        {{
+            put("key1", Pair.create("Pavel", 14));
+            put("key2", Pair.create("Pavel", 26));
+            put("key3", Pair.create("Pavel", 27));
+            put("key4", Pair.create("Jason", 27));
+        }};
+
+        ColumnFamilyStore store = loadData(data, forceFlush);
+
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+        final ByteBuffer age = UTF8Type.instance.decompose("age");
+
+        Set<String> rows;
+
+        rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("av")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("as")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("aw")));
+        Assert.assertEquals(rows.toString(), 0, rows.size());
+
+        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(27)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key3", "key4"}, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(13)));
+        Assert.assertEquals(rows.toString(), 0, rows.size());
+    }
+
+    @Test
+    public void testEmptyTokenizedResults() throws Exception
+    {
+        testEmptyTokenizedResults(false);
+        cleanupData();
+        testEmptyTokenizedResults(true);
+    }
+
+    private void testEmptyTokenizedResults(boolean forceFlush) throws Exception
+    {
+        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key1", Pair.create("  ", 14));
+        }};
+
+        ColumnFamilyStore store = loadData(data, forceFlush);
+
+        Set<String> rows= getIndexed(store, 10, buildExpression(UTF8Type.instance.decompose("first_name"), Operator.EQ, UTF8Type.instance.decompose("doesntmatter")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{}, rows.toArray(new String[rows.size()])));
+    }
+
+    @Test
+    public void testMultiExpressionQueries() throws Exception
+    {
+        testMultiExpressionQueries(false);
+        cleanupData();
+        testMultiExpressionQueries(true);
+    }
+
+    public void testMultiExpressionQueries(boolean forceFlush) throws Exception
+    {
+        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key1", Pair.create("Pavel", 14));
+                put("key2", Pair.create("Pavel", 26));
+                put("key3", Pair.create("Pavel", 27));
+                put("key4", Pair.create("Jason", 27));
+        }};
+
+        ColumnFamilyStore store = loadData(data, forceFlush);
+
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+        final ByteBuffer age = UTF8Type.instance.decompose("age");
+
+        Set<String> rows;
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(14)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(27)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key1", "key2"}, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                         buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                         buildExpression(age, Operator.GT, Int32Type.instance.decompose(14)),
+                         buildExpression(age, Operator.LT, Int32Type.instance.decompose(27)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                         buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                         buildExpression(age, Operator.GT, Int32Type.instance.decompose(12)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                         buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                         buildExpression(age, Operator.GTE, Int32Type.instance.decompose(13)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                         buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                         buildExpression(age, Operator.GTE, Int32Type.instance.decompose(16)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+
+
+        rows = getIndexed(store, 10,
+                         buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                         buildExpression(age, Operator.LT, Int32Type.instance.decompose(30)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                         buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                         buildExpression(age, Operator.LTE, Int32Type.instance.decompose(29)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                         buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                         buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1" }, rows.toArray(new String[rows.size()])));
+    }
+
+    @Test
+    public void testCrossSSTableQueries() throws Exception
+    {
+        testCrossSSTableQueries(false);
+        cleanupData();
+        testCrossSSTableQueries(true);
+
+    }
+
+    private void testCrossSSTableQueries(boolean forceFlush) throws Exception
+    {
+        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key0", Pair.create("Maxie", 43));
+                put("key1", Pair.create("Chelsie", 33));
+                put("key2", Pair.create("Josephine", 43));
+                put("key3", Pair.create("Shanna", 27));
+                put("key4", Pair.create("Amiya", 36));
+            }};
+
+        loadData(part1, forceFlush); // first sstable
+
+        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key5", Pair.create("Americo", 20));
+                put("key6", Pair.create("Fiona", 39));
+                put("key7", Pair.create("Francis", 41));
+                put("key8", Pair.create("Charley", 21));
+                put("key9", Pair.create("Amely", 40));
+            }};
+
+        loadData(part2, forceFlush);
+
+        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key10", Pair.create("Eddie", 42));
+                put("key11", Pair.create("Oswaldo", 35));
+                put("key12", Pair.create("Susana", 35));
+                put("key13", Pair.create("Alivia", 42));
+                put("key14", Pair.create("Demario", 28));
+            }};
+
+        ColumnFamilyStore store = loadData(part3, forceFlush);
+
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+        final ByteBuffer age = UTF8Type.instance.decompose("age");
+
+        Set<String> rows;
+        rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Fiona")),
+                                     buildExpression(age, Operator.LT, Int32Type.instance.decompose(40)));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key6" }, rows.toArray(new String[rows.size()])));
+
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key14",
+                                                                        "key3", "key4", "key6", "key7", "key8" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 5,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+
+        Assert.assertEquals(rows.toString(), 5, rows.size());
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.GTE, Int32Type.instance.decompose(35)));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key4", "key6", "key7" },
+                                                         rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key3", "key8" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(27)),
+                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
+
+        Assert.assertEquals(rows.toString(), 10, rows.size());
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(50)));
+
+        Assert.assertEquals(rows.toString(), 10, rows.size());
+    }
+
+    @Test
+    public void testQueriesThatShouldBeTokenized() throws Exception
+    {
+        testQueriesThatShouldBeTokenized(false);
+        cleanupData();
+        testQueriesThatShouldBeTokenized(true);
+    }
+
+    private void testQueriesThatShouldBeTokenized(boolean forceFlush) throws Exception
+    {
+        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key0", Pair.create("If you can dream it, you can do it.", 43));
+                put("key1", Pair.create("What you get by achieving your goals is not " +
+                        "as important as what you become by achieving your goals, do it.", 33));
+                put("key2", Pair.create("Keep your face always toward the sunshine " +
+                        "- and shadows will fall behind you.", 43));
+                put("key3", Pair.create("We can't help everyone, but everyone can " +
+                        "help someone.", 27));
+            }};
+
+        ColumnFamilyStore store = loadData(part1, forceFlush);
+
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+        final ByteBuffer age = UTF8Type.instance.decompose("age");
+
+        Set<String> rows = getIndexed(store, 10,
+                buildExpression(firstName, Operator.EQ,
+                        UTF8Type.instance.decompose("What you get by achieving your goals")),
+                buildExpression(age, Operator.GT, Int32Type.instance.decompose(32)));
+
+        Assert.assertEquals(rows.toString(), Collections.singleton("key1"), rows);
+
+        rows = getIndexed(store, 10,
+                buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("do it.")));
+
+        Assert.assertEquals(rows.toString(), Arrays.asList("key0", "key1"), Lists.newArrayList(rows));
+    }
+
+    @Test
+    public void testMultiExpressionQueriesWhereRowSplitBetweenSSTables() throws Exception
+    {
+        testMultiExpressionQueriesWhereRowSplitBetweenSSTables(false);
+        cleanupData();
+        testMultiExpressionQueriesWhereRowSplitBetweenSSTables(true);
+    }
+
+    private void testMultiExpressionQueriesWhereRowSplitBetweenSSTables(boolean forceFlush) throws Exception
+    {
+        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key0", Pair.create("Maxie", -1));
+                put("key1", Pair.create("Chelsie", 33));
+                put("key2", Pair.create((String)null, 43));
+                put("key3", Pair.create("Shanna", 27));
+                put("key4", Pair.create("Amiya", 36));
+        }};
+
+        loadData(part1, forceFlush); // first sstable
+
+        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key5", Pair.create("Americo", 20));
+                put("key6", Pair.create("Fiona", 39));
+                put("key7", Pair.create("Francis", 41));
+                put("key8", Pair.create("Charley", 21));
+                put("key9", Pair.create("Amely", 40));
+                put("key14", Pair.create((String)null, 28));
+        }};
+
+        loadData(part2, forceFlush);
+
+        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key0", Pair.create((String)null, 43));
+                put("key10", Pair.create("Eddie", 42));
+                put("key11", Pair.create("Oswaldo", 35));
+                put("key12", Pair.create("Susana", 35));
+                put("key13", Pair.create("Alivia", 42));
+                put("key14", Pair.create("Demario", -1));
+                put("key2", Pair.create("Josephine", -1));
+        }};
+
+        ColumnFamilyStore store = loadData(part3, forceFlush);
+
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+        final ByteBuffer age = UTF8Type.instance.decompose("age");
+
+        Set<String> rows = getIndexed(store, 10,
+                                      buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Fiona")),
+                                      buildExpression(age, Operator.LT, Int32Type.instance.decompose(40)));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key6" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key14",
+                                                                        "key3", "key4", "key6", "key7", "key8" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 5,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+
+        Assert.assertEquals(rows.toString(), 5, rows.size());
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.GTE, Int32Type.instance.decompose(35)));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key4", "key6", "key7" },
+                                                         rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key3", "key8" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(27)),
+                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
+
+        Map<String, Pair<String, Integer>> part4 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key12", Pair.create((String)null, 12));
+                put("key14", Pair.create("Demario", 42));
+                put("key2", Pair.create("Frank", -1));
+        }};
+
+        store = loadData(part4, forceFlush);
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Susana")),
+                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(13)),
+                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key12" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Demario")),
+                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(30)));
+        Assert.assertTrue(rows.toString(), rows.size() == 0);
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Josephine")));
+        Assert.assertTrue(rows.toString(), rows.size() == 0);
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
+
+        Assert.assertEquals(rows.toString(), 10, rows.size());
+
+        rows = getIndexed(store, 10,
+                          buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(50)));
+
+        Assert.assertEquals(rows.toString(), 10, rows.size());
+    }
+
+    @Test
+    public void testPagination() throws Exception
+    {
+        testPagination(false);
+        cleanupData();
+        testPagination(true);
+    }
+
+    private void testPagination(boolean forceFlush) throws Exception
+    {
+        // split data into 3 distinct SSTables to test paging with overlapping token intervals.
+
+        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key01", Pair.create("Ali", 33));
+                put("key02", Pair.create("Jeremy", 41));
+                put("key03", Pair.create("Elvera", 22));
+                put("key04", Pair.create("Bailey", 45));
+                put("key05", Pair.create("Emerson", 32));
+                put("key06", Pair.create("Kadin", 38));
+                put("key07", Pair.create("Maggie", 36));
+                put("key08", Pair.create("Kailey", 36));
+                put("key09", Pair.create("Armand", 21));
+                put("key10", Pair.create("Arnold", 35));
+        }};
+
+        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key11", Pair.create("Ken", 38));
+                put("key12", Pair.create("Penelope", 43));
+                put("key13", Pair.create("Wyatt", 34));
+                put("key14", Pair.create("Johnpaul", 34));
+                put("key15", Pair.create("Trycia", 43));
+                put("key16", Pair.create("Aida", 21));
+                put("key17", Pair.create("Devon", 42));
+        }};
+
+        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key18", Pair.create("Christina", 20));
+                put("key19", Pair.create("Rick", 19));
+                put("key20", Pair.create("Fannie", 22));
+                put("key21", Pair.create("Keegan", 29));
+                put("key22", Pair.create("Ignatius", 36));
+                put("key23", Pair.create("Ellis", 26));
+                put("key24", Pair.create("Annamarie", 29));
+                put("key25", Pair.create("Tianna", 31));
+                put("key26", Pair.create("Dennis", 32));
+        }};
+
+        ColumnFamilyStore store = loadData(part1, forceFlush);
+
+        loadData(part2, forceFlush);
+        loadData(part3, forceFlush);
+
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+        final ByteBuffer age = UTF8Type.instance.decompose("age");
+
+        Set<DecoratedKey> uniqueKeys = getPaged(store, 4,
+                buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                buildExpression(age, Operator.GTE, Int32Type.instance.decompose(21)));
+
+
+        List<String> expected = new ArrayList<String>()
+        {{
+                add("key25");
+                add("key20");
+                add("key13");
+                add("key22");
+                add("key09");
+                add("key14");
+                add("key16");
+                add("key24");
+                add("key03");
+                add("key04");
+                add("key08");
+                add("key07");
+                add("key15");
+                add("key06");
+                add("key21");
+        }};
+
+        Assert.assertEquals(expected, convert(uniqueKeys));
+
+        // now let's test a single equals condition
+
+        uniqueKeys = getPaged(store, 4, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+
+        expected = new ArrayList<String>()
+        {{
+                add("key25");
+                add("key20");
+                add("key13");
+                add("key22");
+                add("key09");
+                add("key14");
+                add("key16");
+                add("key24");
+                add("key03");
+                add("key04");
+                add("key18");
+                add("key08");
+                add("key07");
+                add("key15");
+                add("key06");
+                add("key21");
+        }};
+
+        Assert.assertEquals(expected, convert(uniqueKeys));
+
+        // now let's test something which is smaller than a single page
+        uniqueKeys = getPaged(store, 4,
+                              buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                              buildExpression(age, Operator.EQ, Int32Type.instance.decompose(36)));
+
+        expected = new ArrayList<String>()
+        {{
+                add("key22");
+                add("key08");
+                add("key07");
+        }};
+
+        Assert.assertEquals(expected, convert(uniqueKeys));
+
+        // the same but with the page size of 2 to test minimal pagination windows
+
+        uniqueKeys = getPaged(store, 2,
+                              buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                              buildExpression(age, Operator.EQ, Int32Type.instance.decompose(36)));
+
+        Assert.assertEquals(expected, convert(uniqueKeys));
+
+        // and last but not least, test age range query with pagination
+        uniqueKeys = getPaged(store, 4,
+                buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                buildExpression(age, Operator.GT, Int32Type.instance.decompose(20)),
+                buildExpression(age, Operator.LTE, Int32Type.instance.decompose(36)));
+
+        expected = new ArrayList<String>()
+        {{
+                add("key25");
+                add("key20");
+                add("key13");
+                add("key22");
+                add("key09");
+                add("key14");
+                add("key16");
+                add("key24");
+                add("key03");
+                add("key08");
+                add("key07");
+                add("key21");
+        }};
+
+        Assert.assertEquals(expected, convert(uniqueKeys));
+
+        Set<String> rows;
+
+        rows = executeCQL(String.format("SELECT * FROM %s.%s WHERE first_name = 'a' limit 10 ALLOW FILTERING;", KS_NAME, CF_NAME));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key03", "key04", "key09", "key13", "key14", "key16", "key20", "key22", "key24", "key25" }, rows.toArray(new String[rows.size()])));
+
+        rows = executeCQL(String.format("SELECT * FROM %s.%s WHERE first_name = 'a' and token(id) >= token('key14') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key03", "key04", "key14", "key16", "key24" }, rows.toArray(new String[rows.size()])));
+
+        rows = executeCQL(String.format("SELECT * FROM %s.%s WHERE first_name = 'a' and token(id) >= token('key14') and token(id) <= token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key16", "key24" }, rows.toArray(new String[rows.size()])));
+
+        rows = executeCQL(String.format("SELECT * FROM %s.%s WHERE first_name = 'a' and age > 30 and token(id) >= token('key14') and token(id) <= token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
+    }
+
+    @Test
+    public void testColumnNamesWithSlashes() throws Exception
+    {
+        testColumnNamesWithSlashes(false);
+        cleanupData();
+        testColumnNamesWithSlashes(true);
+    }
+
+    private void testColumnNamesWithSlashes(boolean forceFlush) throws Exception
+    {
+        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+
+        Mutation rm1 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key1")));
+        rm1.add(PartitionUpdate.singleRowUpdate(store.metadata,
+                                                rm1.key(),
+                                                buildRow(buildCell(store.metadata,
+                                                                   UTF8Type.instance.decompose("/data/output/id"),
+                                                                   AsciiType.instance.decompose("jason"),
+                                                                   System.currentTimeMillis()))));
+
+        Mutation rm2 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key2")));
+        rm2.add(PartitionUpdate.singleRowUpdate(store.metadata,
+                                                rm2.key(),
+                                                buildRow(buildCell(store.metadata,
+                                                                   UTF8Type.instance.decompose("/data/output/id"),
+                                                                   AsciiType.instance.decompose("pavel"),
+                                                                   System.currentTimeMillis()))));
+
+        Mutation rm3 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key3")));
+        rm3.add(PartitionUpdate.singleRowUpdate(store.metadata,
+                                                rm3.key(),
+                                                buildRow(buildCell(store.metadata,
+                                                                   UTF8Type.instance.decompose("/data/output/id"),
+                                                                   AsciiType.instance.decompose("Aleksey"),
+                                                                   System.currentTimeMillis()))));
+
+        rm1.apply();
+        rm2.apply();
+        rm3.apply();
+
+        if (forceFlush)
+            store.forceBlockingFlush();
+
+        final ByteBuffer dataOutputId = UTF8Type.instance.decompose("/data/output/id");
+
+        Set<String> rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.EQ, UTF8Type.instance.decompose("A")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key3" }, rows.toArray(new String[rows.size()])));
+
+        // doesn't really make sense to rebuild index for in-memory data
+        if (!forceFlush)
+            return;
+
+        store.indexManager.invalidateAllIndexesBlocking();
+
+        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertTrue(rows.toString(), rows.isEmpty());
+
+        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.EQ, UTF8Type.instance.decompose("A")));
+        Assert.assertTrue(rows.toString(), rows.isEmpty());
+
+        // now let's trigger index rebuild and check if we got the data back
+        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
+
+        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
+
+        // also let's try to build an index for column which has no data to make sure that doesn't fail
+        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("first_name"));
+        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
+
+        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
+    }
+
+    @Test
+    public void testInvalidate() throws Exception
+    {
+        testInvalidate(false);
+        cleanupData();
+        testInvalidate(true);
+    }
+
+    private void testInvalidate(boolean forceFlush) throws Exception
+    {
+        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key0", Pair.create("Maxie", -1));
+                put("key1", Pair.create("Chelsie", 33));
+                put("key2", Pair.create((String) null, 43));
+                put("key3", Pair.create("Shanna", 27));
+                put("key4", Pair.create("Amiya", 36));
+        }};
+
+        ColumnFamilyStore store = loadData(part1, forceFlush);
+
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+        final ByteBuffer age = UTF8Type.instance.decompose("age");
+
+        Set<String> rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key0", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(33)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1" }, rows.toArray(new String[rows.size()])));
+
+        store.indexManager.invalidateAllIndexesBlocking();
+
+        rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertTrue(rows.toString(), rows.isEmpty());
+
+        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(33)));
+        Assert.assertTrue(rows.toString(), rows.isEmpty());
+
+
+        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key5", Pair.create("Americo", 20));
+                put("key6", Pair.create("Fiona", 39));
+                put("key7", Pair.create("Francis", 41));
+                put("key8", Pair.create("Fred", 21));
+                put("key9", Pair.create("Amely", 40));
+                put("key14", Pair.create("Dino", 28));
+        }};
+
+        loadData(part2, forceFlush);
+
+        rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key6", "key7" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(40)));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key9" }, rows.toArray(new String[rows.size()])));
+    }
+
+    @Test
+    public void testTruncate()
+    {
+        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key01", Pair.create("Ali", 33));
+                put("key02", Pair.create("Jeremy", 41));
+                put("key03", Pair.create("Elvera", 22));
+                put("key04", Pair.create("Bailey", 45));
+                put("key05", Pair.create("Emerson", 32));
+                put("key06", Pair.create("Kadin", 38));
+                put("key07", Pair.create("Maggie", 36));
+                put("key08", Pair.create("Kailey", 36));
+                put("key09", Pair.create("Armand", 21));
+                put("key10", Pair.create("Arnold", 35));
+        }};
+
+        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key11", Pair.create("Ken", 38));
+                put("key12", Pair.create("Penelope", 43));
+                put("key13", Pair.create("Wyatt", 34));
+                put("key14", Pair.create("Johnpaul", 34));
+                put("key15", Pair.create("Trycia", 43));
+                put("key16", Pair.create("Aida", 21));
+                put("key17", Pair.create("Devon", 42));
+        }};
+
+        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key18", Pair.create("Christina", 20));
+                put("key19", Pair.create("Rick", 19));
+                put("key20", Pair.create("Fannie", 22));
+                put("key21", Pair.create("Keegan", 29));
+                put("key22", Pair.create("Ignatius", 36));
+                put("key23", Pair.create("Ellis", 26));
+                put("key24", Pair.create("Annamarie", 29));
+                put("key25", Pair.create("Tianna", 31));
+                put("key26", Pair.create("Dennis", 32));
+        }};
+
+        ColumnFamilyStore store = loadData(part1, 1000, true);
+
+        loadData(part2, 2000, true);
+        loadData(part3, 3000, true);
+
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+
+        Set<String> rows = getIndexed(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertEquals(rows.toString(), 16, rows.size());
+
+        // make sure we don't prematurely delete anything
+        store.indexManager.truncateAllIndexesBlocking(500);
+
+        rows = getIndexed(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertEquals(rows.toString(), 16, rows.size());
+
+        store.indexManager.truncateAllIndexesBlocking(1500);
+
+        rows = getIndexed(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertEquals(rows.toString(), 10, rows.size());
+
+        store.indexManager.truncateAllIndexesBlocking(2500);
+
+        rows = getIndexed(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertEquals(rows.toString(), 6, rows.size());
+
+        store.indexManager.truncateAllIndexesBlocking(3500);
+
+        rows = getIndexed(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertEquals(rows.toString(), 0, rows.size());
+
+        // add back in some data just to make sure it all still works
+        Map<String, Pair<String, Integer>> part4 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key40", Pair.create("Tianna", 31));
+                put("key41", Pair.create("Dennis", 32));
+        }};
+
+        loadData(part4, 4000, true);
+
+        rows = getIndexed(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertEquals(rows.toString(), 1, rows.size());
+    }
+
+
+    @Test
+    public void testConcurrentMemtableReadsAndWrites() throws Exception
+    {
+        final ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+
+        ExecutorService scheduler = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
+
+        final int writeCount = 10000;
+        final AtomicInteger updates = new AtomicInteger(0);
+
+        for (int i = 0; i < writeCount; i++)
+        {
+            final String key = "key" + i;
+            final String firstName = "first_name#" + i;
+            final String lastName = "last_name#" + i;
+
+            scheduler.submit((Runnable) () -> {
+                try
+                {
+                    newMutation(key, firstName, lastName, 26, System.currentTimeMillis()).apply();
+                    Uninterruptibles.sleepUninterruptibly(5, TimeUnit.MILLISECONDS); // back up a bit to do more reads
+                }
+                finally
+                {
+                    updates.incrementAndGet();
+                }
+            });
+        }
+
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+        final ByteBuffer age = UTF8Type.instance.decompose("age");
+
+        int previousCount = 0;
+
+        do
+        {
+            // this loop figures out if number of search results monotonically increasing
+            // to make sure that concurrent updates don't interfere with reads, uses first_name and age
+            // indexes to test correctness of both Trie and SkipList ColumnIndex implementations.
+
+            Set<DecoratedKey> rows = getPaged(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                                                          buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
+
+            Assert.assertTrue(previousCount <= rows.size());
+            previousCount = rows.size();
+        }
+        while (updates.get() < writeCount);
+
+        // to make sure that after all of the right are done we can read all "count" worth of rows
+        Set<DecoratedKey> rows = getPaged(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
+
+        Assert.assertEquals(writeCount, rows.size());
+    }
+
+    @Test
+    public void testSameKeyInMemtableAndSSTables()
+    {
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+        final ByteBuffer age = UTF8Type.instance.decompose("age");
+
+        Map<String, Pair<String, Integer>> data1 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key1", Pair.create("Pavel", 14));
+                put("key2", Pair.create("Pavel", 26));
+                put("key3", Pair.create("Pavel", 27));
+                put("key4", Pair.create("Jason", 27));
+        }};
+
+        ColumnFamilyStore store = loadData(data1, true);
+
+        Map<String, Pair<String, Integer>> data2 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key1", Pair.create("Pavel", 14));
+                put("key2", Pair.create("Pavel", 27));
+                put("key4", Pair.create("Jason", 28));
+        }};
+
+        loadData(data2, true);
+
+        Map<String, Pair<String, Integer>> data3 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key1", Pair.create("Pavel", 15));
+                put("key4", Pair.create("Jason", 29));
+        }};
+
+        loadData(data3, false);
+
+        Set<String> rows = getIndexed(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+
+
+        rows = getIndexed(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(15)));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(29)));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 100, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(27)));
+
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key2", "key3"}, rows.toArray(new String[rows.size()])));
+    }
+
+    @Test
+    public void testInsertingIncorrectValuesIntoAgeIndex()
+    {
+        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+        final ByteBuffer age = UTF8Type.instance.decompose("age");
+
+        Mutation rm = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key1")));
+        update(rm, new ArrayList<Cell>()
+        {{
+            add(buildCell(firstName, AsciiType.instance.decompose("pavel"), System.currentTimeMillis()));
+            add(buildCell(age, LongType.instance.decompose(26L), System.currentTimeMillis()));
+        }});
+        rm.apply();
+
+        store.forceBlockingFlush();
+
+        Set<String> rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
+                                                 buildExpression(age, Operator.GTE, Int32Type.instance.decompose(26)));
+
+        // index is expected to have 0 results because age value was of wrong type
+        Assert.assertEquals(0, rows.size());
+    }
+
+
+    @Test
+    public void testUnicodeSupport()
+    {
+        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+
+        final ByteBuffer comment = UTF8Type.instance.decompose("comment");
+
+        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
+        update(rm, comment, UTF8Type.instance.decompose("ⓈⓅⒺⒸⒾⒶⓁ ⒞⒣⒜⒭⒮ and normal ones"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key2"));
+        update(rm, comment, UTF8Type.instance.decompose("龍馭鬱"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key3"));
+        update(rm, comment, UTF8Type.instance.decompose("インディアナ"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key4"));
+        update(rm, comment, UTF8Type.instance.decompose("レストラン"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key5"));
+        update(rm, comment, UTF8Type.instance.decompose("ベンジャミン ウエスト"), System.currentTimeMillis());
+        rm.apply();
+
+
+        Set<String> rows;
+
+        /* Memtable */
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ⓈⓅⒺⒸⒾ")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("normal")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("龍")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("馭鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("龍馭鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ベンジャミン")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("レストラ")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("インディ")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ベンジャミ")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
+
+        store.forceBlockingFlush();
+
+        /* OnDiskSA */
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ⓈⓅⒺⒸⒾ")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("normal")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("龍")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("馭鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("龍馭鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ベンジャミン")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("レストラ")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("インディ")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ベンジャミ")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
+    }
+
+    @Test
+    public void testUnicodeSuffixMode()
+    {
+        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+
+        final ByteBuffer comment = UTF8Type.instance.decompose("comment_suffix_split");
+
+        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
+        update(rm, comment, UTF8Type.instance.decompose("龍馭鬱"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key2"));
+        update(rm, comment, UTF8Type.instance.decompose("インディアナ"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key3"));
+        update(rm, comment, UTF8Type.instance.decompose("レストラン"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key4"));
+        update(rm, comment, UTF8Type.instance.decompose("ベンジャミン ウエスト"), System.currentTimeMillis());
+        rm.apply();
+
+
+        Set<String> rows;
+
+        /* Memtable */
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("龍")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("馭鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("龍馭鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ベンジャミン")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("トラン")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ディア")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ジャミン")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ン")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+
+        store.forceBlockingFlush();
+
+        /* OnDiskSA */
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("龍")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("馭鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("龍馭鬱")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ベンジャミン")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("トラン")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ディア")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ジャミン")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("ン")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+    }
+
+    @Test
+    public void testThatTooBigValueIsRejected()
+    {
+        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+
+        final ByteBuffer comment = UTF8Type.instance.decompose("comment_suffix_split");
+
+        for (int i = 0; i < 10; i++)
+        {
+            byte[] randomBytes = new byte[ThreadLocalRandom.current().nextInt(OnDiskIndexBuilder.MAX_TERM_SIZE, 5 * OnDiskIndexBuilder.MAX_TERM_SIZE)];
+            ThreadLocalRandom.current().nextBytes(randomBytes);
+
+            final ByteBuffer bigValue = UTF8Type.instance.decompose(new String(randomBytes));
+
+            Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
+            update(rm, comment, bigValue, System.currentTimeMillis());
+            rm.apply();
+
+            Set<String> rows;
+
+            rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, bigValue.duplicate()));
+            Assert.assertEquals(0, rows.size());
+
+            store.forceBlockingFlush();
+
+            rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, bigValue.duplicate()));
+            Assert.assertEquals(0, rows.size());
+        }
+    }
+
+    @Test
+    public void testSearchTimeouts() throws Exception
+    {
+        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
+
+        Map<String, Pair<String, Integer>> data1 = new HashMap<String, Pair<String, Integer>>()
+        {{
+                put("key1", Pair.create("Pavel", 14));
+                put("key2", Pair.create("Pavel", 26));
+                put("key3", Pair.create("Pavel", 27));
+                put("key4", Pair.create("Jason", 27));
+        }};
+
+        ColumnFamilyStore store = loadData(data1, true);
+
+        RowFilter filter = RowFilter.create();
+        filter.add(store.metadata.getColumnDefinition(firstName), Operator.EQ, AsciiType.instance.fromString("a"));
+
+        ReadCommand command = new PartitionRangeReadCommand(store.metadata,
+                                                            FBUtilities.nowInSeconds(),
+                                                            ColumnFilter.all(store.metadata),
+                                                            filter,
+                                                            DataLimits.NONE,
+                                                            DataRange.allData(store.metadata.partitioner),
+                                                            Optional.empty());
+
+        try
+        {
+            new QueryPlan(store, command, 0).execute(ReadExecutionController.empty());
+            Assert.fail();
+        }
+        catch (TimeQuotaExceededException e)
+        {
+            // correct behavior
+        }
+        catch (Exception e)
+        {
+            Assert.fail();
+            e.printStackTrace();
+        }
+
+        // to make sure that query doesn't fail in normal conditions
+
+        Set<String> rows = getKeys(new QueryPlan(store, command, DatabaseDescriptor.getRangeRpcTimeout()).execute(ReadExecutionController.empty()));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
+    }
+
+    @Test
+    public void testLowerCaseAnalyzer()
+    {
+        testLowerCaseAnalyzer(false);
+        cleanupData();
+        testLowerCaseAnalyzer(true);
+    }
+
+    @Test
+    public void testChinesePrefixSearch()
+    {
+        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+
+        final ByteBuffer fullName = UTF8Type.instance.decompose("/output/full-name/");
+
+        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
+        update(rm, fullName, UTF8Type.instance.decompose("美加 八田"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key2"));
+        update(rm, fullName, UTF8Type.instance.decompose("仁美 瀧澤"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key3"));
+        update(rm, fullName, UTF8Type.instance.decompose("晃宏 高須"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key4"));
+        update(rm, fullName, UTF8Type.instance.decompose("弘孝 大竹"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key5"));
+        update(rm, fullName, UTF8Type.instance.decompose("満枝 榎本"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key6"));
+        update(rm, fullName, UTF8Type.instance.decompose("飛鳥 上原"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key7"));
+        update(rm, fullName, UTF8Type.instance.decompose("大輝 鎌田"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key8"));
+        update(rm, fullName, UTF8Type.instance.decompose("利久 寺地"), System.currentTimeMillis());
+        rm.apply();
+
+        store.forceBlockingFlush();
+
+
+        Set<String> rows;
+
+        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("美加 八田")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("美加")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("晃宏 高須")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("大輝")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key7" }, rows.toArray(new String[rows.size()])));
+    }
+
+    public void testLowerCaseAnalyzer(boolean forceFlush)
+    {
+        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+
+        final ByteBuffer comment = UTF8Type.instance.decompose("address");
+
+        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
+        update(rm, comment, UTF8Type.instance.decompose("577 Rogahn Valleys Apt. 178"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key2"));
+        update(rm, comment, UTF8Type.instance.decompose("89809 Beverly Course Suite 089"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key3"));
+        update(rm, comment, UTF8Type.instance.decompose("165 clydie oval apt. 399"), System.currentTimeMillis());
+        rm.apply();
+
+        if (forceFlush)
+            store.forceBlockingFlush();
+
+        Set<String> rows;
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("577 Rogahn Valleys")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("577 ROgAhn VallEYs")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("577 rogahn valleys")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("577 rogahn")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("57")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("89809 Beverly Course")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("89809 BEVERly COURSE")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("89809 beverly course")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("89809 Beverly")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("8980")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("165 ClYdie OvAl APT. 399")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("165 Clydie Oval Apt. 399")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("165 clydie oval apt. 399")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("165 ClYdie OvA")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("165 ClYdi")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(comment, Operator.EQ, UTF8Type.instance.decompose("165")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
+    }
+
+    @Test
+    public void testPrefixSSTableLookup()
+    {
+        // This test coverts particular case which interval lookup can return invalid results
+        // when queried on the prefix e.g. "j".
+        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+
+        final ByteBuffer name = UTF8Type.instance.decompose("first_name_prefix");
+
+        Mutation rm;
+
+        rm = new Mutation(KS_NAME, decoratedKey("key1"));
+        update(rm, name, UTF8Type.instance.decompose("Pavel"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key2"));
+        update(rm, name, UTF8Type.instance.decompose("Jordan"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key3"));
+        update(rm, name, UTF8Type.instance.decompose("Mikhail"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key4"));
+        update(rm, name, UTF8Type.instance.decompose("Michael"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key5"));
+        update(rm, name, UTF8Type.instance.decompose("Johnny"), System.currentTimeMillis());
+        rm.apply();
+
+        // first flush would make interval for name - 'johnny' -> 'pavel'
+        store.forceBlockingFlush();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key6"));
+        update(rm, name, UTF8Type.instance.decompose("Jason"), System.currentTimeMillis());
+        rm.apply();
+
+        rm = new Mutation(KS_NAME, decoratedKey("key7"));
+        update(rm, name, UTF8Type.instance.decompose("Vijay"), System.currentTimeMillis());
+        rm.apply();
+
+        // this flush is going to produce range - 'jason' -> 'vijay'
+        store.forceBlockingFlush();
+
+        // make sure that overlap of the prefixes is properly handled across sstables
+        // since simple interval tree lookup is not going to cover it, prefix lookup actually required.
+
+        Set<String> rows;
+        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("J")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key5", "key6" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("j")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key5", "key6" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("m")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3", "key4" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("v")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key7" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("p")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
+
+        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("j")),
+                                     buildExpression(name, Operator.NEQ, UTF8Type.instance.decompose("joh")));
+        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key6" }, rows.toArray(new String[rows.size()])));
+    }
+
+    @Test
+    public void testSettingIsLiteralOption()
+    {
+
+        // special type which is UTF-8 but is only on the inside
+        AbstractType<?> stringType = new AbstractType<String>(AbstractType.ComparisonType.CUSTOM)
+        {
+            public ByteBuffer fromString(String source) throws MarshalException
+            {
+                return UTF8Type.instance.fromString(source);
+            }
+
+            public Term fromJSONObject(Object parsed) throws MarshalException
+            {
+                throw new UnsupportedOperationException();
+            }
+
+            public TypeSerializer<String> getSerializer()
+            {
+                return UTF8Type.instance.getSerializer();
+            }
+
+            public int compareCustom(ByteBuffer a, ByteBuffer b)
+            {
+                return UTF8Type.instance.compare(a, b);
+            }
+        };
+
+        // first let's check that we get 'false' for 'isLiteral' if we don't set the option with special comparator
+        ColumnDefinition columnA = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-A", stringType);
+
+        ColumnIndex indexA = new ColumnIndex(UTF8Type.instance, columnA, IndexMetadata.fromSchemaMetadata("special-index-A", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+        {{
+            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+        }}));
+
+        Assert.assertEquals(true,  indexA.isIndexed());
+        Assert.assertEquals(false, indexA.isLiteral());
+
+        // now let's double-check that we do get 'true' when we set it
+        ColumnDefinition columnB = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-B", stringType);
+
+        ColumnIndex indexB = new ColumnIndex(UTF8Type.instance, columnB, IndexMetadata.fromSchemaMetadata("special-index-B", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+        {{
+            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+            put("is_literal", "true");
+        }}));
+
+        Assert.assertEquals(true, indexB.isIndexed());
+        Assert.assertEquals(true, indexB.isLiteral());
+
+        // and finally we should also get a 'true' if it's built-in UTF-8/ASCII comparator
+        ColumnDefinition columnC = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-C", UTF8Type.instance);
+
+        ColumnIndex indexC = new ColumnIndex(UTF8Type.instance, columnC, IndexMetadata.fromSchemaMetadata("special-index-C", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+        {{
+            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+        }}));
+
+        Assert.assertEquals(true, indexC.isIndexed());
+        Assert.assertEquals(true, indexC.isLiteral());
+
+        ColumnDefinition columnD = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-D", AsciiType.instance);
+
+        ColumnIndex indexD = new ColumnIndex(UTF8Type.instance, columnD, IndexMetadata.fromSchemaMetadata("special-index-D", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+        {{
+            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+        }}));
+
+        Assert.assertEquals(true, indexD.isIndexed());
+        Assert.assertEquals(true, indexD.isLiteral());
+
+        // and option should supersedes the comparator type
+        ColumnDefinition columnE = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-E", UTF8Type.instance);
+
+        ColumnIndex indexE = new ColumnIndex(UTF8Type.instance, columnE, IndexMetadata.fromSchemaMetadata("special-index-E", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+        {{
+            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+            put("is_literal", "false");
+        }}));
+
+        Assert.assertEquals(true,  indexE.isIndexed());
+        Assert.assertEquals(false, indexE.isLiteral());
+    }
+
+    @Test
+    public void testClusteringIndexes() throws Exception
+    {
+        testClusteringIndexes(false);
+        cleanupData();
+        testClusteringIndexes(true);
+    }
+
+    public void testClusteringIndexes(boolean forceFlush) throws Exception
+    {
+        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CLUSTRING_CF_NAME);
+
+        executeCQL("INSERT INTO %s.%s (name, location, age, height, score) VALUES (?, ?, ?, ?, ?)", "Pavel", "US", 27, 183, 1.0);
+        executeCQL("INSERT INTO %s.%s (name, location, age, height, score) VALUES (?, ?, ?, ?, ?)", "Pavel", "BY", 28, 182, 2.0);
+        executeCQL("INSERT INTO %s.%s (name, location, age, height, score) VALUES (?, ?, ?, ?, ?)", "Jordan", "US", 27, 182, 1.0);
+
+        if (forceFlush)
+            store.forceBlockingFlush();
+
+        UntypedResultSet results;
+
+        results = executeCQL("SELECT * FROM %s.%s WHERE location = ? ALLOW FILTERING", "US");
+        Assert.assertNotNull(results);
+        Assert.assertEquals(2, results.size());
+
+        results = executeCQL("SELECT * FROM %s.%s WHERE age >= ? AND height = ? ALLOW FILTERING", 27, 182);
+        Assert.assertNotNull(results);
+        Assert.assertEquals(2, results.size());
+
+        results = executeCQL("SELECT * FROM %s.%s WHERE age = ? AND height = ? ALLOW FILTERING", 28, 182);
+        Assert.assertNotNull(results);
+        Assert.assertEquals(1, results.size());
+
+        results = executeCQL("SELECT * FROM %s.%s WHERE age >= ? AND height = ? AND score >= ? ALLOW FILTERING", 27, 182, 1.0);
+        Assert.assertNotNull(results);
+        Assert.assertEquals(2, results.size());
+
+        results = executeCQL("SELECT * FROM %s.%s WHERE age >= ? AND height = ? AND score = ? ALLOW FILTERING", 27, 182, 1.0);
+        Assert.assertNotNull(results);
+        Assert.assertEquals(1, results.size());
+
+        results = executeCQL("SELECT * FROM %s.%s WHERE location = ? AND age >= ? ALLOW FILTERING", "US", 27);
+        Assert.assertNotNull(results);
+        Assert.assertEquals(2, results.size());
+
+        results = executeCQL("SELECT * FROM %s.%s WHERE location = ? ALLOW FILTERING", "BY");
+        Assert.assertNotNull(results);
+        Assert.assertEquals(1, results.size());
+    }
+
+    private static ColumnFamilyStore loadData(Map<String, Pair<String, Integer>> data, boolean forceFlush)
+    {
+        return loadData(data, System.currentTimeMillis(), forceFlush);
+    }
+
+    private static ColumnFamilyStore loadData(Map<String, Pair<String, Integer>> data, long timestamp, boolean forceFlush)
+    {
+        for (Map.Entry<String, Pair<String, Integer>> e : data.entrySet())
+            newMutation(e.getKey(), e.getValue().left, null, e.getValue().right, timestamp).apply();
+
+        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+
+        if (forceFlush)
+            store.forceBlockingFlush();
+
+        return store;
+    }
+
+    private void cleanupData()
+    {
+        Keyspace ks = Keyspace.open(KS_NAME);
+        ks.getColumnFamilyStore(CF_NAME).truncateBlocking();
+        ks.getColumnFamilyStore(CLUSTRING_CF_NAME).truncateBlocking();
+    }
+
+    private static Set<String> getIndexed(ColumnFamilyStore store, int maxResults, Expression... expressions)
+    {
+        return getIndexed(store, ColumnFilter.all(store.metadata), maxResults, expressions);
+    }
+
+    private static Set<String> getIndexed(ColumnFamilyStore store, ColumnFilter columnFilter, int maxResults, Expression... expressions)
+    {
+        return getKeys(getIndexed(store, columnFilter, null, maxResults, expressions));
+    }
+
+    private static Set<DecoratedKey> getPaged(ColumnFamilyStore store, int pageSize, Expression... expressions)
+    {
+        UnfilteredPartitionIterator currentPage;
+        Set<DecoratedKey> uniqueKeys = new TreeSet<>();
+
+        DecoratedKey lastKey = null;
+
+        int count;
+        do
+        {
+            count = 0;
+            currentPage = getIndexed(store, ColumnFilter.all(store.metadata), lastKey, pageSize, expressions);
+            if (currentPage == null)
+                break;
+
+            while (currentPage.hasNext())
+            {
+                try (UnfilteredRowIterator row = currentPage.next())
+                {
+                    uniqueKeys.add(row.partitionKey());
+                    lastKey = row.partitionKey();
+                    count++;
+                }
+            }
+
+            currentPage.close();
+        }
+        while (count == pageSize);
+
+        return uniqueKeys;
+    }
+
+    private static UnfilteredPartitionIterator getIndexed(ColumnFamilyStore store, ColumnFilter columnFilter, DecoratedKey startKey, int maxResults, Expression... expressions)
+    {
+        DataRange range = (startKey == null)
+                            ? DataRange.allData(PARTITIONER)
+                            : DataRange.forKeyRange(new Range<>(startKey, PARTITIONER.getMinimumToken().maxKeyBound()));
+
+        RowFilter filter = RowFilter.create();
+        for (Expression e : expressions)
+            filter.add(store.metadata.getColumnDefinition(e.name), e.op, e.value);
+
+        ReadCommand command = new PartitionRangeReadCommand(store.metadata,
+                                                            FBUtilities.nowInSeconds(),
+                                                            columnFilter,
+                                                            filter,
+                                                            DataLimits.thriftLimits(maxResults, DataLimits.NO_LIMIT),
+                                                            range,
+                                                            Optional.empty());
+
+        return command.executeLocally(command.executionController());
+    }
+
+    private static Mutation newMutation(String key, String firstName, String lastName, int age, long timestamp)
+    {
+        Mutation rm = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose(key)));
+        List<Cell> cells = new ArrayList<>(3);
+
+        if (age >= 0)
+            cells.add(buildCell(ByteBufferUtil.bytes("age"), Int32Type.instance.decompose(age), timestamp));
+        if (firstName != null)
+            cells.add(buildCell(ByteBufferUtil.bytes("first_name"), UTF8Type.instance.decompose(firstName), timestamp));
+        if (lastName != null)
+            cells.add(buildCell(ByteBufferUtil.bytes("last_name"), UTF8Type.instance.decompose(lastName), timestamp));
+
+        update(rm, cells);
+        return rm;
+    }
+
+    private static Set<String> getKeys(final UnfilteredPartitionIterator rows)
+    {
+        try
+        {
+            return new TreeSet<String>()
+            {{
+                while (rows.hasNext())
+                {
+                    try (UnfilteredRowIterator row = rows.next())
+                    {
+                        add(AsciiType.instance.compose(row.partitionKey().getKey()));
+                    }
+                }
+            }};
+        }
+        finally
+        {
+            rows.close();
+        }
+    }
+
+    private static List<String> convert(final Set<DecoratedKey> keys)
+    {
+        return new ArrayList<String>()
+        {{
+            for (DecoratedKey key : keys)
+                add(AsciiType.instance.getString(key.getKey()));
+        }};
+    }
+
+    private UntypedResultSet executeCQL(String query, Object... values)
+    {
+        return QueryProcessor.executeOnceInternal(String.format(query, KS_NAME, CLUSTRING_CF_NAME), values);
+    }
+
+    private Set<String> executeCQL(String rawStatement) throws Exception
+    {
+        SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(rawStatement).prepare().statement;
+        ResultMessage.Rows cqlRows = statement.executeInternal(QueryState.forInternalCalls(), QueryOptions.DEFAULT);
+
+        Set<String> results = new TreeSet<>();
+        for (CqlRow row : cqlRows.toThriftResult().getRows())
+        {
+            for (org.apache.cassandra.thrift.Column col : row.columns)
+            {
+                String columnName = UTF8Type.instance.getString(col.bufferForName());
+                if (columnName.equals("id"))
+                    results.add(AsciiType.instance.getString(col.bufferForValue()));
+            }
+        }
+
+        return results;
+    }
+
+    private static DecoratedKey decoratedKey(ByteBuffer key)
+    {
+        return PARTITIONER.decorateKey(key);
+    }
+
+    private static DecoratedKey decoratedKey(String key)
+    {
+        return decoratedKey(AsciiType.instance.fromString(key));
+    }
+
+    private static Row buildRow(Collection<Cell> cells)
+    {
+        return buildRow(cells.toArray(new Cell[cells.size()]));

<TRUNCATED>

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

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/plan/Operation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/plan/Operation.java b/src/java/org/apache/cassandra/index/sasi/plan/Operation.java
new file mode 100644
index 0000000..1857c56
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/plan/Operation.java
@@ -0,0 +1,477 @@
+/*
+ * 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.index.sasi.plan;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.ColumnDefinition.Kind;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.filter.RowFilter;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.analyzer.AbstractAnalyzer;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.plan.Expression.Op;
+import org.apache.cassandra.index.sasi.utils.RangeIntersectionIterator;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.index.sasi.utils.RangeUnionIterator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.*;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class Operation extends RangeIterator<Long, Token>
+{
+    public enum OperationType
+    {
+        AND, OR;
+
+        public boolean apply(boolean a, boolean b)
+        {
+            switch (this)
+            {
+                case OR:
+                    return a | b;
+
+                case AND:
+                    return a & b;
+
+                default:
+                    throw new AssertionError();
+            }
+        }
+    }
+
+    private final QueryController controller;
+
+    protected final OperationType op;
+    protected final ListMultimap<ColumnDefinition, Expression> expressions;
+    protected final RangeIterator<Long, Token> range;
+
+    protected Operation left, right;
+
+    private Operation(OperationType operation,
+                      QueryController controller,
+                      ListMultimap<ColumnDefinition, Expression> expressions,
+                      RangeIterator<Long, Token> range,
+                      Operation left, Operation right)
+    {
+        super(range);
+
+        this.op = operation;
+        this.controller = controller;
+        this.expressions = expressions;
+        this.range = range;
+
+        this.left = left;
+        this.right = right;
+    }
+
+    /**
+     * Recursive "satisfies" checks based on operation
+     * and data from the lower level members using depth-first search
+     * and bubbling the results back to the top level caller.
+     *
+     * Most of the work here is done by {@link #localSatisfiedBy(Unfiltered, boolean)}
+     * see it's comment for details, if there are no local expressions
+     * assigned to Operation it will call satisfiedBy(Row) on it's children.
+     *
+     * Query: first_name = X AND (last_name = Y OR address = XYZ AND street = IL AND city = C) OR (state = 'CA' AND country = 'US')
+     * Row: key1: (first_name: X, last_name: Z, address: XYZ, street: IL, city: C, state: NY, country:US)
+     *
+     * #1                       OR
+     *                        /    \
+     * #2       (first_name) AND   AND (state, country)
+     *                          \
+     * #3            (last_name) OR
+     *                             \
+     * #4                          AND (address, street, city)
+     *
+     *
+     * Evaluation of the key1 is top-down depth-first search:
+     *
+     * --- going down ---
+     * Level #1 is evaluated, OR expression has to pull results from it's children which are at level #2 and OR them together,
+     * Level #2 AND (state, country) could be be evaluated right away, AND (first_name) refers to it's "right" child from level #3
+     * Level #3 OR (last_name) requests results from level #4
+     * Level #4 AND (address, street, city) does logical AND between it's 3 fields, returns result back to level #3.
+     * --- bubbling up ---
+     * Level #3 computes OR between AND (address, street, city) result and it's "last_name" expression
+     * Level #2 computes AND between "first_name" and result of level #3, AND (state, country) which is already computed
+     * Level #1 does OR between results of AND (first_name) and AND (state, country) and returns final result.
+     *
+     * @param row The row to check.
+     * @return true if give Row satisfied all of the expressions in the tree,
+     *         false otherwise.
+     */
+    public boolean satisfiedBy(Unfiltered row, boolean allowMissingColumns)
+    {
+        boolean sideL, sideR;
+
+        if (expressions == null || expressions.isEmpty())
+        {
+            sideL =  left != null &&  left.satisfiedBy(row, allowMissingColumns);
+            sideR = right != null && right.satisfiedBy(row, allowMissingColumns);
+
+            // one of the expressions was skipped
+            // because it had no indexes attached
+            if (left == null)
+                return sideR;
+        }
+        else
+        {
+            sideL = localSatisfiedBy(row, allowMissingColumns);
+
+            // if there is no right it means that this expression
+            // is last in the sequence, we can just return result from local expressions
+            if (right == null)
+                return sideL;
+
+            sideR = right.satisfiedBy(row, allowMissingColumns);
+        }
+
+
+        return op.apply(sideL, sideR);
+    }
+
+    /**
+     * Check every expression in the analyzed list to figure out if the
+     * columns in the give row match all of the based on the operation
+     * set to the current operation node.
+     *
+     * The algorithm is as follows: for every given expression from analyzed
+     * list get corresponding column from the Row:
+     *   - apply {@link Expression#contains(ByteBuffer)}
+     *     method to figure out if it's satisfied;
+     *   - apply logical operation between boolean accumulator and current boolean result;
+     *   - if result == false and node's operation is AND return right away;
+     *
+     * After all of the expressions have been evaluated return resulting accumulator variable.
+     *
+     * Example:
+     *
+     * Operation = (op: AND, columns: [first_name = p, 5 < age < 7, last_name: y])
+     * Row = (first_name: pavel, last_name: y, age: 6, timestamp: 15)
+     *
+     * #1 get "first_name" = p (expressions)
+     *      - row-get "first_name"                      => "pavel"
+     *      - compare "pavel" against "p"               => true (current)
+     *      - set accumulator current                   => true (because this is expression #1)
+     *
+     * #2 get "last_name" = y (expressions)
+     *      - row-get "last_name"                       => "y"
+     *      - compare "y" against "y"                   => true (current)
+     *      - set accumulator to accumulator & current  => true
+     *
+     * #3 get 5 < "age" < 7 (expressions)
+     *      - row-get "age"                             => "6"
+     *      - compare 5 < 6 < 7                         => true (current)
+     *      - set accumulator to accumulator & current  => true
+     *
+     * #4 return accumulator => true (row satisfied all of the conditions)
+     *
+     * @param row The row to check.
+     * @return true if give Row satisfied all of the analyzed expressions,
+     *         false otherwise.
+     */
+    private boolean localSatisfiedBy(Unfiltered row, boolean allowMissingColumns)
+    {
+        if (row == null || !row.isRow())
+            return false;
+
+        final int now = FBUtilities.nowInSeconds();
+        boolean result = false;
+        int idx = 0;
+
+        for (ColumnDefinition column : expressions.keySet())
+        {
+            if (column.kind == Kind.PARTITION_KEY)
+                continue;
+
+            ByteBuffer value = ColumnIndex.getValueOf(column, (Row) row, now);
+            boolean isMissingColumn = value == null;
+
+            if (!allowMissingColumns && isMissingColumn)
+                throw new IllegalStateException("All indexed columns should be included into the column slice, missing: " + column);
+
+            boolean isMatch = false;
+            // If there is a column with multiple expressions that effectively means an OR
+            // e.g. comment = 'x y z' could be split into 'comment' EQ 'x', 'comment' EQ 'y', 'comment' EQ 'z'
+            // by analyzer, in situation like that we only need to check if at least one of expressions matches,
+            // and there is no hit on the NOT_EQ (if any) which are always at the end of the filter list.
+            // Loop always starts from the end of the list, which makes it possible to break after the last
+            // NOT_EQ condition on first EQ/RANGE condition satisfied, instead of checking every
+            // single expression in the column filter list.
+            List<Expression> filters = expressions.get(column);
+            for (int i = filters.size() - 1; i >= 0; i--)
+            {
+                Expression expression = filters.get(i);
+                isMatch = !isMissingColumn && expression.contains(value);
+                if (expression.getOp() == Op.NOT_EQ)
+                {
+                    // since this is NOT_EQ operation we have to
+                    // inverse match flag (to check against other expressions),
+                    // and break in case of negative inverse because that means
+                    // that it's a positive hit on the not-eq clause.
+                    isMatch = !isMatch;
+                    if (!isMatch)
+                        break;
+                } // if it was a match on EQ/RANGE or column is missing
+                else if (isMatch || isMissingColumn)
+                    break;
+            }
+
+            if (idx++ == 0)
+            {
+                result = isMatch;
+                continue;
+            }
+
+            result = op.apply(result, isMatch);
+
+            // exit early because we already got a single false
+            if (op == OperationType.AND && !result)
+                return false;
+        }
+
+        return idx == 0 || result;
+    }
+
+    @VisibleForTesting
+    protected static ListMultimap<ColumnDefinition, Expression> analyzeGroup(QueryController controller,
+                                                                             OperationType op,
+                                                                             List<RowFilter.Expression> expressions)
+    {
+        ListMultimap<ColumnDefinition, Expression> analyzed = ArrayListMultimap.create();
+
+        // sort all of the expressions in the operation by name and priority of the logical operator
+        // this gives us an efficient way to handle inequality and combining into ranges without extra processing
+        // and converting expressions from one type to another.
+        Collections.sort(expressions, (a, b) -> {
+            int cmp = a.column().compareTo(b.column());
+            return cmp == 0 ? -Integer.compare(getPriority(a.operator()), getPriority(b.operator())) : cmp;
+        });
+
+        for (final RowFilter.Expression e : expressions)
+        {
+            ColumnIndex columnIndex = controller.getIndex(e);
+            List<Expression> perColumn = analyzed.get(e.column());
+
+            if (columnIndex == null)
+                columnIndex = new ColumnIndex(controller.getKeyValidator(), e.column(), null);
+
+            AbstractAnalyzer analyzer = columnIndex.getAnalyzer();
+            analyzer.reset(e.getIndexValue());
+
+            // EQ/NOT_EQ can have multiple expressions e.g. text = "Hello World",
+            // becomes text = "Hello" OR text = "World" because "space" is always interpreted as a split point (by analyzer),
+            // NOT_EQ is made an independent expression only in case of pre-existing multiple EQ expressions, or
+            // if there is no EQ operations and NOT_EQ is met or a single NOT_EQ expression present,
+            // in such case we know exactly that there would be no more EQ/RANGE expressions for given column
+            // since NOT_EQ has the lowest priority.
+            if (e.operator() == Operator.EQ
+                    || (e.operator() == Operator.NEQ
+                       && (perColumn.size() == 0 || perColumn.size() > 1
+                           || (perColumn.size() == 1 && perColumn.get(0).getOp() == Op.NOT_EQ))))
+            {
+                while (analyzer.hasNext())
+                {
+                    final ByteBuffer token = analyzer.next();
+                    perColumn.add(new Expression(controller, columnIndex).add(e.operator(), token));
+                }
+            }
+            else
+            // "range" or not-equals operator, combines both bounds together into the single expression,
+            // iff operation of the group is AND, otherwise we are forced to create separate expressions,
+            // not-equals is combined with the range iff operator is AND.
+            {
+                Expression range;
+                if (perColumn.size() == 0 || op != OperationType.AND)
+                    perColumn.add((range = new Expression(controller, columnIndex)));
+                else
+                    range = Iterables.getLast(perColumn);
+
+                while (analyzer.hasNext())
+                    range.add(e.operator(), analyzer.next());
+            }
+        }
+
+        return analyzed;
+    }
+
+    private static int getPriority(Operator op)
+    {
+        switch (op)
+        {
+            case EQ:
+                return 4;
+
+            case GTE:
+            case GT:
+                return 3;
+
+            case LTE:
+            case LT:
+                return 2;
+
+            case NEQ:
+                return 1;
+
+            default:
+                return 0;
+        }
+    }
+
+    protected Token computeNext()
+    {
+        return range != null && range.hasNext() ? range.next() : endOfData();
+    }
+
+    protected void performSkipTo(Long nextToken)
+    {
+        if (range != null)
+            range.skipTo(nextToken);
+    }
+
+    public void close() throws IOException
+    {
+        controller.releaseIndexes(this);
+    }
+
+    public static class Builder
+    {
+        private final QueryController controller;
+
+        protected final OperationType op;
+        protected final List<RowFilter.Expression> expressions;
+
+        protected Builder left, right;
+
+        public Builder(OperationType operation, QueryController controller, RowFilter.Expression... columns)
+        {
+            this.op = operation;
+            this.controller = controller;
+            this.expressions = new ArrayList<>();
+            Collections.addAll(expressions, columns);
+        }
+
+        public Builder setRight(Builder operation)
+        {
+            this.right = operation;
+            return this;
+        }
+
+        public Builder setLeft(Builder operation)
+        {
+            this.left = operation;
+            return this;
+        }
+
+        public void add(RowFilter.Expression e)
+        {
+            expressions.add(e);
+        }
+
+        public void add(Collection<RowFilter.Expression> newExpressions)
+        {
+            if (expressions != null)
+                expressions.addAll(newExpressions);
+        }
+
+        public Operation complete()
+        {
+            if (!expressions.isEmpty())
+            {
+                ListMultimap<ColumnDefinition, Expression> analyzedExpressions = analyzeGroup(controller, op, expressions);
+                RangeIterator.Builder<Long, Token> range = controller.getIndexes(op, analyzedExpressions.values());
+
+                Operation rightOp = null;
+                if (right != null)
+                {
+                    rightOp = right.complete();
+                    range.add(rightOp);
+                }
+
+                return new Operation(op, controller, analyzedExpressions, range.build(), null, rightOp);
+            }
+            else
+            {
+                Operation leftOp = null, rightOp = null;
+                boolean leftIndexes = false, rightIndexes = false;
+
+                if (left != null)
+                {
+                    leftOp = left.complete();
+                    leftIndexes = leftOp != null && leftOp.range != null;
+                }
+
+                if (right != null)
+                {
+                    rightOp = right.complete();
+                    rightIndexes = rightOp != null && rightOp.range != null;
+                }
+
+                RangeIterator<Long, Token> join;
+                /**
+                 * Operation should allow one of it's sub-trees to wrap no indexes, that is related  to the fact that we
+                 * have to accept defined-but-not-indexed columns as well as key range as IndexExpressions.
+                 *
+                 * Two cases are possible:
+                 *
+                 * only left child produced indexed iterators, that could happen when there are two columns
+                 * or key range on the right:
+                 *
+                 *                AND
+                 *              /     \
+                 *            OR       \
+                 *           /   \     AND
+                 *          a     b   /   \
+                 *                  key   key
+                 *
+                 * only right child produced indexed iterators:
+                 *
+                 *               AND
+                 *              /    \
+                 *            AND     a
+                 *           /   \
+                 *         key  key
+                 */
+                if (leftIndexes && !rightIndexes)
+                    join = leftOp;
+                else if (!leftIndexes && rightIndexes)
+                    join = rightOp;
+                else if (leftIndexes)
+                {
+                    RangeIterator.Builder<Long, Token> builder = op == OperationType.OR
+                                                ? RangeUnionIterator.<Long, Token>builder()
+                                                : RangeIntersectionIterator.<Long, Token>builder();
+
+                    join = builder.add(leftOp).add(rightOp).build();
+                }
+                else
+                    throw new AssertionError("both sub-trees have 0 indexes.");
+
+                return new Operation(op, controller, null, join, leftOp, rightOp);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java b/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java
new file mode 100644
index 0000000..8e10fd0
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java
@@ -0,0 +1,261 @@
+/*
+ * 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.index.sasi.plan;
+
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.ColumnFamilyStore.RefViewFragment;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.filter.RowFilter;
+import org.apache.cassandra.db.lifecycle.SSTableSet;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.index.Index;
+import org.apache.cassandra.index.sasi.SASIIndex;
+import org.apache.cassandra.index.sasi.SSTableIndex;
+import org.apache.cassandra.index.sasi.TermIterator;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.conf.view.View;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.exceptions.TimeQuotaExceededException;
+import org.apache.cassandra.index.sasi.plan.Operation.OperationType;
+import org.apache.cassandra.index.sasi.utils.RangeIntersectionIterator;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.index.sasi.utils.RangeUnionIterator;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.Pair;
+
+public class QueryController
+{
+    private final long executionQuota;
+    private final long executionStart;
+
+    private final ColumnFamilyStore cfs;
+    private final PartitionRangeReadCommand command;
+    private final Map<Collection<Expression>, List<RangeIterator<Long, Token>>> resources = new HashMap<>();
+    private final RefViewFragment scope;
+    private final Set<SSTableReader> sstables;
+
+    public QueryController(ColumnFamilyStore cfs, PartitionRangeReadCommand command, long timeQuotaMs)
+    {
+        this.cfs = cfs;
+        this.command = command;
+        this.executionQuota = TimeUnit.MILLISECONDS.toNanos(timeQuotaMs);
+        this.executionStart = System.nanoTime();
+        this.scope = getSSTableScope(cfs, command);
+        this.sstables = new HashSet<>(scope.sstables);
+    }
+
+    public boolean isForThrift()
+    {
+        return command.isForThrift();
+    }
+
+    public CFMetaData metadata()
+    {
+        return command.metadata();
+    }
+
+    public Collection<RowFilter.Expression> getExpressions()
+    {
+        return command.rowFilter().getExpressions();
+    }
+
+    public DataRange dataRange()
+    {
+        return command.dataRange();
+    }
+
+    public AbstractType<?> getKeyValidator()
+    {
+        return cfs.metadata.getKeyValidator();
+    }
+
+    public ColumnIndex getIndex(RowFilter.Expression expression)
+    {
+        Optional<Index> index = cfs.indexManager.getBestIndexFor(expression);
+        return index.isPresent() ? ((SASIIndex) index.get()).getIndex() : null;
+    }
+
+
+    public UnfilteredRowIterator getPartition(DecoratedKey key, ReadExecutionController executionController)
+    {
+        if (key == null)
+            throw new NullPointerException();
+        try
+        {
+            SinglePartitionReadCommand partition = SinglePartitionReadCommand.create(command.isForThrift(),
+                                                                                     cfs.metadata,
+                                                                                     command.nowInSec(),
+                                                                                     command.columnFilter(),
+                                                                                     command.rowFilter().withoutExpressions(),
+                                                                                     DataLimits.NONE,
+                                                                                     key,
+                                                                                     command.clusteringIndexFilter(key));
+
+            return partition.queryMemtableAndDisk(cfs, executionController.baseReadOpOrderGroup());
+        }
+        finally
+        {
+            checkpoint();
+        }
+    }
+
+    /**
+     * Build a range iterator from the given list of expressions by applying given operation (OR/AND).
+     * Building of such iterator involves index search, results of which are persisted in the internal resources list
+     * and can be released later via {@link QueryController#releaseIndexes(Operation)}.
+     *
+     * @param op The operation type to coalesce expressions with.
+     * @param expressions The expressions to build range iterator from (expressions with not results are ignored).
+     *
+     * @return The range builder based on given expressions and operation type.
+     */
+    public RangeIterator.Builder<Long, Token> getIndexes(OperationType op, Collection<Expression> expressions)
+    {
+        if (resources.containsKey(expressions))
+            throw new IllegalArgumentException("Can't process the same expressions multiple times.");
+
+        RangeIterator.Builder<Long, Token> builder = op == OperationType.OR
+                                                ? RangeUnionIterator.<Long, Token>builder()
+                                                : RangeIntersectionIterator.<Long, Token>builder();
+
+        List<RangeIterator<Long, Token>> perIndexUnions = new ArrayList<>();
+
+        for (Map.Entry<Expression, Set<SSTableIndex>> e : getView(op, expressions).entrySet())
+        {
+            RangeIterator<Long, Token> index = TermIterator.build(e.getKey(), e.getValue());
+
+            if (index == null)
+                continue;
+
+            builder.add(index);
+            perIndexUnions.add(index);
+        }
+
+        resources.put(expressions, perIndexUnions);
+        return builder;
+    }
+
+    public void checkpoint()
+    {
+        if ((System.nanoTime() - executionStart) >= executionQuota)
+            throw new TimeQuotaExceededException();
+    }
+
+    public void releaseIndexes(Operation operation)
+    {
+        if (operation.expressions != null)
+            releaseIndexes(resources.remove(operation.expressions.values()));
+    }
+
+    private void releaseIndexes(List<RangeIterator<Long, Token>> indexes)
+    {
+        if (indexes == null)
+            return;
+
+        indexes.forEach(FileUtils::closeQuietly);
+    }
+
+    public void finish()
+    {
+        try
+        {
+            resources.values().forEach(this::releaseIndexes);
+        }
+        finally
+        {
+            scope.release();
+        }
+    }
+
+    private Map<Expression, Set<SSTableIndex>> getView(OperationType op, Collection<Expression> expressions)
+    {
+        // first let's determine the primary expression if op is AND
+        Pair<Expression, Set<SSTableIndex>> primary = (op == OperationType.AND) ? calculatePrimary(expressions) : null;
+
+        Map<Expression, Set<SSTableIndex>> indexes = new HashMap<>();
+        for (Expression e : expressions)
+        {
+            // NO_EQ and non-index column query should only act as FILTER BY for satisfiedBy(Row) method
+            // because otherwise it likely to go through the whole index.
+            if (!e.isIndexed() || e.getOp() == Expression.Op.NOT_EQ)
+                continue;
+
+            // primary expression, we'll have to add as is
+            if (primary != null && e.equals(primary.left))
+            {
+                indexes.put(primary.left, primary.right);
+                continue;
+            }
+
+            View view = e.index.getView();
+            if (view == null)
+                continue;
+
+            Set<SSTableIndex> readers = new HashSet<>();
+            if (primary != null && primary.right.size() > 0)
+            {
+                for (SSTableIndex index : primary.right)
+                    readers.addAll(view.match(index.minKey(), index.maxKey()));
+            }
+            else
+            {
+                readers.addAll(view.match(sstables, e));
+            }
+
+            indexes.put(e, readers);
+        }
+
+        return indexes;
+    }
+
+    private Pair<Expression, Set<SSTableIndex>> calculatePrimary(Collection<Expression> expressions)
+    {
+        Expression expression = null;
+        Set<SSTableIndex> primaryIndexes = Collections.emptySet();
+
+        for (Expression e : expressions)
+        {
+            if (!e.isIndexed())
+                continue;
+
+            View view = e.index.getView();
+            if (view == null)
+                continue;
+
+            Set<SSTableIndex> indexes = view.match(sstables, e);
+            if (primaryIndexes.size() > indexes.size())
+            {
+                primaryIndexes = indexes;
+                expression = e;
+            }
+        }
+
+        return expression == null ? null : Pair.create(expression, primaryIndexes);
+    }
+
+    private static RefViewFragment getSSTableScope(ColumnFamilyStore cfs, PartitionRangeReadCommand command)
+    {
+        return cfs.selectAndReference(org.apache.cassandra.db.lifecycle.View.select(SSTableSet.CANONICAL, command.dataRange().keyRange()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java b/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java
new file mode 100644
index 0000000..d34b05a
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java
@@ -0,0 +1,170 @@
+/*
+ * 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.index.sasi.plan;
+
+import java.util.*;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.plan.Operation.OperationType;
+import org.apache.cassandra.exceptions.RequestTimeoutException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.AbstractIterator;
+
+public class QueryPlan
+{
+    private final QueryController controller;
+
+    public QueryPlan(ColumnFamilyStore cfs, ReadCommand command, long executionQuotaMs)
+    {
+        this.controller = new QueryController(cfs, (PartitionRangeReadCommand) command, executionQuotaMs);
+    }
+
+    /**
+     * Converts expressions into operation tree (which is currently just a single AND).
+     *
+     * Operation tree allows us to do a couple of important optimizations
+     * namely, group flattening for AND operations (query rewrite), expression bounds checks,
+     * "satisfies by" checks for resulting rows with an early exit.
+     *
+     * @return root of the operations tree.
+     */
+    private Operation analyze()
+    {
+        try
+        {
+            Operation.Builder and = new Operation.Builder(OperationType.AND, controller);
+            controller.getExpressions().forEach(and::add);
+            return and.complete();
+        }
+        catch (Exception | Error e)
+        {
+            controller.finish();
+            throw e;
+        }
+    }
+
+    public UnfilteredPartitionIterator execute(ReadExecutionController executionController) throws RequestTimeoutException
+    {
+        return new ResultIterator(analyze(), controller, executionController);
+    }
+
+    private static class ResultIterator extends AbstractIterator<UnfilteredRowIterator> implements UnfilteredPartitionIterator
+    {
+        private final AbstractBounds<PartitionPosition> keyRange;
+        private final Operation operationTree;
+        private final QueryController controller;
+        private final ReadExecutionController executionController;
+
+        private Iterator<DecoratedKey> currentKeys = null;
+
+        public ResultIterator(Operation operationTree, QueryController controller, ReadExecutionController executionController)
+        {
+            this.keyRange = controller.dataRange().keyRange();
+            this.operationTree = operationTree;
+            this.controller = controller;
+            this.executionController = executionController;
+            if (operationTree != null)
+                operationTree.skipTo((Long) keyRange.left.getToken().getTokenValue());
+        }
+
+        protected UnfilteredRowIterator computeNext()
+        {
+            if (operationTree == null)
+                return endOfData();
+
+            for (;;)
+            {
+                if (currentKeys == null || !currentKeys.hasNext())
+                {
+                    if (!operationTree.hasNext())
+                         return endOfData();
+
+                    Token token = operationTree.next();
+                    currentKeys = token.iterator();
+                }
+
+                while (currentKeys.hasNext())
+                {
+                    DecoratedKey key = currentKeys.next();
+
+                    if (!keyRange.right.isMinimum() && keyRange.right.compareTo(key) < 0)
+                        return endOfData();
+
+                    try (UnfilteredRowIterator partition = controller.getPartition(key, executionController))
+                    {
+                        List<Unfiltered> clusters = new ArrayList<>();
+                        while (partition.hasNext())
+                        {
+                            Unfiltered row = partition.next();
+                            if (operationTree.satisfiedBy(row, true))
+                                clusters.add(row);
+                        }
+
+                        if (!clusters.isEmpty())
+                            return new PartitionIterator(partition, clusters);
+                    }
+                }
+            }
+        }
+
+        private static class PartitionIterator extends AbstractUnfilteredRowIterator
+        {
+            private final Iterator<Unfiltered> rows;
+
+            public PartitionIterator(UnfilteredRowIterator partition, Collection<Unfiltered> content)
+            {
+                super(partition.metadata(),
+                      partition.partitionKey(),
+                      partition.partitionLevelDeletion(),
+                      partition.columns(),
+                      partition.staticRow(),
+                      partition.isReverseOrder(),
+                      partition.stats());
+
+                rows = content.iterator();
+            }
+
+            @Override
+            protected Unfiltered computeNext()
+            {
+                return rows.hasNext() ? rows.next() : endOfData();
+            }
+        }
+
+        public boolean isForThrift()
+        {
+            return controller.isForThrift();
+        }
+
+        public CFMetaData metadata()
+        {
+            return controller.metadata();
+        }
+
+        public void close()
+        {
+            FileUtils.closeQuietly(operationTree);
+            controller.finish();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/sa/ByteTerm.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/sa/ByteTerm.java b/src/java/org/apache/cassandra/index/sasi/sa/ByteTerm.java
new file mode 100644
index 0000000..c7bbab7
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/sa/ByteTerm.java
@@ -0,0 +1,51 @@
+/*
+ * 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.index.sasi.sa;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.index.sasi.disk.TokenTreeBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public class ByteTerm extends Term<ByteBuffer>
+{
+    public ByteTerm(int position, ByteBuffer value, TokenTreeBuilder tokens)
+    {
+        super(position, value, tokens);
+    }
+
+    public ByteBuffer getTerm()
+    {
+        return value.duplicate();
+    }
+
+    public ByteBuffer getSuffix(int start)
+    {
+        return (ByteBuffer) value.duplicate().position(value.position() + start);
+    }
+
+    public int compareTo(AbstractType<?> comparator, Term other)
+    {
+        return comparator.compare(value, (ByteBuffer) other.value);
+    }
+
+    public int length()
+    {
+        return value.remaining();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/sa/CharTerm.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/sa/CharTerm.java b/src/java/org/apache/cassandra/index/sasi/sa/CharTerm.java
new file mode 100644
index 0000000..533b566
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/sa/CharTerm.java
@@ -0,0 +1,54 @@
+/*
+ * 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.index.sasi.sa;
+
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+
+import org.apache.cassandra.index.sasi.disk.TokenTreeBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+import com.google.common.base.Charsets;
+
+public class CharTerm extends Term<CharBuffer>
+{
+    public CharTerm(int position, CharBuffer value, TokenTreeBuilder tokens)
+    {
+        super(position, value, tokens);
+    }
+
+    public ByteBuffer getTerm()
+    {
+        return Charsets.UTF_8.encode(value.duplicate());
+    }
+
+    public ByteBuffer getSuffix(int start)
+    {
+        return Charsets.UTF_8.encode(value.subSequence(value.position() + start, value.remaining()));
+    }
+
+    public int compareTo(AbstractType<?> comparator, Term other)
+    {
+        return value.compareTo((CharBuffer) other.value);
+    }
+
+    public int length()
+    {
+        return value.length();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/sa/IntegralSA.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/sa/IntegralSA.java b/src/java/org/apache/cassandra/index/sasi/sa/IntegralSA.java
new file mode 100644
index 0000000..8356585
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/sa/IntegralSA.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.index.sasi.sa;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
+import org.apache.cassandra.index.sasi.disk.TokenTreeBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.utils.Pair;
+
+public class IntegralSA extends SA<ByteBuffer>
+{
+    public IntegralSA(AbstractType<?> comparator, OnDiskIndexBuilder.Mode mode)
+    {
+        super(comparator, mode);
+    }
+
+    public Term<ByteBuffer> getTerm(ByteBuffer termValue, TokenTreeBuilder tokens)
+    {
+        return new ByteTerm(charCount, termValue, tokens);
+    }
+
+    public TermIterator finish()
+    {
+        return new IntegralSuffixIterator();
+    }
+
+
+    private class IntegralSuffixIterator extends TermIterator
+    {
+        private final Iterator<Term<ByteBuffer>> termIterator;
+
+        public IntegralSuffixIterator()
+        {
+            Collections.sort(terms, new Comparator<Term<?>>()
+            {
+                public int compare(Term<?> a, Term<?> b)
+                {
+                    return a.compareTo(comparator, b);
+                }
+            });
+
+            termIterator = terms.iterator();
+        }
+
+        public ByteBuffer minTerm()
+        {
+            return terms.get(0).getTerm();
+        }
+
+        public ByteBuffer maxTerm()
+        {
+            return terms.get(terms.size() - 1).getTerm();
+        }
+
+        protected Pair<ByteBuffer, TokenTreeBuilder> computeNext()
+        {
+            if (!termIterator.hasNext())
+                return endOfData();
+
+            Term<ByteBuffer> term = termIterator.next();
+            return Pair.create(term.getTerm(), term.getTokens().finish());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/sa/SA.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/sa/SA.java b/src/java/org/apache/cassandra/index/sasi/sa/SA.java
new file mode 100644
index 0000000..75f9f92
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/sa/SA.java
@@ -0,0 +1,58 @@
+/*
+ * 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.index.sasi.sa;
+
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder.Mode;
+import org.apache.cassandra.index.sasi.disk.TokenTreeBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public abstract class SA<T extends Buffer>
+{
+    protected final AbstractType<?> comparator;
+    protected final Mode mode;
+
+    protected final List<Term<T>> terms = new ArrayList<>();
+    protected int charCount = 0;
+
+    public SA(AbstractType<?> comparator, Mode mode)
+    {
+        this.comparator = comparator;
+        this.mode = mode;
+    }
+
+    public Mode getMode()
+    {
+        return mode;
+    }
+
+    public void add(ByteBuffer termValue, TokenTreeBuilder tokens)
+    {
+        Term<T> term = getTerm(termValue, tokens);
+        terms.add(term);
+        charCount += term.length();
+    }
+
+    public abstract TermIterator finish();
+
+    protected abstract Term<T> getTerm(ByteBuffer termValue, TokenTreeBuilder tokens);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/sa/SuffixSA.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/sa/SuffixSA.java b/src/java/org/apache/cassandra/index/sasi/sa/SuffixSA.java
new file mode 100644
index 0000000..63f6c5b
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/sa/SuffixSA.java
@@ -0,0 +1,143 @@
+/*
+ * 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.index.sasi.sa;
+
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
+import org.apache.cassandra.index.sasi.disk.TokenTreeBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.utils.Pair;
+
+import com.google.common.base.Charsets;
+import net.mintern.primitive.Primitive;
+
+public class SuffixSA extends SA<CharBuffer>
+{
+    public SuffixSA(AbstractType<?> comparator, OnDiskIndexBuilder.Mode mode)
+    {
+        super(comparator, mode);
+    }
+
+    protected Term<CharBuffer> getTerm(ByteBuffer termValue, TokenTreeBuilder tokens)
+    {
+        return new CharTerm(charCount, Charsets.UTF_8.decode(termValue.duplicate()), tokens);
+    }
+
+    public TermIterator finish()
+    {
+        return new SASuffixIterator();
+    }
+
+    private class SASuffixIterator extends TermIterator
+    {
+        private final long[] suffixes;
+
+        private int current = 0;
+        private ByteBuffer lastProcessedSuffix;
+        private TokenTreeBuilder container;
+
+        public SASuffixIterator()
+        {
+            // each element has term index and char position encoded as two 32-bit integers
+            // to avoid binary search per suffix while sorting suffix array.
+            suffixes = new long[charCount];
+
+            long termIndex = -1, currentTermLength = -1;
+            for (int i = 0; i < charCount; i++)
+            {
+                if (i >= currentTermLength || currentTermLength == -1)
+                {
+                    Term currentTerm = terms.get((int) ++termIndex);
+                    currentTermLength = currentTerm.getPosition() + currentTerm.length();
+                }
+
+                suffixes[i] = (termIndex << 32) | i;
+            }
+
+            Primitive.sort(suffixes, (a, b) -> {
+                Term aTerm = terms.get((int) (a >>> 32));
+                Term bTerm = terms.get((int) (b >>> 32));
+                return comparator.compare(aTerm.getSuffix(((int) a) - aTerm.getPosition()),
+                                          bTerm.getSuffix(((int) b) - bTerm.getPosition()));
+            });
+        }
+
+        private Pair<ByteBuffer, TokenTreeBuilder> suffixAt(int position)
+        {
+            long index = suffixes[position];
+            Term term = terms.get((int) (index >>> 32));
+            return Pair.create(term.getSuffix(((int) index) - term.getPosition()), term.getTokens());
+        }
+
+        public ByteBuffer minTerm()
+        {
+            return suffixAt(0).left;
+        }
+
+        public ByteBuffer maxTerm()
+        {
+            return suffixAt(suffixes.length - 1).left;
+        }
+
+        protected Pair<ByteBuffer, TokenTreeBuilder> computeNext()
+        {
+            while (true)
+            {
+                if (current >= suffixes.length)
+                {
+                    if (lastProcessedSuffix == null)
+                        return endOfData();
+
+                    Pair<ByteBuffer, TokenTreeBuilder> result = finishSuffix();
+
+                    lastProcessedSuffix = null;
+                    return result;
+                }
+
+                Pair<ByteBuffer, TokenTreeBuilder> suffix = suffixAt(current++);
+
+                if (lastProcessedSuffix == null)
+                {
+                    lastProcessedSuffix = suffix.left;
+                    container = new TokenTreeBuilder(suffix.right.getTokens());
+                }
+                else if (comparator.compare(lastProcessedSuffix, suffix.left) == 0)
+                {
+                    lastProcessedSuffix = suffix.left;
+                    container.add(suffix.right.getTokens());
+                }
+                else
+                {
+                    Pair<ByteBuffer, TokenTreeBuilder> result = finishSuffix();
+
+                    lastProcessedSuffix = suffix.left;
+                    container = new TokenTreeBuilder(suffix.right.getTokens());
+
+                    return result;
+                }
+            }
+        }
+
+        private Pair<ByteBuffer, TokenTreeBuilder> finishSuffix()
+        {
+            return Pair.create(lastProcessedSuffix, container.finish());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/sa/Term.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/sa/Term.java b/src/java/org/apache/cassandra/index/sasi/sa/Term.java
new file mode 100644
index 0000000..fe6eca8
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/sa/Term.java
@@ -0,0 +1,58 @@
+/*
+ * 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.index.sasi.sa;
+
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.index.sasi.disk.TokenTreeBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public abstract class Term<T extends Buffer>
+{
+    protected final int position;
+    protected final T value;
+    protected TokenTreeBuilder tokens;
+
+
+    public Term(int position, T value, TokenTreeBuilder tokens)
+    {
+        this.position = position;
+        this.value = value;
+        this.tokens = tokens;
+    }
+
+    public int getPosition()
+    {
+        return position;
+    }
+
+    public abstract ByteBuffer getTerm();
+    public abstract ByteBuffer getSuffix(int start);
+
+    public TokenTreeBuilder getTokens()
+    {
+        return tokens;
+    }
+
+    public abstract int compareTo(AbstractType<?> comparator, Term other);
+
+    public abstract int length();
+
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/sa/TermIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/sa/TermIterator.java b/src/java/org/apache/cassandra/index/sasi/sa/TermIterator.java
new file mode 100644
index 0000000..916aa07
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/sa/TermIterator.java
@@ -0,0 +1,31 @@
+/*
+ * 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.index.sasi.sa;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.index.sasi.disk.TokenTreeBuilder;
+import org.apache.cassandra.utils.Pair;
+
+import com.google.common.collect.AbstractIterator;
+
+public abstract class TermIterator extends AbstractIterator<Pair<ByteBuffer, TokenTreeBuilder>>
+{
+    public abstract ByteBuffer minTerm();
+    public abstract ByteBuffer maxTerm();
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/AbstractIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/AbstractIterator.java b/src/java/org/apache/cassandra/index/sasi/utils/AbstractIterator.java
new file mode 100644
index 0000000..cf918c1
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/AbstractIterator.java
@@ -0,0 +1,155 @@
+/*
+ * Copyright (C) 2007 The Guava Authors
+ *
+ * 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;
+
+import java.util.NoSuchElementException;
+
+import com.google.common.collect.PeekingIterator;
+
+import static com.google.common.base.Preconditions.checkState;
+
+// This is fork of the Guava AbstractIterator, the only difference
+// is that state & next variables are now protected, this was required
+// for SkippableIterator.skipTo(..) to void all previous state.
+public abstract class AbstractIterator<T> implements PeekingIterator<T>
+{
+    protected State state = State.NOT_READY;
+
+    /** Constructor for use by subclasses. */
+    protected AbstractIterator() {}
+
+    protected enum State
+    {
+        /** We have computed the next element and haven't returned it yet. */
+        READY,
+
+        /** We haven't yet computed or have already returned the element. */
+        NOT_READY,
+
+        /** We have reached the end of the data and are finished. */
+        DONE,
+
+        /** We've suffered an exception and are kaput. */
+        FAILED,
+    }
+
+    protected T next;
+
+    /**
+     * Returns the next element. <b>Note:</b> the implementation must call {@link
+     * #endOfData()} when there are no elements left in the iteration. Failure to
+     * do so could result in an infinite loop.
+     *
+     * <p>The initial invocation of {@link #hasNext()} or {@link #next()} calls
+     * this method, as does the first invocation of {@code hasNext} or {@code
+     * next} following each successful call to {@code next}. Once the
+     * implementation either invokes {@code endOfData} or throws an exception,
+     * {@code computeNext} is guaranteed to never be called again.
+     *
+     * <p>If this method throws an exception, it will propagate outward to the
+     * {@code hasNext} or {@code next} invocation that invoked this method. Any
+     * further attempts to use the iterator will result in an {@link
+     * IllegalStateException}.
+     *
+     * <p>The implementation of this method may not invoke the {@code hasNext},
+     * {@code next}, or {@link #peek()} methods on this instance; if it does, an
+     * {@code IllegalStateException} will result.
+     *
+     * @return the next element if there was one. If {@code endOfData} was called
+     *     during execution, the return value will be ignored.
+     * @throws RuntimeException if any unrecoverable error happens. This exception
+     *     will propagate outward to the {@code hasNext()}, {@code next()}, or
+     *     {@code peek()} invocation that invoked this method. Any further
+     *     attempts to use the iterator will result in an
+     *     {@link IllegalStateException}.
+     */
+    protected abstract T computeNext();
+
+    /**
+     * Implementations of {@link #computeNext} <b>must</b> invoke this method when
+     * there are no elements left in the iteration.
+     *
+     * @return {@code null}; a convenience so your {@code computeNext}
+     *     implementation can use the simple statement {@code return endOfData();}
+     */
+    protected final T endOfData()
+    {
+        state = State.DONE;
+        return null;
+    }
+
+    public final boolean hasNext()
+    {
+        checkState(state != State.FAILED);
+
+        switch (state)
+        {
+            case DONE:
+                return false;
+
+            case READY:
+                return true;
+
+            default:
+        }
+
+        return tryToComputeNext();
+    }
+
+    protected boolean tryToComputeNext()
+    {
+        state = State.FAILED; // temporary pessimism
+        next = computeNext();
+
+        if (state != State.DONE)
+        {
+            state = State.READY;
+            return true;
+        }
+
+        return false;
+    }
+
+    public final T next()
+    {
+        if (!hasNext())
+            throw new NoSuchElementException();
+
+        state = State.NOT_READY;
+        return next;
+    }
+
+    public void remove()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Returns the next element in the iteration without advancing the iteration,
+     * according to the contract of {@link PeekingIterator#peek()}.
+     *
+     * <p>Implementations of {@code AbstractIterator} that wish to expose this
+     * functionality should implement {@code PeekingIterator}.
+     */
+    public final T peek()
+    {
+        if (!hasNext())
+            throw new NoSuchElementException();
+
+        return next;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/CombinedTerm.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/CombinedTerm.java b/src/java/org/apache/cassandra/index/sasi/utils/CombinedTerm.java
new file mode 100644
index 0000000..2bf5a07
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/CombinedTerm.java
@@ -0,0 +1,103 @@
+/*
+ * 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.index.sasi.utils;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.cassandra.index.sasi.disk.OnDiskIndex.DataTerm;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.disk.TokenTree;
+import org.apache.cassandra.index.sasi.disk.TokenTreeBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+import com.carrotsearch.hppc.LongOpenHashSet;
+import com.carrotsearch.hppc.LongSet;
+import com.carrotsearch.hppc.cursors.LongCursor;
+
+public class CombinedTerm implements CombinedValue<DataTerm>
+{
+    private final AbstractType<?> comparator;
+    private final DataTerm term;
+    private final TreeMap<Long, LongSet> tokens;
+
+    public CombinedTerm(AbstractType<?> comparator, DataTerm term)
+    {
+        this.comparator = comparator;
+        this.term = term;
+        this.tokens = new TreeMap<>();
+
+        RangeIterator<Long, Token> tokens = term.getTokens();
+        while (tokens.hasNext())
+        {
+            Token current = tokens.next();
+            LongSet offsets = this.tokens.get(current.get());
+            if (offsets == null)
+                this.tokens.put(current.get(), (offsets = new LongOpenHashSet()));
+
+            for (Long offset : ((TokenTree.OnDiskToken) current).getOffsets())
+                offsets.add(offset);
+        }
+    }
+
+    public ByteBuffer getTerm()
+    {
+        return term.getTerm();
+    }
+
+    public Map<Long, LongSet> getTokens()
+    {
+        return tokens;
+    }
+
+    public TokenTreeBuilder getTokenTreeBuilder()
+    {
+        return new TokenTreeBuilder(tokens).finish();
+    }
+
+    public void merge(CombinedValue<DataTerm> other)
+    {
+        if (!(other instanceof CombinedTerm))
+            return;
+
+        CombinedTerm o = (CombinedTerm) other;
+
+        assert comparator == o.comparator;
+
+        for (Map.Entry<Long, LongSet> token : o.tokens.entrySet())
+        {
+            LongSet offsets = this.tokens.get(token.getKey());
+            if (offsets == null)
+                this.tokens.put(token.getKey(), (offsets = new LongOpenHashSet()));
+
+            for (LongCursor offset : token.getValue())
+                offsets.add(offset.value);
+        }
+    }
+
+    public DataTerm get()
+    {
+        return term;
+    }
+
+    public int compareTo(CombinedValue<DataTerm> o)
+    {
+        return term.compareTo(comparator, o.get().getTerm());
+    }
+}
\ 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/CombinedTermIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/CombinedTermIterator.java b/src/java/org/apache/cassandra/index/sasi/utils/CombinedTermIterator.java
new file mode 100644
index 0000000..06c27bf
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/CombinedTermIterator.java
@@ -0,0 +1,87 @@
+/*
+ * 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.index.sasi.utils;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.index.sasi.disk.Descriptor;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndex;
+import org.apache.cassandra.index.sasi.disk.TokenTreeBuilder;
+import org.apache.cassandra.index.sasi.sa.TermIterator;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.utils.Pair;
+
+public class CombinedTermIterator extends TermIterator
+{
+    final Descriptor descriptor;
+    final RangeIterator<OnDiskIndex.DataTerm, CombinedTerm> union;
+    final ByteBuffer min;
+    final ByteBuffer max;
+
+    public CombinedTermIterator(OnDiskIndex... sas)
+    {
+        this(Descriptor.CURRENT, sas);
+    }
+
+    public CombinedTermIterator(Descriptor d, OnDiskIndex... parts)
+    {
+        descriptor = d;
+        union = OnDiskIndexIterator.union(parts);
+
+        AbstractType<?> comparator = parts[0].getComparator(); // assumes all SAs have same comparator
+        ByteBuffer minimum = parts[0].minTerm();
+        ByteBuffer maximum = parts[0].maxTerm();
+
+        for (int i = 1; i < parts.length; i++)
+        {
+            OnDiskIndex part = parts[i];
+            if (part == null)
+                continue;
+
+            minimum = comparator.compare(minimum, part.minTerm()) > 0 ? part.minTerm() : minimum;
+            maximum = comparator.compare(maximum, part.maxTerm()) < 0 ? part.maxTerm() : maximum;
+        }
+
+        min = minimum;
+        max = maximum;
+    }
+
+    public ByteBuffer minTerm()
+    {
+        return min;
+    }
+
+    public ByteBuffer maxTerm()
+    {
+        return max;
+    }
+
+    protected Pair<ByteBuffer, TokenTreeBuilder> computeNext()
+    {
+        if (!union.hasNext())
+        {
+            return endOfData();
+        }
+        else
+        {
+            CombinedTerm term = union.next();
+            return Pair.create(term.getTerm(), term.getTokenTreeBuilder());
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/CombinedValue.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/CombinedValue.java b/src/java/org/apache/cassandra/index/sasi/utils/CombinedValue.java
new file mode 100644
index 0000000..ca5f9be
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/CombinedValue.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.index.sasi.utils;
+
+public interface CombinedValue<V> extends Comparable<CombinedValue<V>>
+{
+    void merge(CombinedValue<V> other);
+
+    V get();
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/MappedBuffer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/MappedBuffer.java b/src/java/org/apache/cassandra/index/sasi/utils/MappedBuffer.java
new file mode 100644
index 0000000..37ab1be
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/MappedBuffer.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.index.sasi.utils;
+
+import java.io.Closeable;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel.MapMode;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class MappedBuffer implements Closeable
+{
+    private final MappedByteBuffer[] pages;
+
+    private long position, limit;
+    private final long capacity;
+    private final int pageSize, sizeBits;
+
+    private MappedBuffer(MappedBuffer other)
+    {
+        this.sizeBits = other.sizeBits;
+        this.pageSize = other.pageSize;
+        this.position = other.position;
+        this.limit = other.limit;
+        this.capacity = other.capacity;
+        this.pages = other.pages;
+    }
+
+    public MappedBuffer(RandomAccessReader file)
+    {
+        this(file.getChannel(), 30);
+    }
+
+    public MappedBuffer(ChannelProxy file)
+    {
+        this(file, 30);
+    }
+
+    @VisibleForTesting
+    protected MappedBuffer(ChannelProxy file, int numPageBits)
+    {
+        if (numPageBits > Integer.SIZE - 1)
+            throw new IllegalArgumentException("page size can't be bigger than 1G");
+
+        sizeBits = numPageBits;
+        pageSize = 1 << sizeBits;
+        position = 0;
+        limit = capacity = file.size();
+        pages = new MappedByteBuffer[(int) (file.size() / pageSize) + 1];
+
+        try
+        {
+            long offset = 0;
+            for (int i = 0; i < pages.length; i++)
+            {
+                long pageSize = Math.min(this.pageSize, (capacity - offset));
+                pages[i] = file.map(MapMode.READ_ONLY, offset, pageSize);
+                offset += pageSize;
+            }
+        }
+        finally
+        {
+            file.close();
+        }
+    }
+
+    public int comparePageTo(long offset, int length, AbstractType<?> comparator, ByteBuffer other)
+    {
+        return comparator.compare(getPageRegion(offset, length), other);
+    }
+
+    public long capacity()
+    {
+        return capacity;
+    }
+
+    public long position()
+    {
+        return position;
+    }
+
+    public MappedBuffer position(long newPosition)
+    {
+        if (newPosition < 0 || newPosition > limit)
+            throw new IllegalArgumentException("position: " + newPosition + ", limit: " + limit);
+
+        position = newPosition;
+        return this;
+    }
+
+    public long limit()
+    {
+        return limit;
+    }
+
+    public MappedBuffer limit(long newLimit)
+    {
+        if (newLimit < position || newLimit > capacity)
+            throw new IllegalArgumentException();
+
+        limit = newLimit;
+        return this;
+    }
+
+    public long remaining()
+    {
+        return limit - position;
+    }
+
+    public boolean hasRemaining()
+    {
+        return remaining() > 0;
+    }
+
+    public byte get()
+    {
+        return get(position++);
+    }
+
+    public byte get(long pos)
+    {
+        return pages[getPage(pos)].get(getPageOffset(pos));
+    }
+
+    public short getShort()
+    {
+        short value = getShort(position);
+        position += 2;
+        return value;
+    }
+
+    public short getShort(long pos)
+    {
+        if (isPageAligned(pos, 2))
+            return pages[getPage(pos)].getShort(getPageOffset(pos));
+
+        int ch1 = get(pos)     & 0xff;
+        int ch2 = get(pos + 1) & 0xff;
+        return (short) ((ch1 << 8) + ch2);
+    }
+
+    public int getInt()
+    {
+        int value = getInt(position);
+        position += 4;
+        return value;
+    }
+
+    public int getInt(long pos)
+    {
+        if (isPageAligned(pos, 4))
+            return pages[getPage(pos)].getInt(getPageOffset(pos));
+
+        int ch1 = get(pos)     & 0xff;
+        int ch2 = get(pos + 1) & 0xff;
+        int ch3 = get(pos + 2) & 0xff;
+        int ch4 = get(pos + 3) & 0xff;
+
+        return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4);
+    }
+
+    public long getLong()
+    {
+        long value = getLong(position);
+        position += 8;
+        return value;
+    }
+
+
+    public long getLong(long pos)
+    {
+        // fast path if the long could be retrieved from a single page
+        // that would avoid multiple expensive look-ups into page array.
+        return (isPageAligned(pos, 8))
+                ? pages[getPage(pos)].getLong(getPageOffset(pos))
+                : ((long) (getInt(pos)) << 32) + (getInt(pos + 4) & 0xFFFFFFFFL);
+    }
+
+    public ByteBuffer getPageRegion(long position, int length)
+    {
+        if (!isPageAligned(position, length))
+            throw new IllegalArgumentException(String.format("range: %s-%s wraps more than one page", position, length));
+
+        ByteBuffer slice = pages[getPage(position)].duplicate();
+
+        int pageOffset = getPageOffset(position);
+        slice.position(pageOffset).limit(pageOffset + length);
+
+        return slice;
+    }
+
+    public MappedBuffer duplicate()
+    {
+        return new MappedBuffer(this);
+    }
+
+    public void close()
+    {
+        if (!FileUtils.isCleanerAvailable())
+            return;
+
+        /*
+         * Try forcing the unmapping of pages using undocumented unsafe sun APIs.
+         * If this fails (non Sun JVM), we'll have to wait for the GC to finalize the mapping.
+         * If this works and a thread tries to access any page, hell will unleash on earth.
+         */
+        try
+        {
+            for (MappedByteBuffer segment : pages)
+                FileUtils.clean(segment);
+        }
+        catch (Exception e)
+        {
+            // This is not supposed to happen
+        }
+    }
+
+    private int getPage(long position)
+    {
+        return (int) (position >> sizeBits);
+    }
+
+    private int getPageOffset(long position)
+    {
+        return (int) (position & pageSize - 1);
+    }
+
+    private boolean isPageAligned(long position, int length)
+    {
+        return pageSize - (getPageOffset(position) + length) > 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/OnDiskIndexIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/OnDiskIndexIterator.java b/src/java/org/apache/cassandra/index/sasi/utils/OnDiskIndexIterator.java
new file mode 100644
index 0000000..ae97cab
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/OnDiskIndexIterator.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.index.sasi.utils;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.cassandra.index.sasi.disk.OnDiskIndex;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndex.DataTerm;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public class OnDiskIndexIterator extends RangeIterator<DataTerm, CombinedTerm>
+{
+    private final AbstractType<?> comparator;
+    private final Iterator<DataTerm> terms;
+
+    public OnDiskIndexIterator(OnDiskIndex index)
+    {
+        super(index.min(), index.max(), Long.MAX_VALUE);
+
+        this.comparator = index.getComparator();
+        this.terms = index.iterator();
+    }
+
+    public static RangeIterator<DataTerm, CombinedTerm> union(OnDiskIndex... union)
+    {
+        RangeUnionIterator.Builder<DataTerm, CombinedTerm> builder = RangeUnionIterator.builder();
+        for (OnDiskIndex e : union)
+        {
+            if (e != null)
+                builder.add(new OnDiskIndexIterator(e));
+        }
+
+        return builder.build();
+    }
+
+    protected CombinedTerm computeNext()
+    {
+        return terms.hasNext() ? new CombinedTerm(comparator, terms.next()) : endOfData();
+    }
+
+    protected void performSkipTo(DataTerm nextToken)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public void close() throws IOException
+    {}
+}


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

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java b/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java
new file mode 100644
index 0000000..5d85d00
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java
@@ -0,0 +1,519 @@
+/*
+ * 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.index.sasi.disk;
+
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.index.sasi.utils.AbstractIterator;
+import org.apache.cassandra.index.sasi.utils.CombinedValue;
+import org.apache.cassandra.index.sasi.utils.MappedBuffer;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.utils.MergeIterator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.collect.Iterators;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+import static org.apache.cassandra.index.sasi.disk.TokenTreeBuilder.EntryType;
+
+// Note: all of the seek-able offsets contained in TokenTree should be sizeof(long)
+// even if currently only lower int portion of them if used, because that makes
+// it possible to switch to mmap implementation which supports long positions
+// without any on-disk format changes and/or re-indexing if one day we'll have a need to.
+public class TokenTree
+{
+    private static final int LONG_BYTES = Long.SIZE / 8;
+    private static final int SHORT_BYTES = Short.SIZE / 8;
+
+    private final Descriptor descriptor;
+    private final MappedBuffer file;
+    private final long startPos;
+    private final long treeMinToken;
+    private final long treeMaxToken;
+    private final long tokenCount;
+
+    @VisibleForTesting
+    protected TokenTree(MappedBuffer tokenTree)
+    {
+        this(Descriptor.CURRENT, tokenTree);
+    }
+
+    public TokenTree(Descriptor d, MappedBuffer tokenTree)
+    {
+        descriptor = d;
+        file = tokenTree;
+        startPos = file.position();
+
+        file.position(startPos + TokenTreeBuilder.SHARED_HEADER_BYTES);
+
+        if (!validateMagic())
+            throw new IllegalArgumentException("invalid token tree");
+
+        tokenCount = file.getLong();
+        treeMinToken = file.getLong();
+        treeMaxToken = file.getLong();
+    }
+
+    public long getCount()
+    {
+        return tokenCount;
+    }
+
+    public RangeIterator<Long, Token> iterator(Function<Long, DecoratedKey> keyFetcher)
+    {
+        return new TokenTreeIterator(file.duplicate(), keyFetcher);
+    }
+
+    public OnDiskToken get(final long searchToken, Function<Long, DecoratedKey> keyFetcher)
+    {
+        seekToLeaf(searchToken, file);
+        long leafStart = file.position();
+        short leafSize = file.getShort(leafStart + 1); // skip the info byte
+
+        file.position(leafStart + TokenTreeBuilder.BLOCK_HEADER_BYTES); // skip to tokens
+        short tokenIndex = searchLeaf(searchToken, leafSize);
+
+        file.position(leafStart + TokenTreeBuilder.BLOCK_HEADER_BYTES);
+
+        OnDiskToken token = OnDiskToken.getTokenAt(file, tokenIndex, leafSize, keyFetcher);
+        return token.get().equals(searchToken) ? token : null;
+    }
+
+    private boolean validateMagic()
+    {
+        switch (descriptor.version.toString())
+        {
+            case Descriptor.VERSION_AA:
+                return true;
+            case Descriptor.VERSION_AB:
+                return TokenTreeBuilder.AB_MAGIC == file.getShort();
+            default:
+                return false;
+        }
+    }
+
+    // finds leaf that *could* contain token
+    private void seekToLeaf(long token, MappedBuffer file)
+    {
+        // this loop always seeks forward except for the first iteration
+        // where it may seek back to the root
+        long blockStart = startPos;
+        while (true)
+        {
+            file.position(blockStart);
+
+            byte info = file.get();
+            boolean isLeaf = (info & 1) == 1;
+
+            if (isLeaf)
+            {
+                file.position(blockStart);
+                break;
+            }
+
+            short tokenCount = file.getShort();
+
+            long minToken = file.getLong();
+            long maxToken = file.getLong();
+
+            long seekBase = blockStart + TokenTreeBuilder.BLOCK_HEADER_BYTES;
+            if (minToken > token)
+            {
+                // seek to beginning of child offsets to locate first child
+                file.position(seekBase + tokenCount * LONG_BYTES);
+                blockStart = (startPos + (int) file.getLong());
+            }
+            else if (maxToken < token)
+            {
+                // seek to end of child offsets to locate last child
+                file.position(seekBase + (2 * tokenCount) * LONG_BYTES);
+                blockStart = (startPos + (int) file.getLong());
+            }
+            else
+            {
+                // skip to end of block header/start of interior block tokens
+                file.position(seekBase);
+
+                short offsetIndex = searchBlock(token, tokenCount, file);
+
+                // file pointer is now at beginning of offsets
+                if (offsetIndex == tokenCount)
+                    file.position(file.position() + (offsetIndex * LONG_BYTES));
+                else
+                    file.position(file.position() + ((tokenCount - offsetIndex - 1) + offsetIndex) * LONG_BYTES);
+
+                blockStart = (startPos + (int) file.getLong());
+            }
+        }
+    }
+
+    private short searchBlock(long searchToken, short tokenCount, MappedBuffer file)
+    {
+        short offsetIndex = 0;
+        for (int i = 0; i < tokenCount; i++)
+        {
+            long readToken = file.getLong();
+            if (searchToken < readToken)
+                break;
+
+            offsetIndex++;
+        }
+
+        return offsetIndex;
+    }
+
+    private short searchLeaf(long searchToken, short tokenCount)
+    {
+        long base = file.position();
+
+        int start = 0;
+        int end = tokenCount;
+        int middle = 0;
+
+        while (start <= end)
+        {
+            middle = start + ((end - start) >> 1);
+
+            // each entry is 16 bytes wide, token is in bytes 4-11
+            long token = file.getLong(base + (middle * (2 * LONG_BYTES) + 4));
+
+            if (token == searchToken)
+                break;
+
+            if (token < searchToken)
+                start = middle + 1;
+            else
+                end = middle - 1;
+        }
+
+        return (short) middle;
+    }
+
+    public class TokenTreeIterator extends RangeIterator<Long, Token>
+    {
+        private final Function<Long, DecoratedKey> keyFetcher;
+        private final MappedBuffer file;
+
+        private long currentLeafStart;
+        private int currentTokenIndex;
+
+        private long leafMinToken;
+        private long leafMaxToken;
+        private short leafSize;
+
+        protected boolean firstIteration = true;
+        private boolean lastLeaf;
+
+        TokenTreeIterator(MappedBuffer file, Function<Long, DecoratedKey> keyFetcher)
+        {
+            super(treeMinToken, treeMaxToken, tokenCount);
+
+            this.file = file;
+            this.keyFetcher = keyFetcher;
+        }
+
+        protected Token computeNext()
+        {
+            maybeFirstIteration();
+
+            if (currentTokenIndex >= leafSize && lastLeaf)
+                return endOfData();
+
+            if (currentTokenIndex < leafSize) // tokens remaining in this leaf
+            {
+                return getTokenAt(currentTokenIndex++);
+            }
+            else // no more tokens remaining in this leaf
+            {
+                assert !lastLeaf;
+
+                seekToNextLeaf();
+                setupBlock();
+                return computeNext();
+            }
+        }
+
+        protected void performSkipTo(Long nextToken)
+        {
+            maybeFirstIteration();
+
+            if (nextToken <= leafMaxToken) // next is in this leaf block
+            {
+                searchLeaf(nextToken);
+            }
+            else // next is in a leaf block that needs to be found
+            {
+                seekToLeaf(nextToken, file);
+                setupBlock();
+                findNearest(nextToken);
+            }
+        }
+
+        private void setupBlock()
+        {
+            currentLeafStart = file.position();
+            currentTokenIndex = 0;
+
+            lastLeaf = (file.get() & (1 << TokenTreeBuilder.LAST_LEAF_SHIFT)) > 0;
+            leafSize = file.getShort();
+
+            leafMinToken = file.getLong();
+            leafMaxToken = file.getLong();
+
+            // seek to end of leaf header/start of data
+            file.position(currentLeafStart + TokenTreeBuilder.BLOCK_HEADER_BYTES);
+        }
+
+        private void findNearest(Long next)
+        {
+            if (next > leafMaxToken && !lastLeaf)
+            {
+                seekToNextLeaf();
+                setupBlock();
+                findNearest(next);
+            }
+            else if (next > leafMinToken)
+                searchLeaf(next);
+        }
+
+        private void searchLeaf(long next)
+        {
+            for (int i = currentTokenIndex; i < leafSize; i++)
+            {
+                if (compareTokenAt(currentTokenIndex, next) >= 0)
+                    break;
+
+                currentTokenIndex++;
+            }
+        }
+
+        private int compareTokenAt(int idx, long toToken)
+        {
+            return Long.compare(file.getLong(getTokenPosition(idx)), toToken);
+        }
+
+        private Token getTokenAt(int idx)
+        {
+            return OnDiskToken.getTokenAt(file, idx, leafSize, keyFetcher);
+        }
+
+        private long getTokenPosition(int idx)
+        {
+            // skip 4 byte entry header to get position pointing directly at the entry's token
+            return OnDiskToken.getEntryPosition(idx, file) + (2 * SHORT_BYTES);
+        }
+
+        private void seekToNextLeaf()
+        {
+            file.position(currentLeafStart + TokenTreeBuilder.BLOCK_BYTES);
+        }
+
+        public void close() throws IOException
+        {
+            // nothing to do here
+        }
+
+        private void maybeFirstIteration()
+        {
+            // seek to the first token only when requested for the first time,
+            // highly predictable branch and saves us a lot by not traversing the tree
+            // on creation time because it's not at all required.
+            if (!firstIteration)
+                return;
+
+            seekToLeaf(treeMinToken, file);
+            setupBlock();
+            firstIteration = false;
+        }
+    }
+
+    public static class OnDiskToken extends Token
+    {
+        private final Set<TokenInfo> info = new HashSet<>(2);
+        private final Set<DecoratedKey> loadedKeys = new TreeSet<>(DecoratedKey.comparator);
+
+        public OnDiskToken(MappedBuffer buffer, long position, short leafSize, Function<Long, DecoratedKey> keyFetcher)
+        {
+            super(buffer.getLong(position + (2 * SHORT_BYTES)));
+            info.add(new TokenInfo(buffer, position, leafSize, keyFetcher));
+        }
+
+        public void merge(CombinedValue<Long> other)
+        {
+            if (!(other instanceof Token))
+                return;
+
+            Token o = (Token) other;
+            if (token != o.token)
+                throw new IllegalArgumentException(String.format("%s != %s", token, o.token));
+
+            if (o instanceof OnDiskToken)
+            {
+                info.addAll(((OnDiskToken) other).info);
+            }
+            else
+            {
+                Iterators.addAll(loadedKeys, o.iterator());
+            }
+        }
+
+        public Iterator<DecoratedKey> iterator()
+        {
+            List<Iterator<DecoratedKey>> keys = new ArrayList<>(info.size());
+
+            for (TokenInfo i : info)
+                keys.add(i.iterator());
+
+            if (!loadedKeys.isEmpty())
+                keys.add(loadedKeys.iterator());
+
+            return MergeIterator.get(keys, DecoratedKey.comparator, new MergeIterator.Reducer<DecoratedKey, DecoratedKey>()
+            {
+                DecoratedKey reduced = null;
+
+                public boolean trivialReduceIsTrivial()
+                {
+                    return true;
+                }
+
+                public void reduce(int idx, DecoratedKey current)
+                {
+                    reduced = current;
+                }
+
+                protected DecoratedKey getReduced()
+                {
+                    return reduced;
+                }
+            });
+        }
+
+        public Set<Long> getOffsets()
+        {
+            Set<Long> offsets = new HashSet<>();
+            for (TokenInfo i : info)
+            {
+                for (long offset : i.fetchOffsets())
+                    offsets.add(offset);
+            }
+
+            return offsets;
+        }
+
+        public static OnDiskToken getTokenAt(MappedBuffer buffer, int idx, short leafSize, Function<Long, DecoratedKey> keyFetcher)
+        {
+            return new OnDiskToken(buffer, getEntryPosition(idx, buffer), leafSize, keyFetcher);
+        }
+
+        private static long getEntryPosition(int idx, MappedBuffer file)
+        {
+            // info (4 bytes) + token (8 bytes) + offset (4 bytes) = 16 bytes
+            return file.position() + (idx * (2 * LONG_BYTES));
+        }
+    }
+
+    private static class TokenInfo
+    {
+        private final MappedBuffer buffer;
+        private final Function<Long, DecoratedKey> keyFetcher;
+
+        private final long position;
+        private final short leafSize;
+
+        public TokenInfo(MappedBuffer buffer, long position, short leafSize, Function<Long, DecoratedKey> keyFetcher)
+        {
+            this.keyFetcher = keyFetcher;
+            this.buffer = buffer;
+            this.position = position;
+            this.leafSize = leafSize;
+        }
+
+        public Iterator<DecoratedKey> iterator()
+        {
+            return new KeyIterator(keyFetcher, fetchOffsets());
+        }
+
+        public int hashCode()
+        {
+            return new HashCodeBuilder().append(keyFetcher).append(position).append(leafSize).build();
+        }
+
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof TokenInfo))
+                return false;
+
+            TokenInfo o = (TokenInfo) other;
+            return keyFetcher == o.keyFetcher && position == o.position;
+        }
+
+        private long[] fetchOffsets()
+        {
+            short info = buffer.getShort(position);
+            short offsetShort = buffer.getShort(position + SHORT_BYTES);
+            int offsetInt = buffer.getInt(position + (2 * SHORT_BYTES) + LONG_BYTES);
+
+            EntryType type = EntryType.of(info & TokenTreeBuilder.ENTRY_TYPE_MASK);
+
+            switch (type)
+            {
+                case SIMPLE:
+                    return new long[] { offsetInt };
+
+                case OVERFLOW:
+                    long[] offsets = new long[offsetShort]; // offsetShort contains count of tokens
+                    long offsetPos = (buffer.position() + (2 * (leafSize * LONG_BYTES)) + (offsetInt * LONG_BYTES));
+
+                    for (int i = 0; i < offsetShort; i++)
+                        offsets[i] = buffer.getLong(offsetPos + (i * LONG_BYTES));
+
+                    return offsets;
+
+                case FACTORED:
+                    return new long[] { (((long) offsetInt) << Short.SIZE) + offsetShort };
+
+                case PACKED:
+                    return new long[] { offsetShort, offsetInt };
+
+                default:
+                    throw new IllegalStateException("Unknown entry type: " + type);
+            }
+        }
+    }
+
+    private static class KeyIterator extends AbstractIterator<DecoratedKey>
+    {
+        private final Function<Long, DecoratedKey> keyFetcher;
+        private final long[] offsets;
+        private int index = 0;
+
+        public KeyIterator(Function<Long, DecoratedKey> keyFetcher, long[] offsets)
+        {
+            this.keyFetcher = keyFetcher;
+            this.offsets = offsets;
+        }
+
+        public DecoratedKey computeNext()
+        {
+            return index < offsets.length ? keyFetcher.apply(offsets[index++]) : endOfData();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/disk/TokenTreeBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/TokenTreeBuilder.java b/src/java/org/apache/cassandra/index/sasi/disk/TokenTreeBuilder.java
new file mode 100644
index 0000000..e10b057
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/disk/TokenTreeBuilder.java
@@ -0,0 +1,839 @@
+/*
+ * 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.index.sasi.disk;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+
+import com.carrotsearch.hppc.LongArrayList;
+import com.carrotsearch.hppc.LongSet;
+import com.carrotsearch.hppc.cursors.LongCursor;
+import com.carrotsearch.hppc.LongOpenHashSet;
+import com.google.common.collect.AbstractIterator;
+
+public class TokenTreeBuilder
+{
+    // note: ordinal positions are used here, do not change order
+    enum EntryType
+    {
+        SIMPLE, FACTORED, PACKED, OVERFLOW;
+
+        public static EntryType of(int ordinal)
+        {
+            if (ordinal == SIMPLE.ordinal())
+                return SIMPLE;
+
+            if (ordinal == FACTORED.ordinal())
+                return FACTORED;
+
+            if (ordinal == PACKED.ordinal())
+                return PACKED;
+
+            if (ordinal == OVERFLOW.ordinal())
+                return OVERFLOW;
+
+            throw new IllegalArgumentException("Unknown ordinal: " + ordinal);
+        }
+    }
+
+    public static final int BLOCK_BYTES = 4096;
+    public static final int BLOCK_HEADER_BYTES = 64;
+    public static final int OVERFLOW_TRAILER_BYTES = 64;
+    public static final int OVERFLOW_TRAILER_CAPACITY = OVERFLOW_TRAILER_BYTES / 8;
+    public static final int TOKENS_PER_BLOCK = (BLOCK_BYTES - BLOCK_HEADER_BYTES - OVERFLOW_TRAILER_BYTES) / 16;
+    public static final long MAX_OFFSET = (1L << 47) - 1; // 48 bits for (signed) offset
+    public static final byte LAST_LEAF_SHIFT = 1;
+    public static final byte SHARED_HEADER_BYTES = 19;
+    public static final byte ENTRY_TYPE_MASK = 0x03;
+    public static final short AB_MAGIC = 0x5A51;
+
+    private final SortedMap<Long, LongSet> tokens = new TreeMap<>();
+    private int numBlocks;
+
+    private Node root;
+    private InteriorNode rightmostParent;
+    private Leaf leftmostLeaf;
+    private Leaf rightmostLeaf;
+    private long tokenCount = 0;
+    private long treeMinToken;
+    private long treeMaxToken;
+
+    public TokenTreeBuilder()
+    {}
+
+    public TokenTreeBuilder(SortedMap<Long, LongSet> data)
+    {
+        add(data);
+    }
+
+    public void add(Long token, long keyPosition)
+    {
+        LongSet found = tokens.get(token);
+        if (found == null)
+            tokens.put(token, (found = new LongOpenHashSet(2)));
+
+        found.add(keyPosition);
+    }
+
+    public void add(SortedMap<Long, LongSet> data)
+    {
+        for (Map.Entry<Long, LongSet> newEntry : data.entrySet())
+        {
+            LongSet found = tokens.get(newEntry.getKey());
+            if (found == null)
+                tokens.put(newEntry.getKey(), (found = new LongOpenHashSet(4)));
+
+            for (LongCursor offset : newEntry.getValue())
+                found.add(offset.value);
+        }
+    }
+
+    public TokenTreeBuilder finish()
+    {
+        maybeBulkLoad();
+        return this;
+    }
+
+    public SortedMap<Long, LongSet> getTokens()
+    {
+        return tokens;
+    }
+
+    public long getTokenCount()
+    {
+        return tokenCount;
+    }
+
+    public int serializedSize()
+    {
+        if (numBlocks == 1)
+            return (BLOCK_HEADER_BYTES + ((int) tokenCount * 16));
+        else
+            return numBlocks * BLOCK_BYTES;
+    }
+
+    public void write(DataOutputPlus out) throws IOException
+    {
+        ByteBuffer blockBuffer = ByteBuffer.allocate(BLOCK_BYTES);
+        Iterator<Node> levelIterator = root.levelIterator();
+        long childBlockIndex = 1;
+
+        while (levelIterator != null)
+        {
+
+            Node firstChild = null;
+            while (levelIterator.hasNext())
+            {
+                Node block = levelIterator.next();
+
+                if (firstChild == null && !block.isLeaf())
+                    firstChild = ((InteriorNode) block).children.get(0);
+
+                block.serialize(childBlockIndex, blockBuffer);
+                flushBuffer(blockBuffer, out, numBlocks != 1);
+
+                childBlockIndex += block.childCount();
+            }
+
+            levelIterator = (firstChild == null) ? null : firstChild.levelIterator();
+        }
+    }
+
+    public Iterator<Pair<Long, LongSet>> iterator()
+    {
+        return new TokenIterator(leftmostLeaf.levelIterator());
+    }
+
+    private void maybeBulkLoad()
+    {
+        if (root == null)
+            bulkLoad();
+    }
+
+    private void flushBuffer(ByteBuffer buffer, DataOutputPlus o, boolean align) throws IOException
+    {
+        // seek to end of last block before flushing
+        if (align)
+            alignBuffer(buffer, BLOCK_BYTES);
+
+        buffer.flip();
+        o.write(buffer);
+        buffer.clear();
+    }
+
+    private static void alignBuffer(ByteBuffer buffer, int blockSize)
+    {
+        long curPos = buffer.position();
+        if ((curPos & (blockSize - 1)) != 0) // align on the block boundary if needed
+            buffer.position((int) FBUtilities.align(curPos, blockSize));
+    }
+
+    private void bulkLoad()
+    {
+        tokenCount = tokens.size();
+        treeMinToken = tokens.firstKey();
+        treeMaxToken = tokens.lastKey();
+        numBlocks = 1;
+
+        // special case the tree that only has a single block in it (so we don't create a useless root)
+        if (tokenCount <= TOKENS_PER_BLOCK)
+        {
+            leftmostLeaf = new Leaf(tokens);
+            rightmostLeaf = leftmostLeaf;
+            root = leftmostLeaf;
+        }
+        else
+        {
+            root = new InteriorNode();
+            rightmostParent = (InteriorNode) root;
+
+            int i = 0;
+            Leaf lastLeaf = null;
+            Long firstToken = tokens.firstKey();
+            Long finalToken = tokens.lastKey();
+            Long lastToken;
+            for (Long token : tokens.keySet())
+            {
+                if (i == 0 || (i % TOKENS_PER_BLOCK != 0 && i != (tokenCount - 1)))
+                {
+                    i++;
+                    continue;
+                }
+
+                lastToken = token;
+                Leaf leaf = (i != (tokenCount - 1) || token.equals(finalToken)) ?
+                        new Leaf(tokens.subMap(firstToken, lastToken)) : new Leaf(tokens.tailMap(firstToken));
+
+                if (i == TOKENS_PER_BLOCK)
+                    leftmostLeaf = leaf;
+                else
+                    lastLeaf.next = leaf;
+
+                rightmostParent.add(leaf);
+                lastLeaf = leaf;
+                rightmostLeaf = leaf;
+                firstToken = lastToken;
+                i++;
+                numBlocks++;
+
+                if (token.equals(finalToken))
+                {
+                    Leaf finalLeaf = new Leaf(tokens.tailMap(token));
+                    lastLeaf.next = finalLeaf;
+                    rightmostParent.add(finalLeaf);
+                    rightmostLeaf = finalLeaf;
+                    numBlocks++;
+                }
+            }
+
+        }
+    }
+
+    private abstract class Node
+    {
+        protected InteriorNode parent;
+        protected Node next;
+        protected Long nodeMinToken, nodeMaxToken;
+
+        public abstract void serialize(long childBlockIndex, ByteBuffer buf);
+        public abstract int childCount();
+        public abstract int tokenCount();
+        public abstract Long smallestToken();
+
+        public Iterator<Node> levelIterator()
+        {
+            return new LevelIterator(this);
+        }
+
+        public boolean isLeaf()
+        {
+            return (this instanceof Leaf);
+        }
+
+        protected boolean isLastLeaf()
+        {
+            return this == rightmostLeaf;
+        }
+
+        protected boolean isRoot()
+        {
+            return this == root;
+        }
+
+        protected void updateTokenRange(long token)
+        {
+            nodeMinToken = nodeMinToken == null ? token : Math.min(nodeMinToken, token);
+            nodeMaxToken = nodeMaxToken == null ? token : Math.max(nodeMaxToken, token);
+        }
+
+        protected void serializeHeader(ByteBuffer buf)
+        {
+            Header header;
+            if (isRoot())
+                header = new RootHeader();
+            else if (!isLeaf())
+                header = new InteriorNodeHeader();
+            else
+                header = new LeafHeader();
+
+            header.serialize(buf);
+            alignBuffer(buf, BLOCK_HEADER_BYTES);
+        }
+
+        private abstract class Header
+        {
+            public void serialize(ByteBuffer buf)
+            {
+                buf.put(infoByte())
+                        .putShort((short) (tokenCount()))
+                        .putLong(nodeMinToken)
+                        .putLong(nodeMaxToken);
+            }
+
+            protected abstract byte infoByte();
+        }
+
+        private class RootHeader extends Header
+        {
+            public void serialize(ByteBuffer buf)
+            {
+                super.serialize(buf);
+                writeMagic(buf);
+                buf.putLong(tokenCount)
+                        .putLong(treeMinToken)
+                        .putLong(treeMaxToken);
+            }
+
+            protected byte infoByte()
+            {
+                // if leaf, set leaf indicator and last leaf indicator (bits 0 & 1)
+                // if not leaf, clear both bits
+                return (byte) ((isLeaf()) ? 3 : 0);
+            }
+
+            protected void writeMagic(ByteBuffer buf)
+            {
+                switch (Descriptor.CURRENT_VERSION)
+                {
+                    case Descriptor.VERSION_AB:
+                        buf.putShort(AB_MAGIC);
+                        break;
+                    default:
+                        break;
+                }
+
+            }
+        }
+
+        private class InteriorNodeHeader extends Header
+        {
+            // bit 0 (leaf indicator) & bit 1 (last leaf indicator) cleared
+            protected byte infoByte()
+            {
+                return 0;
+            }
+        }
+
+        private class LeafHeader extends Header
+        {
+            // bit 0 set as leaf indicator
+            // bit 1 set if this is last leaf of data
+            protected byte infoByte()
+            {
+                byte infoByte = 1;
+                infoByte |= (isLastLeaf()) ? (1 << LAST_LEAF_SHIFT) : 0;
+
+                return infoByte;
+            }
+        }
+
+    }
+
+    private class Leaf extends Node
+    {
+        private final SortedMap<Long, LongSet> tokens;
+        private LongArrayList overflowCollisions;
+
+        Leaf(SortedMap<Long, LongSet> data)
+        {
+            nodeMinToken = data.firstKey();
+            nodeMaxToken = data.lastKey();
+            tokens = data;
+        }
+
+        public Long largestToken()
+        {
+            return nodeMaxToken;
+        }
+
+        public void serialize(long childBlockIndex, ByteBuffer buf)
+        {
+            serializeHeader(buf);
+            serializeData(buf);
+            serializeOverflowCollisions(buf);
+        }
+
+        public int childCount()
+        {
+            return 0;
+        }
+
+        public int tokenCount()
+        {
+            return tokens.size();
+        }
+
+        public Long smallestToken()
+        {
+            return nodeMinToken;
+        }
+
+        public Iterator<Map.Entry<Long, LongSet>> tokenIterator()
+        {
+            return tokens.entrySet().iterator();
+        }
+
+        private void serializeData(ByteBuffer buf)
+        {
+            for (Map.Entry<Long, LongSet> entry : tokens.entrySet())
+                createEntry(entry.getKey(), entry.getValue()).serialize(buf);
+        }
+
+        private void serializeOverflowCollisions(ByteBuffer buf)
+        {
+            if (overflowCollisions != null)
+                for (LongCursor offset : overflowCollisions)
+                    buf.putLong(offset.value);
+        }
+
+
+        private LeafEntry createEntry(final long tok, final LongSet offsets)
+        {
+            int offsetCount = offsets.size();
+            switch (offsetCount)
+            {
+                case 0:
+                    throw new AssertionError("no offsets for token " + tok);
+                case 1:
+                    long offset = offsets.toArray()[0];
+                    if (offset > MAX_OFFSET)
+                        throw new AssertionError("offset " + offset + " cannot be greater than " + MAX_OFFSET);
+                    else if (offset <= Integer.MAX_VALUE)
+                        return new SimpleLeafEntry(tok, offset);
+                    else
+                        return new FactoredOffsetLeafEntry(tok, offset);
+                case 2:
+                    long[] rawOffsets = offsets.toArray();
+                    if (rawOffsets[0] <= Integer.MAX_VALUE && rawOffsets[1] <= Integer.MAX_VALUE &&
+                            (rawOffsets[0] <= Short.MAX_VALUE || rawOffsets[1] <= Short.MAX_VALUE))
+                        return new PackedCollisionLeafEntry(tok, rawOffsets);
+                    else
+                        return createOverflowEntry(tok, offsetCount, offsets);
+                default:
+                    return createOverflowEntry(tok, offsetCount, offsets);
+            }
+        }
+
+        private LeafEntry createOverflowEntry(final long tok, final int offsetCount, final LongSet offsets)
+        {
+            if (overflowCollisions == null)
+                overflowCollisions = new LongArrayList();
+
+            LeafEntry entry = new OverflowCollisionLeafEntry(tok, (short) overflowCollisions.size(), (short) offsetCount);
+            for (LongCursor o : offsets) {
+                if (overflowCollisions.size() == OVERFLOW_TRAILER_CAPACITY)
+                    throw new AssertionError("cannot have more than " + OVERFLOW_TRAILER_CAPACITY + " overflow collisions per leaf");
+                else
+                    overflowCollisions.add(o.value);
+            }
+            return entry;
+        }
+
+        private abstract class LeafEntry
+        {
+            protected final long token;
+
+            abstract public EntryType type();
+            abstract public int offsetData();
+            abstract public short offsetExtra();
+
+            public LeafEntry(final long tok)
+            {
+                token = tok;
+            }
+
+            public void serialize(ByteBuffer buf)
+            {
+                buf.putShort((short) type().ordinal())
+                        .putShort(offsetExtra())
+                        .putLong(token)
+                        .putInt(offsetData());
+            }
+
+        }
+
+
+        // assumes there is a single offset and the offset is <= Integer.MAX_VALUE
+        private class SimpleLeafEntry extends LeafEntry
+        {
+            private final long offset;
+
+            public SimpleLeafEntry(final long tok, final long off)
+            {
+                super(tok);
+                offset = off;
+            }
+
+            public EntryType type()
+            {
+                return EntryType.SIMPLE;
+            }
+
+            public int offsetData()
+            {
+                return (int) offset;
+            }
+
+            public short offsetExtra()
+            {
+                return 0;
+            }
+        }
+
+        // assumes there is a single offset and Integer.MAX_VALUE < offset <= MAX_OFFSET
+        // take the middle 32 bits of offset (or the top 32 when considering offset is max 48 bits)
+        // and store where offset is normally stored. take bottom 16 bits of offset and store in entry header
+        private class FactoredOffsetLeafEntry extends LeafEntry
+        {
+            private final long offset;
+
+            public FactoredOffsetLeafEntry(final long tok, final long off)
+            {
+                super(tok);
+                offset = off;
+            }
+
+            public EntryType type()
+            {
+                return EntryType.FACTORED;
+            }
+
+            public int offsetData()
+            {
+                return (int) (offset >>> Short.SIZE);
+            }
+
+            public short offsetExtra()
+            {
+                return (short) offset;
+            }
+        }
+
+        // holds an entry with two offsets that can be packed in an int & a short
+        // the int offset is stored where offset is normally stored. short offset is
+        // stored in entry header
+        private class PackedCollisionLeafEntry extends LeafEntry
+        {
+            private short smallerOffset;
+            private int largerOffset;
+
+            public PackedCollisionLeafEntry(final long tok, final long[] offs)
+            {
+                super(tok);
+
+                smallerOffset = (short) Math.min(offs[0], offs[1]);
+                largerOffset = (int) Math.max(offs[0], offs[1]);
+            }
+
+            public EntryType type()
+            {
+                return EntryType.PACKED;
+            }
+
+            public int offsetData()
+            {
+                return largerOffset;
+            }
+
+            public short offsetExtra()
+            {
+                return smallerOffset;
+            }
+        }
+
+        // holds an entry with three or more offsets, or two offsets that cannot
+        // be packed into an int & a short. the index into the overflow list
+        // is stored where the offset is normally stored. the number of overflowed offsets
+        // for the entry is stored in the entry header
+        private class OverflowCollisionLeafEntry extends LeafEntry
+        {
+            private final short startIndex;
+            private final short count;
+
+            public OverflowCollisionLeafEntry(final long tok, final short collisionStartIndex, final short collisionCount)
+            {
+                super(tok);
+                startIndex = collisionStartIndex;
+                count = collisionCount;
+            }
+
+            public EntryType type()
+            {
+                return EntryType.OVERFLOW;
+            }
+
+            public int offsetData()
+            {
+                return startIndex;
+            }
+
+            public short offsetExtra()
+            {
+                return count;
+            }
+
+        }
+
+    }
+
+    private class InteriorNode extends Node
+    {
+        private List<Long> tokens = new ArrayList<>(TOKENS_PER_BLOCK);
+        private List<Node> children = new ArrayList<>(TOKENS_PER_BLOCK + 1);
+        private int position = 0; // TODO (jwest): can get rid of this and use array size
+
+
+        public void serialize(long childBlockIndex, ByteBuffer buf)
+        {
+            serializeHeader(buf);
+            serializeTokens(buf);
+            serializeChildOffsets(childBlockIndex, buf);
+        }
+
+        public int childCount()
+        {
+            return children.size();
+        }
+
+        public int tokenCount()
+        {
+            return tokens.size();
+        }
+
+        public Long smallestToken()
+        {
+            return tokens.get(0);
+        }
+
+        protected void add(Long token, InteriorNode leftChild, InteriorNode rightChild)
+        {
+            int pos = tokens.size();
+            if (pos == TOKENS_PER_BLOCK)
+            {
+                InteriorNode sibling = split();
+                sibling.add(token, leftChild, rightChild);
+
+            }
+            else {
+                if (leftChild != null)
+                    children.add(pos, leftChild);
+
+                if (rightChild != null)
+                {
+                    children.add(pos + 1, rightChild);
+                    rightChild.parent = this;
+                }
+
+                updateTokenRange(token);
+                tokens.add(pos, token);
+            }
+        }
+
+        protected void add(Leaf node)
+        {
+
+            if (position == (TOKENS_PER_BLOCK + 1))
+            {
+                rightmostParent = split();
+                rightmostParent.add(node);
+            }
+            else
+            {
+
+                node.parent = this;
+                children.add(position, node);
+                position++;
+
+                // the first child is referenced only during bulk load. we don't take a value
+                // to store into the tree, one is subtracted since position has already been incremented
+                // for the next node to be added
+                if (position - 1 == 0)
+                    return;
+
+
+                // tokens are inserted one behind the current position, but 2 is subtracted because
+                // position has already been incremented for the next add
+                Long smallestToken = node.smallestToken();
+                updateTokenRange(smallestToken);
+                tokens.add(position - 2, smallestToken);
+            }
+
+        }
+
+        protected InteriorNode split()
+        {
+            Pair<Long, InteriorNode> splitResult = splitBlock();
+            Long middleValue = splitResult.left;
+            InteriorNode sibling = splitResult.right;
+            InteriorNode leftChild = null;
+
+            // create a new root if necessary
+            if (parent == null)
+            {
+                parent = new InteriorNode();
+                root = parent;
+                sibling.parent = parent;
+                leftChild = this;
+                numBlocks++;
+            }
+
+            parent.add(middleValue, leftChild, sibling);
+
+            return sibling;
+        }
+
+        protected Pair<Long, InteriorNode> splitBlock()
+        {
+            final int splitPosition = TOKENS_PER_BLOCK - 2;
+            InteriorNode sibling = new InteriorNode();
+            sibling.parent = parent;
+            next = sibling;
+
+            Long middleValue = tokens.get(splitPosition);
+
+            for (int i = splitPosition; i < TOKENS_PER_BLOCK; i++)
+            {
+                if (i != TOKENS_PER_BLOCK && i != splitPosition)
+                {
+                    long token = tokens.get(i);
+                    sibling.updateTokenRange(token);
+                    sibling.tokens.add(token);
+                }
+
+                Node child = children.get(i + 1);
+                child.parent = sibling;
+                sibling.children.add(child);
+                sibling.position++;
+            }
+
+            for (int i = TOKENS_PER_BLOCK; i >= splitPosition; i--)
+            {
+                if (i != TOKENS_PER_BLOCK)
+                    tokens.remove(i);
+
+                if (i != splitPosition)
+                    children.remove(i);
+            }
+
+            nodeMinToken = smallestToken();
+            nodeMaxToken = tokens.get(tokens.size() - 1);
+            numBlocks++;
+
+            return Pair.create(middleValue, sibling);
+        }
+
+        protected boolean isFull()
+        {
+            return (position >= TOKENS_PER_BLOCK + 1);
+        }
+
+        private void serializeTokens(ByteBuffer buf)
+        {
+            for (Long token : tokens)
+                buf.putLong(token);
+        }
+
+
+        private void serializeChildOffsets(long childBlockIndex, ByteBuffer buf)
+        {
+            for (int i = 0; i < children.size(); i++)
+                buf.putLong((childBlockIndex + i) * BLOCK_BYTES);
+        }
+    }
+
+    public static class LevelIterator extends AbstractIterator<Node>
+    {
+        private Node currentNode;
+
+        LevelIterator(Node first)
+        {
+            currentNode = first;
+        }
+
+        public Node computeNext()
+        {
+            if (currentNode == null)
+                return endOfData();
+
+            Node returnNode = currentNode;
+            currentNode = returnNode.next;
+
+            return returnNode;
+        }
+
+
+    }
+
+    public static class TokenIterator extends AbstractIterator<Pair<Long, LongSet>>
+    {
+        private Iterator<Node> levelIterator;
+        private Iterator<Map.Entry<Long, LongSet>> currentIterator;
+
+        TokenIterator(Iterator<Node> level)
+        {
+            levelIterator = level;
+            if (levelIterator.hasNext())
+                currentIterator = ((Leaf) levelIterator.next()).tokenIterator();
+        }
+
+        public Pair<Long, LongSet> computeNext()
+        {
+            if (currentIterator != null && currentIterator.hasNext())
+            {
+                Map.Entry<Long, LongSet> next = currentIterator.next();
+                return Pair.create(next.getKey(), next.getValue());
+            }
+            else
+            {
+                if (!levelIterator.hasNext())
+                    return endOfData();
+                else
+                {
+                    currentIterator = ((Leaf) levelIterator.next()).tokenIterator();
+                    return computeNext();
+                }
+            }
+
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/exceptions/TimeQuotaExceededException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/exceptions/TimeQuotaExceededException.java b/src/java/org/apache/cassandra/index/sasi/exceptions/TimeQuotaExceededException.java
new file mode 100644
index 0000000..af577dc
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/exceptions/TimeQuotaExceededException.java
@@ -0,0 +1,21 @@
+/*
+ * 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.index.sasi.exceptions;
+
+public class TimeQuotaExceededException extends RuntimeException
+{}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/memory/IndexMemtable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/memory/IndexMemtable.java b/src/java/org/apache/cassandra/index/sasi/memory/IndexMemtable.java
new file mode 100644
index 0000000..cf7f3a5
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/memory/IndexMemtable.java
@@ -0,0 +1,71 @@
+/*
+ * 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.index.sasi.memory;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.index.sasi.utils.TypeUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class IndexMemtable
+{
+    private static final Logger logger = LoggerFactory.getLogger(IndexMemtable.class);
+
+    private final MemIndex index;
+
+    public IndexMemtable(ColumnIndex columnIndex)
+    {
+        this.index = MemIndex.forColumn(columnIndex.keyValidator(), columnIndex);
+    }
+
+    public long index(DecoratedKey key, ByteBuffer value)
+    {
+        if (value == null || value.remaining() == 0)
+            return 0;
+
+        AbstractType<?> validator = index.columnIndex.getValidator();
+        if (!TypeUtil.isValid(value, validator))
+        {
+            int size = value.remaining();
+            if ((value = TypeUtil.tryUpcast(value, validator)) == null)
+            {
+                logger.error("Can't add column {} to index for key: {}, value size {} bytes, validator: {}.",
+                             index.columnIndex.getColumnName(),
+                             index.columnIndex.keyValidator().getString(key.getKey()),
+                             size,
+                             validator);
+                return 0;
+            }
+        }
+
+        return index.add(key, value);
+    }
+
+    public RangeIterator<Long, Token> search(Expression expression)
+    {
+        return index == null ? null : index.search(expression);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/memory/KeyRangeIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/memory/KeyRangeIterator.java b/src/java/org/apache/cassandra/index/sasi/memory/KeyRangeIterator.java
new file mode 100644
index 0000000..293e2ee
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/memory/KeyRangeIterator.java
@@ -0,0 +1,118 @@
+/*
+ * 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.index.sasi.memory;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentSkipListSet;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.utils.AbstractIterator;
+import org.apache.cassandra.index.sasi.utils.CombinedValue;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+
+import com.google.common.collect.PeekingIterator;
+
+public class KeyRangeIterator extends RangeIterator<Long, Token>
+{
+    private final DKIterator iterator;
+
+    public KeyRangeIterator(ConcurrentSkipListSet<DecoratedKey> keys)
+    {
+        super((Long) keys.first().getToken().getTokenValue(), (Long) keys.last().getToken().getTokenValue(), keys.size());
+        this.iterator = new DKIterator(keys.iterator());
+    }
+
+    protected Token computeNext()
+    {
+        return iterator.hasNext() ? new DKToken(iterator.next()) : endOfData();
+    }
+
+    protected void performSkipTo(Long nextToken)
+    {
+        while (iterator.hasNext())
+        {
+            DecoratedKey key = iterator.peek();
+            if (Long.compare((long) key.getToken().getTokenValue(), nextToken) >= 0)
+                break;
+
+            // consume smaller key
+            iterator.next();
+        }
+    }
+
+    public void close() throws IOException
+    {}
+
+    private static class DKIterator extends AbstractIterator<DecoratedKey> implements PeekingIterator<DecoratedKey>
+    {
+        private final Iterator<DecoratedKey> keys;
+
+        public DKIterator(Iterator<DecoratedKey> keys)
+        {
+            this.keys = keys;
+        }
+
+        protected DecoratedKey computeNext()
+        {
+            return keys.hasNext() ? keys.next() : endOfData();
+        }
+    }
+
+    private static class DKToken extends Token
+    {
+        private final SortedSet<DecoratedKey> keys;
+
+        public DKToken(final DecoratedKey key)
+        {
+            super((long) key.getToken().getTokenValue());
+
+            keys = new TreeSet<DecoratedKey>(DecoratedKey.comparator)
+            {{
+                add(key);
+            }};
+        }
+
+        public void merge(CombinedValue<Long> other)
+        {
+            if (!(other instanceof Token))
+                return;
+
+            Token o = (Token) other;
+            assert o.get().equals(token);
+
+            if (o instanceof DKToken)
+            {
+                keys.addAll(((DKToken) o).keys);
+            }
+            else
+            {
+                for (DecoratedKey key : o)
+                    keys.add(key);
+            }
+        }
+
+        public Iterator<DecoratedKey> iterator()
+        {
+            return keys.iterator();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/memory/MemIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/memory/MemIndex.java b/src/java/org/apache/cassandra/index/sasi/memory/MemIndex.java
new file mode 100644
index 0000000..22d6c9e
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/memory/MemIndex.java
@@ -0,0 +1,51 @@
+/*
+ * 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.index.sasi.memory;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+import org.github.jamm.MemoryMeter;
+
+public abstract class MemIndex
+{
+    protected final AbstractType<?> keyValidator;
+    protected final ColumnIndex columnIndex;
+
+    protected MemIndex(AbstractType<?> keyValidator, ColumnIndex columnIndex)
+    {
+        this.keyValidator = keyValidator;
+        this.columnIndex = columnIndex;
+    }
+
+    public abstract long add(DecoratedKey key, ByteBuffer value);
+    public abstract RangeIterator<Long, Token> search(Expression expression);
+
+    public static MemIndex forColumn(AbstractType<?> keyValidator, ColumnIndex columnIndex)
+    {
+        return columnIndex.isLiteral()
+                ? new TrieMemIndex(keyValidator, columnIndex)
+                : new SkipListMemIndex(keyValidator, columnIndex);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/memory/SkipListMemIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/memory/SkipListMemIndex.java b/src/java/org/apache/cassandra/index/sasi/memory/SkipListMemIndex.java
new file mode 100644
index 0000000..69b57d0
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/memory/SkipListMemIndex.java
@@ -0,0 +1,97 @@
+/*
+ * 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.index.sasi.memory;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.index.sasi.utils.RangeUnionIterator;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public class SkipListMemIndex extends MemIndex
+{
+    public static final int CSLM_OVERHEAD = 128; // average overhead of CSLM
+
+    private final ConcurrentSkipListMap<ByteBuffer, ConcurrentSkipListSet<DecoratedKey>> index;
+
+    public SkipListMemIndex(AbstractType<?> keyValidator, ColumnIndex columnIndex)
+    {
+        super(keyValidator, columnIndex);
+        index = new ConcurrentSkipListMap<>(columnIndex.getValidator());
+    }
+
+    public long add(DecoratedKey key, ByteBuffer value)
+    {
+        long overhead = CSLM_OVERHEAD; // DKs are shared
+        ConcurrentSkipListSet<DecoratedKey> keys = index.get(value);
+
+        if (keys == null)
+        {
+            ConcurrentSkipListSet<DecoratedKey> newKeys = new ConcurrentSkipListSet<>(DecoratedKey.comparator);
+            keys = index.putIfAbsent(value, newKeys);
+            if (keys == null)
+            {
+                overhead += CSLM_OVERHEAD + value.remaining();
+                keys = newKeys;
+            }
+        }
+
+        keys.add(key);
+
+        return overhead;
+    }
+
+    public RangeIterator<Long, Token> search(Expression expression)
+    {
+        ByteBuffer min = expression.lower == null ? null : expression.lower.value;
+        ByteBuffer max = expression.upper == null ? null : expression.upper.value;
+
+        SortedMap<ByteBuffer, ConcurrentSkipListSet<DecoratedKey>> search;
+
+        if (min == null && max == null)
+        {
+            throw new IllegalArgumentException();
+        }
+        if (min != null && max != null)
+        {
+            search = index.subMap(min, expression.lower.inclusive, max, expression.upper.inclusive);
+        }
+        else if (min == null)
+        {
+            search = index.headMap(max, expression.upper.inclusive);
+        }
+        else
+        {
+            search = index.tailMap(min, expression.lower.inclusive);
+        }
+
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+        search.values().stream()
+                       .filter(keys -> !keys.isEmpty())
+                       .forEach(keys -> builder.add(new KeyRangeIterator(keys)));
+
+        return builder.build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java b/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java
new file mode 100644
index 0000000..e4ee6eb
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java
@@ -0,0 +1,254 @@
+/*
+ * 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.index.sasi.memory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.ConcurrentSkipListSet;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.index.sasi.analyzer.AbstractAnalyzer;
+import org.apache.cassandra.index.sasi.utils.RangeUnionIterator;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+import com.googlecode.concurrenttrees.radix.ConcurrentRadixTree;
+import com.googlecode.concurrenttrees.suffix.ConcurrentSuffixTree;
+import com.googlecode.concurrenttrees.radix.node.concrete.SmartArrayBasedNodeFactory;
+import com.googlecode.concurrenttrees.radix.node.Node;
+
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.cassandra.index.sasi.memory.SkipListMemIndex.CSLM_OVERHEAD;
+
+public class TrieMemIndex extends MemIndex
+{
+    private static final Logger logger = LoggerFactory.getLogger(TrieMemIndex.class);
+
+    private final ConcurrentTrie index;
+
+    public TrieMemIndex(AbstractType<?> keyValidator, ColumnIndex columnIndex)
+    {
+        super(keyValidator, columnIndex);
+
+        switch (columnIndex.getMode().mode)
+        {
+            case CONTAINS:
+                index = new ConcurrentSuffixTrie(columnIndex.getDefinition());
+                break;
+
+            case PREFIX:
+                index = new ConcurrentPrefixTrie(columnIndex.getDefinition());
+                break;
+
+            default:
+                throw new IllegalStateException("Unsupported mode: " + columnIndex.getMode().mode);
+        }
+    }
+
+    public long add(DecoratedKey key, ByteBuffer value)
+    {
+        AbstractAnalyzer analyzer = columnIndex.getAnalyzer();
+        analyzer.reset(value.duplicate());
+
+        long size = 0;
+        while (analyzer.hasNext())
+        {
+            ByteBuffer term = analyzer.next();
+
+            if (term.remaining() >= OnDiskIndexBuilder.MAX_TERM_SIZE)
+            {
+                logger.info("Can't add term of column {} to index for key: {}, term size {} bytes, max allowed size {} bytes, use analyzed = true (if not yet set) for that column.",
+                            columnIndex.getColumnName(),
+                            keyValidator.getString(key.getKey()),
+                            term.remaining(),
+                            OnDiskIndexBuilder.MAX_TERM_SIZE);
+                continue;
+            }
+
+            size += index.add(columnIndex.getValidator().getString(term), key);
+        }
+
+        return size;
+    }
+
+    public RangeIterator<Long, Token> search(Expression expression)
+    {
+        return index.search(expression);
+    }
+
+    private static abstract class ConcurrentTrie
+    {
+        public static final SizeEstimatingNodeFactory NODE_FACTORY = new SizeEstimatingNodeFactory();
+
+        protected final ColumnDefinition definition;
+
+        public ConcurrentTrie(ColumnDefinition column)
+        {
+            definition = column;
+        }
+
+        public long add(String value, DecoratedKey key)
+        {
+            long overhead = CSLM_OVERHEAD;
+            ConcurrentSkipListSet<DecoratedKey> keys = get(value);
+            if (keys == null)
+            {
+                ConcurrentSkipListSet<DecoratedKey> newKeys = new ConcurrentSkipListSet<>(DecoratedKey.comparator);
+                keys = putIfAbsent(value, newKeys);
+                if (keys == null)
+                {
+                    overhead += CSLM_OVERHEAD + value.length();
+                    keys = newKeys;
+                }
+            }
+
+            keys.add(key);
+
+            // get and reset new memory size allocated by current thread
+            overhead += NODE_FACTORY.currentUpdateSize();
+            NODE_FACTORY.reset();
+
+            return overhead;
+        }
+
+        public RangeIterator<Long, Token> search(Expression expression)
+        {
+            assert expression.getOp() == Expression.Op.EQ; // means that min == max
+
+            ByteBuffer prefix = expression.lower == null ? null : expression.lower.value;
+
+            Iterable<ConcurrentSkipListSet<DecoratedKey>> search = search(definition.cellValueType().getString(prefix));
+
+            RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+            for (ConcurrentSkipListSet<DecoratedKey> keys : search)
+            {
+                if (!keys.isEmpty())
+                    builder.add(new KeyRangeIterator(keys));
+            }
+
+            return builder.build();
+        }
+
+        protected abstract ConcurrentSkipListSet<DecoratedKey> get(String value);
+        protected abstract Iterable<ConcurrentSkipListSet<DecoratedKey>> search(String value);
+        protected abstract ConcurrentSkipListSet<DecoratedKey> putIfAbsent(String value, ConcurrentSkipListSet<DecoratedKey> key);
+    }
+
+    protected static class ConcurrentPrefixTrie extends ConcurrentTrie
+    {
+        private final ConcurrentRadixTree<ConcurrentSkipListSet<DecoratedKey>> trie;
+
+        private ConcurrentPrefixTrie(ColumnDefinition column)
+        {
+            super(column);
+            trie = new ConcurrentRadixTree<>(NODE_FACTORY);
+        }
+
+        public ConcurrentSkipListSet<DecoratedKey> get(String value)
+        {
+            return trie.getValueForExactKey(value);
+        }
+
+        public ConcurrentSkipListSet<DecoratedKey> putIfAbsent(String value, ConcurrentSkipListSet<DecoratedKey> newKeys)
+        {
+            return trie.putIfAbsent(value, newKeys);
+        }
+
+        public Iterable<ConcurrentSkipListSet<DecoratedKey>> search(String value)
+        {
+            return trie.getValuesForKeysStartingWith(value);
+        }
+    }
+
+    protected static class ConcurrentSuffixTrie extends ConcurrentTrie
+    {
+        private final ConcurrentSuffixTree<ConcurrentSkipListSet<DecoratedKey>> trie;
+
+        private ConcurrentSuffixTrie(ColumnDefinition column)
+        {
+            super(column);
+            trie = new ConcurrentSuffixTree<>(NODE_FACTORY);
+        }
+
+        public ConcurrentSkipListSet<DecoratedKey> get(String value)
+        {
+            return trie.getValueForExactKey(value);
+        }
+
+        public ConcurrentSkipListSet<DecoratedKey> putIfAbsent(String value, ConcurrentSkipListSet<DecoratedKey> newKeys)
+        {
+            return trie.putIfAbsent(value, newKeys);
+        }
+
+        public Iterable<ConcurrentSkipListSet<DecoratedKey>> search(String value)
+        {
+            return trie.getValuesForKeysContaining(value);
+        }
+    }
+
+    // This relies on the fact that all of the tree updates are done under exclusive write lock,
+    // method would overestimate in certain circumstances e.g. when nodes are replaced in place,
+    // but it's still better comparing to underestimate since it gives more breathing room for other memory users.
+    private static class SizeEstimatingNodeFactory extends SmartArrayBasedNodeFactory
+    {
+        private final ThreadLocal<Long> updateSize = ThreadLocal.withInitial(() -> 0L);
+
+        public Node createNode(CharSequence edgeCharacters, Object value, List<Node> childNodes, boolean isRoot)
+        {
+            Node node = super.createNode(edgeCharacters, value, childNodes, isRoot);
+            updateSize.set(updateSize.get() + measure(node));
+            return node;
+        }
+
+        public long currentUpdateSize()
+        {
+            return updateSize.get();
+        }
+
+        public void reset()
+        {
+            updateSize.set(0L);
+        }
+
+        private long measure(Node node)
+        {
+            // node with max overhead is CharArrayNodeLeafWithValue = 24B
+            long overhead = 24;
+
+            // array of chars (2 bytes) + CharSequence overhead
+            overhead += 24 + node.getIncomingEdge().length() * 2;
+
+            if (node.getOutgoingEdges() != null)
+            {
+                // 16 bytes for AtomicReferenceArray
+                overhead += 16;
+                overhead += 24 * node.getOutgoingEdges().size();
+            }
+
+            return overhead;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/plan/Expression.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/plan/Expression.java b/src/java/org/apache/cassandra/index/sasi/plan/Expression.java
new file mode 100644
index 0000000..e215ec7
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/plan/Expression.java
@@ -0,0 +1,340 @@
+/*
+ * 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.index.sasi.plan;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.index.sasi.analyzer.AbstractAnalyzer;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndex;
+import org.apache.cassandra.index.sasi.utils.TypeUtil;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Iterators;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Expression
+{
+    private static final Logger logger = LoggerFactory.getLogger(Expression.class);
+
+    public enum Op
+    {
+        EQ, NOT_EQ, RANGE
+    }
+
+    private final QueryController controller;
+
+    public final AbstractAnalyzer analyzer;
+
+    public final ColumnIndex index;
+    public final AbstractType<?> validator;
+    public final boolean isLiteral;
+
+    @VisibleForTesting
+    protected Op operation;
+
+    public Bound lower, upper;
+    public List<ByteBuffer> exclusions = new ArrayList<>();
+
+    public Expression(Expression other)
+    {
+        this(other.controller, other.index);
+        operation = other.operation;
+    }
+
+    public Expression(QueryController controller, ColumnIndex columnIndex)
+    {
+        this.controller = controller;
+        this.index = columnIndex;
+        this.analyzer = columnIndex.getAnalyzer();
+        this.validator = columnIndex.getValidator();
+        this.isLiteral = columnIndex.isLiteral();
+    }
+
+    @VisibleForTesting
+    public Expression(String name, AbstractType<?> validator)
+    {
+        this(null, new ColumnIndex(UTF8Type.instance, ColumnDefinition.regularDef("sasi", "internal", name, validator), null));
+    }
+
+    public Expression setLower(Bound newLower)
+    {
+        lower = newLower == null ? null : new Bound(newLower.value, newLower.inclusive);
+        return this;
+    }
+
+    public Expression setUpper(Bound newUpper)
+    {
+        upper = newUpper == null ? null : new Bound(newUpper.value, newUpper.inclusive);
+        return this;
+    }
+
+    public Expression setOp(Op op)
+    {
+        this.operation = op;
+        return this;
+    }
+
+    public Expression add(Operator op, ByteBuffer value)
+    {
+        boolean lowerInclusive = false, upperInclusive = false;
+        switch (op)
+        {
+            case EQ:
+                lower = new Bound(value, true);
+                upper = lower;
+                operation = Op.EQ;
+                break;
+
+            case NEQ:
+                // index expressions are priority sorted
+                // and NOT_EQ is the lowest priority, which means that operation type
+                // is always going to be set before reaching it in case of RANGE or EQ.
+                if (operation == null)
+                {
+                    operation = Op.NOT_EQ;
+                    lower = new Bound(value, true);
+                    upper = lower;
+                }
+                else
+                    exclusions.add(value);
+                break;
+
+            case LTE:
+                upperInclusive = true;
+            case LT:
+                operation = Op.RANGE;
+                upper = new Bound(value, upperInclusive);
+                break;
+
+            case GTE:
+                lowerInclusive = true;
+            case GT:
+                operation = Op.RANGE;
+                lower = new Bound(value, lowerInclusive);
+                break;
+        }
+
+        return this;
+    }
+
+    public Expression addExclusion(ByteBuffer value)
+    {
+        exclusions.add(value);
+        return this;
+    }
+
+    public boolean contains(ByteBuffer value)
+    {
+        if (!TypeUtil.isValid(value, validator))
+        {
+            int size = value.remaining();
+            if ((value = TypeUtil.tryUpcast(value, validator)) == null)
+            {
+                logger.error("Can't cast value for {} to size accepted by {}, value size is {} bytes.",
+                             index.getColumnName(),
+                             validator,
+                             size);
+                return false;
+            }
+        }
+
+        if (lower != null)
+        {
+            // suffix check
+            if (isLiteral)
+            {
+                if (!validateStringValue(value, lower.value))
+                    return false;
+            }
+            else
+            {
+                // range or (not-)equals - (mainly) for numeric values
+                int cmp = validator.compare(lower.value, value);
+
+                // in case of (NOT_)EQ lower == upper
+                if (operation == Op.EQ || operation == Op.NOT_EQ)
+                    return cmp == 0;
+
+                if (cmp > 0 || (cmp == 0 && !lower.inclusive))
+                    return false;
+            }
+        }
+
+        if (upper != null && lower != upper)
+        {
+            // string (prefix or suffix) check
+            if (isLiteral)
+            {
+                if (!validateStringValue(value, upper.value))
+                    return false;
+            }
+            else
+            {
+                // range - mainly for numeric values
+                int cmp = validator.compare(upper.value, value);
+                if (cmp < 0 || (cmp == 0 && !upper.inclusive))
+                    return false;
+            }
+        }
+
+        // as a last step let's check exclusions for the given field,
+        // this covers EQ/RANGE with exclusions.
+        for (ByteBuffer term : exclusions)
+        {
+            if (isLiteral && validateStringValue(value, term))
+                return false;
+            else if (validator.compare(term, value) == 0)
+                return false;
+        }
+
+        return true;
+    }
+
+    private boolean validateStringValue(ByteBuffer columnValue, ByteBuffer requestedValue)
+    {
+        analyzer.reset(columnValue.duplicate());
+        while (analyzer.hasNext())
+        {
+            ByteBuffer term = analyzer.next();
+            if (ByteBufferUtil.contains(term, requestedValue))
+                return true;
+        }
+
+        return false;
+    }
+
+    public Op getOp()
+    {
+        return operation;
+    }
+
+    public void checkpoint()
+    {
+        if (controller == null)
+            return;
+
+        controller.checkpoint();
+    }
+
+    public boolean hasLower()
+    {
+        return lower != null;
+    }
+
+    public boolean hasUpper()
+    {
+        return upper != null;
+    }
+
+    public boolean isLowerSatisfiedBy(OnDiskIndex.DataTerm term)
+    {
+        if (!hasLower())
+            return true;
+
+        int cmp = term.compareTo(validator, lower.value, false);
+        return cmp > 0 || cmp == 0 && lower.inclusive;
+    }
+
+    public boolean isUpperSatisfiedBy(OnDiskIndex.DataTerm term)
+    {
+        if (!hasUpper())
+            return true;
+
+        int cmp = term.compareTo(validator, upper.value, false);
+        return cmp < 0 || cmp == 0 && upper.inclusive;
+    }
+
+    public boolean isIndexed()
+    {
+        return index.isIndexed();
+    }
+
+    public String toString()
+    {
+        return String.format("Expression{name: %s, op: %s, lower: (%s, %s), upper: (%s, %s), exclusions: %s}",
+                             index.getColumnName(),
+                             operation,
+                             lower == null ? "null" : validator.getString(lower.value),
+                             lower != null && lower.inclusive,
+                             upper == null ? "null" : validator.getString(upper.value),
+                             upper != null && upper.inclusive,
+                             Iterators.toString(Iterators.transform(exclusions.iterator(), validator::getString)));
+    }
+
+    public int hashCode()
+    {
+        return new HashCodeBuilder().append(index.getColumnName())
+                                    .append(operation)
+                                    .append(validator)
+                                    .append(lower).append(upper)
+                                    .append(exclusions).build();
+    }
+
+    public boolean equals(Object other)
+    {
+        if (!(other instanceof Expression))
+            return false;
+
+        if (this == other)
+            return true;
+
+        Expression o = (Expression) other;
+
+        return Objects.equals(index.getColumnName(), o.index.getColumnName())
+                && validator.equals(o.validator)
+                && operation == o.operation
+                && Objects.equals(lower, o.lower)
+                && Objects.equals(upper, o.upper)
+                && exclusions.equals(o.exclusions);
+    }
+
+
+    public static class Bound
+    {
+        public final ByteBuffer value;
+        public final boolean inclusive;
+
+        public Bound(ByteBuffer value, boolean inclusive)
+        {
+            this.value = value;
+            this.inclusive = inclusive;
+        }
+
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof Bound))
+                return false;
+
+            Bound o = (Bound) other;
+            return value.equals(o.value) && inclusive == o.inclusive;
+        }
+    }
+}


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

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/NonTokenizingOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/NonTokenizingOptions.java b/src/java/org/apache/cassandra/index/sasi/analyzer/NonTokenizingOptions.java
new file mode 100644
index 0000000..303087b
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/NonTokenizingOptions.java
@@ -0,0 +1,147 @@
+/*
+ * 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.index.sasi.analyzer;
+
+import java.util.Map;
+
+public class NonTokenizingOptions
+{
+    public static final String NORMALIZE_LOWERCASE = "normalize_lowercase";
+    public static final String NORMALIZE_UPPERCASE = "normalize_uppercase";
+    public static final String CASE_SENSITIVE = "case_sensitive";
+
+    private boolean caseSensitive;
+    private boolean upperCaseOutput;
+    private boolean lowerCaseOutput;
+
+    public boolean isCaseSensitive()
+    {
+        return caseSensitive;
+    }
+
+    public void setCaseSensitive(boolean caseSensitive)
+    {
+        this.caseSensitive = caseSensitive;
+    }
+
+    public boolean shouldUpperCaseOutput()
+    {
+        return upperCaseOutput;
+    }
+
+    public void setUpperCaseOutput(boolean upperCaseOutput)
+    {
+        this.upperCaseOutput = upperCaseOutput;
+    }
+
+    public boolean shouldLowerCaseOutput()
+    {
+        return lowerCaseOutput;
+    }
+
+    public void setLowerCaseOutput(boolean lowerCaseOutput)
+    {
+        this.lowerCaseOutput = lowerCaseOutput;
+    }
+
+    public static class OptionsBuilder
+    {
+        private boolean caseSensitive = true;
+        private boolean upperCaseOutput = false;
+        private boolean lowerCaseOutput = false;
+
+        public OptionsBuilder()
+        {
+        }
+
+        public OptionsBuilder caseSensitive(boolean caseSensitive)
+        {
+            this.caseSensitive = caseSensitive;
+            return this;
+        }
+
+        public OptionsBuilder upperCaseOutput(boolean upperCaseOutput)
+        {
+            this.upperCaseOutput = upperCaseOutput;
+            return this;
+        }
+
+        public OptionsBuilder lowerCaseOutput(boolean lowerCaseOutput)
+        {
+            this.lowerCaseOutput = lowerCaseOutput;
+            return this;
+        }
+
+        public NonTokenizingOptions build()
+        {
+            if (lowerCaseOutput && upperCaseOutput)
+                throw new IllegalArgumentException("Options to normalize terms cannot be " +
+                        "both uppercase and lowercase at the same time");
+
+            NonTokenizingOptions options = new NonTokenizingOptions();
+            options.setCaseSensitive(caseSensitive);
+            options.setUpperCaseOutput(upperCaseOutput);
+            options.setLowerCaseOutput(lowerCaseOutput);
+            return options;
+        }
+    }
+
+    public static NonTokenizingOptions buildFromMap(Map<String, String> optionsMap)
+    {
+        OptionsBuilder optionsBuilder = new OptionsBuilder();
+
+        if (optionsMap.containsKey(CASE_SENSITIVE) && (optionsMap.containsKey(NORMALIZE_LOWERCASE)
+                || optionsMap.containsKey(NORMALIZE_UPPERCASE)))
+            throw new IllegalArgumentException("case_sensitive option cannot be specified together " +
+                    "with either normalize_lowercase or normalize_uppercase");
+
+        for (Map.Entry<String, String> entry : optionsMap.entrySet())
+        {
+            switch (entry.getKey())
+            {
+                case NORMALIZE_LOWERCASE:
+                {
+                    boolean bool = Boolean.parseBoolean(entry.getValue());
+                    optionsBuilder = optionsBuilder.lowerCaseOutput(bool);
+                    break;
+                }
+                case NORMALIZE_UPPERCASE:
+                {
+                    boolean bool = Boolean.parseBoolean(entry.getValue());
+                    optionsBuilder = optionsBuilder.upperCaseOutput(bool);
+                    break;
+                }
+                case CASE_SENSITIVE:
+                {
+                    boolean bool = Boolean.parseBoolean(entry.getValue());
+                    optionsBuilder = optionsBuilder.caseSensitive(bool);
+                    break;
+                }
+            }
+        }
+        return optionsBuilder.build();
+    }
+
+    public static NonTokenizingOptions getDefaultOptions()
+    {
+        return new OptionsBuilder()
+                .caseSensitive(true).lowerCaseOutput(false)
+                .upperCaseOutput(false)
+                .build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/SUPPLEMENTARY.jflex-macro
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/SUPPLEMENTARY.jflex-macro b/src/java/org/apache/cassandra/index/sasi/analyzer/SUPPLEMENTARY.jflex-macro
new file mode 100644
index 0000000..f5bf68e
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/SUPPLEMENTARY.jflex-macro
@@ -0,0 +1,143 @@
+/*
+ * 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.
+ */
+// Generated using ICU4J 52.1.0.0
+// by org.apache.lucene.analysis.icu.GenerateJFlexSupplementaryMacros
+
+
+ALetterSupp = (
+	  ([\ud83b][\uDE00-\uDE03\uDE05-\uDE1F\uDE21\uDE22\uDE24\uDE27\uDE29-\uDE32\uDE34-\uDE37\uDE39\uDE3B\uDE42\uDE47\uDE49\uDE4B\uDE4D-\uDE4F\uDE51\uDE52\uDE54\uDE57\uDE59\uDE5B\uDE5D\uDE5F\uDE61\uDE62\uDE64\uDE67-\uDE6A\uDE6C-\uDE72\uDE74-\uDE77\uDE79-\uDE7C\uDE7E\uDE80-\uDE89\uDE8B-\uDE9B\uDEA1-\uDEA3\uDEA5-\uDEA9\uDEAB-\uDEBB])
+	| ([\ud81a][\uDC00-\uDE38])
+	| ([\ud81b][\uDF00-\uDF44\uDF50\uDF93-\uDF9F])
+	| ([\ud835][\uDC00-\uDC54\uDC56-\uDC9C\uDC9E\uDC9F\uDCA2\uDCA5\uDCA6\uDCA9-\uDCAC\uDCAE-\uDCB9\uDCBB\uDCBD-\uDCC3\uDCC5-\uDD05\uDD07-\uDD0A\uDD0D-\uDD14\uDD16-\uDD1C\uDD1E-\uDD39\uDD3B-\uDD3E\uDD40-\uDD44\uDD46\uDD4A-\uDD50\uDD52-\uDEA5\uDEA8-\uDEC0\uDEC2-\uDEDA\uDEDC-\uDEFA\uDEFC-\uDF14\uDF16-\uDF34\uDF36-\uDF4E\uDF50-\uDF6E\uDF70-\uDF88\uDF8A-\uDFA8\uDFAA-\uDFC2\uDFC4-\uDFCB])
+	| ([\ud80d][\uDC00-\uDC2E])
+	| ([\ud80c][\uDC00-\uDFFF])
+	| ([\ud809][\uDC00-\uDC62])
+	| ([\ud808][\uDC00-\uDF6E])
+	| ([\ud805][\uDE80-\uDEAA])
+	| ([\ud804][\uDC03-\uDC37\uDC83-\uDCAF\uDCD0-\uDCE8\uDD03-\uDD26\uDD83-\uDDB2\uDDC1-\uDDC4])
+	| ([\ud801][\uDC00-\uDC9D])
+	| ([\ud800][\uDC00-\uDC0B\uDC0D-\uDC26\uDC28-\uDC3A\uDC3C\uDC3D\uDC3F-\uDC4D\uDC50-\uDC5D\uDC80-\uDCFA\uDD40-\uDD74\uDE80-\uDE9C\uDEA0-\uDED0\uDF00-\uDF1E\uDF30-\uDF4A\uDF80-\uDF9D\uDFA0-\uDFC3\uDFC8-\uDFCF\uDFD1-\uDFD5])
+	| ([\ud803][\uDC00-\uDC48])
+	| ([\ud802][\uDC00-\uDC05\uDC08\uDC0A-\uDC35\uDC37\uDC38\uDC3C\uDC3F-\uDC55\uDD00-\uDD15\uDD20-\uDD39\uDD80-\uDDB7\uDDBE\uDDBF\uDE00\uDE10-\uDE13\uDE15-\uDE17\uDE19-\uDE33\uDE60-\uDE7C\uDF00-\uDF35\uDF40-\uDF55\uDF60-\uDF72])
+)
+FormatSupp = (
+	  ([\ud804][\uDCBD])
+	| ([\ud834][\uDD73-\uDD7A])
+	| ([\udb40][\uDC01\uDC20-\uDC7F])
+)
+NumericSupp = (
+	  ([\ud805][\uDEC0-\uDEC9])
+	| ([\ud804][\uDC66-\uDC6F\uDCF0-\uDCF9\uDD36-\uDD3F\uDDD0-\uDDD9])
+	| ([\ud835][\uDFCE-\uDFFF])
+	| ([\ud801][\uDCA0-\uDCA9])
+)
+ExtendSupp = (
+	  ([\ud81b][\uDF51-\uDF7E\uDF8F-\uDF92])
+	| ([\ud805][\uDEAB-\uDEB7])
+	| ([\ud804][\uDC00-\uDC02\uDC38-\uDC46\uDC80-\uDC82\uDCB0-\uDCBA\uDD00-\uDD02\uDD27-\uDD34\uDD80-\uDD82\uDDB3-\uDDC0])
+	| ([\ud834][\uDD65-\uDD69\uDD6D-\uDD72\uDD7B-\uDD82\uDD85-\uDD8B\uDDAA-\uDDAD\uDE42-\uDE44])
+	| ([\ud800][\uDDFD])
+	| ([\udb40][\uDD00-\uDDEF])
+	| ([\ud802][\uDE01-\uDE03\uDE05\uDE06\uDE0C-\uDE0F\uDE38-\uDE3A\uDE3F])
+)
+KatakanaSupp = (
+	  ([\ud82c][\uDC00])
+)
+MidLetterSupp = (
+	  []
+)
+MidNumSupp = (
+	  []
+)
+MidNumLetSupp = (
+	  []
+)
+ExtendNumLetSupp = (
+	  []
+)
+ExtendNumLetSupp = (
+	  []
+)
+ComplexContextSupp = (
+	  []
+)
+HanSupp = (
+	  ([\ud87e][\uDC00-\uDE1D])
+	| ([\ud86b][\uDC00-\uDFFF])
+	| ([\ud86a][\uDC00-\uDFFF])
+	| ([\ud869][\uDC00-\uDED6\uDF00-\uDFFF])
+	| ([\ud868][\uDC00-\uDFFF])
+	| ([\ud86e][\uDC00-\uDC1D])
+	| ([\ud86d][\uDC00-\uDF34\uDF40-\uDFFF])
+	| ([\ud86c][\uDC00-\uDFFF])
+	| ([\ud863][\uDC00-\uDFFF])
+	| ([\ud862][\uDC00-\uDFFF])
+	| ([\ud861][\uDC00-\uDFFF])
+	| ([\ud860][\uDC00-\uDFFF])
+	| ([\ud867][\uDC00-\uDFFF])
+	| ([\ud866][\uDC00-\uDFFF])
+	| ([\ud865][\uDC00-\uDFFF])
+	| ([\ud864][\uDC00-\uDFFF])
+	| ([\ud858][\uDC00-\uDFFF])
+	| ([\ud859][\uDC00-\uDFFF])
+	| ([\ud85a][\uDC00-\uDFFF])
+	| ([\ud85b][\uDC00-\uDFFF])
+	| ([\ud85c][\uDC00-\uDFFF])
+	| ([\ud85d][\uDC00-\uDFFF])
+	| ([\ud85e][\uDC00-\uDFFF])
+	| ([\ud85f][\uDC00-\uDFFF])
+	| ([\ud850][\uDC00-\uDFFF])
+	| ([\ud851][\uDC00-\uDFFF])
+	| ([\ud852][\uDC00-\uDFFF])
+	| ([\ud853][\uDC00-\uDFFF])
+	| ([\ud854][\uDC00-\uDFFF])
+	| ([\ud855][\uDC00-\uDFFF])
+	| ([\ud856][\uDC00-\uDFFF])
+	| ([\ud857][\uDC00-\uDFFF])
+	| ([\ud849][\uDC00-\uDFFF])
+	| ([\ud848][\uDC00-\uDFFF])
+	| ([\ud84b][\uDC00-\uDFFF])
+	| ([\ud84a][\uDC00-\uDFFF])
+	| ([\ud84d][\uDC00-\uDFFF])
+	| ([\ud84c][\uDC00-\uDFFF])
+	| ([\ud84f][\uDC00-\uDFFF])
+	| ([\ud84e][\uDC00-\uDFFF])
+	| ([\ud841][\uDC00-\uDFFF])
+	| ([\ud840][\uDC00-\uDFFF])
+	| ([\ud843][\uDC00-\uDFFF])
+	| ([\ud842][\uDC00-\uDFFF])
+	| ([\ud845][\uDC00-\uDFFF])
+	| ([\ud844][\uDC00-\uDFFF])
+	| ([\ud847][\uDC00-\uDFFF])
+	| ([\ud846][\uDC00-\uDFFF])
+)
+HiraganaSupp = (
+	  ([\ud83c][\uDE00])
+	| ([\ud82c][\uDC01])
+)
+SingleQuoteSupp = (
+	  []
+)
+DoubleQuoteSupp = (
+	  []
+)
+HebrewLetterSupp = (
+	  []
+)
+RegionalIndicatorSupp = (
+	  ([\ud83c][\uDDE6-\uDDFF])
+)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzer.java b/src/java/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzer.java
new file mode 100644
index 0000000..bcc63df
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzer.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.index.sasi.analyzer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import org.apache.cassandra.index.sasi.analyzer.filter.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import com.carrotsearch.hppc.IntObjectMap;
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+
+public class StandardAnalyzer extends AbstractAnalyzer
+{
+    public enum TokenType
+    {
+        EOF(-1),
+        ALPHANUM(0),
+        NUM(6),
+        SOUTHEAST_ASIAN(9),
+        IDEOGRAPHIC(10),
+        HIRAGANA(11),
+        KATAKANA(12),
+        HANGUL(13);
+
+        private static final IntObjectMap<TokenType> TOKENS = new IntObjectOpenHashMap<>();
+
+        static
+        {
+            for (TokenType type : TokenType.values())
+                TOKENS.put(type.value, type);
+        }
+
+        public final int value;
+
+        TokenType(int value)
+        {
+            this.value = value;
+        }
+
+        public int getValue()
+        {
+            return value;
+        }
+
+        public static TokenType fromValue(int val)
+        {
+            return TOKENS.get(val);
+        }
+    }
+
+    private AbstractType validator;
+
+    private StandardTokenizerInterface scanner;
+    private StandardTokenizerOptions options;
+    private FilterPipelineTask filterPipeline;
+
+    protected Reader inputReader = null;
+
+    public String getToken()
+    {
+        return scanner.getText();
+    }
+
+    public final boolean incrementToken() throws IOException
+    {
+        while(true)
+        {
+            TokenType currentTokenType = TokenType.fromValue(scanner.getNextToken());
+            if (currentTokenType == TokenType.EOF)
+                return false;
+            if (scanner.yylength() <= options.getMaxTokenLength()
+                    && scanner.yylength() >= options.getMinTokenLength())
+                return true;
+        }
+    }
+
+    protected String getFilteredCurrentToken() throws IOException
+    {
+        String token = getToken();
+        Object pipelineRes;
+
+        while (true)
+        {
+            pipelineRes = FilterPipelineExecutor.execute(filterPipeline, token);
+            if (pipelineRes != null)
+                break;
+
+            boolean reachedEOF = incrementToken();
+            if (!reachedEOF)
+                break;
+
+            token = getToken();
+        }
+
+        return (String) pipelineRes;
+    }
+
+    private FilterPipelineTask getFilterPipeline()
+    {
+        FilterPipelineBuilder builder = new FilterPipelineBuilder(new BasicResultFilters.NoOperation());
+        if (!options.isCaseSensitive() && options.shouldLowerCaseTerms())
+            builder = builder.add("to_lower", new BasicResultFilters.LowerCase());
+        if (!options.isCaseSensitive() && options.shouldUpperCaseTerms())
+            builder = builder.add("to_upper", new BasicResultFilters.UpperCase());
+        if (options.shouldStemTerms())
+            builder = builder.add("term_stemming", new StemmingFilters.DefaultStemmingFilter(options.getLocale()));
+        if (options.shouldIgnoreStopTerms())
+            builder = builder.add("skip_stop_words", new StopWordFilters.DefaultStopWordFilter(options.getLocale()));
+        return builder.build();
+    }
+
+    public void init(Map<String, String> options, AbstractType validator)
+    {
+        init(StandardTokenizerOptions.buildFromMap(options), validator);
+    }
+
+    @VisibleForTesting
+    protected void init(StandardTokenizerOptions options)
+    {
+        init(options, UTF8Type.instance);
+    }
+
+    public void init(StandardTokenizerOptions tokenizerOptions, AbstractType validator)
+    {
+        this.validator = validator;
+        this.options = tokenizerOptions;
+        this.filterPipeline = getFilterPipeline();
+
+        Reader reader = new InputStreamReader(new DataInputBuffer(ByteBufferUtil.EMPTY_BYTE_BUFFER, false));
+        this.scanner = new StandardTokenizerImpl(reader);
+        this.inputReader = reader;
+    }
+
+    public boolean hasNext()
+    {
+        try
+        {
+            if (incrementToken())
+            {
+                if (getFilteredCurrentToken() != null)
+                {
+                    this.next = validator.fromString(normalize(getFilteredCurrentToken()));
+                    return true;
+                }
+            }
+        }
+        catch (IOException e)
+        {}
+
+        return false;
+    }
+
+    public void reset(ByteBuffer input)
+    {
+        this.next = null;
+        Reader reader = new InputStreamReader(new DataInputBuffer(input, false));
+        scanner.yyreset(reader);
+        this.inputReader = reader;
+    }
+
+    public void reset(InputStream input)
+    {
+        this.next = null;
+        Reader reader = new InputStreamReader(input);
+        scanner.yyreset(reader);
+        this.inputReader = reader;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerImpl.jflex
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerImpl.jflex b/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerImpl.jflex
new file mode 100644
index 0000000..d0270ff
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerImpl.jflex
@@ -0,0 +1,220 @@
+package org.apache.cassandra.index.sasi.analyzer;
+
+/*
+ * 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.
+ */
+
+import java.util.Arrays;
+
+/**
+ * This class implements Word Break rules from the Unicode Text Segmentation 
+ * algorithm, as specified in 
+ * <a href="http://unicode.org/reports/tr29/">Unicode Standard Annex #29</a>. ∂
+ * <p/>
+ * Tokens produced are of the following types:
+ * <ul>
+ *   <li>&lt;ALPHANUM&gt;: A sequence of alphabetic and numeric characters</li>
+ *   <li>&lt;NUM&gt;: A number</li>
+ *   <li>&lt;SOUTHEAST_ASIAN&gt;: A sequence of characters from South and Southeast
+ *       Asian languages, including Thai, Lao, Myanmar, and Khmer</li>
+ *   <li>&lt;IDEOGRAPHIC&gt;: A single CJKV ideographic character</li>
+ *   <li>&lt;HIRAGANA&gt;: A single hiragana character</li>
+ *   <li>&lt;KATAKANA&gt;: A sequence of katakana characters</li>
+ *   <li>&lt;HANGUL&gt;: A sequence of Hangul characters</li>
+ * </ul>
+ */
+%%
+
+%unicode 6.3
+%integer
+%final
+%public
+%class StandardTokenizerImpl
+%implements StandardTokenizerInterface
+%function getNextToken
+%char
+%buffer 4096
+
+%include SUPPLEMENTARY.jflex-macro
+ALetter           = (\p{WB:ALetter}                                     | {ALetterSupp})
+Format            = (\p{WB:Format}                                      | {FormatSupp})
+Numeric           = ([\p{WB:Numeric}[\p{Blk:HalfAndFullForms}&&\p{Nd}]] | {NumericSupp})
+Extend            = (\p{WB:Extend}                                      | {ExtendSupp})
+Katakana          = (\p{WB:Katakana}                                    | {KatakanaSupp})
+MidLetter         = (\p{WB:MidLetter}                                   | {MidLetterSupp})
+MidNum            = (\p{WB:MidNum}                                      | {MidNumSupp})
+MidNumLet         = (\p{WB:MidNumLet}                                   | {MidNumLetSupp})
+ExtendNumLet      = (\p{WB:ExtendNumLet}                                | {ExtendNumLetSupp})
+ComplexContext    = (\p{LB:Complex_Context}                             | {ComplexContextSupp})
+Han               = (\p{Script:Han}                                     | {HanSupp})
+Hiragana          = (\p{Script:Hiragana}                                | {HiraganaSupp})
+SingleQuote       = (\p{WB:Single_Quote}                                | {SingleQuoteSupp})
+DoubleQuote       = (\p{WB:Double_Quote}                                | {DoubleQuoteSupp})
+HebrewLetter      = (\p{WB:Hebrew_Letter}                               | {HebrewLetterSupp})
+RegionalIndicator = (\p{WB:Regional_Indicator}                          | {RegionalIndicatorSupp})
+HebrewOrALetter   = ({HebrewLetter} | {ALetter})
+
+// UAX#29 WB4. X (Extend | Format)* --> X
+//
+HangulEx            = [\p{Script:Hangul}&&[\p{WB:ALetter}\p{WB:Hebrew_Letter}]] ({Format} | {Extend})*
+HebrewOrALetterEx   = {HebrewOrALetter}                                         ({Format} | {Extend})*
+NumericEx           = {Numeric}                                                 ({Format} | {Extend})*
+KatakanaEx          = {Katakana}                                                ({Format} | {Extend})* 
+MidLetterEx         = ({MidLetter} | {MidNumLet} | {SingleQuote})               ({Format} | {Extend})* 
+MidNumericEx        = ({MidNum} | {MidNumLet} | {SingleQuote})                  ({Format} | {Extend})*
+ExtendNumLetEx      = {ExtendNumLet}                                            ({Format} | {Extend})*
+HanEx               = {Han}                                                     ({Format} | {Extend})*
+HiraganaEx          = {Hiragana}                                                ({Format} | {Extend})*
+SingleQuoteEx       = {SingleQuote}                                             ({Format} | {Extend})*                                            
+DoubleQuoteEx       = {DoubleQuote}                                             ({Format} | {Extend})*
+HebrewLetterEx      = {HebrewLetter}                                            ({Format} | {Extend})*
+RegionalIndicatorEx = {RegionalIndicator}                                       ({Format} | {Extend})*
+
+
+%{
+  /** Alphanumeric sequences */
+  public static final int WORD_TYPE = StandardAnalyzer.TokenType.ALPHANUM.value;
+  
+  /** Numbers */
+  public static final int NUMERIC_TYPE = StandardAnalyzer.TokenType.NUM.value;
+  
+  /**
+   * Chars in class \p{Line_Break = Complex_Context} are from South East Asian
+   * scripts (Thai, Lao, Myanmar, Khmer, etc.).  Sequences of these are kept 
+   * together as as a single token rather than broken up, because the logic
+   * required to break them at word boundaries is too complex for UAX#29.
+   * <p>
+   * See Unicode Line Breaking Algorithm: http://www.unicode.org/reports/tr14/#SA
+   */
+  public static final int SOUTH_EAST_ASIAN_TYPE = StandardAnalyzer.TokenType.SOUTHEAST_ASIAN.value;
+  
+  public static final int IDEOGRAPHIC_TYPE = StandardAnalyzer.TokenType.IDEOGRAPHIC.value;
+  
+  public static final int HIRAGANA_TYPE = StandardAnalyzer.TokenType.HIRAGANA.value;
+  
+  public static final int KATAKANA_TYPE = StandardAnalyzer.TokenType.KATAKANA.value;
+  
+  public static final int HANGUL_TYPE = StandardAnalyzer.TokenType.HANGUL.value;
+
+  public final int yychar()
+  {
+    return yychar;
+  }
+
+  public String getText()
+  {
+    return String.valueOf(zzBuffer, zzStartRead, zzMarkedPos-zzStartRead);
+  }
+
+  public char[] getArray()
+  {
+    return Arrays.copyOfRange(zzBuffer, zzStartRead, zzMarkedPos);
+  }
+
+  public byte[] getBytes()
+  {
+    return getText().getBytes();
+  }
+
+%}
+
+%%
+
+// UAX#29 WB1.   sot   ÷
+//        WB2.     ÷   eot
+//
+<<EOF>> { return StandardAnalyzer.TokenType.EOF.value; }
+
+// UAX#29 WB8.   Numeric × Numeric
+//        WB11.  Numeric (MidNum | MidNumLet | Single_Quote) × Numeric
+//        WB12.  Numeric × (MidNum | MidNumLet | Single_Quote) Numeric
+//        WB13a. (ALetter | Hebrew_Letter | Numeric | Katakana | ExtendNumLet) × ExtendNumLet
+//        WB13b. ExtendNumLet × (ALetter | Hebrew_Letter | Numeric | Katakana) 
+//
+{ExtendNumLetEx}* {NumericEx} ( ( {ExtendNumLetEx}* | {MidNumericEx} ) {NumericEx} )* {ExtendNumLetEx}* 
+  { return NUMERIC_TYPE; }
+
+// subset of the below for typing purposes only!
+{HangulEx}+
+  { return HANGUL_TYPE; }
+  
+{KatakanaEx}+
+  { return KATAKANA_TYPE; }
+
+// UAX#29 WB5.   (ALetter | Hebrew_Letter) × (ALetter | Hebrew_Letter)
+//        WB6.   (ALetter | Hebrew_Letter) × (MidLetter | MidNumLet | Single_Quote) (ALetter | Hebrew_Letter)
+//        WB7.   (ALetter | Hebrew_Letter) (MidLetter | MidNumLet | Single_Quote) × (ALetter | Hebrew_Letter)
+//        WB7a.  Hebrew_Letter × Single_Quote
+//        WB7b.  Hebrew_Letter × Double_Quote Hebrew_Letter
+//        WB7c.  Hebrew_Letter Double_Quote × Hebrew_Letter
+//        WB9.   (ALetter | Hebrew_Letter) × Numeric
+//        WB10.  Numeric × (ALetter | Hebrew_Letter)
+//        WB13.  Katakana × Katakana
+//        WB13a. (ALetter | Hebrew_Letter | Numeric | Katakana | ExtendNumLet) × ExtendNumLet
+//        WB13b. ExtendNumLet × (ALetter | Hebrew_Letter | Numeric | Katakana) 
+//
+{ExtendNumLetEx}*  ( {KatakanaEx}          ( {ExtendNumLetEx}*   {KatakanaEx}                           )*
+                   | ( {HebrewLetterEx}    ( {SingleQuoteEx}     | {DoubleQuoteEx}  {HebrewLetterEx}    )
+                     | {NumericEx}         ( ( {ExtendNumLetEx}* | {MidNumericEx} ) {NumericEx}         )*
+                     | {HebrewOrALetterEx} ( ( {ExtendNumLetEx}* | {MidLetterEx}  ) {HebrewOrALetterEx} )*
+                     )+
+                   )
+({ExtendNumLetEx}+ ( {KatakanaEx}          ( {ExtendNumLetEx}*   {KatakanaEx}                           )*
+                   | ( {HebrewLetterEx}    ( {SingleQuoteEx}     | {DoubleQuoteEx}  {HebrewLetterEx}    )
+                     | {NumericEx}         ( ( {ExtendNumLetEx}* | {MidNumericEx} ) {NumericEx}         )*
+                     | {HebrewOrALetterEx} ( ( {ExtendNumLetEx}* | {MidLetterEx}  ) {HebrewOrALetterEx} )*
+                     )+
+                   )
+)*
+{ExtendNumLetEx}* 
+  { return WORD_TYPE; }
+
+
+// From UAX #29:
+//
+//    [C]haracters with the Line_Break property values of Contingent_Break (CB), 
+//    Complex_Context (SA/South East Asian), and XX (Unknown) are assigned word 
+//    boundary property values based on criteria outside of the scope of this
+//    annex.  That means that satisfactory treatment of languages like Chinese
+//    or Thai requires special handling.
+// 
+// In Unicode 6.3, only one character has the \p{Line_Break = Contingent_Break}
+// property: U+FFFC (  ) OBJECT REPLACEMENT CHARACTER.
+//
+// In the ICU implementation of UAX#29, \p{Line_Break = Complex_Context}
+// character sequences (from South East Asian scripts like Thai, Myanmar, Khmer,
+// Lao, etc.) are kept together.  This grammar does the same below.
+//
+// See also the Unicode Line Breaking Algorithm:
+//
+//    http://www.unicode.org/reports/tr14/#SA
+//
+{ComplexContext}+ { return SOUTH_EAST_ASIAN_TYPE; }
+
+// UAX#29 WB14.  Any ÷ Any
+//
+{HanEx} { return IDEOGRAPHIC_TYPE; }
+{HiraganaEx} { return HIRAGANA_TYPE; }
+
+
+// UAX#29 WB3.   CR × LF
+//        WB3a.  (Newline | CR | LF) ÷
+//        WB3b.  ÷ (Newline | CR | LF)
+//        WB13c. Regional_Indicator × Regional_Indicator
+//        WB14.  Any ÷ Any
+//
+{RegionalIndicatorEx} {RegionalIndicatorEx}+ | [^]
+  { /* Break so we don't hit fall-through warning: */ break; /* Not numeric, word, ideographic, hiragana, or SE Asian -- ignore it. */ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerInterface.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerInterface.java b/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerInterface.java
new file mode 100644
index 0000000..57e35d7
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerInterface.java
@@ -0,0 +1,65 @@
+/*
+ * 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.index.sasi.analyzer;
+
+import java.io.IOException;
+import java.io.Reader;
+
+/**
+ * Internal interface for supporting versioned grammars.
+ */
+public interface StandardTokenizerInterface
+{
+
+    String getText();
+
+    char[] getArray();
+
+    byte[] getBytes();
+
+    /**
+     * Returns the current position.
+     */
+    int yychar();
+
+    /**
+     * Returns the length of the matched text region.
+     */
+    int yylength();
+
+    /**
+     * Resumes scanning until the next regular expression is matched,
+     * the end of input is encountered or an I/O-Error occurs.
+     *
+     * @return      the next token, {@link #YYEOF} on end of stream
+     * @exception   java.io.IOException  if any I/O-Error occurs
+     */
+    int getNextToken() throws IOException;
+
+    /**
+     * Resets the scanner to read from a new input stream.
+     * Does not close the old reader.
+     *
+     * All internal variables are reset, the old input stream
+     * <b>cannot</b> be reused (internal buffer is discarded and lost).
+     * Lexical state is set to <tt>ZZ_INITIAL</tt>.
+     *
+     * @param reader   the new input stream
+     */
+    void yyreset(Reader reader);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerOptions.java b/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerOptions.java
new file mode 100644
index 0000000..2a5e4ef
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/StandardTokenizerOptions.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.index.sasi.analyzer;
+
+import java.util.Locale;
+import java.util.Map;
+
+/**
+ * Various options for controlling tokenization and enabling
+ * or disabling features
+ */
+public class StandardTokenizerOptions
+{
+    public static final String TOKENIZATION_ENABLE_STEMMING = "tokenization_enable_stemming";
+    public static final String TOKENIZATION_SKIP_STOP_WORDS = "tokenization_skip_stop_words";
+    public static final String TOKENIZATION_LOCALE = "tokenization_locale";
+    public static final String TOKENIZATION_NORMALIZE_LOWERCASE = "tokenization_normalize_lowercase";
+    public static final String TOKENIZATION_NORMALIZE_UPPERCASE = "tokenization_normalize_uppercase";
+
+    public static final int DEFAULT_MAX_TOKEN_LENGTH = 255;
+    public static final int DEFAULT_MIN_TOKEN_LENGTH = 0;
+
+    private boolean stemTerms;
+    private boolean ignoreStopTerms;
+    private Locale locale;
+    private boolean caseSensitive;
+    private boolean allTermsToUpperCase;
+    private boolean allTermsToLowerCase;
+    private int minTokenLength;
+    private int maxTokenLength;
+
+    public boolean shouldStemTerms()
+    {
+        return stemTerms;
+    }
+
+    public void setStemTerms(boolean stemTerms)
+    {
+        this.stemTerms = stemTerms;
+    }
+
+    public boolean shouldIgnoreStopTerms()
+    {
+        return ignoreStopTerms;
+    }
+
+    public void setIgnoreStopTerms(boolean ignoreStopTerms)
+    {
+        this.ignoreStopTerms = ignoreStopTerms;
+    }
+
+    public Locale getLocale()
+    {
+        return locale;
+    }
+
+    public void setLocale(Locale locale)
+    {
+        this.locale = locale;
+    }
+
+    public boolean isCaseSensitive()
+    {
+        return caseSensitive;
+    }
+
+    public void setCaseSensitive(boolean caseSensitive)
+    {
+        this.caseSensitive = caseSensitive;
+    }
+
+    public boolean shouldUpperCaseTerms()
+    {
+        return allTermsToUpperCase;
+    }
+
+    public void setAllTermsToUpperCase(boolean allTermsToUpperCase)
+    {
+        this.allTermsToUpperCase = allTermsToUpperCase;
+    }
+
+    public boolean shouldLowerCaseTerms()
+    {
+        return allTermsToLowerCase;
+    }
+
+    public void setAllTermsToLowerCase(boolean allTermsToLowerCase)
+    {
+        this.allTermsToLowerCase = allTermsToLowerCase;
+    }
+
+    public int getMinTokenLength()
+    {
+        return minTokenLength;
+    }
+
+    public void setMinTokenLength(int minTokenLength)
+    {
+        this.minTokenLength = minTokenLength;
+    }
+
+    public int getMaxTokenLength()
+    {
+        return maxTokenLength;
+    }
+
+    public void setMaxTokenLength(int maxTokenLength)
+    {
+        this.maxTokenLength = maxTokenLength;
+    }
+
+    public static class OptionsBuilder {
+        private boolean stemTerms;
+        private boolean ignoreStopTerms;
+        private Locale locale;
+        private boolean caseSensitive;
+        private boolean allTermsToUpperCase;
+        private boolean allTermsToLowerCase;
+        private int minTokenLength = DEFAULT_MIN_TOKEN_LENGTH;
+        private int maxTokenLength = DEFAULT_MAX_TOKEN_LENGTH;
+
+        public OptionsBuilder()
+        {
+        }
+
+        public OptionsBuilder stemTerms(boolean stemTerms)
+        {
+            this.stemTerms = stemTerms;
+            return this;
+        }
+
+        public OptionsBuilder ignoreStopTerms(boolean ignoreStopTerms)
+        {
+            this.ignoreStopTerms = ignoreStopTerms;
+            return this;
+        }
+
+        public OptionsBuilder useLocale(Locale locale)
+        {
+            this.locale = locale;
+            return this;
+        }
+
+        public OptionsBuilder caseSensitive(boolean caseSensitive)
+        {
+            this.caseSensitive = caseSensitive;
+            return this;
+        }
+
+        public OptionsBuilder alwaysUpperCaseTerms(boolean allTermsToUpperCase)
+        {
+            this.allTermsToUpperCase = allTermsToUpperCase;
+            return this;
+        }
+
+        public OptionsBuilder alwaysLowerCaseTerms(boolean allTermsToLowerCase)
+        {
+            this.allTermsToLowerCase = allTermsToLowerCase;
+            return this;
+        }
+
+        /**
+         * Set the min allowed token length.  Any token shorter
+         * than this is skipped.
+         */
+        public OptionsBuilder minTokenLength(int minTokenLength)
+        {
+            if (minTokenLength < 1)
+                throw new IllegalArgumentException("minTokenLength must be greater than zero");
+            this.minTokenLength = minTokenLength;
+            return this;
+        }
+
+        /**
+         * Set the max allowed token length.  Any token longer
+         * than this is skipped.
+         */
+        public OptionsBuilder maxTokenLength(int maxTokenLength)
+        {
+            if (maxTokenLength < 1)
+                throw new IllegalArgumentException("maxTokenLength must be greater than zero");
+            this.maxTokenLength = maxTokenLength;
+            return this;
+        }
+
+        public StandardTokenizerOptions build()
+        {
+            if(allTermsToLowerCase && allTermsToUpperCase)
+                throw new IllegalArgumentException("Options to normalize terms cannot be " +
+                        "both uppercase and lowercase at the same time");
+
+            StandardTokenizerOptions options = new StandardTokenizerOptions();
+            options.setIgnoreStopTerms(ignoreStopTerms);
+            options.setStemTerms(stemTerms);
+            options.setLocale(locale);
+            options.setCaseSensitive(caseSensitive);
+            options.setAllTermsToLowerCase(allTermsToLowerCase);
+            options.setAllTermsToUpperCase(allTermsToUpperCase);
+            options.setMinTokenLength(minTokenLength);
+            options.setMaxTokenLength(maxTokenLength);
+            return options;
+        }
+    }
+
+    public static StandardTokenizerOptions buildFromMap(Map<String, String> optionsMap)
+    {
+        OptionsBuilder optionsBuilder = new OptionsBuilder();
+
+        for (Map.Entry<String, String> entry : optionsMap.entrySet())
+        {
+            switch(entry.getKey())
+            {
+                case TOKENIZATION_ENABLE_STEMMING:
+                {
+                    boolean bool = Boolean.parseBoolean(entry.getValue());
+                    optionsBuilder = optionsBuilder.stemTerms(bool);
+                    break;
+                }
+                case TOKENIZATION_SKIP_STOP_WORDS:
+                {
+                    boolean bool = Boolean.parseBoolean(entry.getValue());
+                    optionsBuilder = optionsBuilder.ignoreStopTerms(bool);
+                    break;
+                }
+                case TOKENIZATION_LOCALE:
+                {
+                    Locale locale = new Locale(entry.getValue());
+                    optionsBuilder = optionsBuilder.useLocale(locale);
+                    break;
+                }
+                case TOKENIZATION_NORMALIZE_UPPERCASE:
+                {
+                    boolean bool = Boolean.parseBoolean(entry.getValue());
+                    optionsBuilder = optionsBuilder.alwaysUpperCaseTerms(bool);
+                    break;
+                }
+                case TOKENIZATION_NORMALIZE_LOWERCASE:
+                {
+                    boolean bool = Boolean.parseBoolean(entry.getValue());
+                    optionsBuilder = optionsBuilder.alwaysLowerCaseTerms(bool);
+                    break;
+                }
+                default:
+                {
+                }
+            }
+        }
+        return optionsBuilder.build();
+    }
+
+    public static StandardTokenizerOptions getDefaultOptions()
+    {
+        return new OptionsBuilder()
+                .ignoreStopTerms(true).alwaysLowerCaseTerms(true)
+                .stemTerms(false).useLocale(Locale.ENGLISH).build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/filter/BasicResultFilters.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/BasicResultFilters.java b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/BasicResultFilters.java
new file mode 100644
index 0000000..2b949b8
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/BasicResultFilters.java
@@ -0,0 +1,76 @@
+/*
+ * 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.index.sasi.analyzer.filter;
+
+import java.util.Locale;
+
+/**
+ * Basic/General Token Filters
+ */
+public class BasicResultFilters
+{
+    private static final Locale DEFAULT_LOCALE = Locale.getDefault();
+
+    public static class LowerCase extends FilterPipelineTask<String, String>
+    {
+        private Locale locale;
+
+        public LowerCase(Locale locale)
+        {
+            this.locale = locale;
+        }
+
+        public LowerCase()
+        {
+            this.locale = DEFAULT_LOCALE;
+        }
+
+        public String process(String input) throws Exception
+        {
+            return input.toLowerCase(locale);
+        }
+    }
+
+    public static class UpperCase extends FilterPipelineTask<String, String>
+    {
+        private Locale locale;
+
+        public UpperCase(Locale locale)
+        {
+            this.locale = locale;
+        }
+
+        public UpperCase()
+        {
+            this.locale = DEFAULT_LOCALE;
+        }
+
+        public String process(String input) throws Exception
+        {
+            return input.toUpperCase(locale);
+        }
+    }
+
+    public static class NoOperation extends FilterPipelineTask<Object, Object>
+    {
+        public Object process(Object input) throws Exception
+        {
+            return input;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineBuilder.java b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineBuilder.java
new file mode 100644
index 0000000..e9d262d
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineBuilder.java
@@ -0,0 +1,51 @@
+/*
+ * 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.index.sasi.analyzer.filter;
+
+/**
+ * Creates a Pipeline object for applying n pieces of logic
+ * from the provided methods to the builder in a guaranteed order
+ */
+public class FilterPipelineBuilder
+{
+    private final FilterPipelineTask<?,?> parent;
+    private FilterPipelineTask<?,?> current;
+
+    public FilterPipelineBuilder(FilterPipelineTask<?, ?> first)
+    {
+        this(first, first);
+    }
+
+    private FilterPipelineBuilder(FilterPipelineTask<?, ?> first, FilterPipelineTask<?, ?> current)
+    {
+        this.parent = first;
+        this.current = current;
+    }
+
+    public FilterPipelineBuilder add(String name, FilterPipelineTask<?,?> nextTask)
+    {
+        this.current.setLast(name, nextTask);
+        this.current = nextTask;
+        return this;
+    }
+
+    public FilterPipelineTask<?,?> build()
+    {
+        return this.parent;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineExecutor.java b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineExecutor.java
new file mode 100644
index 0000000..68c055e
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineExecutor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.index.sasi.analyzer.filter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Executes all linked Pipeline Tasks serially and returns
+ * output (if exists) from the executed logic
+ */
+public class FilterPipelineExecutor
+{
+    private static final Logger logger = LoggerFactory.getLogger(FilterPipelineExecutor.class);
+
+    public static <F,T> T execute(FilterPipelineTask<F, T> task, T initialInput)
+    {
+        FilterPipelineTask<?, ?> taskPtr = task;
+        T result = initialInput;
+        try
+        {
+            while (true)
+            {
+                FilterPipelineTask<F,T> taskGeneric = (FilterPipelineTask<F,T>) taskPtr;
+                result = taskGeneric.process((F) result);
+                taskPtr = taskPtr.next;
+                if(taskPtr == null)
+                    return result;
+            }
+        }
+        catch (Exception e)
+        {
+            logger.info("An unhandled exception to occurred while processing " +
+                    "pipeline [{}]", task.getName(), e);
+        }
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineTask.java b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineTask.java
new file mode 100644
index 0000000..13e2a17
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/FilterPipelineTask.java
@@ -0,0 +1,52 @@
+/*
+ * 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.index.sasi.analyzer.filter;
+
+/**
+ * A single task or set of work to process an input
+ * and return a single output. Maintains a link to the
+ * next task to be executed after itself
+ */
+public abstract class FilterPipelineTask<F, T>
+{
+    private String name;
+    public FilterPipelineTask<?, ?> next;
+
+    protected <K, V> void setLast(String name, FilterPipelineTask<K, V> last)
+    {
+        if (last == this)
+            throw new IllegalArgumentException("provided last task [" + last.name + "] cannot be set to itself");
+
+        if (this.next == null)
+        {
+            this.next = last;
+            this.name = name;
+        }
+        else
+        {
+            this.next.setLast(name, last);
+        }
+    }
+
+    public abstract T process(F input) throws Exception;
+
+    public String getName()
+    {
+        return name;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StemmerFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StemmerFactory.java b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StemmerFactory.java
new file mode 100644
index 0000000..04da55c
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StemmerFactory.java
@@ -0,0 +1,101 @@
+/*
+ * 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.index.sasi.analyzer.filter;
+
+import java.lang.reflect.Constructor;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+
+import org.tartarus.snowball.SnowballStemmer;
+import org.tartarus.snowball.ext.*;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Returns a SnowballStemmer instance appropriate for
+ * a given language
+ */
+public class StemmerFactory
+{
+    private static final Logger logger = LoggerFactory.getLogger(StemmerFactory.class);
+    private static final LoadingCache<Class, Constructor<?>> STEMMER_CONSTRUCTOR_CACHE = CacheBuilder.newBuilder()
+            .build(new CacheLoader<Class, Constructor<?>>()
+            {
+                public Constructor<?> load(Class aClass) throws Exception
+                {
+                    try
+                    {
+                        return aClass.getConstructor();
+                    }
+                    catch (Exception e) {
+                        logger.error("Failed to get stemmer constructor", e);
+                    }
+                    return null;
+                }
+            });
+
+    private static final Map<String, Class> SUPPORTED_LANGUAGES;
+
+    static
+    {
+        SUPPORTED_LANGUAGES = new HashMap<>();
+        SUPPORTED_LANGUAGES.put("de", germanStemmer.class);
+        SUPPORTED_LANGUAGES.put("da", danishStemmer.class);
+        SUPPORTED_LANGUAGES.put("es", spanishStemmer.class);
+        SUPPORTED_LANGUAGES.put("en", englishStemmer.class);
+        SUPPORTED_LANGUAGES.put("fl", finnishStemmer.class);
+        SUPPORTED_LANGUAGES.put("fr", frenchStemmer.class);
+        SUPPORTED_LANGUAGES.put("hu", hungarianStemmer.class);
+        SUPPORTED_LANGUAGES.put("it", italianStemmer.class);
+        SUPPORTED_LANGUAGES.put("nl", dutchStemmer.class);
+        SUPPORTED_LANGUAGES.put("no", norwegianStemmer.class);
+        SUPPORTED_LANGUAGES.put("pt", portugueseStemmer.class);
+        SUPPORTED_LANGUAGES.put("ro", romanianStemmer.class);
+        SUPPORTED_LANGUAGES.put("ru", russianStemmer.class);
+        SUPPORTED_LANGUAGES.put("sv", swedishStemmer.class);
+        SUPPORTED_LANGUAGES.put("tr", turkishStemmer.class);
+    }
+
+    public static SnowballStemmer getStemmer(Locale locale)
+    {
+        if (locale == null)
+            return null;
+
+        String rootLang = locale.getLanguage().substring(0, 2);
+        try
+        {
+            Class clazz = SUPPORTED_LANGUAGES.get(rootLang);
+            if(clazz == null)
+                return null;
+            Constructor<?> ctor = STEMMER_CONSTRUCTOR_CACHE.get(clazz);
+            return (SnowballStemmer) ctor.newInstance();
+        }
+        catch (Exception e)
+        {
+            logger.debug("Failed to create new SnowballStemmer instance " +
+                    "for language [{}]", locale.getLanguage(), e);
+        }
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StemmingFilters.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StemmingFilters.java b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StemmingFilters.java
new file mode 100644
index 0000000..9e098d1
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StemmingFilters.java
@@ -0,0 +1,46 @@
+/*
+ * 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.index.sasi.analyzer.filter;
+
+import java.util.Locale;
+
+import org.tartarus.snowball.SnowballStemmer;
+
+/**
+ * Filters for performing Stemming on tokens
+ */
+public class StemmingFilters
+{
+    public static class DefaultStemmingFilter extends FilterPipelineTask<String, String>
+    {
+        private SnowballStemmer stemmer;
+
+        public DefaultStemmingFilter(Locale locale)
+        {
+            stemmer = StemmerFactory.getStemmer(locale);
+        }
+
+        public String process(String input) throws Exception
+        {
+            if (stemmer == null)
+                return input;
+            stemmer.setCurrent(input);
+            return (stemmer.stem()) ? stemmer.getCurrent() : input;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFactory.java b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFactory.java
new file mode 100644
index 0000000..0492822
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFactory.java
@@ -0,0 +1,100 @@
+/*
+ * 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.index.sasi.analyzer.filter;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides a list of Stop Words for a given language
+ */
+public class StopWordFactory
+{
+    private static final Logger logger = LoggerFactory.getLogger(StopWordFactory.class);
+
+    private static final String DEFAULT_RESOURCE_EXT = "_ST.txt";
+    private static final String DEFAULT_RESOURCE_PREFIX = StopWordFactory.class.getPackage()
+            .getName().replace(".", File.separator);
+    private static final Set<String> SUPPORTED_LANGUAGES = new HashSet<>(
+            Arrays.asList("ar","bg","cs","de","en","es","fi","fr","hi","hu","it",
+            "pl","pt","ro","ru","sv"));
+
+    private static final LoadingCache<String, Set<String>> STOP_WORDS_CACHE = CacheBuilder.newBuilder()
+            .build(new CacheLoader<String, Set<String>>()
+            {
+                public Set<String> load(String s) throws Exception
+                {
+                    return getStopWordsFromResource(s);
+                }
+            });
+
+    public static Set<String> getStopWordsForLanguage(Locale locale)
+    {
+        if (locale == null)
+            return null;
+
+        String rootLang = locale.getLanguage().substring(0, 2);
+        try
+        {
+            return (!SUPPORTED_LANGUAGES.contains(rootLang)) ? null : STOP_WORDS_CACHE.get(rootLang);
+        }
+        catch (ExecutionException e)
+        {
+            logger.error("Failed to populate Stop Words Cache for language [{}]", locale.getLanguage(), e);
+            return null;
+        }
+    }
+
+    private static Set<String> getStopWordsFromResource(String language)
+    {
+        Set<String> stopWords = new HashSet<>();
+        String resourceName = DEFAULT_RESOURCE_PREFIX + File.separator + language + DEFAULT_RESOURCE_EXT;
+        try (InputStream is = StopWordFactory.class.getClassLoader().getResourceAsStream(resourceName);
+             BufferedReader r = new BufferedReader(new InputStreamReader(is, StandardCharsets.UTF_8)))
+        {
+                String line;
+                while ((line = r.readLine()) != null)
+                {
+                    //skip comments (lines starting with # char)
+                    if(line.charAt(0) == '#')
+                        continue;
+                    stopWords.add(line.trim());
+                }
+        }
+        catch (Exception e)
+        {
+            logger.error("Failed to retrieve Stop Terms resource for language [{}]", language, e);
+        }
+        return stopWords;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFilters.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFilters.java b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFilters.java
new file mode 100644
index 0000000..4ae849c
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFilters.java
@@ -0,0 +1,42 @@
+/*
+ * 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.index.sasi.analyzer.filter;
+
+import java.util.Locale;
+import java.util.Set;
+
+/**
+ * Filter implementations for input matching Stop Words
+ */
+public class StopWordFilters
+{
+    public static class DefaultStopWordFilter extends FilterPipelineTask<String, String>
+    {
+        private Set<String> stopWords = null;
+
+        public DefaultStopWordFilter(Locale locale)
+        {
+            this.stopWords = StopWordFactory.getStopWordsForLanguage(locale);
+        }
+
+        public String process(String input) throws Exception
+        {
+            return (stopWords != null && stopWords.contains(input)) ? null : input;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java b/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java
new file mode 100644
index 0000000..0bc9a96
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java
@@ -0,0 +1,193 @@
+/*
+ * 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.index.sasi.conf;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.index.sasi.analyzer.AbstractAnalyzer;
+import org.apache.cassandra.index.sasi.conf.view.View;
+import org.apache.cassandra.index.sasi.disk.Token;
+import org.apache.cassandra.index.sasi.memory.IndexMemtable;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.IndexMetadata;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class ColumnIndex
+{
+    private static final String FILE_NAME_FORMAT = "SI_%s.db";
+
+    private final AbstractType<?> keyValidator;
+
+    private final ColumnDefinition column;
+    private final Optional<IndexMetadata> config;
+
+    private final AtomicReference<IndexMemtable> memtable;
+    private final IndexMode mode;
+
+    private final Component component;
+    private final DataTracker tracker;
+
+    public ColumnIndex(AbstractType<?> keyValidator, ColumnDefinition column, IndexMetadata metadata)
+    {
+        this.keyValidator = keyValidator;
+        this.column = column;
+        this.config = metadata == null ? Optional.empty() : Optional.of(metadata);
+        this.mode = IndexMode.getMode(column, config);
+        this.memtable = new AtomicReference<>(new IndexMemtable(this));
+        this.tracker = new DataTracker(keyValidator, this);
+        this.component = new Component(Component.Type.SECONDARY_INDEX, String.format(FILE_NAME_FORMAT, getIndexName()));
+    }
+
+    public void validate() throws ConfigurationException
+    {
+        mode.validate(config);
+    }
+
+    /**
+     * Initialize this column index with specific set of SSTables.
+     *
+     * @param sstables The sstables to be used by index initially.
+     *
+     * @return A collection of sstables which don't have this specific index attached to them.
+     */
+    public Iterable<SSTableReader> init(Set<SSTableReader> sstables)
+    {
+        return tracker.update(Collections.emptySet(), sstables);
+    }
+
+    public AbstractType<?> keyValidator()
+    {
+        return keyValidator;
+    }
+
+    public long index(DecoratedKey key, Row row)
+    {
+        return memtable.get().index(key, getValueOf(column, row, FBUtilities.nowInSeconds()));
+    }
+
+    public void switchMemtable()
+    {
+        memtable.set(new IndexMemtable(this));
+    }
+
+    public RangeIterator<Long, Token> searchMemtable(Expression e)
+    {
+        return memtable.get().search(e);
+    }
+
+    public void update(Collection<SSTableReader> oldSSTables, Collection<SSTableReader> newSSTables)
+    {
+        tracker.update(oldSSTables, newSSTables);
+    }
+
+    public ColumnDefinition getDefinition()
+    {
+        return column;
+    }
+
+    public AbstractType<?> getValidator()
+    {
+        return column.cellValueType();
+    }
+
+    public Component getComponent()
+    {
+        return component;
+    }
+
+    public IndexMode getMode()
+    {
+        return mode;
+    }
+
+    public String getColumnName()
+    {
+        return column.name.toString();
+    }
+
+    public String getIndexName()
+    {
+        return config.isPresent() ? config.get().name : "undefined";
+    }
+
+    public AbstractAnalyzer getAnalyzer()
+    {
+        AbstractAnalyzer analyzer = mode.getAnalyzer(getValidator());
+        analyzer.init(config.isPresent() ? config.get().options : Collections.emptyMap(), column.cellValueType());
+        return analyzer;
+    }
+
+    public View getView()
+    {
+        return tracker.getView();
+    }
+
+    public boolean hasSSTable(SSTableReader sstable)
+    {
+        return tracker.hasSSTable(sstable);
+    }
+
+    public void dropData(long truncateUntil)
+    {
+        switchMemtable();
+        tracker.dropData(truncateUntil);
+    }
+
+    public boolean isIndexed()
+    {
+        return mode != IndexMode.NOT_INDEXED;
+    }
+
+    public boolean isLiteral()
+    {
+        AbstractType<?> validator = getValidator();
+        return isIndexed() ? mode.isLiteral : (validator instanceof UTF8Type || validator instanceof AsciiType);
+    }
+
+    public static ByteBuffer getValueOf(ColumnDefinition column, Row row, int nowInSecs)
+    {
+        switch (column.kind)
+        {
+            case CLUSTERING:
+                return row.clustering().get(column.position());
+
+            case REGULAR:
+                Cell cell = row.getCell(column);
+                return cell == null || !cell.isLive(nowInSecs) ? null : cell.value();
+
+            default:
+                return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/conf/DataTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/DataTracker.java b/src/java/org/apache/cassandra/index/sasi/conf/DataTracker.java
new file mode 100644
index 0000000..9475d12
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/conf/DataTracker.java
@@ -0,0 +1,162 @@
+/*
+ * 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.index.sasi.conf;
+
+import java.io.File;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.index.sasi.SSTableIndex;
+import org.apache.cassandra.index.sasi.conf.view.View;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+import com.google.common.collect.Sets;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** a pared-down version of DataTracker and DT.View. need one for each index of each column family */
+public class DataTracker
+{
+    private static final Logger logger = LoggerFactory.getLogger(DataTracker.class);
+
+    private final AbstractType<?> keyValidator;
+    private final ColumnIndex columnIndex;
+    private final AtomicReference<View> view = new AtomicReference<>();
+
+    public DataTracker(AbstractType<?> keyValidator, ColumnIndex index)
+    {
+        this.keyValidator = keyValidator;
+        this.columnIndex = index;
+        this.view.set(new View(index, Collections.<SSTableIndex>emptySet()));
+    }
+
+    public View getView()
+    {
+        return view.get();
+    }
+
+    /**
+     * Replaces old SSTables with new by creating new immutable tracker.
+     *
+     * @param oldSSTables A set of SSTables to remove.
+     * @param newSSTables A set of SSTables to add to tracker.
+     *
+     * @return A collection of SSTables which don't have component attached for current index.
+     */
+    public Iterable<SSTableReader> update(Collection<SSTableReader> oldSSTables, Collection<SSTableReader> newSSTables)
+    {
+        final Set<SSTableIndex> newIndexes = getIndexes(newSSTables);
+        final Set<SSTableReader> indexedSSTables = getSSTables(newIndexes);
+
+        View currentView, newView;
+        do
+        {
+            currentView = view.get();
+            newView = new View(columnIndex, currentView.getIndexes(), oldSSTables, newIndexes);
+        }
+        while (!view.compareAndSet(currentView, newView));
+
+        return newSSTables.stream().filter(sstable -> !indexedSSTables.contains(sstable)).collect(Collectors.toList());
+    }
+
+    public boolean hasSSTable(SSTableReader sstable)
+    {
+        View currentView = view.get();
+        for (SSTableIndex index : currentView)
+        {
+            if (index.getSSTable().equals(sstable))
+                return true;
+        }
+
+        return false;
+    }
+
+    public void dropData(long truncateUntil)
+    {
+        View currentView = view.get();
+        if (currentView == null)
+            return;
+
+        Set<SSTableReader> toRemove = new HashSet<>();
+        for (SSTableIndex index : currentView)
+        {
+            SSTableReader sstable = index.getSSTable();
+            if (sstable.getMaxTimestamp() > truncateUntil)
+                continue;
+
+            index.markObsolete();
+            toRemove.add(sstable);
+        }
+
+        update(toRemove, Collections.<SSTableReader>emptyList());
+    }
+
+    private Set<SSTableIndex> getIndexes(Collection<SSTableReader> sstables)
+    {
+        Set<SSTableIndex> indexes = new HashSet<>(sstables.size());
+        for (SSTableReader sstable : sstables)
+        {
+            if (sstable.isMarkedCompacted())
+                continue;
+
+            File indexFile = new File(sstable.descriptor.filenameFor(columnIndex.getComponent()));
+            if (!indexFile.exists())
+                continue;
+
+            SSTableIndex index = null;
+
+            try
+            {
+                index = new SSTableIndex(columnIndex, indexFile, sstable);
+
+                logger.info("SSTableIndex.open(column: {}, minTerm: {}, maxTerm: {}, minKey: {}, maxKey: {}, sstable: {})",
+                            columnIndex.getColumnName(),
+                            columnIndex.getValidator().getString(index.minTerm()),
+                            columnIndex.getValidator().getString(index.maxTerm()),
+                            keyValidator.getString(index.minKey()),
+                            keyValidator.getString(index.maxKey()),
+                            index.getSSTable());
+
+                // Try to add new index to the set, if set already has such index, we'll simply release and move on.
+                // This covers situation when sstable collection has the same sstable multiple
+                // times because we don't know what kind of collection it actually is.
+                if (!indexes.add(index))
+                    index.release();
+            }
+            catch (Throwable t)
+            {
+                logger.error("Can't open index file at " + indexFile.getAbsolutePath() + ", skipping.", t);
+                if (index != null)
+                    index.release();
+            }
+        }
+
+        return indexes;
+    }
+
+    private Set<SSTableReader> getSSTables(Set<SSTableIndex> indexes)
+    {
+        return Sets.newHashSet(indexes.stream().map(SSTableIndex::getSSTable).collect(Collectors.toList()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java b/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
new file mode 100644
index 0000000..628a6ce
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.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.index.sasi.conf;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.index.sasi.analyzer.AbstractAnalyzer;
+import org.apache.cassandra.index.sasi.analyzer.NoOpAnalyzer;
+import org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer;
+import org.apache.cassandra.index.sasi.analyzer.StandardAnalyzer;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder.Mode;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.schema.IndexMetadata;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class IndexMode
+{
+    private static final Logger logger = LoggerFactory.getLogger(IndexMode.class);
+
+    public static final IndexMode NOT_INDEXED = new IndexMode(Mode.PREFIX, true, false, NonTokenizingAnalyzer.class, 0);
+
+    private static final Set<AbstractType<?>> TOKENIZABLE_TYPES = new HashSet<AbstractType<?>>()
+    {{
+        add(UTF8Type.instance);
+        add(AsciiType.instance);
+    }};
+
+    private static final String INDEX_MODE_OPTION = "mode";
+    private static final String INDEX_ANALYZED_OPTION = "analyzed";
+    private static final String INDEX_ANALYZER_CLASS_OPTION = "analyzer_class";
+    private static final String INDEX_IS_LITERAL_OPTION = "is_literal";
+    private static final String INDEX_MAX_FLUSH_MEMORY_OPTION = "max_compaction_flush_memory_in_mb";
+    private static final double INDEX_MAX_FLUSH_DEFAULT_MULTIPLIER = 0.15;
+
+    public final Mode mode;
+    public final boolean isAnalyzed, isLiteral;
+    public final Class analyzerClass;
+    public final long maxCompactionFlushMemoryInMb;
+
+    private IndexMode(Mode mode, boolean isLiteral, boolean isAnalyzed, Class analyzerClass, long maxFlushMemMb)
+    {
+        this.mode = mode;
+        this.isLiteral = isLiteral;
+        this.isAnalyzed = isAnalyzed;
+        this.analyzerClass = analyzerClass;
+        this.maxCompactionFlushMemoryInMb = maxFlushMemMb;
+    }
+
+    public void validate(Optional<IndexMetadata> config) throws ConfigurationException
+    {
+        if (!config.isPresent())
+            return;
+
+        Map<String, String> indexOptions = config.get().options;
+        // validate that a valid analyzer class was provided if specified
+        if (indexOptions.containsKey(INDEX_ANALYZER_CLASS_OPTION))
+        {
+            try
+            {
+                Class.forName(indexOptions.get(INDEX_ANALYZER_CLASS_OPTION));
+            }
+            catch (ClassNotFoundException e)
+            {
+                throw new ConfigurationException(String.format("Invalid analyzer class option specified [%s]",
+                        indexOptions.get(INDEX_ANALYZER_CLASS_OPTION)));
+            }
+        }
+    }
+
+    public AbstractAnalyzer getAnalyzer(AbstractType<?> validator)
+    {
+        AbstractAnalyzer analyzer = new NoOpAnalyzer();
+
+        try
+        {
+            if (isAnalyzed)
+            {
+                if (analyzerClass != null)
+                    analyzer = (AbstractAnalyzer) analyzerClass.newInstance();
+                else if (TOKENIZABLE_TYPES.contains(validator))
+                    analyzer = new StandardAnalyzer();
+            }
+        }
+        catch (InstantiationException | IllegalAccessException e)
+        {
+            logger.error("Failed to create new instance of analyzer with class [{}]", analyzerClass.getName(), e);
+        }
+
+        return analyzer;
+    }
+
+    public static IndexMode getMode(ColumnDefinition column, Optional<IndexMetadata> config)
+    {
+        Map<String, String> indexOptions = config.isPresent() ? config.get().options : null;
+        if (indexOptions == null || indexOptions.isEmpty())
+            return IndexMode.NOT_INDEXED;
+
+        Mode mode = indexOptions.get(INDEX_MODE_OPTION) == null
+                        ? Mode.PREFIX
+                        : Mode.mode(indexOptions.get(INDEX_MODE_OPTION));
+
+        boolean isAnalyzed = false;
+        Class analyzerClass = null;
+        try
+        {
+            if (indexOptions.get(INDEX_ANALYZER_CLASS_OPTION) != null)
+            {
+                analyzerClass = Class.forName(indexOptions.get(INDEX_ANALYZER_CLASS_OPTION));
+                isAnalyzed = indexOptions.get(INDEX_ANALYZED_OPTION) == null
+                              ? true : Boolean.valueOf(indexOptions.get(INDEX_ANALYZED_OPTION));
+            }
+            else if (indexOptions.get(INDEX_ANALYZED_OPTION) != null)
+            {
+                isAnalyzed = Boolean.valueOf(indexOptions.get(INDEX_ANALYZED_OPTION));
+            }
+        }
+        catch (ClassNotFoundException e)
+        {
+            // should not happen as we already validated we could instantiate an instance in validateOptions()
+            logger.error("Failed to find specified analyzer class [{}]. Falling back to default analyzer",
+                         indexOptions.get(INDEX_ANALYZER_CLASS_OPTION));
+        }
+
+        boolean isLiteral = false;
+        try
+        {
+            String literalOption = indexOptions.get(INDEX_IS_LITERAL_OPTION);
+            AbstractType<?> validator = column.cellValueType();
+
+            isLiteral = literalOption == null
+                            ? (validator instanceof UTF8Type || validator instanceof AsciiType)
+                            : Boolean.valueOf(literalOption);
+        }
+        catch (Exception e)
+        {
+            logger.error("failed to parse {} option, defaulting to 'false' for {} index.", INDEX_IS_LITERAL_OPTION, config.get().name);
+        }
+
+        Long maxMemMb = indexOptions.get(INDEX_MAX_FLUSH_MEMORY_OPTION) == null
+                ? (long) (1073741824 * INDEX_MAX_FLUSH_DEFAULT_MULTIPLIER) // 1G default for memtable
+                : Long.parseLong(indexOptions.get(INDEX_MAX_FLUSH_MEMORY_OPTION));
+
+        return new IndexMode(mode, isLiteral, isAnalyzed, analyzerClass, maxMemMb);
+    }
+}


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

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/RangeIntersectionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/RangeIntersectionIterator.java b/src/java/org/apache/cassandra/index/sasi/utils/RangeIntersectionIterator.java
new file mode 100644
index 0000000..0d2214a
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/RangeIntersectionIterator.java
@@ -0,0 +1,281 @@
+/*
+ * 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.index.sasi.utils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import com.google.common.collect.Iterators;
+import org.apache.cassandra.io.util.FileUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class RangeIntersectionIterator
+{
+    protected enum Strategy
+    {
+        BOUNCE, LOOKUP, ADAPTIVE
+    }
+
+    public static <K extends Comparable<K>, D extends CombinedValue<K>> Builder<K, D> builder()
+    {
+        return builder(Strategy.ADAPTIVE);
+    }
+
+    @VisibleForTesting
+    protected static <K extends Comparable<K>, D extends CombinedValue<K>> Builder<K, D> builder(Strategy strategy)
+    {
+        return new Builder<>(strategy);
+    }
+
+    public static class Builder<K extends Comparable<K>, D extends CombinedValue<K>> extends RangeIterator.Builder<K, D>
+    {
+        private final Strategy strategy;
+
+        public Builder(Strategy strategy)
+        {
+            super(IteratorType.INTERSECTION);
+            this.strategy = strategy;
+        }
+
+        protected RangeIterator<K, D> buildIterator()
+        {
+            // if the range is disjoint we can simply return empty
+            // iterator of any type, because it's not going to produce any results.
+            if (statistics.isDisjoint())
+                return new BounceIntersectionIterator<>(statistics, new PriorityQueue<RangeIterator<K, D>>(1));
+
+            switch (strategy)
+            {
+                case LOOKUP:
+                    return new LookupIntersectionIterator<>(statistics, ranges);
+
+                case BOUNCE:
+                    return new BounceIntersectionIterator<>(statistics, ranges);
+
+                case ADAPTIVE:
+                    return statistics.sizeRatio() <= 0.01d
+                            ? new LookupIntersectionIterator<>(statistics, ranges)
+                            : new BounceIntersectionIterator<>(statistics, ranges);
+
+                default:
+                    throw new IllegalStateException("Unknown strategy: " + strategy);
+            }
+        }
+    }
+
+    private static abstract class AbstractIntersectionIterator<K extends Comparable<K>, D extends CombinedValue<K>> extends RangeIterator<K, D>
+    {
+        protected final PriorityQueue<RangeIterator<K, D>> ranges;
+
+        private AbstractIntersectionIterator(Builder.Statistics<K, D> statistics, PriorityQueue<RangeIterator<K, D>> ranges)
+        {
+            super(statistics);
+            this.ranges = ranges;
+        }
+
+        public void close() throws IOException
+        {
+            for (RangeIterator<K, D> range : ranges)
+                FileUtils.closeQuietly(range);
+        }
+    }
+
+    /**
+     * Iterator which performs intersection of multiple ranges by using bouncing (merge-join) technique to identify
+     * common elements in the given ranges. Aforementioned "bounce" works as follows: range queue is poll'ed for the
+     * range with the smallest current token (main loop), that token is used to {@link RangeIterator#skipTo(Comparable)}
+     * other ranges, if token produced by {@link RangeIterator#skipTo(Comparable)} is equal to current "candidate" token,
+     * both get merged together and the same operation is repeated for next range from the queue, if returned token
+     * is not equal than candidate, candidate's range gets put back into the queue and the main loop gets repeated until
+     * next intersection token is found or at least one iterator runs out of tokens.
+     *
+     * This technique is every efficient to jump over gaps in the ranges.
+     *
+     * @param <K> The type used to sort ranges.
+     * @param <D> The container type which is going to be returned by {@link Iterator#next()}.
+     */
+    @VisibleForTesting
+    protected static class BounceIntersectionIterator<K extends Comparable<K>, D extends CombinedValue<K>> extends AbstractIntersectionIterator<K, D>
+    {
+        private BounceIntersectionIterator(Builder.Statistics<K, D> statistics, PriorityQueue<RangeIterator<K, D>> ranges)
+        {
+            super(statistics, ranges);
+        }
+
+        protected D computeNext()
+        {
+            while (!ranges.isEmpty())
+            {
+                RangeIterator<K, D> head = ranges.poll();
+
+                // jump right to the beginning of the intersection or return next element
+                if (head.getCurrent().compareTo(getMinimum()) < 0)
+                    head.skipTo(getMinimum());
+
+                D candidate = head.hasNext() ? head.next() : null;
+                if (candidate == null || candidate.get().compareTo(getMaximum()) > 0)
+                {
+                    ranges.add(head);
+                    return endOfData();
+                }
+
+                List<RangeIterator<K, D>> processed = new ArrayList<>();
+
+                boolean intersectsAll = true, exhausted = false;
+                while (!ranges.isEmpty())
+                {
+                    RangeIterator<K, D> range = ranges.poll();
+
+                    // found a range which doesn't overlap with one (or possibly more) other range(s)
+                    if (!isOverlapping(head, range))
+                    {
+                        exhausted = true;
+                        intersectsAll = false;
+                        break;
+                    }
+
+                    D point = range.skipTo(candidate.get());
+
+                    if (point == null) // other range is exhausted
+                    {
+                        exhausted = true;
+                        intersectsAll = false;
+                        break;
+                    }
+
+                    processed.add(range);
+
+                    if (candidate.get().equals(point.get()))
+                    {
+                        candidate.merge(point);
+                        // advance skipped range to the next element if any
+                        Iterators.getNext(range, null);
+                    }
+                    else
+                    {
+                        intersectsAll = false;
+                        break;
+                    }
+                }
+
+                ranges.add(head);
+
+                for (RangeIterator<K, D> range : processed)
+                    ranges.add(range);
+
+                if (exhausted)
+                    return endOfData();
+
+                if (intersectsAll)
+                    return candidate;
+            }
+
+            return endOfData();
+        }
+
+        protected void performSkipTo(K nextToken)
+        {
+            List<RangeIterator<K, D>> skipped = new ArrayList<>();
+
+            while (!ranges.isEmpty())
+            {
+                RangeIterator<K, D> range = ranges.poll();
+                range.skipTo(nextToken);
+                skipped.add(range);
+            }
+
+            for (RangeIterator<K, D> range : skipped)
+                ranges.add(range);
+        }
+    }
+
+    /**
+     * Iterator which performs a linear scan over a primary range (the smallest of the ranges)
+     * and O(log(n)) lookup into secondary ranges using values from the primary iterator.
+     * This technique is efficient when one of the intersection ranges is smaller than others
+     * e.g. ratio 0.01d (default), in such situation scan + lookup is more efficient comparing
+     * to "bounce" merge because "bounce" distance is never going to be big.
+     *
+     * @param <K> The type used to sort ranges.
+     * @param <D> The container type which is going to be returned by {@link Iterator#next()}.
+     */
+    @VisibleForTesting
+    protected static class LookupIntersectionIterator<K extends Comparable<K>, D extends CombinedValue<K>> extends AbstractIntersectionIterator<K, D>
+    {
+        private final RangeIterator<K, D> smallestIterator;
+
+        private LookupIntersectionIterator(Builder.Statistics<K, D> statistics, PriorityQueue<RangeIterator<K, D>> ranges)
+        {
+            super(statistics, ranges);
+
+            smallestIterator = statistics.minRange;
+
+            if (smallestIterator.getCurrent().compareTo(getMinimum()) < 0)
+                smallestIterator.skipTo(getMinimum());
+        }
+
+        protected D computeNext()
+        {
+            while (smallestIterator.hasNext())
+            {
+                D candidate = smallestIterator.next();
+                K token = candidate.get();
+
+                boolean intersectsAll = true;
+                for (RangeIterator<K, D> range : ranges)
+                {
+                    // avoid checking against self, much cheaper than changing queue comparator
+                    // to compare based on the size and re-populating such queue.
+                    if (range.equals(smallestIterator))
+                        continue;
+
+                    // found a range which doesn't overlap with one (or possibly more) other range(s)
+                    if (!isOverlapping(smallestIterator, range))
+                        return endOfData();
+
+                    D point = range.skipTo(token);
+
+                    if (point == null) // one of the iterators is exhausted
+                        return endOfData();
+
+                    if (!point.get().equals(token))
+                    {
+                        intersectsAll = false;
+                        break;
+                    }
+
+                    candidate.merge(point);
+                }
+
+                if (intersectsAll)
+                    return candidate;
+            }
+
+            return endOfData();
+        }
+
+        protected void performSkipTo(K nextToken)
+        {
+            smallestIterator.skipTo(nextToken);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/RangeIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/RangeIterator.java b/src/java/org/apache/cassandra/index/sasi/utils/RangeIterator.java
new file mode 100644
index 0000000..1b5aee4
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/RangeIterator.java
@@ -0,0 +1,279 @@
+/*
+ * 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.index.sasi.utils;
+
+import java.io.Closeable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public abstract class RangeIterator<K extends Comparable<K>, T extends CombinedValue<K>> extends AbstractIterator<T> implements Closeable
+{
+    private final K min, max;
+    private final long count;
+    private K current;
+
+    protected RangeIterator(Builder.Statistics<K, T> statistics)
+    {
+        this(statistics.min, statistics.max, statistics.tokenCount);
+    }
+
+    public RangeIterator(RangeIterator<K, T> range)
+    {
+        this(range == null ? null : range.min, range == null ? null : range.max, range == null ? -1 : range.count);
+    }
+
+    public RangeIterator(K min, K max, long count)
+    {
+        this.min = min;
+        this.current = min;
+        this.max = max;
+        this.count = count;
+    }
+
+    public final K getMinimum()
+    {
+        return min;
+    }
+
+    public final K getCurrent()
+    {
+        return current;
+    }
+
+    public final K getMaximum()
+    {
+        return max;
+    }
+
+    public final long getCount()
+    {
+        return count;
+    }
+
+    /**
+     * When called, this iterators current position should
+     * be skipped forwards until finding either:
+     *   1) an element equal to or bigger than next
+     *   2) the end of the iterator
+     *
+     * @param nextToken value to skip the iterator forward until matching
+     *
+     * @return The next current token after the skip was performed
+     */
+    public final T skipTo(K nextToken)
+    {
+        if (min == null || max == null)
+            return endOfData();
+
+        if (current.compareTo(nextToken) >= 0)
+            return next == null ? recomputeNext() : next;
+
+        if (max.compareTo(nextToken) < 0)
+            return endOfData();
+
+        performSkipTo(nextToken);
+        return recomputeNext();
+    }
+
+    protected abstract void performSkipTo(K nextToken);
+
+    protected T recomputeNext()
+    {
+        return tryToComputeNext() ? peek() : endOfData();
+    }
+
+    protected boolean tryToComputeNext()
+    {
+        boolean hasNext = super.tryToComputeNext();
+        current = hasNext ? next.get() : getMaximum();
+        return hasNext;
+    }
+
+    public static abstract class Builder<K extends Comparable<K>, D extends CombinedValue<K>>
+    {
+        public enum IteratorType
+        {
+            UNION, INTERSECTION
+        }
+
+        @VisibleForTesting
+        protected final Statistics<K, D> statistics;
+
+        @VisibleForTesting
+        protected final PriorityQueue<RangeIterator<K, D>> ranges;
+
+        public Builder(IteratorType type)
+        {
+            statistics = new Statistics<>(type);
+            ranges = new PriorityQueue<>(16, (Comparator<RangeIterator<K, D>>) (a, b) -> a.getCurrent().compareTo(b.getCurrent()));
+        }
+
+        public K getMinimum()
+        {
+            return statistics.min;
+        }
+
+        public K getMaximum()
+        {
+            return statistics.max;
+        }
+
+        public long getTokenCount()
+        {
+            return statistics.tokenCount;
+        }
+
+        public int rangeCount()
+        {
+            return ranges.size();
+        }
+
+        public Builder<K, D> add(RangeIterator<K, D> range)
+        {
+            if (range == null || range.getMinimum() == null || range.getMaximum() == null)
+                return this;
+
+            ranges.add(range);
+            statistics.update(range);
+
+            return this;
+        }
+
+        public Builder<K, D> add(List<RangeIterator<K, D>> ranges)
+        {
+            if (ranges == null || ranges.isEmpty())
+                return this;
+
+            ranges.forEach(this::add);
+            return this;
+        }
+
+        public final RangeIterator<K, D> build()
+        {
+            switch (rangeCount())
+            {
+                case 0:
+                    return null;
+
+                case 1:
+                    return ranges.poll();
+
+                default:
+                    return buildIterator();
+            }
+        }
+
+        protected abstract RangeIterator<K, D> buildIterator();
+
+        public static class Statistics<K extends Comparable<K>, D extends CombinedValue<K>>
+        {
+            protected final IteratorType iteratorType;
+
+            protected K min, max;
+            protected long tokenCount;
+
+            // iterator with the least number of items
+            protected RangeIterator<K, D> minRange;
+            // iterator with the most number of items
+            protected RangeIterator<K, D> maxRange;
+
+            // tracks if all of the added ranges overlap, which is useful in case of intersection,
+            // as it gives direct answer as to such iterator is going to produce any results.
+            protected boolean isOverlapping = true;
+
+            public Statistics(IteratorType iteratorType)
+            {
+                this.iteratorType = iteratorType;
+            }
+
+            /**
+             * Update statistics information with the given range.
+             *
+             * Updates min/max of the combined range, token count and
+             * tracks range with the least/most number of tokens.
+             *
+             * @param range The range to update statistics with.
+             */
+            public void update(RangeIterator<K, D> range)
+            {
+                switch (iteratorType)
+                {
+                    case UNION:
+                        min = min == null || min.compareTo(range.getMinimum()) > 0 ? range.getMinimum() : min;
+                        max = max == null || max.compareTo(range.getMaximum()) < 0 ? range.getMaximum() : max;
+                        break;
+
+                    case INTERSECTION:
+                        // minimum of the intersection is the biggest minimum of individual iterators
+                        min = min == null || min.compareTo(range.getMinimum()) < 0 ? range.getMinimum() : min;
+                        // maximum of the intersection is the smallest maximum of individual iterators
+                        max = max == null || max.compareTo(range.getMaximum()) > 0 ? range.getMaximum() : max;
+                        break;
+
+                    default:
+                        throw new IllegalStateException("Unknown iterator type: " + iteratorType);
+                }
+
+                // check if new range is disjoint with already added ranges, which means that this intersection
+                // is not going to produce any results, so we can cleanup range storage and never added anything to it.
+                isOverlapping &= isOverlapping(min, max, range);
+
+                minRange = minRange == null ? range : min(minRange, range);
+                maxRange = maxRange == null ? range : max(maxRange, range);
+
+                tokenCount += range.getCount();
+
+            }
+
+            private RangeIterator<K, D> min(RangeIterator<K, D> a, RangeIterator<K, D> b)
+            {
+                return a.getCount() > b.getCount() ? b : a;
+            }
+
+            private RangeIterator<K, D> max(RangeIterator<K, D> a, RangeIterator<K, D> b)
+            {
+                return a.getCount() > b.getCount() ? a : b;
+            }
+
+            public boolean isDisjoint()
+            {
+                return !isOverlapping;
+            }
+
+            public double sizeRatio()
+            {
+                return minRange.getCount() * 1d / maxRange.getCount();
+            }
+        }
+    }
+
+    @VisibleForTesting
+    protected static <K extends Comparable<K>, D extends CombinedValue<K>> boolean isOverlapping(RangeIterator<K, D> a, RangeIterator<K, D> b)
+    {
+        return isOverlapping(a.getCurrent(), a.getMaximum(), b);
+    }
+
+    @VisibleForTesting
+    protected static <K extends Comparable<K>, D extends CombinedValue<K>> boolean isOverlapping(K min, K max, RangeIterator<K, D> b)
+    {
+        return min.compareTo(b.getMaximum()) <= 0 && b.getCurrent().compareTo(max) <= 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/RangeUnionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/RangeUnionIterator.java b/src/java/org/apache/cassandra/index/sasi/utils/RangeUnionIterator.java
new file mode 100644
index 0000000..310b6d0
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/RangeUnionIterator.java
@@ -0,0 +1,158 @@
+/*
+ * 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.index.sasi.utils;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.io.util.FileUtils;
+
+/**
+ * Range Union Iterator is used to return sorted stream of elements from multiple RangeIterator instances.
+ *
+ * PriorityQueue is used as a sorting mechanism for the ranges, where each computeNext() operation would poll
+ * from the queue (and push when done), which returns range that contains the smallest element, because
+ * sorting is done on the moving window of range iteration {@link RangeIterator#getCurrent()}. Once retrieved
+ * the smallest element (return candidate) is attempted to be merged with other ranges, because there could
+ * be equal elements in adjacent ranges, such ranges are poll'ed only if their {@link RangeIterator#getCurrent()}
+ * equals to the return candidate.
+ *
+ * @param <K> The type used to sort ranges.
+ * @param <D> The container type which is going to be returned by {@link Iterator#next()}.
+ */
+public class RangeUnionIterator<K extends Comparable<K>, D extends CombinedValue<K>> extends RangeIterator<K, D>
+{
+    private final PriorityQueue<RangeIterator<K, D>> ranges;
+
+    private RangeUnionIterator(Builder.Statistics<K, D> statistics, PriorityQueue<RangeIterator<K, D>> ranges)
+    {
+        super(statistics);
+        this.ranges = ranges;
+    }
+
+    public D computeNext()
+    {
+        RangeIterator<K, D> head = null;
+
+        while (!ranges.isEmpty())
+        {
+            head = ranges.poll();
+            if (head.hasNext())
+                break;
+
+            FileUtils.closeQuietly(head);
+        }
+
+        if (head == null || !head.hasNext())
+            return endOfData();
+
+        D candidate = head.next();
+
+        List<RangeIterator<K, D>> processedRanges = new ArrayList<>();
+
+        if (head.hasNext())
+            processedRanges.add(head);
+        else
+            FileUtils.closeQuietly(head);
+
+        while (!ranges.isEmpty())
+        {
+            // peek here instead of poll is an optimization
+            // so we can re-insert less ranges back if candidate
+            // is less than head of the current range.
+            RangeIterator<K, D> range = ranges.peek();
+
+            int cmp = candidate.get().compareTo(range.getCurrent());
+
+            assert cmp <= 0;
+
+            if (cmp < 0)
+            {
+                break; // candidate is smaller than next token, return immediately
+            }
+            else if (cmp == 0)
+            {
+                candidate.merge(range.next()); // consume and merge
+
+                range = ranges.poll();
+                // re-prioritize changed range
+
+                if (range.hasNext())
+                    processedRanges.add(range);
+                else
+                    FileUtils.closeQuietly(range);
+            }
+        }
+
+        ranges.addAll(processedRanges);
+        return candidate;
+    }
+
+    protected void performSkipTo(K nextToken)
+    {
+        List<RangeIterator<K, D>> changedRanges = new ArrayList<>();
+
+        while (!ranges.isEmpty())
+        {
+            if (ranges.peek().getCurrent().compareTo(nextToken) >= 0)
+                break;
+
+            RangeIterator<K, D> head = ranges.poll();
+
+            if (head.getMaximum().compareTo(nextToken) >= 0)
+            {
+                head.skipTo(nextToken);
+                changedRanges.add(head);
+                continue;
+            }
+
+            FileUtils.closeQuietly(head);
+        }
+
+        ranges.addAll(changedRanges.stream().collect(Collectors.toList()));
+    }
+
+    public void close() throws IOException
+    {
+        ranges.forEach(FileUtils::closeQuietly);
+    }
+
+    public static <K extends Comparable<K>, D extends CombinedValue<K>> Builder<K, D> builder()
+    {
+        return new Builder<>();
+    }
+
+    public static <K extends Comparable<K>, D extends CombinedValue<K>> RangeIterator<K, D> build(List<RangeIterator<K, D>> tokens)
+    {
+        return new Builder<K, D>().add(tokens).build();
+    }
+
+    public static class Builder<K extends Comparable<K>, D extends CombinedValue<K>> extends RangeIterator.Builder<K, D>
+    {
+        public Builder()
+        {
+            super(IteratorType.UNION);
+        }
+
+        protected RangeIterator<K, D> buildIterator()
+        {
+            return new RangeUnionIterator<>(statistics, ranges);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/TypeUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/TypeUtil.java b/src/java/org/apache/cassandra/index/sasi/utils/TypeUtil.java
new file mode 100644
index 0000000..8b38530
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/TypeUtil.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.index.sasi.utils;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.serializers.MarshalException;
+
+public class TypeUtil
+{
+    public static boolean isValid(ByteBuffer term, AbstractType<?> validator)
+    {
+        try
+        {
+            validator.validate(term);
+            return true;
+        }
+        catch (MarshalException e)
+        {
+            return false;
+        }
+    }
+
+    public static ByteBuffer tryUpcast(ByteBuffer term, AbstractType<?> validator)
+    {
+        if (term.remaining() == 0)
+            return null;
+
+        try
+        {
+            if (validator instanceof Int32Type && term.remaining() == 2)
+            {
+                return Int32Type.instance.decompose((int) term.getShort(term.position()));
+            }
+            else if (validator instanceof LongType)
+            {
+                long upcastToken;
+
+                switch (term.remaining())
+                {
+                    case 2:
+                        upcastToken = (long) term.getShort(term.position());
+                        break;
+
+                    case 4:
+                        upcastToken = (long) Int32Type.instance.compose(term);
+                        break;
+
+                    default:
+                        upcastToken = Long.valueOf(UTF8Type.instance.getString(term));
+                }
+
+                return LongType.instance.decompose(upcastToken);
+            }
+            else if (validator instanceof DoubleType && term.remaining() == 4)
+            {
+                return DoubleType.instance.decompose((double) FloatType.instance.compose(term));
+            }
+
+            // maybe it was a string after all
+            return validator.fromString(UTF8Type.instance.getString(term));
+        }
+        catch (Exception e)
+        {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/trie/AbstractPatriciaTrie.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/trie/AbstractPatriciaTrie.java b/src/java/org/apache/cassandra/index/sasi/utils/trie/AbstractPatriciaTrie.java
new file mode 100644
index 0000000..b359416
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/trie/AbstractPatriciaTrie.java
@@ -0,0 +1,1151 @@
+/*
+ * 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.
+ */
+
+/**
+ * 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;
+
+import java.util.AbstractCollection;
+import java.util.AbstractSet;
+import java.util.Collection;
+import java.util.ConcurrentModificationException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+import org.apache.cassandra.index.sasi.utils.trie.Cursor.Decision;
+
+/**
+ * This class implements the base PATRICIA algorithm and everything that
+ * is related to the {@link Map} interface.
+ */
+abstract class AbstractPatriciaTrie<K, V> extends AbstractTrie<K, V>
+{
+    private static final long serialVersionUID = -2303909182832019043L;
+
+    /**
+     * The root node of the {@link Trie}. 
+     */
+    final TrieEntry<K, V> root = new TrieEntry<>(null, null, -1);
+    
+    /**
+     * Each of these fields are initialized to contain an instance of the
+     * appropriate view the first time this view is requested. The views are
+     * stateless, so there's no reason to create more than one of each.
+     */
+    private transient volatile Set<K> keySet;
+    private transient volatile Collection<V> values;
+    private transient volatile Set<Map.Entry<K,V>> entrySet;
+    
+    /**
+     * The current size of the {@link Trie}
+     */
+    private int size = 0;
+    
+    /**
+     * The number of times this {@link Trie} has been modified.
+     * It's used to detect concurrent modifications and fail-fast
+     * the {@link Iterator}s.
+     */
+    transient int modCount = 0;
+    
+    public AbstractPatriciaTrie(KeyAnalyzer<? super K> keyAnalyzer)
+    {
+        super(keyAnalyzer);
+    }
+    
+    public AbstractPatriciaTrie(KeyAnalyzer<? super K> keyAnalyzer, Map<? extends K, ? extends V> m)
+    {
+        super(keyAnalyzer);
+        putAll(m);
+    }
+    
+    @Override
+    public void clear()
+    {
+        root.key = null;
+        root.bitIndex = -1;
+        root.value = null;
+        
+        root.parent = null;
+        root.left = root;
+        root.right = null;
+        root.predecessor = root;
+        
+        size = 0;
+        incrementModCount();
+    }
+    
+    @Override
+    public int size()
+    {
+        return size;
+    }
+   
+    /**
+     * A helper method to increment the {@link Trie} size
+     * and the modification counter.
+     */
+    void incrementSize()
+    {
+        size++;
+        incrementModCount();
+    }
+    
+    /**
+     * A helper method to decrement the {@link Trie} size
+     * and increment the modification counter.
+     */
+    void decrementSize()
+    {
+        size--;
+        incrementModCount();
+    }
+    
+    /**
+     * A helper method to increment the modification counter.
+     */
+    private void incrementModCount()
+    {
+        ++modCount;
+    }
+    
+    @Override
+    public V put(K key, V value)
+    {
+        if (key == null)
+            throw new NullPointerException("Key cannot be null");
+        
+        int lengthInBits = lengthInBits(key);
+        
+        // The only place to store a key with a length
+        // of zero bits is the root node
+        if (lengthInBits == 0)
+        {
+            if (root.isEmpty())
+                incrementSize();
+            else
+                incrementModCount();
+
+            return root.setKeyValue(key, value);
+        }
+        
+        TrieEntry<K, V> found = getNearestEntryForKey(key);
+        if (compareKeys(key, found.key))
+        {
+            if (found.isEmpty()) // <- must be the root
+                incrementSize();
+            else
+                incrementModCount();
+
+            return found.setKeyValue(key, value);
+        }
+        
+        int bitIndex = bitIndex(key, found.key);
+        if (!Tries.isOutOfBoundsIndex(bitIndex))
+        {
+            if (Tries.isValidBitIndex(bitIndex)) // in 99.999...9% the case
+            {
+                /* NEW KEY+VALUE TUPLE */
+                TrieEntry<K, V> t = new TrieEntry<>(key, value, bitIndex);
+                addEntry(t);
+                incrementSize();
+                return null;
+            }
+            else if (Tries.isNullBitKey(bitIndex))
+            {
+                // A bits of the Key are zero. The only place to
+                // store such a Key is the root Node!
+                
+                /* NULL BIT KEY */
+                if (root.isEmpty())
+                    incrementSize();
+                else
+                    incrementModCount();
+
+                return root.setKeyValue(key, value);
+                
+            }
+            else if (Tries.isEqualBitKey(bitIndex))
+            {
+                // This is a very special and rare case.
+                
+                /* REPLACE OLD KEY+VALUE */
+                if (found != root) {
+                    incrementModCount();
+                    return found.setKeyValue(key, value);
+                }
+            }
+        }
+        
+        throw new IndexOutOfBoundsException("Failed to put: " 
+                + key + " -> " + value + ", " + bitIndex);
+    }
+    
+    /**
+     * Adds the given {@link TrieEntry} to the {@link Trie}
+     */
+    TrieEntry<K, V> addEntry(TrieEntry<K, V> entry)
+    {
+        TrieEntry<K, V> current = root.left;
+        TrieEntry<K, V> path = root;
+
+        while(true)
+        {
+            if (current.bitIndex >= entry.bitIndex || current.bitIndex <= path.bitIndex)
+            {
+                entry.predecessor = entry;
+                
+                if (!isBitSet(entry.key, entry.bitIndex))
+                {
+                    entry.left = entry;
+                    entry.right = current;
+                }
+                else
+                {
+                    entry.left = current;
+                    entry.right = entry;
+                }
+               
+                entry.parent = path;
+                if (current.bitIndex >= entry.bitIndex)
+                    current.parent = entry;
+                
+                // if we inserted an uplink, set the predecessor on it
+                if (current.bitIndex <= path.bitIndex)
+                    current.predecessor = entry;
+         
+                if (path == root || !isBitSet(entry.key, path.bitIndex))
+                    path.left = entry;
+                else
+                    path.right = entry;
+                
+                return entry;
+            }
+                
+            path = current;
+            
+            current = !isBitSet(entry.key, current.bitIndex)
+                       ? current.left : current.right;
+        }
+    }
+    
+    @Override
+    public V get(Object k)
+    {
+        TrieEntry<K, V> entry = getEntry(k);
+        return entry != null ? entry.getValue() : null;
+    }
+
+    /**
+     * Returns the entry associated with the specified key in the
+     * AbstractPatriciaTrie.  Returns null if the map contains no mapping
+     * for this key.
+     * 
+     * This may throw ClassCastException if the object is not of type K.
+     */
+    TrieEntry<K,V> getEntry(Object k)
+    {
+        K key = Tries.cast(k);
+        if (key == null)
+            return null;
+        
+        TrieEntry<K,V> entry = getNearestEntryForKey(key);
+        return !entry.isEmpty() && compareKeys(key, entry.key) ? entry : null;
+    }
+    
+    @Override
+    public Map.Entry<K, V> select(K key)
+    {
+        Reference<Map.Entry<K, V>> reference = new Reference<>();
+        return !selectR(root.left, -1, key, reference) ? reference.get() : null;
+    }
+    
+    @Override
+    public Map.Entry<K,V> select(K key, Cursor<? super K, ? super V> cursor)
+    {
+        Reference<Map.Entry<K, V>> reference = new Reference<>();
+        selectR(root.left, -1, key, cursor, reference);
+        return reference.get();
+    }
+
+    /**
+     * This is equivalent to the other {@link #selectR(TrieEntry, int,
+     * K, Cursor, Reference)} method but without its overhead
+     * because we're selecting only one best matching Entry from the
+     * {@link Trie}.
+     */
+    private boolean selectR(TrieEntry<K, V> h, int bitIndex, final K key, final Reference<Map.Entry<K, V>> reference)
+    {
+        if (h.bitIndex <= bitIndex)
+        {
+            // If we hit the root Node and it is empty
+            // we have to look for an alternative best
+            // matching node.
+            if (!h.isEmpty())
+            {
+                reference.set(h);
+                return false;
+            }
+            return true;
+        }
+
+        if (!isBitSet(key, h.bitIndex))
+        {
+            if (selectR(h.left, h.bitIndex, key, reference))
+            {
+                return selectR(h.right, h.bitIndex, key, reference);
+            }
+        }
+        else
+        {
+            if (selectR(h.right, h.bitIndex, key, reference))
+            {
+                return selectR(h.left, h.bitIndex, key, reference);
+            }
+        }
+
+        return false;
+    }
+    
+    /**
+     * 
+     */
+    private boolean selectR(TrieEntry<K,V> h, int bitIndex, 
+                            final K key, final Cursor<? super K, ? super V> cursor,
+                            final Reference<Map.Entry<K, V>> reference)
+    {
+        if (h.bitIndex <= bitIndex)
+        {
+            if (!h.isEmpty())
+            {
+                Decision decision = cursor.select(h);
+                switch(decision)
+                {
+                    case REMOVE:
+                        throw new UnsupportedOperationException("Cannot remove during select");
+
+                    case EXIT:
+                        reference.set(h);
+                        return false; // exit
+
+                    case REMOVE_AND_EXIT:
+                        TrieEntry<K, V> entry = new TrieEntry<>(h.getKey(), h.getValue(), -1);
+                        reference.set(entry);
+                        removeEntry(h);
+                        return false;
+
+                    case CONTINUE:
+                        // fall through.
+                }
+            }
+
+            return true; // continue
+        }
+
+        if (!isBitSet(key, h.bitIndex))
+        {
+            if (selectR(h.left, h.bitIndex, key, cursor, reference))
+            {
+                return selectR(h.right, h.bitIndex, key, cursor, reference);
+            }
+        }
+        else
+        {
+            if (selectR(h.right, h.bitIndex, key, cursor, reference))
+            {
+                return selectR(h.left, h.bitIndex, key, cursor, reference);
+            }
+        }
+        
+        return false;
+    }
+    
+    @Override
+    public Map.Entry<K, V> traverse(Cursor<? super K, ? super V> cursor)
+    {
+        TrieEntry<K, V> entry = nextEntry(null);
+        while (entry != null)
+        {
+            TrieEntry<K, V> current = entry;
+            
+            Decision decision = cursor.select(current);
+            entry = nextEntry(current);
+            
+            switch(decision)
+            {
+                case EXIT:
+                    return current;
+
+                case REMOVE:
+                    removeEntry(current);
+                    break; // out of switch, stay in while loop
+
+                case REMOVE_AND_EXIT:
+                    Map.Entry<K, V> value = new TrieEntry<>(current.getKey(), current.getValue(), -1);
+                    removeEntry(current);
+                    return value;
+
+                case CONTINUE: // do nothing.
+            }
+        }
+        
+        return null;
+    }
+    
+    @Override
+    public boolean containsKey(Object k)
+    {
+        if (k == null)
+            return false;
+        
+        K key = Tries.cast(k);
+        TrieEntry<K, V> entry = getNearestEntryForKey(key);
+        return !entry.isEmpty() && compareKeys(key, entry.key);
+    }
+    
+    @Override
+    public Set<Map.Entry<K,V>> entrySet()
+    {
+        if (entrySet == null)
+            entrySet = new EntrySet();
+
+        return entrySet;
+    }
+    
+    @Override
+    public Set<K> keySet()
+    {
+        if (keySet == null)
+            keySet = new KeySet();
+        return keySet;
+    }
+    
+    @Override
+    public Collection<V> values()
+    {
+        if (values == null)
+            values = new Values();
+        return values;
+    }
+    
+    /**
+     * {@inheritDoc}
+     * 
+     * @throws ClassCastException if provided key is of an incompatible type 
+     */
+    @Override
+    public V remove(Object k)
+    {
+        if (k == null)
+            return null;
+        
+        K key = Tries.cast(k);
+        TrieEntry<K, V> current = root.left;
+        TrieEntry<K, V> path = root;
+        while (true)
+        {
+            if (current.bitIndex <= path.bitIndex)
+            {
+                if (!current.isEmpty() && compareKeys(key, current.key))
+                {
+                    return removeEntry(current);
+                }
+                else
+                {
+                    return null;
+                }
+            }
+            
+            path = current;
+            current = !isBitSet(key, current.bitIndex) ? current.left : current.right;
+        }
+    }
+    
+    /**
+     * Returns the nearest entry for a given key.  This is useful
+     * for finding knowing if a given key exists (and finding the value
+     * for it), or for inserting the key.
+     * 
+     * The actual get implementation. This is very similar to
+     * selectR but with the exception that it might return the
+     * root Entry even if it's empty.
+     */
+    TrieEntry<K, V> getNearestEntryForKey(K key)
+    {
+        TrieEntry<K, V> current = root.left;
+        TrieEntry<K, V> path = root;
+
+        while(true)
+        {
+            if (current.bitIndex <= path.bitIndex)
+                return current;
+            
+            path = current;
+            current = !isBitSet(key, current.bitIndex) ? current.left : current.right;
+        }
+    }
+    
+    /**
+     * Removes a single entry from the {@link Trie}.
+     * 
+     * If we found a Key (Entry h) then figure out if it's
+     * an internal (hard to remove) or external Entry (easy 
+     * to remove)
+     */
+    V removeEntry(TrieEntry<K, V> h)
+    {
+        if (h != root)
+        {
+            if (h.isInternalNode())
+            {
+                removeInternalEntry(h);
+            }
+            else
+            {
+                removeExternalEntry(h);
+            }
+        }
+        
+        decrementSize();
+        return h.setKeyValue(null, null);
+    }
+    
+    /**
+     * Removes an external entry from the {@link Trie}.
+     * 
+     * If it's an external Entry then just remove it.
+     * This is very easy and straight forward.
+     */
+    private void removeExternalEntry(TrieEntry<K, V> h)
+    {
+        if (h == root)
+        {
+            throw new IllegalArgumentException("Cannot delete root Entry!");
+        }
+        else if (!h.isExternalNode())
+        {
+            throw new IllegalArgumentException(h + " is not an external Entry!");
+        } 
+        
+        TrieEntry<K, V> parent = h.parent;
+        TrieEntry<K, V> child = (h.left == h) ? h.right : h.left;
+        
+        if (parent.left == h)
+        {
+            parent.left = child;
+        }
+        else
+        {
+            parent.right = child;
+        }
+        
+        // either the parent is changing, or the predecessor is changing.
+        if (child.bitIndex > parent.bitIndex)
+        {
+            child.parent = parent;
+        }
+        else
+        {
+            child.predecessor = parent;
+        }
+        
+    }
+    
+    /**
+     * Removes an internal entry from the {@link Trie}.
+     * 
+     * If it's an internal Entry then "good luck" with understanding
+     * this code. The Idea is essentially that Entry p takes Entry h's
+     * place in the trie which requires some re-wiring.
+     */
+    private void removeInternalEntry(TrieEntry<K, V> h)
+    {
+        if (h == root)
+        {
+            throw new IllegalArgumentException("Cannot delete root Entry!");
+        }
+        else if (!h.isInternalNode())
+        {
+            throw new IllegalArgumentException(h + " is not an internal Entry!");
+        } 
+        
+        TrieEntry<K, V> p = h.predecessor;
+        
+        // Set P's bitIndex
+        p.bitIndex = h.bitIndex;
+        
+        // Fix P's parent, predecessor and child Nodes
+        {
+            TrieEntry<K, V> parent = p.parent;
+            TrieEntry<K, V> child = (p.left == h) ? p.right : p.left;
+            
+            // if it was looping to itself previously,
+            // it will now be pointed from it's parent
+            // (if we aren't removing it's parent --
+            //  in that case, it remains looping to itself).
+            // otherwise, it will continue to have the same
+            // predecessor.
+            if (p.predecessor == p && p.parent != h)
+                p.predecessor = p.parent;
+            
+            if (parent.left == p)
+            {
+                parent.left = child;
+            }
+            else
+            {
+                parent.right = child;
+            }
+            
+            if (child.bitIndex > parent.bitIndex)
+            {
+                child.parent = parent;
+            }
+        }
+        
+        // Fix H's parent and child Nodes
+        {         
+            // If H is a parent of its left and right child 
+            // then change them to P
+            if (h.left.parent == h)
+                h.left.parent = p;
+
+            if (h.right.parent == h)
+                h.right.parent = p;
+            
+            // Change H's parent
+            if (h.parent.left == h)
+            {
+                h.parent.left = p;
+            }
+            else
+            {
+                h.parent.right = p;
+            }
+        }
+        
+        // Copy the remaining fields from H to P
+        //p.bitIndex = h.bitIndex;
+        p.parent = h.parent;
+        p.left = h.left;
+        p.right = h.right;
+        
+        // Make sure that if h was pointing to any uplinks,
+        // p now points to them.
+        if (isValidUplink(p.left, p))
+            p.left.predecessor = p;
+        
+        if (isValidUplink(p.right, p))
+            p.right.predecessor = p;
+    }
+    
+    /**
+     * Returns the entry lexicographically after the given entry.
+     * If the given entry is null, returns the first node.
+     */
+    TrieEntry<K, V> nextEntry(TrieEntry<K, V> node)
+    {
+        return (node == null) ? firstEntry() : nextEntryImpl(node.predecessor, node, null);
+    }
+    
+    /**
+     * Scans for the next node, starting at the specified point, and using 'previous'
+     * as a hint that the last node we returned was 'previous' (so we know not to return
+     * it again).  If 'tree' is non-null, this will limit the search to the given tree.
+     * 
+     * The basic premise is that each iteration can follow the following steps:
+     * 
+     * 1) Scan all the way to the left.
+     *   a) If we already started from this node last time, proceed to Step 2.
+     *   b) If a valid uplink is found, use it.
+     *   c) If the result is an empty node (root not set), break the scan.
+     *   d) If we already returned the left node, break the scan.
+     *   
+     * 2) Check the right.
+     *   a) If we already returned the right node, proceed to Step 3.
+     *   b) If it is a valid uplink, use it.
+     *   c) Do Step 1 from the right node.
+     *   
+     * 3) Back up through the parents until we encounter find a parent
+     *    that we're not the right child of.
+     *    
+     * 4) If there's no right child of that parent, the iteration is finished.
+     *    Otherwise continue to Step 5.
+     * 
+     * 5) Check to see if the right child is a valid uplink.
+     *    a) If we already returned that child, proceed to Step 6.
+     *       Otherwise, use it.
+     *    
+     * 6) If the right child of the parent is the parent itself, we've
+     *    already found & returned the end of the Trie, so exit.
+     *    
+     * 7) Do Step 1 on the parent's right child.
+     */
+    TrieEntry<K, V> nextEntryImpl(TrieEntry<K, V> start, TrieEntry<K, V> previous, TrieEntry<K, V> tree)
+    {
+        TrieEntry<K, V> current = start;
+
+        // Only look at the left if this was a recursive or
+        // the first check, otherwise we know we've already looked
+        // at the left.
+        if (previous == null || start != previous.predecessor)
+        {
+            while (!current.left.isEmpty())
+            {
+                // stop traversing if we've already
+                // returned the left of this node.
+                if (previous == current.left)
+                    break;
+                
+                if (isValidUplink(current.left, current))
+                    return current.left;
+                
+                current = current.left;
+            }
+        }
+        
+        // If there's no data at all, exit.
+        if (current.isEmpty())
+            return null;
+        
+        // If we've already returned the left,
+        // and the immediate right is null,
+        // there's only one entry in the Trie
+        // which is stored at the root.
+        //
+        //  / ("")   <-- root
+        //  \_/  \
+        //       null <-- 'current'
+        //
+        if (current.right == null)
+            return null;
+        
+        // If nothing valid on the left, try the right.
+        if (previous != current.right)
+        {
+            // See if it immediately is valid.
+            if (isValidUplink(current.right, current))
+                return current.right;
+            
+            // Must search on the right's side if it wasn't initially valid.
+            return nextEntryImpl(current.right, previous, tree);
+        }
+        
+        // Neither left nor right are valid, find the first parent
+        // whose child did not come from the right & traverse it.
+        while (current == current.parent.right)
+        {
+            // If we're going to traverse to above the subtree, stop.
+            if (current == tree)
+                return null;
+            
+            current = current.parent;
+        }
+
+        // If we're on the top of the subtree, we can't go any higher.
+        if (current == tree)
+            return null;
+        
+        // If there's no right, the parent must be root, so we're done.
+        if (current.parent.right == null)
+            return null;
+        
+        // If the parent's right points to itself, we've found one.
+        if (previous != current.parent.right && isValidUplink(current.parent.right, current.parent))
+            return current.parent.right;
+        
+        // If the parent's right is itself, there can't be any more nodes.
+        if (current.parent.right == current.parent)
+            return null;
+        
+        // We need to traverse down the parent's right's path.
+        return nextEntryImpl(current.parent.right, previous, tree);
+    }
+    
+    /**
+     * Returns the first entry the {@link Trie} is storing.
+     * 
+     * This is implemented by going always to the left until
+     * we encounter a valid uplink. That uplink is the first key.
+     */
+    TrieEntry<K, V> firstEntry()
+    {
+        // if Trie is empty, no first node.
+        return isEmpty() ? null : followLeft(root);
+    }
+    
+    /** 
+     * Goes left through the tree until it finds a valid node. 
+     */
+    TrieEntry<K, V> followLeft(TrieEntry<K, V> node)
+    {
+        while(true)
+        {
+            TrieEntry<K, V> child = node.left;
+            // if we hit root and it didn't have a node, go right instead.
+            if (child.isEmpty())
+                child = node.right;
+            
+            if (child.bitIndex <= node.bitIndex)
+                return child;
+            
+            node = child;
+        }
+    }
+    
+    /** 
+     * Returns true if 'next' is a valid uplink coming from 'from'. 
+     */
+    static boolean isValidUplink(TrieEntry<?, ?> next, TrieEntry<?, ?> from)
+    {
+        return next != null && next.bitIndex <= from.bitIndex && !next.isEmpty();
+    }
+    
+    /**
+     * A {@link Reference} allows us to return something through a Method's 
+     * argument list. An alternative would be to an Array with a length of 
+     * one (1) but that leads to compiler warnings. Computationally and memory
+     * wise there's no difference (except for the need to load the 
+     * {@link Reference} Class but that happens only once).
+     */
+    private static class Reference<E>
+    {
+        
+        private E item;
+        
+        public void set(E item)
+        {
+            this.item = item;
+        }
+        
+        public E get()
+        {
+            return item;
+        }
+    }
+    
+    /**
+     *  A {@link Trie} is a set of {@link TrieEntry} nodes
+     */
+    static class TrieEntry<K,V> extends BasicEntry<K, V>
+    {
+        
+        private static final long serialVersionUID = 4596023148184140013L;
+        
+        /** The index this entry is comparing. */
+        protected int bitIndex;
+        
+        /** The parent of this entry. */
+        protected TrieEntry<K,V> parent;
+        
+        /** The left child of this entry. */
+        protected TrieEntry<K,V> left;
+        
+        /** The right child of this entry. */
+        protected TrieEntry<K,V> right;
+        
+        /** The entry who uplinks to this entry. */ 
+        protected TrieEntry<K,V> predecessor;
+        
+        public TrieEntry(K key, V value, int bitIndex)
+        {
+            super(key, value);
+            
+            this.bitIndex = bitIndex;
+            
+            this.parent = null;
+            this.left = this;
+            this.right = null;
+            this.predecessor = this;
+        }
+        
+        /**
+         * Whether or not the entry is storing a key.
+         * Only the root can potentially be empty, all other
+         * nodes must have a key.
+         */
+        public boolean isEmpty()
+        {
+            return key == null;
+        }
+        
+        /** 
+         * Neither the left nor right child is a loopback 
+         */
+        public boolean isInternalNode()
+        {
+            return left != this && right != this;
+        }
+        
+        /** 
+         * Either the left or right child is a loopback 
+         */
+        public boolean isExternalNode()
+        {
+            return !isInternalNode();
+        }
+    }
+    
+
+    /**
+     * This is a entry set view of the {@link Trie} as returned 
+     * by {@link Map#entrySet()}
+     */
+    private class EntrySet extends AbstractSet<Map.Entry<K,V>>
+    {
+        @Override
+        public Iterator<Map.Entry<K,V>> iterator()
+        {
+            return new EntryIterator();
+        }
+        
+        @Override
+        public boolean contains(Object o)
+        {
+            if (!(o instanceof Map.Entry))
+                return false;
+            
+            TrieEntry<K,V> candidate = getEntry(((Map.Entry<?, ?>)o).getKey());
+            return candidate != null && candidate.equals(o);
+        }
+        
+        @Override
+        public boolean remove(Object o)
+        {
+            int size = size();
+            AbstractPatriciaTrie.this.remove(o);
+            return size != size();
+        }
+        
+        @Override
+        public int size()
+        {
+            return AbstractPatriciaTrie.this.size();
+        }
+        
+        @Override
+        public void clear()
+        {
+            AbstractPatriciaTrie.this.clear();
+        }
+        
+        /**
+         * An {@link Iterator} that returns {@link Entry} Objects
+         */
+        private class EntryIterator extends TrieIterator<Map.Entry<K,V>>
+        {
+            @Override
+            public Map.Entry<K,V> next()
+            {
+                return nextEntry();
+            }
+        }
+    }
+    
+    /**
+     * This is a key set view of the {@link Trie} as returned 
+     * by {@link Map#keySet()}
+     */
+    private class KeySet extends AbstractSet<K>
+    {
+        @Override
+        public Iterator<K> iterator()
+        {
+            return new KeyIterator();
+        }
+        
+        @Override
+        public int size()
+        {
+            return AbstractPatriciaTrie.this.size();
+        }
+        
+        @Override
+        public boolean contains(Object o)
+        {
+            return containsKey(o);
+        }
+        
+        @Override
+        public boolean remove(Object o)
+        {
+            int size = size();
+            AbstractPatriciaTrie.this.remove(o);
+            return size != size();
+        }
+        
+        @Override
+        public void clear()
+        {
+            AbstractPatriciaTrie.this.clear();
+        }
+        
+        /**
+         * An {@link Iterator} that returns Key Objects
+         */
+        private class KeyIterator extends TrieIterator<K>
+        {
+            @Override
+            public K next()
+            {
+                return nextEntry().getKey();
+            }
+        }
+    }
+    
+    /**
+     * This is a value view of the {@link Trie} as returned 
+     * by {@link Map#values()}
+     */
+    private class Values extends AbstractCollection<V>
+    {
+        @Override
+        public Iterator<V> iterator()
+        {
+            return new ValueIterator();
+        }
+        
+        @Override
+        public int size()
+        {
+            return AbstractPatriciaTrie.this.size();
+        }
+        
+        @Override
+        public boolean contains(Object o)
+        {
+            return containsValue(o);
+        }
+        
+        @Override
+        public void clear()
+        {
+            AbstractPatriciaTrie.this.clear();
+        }
+        
+        @Override
+        public boolean remove(Object o)
+        {
+            for (Iterator<V> it = iterator(); it.hasNext(); )
+            {
+                V value = it.next();
+                if (Tries.areEqual(value, o))
+                {
+                    it.remove();
+                    return true;
+                }
+            }
+            return false;
+        }
+        
+        /**
+         * An {@link Iterator} that returns Value Objects
+         */
+        private class ValueIterator extends TrieIterator<V>
+        {
+            @Override
+            public V next()
+            {
+                return nextEntry().getValue();
+            }
+        }
+    }
+    
+    /** 
+     * An iterator for the entries. 
+     */
+    abstract class TrieIterator<E> implements Iterator<E>
+    {
+        /**
+         * For fast-fail
+         */
+        protected int expectedModCount = AbstractPatriciaTrie.this.modCount;
+        
+        protected TrieEntry<K, V> next; // the next node to return
+        protected TrieEntry<K, V> current; // the current entry we're on
+        
+        /**
+         * Starts iteration from the root
+         */
+        protected TrieIterator()
+        {
+            next = AbstractPatriciaTrie.this.nextEntry(null);
+        }
+        
+        /**
+         * Starts iteration at the given entry
+         */
+        protected TrieIterator(TrieEntry<K, V> firstEntry)
+        {
+            next = firstEntry;
+        }
+        
+        /**
+         * Returns the next {@link TrieEntry}
+         */
+        protected TrieEntry<K,V> nextEntry()
+        {
+            if (expectedModCount != AbstractPatriciaTrie.this.modCount)
+                throw new ConcurrentModificationException();
+            
+            TrieEntry<K,V> e = next;
+            if (e == null)
+                throw new NoSuchElementException();
+            
+            next = findNext(e);
+            current = e;
+            return e;
+        }
+        
+        /**
+         * @see PatriciaTrie#nextEntry(TrieEntry)
+         */
+        protected TrieEntry<K, V> findNext(TrieEntry<K, V> prior)
+        {
+            return AbstractPatriciaTrie.this.nextEntry(prior);
+        }
+        
+        @Override
+        public boolean hasNext()
+        {
+            return next != null;
+        }
+        
+        @Override
+        public void remove()
+        {
+            if (current == null)
+                throw new IllegalStateException();
+            
+            if (expectedModCount != AbstractPatriciaTrie.this.modCount)
+                throw new ConcurrentModificationException();
+            
+            TrieEntry<K, V> node = current;
+            current = null;
+            AbstractPatriciaTrie.this.removeEntry(node);
+            
+            expectedModCount = AbstractPatriciaTrie.this.modCount;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/trie/AbstractTrie.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/trie/AbstractTrie.java b/src/java/org/apache/cassandra/index/sasi/utils/trie/AbstractTrie.java
new file mode 100644
index 0000000..0bf9c20
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/trie/AbstractTrie.java
@@ -0,0 +1,230 @@
+/*
+ * 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.AbstractMap;
+import java.util.Map;
+
+/**
+ * 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.
+ */
+
+/**
+ * This class provides some basic {@link Trie} functionality and 
+ * utility methods for actual {@link Trie} implementations.
+ */
+abstract class AbstractTrie<K, V> extends AbstractMap<K, V> implements Serializable, Trie<K, V>
+{
+    private static final long serialVersionUID = -6358111100045408883L;
+    
+    /**
+     * The {@link KeyAnalyzer} that's being used to build the 
+     * PATRICIA {@link Trie}
+     */
+    protected final KeyAnalyzer<? super K> keyAnalyzer;
+    
+    /** 
+     * Constructs a new {@link Trie} using the given {@link KeyAnalyzer} 
+     */
+    public AbstractTrie(KeyAnalyzer<? super K> keyAnalyzer)
+    {
+        this.keyAnalyzer = Tries.notNull(keyAnalyzer, "keyAnalyzer");
+    }
+    
+    @Override
+    public K selectKey(K key)
+    {
+        Map.Entry<K, V> entry = select(key);
+        return entry != null ? entry.getKey() : null;
+    }
+    
+    @Override
+    public V selectValue(K key)
+    {
+        Map.Entry<K, V> entry = select(key);
+        return entry != null ? entry.getValue() : null;
+    }
+        
+    @Override
+    public String toString()
+    {
+        StringBuilder buffer = new StringBuilder();
+        buffer.append("Trie[").append(size()).append("]={\n");
+        for (Map.Entry<K, V> entry : entrySet()) {
+            buffer.append("  ").append(entry).append("\n");
+        }
+        buffer.append("}\n");
+        return buffer.toString();
+    }
+    
+    /**
+     * Returns the length of the given key in bits
+     * 
+     * @see KeyAnalyzer#lengthInBits(Object)
+     */
+    final int lengthInBits(K key)
+    {
+        return key == null ? 0 : keyAnalyzer.lengthInBits(key);
+    }
+    
+    /**
+     * Returns whether or not the given bit on the 
+     * key is set or false if the key is null.
+     * 
+     * @see KeyAnalyzer#isBitSet(Object, int)
+     */
+    final boolean isBitSet(K key, int bitIndex)
+    {
+        return key != null && keyAnalyzer.isBitSet(key, bitIndex);
+    }
+    
+    /**
+     * Utility method for calling {@link KeyAnalyzer#bitIndex(Object, Object)}
+     */
+    final int bitIndex(K key, K otherKey)
+    {
+        if (key != null && otherKey != null)
+        {
+            return keyAnalyzer.bitIndex(key, otherKey);            
+        }
+        else if (key != null)
+        {
+            return bitIndex(key);
+        }
+        else if (otherKey != null)
+        {
+            return bitIndex(otherKey);
+        }
+        
+        return KeyAnalyzer.NULL_BIT_KEY;
+    }
+    
+    private int bitIndex(K key)
+    {
+        int lengthInBits = lengthInBits(key);
+        for (int i = 0; i < lengthInBits; i++)
+        {
+            if (isBitSet(key, i))
+                return i;
+        }
+        
+        return KeyAnalyzer.NULL_BIT_KEY;
+    }
+    
+    /**
+     * An utility method for calling {@link KeyAnalyzer#compare(Object, Object)}
+     */
+    final boolean compareKeys(K key, K other)
+    {
+        if (key == null)
+        {
+            return (other == null);
+        }
+        else if (other == null)
+        {
+            return false;
+        }
+        
+        return keyAnalyzer.compare(key, other) == 0;
+    }
+    
+    /**
+     * A basic implementation of {@link Entry}
+     */
+    abstract static class BasicEntry<K, V> implements Map.Entry<K, V>, Serializable
+    {
+        private static final long serialVersionUID = -944364551314110330L;
+
+        protected K key;
+        
+        protected V value;
+        
+        private transient int hashCode = 0;
+        
+        public BasicEntry(K key, V value)
+        {
+            this.key = key;
+            this.value = value;
+        }
+        
+        /**
+         * Replaces the current key and value with the provided
+         * key &amp; value
+         */
+        public V setKeyValue(K key, V value)
+        {
+            this.key = key;
+            this.hashCode = 0;
+            return setValue(value);
+        }
+        
+        @Override
+        public K getKey()
+        {
+            return key;
+        }
+        
+        @Override
+        public V getValue()
+        {
+            return value;
+        }
+        
+        @Override
+        public V setValue(V value)
+        {
+            V previous = this.value;
+            this.value = value;
+            return previous;
+        }
+        
+        @Override
+        public int hashCode()
+        {
+            if (hashCode == 0)
+                hashCode = (key != null ? key.hashCode() : 0);
+            return hashCode;
+        }
+        
+        @Override
+        public boolean equals(Object o)
+        {
+            if (o == this)
+            {
+                return true;
+            }
+            else if (!(o instanceof Map.Entry<?, ?>))
+            {
+                return false;
+            }
+            
+            Map.Entry<?, ?> other = (Map.Entry<?, ?>)o;
+            return Tries.areEqual(key, other.getKey()) && Tries.areEqual(value, other.getValue());
+        }
+        
+        @Override
+        public String toString()
+        {
+            return key + "=" + value;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/utils/trie/Cursor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/trie/Cursor.java b/src/java/org/apache/cassandra/index/sasi/utils/trie/Cursor.java
new file mode 100644
index 0000000..513fae0
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/trie/Cursor.java
@@ -0,0 +1,83 @@
+/*
+ * 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.Map.Entry;
+
+/**
+ * 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.
+ */
+
+/**
+ * A {@link Cursor} can be used to traverse a {@link Trie}, visit each node 
+ * step by step and make {@link Decision}s on each step how to continue with 
+ * traversing the {@link Trie}.
+ */
+public interface Cursor<K, V>
+{
+    
+    /**
+     * The {@link Decision} tells the {@link Cursor} what to do on each step 
+     * while traversing the {@link Trie}.
+     * 
+     * NOTE: Not all operations that work with a {@link Cursor} support all 
+     * {@link Decision} types
+     */
+    enum Decision
+    {
+        
+        /**
+         * Exit the traverse operation
+         */
+        EXIT, 
+        
+        /**
+         * Continue with the traverse operation
+         */
+        CONTINUE, 
+        
+        /**
+         * Remove the previously returned element
+         * from the {@link Trie} and continue
+         */
+        REMOVE, 
+        
+        /**
+         * Remove the previously returned element
+         * from the {@link Trie} and exit from the
+         * traverse operation
+         */
+        REMOVE_AND_EXIT
+    }
+    
+    /**
+     * Called for each {@link Entry} in the {@link Trie}. Return 
+     * {@link Decision#EXIT} to finish the {@link Trie} operation,
+     * {@link Decision#CONTINUE} to go to the next {@link Entry},
+     * {@link Decision#REMOVE} to remove the {@link Entry} and
+     * continue iterating or {@link Decision#REMOVE_AND_EXIT} to
+     * remove the {@link Entry} and stop iterating.
+     * 
+     * Note: Not all operations support {@link Decision#REMOVE}.
+     */
+    Decision select(Map.Entry<? extends K, ? extends V> entry);
+}
\ 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/KeyAnalyzer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/trie/KeyAnalyzer.java b/src/java/org/apache/cassandra/index/sasi/utils/trie/KeyAnalyzer.java
new file mode 100644
index 0000000..9cab4ae
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/utils/trie/KeyAnalyzer.java
@@ -0,0 +1,73 @@
+/*
+ * Copyright 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.
+ */
+
+package org.apache.cassandra.index.sasi.utils.trie;
+
+import java.util.Comparator;
+
+/**
+ * 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.
+ */
+
+/**
+ * The {@link KeyAnalyzer} provides bit-level access to keys
+ * for the {@link PatriciaTrie}.
+ */
+public interface KeyAnalyzer<K> extends Comparator<K>
+{
+    /**
+     * Returned by {@link #bitIndex(Object, Object)} if a key's
+     * bits were all zero (0).
+     */
+    int NULL_BIT_KEY = -1;
+    
+    /** 
+     * Returned by {@link #bitIndex(Object, Object)} if a the
+     * bits of two keys were all equal.
+     */
+    int EQUAL_BIT_KEY = -2;
+    
+    /**
+     * Returned by {@link #bitIndex(Object, Object)} if a keys 
+     * indices are out of bounds.
+     */
+    int OUT_OF_BOUNDS_BIT_KEY = -3;
+    
+    /**
+     * Returns the key's length in bits.
+     */
+    int lengthInBits(K key);
+    
+    /**
+     * Returns {@code true} if a key's bit it set at the given index.
+     */
+    boolean isBitSet(K key, int bitIndex);
+    
+    /**
+     * Returns the index of the first bit that is different in the two keys.
+     */
+    int bitIndex(K key, K otherKey);
+    
+    /**
+     * Returns {@code true} if the second argument is a 
+     * prefix of the first argument.
+     */
+    boolean isPrefix(K key, K prefix);
+}


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

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/conf/view/PrefixTermTree.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/view/PrefixTermTree.java b/src/java/org/apache/cassandra/index/sasi/conf/view/PrefixTermTree.java
new file mode 100644
index 0000000..72b6daf
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/conf/view/PrefixTermTree.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.index.sasi.conf.view;
+
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.cassandra.index.sasi.SSTableIndex;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.index.sasi.utils.trie.KeyAnalyzer;
+import org.apache.cassandra.index.sasi.utils.trie.PatriciaTrie;
+import org.apache.cassandra.index.sasi.utils.trie.Trie;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.utils.Interval;
+import org.apache.cassandra.utils.IntervalTree;
+
+import com.google.common.collect.Sets;
+
+/**
+ * This class is an extension over RangeTermTree for string terms,
+ * it is required because interval tree can't handle matching if search is on the
+ * prefix of min/max of the range, so for ascii/utf8 fields we build an additional
+ * prefix trie (including both min/max terms of the index) and do union of the results
+ * of the prefix tree search and results from the interval tree lookup.
+ */
+public class PrefixTermTree extends RangeTermTree
+{
+    private final OnDiskIndexBuilder.Mode mode;
+    private final Trie<ByteBuffer, Set<SSTableIndex>> trie;
+
+    public PrefixTermTree(ByteBuffer min, ByteBuffer max,
+                          Trie<ByteBuffer, Set<SSTableIndex>> trie,
+                          IntervalTree<ByteBuffer, SSTableIndex, Interval<ByteBuffer, SSTableIndex>> ranges,
+                          OnDiskIndexBuilder.Mode mode)
+    {
+        super(min, max, ranges);
+
+        this.mode = mode;
+        this.trie = trie;
+    }
+
+    public Set<SSTableIndex> search(Expression e)
+    {
+        Map<ByteBuffer, Set<SSTableIndex>> indexes = (e == null || e.lower == null || mode == OnDiskIndexBuilder.Mode.CONTAINS)
+                                                        ? trie : trie.prefixMap(e.lower.value);
+
+        Set<SSTableIndex> view = new HashSet<>(indexes.size());
+        indexes.values().forEach(view::addAll);
+
+        return Sets.union(view, super.search(e));
+    }
+
+    public static class Builder extends RangeTermTree.Builder
+    {
+        private final PatriciaTrie<ByteBuffer, Set<SSTableIndex>> trie;
+
+        protected Builder(OnDiskIndexBuilder.Mode mode, final AbstractType<?> comparator)
+        {
+            super(mode, comparator);
+            trie = new PatriciaTrie<>(new ByteBufferKeyAnalyzer(comparator));
+        }
+
+        public void addIndex(SSTableIndex index)
+        {
+            super.addIndex(index);
+            addTerm(index.minTerm(), index);
+            addTerm(index.maxTerm(), index);
+        }
+
+        public TermTree build()
+        {
+            return new PrefixTermTree(min, max, trie, IntervalTree.build(intervals), mode);
+        }
+
+        private void addTerm(ByteBuffer term, SSTableIndex index)
+        {
+            Set<SSTableIndex> indexes = trie.get(term);
+            if (indexes == null)
+                trie.put(term, (indexes = new HashSet<>()));
+
+            indexes.add(index);
+        }
+    }
+
+    private static class ByteBufferKeyAnalyzer implements KeyAnalyzer<ByteBuffer>
+    {
+        private final AbstractType<?> comparator;
+
+        public ByteBufferKeyAnalyzer(AbstractType<?> comparator)
+        {
+            this.comparator = comparator;
+        }
+
+        /**
+         * A bit mask where the first bit is 1 and the others are zero
+         */
+        private static final int MSB = 1 << Byte.SIZE-1;
+
+        public int compare(ByteBuffer a, ByteBuffer b)
+        {
+            return comparator.compare(a, b);
+        }
+
+        public int lengthInBits(ByteBuffer o)
+        {
+            return o.remaining() * Byte.SIZE;
+        }
+
+        public boolean isBitSet(ByteBuffer key, int bitIndex)
+        {
+            if (bitIndex >= lengthInBits(key))
+                return false;
+
+            int index = bitIndex / Byte.SIZE;
+            int bit = bitIndex % Byte.SIZE;
+            return (key.get(index) & mask(bit)) != 0;
+        }
+
+        public int bitIndex(ByteBuffer key, ByteBuffer otherKey)
+        {
+            int length = Math.max(key.remaining(), otherKey.remaining());
+
+            boolean allNull = true;
+            for (int i = 0; i < length; i++)
+            {
+                byte b1 = valueAt(key, i);
+                byte b2 = valueAt(otherKey, i);
+
+                if (b1 != b2)
+                {
+                    int xor = b1 ^ b2;
+                    for (int j = 0; j < Byte.SIZE; j++)
+                    {
+                        if ((xor & mask(j)) != 0)
+                            return (i * Byte.SIZE) + j;
+                    }
+                }
+
+                if (b1 != 0)
+                    allNull = false;
+            }
+
+            return allNull ? KeyAnalyzer.NULL_BIT_KEY : KeyAnalyzer.EQUAL_BIT_KEY;
+        }
+
+        public boolean isPrefix(ByteBuffer key, ByteBuffer prefix)
+        {
+            if (key.remaining() < prefix.remaining())
+                return false;
+
+            for (int i = 0; i < prefix.remaining(); i++)
+            {
+                if (key.get(i) != prefix.get(i))
+                    return false;
+            }
+
+            return true;
+        }
+
+        /**
+         * Returns the {@code byte} value at the given index.
+         */
+        private byte valueAt(ByteBuffer value, int index)
+        {
+            return index >= 0 && index < value.remaining() ? value.get(index) : 0;
+        }
+
+        /**
+         * Returns a bit mask where the given bit is set
+         */
+        private int mask(int bit)
+        {
+            return MSB >>> bit;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/conf/view/RangeTermTree.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/view/RangeTermTree.java b/src/java/org/apache/cassandra/index/sasi/conf/view/RangeTermTree.java
new file mode 100644
index 0000000..62e5636
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/conf/view/RangeTermTree.java
@@ -0,0 +1,77 @@
+/*
+ * 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.index.sasi.conf.view;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.cassandra.index.sasi.SSTableIndex;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.utils.Interval;
+import org.apache.cassandra.utils.IntervalTree;
+
+public class RangeTermTree implements TermTree
+{
+    protected final ByteBuffer min, max;
+    protected final IntervalTree<ByteBuffer, SSTableIndex, Interval<ByteBuffer, SSTableIndex>> rangeTree;
+
+    public RangeTermTree(ByteBuffer min, ByteBuffer max, IntervalTree<ByteBuffer, SSTableIndex, Interval<ByteBuffer, SSTableIndex>> rangeTree)
+    {
+        this.min = min;
+        this.max = max;
+        this.rangeTree = rangeTree;
+    }
+
+    public Set<SSTableIndex> search(Expression e)
+    {
+        ByteBuffer minTerm = e.lower == null ? min : e.lower.value;
+        ByteBuffer maxTerm = e.upper == null ? max : e.upper.value;
+
+        return new HashSet<>(rangeTree.search(Interval.create(minTerm, maxTerm, (SSTableIndex) null)));
+    }
+
+    public int intervalCount()
+    {
+        return rangeTree.intervalCount();
+    }
+
+    static class Builder extends TermTree.Builder
+    {
+        protected final List<Interval<ByteBuffer, SSTableIndex>> intervals = new ArrayList<>();
+
+        protected Builder(OnDiskIndexBuilder.Mode mode, AbstractType<?> comparator)
+        {
+            super(mode, comparator);
+        }
+
+        public void addIndex(SSTableIndex index)
+        {
+            intervals.add(Interval.create(index.minTerm(), index.maxTerm(), index));
+        }
+
+        public TermTree build()
+        {
+            return new RangeTermTree(min, max, IntervalTree.build(intervals));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/conf/view/TermTree.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/view/TermTree.java b/src/java/org/apache/cassandra/index/sasi/conf/view/TermTree.java
new file mode 100644
index 0000000..a175e22
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/conf/view/TermTree.java
@@ -0,0 +1,58 @@
+/*
+ * 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.index.sasi.conf.view;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+import org.apache.cassandra.index.sasi.SSTableIndex;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public interface TermTree
+{
+    Set<SSTableIndex> search(Expression e);
+
+    int intervalCount();
+
+    abstract class Builder
+    {
+        protected final OnDiskIndexBuilder.Mode mode;
+        protected final AbstractType<?> comparator;
+        protected ByteBuffer min, max;
+
+        protected Builder(OnDiskIndexBuilder.Mode mode, AbstractType<?> comparator)
+        {
+            this.mode = mode;
+            this.comparator = comparator;
+        }
+
+        public final void add(SSTableIndex index)
+        {
+            addIndex(index);
+
+            min = min == null || comparator.compare(min, index.minTerm()) > 0 ? index.minTerm() : min;
+            max = max == null || comparator.compare(max, index.maxTerm()) < 0 ? index.maxTerm() : max;
+        }
+
+        protected abstract void addIndex(SSTableIndex index);
+
+        public abstract TermTree build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/conf/view/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/view/View.java b/src/java/org/apache/cassandra/index/sasi/conf/view/View.java
new file mode 100644
index 0000000..378c3c6
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/conf/view/View.java
@@ -0,0 +1,104 @@
+/*
+ * 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.index.sasi.conf.view;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.index.sasi.SSTableIndex;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.Interval;
+import org.apache.cassandra.utils.IntervalTree;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+
+public class View implements Iterable<SSTableIndex>
+{
+    private final Map<Descriptor, SSTableIndex> view;
+
+    private final TermTree termTree;
+    private final IntervalTree<ByteBuffer, SSTableIndex, Interval<ByteBuffer, SSTableIndex>> keyIntervalTree;
+
+    public View(ColumnIndex index, Set<SSTableIndex> indexes)
+    {
+        this(index, Collections.<SSTableIndex>emptyList(), Collections.<SSTableReader>emptyList(), indexes);
+    }
+
+    public View(ColumnIndex index,
+                Collection<SSTableIndex> currentView,
+                Collection<SSTableReader> oldSSTables,
+                Set<SSTableIndex> newIndexes)
+    {
+        Map<Descriptor, SSTableIndex> newView = new HashMap<>();
+
+        AbstractType<?> validator = index.getValidator();
+        TermTree.Builder termTreeBuilder = (validator instanceof AsciiType || validator instanceof UTF8Type)
+                                            ? new PrefixTermTree.Builder(index.getMode().mode, validator)
+                                            : new RangeTermTree.Builder(index.getMode().mode, validator);
+
+        List<Interval<ByteBuffer, SSTableIndex>> keyIntervals = new ArrayList<>();
+        for (SSTableIndex sstableIndex : Iterables.concat(currentView, newIndexes))
+        {
+            SSTableReader sstable = sstableIndex.getSSTable();
+            if (oldSSTables.contains(sstable) || sstable.isMarkedCompacted() || newView.containsKey(sstable.descriptor))
+            {
+                sstableIndex.release();
+                continue;
+            }
+
+            newView.put(sstable.descriptor, sstableIndex);
+
+            termTreeBuilder.add(sstableIndex);
+            keyIntervals.add(Interval.create(sstableIndex.minKey(), sstableIndex.maxKey(), sstableIndex));
+        }
+
+        this.view = newView;
+        this.termTree = termTreeBuilder.build();
+        this.keyIntervalTree = IntervalTree.build(keyIntervals);
+
+        if (keyIntervalTree.intervalCount() != termTree.intervalCount())
+            throw new IllegalStateException(String.format("mismatched sizes for intervals tree for keys vs terms: %d != %d", keyIntervalTree.intervalCount(), termTree.intervalCount()));
+    }
+
+    public Set<SSTableIndex> match(final Set<SSTableReader> scope, Expression expression)
+    {
+        return Sets.filter(termTree.search(expression), index -> scope.contains(index.getSSTable()));
+    }
+
+    public List<SSTableIndex> match(ByteBuffer minKey, ByteBuffer maxKey)
+    {
+        return keyIntervalTree.search(Interval.create(minKey, maxKey, (SSTableIndex) null));
+    }
+
+    public Iterator<SSTableIndex> iterator()
+    {
+        return view.values().iterator();
+    }
+
+    public Collection<SSTableIndex> getIndexes()
+    {
+        return view.values();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/disk/Descriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/Descriptor.java b/src/java/org/apache/cassandra/index/sasi/disk/Descriptor.java
new file mode 100644
index 0000000..a719f50
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/disk/Descriptor.java
@@ -0,0 +1,51 @@
+/*
+ * 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.index.sasi.disk;
+
+/**
+ * Object descriptor for SSTableAttachedSecondaryIndex files. Similar to, and based upon, the sstable descriptor.
+ */
+public class Descriptor
+{
+    public static final String VERSION_AA = "aa";
+    public static final String VERSION_AB = "ab";
+    public static final String CURRENT_VERSION = VERSION_AB;
+    public static final Descriptor CURRENT = new Descriptor(CURRENT_VERSION);
+
+    public static class Version
+    {
+        public final String version;
+
+        public Version(String version)
+        {
+            this.version = version;
+        }
+
+        public String toString()
+        {
+            return version;
+        }
+    }
+
+    public final Version version;
+
+    public Descriptor(String v)
+    {
+        this.version = new Version(v);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/disk/OnDiskBlock.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskBlock.java b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskBlock.java
new file mode 100644
index 0000000..32cda53
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskBlock.java
@@ -0,0 +1,142 @@
+/*
+ * 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.index.sasi.disk;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.index.sasi.Term;
+import org.apache.cassandra.index.sasi.utils.MappedBuffer;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public abstract class OnDiskBlock<T extends Term>
+{
+    public enum BlockType
+    {
+        POINTER, DATA
+    }
+
+    // this contains offsets of the terms and term data
+    protected final MappedBuffer blockIndex;
+    protected final int blockIndexSize;
+
+    protected final boolean hasCombinedIndex;
+    protected final TokenTree combinedIndex;
+
+    public OnDiskBlock(Descriptor descriptor, MappedBuffer block, BlockType blockType)
+    {
+        blockIndex = block;
+
+        if (blockType == BlockType.POINTER)
+        {
+            hasCombinedIndex = false;
+            combinedIndex = null;
+            blockIndexSize = block.getInt() << 1; // num terms * sizeof(short)
+            return;
+        }
+
+        long blockOffset = block.position();
+        int combinedIndexOffset = block.getInt(blockOffset + OnDiskIndexBuilder.BLOCK_SIZE);
+
+        hasCombinedIndex = (combinedIndexOffset >= 0);
+        long blockIndexOffset = blockOffset + OnDiskIndexBuilder.BLOCK_SIZE + 4 + combinedIndexOffset;
+
+        combinedIndex = hasCombinedIndex ? new TokenTree(descriptor, blockIndex.duplicate().position(blockIndexOffset)) : null;
+        blockIndexSize = block.getInt() * 2;
+    }
+
+    public SearchResult<T> search(AbstractType<?> comparator, ByteBuffer query)
+    {
+        int cmp = -1, start = 0, end = termCount() - 1, middle = 0;
+
+        T element = null;
+        while (start <= end)
+        {
+            middle = start + ((end - start) >> 1);
+            element = getTerm(middle);
+
+            cmp = element.compareTo(comparator, query);
+            if (cmp == 0)
+                return new SearchResult<>(element, cmp, middle);
+            else if (cmp < 0)
+                start = middle + 1;
+            else
+                end = middle - 1;
+        }
+
+        return new SearchResult<>(element, cmp, middle);
+    }
+
+    protected T getTerm(int index)
+    {
+        MappedBuffer dup = blockIndex.duplicate();
+        long startsAt = getTermPosition(index);
+        if (termCount() - 1 == index) // last element
+            dup.position(startsAt);
+        else
+            dup.position(startsAt).limit(getTermPosition(index + 1));
+
+        return cast(dup);
+    }
+
+    protected long getTermPosition(int idx)
+    {
+        return getTermPosition(blockIndex, idx, blockIndexSize);
+    }
+
+    protected int termCount()
+    {
+        return blockIndexSize >> 1;
+    }
+
+    protected abstract T cast(MappedBuffer data);
+
+    static long getTermPosition(MappedBuffer data, int idx, int indexSize)
+    {
+        idx <<= 1;
+        assert idx < indexSize;
+        return data.position() + indexSize + data.getShort(data.position() + idx);
+    }
+
+    public TokenTree getBlockIndex()
+    {
+        return combinedIndex;
+    }
+
+    public int minOffset(OnDiskIndex.IteratorOrder order)
+    {
+        return order == OnDiskIndex.IteratorOrder.DESC ? 0 : termCount() - 1;
+    }
+
+    public int maxOffset(OnDiskIndex.IteratorOrder order)
+    {
+        return minOffset(order) == 0 ? termCount() - 1 : 0;
+    }
+
+    public static class SearchResult<T>
+    {
+        public final T result;
+        public final int index, cmp;
+
+        public SearchResult(T result, int cmp, int index)
+        {
+            this.result = result;
+            this.index = index;
+            this.cmp = cmp;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java
new file mode 100644
index 0000000..0f9e389
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java
@@ -0,0 +1,773 @@
+/*
+ * 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.index.sasi.disk;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.index.sasi.Term;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.index.sasi.plan.Expression.Op;
+import org.apache.cassandra.index.sasi.utils.MappedBuffer;
+import org.apache.cassandra.index.sasi.utils.RangeUnionIterator;
+import org.apache.cassandra.index.sasi.utils.AbstractIterator;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+
+import static org.apache.cassandra.index.sasi.disk.OnDiskBlock.SearchResult;
+
+public class OnDiskIndex implements Iterable<OnDiskIndex.DataTerm>, Closeable
+{
+    public enum IteratorOrder
+    {
+        DESC(1), ASC(-1);
+
+        public final int step;
+
+        IteratorOrder(int step)
+        {
+            this.step = step;
+        }
+
+        public int startAt(OnDiskBlock<DataTerm> block, Expression e)
+        {
+            switch (this)
+            {
+                case DESC:
+                    return e.lower == null
+                            ? 0
+                            : startAt(block.search(e.validator, e.lower.value), e.lower.inclusive);
+
+                case ASC:
+                    return e.upper == null
+                            ? block.termCount() - 1
+                            : startAt(block.search(e.validator, e.upper.value), e.upper.inclusive);
+
+                default:
+                    throw new IllegalArgumentException("Unknown order: " + this);
+            }
+        }
+
+        public int startAt(SearchResult<DataTerm> found, boolean inclusive)
+        {
+            switch (this)
+            {
+                case DESC:
+                    if (found.cmp < 0)
+                        return found.index + 1;
+
+                    return inclusive || found.cmp != 0 ? found.index : found.index + 1;
+
+                case ASC:
+                    if (found.cmp < 0) // search term was bigger then whole data set
+                        return found.index;
+                    return inclusive && (found.cmp == 0 || found.cmp < 0) ? found.index : found.index - 1;
+
+                default:
+                    throw new IllegalArgumentException("Unknown order: " + this);
+            }
+        }
+    }
+
+    public final Descriptor descriptor;
+    protected final OnDiskIndexBuilder.Mode mode;
+    protected final OnDiskIndexBuilder.TermSize termSize;
+
+    protected final AbstractType<?> comparator;
+    protected final MappedBuffer indexFile;
+    protected final long indexSize;
+
+    protected final Function<Long, DecoratedKey> keyFetcher;
+
+    protected final String indexPath;
+
+    protected final PointerLevel[] levels;
+    protected final DataLevel dataLevel;
+
+    protected final ByteBuffer minTerm, maxTerm, minKey, maxKey;
+
+    public OnDiskIndex(File index, AbstractType<?> cmp, Function<Long, DecoratedKey> keyReader)
+    {
+        keyFetcher = keyReader;
+
+        comparator = cmp;
+        indexPath = index.getAbsolutePath();
+
+        RandomAccessFile backingFile = null;
+        try
+        {
+            backingFile = new RandomAccessFile(index, "r");
+
+            descriptor = new Descriptor(backingFile.readUTF());
+
+            termSize = OnDiskIndexBuilder.TermSize.of(backingFile.readShort());
+
+            minTerm = ByteBufferUtil.readWithShortLength(backingFile);
+            maxTerm = ByteBufferUtil.readWithShortLength(backingFile);
+
+            minKey = ByteBufferUtil.readWithShortLength(backingFile);
+            maxKey = ByteBufferUtil.readWithShortLength(backingFile);
+
+            mode = OnDiskIndexBuilder.Mode.mode(backingFile.readUTF());
+
+            indexSize = backingFile.length();
+            indexFile = new MappedBuffer(new ChannelProxy(indexPath, backingFile.getChannel()));
+
+            // start of the levels
+            indexFile.position(indexFile.getLong(indexSize - 8));
+
+            int numLevels = indexFile.getInt();
+            levels = new PointerLevel[numLevels];
+            for (int i = 0; i < levels.length; i++)
+            {
+                int blockCount = indexFile.getInt();
+                levels[i] = new PointerLevel(indexFile.position(), blockCount);
+                indexFile.position(indexFile.position() + blockCount * 8);
+            }
+
+            int blockCount = indexFile.getInt();
+            dataLevel = new DataLevel(indexFile.position(), blockCount);
+        }
+        catch (IOException e)
+        {
+            throw new FSReadError(e, index);
+        }
+        finally
+        {
+            FileUtils.closeQuietly(backingFile);
+        }
+    }
+
+    public ByteBuffer minTerm()
+    {
+        return minTerm;
+    }
+
+    public ByteBuffer maxTerm()
+    {
+        return maxTerm;
+    }
+
+    public ByteBuffer minKey()
+    {
+        return minKey;
+    }
+
+    public ByteBuffer maxKey()
+    {
+        return maxKey;
+    }
+
+    public DataTerm min()
+    {
+        return dataLevel.getBlock(0).getTerm(0);
+    }
+
+    public DataTerm max()
+    {
+        DataBlock block = dataLevel.getBlock(dataLevel.blockCount - 1);
+        return block.getTerm(block.termCount() - 1);
+    }
+
+    /**
+     * Search for rows which match all of the terms inside the given expression in the index file.
+     *
+     * @param exp The expression to use for the query.
+     *
+     * @return Iterator which contains rows for all of the terms from the given range.
+     */
+    public RangeIterator<Long, Token> search(Expression exp)
+    {
+        // convert single NOT_EQ to range with exclusion
+        final Expression expression = (exp.getOp() != Op.NOT_EQ)
+                                        ? exp
+                                        : new Expression(exp).setOp(Op.RANGE)
+                                                .setLower(new Expression.Bound(minTerm, true))
+                                                .setUpper(new Expression.Bound(maxTerm, true))
+                                                .addExclusion(exp.lower.value);
+
+        List<ByteBuffer> exclusions = new ArrayList<>(expression.exclusions.size());
+
+        Iterables.addAll(exclusions, expression.exclusions.stream().filter(exclusion -> {
+            // accept only exclusions which are in the bounds of lower/upper
+            return !(expression.lower != null && comparator.compare(exclusion, expression.lower.value) < 0)
+                && !(expression.upper != null && comparator.compare(exclusion, expression.upper.value) > 0);
+        }).collect(Collectors.toList()));
+
+        Collections.sort(exclusions, comparator);
+
+        if (exclusions.size() == 0)
+            return searchRange(expression);
+
+        List<Expression> ranges = new ArrayList<>(exclusions.size());
+
+        // calculate range splits based on the sorted exclusions
+        Iterator<ByteBuffer> exclusionsIterator = exclusions.iterator();
+
+        Expression.Bound min = expression.lower, max = null;
+        while (exclusionsIterator.hasNext())
+        {
+            max = new Expression.Bound(exclusionsIterator.next(), false);
+            ranges.add(new Expression(expression).setOp(Op.RANGE).setLower(min).setUpper(max));
+            min = max;
+        }
+
+        assert max != null;
+        ranges.add(new Expression(expression).setOp(Op.RANGE).setLower(max).setUpper(expression.upper));
+
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+        for (Expression e : ranges)
+        {
+            RangeIterator<Long, Token> range = searchRange(e);
+            if (range != null)
+                builder.add(range);
+        }
+
+        return builder.build();
+    }
+
+    private RangeIterator<Long, Token> searchRange(Expression range)
+    {
+        Expression.Bound lower = range.lower;
+        Expression.Bound upper = range.upper;
+
+        int lowerBlock = lower == null ? 0 : getDataBlock(lower.value);
+        int upperBlock = upper == null
+                ? dataLevel.blockCount - 1
+                // optimization so we don't have to fetch upperBlock when query has lower == upper
+                : (lower != null && comparator.compare(lower.value, upper.value) == 0) ? lowerBlock : getDataBlock(upper.value);
+
+        return (mode != OnDiskIndexBuilder.Mode.SPARSE || lowerBlock == upperBlock || upperBlock - lowerBlock <= 1)
+                ? searchPoint(lowerBlock, range)
+                : searchRange(lowerBlock, lower, upperBlock, upper);
+    }
+
+    private RangeIterator<Long, Token> searchRange(int lowerBlock, Expression.Bound lower, int upperBlock, Expression.Bound upper)
+    {
+        // if lower is at the beginning of the block that means we can just do a single iterator per block
+        SearchResult<DataTerm> lowerPosition = (lower == null) ? null : searchIndex(lower.value, lowerBlock);
+        SearchResult<DataTerm> upperPosition = (upper == null) ? null : searchIndex(upper.value, upperBlock);
+
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+
+        // optimistically assume that first and last blocks are full block reads, saves at least 3 'else' conditions
+        int firstFullBlockIdx = lowerBlock, lastFullBlockIdx = upperBlock;
+
+        // 'lower' doesn't cover the whole block so we need to do a partial iteration
+        // Two reasons why that can happen:
+        //   - 'lower' is not the first element of the block
+        //   - 'lower' is first element but it's not inclusive in the query
+        if (lowerPosition != null && (lowerPosition.index > 0 || !lower.inclusive))
+        {
+            DataBlock block = dataLevel.getBlock(lowerBlock);
+            int start = (lower.inclusive || lowerPosition.cmp != 0) ? lowerPosition.index : lowerPosition.index + 1;
+
+            builder.add(block.getRange(start, block.termCount()));
+            firstFullBlockIdx = lowerBlock + 1;
+        }
+
+        if (upperPosition != null)
+        {
+            DataBlock block = dataLevel.getBlock(upperBlock);
+            int lastIndex = block.termCount() - 1;
+
+            // The save as with 'lower' but here we need to check if the upper is the last element of the block,
+            // which means that we only have to get individual results if:
+            //  - if it *is not* the last element, or
+            //  - it *is* but shouldn't be included (dictated by upperInclusive)
+            if (upperPosition.index != lastIndex || !upper.inclusive)
+            {
+                int end = (upperPosition.cmp < 0 || (upperPosition.cmp == 0 && upper.inclusive))
+                                ? upperPosition.index + 1 : upperPosition.index;
+
+                builder.add(block.getRange(0, end));
+                lastFullBlockIdx = upperBlock - 1;
+            }
+        }
+
+        int totalSuperBlocks = (lastFullBlockIdx - firstFullBlockIdx) / OnDiskIndexBuilder.SUPER_BLOCK_SIZE;
+
+        // if there are no super-blocks, we can simply read all of the block iterators in sequence
+        if (totalSuperBlocks == 0)
+        {
+            for (int i = firstFullBlockIdx; i <= lastFullBlockIdx; i++)
+                builder.add(dataLevel.getBlock(i).getBlockIndex().iterator(keyFetcher));
+
+            return builder.build();
+        }
+
+        // first get all of the blocks which are aligned before the first super-block in the sequence,
+        // e.g. if the block range was (1, 9) and super-block-size = 4, we need to read 1, 2, 3, 4 - 7 is covered by
+        // super-block, 8, 9 is a remainder.
+
+        int superBlockAlignedStart = firstFullBlockIdx == 0 ? 0 : (int) FBUtilities.align(firstFullBlockIdx, OnDiskIndexBuilder.SUPER_BLOCK_SIZE);
+        for (int blockIdx = firstFullBlockIdx; blockIdx < Math.min(superBlockAlignedStart, lastFullBlockIdx); blockIdx++)
+            builder.add(getBlockIterator(blockIdx));
+
+        // now read all of the super-blocks matched by the request, from the previous comment
+        // it's a block with index 1 (which covers everything from 4 to 7)
+
+        int superBlockIdx = superBlockAlignedStart / OnDiskIndexBuilder.SUPER_BLOCK_SIZE;
+        for (int offset = 0; offset < totalSuperBlocks - 1; offset++)
+            builder.add(dataLevel.getSuperBlock(superBlockIdx++).iterator());
+
+        // now it's time for a remainder read, again from the previous example it's 8, 9 because
+        // we have over-shot previous block but didn't request enough to cover next super-block.
+
+        int lastCoveredBlock = superBlockIdx * OnDiskIndexBuilder.SUPER_BLOCK_SIZE;
+        for (int offset = 0; offset <= (lastFullBlockIdx - lastCoveredBlock); offset++)
+            builder.add(getBlockIterator(lastCoveredBlock + offset));
+
+        return builder.build();
+    }
+
+    private RangeIterator<Long, Token> searchPoint(int lowerBlock, Expression expression)
+    {
+        Iterator<DataTerm> terms = new TermIterator(lowerBlock, expression, IteratorOrder.DESC);
+        RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+
+        while (terms.hasNext())
+        {
+            try
+            {
+                builder.add(terms.next().getTokens());
+            }
+            finally
+            {
+                expression.checkpoint();
+            }
+        }
+
+        return builder.build();
+    }
+
+    private RangeIterator<Long, Token> getBlockIterator(int blockIdx)
+    {
+        DataBlock block = dataLevel.getBlock(blockIdx);
+        return (block.hasCombinedIndex)
+                ? block.getBlockIndex().iterator(keyFetcher)
+                : block.getRange(0, block.termCount());
+    }
+
+    public Iterator<DataTerm> iteratorAt(ByteBuffer query, IteratorOrder order, boolean inclusive)
+    {
+        Expression e = new Expression("", comparator);
+        Expression.Bound bound = new Expression.Bound(query, inclusive);
+
+        switch (order)
+        {
+            case DESC:
+                e.setLower(bound);
+                break;
+
+            case ASC:
+                e.setUpper(bound);
+                break;
+
+            default:
+                throw new IllegalArgumentException("Unknown order: " + order);
+        }
+
+        return new TermIterator(levels.length == 0 ? 0 : getBlockIdx(findPointer(query), query), e, order);
+    }
+
+    private int getDataBlock(ByteBuffer query)
+    {
+        return levels.length == 0 ? 0 : getBlockIdx(findPointer(query), query);
+    }
+
+    public Iterator<DataTerm> iterator()
+    {
+        return new TermIterator(0, new Expression("", comparator), IteratorOrder.DESC);
+    }
+
+    public void close() throws IOException
+    {
+        FileUtils.closeQuietly(indexFile);
+    }
+
+    private PointerTerm findPointer(ByteBuffer query)
+    {
+        PointerTerm ptr = null;
+        for (PointerLevel level : levels)
+        {
+            if ((ptr = level.getPointer(ptr, query)) == null)
+                return null;
+        }
+
+        return ptr;
+    }
+
+    private SearchResult<DataTerm> searchIndex(ByteBuffer query, int blockIdx)
+    {
+        return dataLevel.getBlock(blockIdx).search(comparator, query);
+    }
+
+    private int getBlockIdx(PointerTerm ptr, ByteBuffer query)
+    {
+        int blockIdx = 0;
+        if (ptr != null)
+        {
+            int cmp = ptr.compareTo(comparator, query);
+            blockIdx = (cmp == 0 || cmp > 0) ? ptr.getBlock() : ptr.getBlock() + 1;
+        }
+
+        return blockIdx;
+    }
+
+    protected class PointerLevel extends Level<PointerBlock>
+    {
+        public PointerLevel(long offset, int count)
+        {
+            super(offset, count);
+        }
+
+        public PointerTerm getPointer(PointerTerm parent, ByteBuffer query)
+        {
+            return getBlock(getBlockIdx(parent, query)).search(comparator, query).result;
+        }
+
+        protected PointerBlock cast(MappedBuffer block)
+        {
+            return new PointerBlock(block);
+        }
+    }
+
+    protected class DataLevel extends Level<DataBlock>
+    {
+        protected final int superBlockCnt;
+        protected final long superBlocksOffset;
+
+        public DataLevel(long offset, int count)
+        {
+            super(offset, count);
+            long baseOffset = blockOffsets + blockCount * 8;
+            superBlockCnt = indexFile.getInt(baseOffset);
+            superBlocksOffset = baseOffset + 4;
+        }
+
+        protected DataBlock cast(MappedBuffer block)
+        {
+            return new DataBlock(block);
+        }
+
+        public OnDiskSuperBlock getSuperBlock(int idx)
+        {
+            assert idx < superBlockCnt : String.format("requested index %d is greater than super block count %d", idx, superBlockCnt);
+            long blockOffset = indexFile.getLong(superBlocksOffset + idx * 8);
+            return new OnDiskSuperBlock(indexFile.duplicate().position(blockOffset));
+        }
+    }
+
+    protected class OnDiskSuperBlock
+    {
+        private final TokenTree tokenTree;
+
+        public OnDiskSuperBlock(MappedBuffer buffer)
+        {
+            tokenTree = new TokenTree(descriptor, buffer);
+        }
+
+        public RangeIterator<Long, Token> iterator()
+        {
+            return tokenTree.iterator(keyFetcher);
+        }
+    }
+
+    protected abstract class Level<T extends OnDiskBlock>
+    {
+        protected final long blockOffsets;
+        protected final int blockCount;
+
+        public Level(long offsets, int count)
+        {
+            this.blockOffsets = offsets;
+            this.blockCount = count;
+        }
+
+        public T getBlock(int idx) throws FSReadError
+        {
+            assert idx >= 0 && idx < blockCount;
+
+            // calculate block offset and move there
+            // (long is intentional, we'll just need mmap implementation which supports long positions)
+            long blockOffset = indexFile.getLong(blockOffsets + idx * 8);
+            return cast(indexFile.duplicate().position(blockOffset));
+        }
+
+        protected abstract T cast(MappedBuffer block);
+    }
+
+    protected class DataBlock extends OnDiskBlock<DataTerm>
+    {
+        public DataBlock(MappedBuffer data)
+        {
+            super(descriptor, data, BlockType.DATA);
+        }
+
+        protected DataTerm cast(MappedBuffer data)
+        {
+            return new DataTerm(data, termSize, getBlockIndex());
+        }
+
+        public RangeIterator<Long, Token> getRange(int start, int end)
+        {
+            RangeUnionIterator.Builder<Long, Token> builder = RangeUnionIterator.builder();
+            NavigableMap<Long, Token> sparse = new TreeMap<>();
+
+            for (int i = start; i < end; i++)
+            {
+                DataTerm term = getTerm(i);
+
+                if (term.isSparse())
+                {
+                    NavigableMap<Long, Token> tokens = term.getSparseTokens();
+                    for (Map.Entry<Long, Token> t : tokens.entrySet())
+                    {
+                        Token token = sparse.get(t.getKey());
+                        if (token == null)
+                            sparse.put(t.getKey(), t.getValue());
+                        else
+                            token.merge(t.getValue());
+                    }
+                }
+                else
+                {
+                    builder.add(term.getTokens());
+                }
+            }
+
+            PrefetchedTokensIterator prefetched = sparse.isEmpty() ? null : new PrefetchedTokensIterator(sparse);
+
+            if (builder.rangeCount() == 0)
+                return prefetched;
+
+            builder.add(prefetched);
+            return builder.build();
+        }
+    }
+
+    protected class PointerBlock extends OnDiskBlock<PointerTerm>
+    {
+        public PointerBlock(MappedBuffer block)
+        {
+            super(descriptor, block, BlockType.POINTER);
+        }
+
+        protected PointerTerm cast(MappedBuffer data)
+        {
+            return new PointerTerm(data, termSize);
+        }
+    }
+
+    public class DataTerm extends Term implements Comparable<DataTerm>
+    {
+        private final TokenTree perBlockIndex;
+
+        protected DataTerm(MappedBuffer content, OnDiskIndexBuilder.TermSize size, TokenTree perBlockIndex)
+        {
+            super(content, size);
+            this.perBlockIndex = perBlockIndex;
+        }
+
+        public RangeIterator<Long, Token> getTokens()
+        {
+            final long blockEnd = FBUtilities.align(content.position(), OnDiskIndexBuilder.BLOCK_SIZE);
+
+            if (isSparse())
+                return new PrefetchedTokensIterator(getSparseTokens());
+
+            long offset = blockEnd + 4 + content.getInt(getDataOffset() + 1);
+            return new TokenTree(descriptor, indexFile.duplicate().position(offset)).iterator(keyFetcher);
+        }
+
+        public boolean isSparse()
+        {
+            return content.get(getDataOffset()) > 0;
+        }
+
+        public NavigableMap<Long, Token> getSparseTokens()
+        {
+            long ptrOffset = getDataOffset();
+
+            byte size = content.get(ptrOffset);
+
+            assert size > 0;
+
+            NavigableMap<Long, Token> individualTokens = new TreeMap<>();
+            for (int i = 0; i < size; i++)
+            {
+                Token token = perBlockIndex.get(content.getLong(ptrOffset + 1 + (8 * i)), keyFetcher);
+
+                assert token != null;
+                individualTokens.put(token.get(), token);
+            }
+
+            return individualTokens;
+        }
+
+        public int compareTo(DataTerm other)
+        {
+            return other == null ? 1 : compareTo(comparator, other.getTerm());
+        }
+    }
+
+    protected static class PointerTerm extends Term
+    {
+        public PointerTerm(MappedBuffer content, OnDiskIndexBuilder.TermSize size)
+        {
+            super(content, size);
+        }
+
+        public int getBlock()
+        {
+            return content.getInt(getDataOffset());
+        }
+    }
+
+    private static class PrefetchedTokensIterator extends RangeIterator<Long, Token>
+    {
+        private final NavigableMap<Long, Token> tokens;
+        private PeekingIterator<Token> currentIterator;
+
+        public PrefetchedTokensIterator(NavigableMap<Long, Token> tokens)
+        {
+            super(tokens.firstKey(), tokens.lastKey(), tokens.size());
+            this.tokens = tokens;
+            this.currentIterator = Iterators.peekingIterator(tokens.values().iterator());
+        }
+
+        protected Token computeNext()
+        {
+            return currentIterator != null && currentIterator.hasNext()
+                    ? currentIterator.next()
+                    : endOfData();
+        }
+
+        protected void performSkipTo(Long nextToken)
+        {
+            currentIterator = Iterators.peekingIterator(tokens.tailMap(nextToken, true).values().iterator());
+        }
+
+        public void close() throws IOException
+        {
+            endOfData();
+        }
+    }
+
+    public AbstractType<?> getComparator()
+    {
+        return comparator;
+    }
+
+    public String getIndexPath()
+    {
+        return indexPath;
+    }
+
+    private class TermIterator extends AbstractIterator<DataTerm>
+    {
+        private final Expression e;
+        private final IteratorOrder order;
+
+        protected OnDiskBlock<DataTerm> currentBlock;
+        protected int blockIndex, offset;
+
+        private boolean checkLower = true, checkUpper = true;
+
+        public TermIterator(int startBlock, Expression expression, IteratorOrder order)
+        {
+            this.e = expression;
+            this.order = order;
+            this.blockIndex = startBlock;
+
+            nextBlock();
+        }
+
+        protected DataTerm computeNext()
+        {
+            for (;;)
+            {
+                if (currentBlock == null)
+                    return endOfData();
+
+                if (offset >= 0 && offset < currentBlock.termCount())
+                {
+                    DataTerm currentTerm = currentBlock.getTerm(nextOffset());
+
+                    if (checkLower && !e.isLowerSatisfiedBy(currentTerm))
+                        continue;
+
+                    // flip the flag right on the first bounds match
+                    // to avoid expensive comparisons
+                    checkLower = false;
+
+                    if (checkUpper && !e.isUpperSatisfiedBy(currentTerm))
+                        return endOfData();
+
+                    return currentTerm;
+                }
+
+                nextBlock();
+            }
+        }
+
+        protected void nextBlock()
+        {
+            currentBlock = null;
+
+            if (blockIndex < 0 || blockIndex >= dataLevel.blockCount)
+                return;
+
+            currentBlock = dataLevel.getBlock(nextBlockIndex());
+            offset = checkLower ? order.startAt(currentBlock, e) : currentBlock.minOffset(order);
+
+            // let's check the last term of the new block right away
+            // if expression's upper bound is satisfied by it such means that we can avoid
+            // doing any expensive upper bound checks for that block.
+            checkUpper = e.hasUpper() && !e.isUpperSatisfiedBy(currentBlock.getTerm(currentBlock.maxOffset(order)));
+        }
+
+        protected int nextBlockIndex()
+        {
+            int current = blockIndex;
+            blockIndex += order.step;
+            return current;
+        }
+
+        protected int nextOffset()
+        {
+            int current = offset;
+            offset += order.step;
+            return current;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java
new file mode 100644
index 0000000..7b8f5c9
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java
@@ -0,0 +1,627 @@
+/*
+ * 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.index.sasi.disk;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.index.sasi.sa.IntegralSA;
+import org.apache.cassandra.index.sasi.sa.SA;
+import org.apache.cassandra.index.sasi.sa.TermIterator;
+import org.apache.cassandra.index.sasi.sa.SuffixSA;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+
+import com.carrotsearch.hppc.LongArrayList;
+import com.carrotsearch.hppc.LongSet;
+import com.carrotsearch.hppc.ShortArrayList;
+import com.google.common.annotations.VisibleForTesting;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OnDiskIndexBuilder
+{
+    private static final Logger logger = LoggerFactory.getLogger(OnDiskIndexBuilder.class);
+
+    public enum Mode
+    {
+        PREFIX, CONTAINS, SPARSE;
+
+        public static Mode mode(String mode)
+        {
+            return Mode.valueOf(mode.toUpperCase());
+        }
+    }
+
+    public enum TermSize
+    {
+        INT(4), LONG(8), UUID(16), VARIABLE(-1);
+
+        public final int size;
+
+        TermSize(int size)
+        {
+            this.size = size;
+        }
+
+        public boolean isConstant()
+        {
+            return this != VARIABLE;
+        }
+
+        public static TermSize of(int size)
+        {
+            switch (size)
+            {
+                case -1:
+                    return VARIABLE;
+
+                case 4:
+                    return INT;
+
+                case 8:
+                    return LONG;
+
+                case 16:
+                    return UUID;
+
+                default:
+                    throw new IllegalStateException("unknown state: " + size);
+            }
+        }
+
+        public static TermSize sizeOf(AbstractType<?> comparator)
+        {
+            if (comparator instanceof Int32Type || comparator instanceof FloatType)
+                return INT;
+
+            if (comparator instanceof LongType || comparator instanceof DoubleType
+                    || comparator instanceof TimestampType || comparator instanceof DateType)
+                return LONG;
+
+            if (comparator instanceof TimeUUIDType || comparator instanceof UUIDType)
+                return UUID;
+
+            return VARIABLE;
+        }
+    }
+
+    public static final int BLOCK_SIZE = 4096;
+    public static final int MAX_TERM_SIZE = 1024;
+    public static final int SUPER_BLOCK_SIZE = 64;
+
+    private final List<MutableLevel<InMemoryPointerTerm>> levels = new ArrayList<>();
+    private MutableLevel<InMemoryDataTerm> dataLevel;
+
+    private final TermSize termSize;
+
+    private final AbstractType<?> keyComparator, termComparator;
+
+    private final Map<ByteBuffer, TokenTreeBuilder> terms;
+    private final Mode mode;
+
+    private ByteBuffer minKey, maxKey;
+    private long estimatedBytes;
+
+    public OnDiskIndexBuilder(AbstractType<?> keyComparator, AbstractType<?> comparator, Mode mode)
+    {
+        this.keyComparator = keyComparator;
+        this.termComparator = comparator;
+        this.terms = new HashMap<>();
+        this.termSize = TermSize.sizeOf(comparator);
+        this.mode = mode;
+    }
+
+    public OnDiskIndexBuilder add(ByteBuffer term, DecoratedKey key, long keyPosition)
+    {
+        if (term.remaining() >= MAX_TERM_SIZE)
+        {
+            logger.error("Rejecting value (value size {}, maximum size {} bytes).", term.remaining(), Short.MAX_VALUE);
+            return this;
+        }
+
+        TokenTreeBuilder tokens = terms.get(term);
+        if (tokens == null)
+        {
+            terms.put(term, (tokens = new TokenTreeBuilder()));
+
+            // on-heap size estimates from jol
+            // 64 bytes for TTB + 48 bytes for TreeMap in TTB + size bytes for the term (map key)
+            estimatedBytes += 64 + 48 + term.remaining();
+        }
+
+        tokens.add((Long) key.getToken().getTokenValue(), keyPosition);
+
+        // calculate key range (based on actual key values) for current index
+        minKey = (minKey == null || keyComparator.compare(minKey, key.getKey()) > 0) ? key.getKey() : minKey;
+        maxKey = (maxKey == null || keyComparator.compare(maxKey, key.getKey()) < 0) ? key.getKey() : maxKey;
+
+        // 60 ((boolean(1)*4) + (long(8)*4) + 24) bytes for the LongOpenHashSet created when the keyPosition was added
+        // + 40 bytes for the TreeMap.Entry + 8 bytes for the token (key).
+        // in the case of hash collision for the token we may overestimate but this is extremely rare
+        estimatedBytes += 60 + 40 + 8;
+
+        return this;
+    }
+
+    public long estimatedMemoryUse()
+    {
+        return estimatedBytes;
+    }
+
+    private void addTerm(InMemoryDataTerm term, SequentialWriter out) throws IOException
+    {
+        InMemoryPointerTerm ptr = dataLevel.add(term);
+        if (ptr == null)
+            return;
+
+        int levelIdx = 0;
+        for (;;)
+        {
+            MutableLevel<InMemoryPointerTerm> level = getIndexLevel(levelIdx++, out);
+            if ((ptr = level.add(ptr)) == null)
+                break;
+        }
+    }
+
+    public boolean isEmpty()
+    {
+        return terms.isEmpty();
+    }
+
+    public void finish(Pair<ByteBuffer, ByteBuffer> range, File file, TermIterator terms)
+    {
+        finish(Descriptor.CURRENT, range, file, terms);
+    }
+
+    /**
+     * Finishes up index building process by creating/populating index file.
+     *
+     * @param indexFile The file to write index contents to.
+     *
+     * @return true if index was written successfully, false otherwise (e.g. if index was empty).
+     *
+     * @throws FSWriteError on I/O error.
+     */
+    public boolean finish(File indexFile) throws FSWriteError
+    {
+        return finish(Descriptor.CURRENT, indexFile);
+    }
+
+    @VisibleForTesting
+    protected boolean finish(Descriptor descriptor, File file) throws FSWriteError
+    {
+        // no terms means there is nothing to build
+        if (terms.isEmpty())
+            return false;
+
+        // split terms into suffixes only if it's text, otherwise (even if CONTAINS is set) use terms in original form
+        SA sa = ((termComparator instanceof UTF8Type || termComparator instanceof AsciiType) && mode == Mode.CONTAINS)
+                    ? new SuffixSA(termComparator, mode) : new IntegralSA(termComparator, mode);
+
+        for (Map.Entry<ByteBuffer, TokenTreeBuilder> term : terms.entrySet())
+            sa.add(term.getKey(), term.getValue());
+
+        finish(descriptor, Pair.create(minKey, maxKey), file, sa.finish());
+        return true;
+    }
+
+    protected void finish(Descriptor descriptor, Pair<ByteBuffer, ByteBuffer> range, File file, TermIterator terms)
+    {
+        SequentialWriter out = null;
+
+        try
+        {
+            out = new SequentialWriter(file, BLOCK_SIZE, BufferType.ON_HEAP);
+
+            out.writeUTF(descriptor.version.toString());
+
+            out.writeShort(termSize.size);
+
+            // min, max term (useful to find initial scan range from search expressions)
+            ByteBufferUtil.writeWithShortLength(terms.minTerm(), out);
+            ByteBufferUtil.writeWithShortLength(terms.maxTerm(), out);
+
+            // min, max keys covered by index (useful when searching across multiple indexes)
+            ByteBufferUtil.writeWithShortLength(range.left, out);
+            ByteBufferUtil.writeWithShortLength(range.right, out);
+
+            out.writeUTF(mode.toString());
+
+            out.skipBytes((int) (BLOCK_SIZE - out.position()));
+
+            dataLevel = mode == Mode.SPARSE ? new DataBuilderLevel(out, new MutableDataBlock(mode))
+                                            : new MutableLevel<>(out, new MutableDataBlock(mode));
+            while (terms.hasNext())
+            {
+                Pair<ByteBuffer, TokenTreeBuilder> term = terms.next();
+                addTerm(new InMemoryDataTerm(term.left, term.right), out);
+            }
+
+            dataLevel.finalFlush();
+            for (MutableLevel l : levels)
+                l.flush(); // flush all of the buffers
+
+            // and finally write levels index
+            final long levelIndexPosition = out.position();
+
+            out.writeInt(levels.size());
+            for (int i = levels.size() - 1; i >= 0; i--)
+                levels.get(i).flushMetadata();
+
+            dataLevel.flushMetadata();
+
+            out.writeLong(levelIndexPosition);
+
+            // sync contents of the output and disk,
+            // since it's not done implicitly on close
+            out.sync();
+        }
+        catch (IOException e)
+        {
+            throw new FSWriteError(e, file);
+        }
+        finally
+        {
+            FileUtils.closeQuietly(out);
+        }
+    }
+
+    private MutableLevel<InMemoryPointerTerm> getIndexLevel(int idx, SequentialWriter out)
+    {
+        if (levels.size() == 0)
+            levels.add(new MutableLevel<>(out, new MutableBlock<>()));
+
+        if (levels.size() - 1 < idx)
+        {
+            int toAdd = idx - (levels.size() - 1);
+            for (int i = 0; i < toAdd; i++)
+                levels.add(new MutableLevel<>(out, new MutableBlock<>()));
+        }
+
+        return levels.get(idx);
+    }
+
+    protected static void alignToBlock(SequentialWriter out) throws IOException
+    {
+        long endOfBlock = out.position();
+        if ((endOfBlock & (BLOCK_SIZE - 1)) != 0) // align on the block boundary if needed
+            out.skipBytes((int) (FBUtilities.align(endOfBlock, BLOCK_SIZE) - endOfBlock));
+    }
+
+    private class InMemoryTerm
+    {
+        protected final ByteBuffer term;
+
+        public InMemoryTerm(ByteBuffer term)
+        {
+            this.term = term;
+        }
+
+        public int serializedSize()
+        {
+            return (termSize.isConstant() ? 0 : 2) + term.remaining();
+        }
+
+        public void serialize(DataOutputPlus out) throws IOException
+        {
+            if (termSize.isConstant())
+                out.write(term);
+            else
+                ByteBufferUtil.writeWithShortLength(term, out);
+        }
+    }
+
+    private class InMemoryPointerTerm extends InMemoryTerm
+    {
+        protected final int blockCnt;
+
+        public InMemoryPointerTerm(ByteBuffer term, int blockCnt)
+        {
+            super(term);
+            this.blockCnt = blockCnt;
+        }
+
+        public int serializedSize()
+        {
+            return super.serializedSize() + 4;
+        }
+
+        public void serialize(DataOutputPlus out) throws IOException
+        {
+            super.serialize(out);
+            out.writeInt(blockCnt);
+        }
+    }
+
+    private class InMemoryDataTerm extends InMemoryTerm
+    {
+        private final TokenTreeBuilder keys;
+
+        public InMemoryDataTerm(ByteBuffer term, TokenTreeBuilder keys)
+        {
+            super(term);
+            this.keys = keys;
+        }
+    }
+
+    private class MutableLevel<T extends InMemoryTerm>
+    {
+        private final LongArrayList blockOffsets = new LongArrayList();
+
+        protected final SequentialWriter out;
+
+        private final MutableBlock<T> inProcessBlock;
+        private InMemoryPointerTerm lastTerm;
+
+        public MutableLevel(SequentialWriter out, MutableBlock<T> block)
+        {
+            this.out = out;
+            this.inProcessBlock = block;
+        }
+
+        /**
+         * @return If we flushed a block, return the last term of that block; else, null.
+         */
+        public InMemoryPointerTerm add(T term) throws IOException
+        {
+            InMemoryPointerTerm toPromote = null;
+
+            if (!inProcessBlock.hasSpaceFor(term))
+            {
+                flush();
+                toPromote = lastTerm;
+            }
+
+            inProcessBlock.add(term);
+
+            lastTerm = new InMemoryPointerTerm(term.term, blockOffsets.size());
+            return toPromote;
+        }
+
+        public void flush() throws IOException
+        {
+            blockOffsets.add(out.position());
+            inProcessBlock.flushAndClear(out);
+        }
+
+        public void finalFlush() throws IOException
+        {
+            flush();
+        }
+
+        public void flushMetadata() throws IOException
+        {
+            flushMetadata(blockOffsets);
+        }
+
+        protected void flushMetadata(LongArrayList longArrayList) throws IOException
+        {
+            out.writeInt(longArrayList.size());
+            for (int i = 0; i < longArrayList.size(); i++)
+                out.writeLong(longArrayList.get(i));
+        }
+    }
+
+    /** builds standard data blocks and super blocks, as well */
+    private class DataBuilderLevel extends MutableLevel<InMemoryDataTerm>
+    {
+        private final LongArrayList superBlockOffsets = new LongArrayList();
+
+        /** count of regular data blocks written since current super block was init'd */
+        private int dataBlocksCnt;
+        private TokenTreeBuilder superBlockTree;
+
+        public DataBuilderLevel(SequentialWriter out, MutableBlock<InMemoryDataTerm> block)
+        {
+            super(out, block);
+            superBlockTree = new TokenTreeBuilder();
+        }
+
+        public InMemoryPointerTerm add(InMemoryDataTerm term) throws IOException
+        {
+            InMemoryPointerTerm ptr = super.add(term);
+            if (ptr != null)
+            {
+                dataBlocksCnt++;
+                flushSuperBlock(false);
+            }
+            superBlockTree.add(term.keys.getTokens());
+            return ptr;
+        }
+
+        public void flushSuperBlock(boolean force) throws IOException
+        {
+            if (dataBlocksCnt == SUPER_BLOCK_SIZE || (force && !superBlockTree.getTokens().isEmpty()))
+            {
+                superBlockOffsets.add(out.position());
+                superBlockTree.finish().write(out);
+                alignToBlock(out);
+
+                dataBlocksCnt = 0;
+                superBlockTree = new TokenTreeBuilder();
+            }
+        }
+
+        public void finalFlush() throws IOException
+        {
+            super.flush();
+            flushSuperBlock(true);
+        }
+
+        public void flushMetadata() throws IOException
+        {
+            super.flushMetadata();
+            flushMetadata(superBlockOffsets);
+        }
+    }
+
+    private static class MutableBlock<T extends InMemoryTerm>
+    {
+        protected final DataOutputBufferFixed buffer;
+        protected final ShortArrayList offsets;
+
+        public MutableBlock()
+        {
+            buffer = new DataOutputBufferFixed(BLOCK_SIZE);
+            offsets = new ShortArrayList();
+        }
+
+        public final void add(T term) throws IOException
+        {
+            offsets.add((short) buffer.position());
+            addInternal(term);
+        }
+
+        protected void addInternal(T term) throws IOException
+        {
+            term.serialize(buffer);
+        }
+
+        public boolean hasSpaceFor(T element)
+        {
+            return sizeAfter(element) < BLOCK_SIZE;
+        }
+
+        protected int sizeAfter(T element)
+        {
+            return getWatermark() + 4 + element.serializedSize();
+        }
+
+        protected int getWatermark()
+        {
+            return 4 + offsets.size() * 2 + (int) buffer.position();
+        }
+
+        public void flushAndClear(SequentialWriter out) throws IOException
+        {
+            out.writeInt(offsets.size());
+            for (int i = 0; i < offsets.size(); i++)
+                out.writeShort(offsets.get(i));
+
+            out.write(buffer.buffer());
+
+            alignToBlock(out);
+
+            offsets.clear();
+            buffer.clear();
+        }
+    }
+
+    private static class MutableDataBlock extends MutableBlock<InMemoryDataTerm>
+    {
+        private final Mode mode;
+
+        private int offset = 0;
+        private int sparseValueTerms = 0;
+
+        private final List<TokenTreeBuilder> containers = new ArrayList<>();
+        private TokenTreeBuilder combinedIndex;
+
+        public MutableDataBlock(Mode mode)
+        {
+            this.mode = mode;
+            this.combinedIndex = new TokenTreeBuilder();
+        }
+
+        protected void addInternal(InMemoryDataTerm term) throws IOException
+        {
+            TokenTreeBuilder keys = term.keys;
+
+            if (mode == Mode.SPARSE && keys.getTokenCount() <= 5)
+            {
+                writeTerm(term, keys);
+                sparseValueTerms++;
+            }
+            else
+            {
+                writeTerm(term, offset);
+
+                offset += keys.serializedSize();
+                containers.add(keys);
+            }
+
+            if (mode == Mode.SPARSE)
+                combinedIndex.add(keys.getTokens());
+        }
+
+        protected int sizeAfter(InMemoryDataTerm element)
+        {
+            return super.sizeAfter(element) + ptrLength(element);
+        }
+
+        public void flushAndClear(SequentialWriter out) throws IOException
+        {
+            super.flushAndClear(out);
+
+            out.writeInt((sparseValueTerms == 0) ? -1 : offset);
+
+            if (containers.size() > 0)
+            {
+                for (TokenTreeBuilder tokens : containers)
+                    tokens.write(out);
+            }
+
+            if (sparseValueTerms > 0)
+            {
+                combinedIndex.finish().write(out);
+            }
+
+            alignToBlock(out);
+
+            containers.clear();
+            combinedIndex = new TokenTreeBuilder();
+
+            offset = 0;
+            sparseValueTerms = 0;
+        }
+
+        private int ptrLength(InMemoryDataTerm term)
+        {
+            return (term.keys.getTokenCount() > 5)
+                    ? 5 // 1 byte type + 4 byte offset to the tree
+                    : 1 + (8 * (int) term.keys.getTokenCount()); // 1 byte size + n 8 byte tokens
+        }
+
+        private void writeTerm(InMemoryTerm term, TokenTreeBuilder keys) throws IOException
+        {
+            term.serialize(buffer);
+            buffer.writeByte((byte) keys.getTokenCount());
+
+            Iterator<Pair<Long, LongSet>> tokens = keys.iterator();
+            while (tokens.hasNext())
+                buffer.writeLong(tokens.next().left);
+        }
+
+        private void writeTerm(InMemoryTerm term, int offset) throws IOException
+        {
+            term.serialize(buffer);
+            buffer.writeByte(0x0);
+            buffer.writeInt(offset);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
new file mode 100644
index 0000000..6e63c71
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
@@ -0,0 +1,361 @@
+/*
+ * 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.index.sasi.disk;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.index.sasi.analyzer.AbstractAnalyzer;
+import org.apache.cassandra.index.sasi.conf.ColumnIndex;
+import org.apache.cassandra.index.sasi.utils.CombinedTermIterator;
+import org.apache.cassandra.index.sasi.utils.TypeUtil;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableFlushObserver;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.Uninterruptibles;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerSSTableIndexWriter implements SSTableFlushObserver
+{
+    private static final Logger logger = LoggerFactory.getLogger(PerSSTableIndexWriter.class);
+
+    private static final ThreadPoolExecutor INDEX_FLUSHER_MEMTABLE;
+    private static final ThreadPoolExecutor INDEX_FLUSHER_GENERAL;
+
+    static
+    {
+        INDEX_FLUSHER_GENERAL = new JMXEnabledThreadPoolExecutor(1, 8, 60, TimeUnit.SECONDS,
+                                                                 new LinkedBlockingQueue<>(),
+                                                                 new NamedThreadFactory("SASI-General"),
+                                                                 "internal");
+        INDEX_FLUSHER_GENERAL.allowCoreThreadTimeOut(true);
+
+        INDEX_FLUSHER_MEMTABLE = new JMXEnabledThreadPoolExecutor(1, 8, 60, TimeUnit.SECONDS,
+                                                                  new LinkedBlockingQueue<>(),
+                                                                  new NamedThreadFactory("SASI-Memtable"),
+                                                                  "internal");
+        INDEX_FLUSHER_MEMTABLE.allowCoreThreadTimeOut(true);
+    }
+
+    private final int nowInSec = FBUtilities.nowInSeconds();
+
+    private final Descriptor descriptor;
+    private final OperationType source;
+
+    private final AbstractType<?> keyValidator;
+    private final Map<ColumnDefinition, ColumnIndex> supportedIndexes;
+
+    @VisibleForTesting
+    protected final Map<ColumnDefinition, Index> indexes;
+
+    private DecoratedKey currentKey;
+    private long currentKeyPosition;
+    private boolean isComplete;
+
+    public PerSSTableIndexWriter(AbstractType<?> keyValidator,
+                                 Descriptor descriptor,
+                                 OperationType source,
+                                 Map<ColumnDefinition, ColumnIndex> supportedIndexes)
+    {
+        this.keyValidator = keyValidator;
+        this.descriptor = descriptor;
+        this.source = source;
+        this.supportedIndexes = supportedIndexes;
+        this.indexes = new HashMap<>();
+    }
+
+    public void begin()
+    {}
+
+    public void startPartition(DecoratedKey key, long curPosition)
+    {
+        currentKey = key;
+        currentKeyPosition = curPosition;
+    }
+
+    public void nextUnfilteredCluster(Unfiltered unfiltered)
+    {
+        if (!unfiltered.isRow())
+            return;
+
+        Row row = (Row) unfiltered;
+
+        supportedIndexes.keySet().forEach((column) -> {
+            ByteBuffer value = ColumnIndex.getValueOf(column, row, nowInSec);
+            if (value == null)
+                return;
+
+            ColumnIndex columnIndex = supportedIndexes.get(column);
+            if (columnIndex == null)
+                return;
+
+            Index index = indexes.get(column);
+            if (index == null)
+                indexes.put(column, (index = new Index(columnIndex)));
+
+            index.add(value.duplicate(), currentKey, currentKeyPosition);
+        });
+    }
+
+    public void complete()
+    {
+        if (isComplete)
+            return;
+
+        currentKey = null;
+
+        try
+        {
+            CountDownLatch latch = new CountDownLatch(indexes.size());
+            for (Index index : indexes.values())
+                index.complete(latch);
+
+            Uninterruptibles.awaitUninterruptibly(latch);
+        }
+        finally
+        {
+            indexes.clear();
+            isComplete = true;
+        }
+    }
+
+    public Index getIndex(ColumnDefinition columnDef)
+    {
+        return indexes.get(columnDef);
+    }
+
+    public Descriptor getDescriptor()
+    {
+        return descriptor;
+    }
+
+    @VisibleForTesting
+    protected class Index
+    {
+        private final ColumnIndex columnIndex;
+        private final String outputFile;
+        private final AbstractAnalyzer analyzer;
+        private final long maxMemorySize;
+
+        @VisibleForTesting
+        protected final Set<Future<OnDiskIndex>> segments;
+        private int segmentNumber = 0;
+
+        private OnDiskIndexBuilder currentBuilder;
+
+        public Index(ColumnIndex columnIndex)
+        {
+            this.columnIndex = columnIndex;
+            this.outputFile = descriptor.filenameFor(columnIndex.getComponent());
+            this.analyzer = columnIndex.getAnalyzer();
+            this.segments = new HashSet<>();
+            this.maxMemorySize = maxMemorySize(columnIndex);
+            this.currentBuilder = newIndexBuilder();
+        }
+
+        public void add(ByteBuffer term, DecoratedKey key, long keyPosition)
+        {
+            if (term.remaining() == 0)
+                return;
+
+            boolean isAdded = false;
+
+            analyzer.reset(term);
+            while (analyzer.hasNext())
+            {
+                ByteBuffer token = analyzer.next();
+                int size = token.remaining();
+
+                if (token.remaining() >= OnDiskIndexBuilder.MAX_TERM_SIZE)
+                {
+                    logger.info("Rejecting value (size {}, maximum {} bytes) for column {} (analyzed {}) at {} SSTable.",
+                            term.remaining(),
+                            OnDiskIndexBuilder.MAX_TERM_SIZE,
+                            columnIndex.getColumnName(),
+                            columnIndex.getMode().isAnalyzed,
+                            descriptor);
+                    continue;
+                }
+
+                if (!TypeUtil.isValid(token, columnIndex.getValidator()))
+                {
+                    if ((token = TypeUtil.tryUpcast(token, columnIndex.getValidator())) == null)
+                    {
+                        logger.info("({}) Failed to add {} to index for key: {}, value size was {} bytes, validator is {}.",
+                                    outputFile,
+                                    columnIndex.getColumnName(),
+                                    keyValidator.getString(key.getKey()),
+                                    size,
+                                    columnIndex.getValidator());
+                        continue;
+                    }
+                }
+
+                currentBuilder.add(token, key, keyPosition);
+                isAdded = true;
+            }
+
+            if (!isAdded || currentBuilder.estimatedMemoryUse() < maxMemorySize)
+                return; // non of the generated tokens were added to the index or memory size wasn't reached
+
+            segments.add(getExecutor().submit(scheduleSegmentFlush(false)));
+        }
+
+        @VisibleForTesting
+        protected Callable<OnDiskIndex> scheduleSegmentFlush(final boolean isFinal)
+        {
+            final OnDiskIndexBuilder builder = currentBuilder;
+            currentBuilder = newIndexBuilder();
+
+            final String segmentFile = filename(isFinal);
+
+            return () -> {
+                long start1 = System.nanoTime();
+
+                try
+                {
+                    File index = new File(segmentFile);
+                    return builder.finish(index) ? new OnDiskIndex(index, columnIndex.getValidator(), null) : null;
+                }
+                finally
+                {
+                    if (!isFinal)
+                        logger.info("Flushed index segment {}, took {} ms.", segmentFile, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start1));
+                }
+            };
+        }
+
+        public void complete(final CountDownLatch latch)
+        {
+            logger.info("Scheduling index flush to {}", outputFile);
+
+            getExecutor().submit((Runnable) () -> {
+                long start1 = System.nanoTime();
+
+                OnDiskIndex[] parts = new OnDiskIndex[segments.size() + 1];
+
+                try
+                {
+                    // no parts present, build entire index from memory
+                    if (segments.isEmpty())
+                    {
+                        scheduleSegmentFlush(true).call();
+                        return;
+                    }
+
+                    // parts are present but there is something still in memory, let's flush that inline
+                    if (!currentBuilder.isEmpty())
+                    {
+                        OnDiskIndex last = scheduleSegmentFlush(false).call();
+                        segments.add(Futures.immediateFuture(last));
+                    }
+
+                    int index = 0;
+                    ByteBuffer combinedMin = null, combinedMax = null;
+
+                    for (Future<OnDiskIndex> f : segments)
+                    {
+                        OnDiskIndex part = Futures.getUnchecked(f);
+                        if (part == null)
+                            continue;
+
+                        parts[index++] = part;
+                        combinedMin = (combinedMin == null || keyValidator.compare(combinedMin, part.minKey()) > 0) ? part.minKey() : combinedMin;
+                        combinedMax = (combinedMax == null || keyValidator.compare(combinedMax, part.maxKey()) < 0) ? part.maxKey() : combinedMax;
+                    }
+
+                    OnDiskIndexBuilder builder = newIndexBuilder();
+                    builder.finish(Pair.create(combinedMin, combinedMax),
+                                   new File(outputFile),
+                                   new CombinedTermIterator(parts));
+                }
+                catch (Exception e)
+                {
+                    logger.error("Failed to flush index {}.", outputFile, e);
+                    FileUtils.delete(outputFile);
+                }
+                finally
+                {
+                    logger.info("Index flush to {} took {} ms.", outputFile, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start1));
+
+                    for (OnDiskIndex part : parts)
+                    {
+                        if (part == null)
+                            continue;
+
+                        FileUtils.closeQuietly(part);
+                        FileUtils.delete(part.getIndexPath());
+                    }
+
+                    latch.countDown();
+                }
+            });
+        }
+
+        private ExecutorService getExecutor()
+        {
+            return source == OperationType.FLUSH ? INDEX_FLUSHER_MEMTABLE : INDEX_FLUSHER_GENERAL;
+        }
+
+        private OnDiskIndexBuilder newIndexBuilder()
+        {
+            return new OnDiskIndexBuilder(keyValidator, columnIndex.getValidator(), columnIndex.getMode().mode);
+        }
+
+        public String filename(boolean isFinal)
+        {
+            return outputFile + (isFinal ? "" : "_" + segmentNumber++);
+        }
+    }
+
+    protected long maxMemorySize(ColumnIndex columnIndex)
+    {
+        // 1G for memtable and configuration for compaction
+        return source == OperationType.FLUSH ? 1073741824L : columnIndex.getMode().maxCompactionFlushMemoryInMb;
+    }
+
+    public int hashCode()
+    {
+        return descriptor.hashCode();
+    }
+
+    public boolean equals(Object o)
+    {
+        return !(o == null || !(o instanceof PerSSTableIndexWriter)) && descriptor.equals(((PerSSTableIndexWriter) o).descriptor);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/disk/Token.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/Token.java b/src/java/org/apache/cassandra/index/sasi/disk/Token.java
new file mode 100644
index 0000000..02130a3
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/disk/Token.java
@@ -0,0 +1,42 @@
+/*
+ * 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.index.sasi.disk;
+
+import com.google.common.primitives.Longs;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.index.sasi.utils.CombinedValue;
+
+public abstract class Token implements CombinedValue<Long>, Iterable<DecoratedKey>
+{
+    protected final long token;
+
+    public Token(long token)
+    {
+        this.token = token;
+    }
+
+    public Long get()
+    {
+        return token;
+    }
+
+    public int compareTo(CombinedValue<Long> o)
+    {
+        return Longs.compare(token, ((Token) o).token);
+    }
+}


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

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzerTest.java b/test/unit/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzerTest.java
new file mode 100644
index 0000000..ba67853
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzerTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.index.sasi.analyzer;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for the non-tokenizing analyzer
+ */
+public class NonTokenizingAnalyzerTest
+{
+    @Test
+    public void caseInsensitiveAnalizer() throws Exception
+    {
+        NonTokenizingAnalyzer analyzer = new NonTokenizingAnalyzer();
+        NonTokenizingOptions options = NonTokenizingOptions.getDefaultOptions();
+        options.setCaseSensitive(false);
+        analyzer.init(options, UTF8Type.instance);
+
+        String testString = "Nip it in the bud";
+        ByteBuffer toAnalyze = ByteBuffer.wrap(testString.getBytes());
+        analyzer.reset(toAnalyze);
+        ByteBuffer analyzed = null;
+        while (analyzer.hasNext())
+            analyzed = analyzer.next();
+        Assert.assertTrue(testString.toLowerCase().equals(ByteBufferUtil.string(analyzed)));
+    }
+
+    @Test
+    public void caseSensitiveAnalizer() throws Exception
+    {
+        NonTokenizingAnalyzer analyzer = new NonTokenizingAnalyzer();
+        NonTokenizingOptions options = NonTokenizingOptions.getDefaultOptions();
+        analyzer.init(options, UTF8Type.instance);
+
+        String testString = "Nip it in the bud";
+        ByteBuffer toAnalyze = ByteBuffer.wrap(testString.getBytes());
+        analyzer.reset(toAnalyze);
+        ByteBuffer analyzed = null;
+        while (analyzer.hasNext())
+            analyzed = analyzer.next();
+        Assert.assertFalse(testString.toLowerCase().equals(ByteBufferUtil.string(analyzed)));
+    }
+
+    @Test
+    public void ensureIncompatibleInputSkipped() throws Exception
+    {
+        NonTokenizingAnalyzer analyzer = new NonTokenizingAnalyzer();
+        NonTokenizingOptions options = NonTokenizingOptions.getDefaultOptions();
+        analyzer.init(options, Int32Type.instance);
+
+        ByteBuffer toAnalyze = ByteBufferUtil.bytes(1);
+        analyzer.reset(toAnalyze);
+        Assert.assertTrue(!analyzer.hasNext());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzerTest.java b/test/unit/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzerTest.java
new file mode 100644
index 0000000..e307512
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzerTest.java
@@ -0,0 +1,196 @@
+/*
+ * 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.index.sasi.analyzer;
+
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class StandardAnalyzerTest
+{
+    @Test
+    public void testTokenizationAscii() throws Exception
+    {
+        InputStream is = StandardAnalyzerTest.class.getClassLoader()
+                .getResourceAsStream("tokenization/apache_license_header.txt");
+
+        StandardTokenizerOptions options = new StandardTokenizerOptions.OptionsBuilder()
+                .maxTokenLength(5).build();
+        StandardAnalyzer tokenizer = new StandardAnalyzer();
+        tokenizer.init(options);
+
+        List<ByteBuffer> tokens = new ArrayList<>();
+        tokenizer.reset(is);
+        while (tokenizer.hasNext())
+            tokens.add(tokenizer.next());
+
+        assertEquals(67, tokens.size());
+    }
+
+    @Test
+    public void testTokenizationLoremIpsum() throws Exception
+    {
+        InputStream is = StandardAnalyzerTest.class.getClassLoader()
+                .getResourceAsStream("tokenization/lorem_ipsum.txt");
+
+        StandardAnalyzer tokenizer = new StandardAnalyzer();
+        tokenizer.init(StandardTokenizerOptions.getDefaultOptions());
+
+        List<ByteBuffer> tokens = new ArrayList<>();
+        tokenizer.reset(is);
+        while (tokenizer.hasNext())
+            tokens.add(tokenizer.next());
+
+        assertEquals(62, tokens.size());
+
+    }
+
+    @Test
+    public void testTokenizationJaJp1() throws Exception
+    {
+        InputStream is = StandardAnalyzerTest.class.getClassLoader()
+                .getResourceAsStream("tokenization/ja_jp_1.txt");
+
+        StandardAnalyzer tokenizer = new StandardAnalyzer();
+        tokenizer.init(StandardTokenizerOptions.getDefaultOptions());
+
+        tokenizer.reset(is);
+        List<ByteBuffer> tokens = new ArrayList<>();
+        while (tokenizer.hasNext())
+            tokens.add(tokenizer.next());
+
+        assertEquals(210, tokens.size());
+    }
+
+    @Test
+    public void testTokenizationJaJp2() throws Exception
+    {
+        InputStream is = StandardAnalyzerTest.class.getClassLoader()
+                .getResourceAsStream("tokenization/ja_jp_2.txt");
+
+        StandardTokenizerOptions options = new StandardTokenizerOptions.OptionsBuilder().stemTerms(true)
+                .ignoreStopTerms(true).alwaysLowerCaseTerms(true).build();
+        StandardAnalyzer tokenizer = new StandardAnalyzer();
+        tokenizer.init(options);
+
+        tokenizer.reset(is);
+        List<ByteBuffer> tokens = new ArrayList<>();
+        while (tokenizer.hasNext())
+            tokens.add(tokenizer.next());
+
+        assertEquals(57, tokens.size());
+    }
+
+    @Test
+    public void testTokenizationRuRu1() throws Exception
+    {
+        InputStream is = StandardAnalyzerTest.class.getClassLoader()
+                .getResourceAsStream("tokenization/ru_ru_1.txt");
+        StandardAnalyzer tokenizer = new StandardAnalyzer();
+        tokenizer.init(StandardTokenizerOptions.getDefaultOptions());
+
+        List<ByteBuffer> tokens = new ArrayList<>();
+        tokenizer.reset(is);
+        while (tokenizer.hasNext())
+            tokens.add(tokenizer.next());
+
+        assertEquals(456, tokens.size());
+    }
+
+    @Test
+    public void testTokenizationZnTw1() throws Exception
+    {
+        InputStream is = StandardAnalyzerTest.class.getClassLoader()
+                .getResourceAsStream("tokenization/zn_tw_1.txt");
+        StandardAnalyzer tokenizer = new StandardAnalyzer();
+        tokenizer.init(StandardTokenizerOptions.getDefaultOptions());
+
+        List<ByteBuffer> tokens = new ArrayList<>();
+        tokenizer.reset(is);
+        while (tokenizer.hasNext())
+            tokens.add(tokenizer.next());
+
+        assertEquals(963, tokens.size());
+    }
+
+    @Test
+    public void testTokenizationAdventuresOfHuckFinn() throws Exception
+    {
+        InputStream is = StandardAnalyzerTest.class.getClassLoader()
+                .getResourceAsStream("tokenization/adventures_of_huckleberry_finn_mark_twain.txt");
+
+        StandardTokenizerOptions options = new StandardTokenizerOptions.OptionsBuilder().stemTerms(true)
+                .ignoreStopTerms(true).useLocale(Locale.ENGLISH)
+                .alwaysLowerCaseTerms(true).build();
+        StandardAnalyzer tokenizer = new StandardAnalyzer();
+        tokenizer.init(options);
+
+        List<ByteBuffer> tokens = new ArrayList<>();
+        tokenizer.reset(is);
+        while (tokenizer.hasNext())
+            tokens.add(tokenizer.next());
+
+        assertEquals(40249, tokens.size());
+    }
+
+    @Test
+    public void tokenizeDomainNamesAndUrls() throws Exception
+    {
+        InputStream is = StandardAnalyzerTest.class.getClassLoader()
+                .getResourceAsStream("tokenization/top_visited_domains.txt");
+
+        StandardAnalyzer tokenizer = new StandardAnalyzer();
+        tokenizer.init(StandardTokenizerOptions.getDefaultOptions());
+        tokenizer.reset(is);
+
+        List<ByteBuffer> tokens = new ArrayList<>();
+        while (tokenizer.hasNext())
+            tokens.add(tokenizer.next());
+
+        assertEquals(15, tokens.size());
+    }
+
+    @Test
+    public void testReuseAndResetTokenizerInstance() throws Exception
+    {
+        List<ByteBuffer> bbToTokenize = new ArrayList<>();
+        bbToTokenize.add(ByteBuffer.wrap("Nip it in the bud".getBytes()));
+        bbToTokenize.add(ByteBuffer.wrap("I couldn’t care less".getBytes()));
+        bbToTokenize.add(ByteBuffer.wrap("One and the same".getBytes()));
+        bbToTokenize.add(ByteBuffer.wrap("The squeaky wheel gets the grease.".getBytes()));
+        bbToTokenize.add(ByteBuffer.wrap("The pen is mightier than the sword.".getBytes()));
+
+        StandardAnalyzer tokenizer = new StandardAnalyzer();
+        tokenizer.init(StandardTokenizerOptions.getDefaultOptions());
+
+        List<ByteBuffer> tokens = new ArrayList<>();
+        for (ByteBuffer bb : bbToTokenize)
+        {
+            tokenizer.reset(bb);
+            while (tokenizer.hasNext())
+                tokens.add(tokenizer.next());
+        }
+        assertEquals(10, tokens.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/index/sasi/disk/OnDiskIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/disk/OnDiskIndexTest.java b/test/unit/org/apache/cassandra/index/sasi/disk/OnDiskIndexTest.java
new file mode 100644
index 0000000..57629b6
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/disk/OnDiskIndexTest.java
@@ -0,0 +1,856 @@
+/*
+ * 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.index.sasi.disk;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.BufferDecoratedKey;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.index.sasi.plan.Expression;
+import org.apache.cassandra.index.sasi.utils.CombinedTerm;
+import org.apache.cassandra.index.sasi.utils.CombinedTermIterator;
+import org.apache.cassandra.index.sasi.utils.OnDiskIndexIterator;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.utils.MurmurHash;
+import org.apache.cassandra.utils.Pair;
+
+import com.carrotsearch.hppc.LongSet;
+import com.carrotsearch.hppc.cursors.LongCursor;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
+
+import junit.framework.Assert;
+import org.junit.Test;
+
+public class OnDiskIndexTest
+{
+    @Test
+    public void testStringSAConstruction() throws Exception
+    {
+        Map<ByteBuffer, TokenTreeBuilder> data = new HashMap<ByteBuffer, TokenTreeBuilder>()
+        {{
+                put(UTF8Type.instance.decompose("scat"), keyBuilder(1L));
+                put(UTF8Type.instance.decompose("mat"),  keyBuilder(2L));
+                put(UTF8Type.instance.decompose("fat"),  keyBuilder(3L));
+                put(UTF8Type.instance.decompose("cat"),  keyBuilder(1L, 4L));
+                put(UTF8Type.instance.decompose("till"), keyBuilder(2L, 6L));
+                put(UTF8Type.instance.decompose("bill"), keyBuilder(5L));
+                put(UTF8Type.instance.decompose("foo"),  keyBuilder(7L));
+                put(UTF8Type.instance.decompose("bar"),  keyBuilder(9L, 10L));
+                put(UTF8Type.instance.decompose("michael"), keyBuilder(11L, 12L, 1L));
+        }};
+
+        OnDiskIndexBuilder builder = new OnDiskIndexBuilder(UTF8Type.instance, UTF8Type.instance, OnDiskIndexBuilder.Mode.CONTAINS);
+        for (Map.Entry<ByteBuffer, TokenTreeBuilder> e : data.entrySet())
+            addAll(builder, e.getKey(), e.getValue());
+
+        File index = File.createTempFile("on-disk-sa-string", "db");
+        index.deleteOnExit();
+
+        builder.finish(index);
+
+        OnDiskIndex onDisk = new OnDiskIndex(index, UTF8Type.instance, new KeyConverter());
+
+        // first check if we can find exact matches
+        for (Map.Entry<ByteBuffer, TokenTreeBuilder> e : data.entrySet())
+        {
+            if (UTF8Type.instance.getString(e.getKey()).equals("cat"))
+                continue; // cat is embedded into scat, we'll test it in next section
+
+            Assert.assertEquals("Key was: " + UTF8Type.instance.compose(e.getKey()), convert(e.getValue()), convert(onDisk.search(expressionFor(UTF8Type.instance, e.getKey()))));
+        }
+
+        // check that cat returns positions for scat & cat
+        Assert.assertEquals(convert(1, 4), convert(onDisk.search(expressionFor("cat"))));
+
+        // random suffix queries
+        Assert.assertEquals(convert(9, 10), convert(onDisk.search(expressionFor("ar"))));
+        Assert.assertEquals(convert(1, 2, 3, 4), convert(onDisk.search(expressionFor("at"))));
+        Assert.assertEquals(convert(1, 11, 12), convert(onDisk.search(expressionFor("mic"))));
+        Assert.assertEquals(convert(1, 11, 12), convert(onDisk.search(expressionFor("ae"))));
+        Assert.assertEquals(convert(2, 5, 6), convert(onDisk.search(expressionFor("ll"))));
+        Assert.assertEquals(convert(1, 2, 5, 6, 11, 12), convert(onDisk.search(expressionFor("l"))));
+        Assert.assertEquals(convert(7), convert(onDisk.search(expressionFor("oo"))));
+        Assert.assertEquals(convert(7), convert(onDisk.search(expressionFor("o"))));
+        Assert.assertEquals(convert(1, 2, 3, 4, 6), convert(onDisk.search(expressionFor("t"))));
+
+        Assert.assertEquals(Collections.<DecoratedKey>emptySet(), convert(onDisk.search(expressionFor("hello"))));
+
+        onDisk.close();
+    }
+
+    @Test
+    public void testIntegerSAConstruction() throws Exception
+    {
+        final Map<ByteBuffer, TokenTreeBuilder> data = new HashMap<ByteBuffer, TokenTreeBuilder>()
+        {{
+                put(Int32Type.instance.decompose(5),  keyBuilder(1L));
+                put(Int32Type.instance.decompose(7),  keyBuilder(2L));
+                put(Int32Type.instance.decompose(1),  keyBuilder(3L));
+                put(Int32Type.instance.decompose(3),  keyBuilder(1L, 4L));
+                put(Int32Type.instance.decompose(8),  keyBuilder(2L, 6L));
+                put(Int32Type.instance.decompose(10), keyBuilder(5L));
+                put(Int32Type.instance.decompose(6),  keyBuilder(7L));
+                put(Int32Type.instance.decompose(4),  keyBuilder(9L, 10L));
+                put(Int32Type.instance.decompose(0),  keyBuilder(11L, 12L, 1L));
+        }};
+
+        OnDiskIndexBuilder builder = new OnDiskIndexBuilder(UTF8Type.instance, Int32Type.instance, OnDiskIndexBuilder.Mode.PREFIX);
+        for (Map.Entry<ByteBuffer, TokenTreeBuilder> e : data.entrySet())
+            addAll(builder, e.getKey(), e.getValue());
+
+        File index = File.createTempFile("on-disk-sa-int", "db");
+        index.deleteOnExit();
+
+        builder.finish(index);
+
+        OnDiskIndex onDisk = new OnDiskIndex(index, Int32Type.instance, new KeyConverter());
+
+        for (Map.Entry<ByteBuffer, TokenTreeBuilder> e : data.entrySet())
+        {
+            Assert.assertEquals(convert(e.getValue()), convert(onDisk.search(expressionFor(Int32Type.instance, e.getKey()))));
+        }
+
+        List<ByteBuffer> sortedNumbers = new ArrayList<ByteBuffer>()
+        {{
+            addAll(data.keySet().stream().collect(Collectors.toList()));
+        }};
+
+        Collections.sort(sortedNumbers, Int32Type.instance::compare);
+
+        // test full iteration
+        int idx = 0;
+        for (OnDiskIndex.DataTerm term : onDisk)
+        {
+            ByteBuffer number = sortedNumbers.get(idx++);
+            Assert.assertEquals(number, term.getTerm());
+            Assert.assertEquals(convert(data.get(number)), convert(term.getTokens()));
+        }
+
+        // test partial iteration (descending)
+        idx = 3; // start from the 3rd element
+        Iterator<OnDiskIndex.DataTerm> partialIter = onDisk.iteratorAt(sortedNumbers.get(idx), OnDiskIndex.IteratorOrder.DESC, true);
+        while (partialIter.hasNext())
+        {
+            OnDiskIndex.DataTerm term = partialIter.next();
+            ByteBuffer number = sortedNumbers.get(idx++);
+
+            Assert.assertEquals(number, term.getTerm());
+            Assert.assertEquals(convert(data.get(number)), convert(term.getTokens()));
+        }
+
+        idx = 3; // start from the 3rd element exclusive
+        partialIter = onDisk.iteratorAt(sortedNumbers.get(idx++), OnDiskIndex.IteratorOrder.DESC, false);
+        while (partialIter.hasNext())
+        {
+            OnDiskIndex.DataTerm term = partialIter.next();
+            ByteBuffer number = sortedNumbers.get(idx++);
+
+            Assert.assertEquals(number, term.getTerm());
+            Assert.assertEquals(convert(data.get(number)), convert(term.getTokens()));
+        }
+
+        // test partial iteration (ascending)
+        idx = 6; // start from the 6rd element
+        partialIter = onDisk.iteratorAt(sortedNumbers.get(idx), OnDiskIndex.IteratorOrder.ASC, true);
+        while (partialIter.hasNext())
+        {
+            OnDiskIndex.DataTerm term = partialIter.next();
+            ByteBuffer number = sortedNumbers.get(idx--);
+
+            Assert.assertEquals(number, term.getTerm());
+            Assert.assertEquals(convert(data.get(number)), convert(term.getTokens()));
+        }
+
+        idx = 6; // start from the 6rd element exclusive
+        partialIter = onDisk.iteratorAt(sortedNumbers.get(idx--), OnDiskIndex.IteratorOrder.ASC, false);
+        while (partialIter.hasNext())
+        {
+            OnDiskIndex.DataTerm term = partialIter.next();
+            ByteBuffer number = sortedNumbers.get(idx--);
+
+            Assert.assertEquals(number, term.getTerm());
+            Assert.assertEquals(convert(data.get(number)), convert(term.getTokens()));
+        }
+
+        onDisk.close();
+
+        List<ByteBuffer> iterCheckNums = new ArrayList<ByteBuffer>()
+        {{
+            add(Int32Type.instance.decompose(3));
+            add(Int32Type.instance.decompose(9));
+            add(Int32Type.instance.decompose(14));
+            add(Int32Type.instance.decompose(42));
+        }};
+
+        OnDiskIndexBuilder iterTest = new OnDiskIndexBuilder(UTF8Type.instance, Int32Type.instance, OnDiskIndexBuilder.Mode.PREFIX);
+        for (int i = 0; i < iterCheckNums.size(); i++)
+            iterTest.add(iterCheckNums.get(i), keyAt((long) i), i);
+
+        File iterIndex = File.createTempFile("sa-iter", ".db");
+        iterIndex.deleteOnExit();
+
+        iterTest.finish(iterIndex);
+
+        onDisk = new OnDiskIndex(iterIndex, Int32Type.instance, new KeyConverter());
+
+        ByteBuffer number = Int32Type.instance.decompose(1);
+        Assert.assertEquals(0, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.ASC, false)));
+        Assert.assertEquals(0, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.ASC, true)));
+        Assert.assertEquals(4, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.DESC, false)));
+        Assert.assertEquals(4, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.DESC, true)));
+
+        number = Int32Type.instance.decompose(44);
+        Assert.assertEquals(4, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.ASC, false)));
+        Assert.assertEquals(4, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.ASC, true)));
+        Assert.assertEquals(0, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.DESC, false)));
+        Assert.assertEquals(0, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.DESC, true)));
+
+        number = Int32Type.instance.decompose(20);
+        Assert.assertEquals(3, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.ASC, false)));
+        Assert.assertEquals(3, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.ASC, true)));
+        Assert.assertEquals(1, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.DESC, false)));
+        Assert.assertEquals(1, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.DESC, true)));
+
+        number = Int32Type.instance.decompose(5);
+        Assert.assertEquals(1, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.ASC, false)));
+        Assert.assertEquals(1, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.ASC, true)));
+        Assert.assertEquals(3, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.DESC, false)));
+        Assert.assertEquals(3, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.DESC, true)));
+
+        number = Int32Type.instance.decompose(10);
+        Assert.assertEquals(2, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.ASC, false)));
+        Assert.assertEquals(2, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.ASC, true)));
+        Assert.assertEquals(2, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.DESC, false)));
+        Assert.assertEquals(2, Iterators.size(onDisk.iteratorAt(number, OnDiskIndex.IteratorOrder.DESC, true)));
+
+        onDisk.close();
+    }
+
+    @Test
+    public void testMultiSuffixMatches() throws Exception
+    {
+        OnDiskIndexBuilder builder = new OnDiskIndexBuilder(UTF8Type.instance, UTF8Type.instance, OnDiskIndexBuilder.Mode.CONTAINS)
+        {{
+                addAll(this, UTF8Type.instance.decompose("Eliza"), keyBuilder(1L, 2L));
+                addAll(this, UTF8Type.instance.decompose("Elizabeth"), keyBuilder(3L, 4L));
+                addAll(this, UTF8Type.instance.decompose("Aliza"), keyBuilder(5L, 6L));
+                addAll(this, UTF8Type.instance.decompose("Taylor"), keyBuilder(7L, 8L));
+                addAll(this, UTF8Type.instance.decompose("Pavel"), keyBuilder(9L, 10L));
+        }};
+
+        File index = File.createTempFile("on-disk-sa-multi-suffix-match", ".db");
+        index.deleteOnExit();
+
+        builder.finish(index);
+
+        OnDiskIndex onDisk = new OnDiskIndex(index, UTF8Type.instance, new KeyConverter());
+
+        Assert.assertEquals(convert(1, 2, 3, 4, 5, 6), convert(onDisk.search(expressionFor("liz"))));
+        Assert.assertEquals(convert(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), convert(onDisk.search(expressionFor("a"))));
+        Assert.assertEquals(convert(5, 6), convert(onDisk.search(expressionFor("A"))));
+        Assert.assertEquals(convert(1, 2, 3, 4), convert(onDisk.search(expressionFor("E"))));
+        Assert.assertEquals(convert(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), convert(onDisk.search(expressionFor("l"))));
+        Assert.assertEquals(convert(3, 4), convert(onDisk.search(expressionFor("bet"))));
+        Assert.assertEquals(convert(3, 4, 9, 10), convert(onDisk.search(expressionFor("e"))));
+        Assert.assertEquals(convert(7, 8), convert(onDisk.search(expressionFor("yl"))));
+        Assert.assertEquals(convert(7, 8), convert(onDisk.search(expressionFor("T"))));
+        Assert.assertEquals(convert(1, 2, 3, 4, 5, 6), convert(onDisk.search(expressionFor("za"))));
+        Assert.assertEquals(convert(3, 4), convert(onDisk.search(expressionFor("ab"))));
+
+        Assert.assertEquals(Collections.<DecoratedKey>emptySet(), convert(onDisk.search(expressionFor("Pi"))));
+        Assert.assertEquals(Collections.<DecoratedKey>emptySet(), convert(onDisk.search(expressionFor("ethz"))));
+        Assert.assertEquals(Collections.<DecoratedKey>emptySet(), convert(onDisk.search(expressionFor("liw"))));
+        Assert.assertEquals(Collections.<DecoratedKey>emptySet(), convert(onDisk.search(expressionFor("Taw"))));
+        Assert.assertEquals(Collections.<DecoratedKey>emptySet(), convert(onDisk.search(expressionFor("Av"))));
+
+        onDisk.close();
+    }
+
+    @Test
+    public void testSparseMode() throws Exception
+    {
+        OnDiskIndexBuilder builder = new OnDiskIndexBuilder(UTF8Type.instance, LongType.instance, OnDiskIndexBuilder.Mode.SPARSE);
+
+        final long start = System.currentTimeMillis();
+        final int numIterations = 100000;
+
+        for (long i = 0; i < numIterations; i++)
+            builder.add(LongType.instance.decompose(start + i), keyAt(i), i);
+
+        File index = File.createTempFile("on-disk-sa-sparse", "db");
+        index.deleteOnExit();
+
+        builder.finish(index);
+
+        OnDiskIndex onDisk = new OnDiskIndex(index, LongType.instance, new KeyConverter());
+
+        ThreadLocalRandom random = ThreadLocalRandom.current();
+
+        for (long step = start; step < (start + numIterations); step += 1000)
+        {
+            boolean lowerInclusive = random.nextBoolean();
+            boolean upperInclusive = random.nextBoolean();
+
+            long limit = random.nextLong(step, start + numIterations);
+            RangeIterator<Long, Token> rows = onDisk.search(expressionFor(step, lowerInclusive, limit, upperInclusive));
+
+            long lowerKey = step - start;
+            long upperKey = lowerKey + (limit - step);
+
+            if (!lowerInclusive)
+                lowerKey += 1;
+
+            if (upperInclusive)
+                upperKey += 1;
+
+            Set<DecoratedKey> actual = convert(rows);
+            for (long key = lowerKey; key < upperKey; key++)
+                Assert.assertTrue("key" + key + " wasn't found", actual.contains(keyAt(key)));
+
+            Assert.assertEquals((upperKey - lowerKey), actual.size());
+        }
+
+        // let's also explicitly test whole range search
+        RangeIterator<Long, Token> rows = onDisk.search(expressionFor(start, true, start + numIterations, true));
+
+        Set<DecoratedKey> actual = convert(rows);
+        Assert.assertEquals(numIterations, actual.size());
+    }
+
+    @Test
+    public void testNotEqualsQueryForStrings() throws Exception
+    {
+        Map<ByteBuffer, TokenTreeBuilder> data = new HashMap<ByteBuffer, TokenTreeBuilder>()
+        {{
+                put(UTF8Type.instance.decompose("Pavel"),   keyBuilder(1L, 2L));
+                put(UTF8Type.instance.decompose("Jason"),   keyBuilder(3L));
+                put(UTF8Type.instance.decompose("Jordan"),  keyBuilder(4L));
+                put(UTF8Type.instance.decompose("Michael"), keyBuilder(5L, 6L));
+                put(UTF8Type.instance.decompose("Vijay"),   keyBuilder(7L));
+                put(UTF8Type.instance.decompose("Travis"),  keyBuilder(8L));
+                put(UTF8Type.instance.decompose("Aleksey"), keyBuilder(9L, 10L));
+        }};
+
+        OnDiskIndexBuilder builder = new OnDiskIndexBuilder(UTF8Type.instance, UTF8Type.instance, OnDiskIndexBuilder.Mode.PREFIX);
+        for (Map.Entry<ByteBuffer, TokenTreeBuilder> e : data.entrySet())
+            addAll(builder, e.getKey(), e.getValue());
+
+        File index = File.createTempFile("on-disk-sa-except-test", "db");
+        index.deleteOnExit();
+
+        builder.finish(index);
+
+        OnDiskIndex onDisk = new OnDiskIndex(index, UTF8Type.instance, new KeyConverter());
+
+        // test whole words first
+        Assert.assertEquals(convert(3, 4, 5, 6, 7, 8, 9, 10), convert(onDisk.search(expressionForNot("Aleksey", "Vijay", "Pavel"))));
+
+        Assert.assertEquals(convert(3, 4, 7, 8, 9, 10), convert(onDisk.search(expressionForNot("Aleksey", "Vijay", "Pavel", "Michael"))));
+
+        Assert.assertEquals(convert(3, 4, 7, 9, 10), convert(onDisk.search(expressionForNot("Aleksey", "Vijay", "Pavel", "Michael", "Travis"))));
+
+        // now test prefixes
+        Assert.assertEquals(convert(3, 4, 5, 6, 7, 8, 9, 10), convert(onDisk.search(expressionForNot("Aleksey", "Vijay", "Pav"))));
+
+        Assert.assertEquals(convert(3, 4, 7, 8, 9, 10), convert(onDisk.search(expressionForNot("Aleksey", "Vijay", "Pavel", "Mic"))));
+
+        Assert.assertEquals(convert(3, 4, 7, 9, 10), convert(onDisk.search(expressionForNot("Aleksey", "Vijay", "Pavel", "Micha", "Tr"))));
+
+        onDisk.close();
+    }
+
+    @Test
+    public void testNotEqualsQueryForNumbers() throws Exception
+    {
+        final Map<ByteBuffer, TokenTreeBuilder> data = new HashMap<ByteBuffer, TokenTreeBuilder>()
+        {{
+                put(Int32Type.instance.decompose(5),  keyBuilder(1L));
+                put(Int32Type.instance.decompose(7),  keyBuilder(2L));
+                put(Int32Type.instance.decompose(1),  keyBuilder(3L));
+                put(Int32Type.instance.decompose(3),  keyBuilder(1L, 4L));
+                put(Int32Type.instance.decompose(8),  keyBuilder(8L, 6L));
+                put(Int32Type.instance.decompose(10), keyBuilder(5L));
+                put(Int32Type.instance.decompose(6),  keyBuilder(7L));
+                put(Int32Type.instance.decompose(4),  keyBuilder(9L, 10L));
+                put(Int32Type.instance.decompose(0),  keyBuilder(11L, 12L, 1L));
+        }};
+
+        OnDiskIndexBuilder builder = new OnDiskIndexBuilder(UTF8Type.instance, Int32Type.instance, OnDiskIndexBuilder.Mode.PREFIX);
+        for (Map.Entry<ByteBuffer, TokenTreeBuilder> e : data.entrySet())
+            addAll(builder, e.getKey(), e.getValue());
+
+        File index = File.createTempFile("on-disk-sa-except-int-test", "db");
+        index.deleteOnExit();
+
+        builder.finish(index);
+
+        OnDiskIndex onDisk = new OnDiskIndex(index, Int32Type.instance, new KeyConverter());
+
+        Assert.assertEquals(convert(1, 2, 4, 5, 6, 7, 8, 9, 10, 11, 12), convert(onDisk.search(expressionForNot(0, 10, 1))));
+        Assert.assertEquals(convert(1, 2, 4, 5, 7, 9, 10, 11, 12), convert(onDisk.search(expressionForNot(0, 10, 1, 8))));
+        Assert.assertEquals(convert(1, 2, 4, 5, 7, 11, 12), convert(onDisk.search(expressionForNot(0, 10, 1, 8, 4))));
+
+        onDisk.close();
+    }
+
+    @Test
+    public void testRangeQueryWithExclusions() throws Exception
+    {
+        final long lower = 0;
+        final long upper = 100000;
+
+        OnDiskIndexBuilder builder = new OnDiskIndexBuilder(UTF8Type.instance, LongType.instance, OnDiskIndexBuilder.Mode.SPARSE);
+        for (long i = lower; i <= upper; i++)
+            builder.add(LongType.instance.decompose(i), keyAt(i), i);
+
+        File index = File.createTempFile("on-disk-sa-except-long-ranges", "db");
+        index.deleteOnExit();
+
+        builder.finish(index);
+
+        OnDiskIndex onDisk = new OnDiskIndex(index, LongType.instance, new KeyConverter());
+
+        ThreadLocalRandom random = ThreadLocalRandom.current();
+
+        // single exclusion
+
+        // let's do small range first to figure out if searchPoint works properly
+        validateExclusions(onDisk, lower, 50, Sets.newHashSet(42L));
+        // now let's do whole data set to test SPARSE searching
+        validateExclusions(onDisk, lower, upper, Sets.newHashSet(31337L));
+
+        // pair of exclusions which would generate a split
+
+        validateExclusions(onDisk, lower, random.nextInt(400, 800), Sets.newHashSet(42L, 154L));
+        validateExclusions(onDisk, lower, upper, Sets.newHashSet(31337L, 54631L));
+
+        // 3 exclusions which would generate a split and change bounds
+
+        validateExclusions(onDisk, lower, random.nextInt(400, 800), Sets.newHashSet(42L, 154L));
+        validateExclusions(onDisk, lower, upper, Sets.newHashSet(31337L, 54631L));
+
+        validateExclusions(onDisk, lower, random.nextLong(400, upper), Sets.newHashSet(42L, 55L));
+        validateExclusions(onDisk, lower, random.nextLong(400, upper), Sets.newHashSet(42L, 55L, 93L));
+        validateExclusions(onDisk, lower, random.nextLong(400, upper), Sets.newHashSet(42L, 55L, 93L, 205L));
+
+        Set<Long> exclusions = Sets.newHashSet(3L, 12L, 13L, 14L, 27L, 54L, 81L, 125L, 384L, 771L, 1054L, 2048L, 78834L);
+
+        // test that exclusions are properly bound by lower/upper of the expression
+        Assert.assertEquals(392, validateExclusions(onDisk, lower, 400, exclusions, false));
+        Assert.assertEquals(101, validateExclusions(onDisk, lower, 100, Sets.newHashSet(-10L, -5L, -1L), false));
+
+        validateExclusions(onDisk, lower, upper, exclusions);
+
+        Assert.assertEquals(100000, convert(onDisk.search(new Expression("", LongType.instance)
+                                                    .add(Operator.NEQ, LongType.instance.decompose(100L)))).size());
+
+        Assert.assertEquals(49, convert(onDisk.search(new Expression("", LongType.instance)
+                                                    .add(Operator.LT, LongType.instance.decompose(50L))
+                                                    .add(Operator.NEQ, LongType.instance.decompose(10L)))).size());
+
+        Assert.assertEquals(99998, convert(onDisk.search(new Expression("", LongType.instance)
+                                                    .add(Operator.GT, LongType.instance.decompose(1L))
+                                                    .add(Operator.NEQ, LongType.instance.decompose(20L)))).size());
+
+        onDisk.close();
+    }
+
+    private void validateExclusions(OnDiskIndex sa, long lower, long upper, Set<Long> exclusions)
+    {
+        validateExclusions(sa, lower, upper, exclusions, true);
+    }
+
+    private int validateExclusions(OnDiskIndex sa, long lower, long upper, Set<Long> exclusions, boolean checkCount)
+    {
+        int count = 0;
+        for (DecoratedKey key : convert(sa.search(rangeWithExclusions(lower, true, upper, true, exclusions))))
+        {
+            String keyId = UTF8Type.instance.getString(key.getKey()).split("key")[1];
+            Assert.assertFalse("key" + keyId + " is present.", exclusions.contains(Long.valueOf(keyId)));
+            count++;
+        }
+
+        if (checkCount)
+            Assert.assertEquals(upper - (lower == 0 ? -1 : lower) - exclusions.size(), count);
+
+        return count;
+    }
+
+    @Test
+    public void testDescriptor() throws Exception
+    {
+        final Map<ByteBuffer, Pair<DecoratedKey, Long>> data = new HashMap<ByteBuffer, Pair<DecoratedKey, Long>>()
+        {{
+                put(Int32Type.instance.decompose(5), Pair.create(keyAt(1L), 1L));
+        }};
+
+        OnDiskIndexBuilder builder1 = new OnDiskIndexBuilder(UTF8Type.instance, Int32Type.instance, OnDiskIndexBuilder.Mode.PREFIX);
+        OnDiskIndexBuilder builder2 = new OnDiskIndexBuilder(UTF8Type.instance, Int32Type.instance, OnDiskIndexBuilder.Mode.PREFIX);
+        for (Map.Entry<ByteBuffer, Pair<DecoratedKey, Long>> e : data.entrySet())
+        {
+            DecoratedKey key = e.getValue().left;
+            Long position = e.getValue().right;
+
+            builder1.add(e.getKey(), key, position);
+            builder2.add(e.getKey(), key, position);
+        }
+
+        File index1 = File.createTempFile("on-disk-sa-int", "db");
+        File index2 = File.createTempFile("on-disk-sa-int2", "db");
+        index1.deleteOnExit();
+        index2.deleteOnExit();
+
+        builder1.finish(index1);
+        builder2.finish(new Descriptor(Descriptor.VERSION_AA), index2);
+
+        OnDiskIndex onDisk1 = new OnDiskIndex(index1, Int32Type.instance, new KeyConverter());
+        OnDiskIndex onDisk2 = new OnDiskIndex(index2, Int32Type.instance, new KeyConverter());
+
+        ByteBuffer number = Int32Type.instance.decompose(5);
+
+        Assert.assertEquals(Collections.singleton(data.get(number).left), convert(onDisk1.search(expressionFor(Int32Type.instance, number))));
+        Assert.assertEquals(Collections.singleton(data.get(number).left), convert(onDisk2.search(expressionFor(Int32Type.instance, number))));
+
+        Assert.assertEquals(onDisk1.descriptor.version.version, Descriptor.CURRENT_VERSION);
+        Assert.assertEquals(onDisk2.descriptor.version.version, Descriptor.VERSION_AA);
+    }
+
+    @Test
+    public void testSuperBlocks() throws Exception
+    {
+        Map<ByteBuffer, TokenTreeBuilder> terms = new HashMap<>();
+        terms.put(UTF8Type.instance.decompose("1234"), keyBuilder(1L, 2L));
+        terms.put(UTF8Type.instance.decompose("2345"), keyBuilder(3L, 4L));
+        terms.put(UTF8Type.instance.decompose("3456"), keyBuilder(5L, 6L));
+        terms.put(UTF8Type.instance.decompose("4567"), keyBuilder(7L, 8L));
+        terms.put(UTF8Type.instance.decompose("5678"), keyBuilder(9L, 10L));
+
+        OnDiskIndexBuilder builder = new OnDiskIndexBuilder(UTF8Type.instance, Int32Type.instance, OnDiskIndexBuilder.Mode.SPARSE);
+        for (Map.Entry<ByteBuffer, TokenTreeBuilder> entry : terms.entrySet())
+            addAll(builder, entry.getKey(), entry.getValue());
+
+        File index = File.createTempFile("on-disk-sa-try-superblocks", ".db");
+        index.deleteOnExit();
+
+        builder.finish(index);
+
+        OnDiskIndex onDisk = new OnDiskIndex(index, Int32Type.instance, new KeyConverter());
+        OnDiskIndex.OnDiskSuperBlock superBlock = onDisk.dataLevel.getSuperBlock(0);
+        Iterator<Token> iter = superBlock.iterator();
+
+        Long lastToken = null;
+        while (iter.hasNext())
+        {
+            Token token = iter.next();
+
+            if (lastToken != null)
+                Assert.assertTrue(lastToken.compareTo(token.get()) < 0);
+
+            lastToken = token.get();
+        }
+    }
+
+    @Test
+    public void testSuperBlockRetrieval() throws Exception
+    {
+        OnDiskIndexBuilder builder = new OnDiskIndexBuilder(UTF8Type.instance, LongType.instance, OnDiskIndexBuilder.Mode.SPARSE);
+        for (long i = 0; i < 100000; i++)
+            builder.add(LongType.instance.decompose(i), keyAt(i), i);
+
+        File index = File.createTempFile("on-disk-sa-multi-superblock-match", ".db");
+        index.deleteOnExit();
+
+        builder.finish(index);
+
+        OnDiskIndex onDiskIndex = new OnDiskIndex(index, LongType.instance, new KeyConverter());
+
+        testSearchRangeWithSuperBlocks(onDiskIndex, 0, 500);
+        testSearchRangeWithSuperBlocks(onDiskIndex, 300, 93456);
+        testSearchRangeWithSuperBlocks(onDiskIndex, 210, 1700);
+        testSearchRangeWithSuperBlocks(onDiskIndex, 530, 3200);
+
+        Random random = new Random(0xdeadbeef);
+        for (int i = 0; i < 100000; i += random.nextInt(1500)) // random steps with max of 1500 elements
+        {
+            for (int j = 0; j < 3; j++)
+                testSearchRangeWithSuperBlocks(onDiskIndex, i, ThreadLocalRandom.current().nextInt(i, 100000));
+        }
+    }
+
+    @Test
+    public void testCombiningOfThePartitionedSA() throws Exception
+    {
+        OnDiskIndexBuilder builderA = new OnDiskIndexBuilder(UTF8Type.instance, LongType.instance, OnDiskIndexBuilder.Mode.PREFIX);
+        OnDiskIndexBuilder builderB = new OnDiskIndexBuilder(UTF8Type.instance, LongType.instance, OnDiskIndexBuilder.Mode.PREFIX);
+
+        TreeMap<Long, TreeMap<Long, LongSet>> expected = new TreeMap<>();
+
+        for (long i = 0; i <= 100; i++)
+        {
+            TreeMap<Long, LongSet> offsets = expected.get(i);
+            if (offsets == null)
+                expected.put(i, (offsets = new TreeMap<>()));
+
+            builderA.add(LongType.instance.decompose(i), keyAt(i), i);
+            offsets.putAll(keyBuilder(i).getTokens());
+        }
+
+        for (long i = 50; i < 100; i++)
+        {
+            TreeMap<Long, LongSet> offsets = expected.get(i);
+            if (offsets == null)
+                expected.put(i, (offsets = new TreeMap<>()));
+
+            long position = 100L + i;
+            builderB.add(LongType.instance.decompose(i), keyAt(position), position);
+            offsets.putAll(keyBuilder(100L + i).getTokens());
+        }
+
+        File indexA = File.createTempFile("on-disk-sa-partition-a", ".db");
+        indexA.deleteOnExit();
+
+        File indexB = File.createTempFile("on-disk-sa-partition-b", ".db");
+        indexB.deleteOnExit();
+
+        builderA.finish(indexA);
+        builderB.finish(indexB);
+
+        OnDiskIndex a = new OnDiskIndex(indexA, LongType.instance, new KeyConverter());
+        OnDiskIndex b = new OnDiskIndex(indexB, LongType.instance, new KeyConverter());
+
+        RangeIterator<OnDiskIndex.DataTerm, CombinedTerm> union = OnDiskIndexIterator.union(a, b);
+
+        TreeMap<Long, TreeMap<Long, LongSet>> actual = new TreeMap<>();
+        while (union.hasNext())
+        {
+            CombinedTerm term = union.next();
+
+            Long composedTerm = LongType.instance.compose(term.getTerm());
+
+            TreeMap<Long, LongSet> offsets = actual.get(composedTerm);
+            if (offsets == null)
+                actual.put(composedTerm, (offsets = new TreeMap<>()));
+
+            offsets.putAll(term.getTokens());
+        }
+
+        Assert.assertEquals(actual, expected);
+
+        File indexC = File.createTempFile("on-disk-sa-partition-final", ".db");
+        indexC.deleteOnExit();
+
+        OnDiskIndexBuilder combined = new OnDiskIndexBuilder(UTF8Type.instance, LongType.instance, OnDiskIndexBuilder.Mode.PREFIX);
+        combined.finish(Pair.create(keyAt(0).getKey(), keyAt(100).getKey()), indexC, new CombinedTermIterator(a, b));
+
+        OnDiskIndex c = new OnDiskIndex(indexC, LongType.instance, new KeyConverter());
+        union = OnDiskIndexIterator.union(c);
+        actual.clear();
+
+        while (union.hasNext())
+        {
+            CombinedTerm term = union.next();
+
+            Long composedTerm = LongType.instance.compose(term.getTerm());
+
+            TreeMap<Long, LongSet> offsets = actual.get(composedTerm);
+            if (offsets == null)
+                actual.put(composedTerm, (offsets = new TreeMap<>()));
+
+            offsets.putAll(term.getTokens());
+        }
+
+        Assert.assertEquals(actual, expected);
+
+        a.close();
+        b.close();
+    }
+
+    private void testSearchRangeWithSuperBlocks(OnDiskIndex onDiskIndex, long start, long end)
+    {
+        RangeIterator<Long, Token> tokens = onDiskIndex.search(expressionFor(start, true, end, false));
+
+        int keyCount = 0;
+        Long lastToken = null;
+        while (tokens.hasNext())
+        {
+            Token token = tokens.next();
+            Iterator<DecoratedKey> keys = token.iterator();
+
+            // each of the values should have exactly a single key
+            Assert.assertTrue(keys.hasNext());
+            keys.next();
+            Assert.assertFalse(keys.hasNext());
+
+            // and it's last should always smaller than current
+            if (lastToken != null)
+                Assert.assertTrue("last should be less than current", lastToken.compareTo(token.get()) < 0);
+
+            lastToken = token.get();
+            keyCount++;
+        }
+
+        Assert.assertEquals(end - start, keyCount);
+    }
+
+    private static DecoratedKey keyAt(long rawKey)
+    {
+        ByteBuffer key = ByteBuffer.wrap(("key" + rawKey).getBytes());
+        return new BufferDecoratedKey(new Murmur3Partitioner.LongToken(MurmurHash.hash2_64(key, key.position(), key.remaining(), 0)), key);
+    }
+
+    private static TokenTreeBuilder keyBuilder(Long... keys)
+    {
+        TokenTreeBuilder builder = new TokenTreeBuilder();
+
+        for (final Long key : keys)
+        {
+            DecoratedKey dk = keyAt(key);
+            builder.add((Long) dk.getToken().getTokenValue(), key);
+        }
+
+        return builder.finish();
+    }
+
+    private static Set<DecoratedKey> convert(TokenTreeBuilder offsets)
+    {
+        Set<DecoratedKey> result = new HashSet<>();
+
+        Iterator<Pair<Long, LongSet>> offsetIter = offsets.iterator();
+        while (offsetIter.hasNext())
+        {
+            LongSet v = offsetIter.next().right;
+
+            for (LongCursor offset : v)
+                result.add(keyAt(offset.value));
+        }
+        return result;
+    }
+
+    private static Set<DecoratedKey> convert(long... keyOffsets)
+    {
+        Set<DecoratedKey> result = new HashSet<>();
+        for (long offset : keyOffsets)
+            result.add(keyAt(offset));
+
+        return result;
+    }
+
+    private static Set<DecoratedKey> convert(RangeIterator<Long, Token> results)
+    {
+        if (results == null)
+            return Collections.emptySet();
+
+        Set<DecoratedKey> keys = new TreeSet<>(DecoratedKey.comparator);
+
+        while (results.hasNext())
+        {
+            for (DecoratedKey key : results.next())
+                keys.add(key);
+        }
+
+        return keys;
+    }
+
+    private static Expression expressionFor(long lower, boolean lowerInclusive, long upper, boolean upperInclusive)
+    {
+        Expression expression = new Expression("", LongType.instance);
+        expression.add(lowerInclusive ? Operator.GTE : Operator.GT, LongType.instance.decompose(lower));
+        expression.add(upperInclusive ? Operator.LTE : Operator.LT, LongType.instance.decompose(upper));
+        return expression;
+    }
+
+    private static Expression expressionFor(AbstractType<?> validator, ByteBuffer term)
+    {
+        Expression expression = new Expression("", validator);
+        expression.add(Operator.EQ, term);
+        return expression;
+    }
+
+    private static Expression expressionForNot(AbstractType<?> validator, ByteBuffer lower, ByteBuffer upper, Iterable<ByteBuffer> terms)
+    {
+        Expression expression = new Expression("", validator);
+        expression.setOp(Expression.Op.RANGE);
+        expression.setLower(new Expression.Bound(lower, true));
+        expression.setUpper(new Expression.Bound(upper, true));
+        for (ByteBuffer term : terms)
+            expression.add(Operator.NEQ, term);
+        return expression;
+
+    }
+
+    private static Expression expressionForNot(Integer lower, Integer upper, Integer... terms)
+    {
+        return expressionForNot(Int32Type.instance,
+                Int32Type.instance.decompose(lower),
+                Int32Type.instance.decompose(upper),
+                Arrays.asList(terms).stream().map(Int32Type.instance::decompose).collect(Collectors.toList()));
+    }
+
+    private static Expression rangeWithExclusions(long lower, boolean lowerInclusive, long upper, boolean upperInclusive, Set<Long> exclusions)
+    {
+        Expression expression = expressionFor(lower, lowerInclusive, upper, upperInclusive);
+        for (long e : exclusions)
+            expression.add(Operator.NEQ, LongType.instance.decompose(e));
+
+        return expression;
+    }
+
+    private static Expression expressionForNot(String lower, String upper, String... terms)
+    {
+        return expressionForNot(UTF8Type.instance,
+                UTF8Type.instance.decompose(lower),
+                UTF8Type.instance.decompose(upper),
+                Arrays.asList(terms).stream().map(UTF8Type.instance::decompose).collect(Collectors.toList()));
+    }
+
+    private static Expression expressionFor(String term)
+    {
+        return expressionFor(UTF8Type.instance, UTF8Type.instance.decompose(term));
+    }
+
+    private static void addAll(OnDiskIndexBuilder builder, ByteBuffer term, TokenTreeBuilder tokens)
+    {
+        for (Map.Entry<Long, LongSet> token : tokens.getTokens().entrySet())
+        {
+            for (long position : token.getValue().toArray())
+                builder.add(term, keyAt(position), position);
+        }
+    }
+
+    private static class KeyConverter implements Function<Long, DecoratedKey>
+    {
+        @Override
+        public DecoratedKey apply(Long offset)
+        {
+            return keyAt(offset);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java b/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
new file mode 100644
index 0000000..4663692
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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.index.sasi.disk;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.rows.BTreeRow;
+import org.apache.cassandra.db.rows.BufferCell;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.index.sasi.SASIIndex;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Tables;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import com.google.common.util.concurrent.Futures;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class PerSSTableIndexWriterTest extends SchemaLoader
+{
+    private static final String KS_NAME = "sasi";
+    private static final String CF_NAME = "test_cf";
+
+    @BeforeClass
+    public static void loadSchema() throws ConfigurationException
+    {
+        System.setProperty("cassandra.config", "cassandra-murmur.yaml");
+        SchemaLoader.loadSchema();
+        MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(KS_NAME,
+                                                                     KeyspaceParams.simpleTransient(1),
+                                                                     Tables.of(SchemaLoader.sasiCFMD(KS_NAME, CF_NAME))));
+    }
+
+    @Test
+    public void testPartialIndexWrites() throws Exception
+    {
+        final int maxKeys = 100000, numParts = 4, partSize = maxKeys / numParts;
+        final String keyFormat = "key%06d";
+        final long timestamp = System.currentTimeMillis();
+
+        ColumnFamilyStore cfs = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
+        ColumnDefinition column = cfs.metadata.getColumnDefinition(UTF8Type.instance.decompose("age"));
+
+        SASIIndex sasi = (SASIIndex) cfs.indexManager.getIndexByName("age");
+
+        File directory = cfs.getDirectories().getDirectoryForNewSSTables();
+        Descriptor descriptor = Descriptor.fromFilename(cfs.getSSTablePath(directory));
+        PerSSTableIndexWriter indexWriter = (PerSSTableIndexWriter) sasi.getFlushObserver(descriptor, OperationType.FLUSH);
+
+        SortedMap<DecoratedKey, Row> expectedKeys = new TreeMap<>(DecoratedKey.comparator);
+
+        for (int i = 0; i < maxKeys; i++)
+        {
+            ByteBuffer key = ByteBufferUtil.bytes(String.format(keyFormat, i));
+            expectedKeys.put(cfs.metadata.partitioner.decorateKey(key),
+                             BTreeRow.singleCellRow(Clustering.EMPTY,
+                                                    BufferCell.live(cfs.metadata, column, timestamp, Int32Type.instance.decompose(i))));
+        }
+
+        indexWriter.begin();
+
+        Iterator<Map.Entry<DecoratedKey, Row>> keyIterator = expectedKeys.entrySet().iterator();
+        long position = 0;
+
+        Set<String> segments = new HashSet<>();
+        outer:
+        for (;;)
+        {
+            for (int i = 0; i < partSize; i++)
+            {
+                if (!keyIterator.hasNext())
+                    break outer;
+
+                Map.Entry<DecoratedKey, Row> key = keyIterator.next();
+
+                indexWriter.startPartition(key.getKey(), position++);
+                indexWriter.nextUnfilteredCluster(key.getValue());
+            }
+
+            PerSSTableIndexWriter.Index index = indexWriter.getIndex(column);
+
+            OnDiskIndex segment = index.scheduleSegmentFlush(false).call();
+            index.segments.add(Futures.immediateFuture(segment));
+            segments.add(segment.getIndexPath());
+        }
+
+        for (String segment : segments)
+            Assert.assertTrue(new File(segment).exists());
+
+        String indexFile = indexWriter.indexes.get(column).filename(true);
+
+        // final flush
+        indexWriter.complete();
+
+        for (String segment : segments)
+            Assert.assertFalse(new File(segment).exists());
+
+        OnDiskIndex index = new OnDiskIndex(new File(indexFile), Int32Type.instance, keyPosition -> {
+            ByteBuffer key = ByteBufferUtil.bytes(String.format(keyFormat, keyPosition));
+            return cfs.metadata.partitioner.decorateKey(key);
+        });
+
+        Assert.assertEquals(0, UTF8Type.instance.compare(index.minKey(), ByteBufferUtil.bytes(String.format(keyFormat, 0))));
+        Assert.assertEquals(0, UTF8Type.instance.compare(index.maxKey(), ByteBufferUtil.bytes(String.format(keyFormat, maxKeys - 1))));
+
+        Set<DecoratedKey> actualKeys = new HashSet<>();
+        int count = 0;
+        for (OnDiskIndex.DataTerm term : index)
+        {
+            RangeIterator<Long, Token> tokens = term.getTokens();
+
+            while (tokens.hasNext())
+            {
+                for (DecoratedKey key : tokens.next())
+                    actualKeys.add(key);
+            }
+
+            Assert.assertEquals(count++, (int) Int32Type.instance.compose(term.getTerm()));
+        }
+
+        Assert.assertEquals(expectedKeys.size(), actualKeys.size());
+        for (DecoratedKey key : expectedKeys.keySet())
+            Assert.assertTrue(actualKeys.contains(key));
+
+        FileUtils.closeQuietly(index);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/unit/org/apache/cassandra/index/sasi/disk/TokenTreeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/disk/TokenTreeTest.java b/test/unit/org/apache/cassandra/index/sasi/disk/TokenTreeTest.java
new file mode 100644
index 0000000..d09b8d4
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/disk/TokenTreeTest.java
@@ -0,0 +1,535 @@
+/*
+ * 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.index.sasi.disk;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+import org.apache.cassandra.db.BufferDecoratedKey;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.index.sasi.disk.TokenTreeBuilder.EntryType;
+import org.apache.cassandra.index.sasi.utils.CombinedValue;
+import org.apache.cassandra.index.sasi.utils.MappedBuffer;
+import org.apache.cassandra.index.sasi.utils.RangeIterator;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.MurmurHash;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+import junit.framework.Assert;
+import org.junit.Test;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import com.carrotsearch.hppc.LongOpenHashSet;
+import com.carrotsearch.hppc.LongSet;
+import com.carrotsearch.hppc.cursors.LongCursor;
+import com.google.common.base.Function;
+
+public class TokenTreeTest
+{
+    private static final Function<Long, DecoratedKey> KEY_CONVERTER = new KeyConverter();
+
+    static LongSet singleOffset = new LongOpenHashSet() {{ add(1); }};
+    static LongSet bigSingleOffset = new LongOpenHashSet() {{ add(((long) Integer.MAX_VALUE) + 10); }};
+    static LongSet shortPackableCollision = new LongOpenHashSet() {{ add(2L); add(3L); }}; // can pack two shorts
+    static LongSet intPackableCollision = new LongOpenHashSet() {{ add(6L); add(((long) Short.MAX_VALUE) + 1); }}; // can pack int & short
+    static LongSet multiCollision =  new LongOpenHashSet() {{ add(3L); add(4L); add(5L); }}; // can't pack
+    static LongSet unpackableCollision = new LongOpenHashSet() {{ add(((long) Short.MAX_VALUE) + 1); add(((long) Short.MAX_VALUE) + 2); }}; // can't pack
+
+    final static SortedMap<Long, LongSet> simpleTokenMap = new TreeMap<Long, LongSet>()
+    {{
+            put(1L, bigSingleOffset); put(3L, shortPackableCollision); put(4L, intPackableCollision); put(6L, singleOffset);
+            put(9L, multiCollision); put(10L, unpackableCollision); put(12L, singleOffset); put(13L, singleOffset);
+            put(15L, singleOffset); put(16L, singleOffset); put(20L, singleOffset); put(22L, singleOffset);
+            put(25L, singleOffset); put(26L, singleOffset); put(27L, singleOffset); put(28L, singleOffset);
+            put(40L, singleOffset); put(50L, singleOffset); put(100L, singleOffset); put(101L, singleOffset);
+            put(102L, singleOffset); put(103L, singleOffset); put(108L, singleOffset); put(110L, singleOffset);
+            put(112L, singleOffset); put(115L, singleOffset); put(116L, singleOffset); put(120L, singleOffset);
+            put(121L, singleOffset); put(122L, singleOffset); put(123L, singleOffset); put(125L, singleOffset);
+    }};
+
+    final static SortedMap<Long, LongSet> bigTokensMap = new TreeMap<Long, LongSet>()
+    {{
+            for (long i = 0; i < 1000000; i++)
+                put(i, singleOffset);
+    }};
+
+    final static SortedMap<Long, LongSet> collidingTokensMap = new TreeMap<Long, LongSet>()
+    {{
+            put(1L, singleOffset); put(7L, singleOffset); put(8L, singleOffset);
+    }};
+
+    final static SortedMap<Long, LongSet> tokens = bigTokensMap;
+
+    @Test
+    public void buildAndIterate() throws Exception
+    {
+        final TokenTreeBuilder builder = new TokenTreeBuilder(tokens).finish();
+        final Iterator<Pair<Long, LongSet>> tokenIterator = builder.iterator();
+        final Iterator<Map.Entry<Long, LongSet>> listIterator = tokens.entrySet().iterator();
+        while (tokenIterator.hasNext() && listIterator.hasNext())
+        {
+            Pair<Long, LongSet> tokenNext = tokenIterator.next();
+            Map.Entry<Long, LongSet> listNext = listIterator.next();
+
+            Assert.assertEquals(listNext.getKey(), tokenNext.left);
+            Assert.assertEquals(listNext.getValue(), tokenNext.right);
+        }
+
+        Assert.assertFalse("token iterator not finished", tokenIterator.hasNext());
+        Assert.assertFalse("list iterator not finished", listIterator.hasNext());
+    }
+
+    @Test
+    public void buildWithMultipleMapsAndIterate() throws Exception
+    {
+        final SortedMap<Long, LongSet> merged = new TreeMap<>();
+        final TokenTreeBuilder builder = new TokenTreeBuilder(simpleTokenMap).finish();
+        builder.add(collidingTokensMap);
+
+        merged.putAll(collidingTokensMap);
+        for (Map.Entry<Long, LongSet> entry : simpleTokenMap.entrySet())
+        {
+            if (merged.containsKey(entry.getKey()))
+            {
+                LongSet mergingOffsets  = entry.getValue();
+                LongSet existingOffsets = merged.get(entry.getKey());
+
+                if (mergingOffsets.equals(existingOffsets))
+                    continue;
+
+                Set<Long> mergeSet = new HashSet<>();
+                for (LongCursor merging : mergingOffsets)
+                    mergeSet.add(merging.value);
+
+                for (LongCursor existing : existingOffsets)
+                    mergeSet.add(existing.value);
+
+                LongSet mergedResults = new LongOpenHashSet();
+                for (Long result : mergeSet)
+                    mergedResults.add(result);
+
+                merged.put(entry.getKey(), mergedResults);
+            }
+            else
+            {
+                merged.put(entry.getKey(), entry.getValue());
+            }
+        }
+
+        final Iterator<Pair<Long, LongSet>> tokenIterator = builder.iterator();
+        final Iterator<Map.Entry<Long, LongSet>> listIterator = merged.entrySet().iterator();
+        while (tokenIterator.hasNext() && listIterator.hasNext())
+        {
+            Pair<Long, LongSet> tokenNext = tokenIterator.next();
+            Map.Entry<Long, LongSet> listNext = listIterator.next();
+
+            Assert.assertEquals(listNext.getKey(), tokenNext.left);
+            Assert.assertEquals(listNext.getValue(), tokenNext.right);
+        }
+
+        Assert.assertFalse("token iterator not finished", tokenIterator.hasNext());
+        Assert.assertFalse("list iterator not finished", listIterator.hasNext());
+
+    }
+
+    @Test
+    public void testSerializedSize() throws Exception
+    {
+        final TokenTreeBuilder builder = new TokenTreeBuilder(tokens).finish();
+
+        final File treeFile = File.createTempFile("token-tree-size-test", "tt");
+        treeFile.deleteOnExit();
+
+        try (SequentialWriter writer = new SequentialWriter(treeFile, 4096, BufferType.ON_HEAP))
+        {
+            builder.write(writer);
+            writer.sync();
+        }
+
+        final RandomAccessReader reader = RandomAccessReader.open(treeFile);
+        Assert.assertEquals((int) reader.bytesRemaining(), builder.serializedSize());
+    }
+
+    @Test
+    public void buildSerializeAndIterate() throws Exception
+    {
+        final TokenTreeBuilder builder = new TokenTreeBuilder(simpleTokenMap).finish();
+
+        final File treeFile = File.createTempFile("token-tree-iterate-test1", "tt");
+        treeFile.deleteOnExit();
+
+        try (SequentialWriter writer = new SequentialWriter(treeFile, 4096, BufferType.ON_HEAP))
+        {
+            builder.write(writer);
+            writer.sync();
+        }
+
+        final RandomAccessReader reader = RandomAccessReader.open(treeFile);
+        final TokenTree tokenTree = new TokenTree(new MappedBuffer(reader));
+
+        final Iterator<Token> tokenIterator = tokenTree.iterator(KEY_CONVERTER);
+        final Iterator<Map.Entry<Long, LongSet>> listIterator = simpleTokenMap.entrySet().iterator();
+        while (tokenIterator.hasNext() && listIterator.hasNext())
+        {
+            Token treeNext = tokenIterator.next();
+            Map.Entry<Long, LongSet> listNext = listIterator.next();
+
+            Assert.assertEquals(listNext.getKey(), treeNext.get());
+            Assert.assertEquals(convert(listNext.getValue()), convert(treeNext));
+        }
+
+        Assert.assertFalse("token iterator not finished", tokenIterator.hasNext());
+        Assert.assertFalse("list iterator not finished", listIterator.hasNext());
+
+        reader.close();
+    }
+
+    @Test
+    public void buildSerializeAndGet() throws Exception
+    {
+        final long tokMin = 0;
+        final long tokMax = 1000;
+
+        final TokenTree tokenTree = generateTree(tokMin, tokMax);
+
+        for (long i = 0; i <= tokMax; i++)
+        {
+            TokenTree.OnDiskToken result = tokenTree.get(i, KEY_CONVERTER);
+            Assert.assertNotNull("failed to find object for token " + i, result);
+
+            Set<Long> found = result.getOffsets();
+            Assert.assertEquals(1, found.size());
+            Assert.assertEquals(i, found.toArray()[0]);
+        }
+
+        Assert.assertNull("found missing object", tokenTree.get(tokMax + 10, KEY_CONVERTER));
+    }
+
+    @Test
+    public void buildSerializeIterateAndSkip() throws Exception
+    {
+        final TokenTreeBuilder builder = new TokenTreeBuilder(tokens).finish();
+
+        final File treeFile = File.createTempFile("token-tree-iterate-test2", "tt");
+        treeFile.deleteOnExit();
+
+        try (SequentialWriter writer = new SequentialWriter(treeFile, 4096, BufferType.ON_HEAP))
+        {
+            builder.write(writer);
+            writer.sync();
+        }
+
+        final RandomAccessReader reader = RandomAccessReader.open(treeFile);
+        final TokenTree tokenTree = new TokenTree(new MappedBuffer(reader));
+
+        final RangeIterator<Long, Token> treeIterator = tokenTree.iterator(KEY_CONVERTER);
+        final RangeIterator<Long, TokenWithOffsets> listIterator = new EntrySetSkippableIterator(tokens);
+
+        long lastToken = 0L;
+        while (treeIterator.hasNext() && lastToken < 12)
+        {
+            Token treeNext = treeIterator.next();
+            TokenWithOffsets listNext = listIterator.next();
+
+            Assert.assertEquals(listNext.token, (lastToken = treeNext.get()));
+            Assert.assertEquals(convert(listNext.offsets), convert(treeNext));
+        }
+
+        treeIterator.skipTo(100548L);
+        listIterator.skipTo(100548L);
+
+        while (treeIterator.hasNext() && listIterator.hasNext())
+        {
+            Token treeNext = treeIterator.next();
+            TokenWithOffsets listNext = listIterator.next();
+
+            Assert.assertEquals(listNext.token, (long) treeNext.get());
+            Assert.assertEquals(convert(listNext.offsets), convert(treeNext));
+
+        }
+
+        Assert.assertFalse("Tree iterator not completed", treeIterator.hasNext());
+        Assert.assertFalse("List iterator not completed", listIterator.hasNext());
+
+        reader.close();
+    }
+
+    @Test
+    public void skipPastEnd() throws Exception
+    {
+        final TokenTreeBuilder builder = new TokenTreeBuilder(simpleTokenMap).finish();
+
+        final File treeFile = File.createTempFile("token-tree-skip-past-test", "tt");
+        treeFile.deleteOnExit();
+
+        try (SequentialWriter writer = new SequentialWriter(treeFile, 4096, BufferType.ON_HEAP))
+        {
+            builder.write(writer);
+            writer.sync();
+        }
+
+        final RandomAccessReader reader = RandomAccessReader.open(treeFile);
+        final RangeIterator<Long, Token> tokenTree = new TokenTree(new MappedBuffer(reader)).iterator(KEY_CONVERTER);
+
+        tokenTree.skipTo(simpleTokenMap.lastKey() + 10);
+    }
+
+    @Test
+    public void testTokenMerge() throws Exception
+    {
+        final long min = 0, max = 1000;
+
+        // two different trees with the same offsets
+        TokenTree treeA = generateTree(min, max);
+        TokenTree treeB = generateTree(min, max);
+
+        RangeIterator<Long, Token> a = treeA.iterator(new KeyConverter());
+        RangeIterator<Long, Token> b = treeB.iterator(new KeyConverter());
+
+        long count = min;
+        while (a.hasNext() && b.hasNext())
+        {
+            final Token tokenA = a.next();
+            final Token tokenB = b.next();
+
+            // merging of two OnDiskToken
+            tokenA.merge(tokenB);
+            // merging with RAM Token with different offset
+            tokenA.merge(new TokenWithOffsets(tokenA.get(), convert(count + 1)));
+            // and RAM token with the same offset
+            tokenA.merge(new TokenWithOffsets(tokenA.get(), convert(count)));
+
+            // should fail when trying to merge different tokens
+            try
+            {
+                tokenA.merge(new TokenWithOffsets(tokenA.get() + 1, convert(count)));
+                Assert.fail();
+            }
+            catch (IllegalArgumentException e)
+            {
+                // expected
+            }
+
+            final Set<Long> offsets = new TreeSet<>();
+            for (DecoratedKey key : tokenA)
+                 offsets.add(LongType.instance.compose(key.getKey()));
+
+            Set<Long> expected = new TreeSet<>();
+            {
+                expected.add(count);
+                expected.add(count + 1);
+            }
+
+            Assert.assertEquals(expected, offsets);
+            count++;
+        }
+
+        Assert.assertEquals(max, count - 1);
+    }
+
+    @Test
+    public void testEntryTypeOrdinalLookup()
+    {
+        Assert.assertEquals(EntryType.SIMPLE, EntryType.of(EntryType.SIMPLE.ordinal()));
+        Assert.assertEquals(EntryType.PACKED, EntryType.of(EntryType.PACKED.ordinal()));
+        Assert.assertEquals(EntryType.FACTORED, EntryType.of(EntryType.FACTORED.ordinal()));
+        Assert.assertEquals(EntryType.OVERFLOW, EntryType.of(EntryType.OVERFLOW.ordinal()));
+    }
+
+    private static class EntrySetSkippableIterator extends RangeIterator<Long, TokenWithOffsets>
+    {
+        private final PeekingIterator<Map.Entry<Long, LongSet>> elements;
+
+        EntrySetSkippableIterator(SortedMap<Long, LongSet> elms)
+        {
+            super(elms.firstKey(), elms.lastKey(), elms.size());
+            elements = Iterators.peekingIterator(elms.entrySet().iterator());
+        }
+
+        @Override
+        public TokenWithOffsets computeNext()
+        {
+            if (!elements.hasNext())
+                return endOfData();
+
+            Map.Entry<Long, LongSet> next = elements.next();
+            return new TokenWithOffsets(next.getKey(), next.getValue());
+        }
+
+        @Override
+        protected void performSkipTo(Long nextToken)
+        {
+            while (elements.hasNext())
+            {
+                if (Long.compare(elements.peek().getKey(), nextToken) >= 0)
+                {
+                    break;
+                }
+
+                elements.next();
+            }
+        }
+
+        @Override
+        public void close() throws IOException
+        {
+            // nothing to do here
+        }
+    }
+
+    public static class TokenWithOffsets extends Token
+    {
+        private final LongSet offsets;
+
+        public TokenWithOffsets(long token, final LongSet offsets)
+        {
+            super(token);
+            this.offsets = offsets;
+        }
+
+        @Override
+        public void merge(CombinedValue<Long> other)
+        {}
+
+        @Override
+        public int compareTo(CombinedValue<Long> o)
+        {
+            return Long.compare(token, o.get());
+        }
+
+        @Override
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof TokenWithOffsets))
+                return false;
+
+            TokenWithOffsets o = (TokenWithOffsets) other;
+            return token == o.token && offsets.equals(o.offsets);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return new HashCodeBuilder().append(token).build();
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("TokenValue(token: %d, offsets: %s)", token, offsets);
+        }
+
+        @Override
+        public Iterator<DecoratedKey> iterator()
+        {
+            List<DecoratedKey> keys = new ArrayList<>(offsets.size());
+            for (LongCursor offset : offsets)
+                 keys.add(dk(offset.value));
+
+            return keys.iterator();
+        }
+    }
+
+    private static Set<DecoratedKey> convert(LongSet offsets)
+    {
+        Set<DecoratedKey> keys = new HashSet<>();
+        for (LongCursor offset : offsets)
+            keys.add(KEY_CONVERTER.apply(offset.value));
+
+        return keys;
+    }
+
+    private static Set<DecoratedKey> convert(Token results)
+    {
+        Set<DecoratedKey> keys = new HashSet<>();
+        for (DecoratedKey key : results)
+            keys.add(key);
+
+        return keys;
+    }
+
+    private static LongSet convert(long... values)
+    {
+        LongSet result = new LongOpenHashSet(values.length);
+        for (long v : values)
+            result.add(v);
+
+        return result;
+    }
+
+    private static class KeyConverter implements Function<Long, DecoratedKey>
+    {
+        @Override
+        public DecoratedKey apply(Long offset)
+        {
+            return dk(offset);
+        }
+    }
+
+    private static DecoratedKey dk(Long token)
+    {
+        ByteBuffer buf = ByteBuffer.allocate(8);
+        buf.putLong(token);
+        buf.flip();
+        Long hashed = MurmurHash.hash2_64(buf, buf.position(), buf.remaining(), 0);
+        return new BufferDecoratedKey(new Murmur3Partitioner.LongToken(hashed), buf);
+    }
+
+    private static TokenTree generateTree(final long minToken, final long maxToken) throws IOException
+    {
+        final SortedMap<Long, LongSet> toks = new TreeMap<Long, LongSet>()
+        {{
+                for (long i = minToken; i <= maxToken; i++)
+                {
+                    LongSet offsetSet = new LongOpenHashSet();
+                    offsetSet.add(i);
+                    put(i, offsetSet);
+                }
+        }};
+
+        final TokenTreeBuilder builder = new TokenTreeBuilder(toks).finish();
+        final File treeFile = File.createTempFile("token-tree-get-test", "tt");
+        treeFile.deleteOnExit();
+
+        try (SequentialWriter writer = new SequentialWriter(treeFile, 4096, BufferType.ON_HEAP))
+        {
+            builder.write(writer);
+            writer.sync();
+        }
+
+        RandomAccessReader reader = null;
+
+        try
+        {
+            reader = RandomAccessReader.open(treeFile);
+            return new TokenTree(new MappedBuffer(reader));
+        }
+        finally
+        {
+            FileUtils.closeQuietly(reader);
+        }
+    }
+}


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

Posted by xe...@apache.org.
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


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

Posted by xe...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/fi_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/fi_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/fi_ST.txt
new file mode 100644
index 0000000..3c8bfd5
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/fi_ST.txt
@@ -0,0 +1,748 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+aiemmin
+aika
+aikaa
+aikaan
+aikaisemmin
+aikaisin
+aikajen
+aikana
+aikoina
+aikoo
+aikovat
+aina
+ainakaan
+ainakin
+ainoa
+ainoat
+aiomme
+aion
+aiotte
+aist
+aivan
+ajan
+�l�
+alas
+alemmas
+�lk��n
+alkuisin
+alkuun
+alla
+alle
+aloitamme
+aloitan
+aloitat
+aloitatte
+aloitattivat
+aloitettava
+aloitettevaksi
+aloitettu
+aloitimme
+aloitin
+aloitit
+aloititte
+aloittaa
+aloittamatta
+aloitti
+aloittivat
+alta
+aluksi
+alussa
+alusta
+annettavaksi
+annetteva
+annettu
+antaa
+antamatta
+antoi
+aoua
+apu
+asia
+asiaa
+asian
+asiasta
+asiat
+asioiden
+asioihin
+asioita
+asti
+avuksi
+avulla
+avun
+avutta
+edell�
+edelle
+edelleen
+edelt�
+edemm�s
+edes
+edess�
+edest�
+ehk�
+ei
+eik�
+eilen
+eiv�t
+eli
+ellei
+elleiv�t
+ellemme
+ellen
+ellet
+ellette
+emme
+en
+en��
+enemm�n
+eniten
+ennen
+ensi
+ensimm�inen
+ensimm�iseksi
+ensimm�isen
+ensimm�isen�
+ensimm�iset
+ensimm�isi�
+ensimm�isiksi
+ensimm�isin�
+ensimm�ist�
+ensin
+entinen
+entisen
+entisi�
+entist�
+entisten
+er��t
+er�iden
+er�s
+eri
+eritt�in
+erityisesti
+esi
+esiin
+esill�
+esimerkiksi
+et
+eteen
+etenkin
+ett�
+ette
+ettei
+halua
+haluaa
+haluamatta
+haluamme
+haluan
+haluat
+haluatte
+haluavat
+halunnut
+halusi
+halusimme
+halusin
+halusit
+halusitte
+halusivat
+halutessa
+haluton
+h�n
+h�neen
+h�nell�
+h�nelle
+h�nelt�
+h�nen
+h�ness�
+h�nest�
+h�net
+he
+hei
+heid�n
+heihin
+heille
+heilt�
+heiss�
+heist�
+heit�
+helposti
+heti
+hetkell�
+hieman
+huolimatta
+huomenna
+hyv�
+hyv��
+hyv�t
+hyvi�
+hyvien
+hyviin
+hyviksi
+hyville
+hyvilt�
+hyvin
+hyvin�
+hyviss�
+hyvist�
+ihan
+ilman
+ilmeisesti
+itse
+itse��n
+itsens�
+ja
+j��
+j�lkeen
+j�lleen
+jo
+johon
+joiden
+joihin
+joiksi
+joilla
+joille
+joilta
+joissa
+joista
+joita
+joka
+jokainen
+jokin
+joko
+joku
+jolla
+jolle
+jolloin
+jolta
+jompikumpi
+jonka
+jonkin
+jonne
+joo
+jopa
+jos
+joskus
+jossa
+josta
+jota
+jotain
+joten
+jotenkin
+jotenkuten
+jotka
+jotta
+jouduimme
+jouduin
+jouduit
+jouduitte
+joudumme
+joudun
+joudutte
+joukkoon
+joukossa
+joukosta
+joutua
+joutui
+joutuivat
+joutumaan
+joutuu
+joutuvat
+juuri
+kahdeksan
+kahdeksannen
+kahdella
+kahdelle
+kahdelta
+kahden
+kahdessa
+kahdesta
+kahta
+kahteen
+kai
+kaiken
+kaikille
+kaikilta
+kaikkea
+kaikki
+kaikkia
+kaikkiaan
+kaikkialla
+kaikkialle
+kaikkialta
+kaikkien
+kaikkin
+kaksi
+kannalta
+kannattaa
+kanssa
+kanssaan
+kanssamme
+kanssani
+kanssanne
+kanssasi
+kauan
+kauemmas
+kautta
+kehen
+keiden
+keihin
+keiksi
+keill�
+keille
+keilt�
+kein�
+keiss�
+keist�
+keit�
+keitt�
+keitten
+keneen
+keneksi
+kenell�
+kenelle
+kenelt�
+kenen
+kenen�
+keness�
+kenest�
+kenet
+kenett�
+kenness�st�
+kerran
+kerta
+kertaa
+kesken
+keskim��rin
+ket�
+ketk�
+kiitos
+kohti
+koko
+kokonaan
+kolmas
+kolme
+kolmen
+kolmesti
+koska
+koskaan
+kovin
+kuin
+kuinka
+kuitenkaan
+kuitenkin
+kuka
+kukaan
+kukin
+kumpainen
+kumpainenkaan
+kumpi
+kumpikaan
+kumpikin
+kun
+kuten
+kuuden
+kuusi
+kuutta
+kyll�
+kymmenen
+kyse
+l�hekk�in
+l�hell�
+l�helle
+l�helt�
+l�hemm�s
+l�hes
+l�hinn�
+l�htien
+l�pi
+liian
+liki
+lis��
+lis�ksi
+luo
+mahdollisimman
+mahdollista
+me
+meid�n
+meill�
+meille
+melkein
+melko
+menee
+meneet
+menemme
+menen
+menet
+menette
+menev�t
+meni
+menimme
+menin
+menit
+meniv�t
+menness�
+mennyt
+menossa
+mihin
+mik�
+mik��n
+mik�li
+mikin
+miksi
+milloin
+min�
+minne
+minun
+minut
+miss�
+mist�
+mit�
+mit��n
+miten
+moi
+molemmat
+mones
+monesti
+monet
+moni
+moniaalla
+moniaalle
+moniaalta
+monta
+muassa
+muiden
+muita
+muka
+mukaan
+mukaansa
+mukana
+mutta
+muu
+muualla
+muualle
+muualta
+muuanne
+muulloin
+muun
+muut
+muuta
+muutama
+muutaman
+muuten
+my�hemmin
+my�s
+my�sk��n
+my�skin
+my�t�
+n�iden
+n�in
+n�iss�
+n�iss�hin
+n�iss�lle
+n�iss�lt�
+n�iss�st�
+n�it�
+n�m�
+ne
+nelj�
+nelj��
+nelj�n
+niiden
+niin
+niist�
+niit�
+noin
+nopeammin
+nopeasti
+nopeiten
+nro
+nuo
+nyt
+ohi
+oikein
+ole
+olemme
+olen
+olet
+olette
+oleva
+olevan
+olevat
+oli
+olimme
+olin
+olisi
+olisimme
+olisin
+olisit
+olisitte
+olisivat
+olit
+olitte
+olivat
+olla
+olleet
+olli
+ollut
+oma
+omaa
+omaan
+omaksi
+omalle
+omalta
+oman
+omassa
+omat
+omia
+omien
+omiin
+omiksi
+omille
+omilta
+omissa
+omista
+on
+onkin
+onko
+ovat
+p��lle
+paikoittain
+paitsi
+pakosti
+paljon
+paremmin
+parempi
+parhaillaan
+parhaiten
+per�ti
+perusteella
+pian
+pieneen
+pieneksi
+pienell�
+pienelle
+pienelt�
+pienempi
+pienest�
+pieni
+pienin
+puolesta
+puolestaan
+runsaasti
+saakka
+sadam
+sama
+samaa
+samaan
+samalla
+samallalta
+samallassa
+samallasta
+saman
+samat
+samoin
+sata
+sataa
+satojen
+se
+seitsem�n
+sek�
+sen
+seuraavat
+siell�
+sielt�
+siihen
+siin�
+siis
+siit�
+sijaan
+siksi
+sill�
+silloin
+silti
+sin�
+sinne
+sinua
+sinulle
+sinulta
+sinun
+sinussa
+sinusta
+sinut
+sis�kk�in
+sis�ll�
+sit�
+siten
+sitten
+suoraan
+suuntaan
+suuren
+suuret
+suuri
+suuria
+suurin
+suurten
+taa
+t��ll�
+t��lt�
+taas
+taemmas
+t�h�n
+tahansa
+tai
+takaa
+takaisin
+takana
+takia
+t�ll�
+t�ll�in
+t�m�
+t�m�n
+t�n�
+t�n��n
+t�nne
+tapauksessa
+t�ss�
+t�st�
+t�t�
+t�ten
+tavalla
+tavoitteena
+t�ysin
+t�ytyv�t
+t�ytyy
+te
+tietysti
+todella
+toinen
+toisaalla
+toisaalle
+toisaalta
+toiseen
+toiseksi
+toisella
+toiselle
+toiselta
+toisemme
+toisen
+toisensa
+toisessa
+toisesta
+toista
+toistaiseksi
+toki
+tosin
+tuhannen
+tuhat
+tule
+tulee
+tulemme
+tulen
+tulet
+tulette
+tulevat
+tulimme
+tulin
+tulisi
+tulisimme
+tulisin
+tulisit
+tulisitte
+tulisivat
+tulit
+tulitte
+tulivat
+tulla
+tulleet
+tullut
+tuntuu
+tuo
+tuolla
+tuolloin
+tuolta
+tuonne
+tuskin
+tyk�
+usea
+useasti
+useimmiten
+usein
+useita
+uudeksi
+uudelleen
+uuden
+uudet
+uusi
+uusia
+uusien
+uusinta
+uuteen
+uutta
+vaan
+v�h�n
+v�hemm�n
+v�hint��n
+v�hiten
+vai
+vaiheessa
+vaikea
+vaikean
+vaikeat
+vaikeilla
+vaikeille
+vaikeilta
+vaikeissa
+vaikeista
+vaikka
+vain
+v�lill�
+varmasti
+varsin
+varsinkin
+varten
+vasta
+vastaan
+vastakkain
+verran
+viel�
+vierekk�in
+vieri
+viiden
+viime
+viimeinen
+viimeisen
+viimeksi
+viisi
+voi
+voidaan
+voimme
+voin
+voisi
+voit
+voitte
+voivat
+vuoden
+vuoksi
+vuosi
+vuosien
+vuosina
+vuotta
+yh�
+yhdeks�n
+yhden
+yhdess�
+yht�
+yht��ll�
+yht��lle
+yht��lt�
+yht��n
+yhteen
+yhteens�
+yhteydess�
+yhteyteen
+yksi
+yksin
+yksitt�in
+yleens�
+ylemm�s
+yli
+yl�s
+ymp�ri

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/fr_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/fr_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/fr_ST.txt
new file mode 100644
index 0000000..c84d8c1
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/fr_ST.txt
@@ -0,0 +1,464 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+a
+�
+�
+abord
+afin
+ah
+ai
+aie
+ainsi
+allaient
+allo
+all�
+allons
+apr�s
+assez
+attendu
+au
+aucun
+aucune
+aujourd
+aujourd'hui
+auquel
+aura
+auront
+aussi
+autre
+autres
+aux
+auxquelles
+auxquels
+avaient
+avais
+avait
+avant
+avec
+avoir
+ayant
+b
+bah
+beaucoup
+bien
+bigre
+boum
+bravo
+brrr
+c
+�a
+car
+ce
+ceci
+cela
+celle
+celle-ci
+celle-l�
+celles
+celles-ci
+celles-l�
+celui
+celui-ci
+celui-l�
+cent
+cependant
+certain
+certaine
+certaines
+certains
+certes
+ces
+cet
+cette
+ceux
+ceux-ci
+ceux-l�
+chacun
+chaque
+cher
+ch�re
+ch�res
+chers
+chez
+chiche
+chut
+ci
+cinq
+cinquantaine
+cinquante
+cinquanti�me
+cinqui�me
+clac
+clic
+combien
+comme
+comment
+compris
+concernant
+contre
+couic
+crac
+d
+da
+dans
+de
+debout
+dedans
+dehors
+del�
+depuis
+derri�re
+des
+d�s
+d�sormais
+desquelles
+desquels
+dessous
+dessus
+deux
+deuxi�me
+deuxi�mement
+devant
+devers
+devra
+diff�rent
+diff�rente
+diff�rentes
+diff�rents
+dire
+divers
+diverse
+diverses
+dix
+dix-huit
+dixi�me
+dix-neuf
+dix-sept
+doit
+doivent
+donc
+dont
+douze
+douzi�me
+dring
+du
+duquel
+durant
+e
+effet
+eh
+elle
+elle-m�me
+elles
+elles-m�mes
+en
+encore
+entre
+envers
+environ
+es
+�s
+est
+et
+etant
+�taient
+�tais
+�tait
+�tant
+etc
+�t�
+etre
+�tre
+eu
+euh
+eux
+eux-m�mes
+except�
+f
+fa�on
+fais
+faisaient
+faisant
+fait
+feront
+fi
+flac
+floc
+font
+g
+gens
+h
+ha
+h�
+hein
+h�las
+hem
+hep
+hi
+ho
+hol�
+hop
+hormis
+hors
+hou
+houp
+hue
+hui
+huit
+huiti�me
+hum
+hurrah
+i
+il
+ils
+importe
+j
+je
+jusqu
+jusque
+k
+l
+la
+l�
+laquelle
+las
+le
+lequel
+les
+l�s
+lesquelles
+lesquels
+leur
+leurs
+longtemps
+lorsque
+lui
+lui-m�me
+m
+ma
+maint
+mais
+malgr�
+me
+m�me
+m�mes
+merci
+mes
+mien
+mienne
+miennes
+miens
+mille
+mince
+moi
+moi-m�me
+moins
+mon
+moyennant
+n
+na
+ne
+n�anmoins
+neuf
+neuvi�me
+ni
+nombreuses
+nombreux
+non
+nos
+notre
+n�tre
+n�tres
+nous
+nous-m�mes
+nul
+o
+o|
+�
+oh
+oh�
+ol�
+oll�
+on
+ont
+onze
+onzi�me
+ore
+ou
+o�
+ouf
+ouias
+oust
+ouste
+outre
+p
+paf
+pan
+par
+parmi
+partant
+particulier
+particuli�re
+particuli�rement
+pas
+pass�
+pendant
+personne
+peu
+peut
+peuvent
+peux
+pff
+pfft
+pfut
+pif
+plein
+plouf
+plus
+plusieurs
+plut�t
+pouah
+pour
+pourquoi
+premier
+premi�re
+premi�rement
+pr�s
+proche
+psitt
+puisque
+q
+qu
+quand
+quant
+quanta
+quant-�-soi
+quarante
+quatorze
+quatre
+quatre-vingt
+quatri�me
+quatri�mement
+que
+quel
+quelconque
+quelle
+quelles
+quelque
+quelques
+quelqu'un
+quels
+qui
+quiconque
+quinze
+quoi
+quoique
+r
+revoici
+revoil�
+rien
+s
+sa
+sacrebleu
+sans
+sapristi
+sauf
+se
+seize
+selon
+sept
+septi�me
+sera
+seront
+ses
+si
+sien
+sienne
+siennes
+siens
+sinon
+six
+sixi�me
+soi
+soi-m�me
+soit
+soixante
+son
+sont
+sous
+stop
+suis
+suivant
+sur
+surtout
+t
+ta
+tac
+tant
+te
+t�
+tel
+telle
+tellement
+telles
+tels
+tenant
+tes
+tic
+tien
+tienne
+tiennes
+tiens
+toc
+toi
+toi-m�me
+ton
+touchant
+toujours
+tous
+tout
+toute
+toutes
+treize
+trente
+tr�s
+trois
+troisi�me
+troisi�mement
+trop
+tsoin
+tsouin
+tu
+u
+un
+une
+unes
+uns
+v
+va
+vais
+vas
+v�
+vers
+via
+vif
+vifs
+vingt
+vivat
+vive
+vives
+vlan
+voici
+voil�
+vont
+vos
+votre
+v�tre
+v�tres
+vous
+vous-m�mes
+vu
+w
+x
+y
+z
+zut

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/hi_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/hi_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/hi_ST.txt
new file mode 100644
index 0000000..426fc2d
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/hi_ST.txt
@@ -0,0 +1,164 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+पर
+इन 
+वह 
+यिह 
+वुह 
+जिन्हें
+जिन्हों
+तिन्हें
+तिन्हों
+किन्हों
+किन्हें
+इत्यादि
+द्वारा
+इन्हें
+इन्हों
+उन्हों
+बिलकुल
+निहायत
+ऱ्वासा
+इन्हीं
+उन्हीं
+उन्हें
+इसमें
+जितना
+दुसरा
+कितना
+दबारा
+साबुत
+वग़ैरह
+दूसरे
+कौनसा
+लेकिन
+होता
+करने
+किया
+लिये
+अपने
+नहीं
+दिया
+इसका
+करना
+वाले
+सकते
+इसके
+सबसे
+होने
+करते
+बहुत
+वर्ग
+करें
+होती
+अपनी
+उनके
+कहते
+होते
+करता
+उनकी
+इसकी
+सकता
+रखें
+अपना
+उसके
+जिसे
+तिसे
+किसे
+किसी
+काफ़ी
+पहले
+नीचे
+बाला
+यहाँ
+जैसा
+जैसे
+मानो
+अंदर
+भीतर
+पूरा
+सारा
+होना
+उनको
+वहाँ
+वहीं
+जहाँ
+जीधर
+उनका
+इनका
+के
+हैं
+गया
+बनी
+एवं
+हुआ
+साथ
+बाद
+लिए
+कुछ
+कहा
+यदि
+हुई
+इसे
+हुए
+अभी
+सभी
+कुल
+रहा
+रहे
+इसी
+उसे
+जिस
+जिन
+तिस
+तिन
+कौन
+किस
+कोई
+ऐसे
+तरह
+किर
+साभ
+संग
+यही
+बही
+उसी
+फिर
+मगर
+का
+एक
+यह
+से
+को
+इस
+कि
+जो
+कर
+मे
+ने
+तो
+ही
+या
+हो
+था
+तक
+आप
+ये
+थे
+दो
+वे
+थी
+जा
+ना
+उस
+एस
+पे
+उन
+सो
+भी
+और
+घर
+तब
+जब
+अत
+व
+न

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/hu_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/hu_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/hu_ST.txt
new file mode 100644
index 0000000..7cf3e1c
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/hu_ST.txt
@@ -0,0 +1,738 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+a
+abba
+abban
+abból
+addig
+ahhoz
+ahol
+akár
+aki
+akik
+akkor
+alá
+alád
+alájuk
+alám
+alánk
+alapján
+alátok
+alatt
+alatta
+alattad
+alattam
+alattatok
+alattuk
+alattunk
+alól
+alóla
+alólad
+alólam
+alólatok
+alóluk
+alólunk
+által
+általában
+ám
+amely
+amelybol
+amelyek
+amelyekben
+amelyeket
+amelyet
+amelyik
+amelynek
+ami
+amíg
+amikor
+amit
+amott
+annak
+annál
+arra
+arról
+át
+attól
+az
+azért
+aznap
+azok
+azokat
+azokba
+azokban
+azokból
+azokért
+azokhoz
+azokig
+azokká
+azokkal
+azoknak
+azoknál
+azokon
+azokra
+azokról
+azoktól
+azon
+azonban
+azonnal
+azt
+aztán
+azzá
+azzal
+bal
+balra
+ban
+bár
+bárcsak
+bármilyen
+be
+belé
+beléd
+beléjük
+belém
+belénk
+belétek
+belőle
+belőled
+belőlem
+belőletek
+belőlük
+belőlünk
+belül
+ben
+benne
+benned
+bennem
+bennetek
+bennük
+bennünk
+búcsú
+csak
+csakhogy
+csupán
+de
+dehogy
+ebbe
+ebben
+ebből
+eddig
+egész
+egészen
+egy
+egyéb
+egyebek
+egyebet
+egyedül
+egyelőre
+egyet
+egyik
+egymás
+egyre
+egyszerre
+együtt
+ehhez
+el
+elé
+eléd
+elég
+eleinte
+eléjük
+elém
+elénk
+elétek
+éljen
+ellen
+ellenére
+ellenes
+elleni
+elmondta
+előbb
+elől
+előle
+előled
+előlem
+előletek
+előlük
+előlünk
+először
+előtt
+előtte
+előtted
+előttem
+előttetek
+előttük
+előttünk
+előző
+első
+elsők
+elsősorban
+elsőt
+én
+engem
+ennek
+ennél
+ennyi
+enyém
+erre
+erről
+érte
+érted
+értem
+értetek
+értük
+értünk
+és
+esetben
+ettől
+év
+évben
+éve
+évek
+éves
+évi
+évvel
+ez
+ezek
+ezekbe
+ezekben
+ezekből
+ezeken
+ezekért
+ezeket
+ezekhez
+ezekig
+ezekké
+ezekkel
+ezeknek
+ezeknél
+ezekre
+ezekről
+ezektől
+ezen
+ezentúl
+ezer
+ezért
+ezret
+ezt
+ezután
+ezzé
+ezzel
+fel
+fél
+fele
+felé
+felek
+felet
+felett
+fent
+fenti
+fölé
+gyakran
+ha
+halló
+hamar
+hanem
+hány
+hányszor
+harmadik
+harmadikat
+hármat
+harminc
+három
+hat
+hát
+hátha
+hatodik
+hatodikat
+hatot
+hátulsó
+hatvan
+helyett
+hét
+hetedik
+hetediket
+hetet
+hetven
+hiába
+hirtelen
+hiszen
+hogy
+hol
+holnap
+holnapot
+honnan
+hova
+hozzá
+hozzád
+hozzájuk
+hozzám
+hozzánk
+hozzátok
+hurrá
+húsz
+huszadik
+idén
+ide-оda
+igazán
+igen
+így
+illetve
+ilyen
+immár
+inkább
+is
+ismét
+itt
+jelenleg
+jó
+jobban
+jobbra
+jól
+jólesik
+jóval
+jövőre
+kell
+kellene
+kellett
+kelljen
+képest
+kérem
+kérlek
+késő
+később
+későn
+kész
+két
+kétszer
+ketten
+kettő
+kettőt
+kevés
+ki
+kiben
+kiből
+kicsit
+kicsoda
+kié
+kiért
+kihez
+kik
+kikbe
+kikben
+kikből
+kiken
+kikért
+kiket
+kikhez
+kikké
+kikkel
+kiknek
+kiknél
+kikre
+kikről
+kiktől
+kilenc
+kilencedik
+kilencediket
+kilencet
+kilencven
+kin
+kinek
+kinél
+kire
+kiről
+kit
+kitől
+kivé
+kivel
+korábban
+körül
+köszönhetően
+köszönöm
+közben
+közé
+közel
+közepén
+közepesen
+között
+közül
+külön
+különben
+különböző
+különbözőbb
+különbözőek
+lassan
+le
+legalább
+legyen
+lehet
+lehetetlen
+lehetőleg
+lehetőség
+lenne
+lennék
+lennének
+lesz
+leszek
+lesznek
+leszünk
+lett
+lettek
+lettem
+lettünk
+lévő
+ma
+maga
+magad
+magam
+magát
+magatokat
+magukat
+magunkat
+mai
+majd
+majdnem
+manapság
+már
+más
+másik
+másikat
+másnap
+második
+másodszor
+mások
+másokat
+mást
+meg
+még
+megcsinál
+megcsinálnak
+megint
+mégis
+megvan
+mellé
+melléd
+melléjük
+mellém
+mellénk
+mellétek
+mellett
+mellette
+melletted
+mellettem
+mellettetek
+mellettük
+mellettünk
+mellől
+mellőle
+mellőled
+mellőlem
+mellőletek
+mellőlük
+mellőlünk
+melyik
+mennyi
+mert
+mi
+miatt
+miatta
+miattad
+miattam
+miattatok
+miattuk
+miattunk
+mibe
+miben
+miből
+miért
+míg
+mihez
+mik
+mikbe
+mikben
+mikből
+miken
+mikért
+miket
+mikhez
+mikké
+mikkel
+miknek
+miknél
+mikor
+mikre
+mikről
+miktől
+milyen
+min
+mind
+mindegyik
+mindegyiket
+minden
+mindenesetre
+mindenki
+mindent
+mindenütt
+mindig
+mindketten
+minek
+minél
+minket
+mint
+mire
+miről
+mit
+mitől
+mivé
+mivel
+mögé
+mögéd
+mögéjük
+mögém
+mögénk
+mögétek
+mögött
+mögötte
+mögötted
+mögöttem
+mögöttetek
+mögöttük
+mögöttünk
+mögül
+mögüle
+mögüled
+mögülem
+mögületek
+mögülük
+mögülünk
+mondta
+most
+mostanáig
+múltkor
+múlva
+na
+nagyon
+nála
+nálad
+nálam
+nálatok
+náluk
+nálunk
+naponta
+napot
+ne
+négy
+negyedik
+negyediket
+négyet
+negyven
+néha
+néhány
+neked
+nekem
+neki
+nekik
+nektek
+nekünk
+nélkül
+nem
+nemcsak
+nemrég
+nincs
+nyolc
+nyolcadik
+nyolcadikat
+nyolcat
+nyolcvan
+ő
+ők
+őket
+olyan
+ön
+önbe
+önben
+önből
+önért
+önhöz
+onnan
+önnek
+önnel
+önnél
+önök
+önökbe
+önökben
+önökből
+önökért
+önöket
+önökhöz
+önökkel
+önöknek
+önöknél
+önökön
+önökre
+önökről
+önöktől
+önön
+önre
+önről
+önt
+öntől
+öt
+őt
+óta
+ötödik
+ötödiket
+ötöt
+ott
+ötven
+pár
+pedig
+például
+persze
+rá
+rád
+rajta
+rajtad
+rajtam
+rajtatok
+rajtuk
+rajtunk
+rájuk
+rám
+ránk
+rátok
+régen
+régóta
+rendben
+részére
+rögtön
+róla
+rólad
+rólam
+rólatok
+róluk
+rólunk
+rosszul
+se
+sem
+semmi
+semmilyen
+semmiség
+senki
+soha
+sok
+sokáig
+sokan
+sokszor
+során
+sőt
+stb.
+számára
+száz
+századik
+százat
+szemben
+szépen
+szerbusz
+szerint
+szerinte
+szerinted
+szerintem
+szerintetek
+szerintük
+szerintünk
+szervusz
+szinte
+szíves
+szívesen
+szíveskedjék
+talán
+tavaly
+távol
+te
+téged
+tegnap
+tegnapelőtt
+tehát
+tele
+tényleg
+tessék
+ti
+tied
+titeket
+tíz
+tizedik
+tizediket
+tizenegy
+tizenegyedik
+tizenhárom
+tizenhat
+tizenhét
+tizenkét
+tizenkettedik
+tizenkettő
+tizenkilenc
+tizennégy
+tizennyolc
+tizenöt
+tizet
+több
+többi
+többször
+tőle
+tőled
+tőlem
+tőletek
+tőlük
+tőlünk
+tovább
+további
+túl
+úgy
+ugyanakkor
+ugyanez
+ugyanis
+ugye
+úgyis
+úgynevezett
+újra
+úr
+urak
+uram
+urat
+után
+utoljára
+utolsó
+vagy
+vagyis
+vagyok
+vagytok
+vagyunk
+vajon
+valahol
+valaki
+valakit
+valamelyik
+valami
+valamint
+van
+vannak
+végén
+végre
+végül
+vele
+veled
+velem
+veletek
+velük
+velünk
+viszlát
+viszont
+viszontlátásra
+volna
+volnának
+volnék
+volt
+voltak
+voltam
+voltunk

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/it_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/it_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/it_ST.txt
new file mode 100644
index 0000000..23c80a2
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/it_ST.txt
@@ -0,0 +1,400 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+a
+abbastanza
+accidenti
+ad
+adesso
+affinche
+agli
+ahime
+ahimè
+ai
+al
+alcuna
+alcuni
+alcuno
+all
+alla
+alle
+allo
+altri
+altrimenti
+altro
+altrui
+anche
+ancora
+anni
+anno
+ansa
+assai
+attesa
+avanti
+avendo
+avente
+aver
+avere
+avete
+aveva
+avuta
+avute
+avuti
+avuto
+basta
+bene
+benissimo
+berlusconi
+brava
+bravo
+c
+casa
+caso
+cento
+certa
+certe
+certi
+certo
+che
+chi
+chicchessia
+chiunque
+ci
+ciascuna
+ciascuno
+cima
+cio
+ciò
+cioe
+cioè
+circa
+citta
+città
+codesta
+codesti
+codesto
+cogli
+coi
+col
+colei
+coll
+coloro
+colui
+come
+con
+concernente
+consiglio
+contro
+cortesia
+cos
+cosa
+cosi
+così
+cui
+d
+da
+dagli
+dai
+dal
+dall
+dalla
+dalle
+dallo
+davanti
+degli
+dei
+del
+dell
+della
+delle
+dello
+dentro
+detto
+deve
+di
+dice
+dietro
+dire
+dirimpetto
+dopo
+dove
+dovra
+dovrà
+due
+dunque
+durante
+e
+è
+ecco
+ed
+egli
+ella
+eppure
+era
+erano
+esse
+essendo
+esser
+essere
+essi
+ex
+fa
+fare
+fatto
+favore
+fin
+finalmente
+finche
+fine
+fino
+forse
+fra
+fuori
+gia
+già
+giacche
+giorni
+giorno
+gli
+gliela
+gliele
+glieli
+glielo
+gliene
+governo
+grande
+grazie
+gruppo
+ha
+hai
+hanno
+ho
+i
+ieri
+il
+improvviso
+in
+infatti
+insieme
+intanto
+intorno
+invece
+io
+l
+la
+là
+lavoro
+le
+lei
+li
+lo
+lontano
+loro
+lui
+lungo
+ma
+macche
+magari
+mai
+male
+malgrado
+malissimo
+me
+medesimo
+mediante
+meglio
+meno
+mentre
+mesi
+mezzo
+mi
+mia
+mie
+miei
+mila
+miliardi
+milioni
+ministro
+mio
+moltissimo
+molto
+mondo
+nazionale
+ne
+negli
+nei
+nel
+nell
+nella
+nelle
+nello
+nemmeno
+neppure
+nessuna
+nessuno
+niente
+no
+noi
+non
+nondimeno
+nostra
+nostre
+nostri
+nostro
+nulla
+nuovo
+o
+od
+oggi
+ogni
+ognuna
+ognuno
+oltre
+oppure
+ora
+ore
+osi
+ossia
+paese
+parecchi
+parecchie
+parecchio
+parte
+partendo
+peccato
+peggio
+per
+perche
+perchè
+percio
+perciò
+perfino
+pero
+però
+persone
+piedi
+pieno
+piglia
+piu
+più
+po
+pochissimo
+poco
+poi
+poiche
+press
+prima
+primo
+proprio
+puo
+può
+pure
+purtroppo
+qualche
+qualcuna
+qualcuno
+quale
+quali
+qualunque
+quando
+quanta
+quante
+quanti
+quanto
+quantunque
+quasi
+quattro
+quel
+quella
+quelli
+quello
+quest
+questa
+queste
+questi
+questo
+qui
+quindi
+riecco
+salvo
+sara
+sarà
+sarebbe
+scopo
+scorso
+se
+secondo
+seguente
+sei
+sempre
+senza
+si
+sia
+siamo
+siete
+solito
+solo
+sono
+sopra
+sotto
+sta
+staranno
+stata
+state
+stati
+stato
+stesso
+su
+sua
+successivo
+sue
+sugli
+sui
+sul
+sull
+sulla
+sulle
+sullo
+suo
+suoi
+tale
+talvolta
+tanto
+te
+tempo
+ti
+torino
+tra
+tranne
+tre
+troppo
+tu
+tua
+tue
+tuo
+tuoi
+tutta
+tuttavia
+tutte
+tutti
+tutto
+uguali
+un
+una
+uno
+uomo
+va
+vale
+varia
+varie
+vario
+verso
+vi
+via
+vicino
+visto
+vita
+voi
+volta
+vostra
+vostre
+vostri
+vostro

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/pl_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/pl_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/pl_ST.txt
new file mode 100644
index 0000000..e27c30e
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/pl_ST.txt
@@ -0,0 +1,139 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+ach
+aj
+albo
+bardzo
+bez
+bo
+być
+ci
+cię
+ciebie
+co
+czy
+daleko
+dla
+dlaczego
+dlatego
+do
+dobrze
+dokąd
+dość
+dużo
+dwa
+dwaj
+dwie
+dwoje
+dziś
+dzisiaj
+gdyby
+gdzie
+go
+ich
+ile
+im
+inny
+ja
+ją
+jak
+jakby
+jaki
+je
+jeden
+jedna
+jedno
+jego
+jej
+jemu
+jeśli
+jest
+jestem
+jeżeli
+już
+każdy
+kiedy
+kierunku
+kto
+ku
+lub
+ma
+mają
+mam
+mi
+mną
+mnie
+moi
+mój
+moja
+moje
+może
+mu
+my
+na
+nam
+nami
+nas
+nasi
+nasz
+nasza
+nasze
+natychmiast
+nią
+nic
+nich
+nie
+niego
+niej
+niemu
+nigdy
+nim
+nimi
+niż
+obok
+od
+około
+on
+ona
+one
+oni
+ono
+owszem
+po
+pod
+ponieważ
+przed
+przedtem
+są
+sam
+sama
+się
+skąd
+tak
+taki
+tam
+ten
+to
+tobą
+tobie
+tu
+tutaj
+twoi
+twój
+twoja
+twoje
+ty
+wam
+wami
+was
+wasi
+wasz
+wasza
+wasze
+we
+więc
+wszystko
+wtedy
+wy
+żaden
+zawsze
+że

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/pt_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/pt_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/pt_ST.txt
new file mode 100644
index 0000000..da60644
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/pt_ST.txt
@@ -0,0 +1,357 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+a
+�
+adeus
+agora
+a�
+ainda
+al�m
+algo
+algumas
+alguns
+ali
+ano
+anos
+antes
+ao
+aos
+apenas
+apoio
+ap�s
+aquela
+aquelas
+aquele
+aqueles
+aqui
+aquilo
+�rea
+as
+�s
+assim
+at�
+atr�s
+atrav�s
+baixo
+bastante
+bem
+bom
+breve
+c�
+cada
+catorze
+cedo
+cento
+certamente
+certeza
+cima
+cinco
+coisa
+com
+como
+conselho
+contra
+custa
+da
+d�
+d�o
+daquela
+daquele
+dar
+das
+de
+debaixo
+demais
+dentro
+depois
+desde
+dessa
+desse
+desta
+deste
+deve
+dever�
+dez
+dezanove
+dezasseis
+dezassete
+dezoito
+dia
+diante
+diz
+dizem
+dizer
+do
+dois
+dos
+doze
+duas
+d�vida
+e
+�
+ela
+elas
+ele
+eles
+em
+embora
+entre
+era
+�s
+essa
+essas
+esse
+esses
+esta
+est�
+estar
+estas
+est�s
+estava
+este
+estes
+esteve
+estive
+estivemos
+estiveram
+estiveste
+estivestes
+estou
+eu
+exemplo
+fa�o
+falta
+favor
+faz
+fazeis
+fazem
+fazemos
+fazer
+fazes
+fez
+fim
+final
+foi
+fomos
+for
+foram
+forma
+foste
+fostes
+fui
+geral
+grande
+grandes
+grupo
+h�
+hoje
+horas
+isso
+isto
+j�
+l�
+lado
+local
+logo
+longe
+lugar
+maior
+maioria
+mais
+mal
+mas
+m�ximo
+me
+meio
+menor
+menos
+m�s
+meses
+meu
+meus
+mil
+minha
+minhas
+momento
+muito
+muitos
+na
+nada
+n�o
+naquela
+naquele
+nas
+nem
+nenhuma
+nessa
+nesse
+nesta
+neste
+n�vel
+no
+noite
+nome
+nos
+n�s
+nossa
+nossas
+nosso
+nossos
+nova
+nove
+novo
+novos
+num
+numa
+n�mero
+nunca
+o
+obra
+obrigada
+obrigado
+oitava
+oitavo
+oito
+onde
+ontem
+onze
+os
+ou
+outra
+outras
+outro
+outros
+para
+parece
+parte
+partir
+pela
+pelas
+pelo
+pelos
+perto
+pode
+p�de
+podem
+poder
+p�e
+p�em
+ponto
+pontos
+por
+porque
+porqu�
+posi��o
+poss�vel
+possivelmente
+posso
+pouca
+pouco
+primeira
+primeiro
+pr�prio
+pr�ximo
+puderam
+qual
+quando
+quanto
+quarta
+quarto
+quatro
+que
+qu�
+quem
+quer
+quero
+quest�o
+quinta
+quinto
+quinze
+rela��o
+sabe
+s�o
+se
+segunda
+segundo
+sei
+seis
+sem
+sempre
+ser
+seria
+sete
+s�tima
+s�timo
+seu
+seus
+sexta
+sexto
+sim
+sistema
+sob
+sobre
+sois
+somos
+sou
+sua
+suas
+tal
+talvez
+tamb�m
+tanto
+t�o
+tarde
+te
+tem
+t�m
+temos
+tendes
+tenho
+tens
+ter
+terceira
+terceiro
+teu
+teus
+teve
+tive
+tivemos
+tiveram
+tiveste
+tivestes
+toda
+todas
+todo
+todos
+trabalho
+tr�s
+treze
+tu
+tua
+tuas
+tudo
+um
+uma
+umas
+uns
+vai
+vais
+v�o
+v�rios
+vem
+v�m
+vens
+ver
+vez
+vezes
+viagem
+vindo
+vinte
+voc�
+voc�s
+vos
+v�s
+vossa
+vossas
+vosso
+vossos
+zero

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/ro_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/ro_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/ro_ST.txt
new file mode 100644
index 0000000..ec7c517
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/ro_ST.txt
@@ -0,0 +1,283 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+acea
+aceasta
+această
+aceea
+acei
+aceia
+acel
+acela
+acele
+acelea
+acest
+acesta
+aceste
+acestea
+aceşti
+aceştia
+acolo
+acord
+acum
+ai
+aia
+aibă
+aici
+al
+ăla
+ale
+alea
+ălea
+altceva
+altcineva
+am
+ar
+are
+aş
+aşadar
+asemenea
+asta
+ăsta
+astăzi
+astea
+ăstea
+ăştia
+asupra
+aţi
+au
+avea
+avem
+aveţi
+azi
+bine
+bucur
+bună
+ca
+că
+căci
+când
+care
+cărei
+căror
+cărui
+cât
+câte
+câţi
+către
+câtva
+caut
+ce
+cel
+ceva
+chiar
+cinci
+cînd
+cine
+cineva
+cît
+cîte
+cîţi
+cîtva
+contra
+cu
+cum
+cumva
+curând
+curînd
+da
+dă
+dacă
+dar
+dată
+datorită
+dau
+de
+deci
+deja
+deoarece
+departe
+deşi
+din
+dinaintea
+dintr-
+dintre
+doi
+doilea
+două
+drept
+după
+ea
+ei
+el
+ele
+eram
+este
+eşti
+eu
+face
+fără
+fata
+fi
+fie
+fiecare
+fii
+fim
+fiţi
+fiu
+frumos
+graţie
+halbă
+iar
+ieri
+îi
+îl
+îmi
+împotriva
+în 
+înainte
+înaintea
+încât
+încît
+încotro
+între
+întrucât
+întrucît
+îţi
+la
+lângă
+le
+li
+lîngă
+lor
+lui
+mă
+mai
+mâine
+mea
+mei
+mele
+mereu
+meu
+mi
+mie
+mîine
+mine
+mult
+multă
+mulţi
+mulţumesc
+ne
+nevoie
+nicăieri
+nici
+nimeni
+nimeri
+nimic
+nişte
+noastră
+noastre
+noi
+noroc
+noştri
+nostru
+nouă
+nu
+opt
+ori
+oricând
+oricare
+oricât
+orice
+oricînd
+oricine
+oricît
+oricum
+oriunde
+până
+patra
+patru
+patrulea
+pe
+pentru
+peste
+pic
+pînă
+poate
+pot
+prea
+prima
+primul
+prin
+printr-
+puţin
+puţina
+puţină
+rog
+sa
+să
+săi
+sale
+şapte
+şase
+sau
+său
+se
+şi
+sînt
+sîntem
+sînteţi
+spate
+spre
+ştiu
+sub
+sunt
+suntem
+sunteţi
+sută
+ta
+tăi
+tale
+tău
+te
+ţi
+ţie
+timp
+tine
+toată
+toate
+tot
+toţi
+totuşi
+trei
+treia
+treilea
+tu
+un
+una
+unde
+undeva
+unei
+uneia
+unele
+uneori
+unii
+unor
+unora
+unu
+unui
+unuia
+unul
+vă
+vi
+voastră
+voastre
+voi
+voştri
+vostru
+vouă
+vreme
+vreo
+vreun
+zece
+zero
+zi
+zice

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/sv_ST.txt
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/sv_ST.txt b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/sv_ST.txt
new file mode 100644
index 0000000..582ab5a
--- /dev/null
+++ b/src/resources/org/apache/cassandra/index/sasi/analyzer/filter/sv_ST.txt
@@ -0,0 +1,387 @@
+# Stop Words List from http://members.unine.ch/jacques.savoy/clef/index.html
+aderton
+adertonde
+adj�
+aldrig
+alla
+allas
+allt
+alltid
+allts�
+�n
+andra
+andras
+annan
+annat
+�nnu
+artonde
+artonn
+�tminstone
+att
+�tta
+�ttio
+�ttionde
+�ttonde
+av
+�ven
+b�da
+b�das
+bakom
+bara
+b�st
+b�ttre
+beh�va
+beh�vas
+beh�vde
+beh�vt
+beslut
+beslutat
+beslutit
+bland
+blev
+bli
+blir
+blivit
+bort
+borta
+bra
+d�
+dag
+dagar
+dagarna
+dagen
+d�r
+d�rf�r
+de
+del
+delen
+dem
+den
+deras
+dess
+det
+detta
+dig
+din
+dina
+dit
+ditt
+dock
+du
+efter
+eftersom
+elfte
+eller
+elva
+en
+enkel
+enkelt
+enkla
+enligt
+er
+era
+ert
+ett
+ettusen
+f� 
+fanns
+f�r
+f�tt 
+fem
+femte
+femtio
+femtionde
+femton
+femtonde
+fick
+fin
+finnas
+finns
+fj�rde
+fjorton
+fjortonde
+fler
+flera
+flesta
+f�ljande
+f�r
+f�re
+f�rl�t
+f�rra
+f�rsta
+fram
+framf�r
+fr�n
+fyra
+fyrtio
+fyrtionde
+g�
+g�lla
+g�ller
+g�llt
+g�r
+g�rna
+g�tt
+genast
+genom
+gick
+gjorde
+gjort
+god
+goda
+godare
+godast
+g�r
+g�ra
+gott
+ha
+hade
+haft
+han
+hans
+har
+h�r
+heller
+hellre
+helst
+helt
+henne
+hennes
+hit
+h�g
+h�ger
+h�gre
+h�gst
+hon
+honom
+hundra
+hundraen
+hundraett
+hur
+i
+ibland
+idag
+ig�r
+igen
+imorgon
+in
+inf�r
+inga
+ingen
+ingenting
+inget
+innan
+inne
+inom
+inte
+inuti
+ja
+jag
+j�mf�rt
+kan
+kanske
+knappast
+kom
+komma
+kommer
+kommit
+kr
+kunde
+kunna
+kunnat
+kvar
+l�nge
+l�ngre
+l�ngsam
+l�ngsammare
+l�ngsammast
+l�ngsamt
+l�ngst
+l�ngt
+l�tt
+l�ttare
+l�ttast
+legat
+ligga
+ligger
+lika
+likst�lld
+likst�llda
+lilla
+lite
+liten
+litet
+man
+m�nga
+m�ste
+med
+mellan
+men
+mer
+mera
+mest
+mig
+min
+mina
+mindre
+minst
+mitt
+mittemot
+m�jlig
+m�jligen
+m�jligt
+m�jligtvis
+mot
+mycket
+n�gon
+n�gonting
+n�got
+n�gra
+n�r
+n�sta
+ned
+nederst
+nedersta
+nedre
+nej
+ner
+ni
+nio
+nionde
+nittio
+nittionde
+nitton
+nittonde
+n�dv�ndig
+n�dv�ndiga
+n�dv�ndigt
+n�dv�ndigtvis
+nog
+noll
+nr
+nu
+nummer
+och
+ocks�
+ofta
+oftast
+olika
+olikt
+om
+oss
+�ver
+�vermorgon
+�verst
+�vre
+p�
+rakt
+r�tt
+redan
+s�
+sade
+s�ga
+s�ger
+sagt
+samma
+s�mre
+s�mst
+sedan
+senare
+senast
+sent
+sex
+sextio
+sextionde
+sexton
+sextonde
+sig
+sin
+sina
+sist
+sista
+siste
+sitt
+sj�tte
+sju
+sjunde
+sjuttio
+sjuttionde
+sjutton
+sjuttonde
+ska
+skall
+skulle
+slutligen
+sm�
+sm�tt
+snart
+som
+stor
+stora
+st�rre
+st�rst
+stort
+tack
+tidig
+tidigare
+tidigast
+tidigt
+till
+tills
+tillsammans
+tio
+tionde
+tjugo
+tjugoen
+tjugoett
+tjugonde
+tjugotre
+tjugotv�
+tjungo
+tolfte
+tolv
+tre
+tredje
+trettio
+trettionde
+tretton
+trettonde
+tv�
+tv�hundra
+under
+upp
+ur
+urs�kt
+ut
+utan
+utanf�r
+ute
+vad
+v�nster
+v�nstra
+var
+v�r
+vara
+v�ra
+varf�r
+varifr�n
+varit
+varken
+v�rre
+vars�god
+vart
+v�rt
+vem
+vems
+verkligen
+vi
+vid
+vidare
+viktig
+viktigare
+viktigast
+viktigt
+vilka
+vilken
+vilket
+vill

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/test/conf/cassandra-murmur.yaml
----------------------------------------------------------------------
diff --git a/test/conf/cassandra-murmur.yaml b/test/conf/cassandra-murmur.yaml
new file mode 100644
index 0000000..00f8b4c
--- /dev/null
+++ b/test/conf/cassandra-murmur.yaml
@@ -0,0 +1,43 @@
+#
+# Warning!
+# Consider the effects on 'o.a.c.i.s.LegacySSTableTest' before changing schemas in this file.
+#
+cluster_name: Test Cluster
+memtable_allocation_type: heap_buffers
+commitlog_sync: batch
+commitlog_sync_batch_window_in_ms: 1.0
+commitlog_segment_size_in_mb: 5
+commitlog_directory: build/test/cassandra/commitlog
+hints_directory: build/test/cassandra/hints
+partitioner: org.apache.cassandra.dht.Murmur3Partitioner
+listen_address: 127.0.0.1
+storage_port: 7010
+rpc_port: 9170
+start_native_transport: true
+native_transport_port: 9042
+column_index_size_in_kb: 4
+saved_caches_directory: build/test/cassandra/saved_caches
+data_file_directories:
+    - build/test/cassandra/data
+disk_access_mode: mmap
+seed_provider:
+    - class_name: org.apache.cassandra.locator.SimpleSeedProvider
+      parameters:
+          - seeds: "127.0.0.1"
+endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch
+dynamic_snitch: true
+request_scheduler: org.apache.cassandra.scheduler.RoundRobinScheduler
+request_scheduler_id: keyspace
+server_encryption_options:
+    internode_encryption: none
+    keystore: conf/.keystore
+    keystore_password: cassandra
+    truststore: conf/.truststore
+    truststore_password: cassandra
+incremental_backups: true
+concurrent_compactors: 4
+compaction_throughput_mb_per_sec: 0
+row_cache_class_name: org.apache.cassandra.cache.OHCProvider
+row_cache_size_in_mb: 16
+enable_user_defined_functions: true
+enable_scripted_user_defined_functions: true