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

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/72790dc8/src/java/org/apache/cassandra/index/sasi/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);
+    }
+}