You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by mc...@apache.org on 2018/03/15 09:45:46 UTC

[6/7] cassandra git commit: SASI tokenizer for simple delimiter based entries

SASI tokenizer for simple delimiter based entries

Patch by Mick Semb Wever; reviewed by Michael Kjellman for CASSANDRA-14247


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

Branch: refs/heads/trunk
Commit: 4c6a10af8bd5247348d5445e6fd3a2f141e50bc1
Parents: 5a662ea
Author: Mick Semb Wever <mc...@apache.org>
Authored: Wed Feb 21 21:16:05 2018 +1100
Committer: Mick Semb Wever <mc...@apache.org>
Committed: Thu Mar 15 17:41:50 2018 +1100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../index/sasi/analyzer/DelimiterAnalyzer.java  | 103 +++++++
 .../analyzer/DelimiterTokenizingOptions.java    |  71 +++++
 test/resources/tokenization/world_cities_a.csv  |   1 +
 .../sasi/analyzer/DelimiterAnalyzerTest.java    | 282 +++++++++++++++++++
 5 files changed, 458 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4c6a10af/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8f913cd..fcd86ed 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.3
+ * SASI tokenizer for simple delimiter based entries (CASSANDRA-14247)
  * Fix Loss of digits when doing CAST from varint/bigint to decimal (CASSANDRA-14170)
  * RateBasedBackPressure unnecessarily invokes a lock on the Guava RateLimiter (CASSANDRA-14163)
  * Fix wildcard GROUP BY queries (CASSANDRA-14209)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4c6a10af/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzer.java b/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzer.java
new file mode 100644
index 0000000..794a8b9
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzer.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.analyzer;
+
+import java.nio.CharBuffer;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import com.google.common.annotations.Beta;
+import com.google.common.base.Preconditions;
+
+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.utils.AbstractIterator;
+
+@Beta
+public class DelimiterAnalyzer extends AbstractAnalyzer
+{
+
+    private static final Map<AbstractType<?>,Charset> VALID_ANALYZABLE_TYPES = new HashMap<AbstractType<?>,Charset>()
+    {{
+            put(UTF8Type.instance, StandardCharsets.UTF_8);
+            put(AsciiType.instance, StandardCharsets.US_ASCII);
+    }};
+
+    private char delimiter;
+    private Charset charset;
+    private Iterator<ByteBuffer> iter;
+
+    public DelimiterAnalyzer()
+    {
+    }
+
+    public ByteBuffer next()
+    {
+        return iter.next();
+    }
+
+    public void init(Map<String, String> options, AbstractType validator)
+    {
+        DelimiterTokenizingOptions tokenizingOptions = DelimiterTokenizingOptions.buildFromMap(options);
+        delimiter = tokenizingOptions.getDelimiter();
+
+        if (!VALID_ANALYZABLE_TYPES.containsKey(validator))
+            throw new IllegalArgumentException(String.format("Only text types supported, got %s", validator));
+
+        charset = VALID_ANALYZABLE_TYPES.get(validator);
+    }
+
+    public boolean hasNext()
+    {
+        return iter.hasNext();
+    }
+
+    public void reset(ByteBuffer input)
+    {
+        Preconditions.checkNotNull(input);
+        final CharBuffer cb = charset.decode(input);
+
+        this.iter = new AbstractIterator<ByteBuffer>() {
+            protected ByteBuffer computeNext() {
+
+                if (!cb.hasRemaining())
+                    return endOfData();
+
+                CharBuffer readahead = cb.duplicate();
+                // loop until we see the next delimiter character, or reach end of data
+                while (readahead.hasRemaining() && readahead.get() != delimiter);
+
+                char[] chars = new char[readahead.position() - cb.position() - (readahead.hasRemaining() ? 1 : 0)];
+                cb.get(chars);
+                Preconditions.checkState(!cb.hasRemaining() || cb.get() == delimiter);
+                return charset.encode(CharBuffer.wrap(chars));
+            }
+        };
+    }
+
+
+    public boolean isTokenizing()
+    {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4c6a10af/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterTokenizingOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterTokenizingOptions.java b/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterTokenizingOptions.java
new file mode 100644
index 0000000..c2c8ef7
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterTokenizingOptions.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.analyzer;
+
+import java.util.Map;
+
+/** Simple tokenizer based on a specified delimiter (rather than whitespace).
+ */
+public class DelimiterTokenizingOptions
+{
+    public static final String DELIMITER = "delimiter";
+
+    private final char delimiter;
+
+    private DelimiterTokenizingOptions(char delimiter)
+    {
+        this.delimiter = delimiter;
+    }
+
+    char getDelimiter()
+    {
+        return delimiter;
+    }
+
+    private static class OptionsBuilder
+    {
+        private char delimiter = ',';
+
+        public DelimiterTokenizingOptions build()
+        {
+            return new DelimiterTokenizingOptions(delimiter);
+        }
+    }
+
+    static DelimiterTokenizingOptions buildFromMap(Map<String, String> optionsMap)
+    {
+        OptionsBuilder optionsBuilder = new OptionsBuilder();
+
+        for (Map.Entry<String, String> entry : optionsMap.entrySet())
+        {
+            switch (entry.getKey())
+            {
+                case DELIMITER:
+                {
+                    String value = entry.getValue();
+                    if (1 != value.length())
+                        throw new IllegalArgumentException(String.format("Only single character delimiters supported, was %s", value));
+
+                    optionsBuilder.delimiter = entry.getValue().charAt(0);
+                    break;
+                }
+            }
+        }
+        return optionsBuilder.build();
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org