[jira] [Commented] (CASSANDRA-11973) Is MemoryUtil.getShort() supposed to return a sign-extended or non-sign-extended value?
[ https://issues.apache.org/jira/browse/CASSANDRA-11973?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350391#comment-15350391 ] Stefania commented on CASSANDRA-11973: -- Thank you for the patch, +1. >From what I could understand, the intention was to return an unsigned short >and therefore an int with non-sign extension is correct. There is a comment >[here|https://issues.apache.org/jira/browse/CASSANDRA-10579?focusedCommentId=14985202=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14985202] > on the ticket that fixed it for the {{UNALIGNED}} case, CASSANDRA-10579, >according to which {{MemoryUtil.getShortByByte()}} is already returning an >unsigned short, but that's not correct unless we do a {{& 0x}}. Therefore, I agree with your patch and, if the tests are OK, I propose to commit it to 2.2+ with a slight variation to avoid duplicating {{& 0x}}: ||2.2||3.0||trunk|| |[patch|https://github.com/stef1927/cassandra/commits/11973-2.2]|[patch|https://github.com/stef1927/cassandra/commits/11973-3.0]|[patch|https://github.com/stef1927/cassandra/commits/11973]| |[testall|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-11973-2.2-testall/]|[testall|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-11973-3.0-testall/]|[testall|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-11973-testall/]| |[dtest|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-11973-2.2-dtest/]|[dtest|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-11973-3.0-dtest/]|[dtest|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-11973-dtest/]| [~philipthompson]: would it be possible to run the offheap dtests at least with one of these patches? > Is MemoryUtil.getShort() supposed to return a sign-extended or > non-sign-extended value? > --- > > Key: CASSANDRA-11973 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11973 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Rei Odaira >Assignee: Rei Odaira >Priority: Minor > Fix For: 2.2.x, 3.0.x, 3.x > > Attachments: 11973-2.2.txt > > > In org.apache.cassandra.utils.memory.MemoryUtil.getShort(), the returned > value of unsafe.getShort(address) is bit-wise-AND'ed with 0x, while that > of getShortByByte(address) is not. This inconsistency results in different > returned values when the short integer is negative. Which is preferred > behavior? Looking at NativeClustering and NativeCellTest, it seems like > non-sign-extension is assumed. > By the way, is there any reason MemoryUtil.getShort() and > MemoryUtil.getShortByByte() return "int", not "short"? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11993) Cannot read Snappy compressed tables with 3.6
[ https://issues.apache.org/jira/browse/CASSANDRA-11993?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefania updated CASSANDRA-11993: - Resolution: Fixed Fix Version/s: (was: 3.6) 3.8 Reproduced In: 3.6 Status: Resolved (was: Patch Available) > Cannot read Snappy compressed tables with 3.6 > - > > Key: CASSANDRA-11993 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11993 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: Nimi Wariboko Jr. >Assignee: Branimir Lambov > Fix For: 3.8 > > > After upgrading to 3.6, I can no longer read/compact sstables compressed with > snappy compression. The memtable_allocation_type makes no difference both > offheap_buffers and heap_buffers cause the errors. > {code} > WARN [SharedPool-Worker-5] 2016-06-10 15:45:18,731 > AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread > Thread[SharedPool-Worker-5,5,main]: {} > org.xerial.snappy.SnappyError: [NOT_A_DIRECT_BUFFER] destination is not a > direct buffer > at org.xerial.snappy.Snappy.uncompress(Snappy.java:509) > ~[snappy-java-1.1.1.7.jar:na] > at > org.apache.cassandra.io.compress.SnappyCompressor.uncompress(SnappyCompressor.java:102) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.CompressedSegmentedFile$Mmap.readChunk(CompressedSegmentedFile.java:323) > ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:137) > ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:19) > ~[apache-cassandra-3.6.jar:3.6] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache$BoundedLocalLoadingCache.lambda$new$0(BoundedLocalCache.java:2949) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$15(BoundedLocalCache.java:1807) > ~[caffeine-2.2.6.jar:na] > at > java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1853) > ~[na:1.8.0_66] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:1805) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:1788) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:97) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:66) > ~[caffeine-2.2.6.jar:na] > at > org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:215) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:193) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:78) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:220) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.SegmentedFile.createReader(SegmentedFile.java:138) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.sstable.format.SSTableReader.getFileDataInput(SSTableReader.java:1779) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator.(AbstractSSTableIterator.java:103) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:44) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:72) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:65) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound.initializeIterator(UnfilteredRowIteratorWithLowerBound.java:85) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.maybeInit(LazilyInitializedUnfilteredRowIterator.java:48) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:99) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound.computeNext(UnfilteredRowIteratorWithLowerBound.java:94) > ~[apache-cassandra-3.6.jar:3.6] > at >
[jira] [Updated] (CASSANDRA-11993) Cannot read Snappy compressed tables with 3.6
[ https://issues.apache.org/jira/browse/CASSANDRA-11993?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefania updated CASSANDRA-11993: - Component/s: Local Write-Read Paths > Cannot read Snappy compressed tables with 3.6 > - > > Key: CASSANDRA-11993 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11993 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: Nimi Wariboko Jr. >Assignee: Branimir Lambov > Fix For: 3.8 > > > After upgrading to 3.6, I can no longer read/compact sstables compressed with > snappy compression. The memtable_allocation_type makes no difference both > offheap_buffers and heap_buffers cause the errors. > {code} > WARN [SharedPool-Worker-5] 2016-06-10 15:45:18,731 > AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread > Thread[SharedPool-Worker-5,5,main]: {} > org.xerial.snappy.SnappyError: [NOT_A_DIRECT_BUFFER] destination is not a > direct buffer > at org.xerial.snappy.Snappy.uncompress(Snappy.java:509) > ~[snappy-java-1.1.1.7.jar:na] > at > org.apache.cassandra.io.compress.SnappyCompressor.uncompress(SnappyCompressor.java:102) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.CompressedSegmentedFile$Mmap.readChunk(CompressedSegmentedFile.java:323) > ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:137) > ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:19) > ~[apache-cassandra-3.6.jar:3.6] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache$BoundedLocalLoadingCache.lambda$new$0(BoundedLocalCache.java:2949) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$15(BoundedLocalCache.java:1807) > ~[caffeine-2.2.6.jar:na] > at > java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1853) > ~[na:1.8.0_66] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:1805) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:1788) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:97) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:66) > ~[caffeine-2.2.6.jar:na] > at > org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:215) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:193) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:78) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:220) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.SegmentedFile.createReader(SegmentedFile.java:138) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.sstable.format.SSTableReader.getFileDataInput(SSTableReader.java:1779) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator.(AbstractSSTableIterator.java:103) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:44) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:72) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:65) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound.initializeIterator(UnfilteredRowIteratorWithLowerBound.java:85) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.maybeInit(LazilyInitializedUnfilteredRowIterator.java:48) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:99) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound.computeNext(UnfilteredRowIteratorWithLowerBound.java:94) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound.computeNext(UnfilteredRowIteratorWithLowerBound.java:26) > ~[apache-cassandra-3.6.jar:3.6] > at >
[jira] [Commented] (CASSANDRA-11993) Cannot read Snappy compressed tables with 3.6
[ https://issues.apache.org/jira/browse/CASSANDRA-11993?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350364#comment-15350364 ] Stefania commented on CASSANDRA-11993: -- All test failures accounted for, and passing locally; committed to trunk as ac53b720bd53641eb178a0a26f84df85108dfe7c, thank you! > Cannot read Snappy compressed tables with 3.6 > - > > Key: CASSANDRA-11993 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11993 > Project: Cassandra > Issue Type: Bug >Reporter: Nimi Wariboko Jr. >Assignee: Branimir Lambov > Fix For: 3.6 > > > After upgrading to 3.6, I can no longer read/compact sstables compressed with > snappy compression. The memtable_allocation_type makes no difference both > offheap_buffers and heap_buffers cause the errors. > {code} > WARN [SharedPool-Worker-5] 2016-06-10 15:45:18,731 > AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread > Thread[SharedPool-Worker-5,5,main]: {} > org.xerial.snappy.SnappyError: [NOT_A_DIRECT_BUFFER] destination is not a > direct buffer > at org.xerial.snappy.Snappy.uncompress(Snappy.java:509) > ~[snappy-java-1.1.1.7.jar:na] > at > org.apache.cassandra.io.compress.SnappyCompressor.uncompress(SnappyCompressor.java:102) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.CompressedSegmentedFile$Mmap.readChunk(CompressedSegmentedFile.java:323) > ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:137) > ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:19) > ~[apache-cassandra-3.6.jar:3.6] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache$BoundedLocalLoadingCache.lambda$new$0(BoundedLocalCache.java:2949) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$15(BoundedLocalCache.java:1807) > ~[caffeine-2.2.6.jar:na] > at > java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1853) > ~[na:1.8.0_66] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:1805) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:1788) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:97) > ~[caffeine-2.2.6.jar:na] > at > com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:66) > ~[caffeine-2.2.6.jar:na] > at > org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:215) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:193) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:78) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:220) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.util.SegmentedFile.createReader(SegmentedFile.java:138) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.sstable.format.SSTableReader.getFileDataInput(SSTableReader.java:1779) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator.(AbstractSSTableIterator.java:103) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:44) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:72) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:65) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound.initializeIterator(UnfilteredRowIteratorWithLowerBound.java:85) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.maybeInit(LazilyInitializedUnfilteredRowIterator.java:48) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:99) > ~[apache-cassandra-3.6.jar:3.6] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound.computeNext(UnfilteredRowIteratorWithLowerBound.java:94) > ~[apache-cassandra-3.6.jar:3.6] > at >
cassandra git commit: Chunk cache to request compressor-compatible buffers if pool space is exhausted
Repository: cassandra Updated Branches: refs/heads/trunk fb221095c -> ac53b720b Chunk cache to request compressor-compatible buffers if pool space is exhausted pathch by Branimir Lambov; reviewed by Stefania Alborghetti for CASSANDRA-11993 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ac53b720 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ac53b720 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ac53b720 Branch: refs/heads/trunk Commit: ac53b720bd53641eb178a0a26f84df85108dfe7c Parents: fb22109 Author: Branimir LambovAuthored: Fri Jun 24 18:04:50 2016 +0200 Committer: Stefania Alborghetti Committed: Mon Jun 27 10:40:29 2016 +0800 -- CHANGES.txt| 1 + src/java/org/apache/cassandra/cache/ChunkCache.java| 3 ++- src/java/org/apache/cassandra/io/compress/ICompressor.java | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ac53b720/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 5b72016..9486c53 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.8 + * Chunk cache to request compressor-compatible buffers if pool space is exhausted (CASSANDRA-11993) * Remove DatabaseDescriptor dependencies from SequentialWriter (CASSANDRA-11579) * Move skip_stop_words filter before stemming (CASSANDRA-12078) * Support seek() in EncryptedFileSegmentInputStream (CASSANDRA-11957) http://git-wip-us.apache.org/repos/asf/cassandra/blob/ac53b720/src/java/org/apache/cassandra/cache/ChunkCache.java -- diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java index 9c32746..e6296bd 100644 --- a/src/java/org/apache/cassandra/cache/ChunkCache.java +++ b/src/java/org/apache/cassandra/cache/ChunkCache.java @@ -31,6 +31,7 @@ import com.google.common.util.concurrent.MoreExecutors; import com.github.benmanes.caffeine.cache.*; import com.codahale.metrics.Timer; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.compress.BufferType; import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.io.util.*; import org.apache.cassandra.metrics.CacheMissMetrics; @@ -152,7 +153,7 @@ public class ChunkCache metrics.misses.mark(); try (Timer.Context ctx = metrics.missLatency.time()) { -ByteBuffer buffer = BufferPool.get(key.file.chunkSize()); +ByteBuffer buffer = BufferPool.get(key.file.chunkSize(), key.file.preferredBufferType()); assert buffer != null; rebufferer.readChunk(key.position, buffer); return new Buffer(buffer, key.position); http://git-wip-us.apache.org/repos/asf/cassandra/blob/ac53b720/src/java/org/apache/cassandra/io/compress/ICompressor.java -- diff --git a/src/java/org/apache/cassandra/io/compress/ICompressor.java b/src/java/org/apache/cassandra/io/compress/ICompressor.java index 5719834..40dc7c2 100644 --- a/src/java/org/apache/cassandra/io/compress/ICompressor.java +++ b/src/java/org/apache/cassandra/io/compress/ICompressor.java @@ -49,7 +49,7 @@ public interface ICompressor public BufferType preferredBufferType(); /** - * Checks if the given buffer would be supported by the compressor. If a type is supported the compressor must be + * Checks if the given buffer would be supported by the compressor. If a type is supported, the compressor must be * able to use it in combination with all other supported types. * * Direct and memory-mapped buffers must be supported by all compressors.
[jira] [Commented] (CASSANDRA-11580) remove DatabaseDescriptor dependency from SegmentedFile
[ https://issues.apache.org/jira/browse/CASSANDRA-11580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350339#comment-15350339 ] Yuki Morishita commented on CASSANDRA-11580: Change and tests looks good to me, thanks! The patch needs some rebasing, will do before code freeze. > remove DatabaseDescriptor dependency from SegmentedFile > --- > > Key: CASSANDRA-11580 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11580 > Project: Cassandra > Issue Type: Sub-task >Reporter: Yuki Morishita >Assignee: Yuki Morishita > Fix For: 3.x > > > Several configurable parameters are pulled from {{DatabaseDescriptor}} from > {{SegmentedFile}} and its subclasses. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11579) remove DatabaseDescriptor dependency from SequentialWriter
[ https://issues.apache.org/jira/browse/CASSANDRA-11579?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuki Morishita updated CASSANDRA-11579: --- Resolution: Fixed Fix Version/s: 3.8 Status: Resolved (was: Ready to Commit) I had time to commit today, so here it is. Committed as {{fb221095cb2a18cf8f027a8a084700d606bb9ca3}}, thanks! > remove DatabaseDescriptor dependency from SequentialWriter > -- > > Key: CASSANDRA-11579 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11579 > Project: Cassandra > Issue Type: Sub-task >Reporter: Yuki Morishita >Assignee: Yuki Morishita >Priority: Minor > Fix For: 3.8 > > > {{SequentialWriter}} and its subclass is widely used in Cassandra, mainly > from SSTable. Removing dependency to {{DatabaseDescriptor}} improve > reusability of this class. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
cassandra git commit: Remove DatabaseDescriptor dependency from Sequentialwriter
Repository: cassandra Updated Branches: refs/heads/trunk 5bc2f0130 -> fb221095c Remove DatabaseDescriptor dependency from Sequentialwriter patch by yukim; reviewed by snazy for CASSANDRA-11579 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/fb221095 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/fb221095 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/fb221095 Branch: refs/heads/trunk Commit: fb221095cb2a18cf8f027a8a084700d606bb9ca3 Parents: 5bc2f01 Author: Yuki MorishitaAuthored: Mon Jun 27 10:51:10 2016 +0900 Committer: Yuki Morishita Committed: Mon Jun 27 10:55:47 2016 +0900 -- CHANGES.txt | 1 + .../apache/cassandra/cache/AutoSavingCache.java | 7 +- .../index/sasi/disk/OnDiskIndexBuilder.java | 7 +- .../io/compress/CompressedSequentialWriter.java | 39 +++-- .../io/sstable/format/big/BigTableWriter.java | 62 .../cassandra/io/util/ChecksumWriter.java | 103 + .../io/util/ChecksummedSequentialWriter.java| 24 +-- .../io/util/DataIntegrityMetadata.java | 86 --- .../cassandra/io/util/SequentialWriter.java | 82 -- .../io/util/SequentialWriterOption.java | 154 +++ .../apache/cassandra/db/RowIndexEntryTest.java | 14 +- .../hints/ChecksummedDataInputTest.java | 6 +- .../index/sasi/disk/TokenTreeTest.java | 16 +- .../CompressedRandomAccessReaderTest.java | 21 +-- .../CompressedSequentialWriterTest.java | 58 ++- .../cassandra/io/sstable/DescriptorTest.java| 5 +- .../io/util/BufferedRandomAccessFileTest.java | 8 +- .../util/ChecksummedRandomAccessReaderTest.java | 28 ++-- .../util/ChecksummedSequentialWriterTest.java | 4 +- .../cassandra/io/util/DataOutputTest.java | 6 +- .../cassandra/io/util/MmappedRegionsTest.java | 9 +- .../io/util/RandomAccessReaderTest.java | 8 +- .../cassandra/io/util/SequentialWriterTest.java | 50 +- .../compression/CompressedInputStreamTest.java | 9 +- 24 files changed, 539 insertions(+), 268 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/fb221095/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index d40cab4..5b72016 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.8 + * Remove DatabaseDescriptor dependencies from SequentialWriter (CASSANDRA-11579) * Move skip_stop_words filter before stemming (CASSANDRA-12078) * Support seek() in EncryptedFileSegmentInputStream (CASSANDRA-11957) * SSTable tools mishandling LocalPartitioner (CASSANDRA-12002) http://git-wip-us.apache.org/repos/asf/cassandra/blob/fb221095/src/java/org/apache/cassandra/cache/AutoSavingCache.java -- diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java index 1b48d4f..cb2ad8a 100644 --- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java +++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java @@ -84,6 +84,11 @@ public class AutoSavingCache extends InstrumentingCache extends InstrumentingCachehttp://git-wip-us.apache.org/repos/asf/cassandra/blob/fb221095/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java -- diff --git a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java index 8acbb05..4946f06 100644 --- a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java +++ b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java @@ -31,7 +31,6 @@ import org.apache.cassandra.index.sasi.sa.TermIterator; import org.apache.cassandra.index.sasi.sa.SuffixSA; import org.apache.cassandra.db.marshal.*; import org.apache.cassandra.io.FSWriteError; -import org.apache.cassandra.io.compress.BufferType; import org.apache.cassandra.io.util.*; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; @@ -131,6 +130,10 @@ public class OnDiskIndexBuilder public static final int SUPER_BLOCK_SIZE = 64; public static final int IS_PARTIAL_BIT = 15; +private static final SequentialWriterOption WRITER_OPTION = SequentialWriterOption.newBuilder() + .bufferSize(BLOCK_SIZE) + .build(); + private final List
[jira] [Commented] (CASSANDRA-11393) dtest failure in upgrade_tests.upgrade_through_versions_test.ProtoV3Upgrade_2_1_UpTo_3_0_HEAD.rolling_upgrade_test
[ https://issues.apache.org/jira/browse/CASSANDRA-11393?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350332#comment-15350332 ] Stefania commented on CASSANDRA-11393: -- [~rhatch], if you encounter any "LEAK DETECTED" errors in your tests, we have CASSANDRA-11767 open for them. Would it be possible to run the tests with {{-Dcassandra.debugrefcount=true}}, so we avoid repeating the tests twice? > dtest failure in > upgrade_tests.upgrade_through_versions_test.ProtoV3Upgrade_2_1_UpTo_3_0_HEAD.rolling_upgrade_test > -- > > Key: CASSANDRA-11393 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11393 > Project: Cassandra > Issue Type: Bug > Components: Coordination, Streaming and Messaging >Reporter: Philip Thompson >Assignee: Benjamin Lerer > Labels: dtest > Fix For: 3.0.x, 3.x > > Attachments: 11393-3.0.txt > > > We are seeing a failure in the upgrade tests that go from 2.1 to 3.0 > {code} > node2: ERROR [SharedPool-Worker-2] 2016-03-10 20:05:17,865 Message.java:611 - > Unexpected exception during request; channel = [id: 0xeb79b477, > /127.0.0.1:39613 => /127.0.0.2:9042] > java.lang.AssertionError: null > at > org.apache.cassandra.db.ReadCommand$LegacyReadCommandSerializer.serializedSize(ReadCommand.java:1208) > ~[main/:na] > at > org.apache.cassandra.db.ReadCommand$LegacyReadCommandSerializer.serializedSize(ReadCommand.java:1155) > ~[main/:na] > at org.apache.cassandra.net.MessageOut.payloadSize(MessageOut.java:166) > ~[main/:na] > at > org.apache.cassandra.net.OutboundTcpConnectionPool.getConnection(OutboundTcpConnectionPool.java:72) > ~[main/:na] > at > org.apache.cassandra.net.MessagingService.getConnection(MessagingService.java:609) > ~[main/:na] > at > org.apache.cassandra.net.MessagingService.sendOneWay(MessagingService.java:758) > ~[main/:na] > at > org.apache.cassandra.net.MessagingService.sendRR(MessagingService.java:701) > ~[main/:na] > at > org.apache.cassandra.net.MessagingService.sendRRWithFailure(MessagingService.java:684) > ~[main/:na] > at > org.apache.cassandra.service.AbstractReadExecutor.makeRequests(AbstractReadExecutor.java:110) > ~[main/:na] > at > org.apache.cassandra.service.AbstractReadExecutor.makeDataRequests(AbstractReadExecutor.java:85) > ~[main/:na] > at > org.apache.cassandra.service.AbstractReadExecutor$AlwaysSpeculatingReadExecutor.executeAsync(AbstractReadExecutor.java:330) > ~[main/:na] > at > org.apache.cassandra.service.StorageProxy$SinglePartitionReadLifecycle.doInitialQueries(StorageProxy.java:1699) > ~[main/:na] > at > org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:1654) > ~[main/:na] > at > org.apache.cassandra.service.StorageProxy.readRegular(StorageProxy.java:1601) > ~[main/:na] > at > org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:1520) > ~[main/:na] > at > org.apache.cassandra.db.SinglePartitionReadCommand.execute(SinglePartitionReadCommand.java:302) > ~[main/:na] > at > org.apache.cassandra.service.pager.AbstractQueryPager.fetchPage(AbstractQueryPager.java:67) > ~[main/:na] > at > org.apache.cassandra.service.pager.SinglePartitionPager.fetchPage(SinglePartitionPager.java:34) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.SelectStatement$Pager$NormalPager.fetchPage(SelectStatement.java:297) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:333) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:209) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:76) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:472) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:449) > ~[main/:na] > at > org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:130) > ~[main/:na] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:507) > [main/:na] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:401) > [main/:na] > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > [netty-all-4.0.23.Final.jar:4.0.23.Final] > at >
cassandra git commit: move 'resource' warning suppression to where it's needed
Repository: cassandra Updated Branches: refs/heads/trunk 79a2875a7 -> 5bc2f0130 move 'resource' warning suppression to where it's needed Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/5bc2f013 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5bc2f013 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5bc2f013 Branch: refs/heads/trunk Commit: 5bc2f0130986b98beca40e4e18a27d2803248725 Parents: 79a2875 Author: Dave BrosiusAuthored: Sun Jun 26 20:22:35 2016 -0400 Committer: Dave Brosius Committed: Sun Jun 26 20:22:35 2016 -0400 -- src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/5bc2f013/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java -- diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java index 6110afe..1f4fdc2 100644 --- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java +++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java @@ -85,6 +85,7 @@ public class IndexSummaryBuilder implements AutoCloseable } } +@SuppressWarnings("resource") public IndexSummaryBuilder(long expectedKeys, int minIndexInterval, int samplingLevel) { this.samplingLevel = samplingLevel; @@ -281,7 +282,6 @@ public class IndexSummaryBuilder implements AutoCloseable * @param partitioner the partitioner used for the index summary * @return a new IndexSummary */ -@SuppressWarnings("resource") public static IndexSummary downsample(IndexSummary existing, int newSamplingLevel, int minIndexInterval, IPartitioner partitioner) { // To downsample the old index summary, we'll go through (potentially) several rounds of downsampling.
[jira] [Commented] (CASSANDRA-10769) "received out of order wrt DecoratedKey" after scrub
[ https://issues.apache.org/jira/browse/CASSANDRA-10769?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350282#comment-15350282 ] Paulo Motta commented on CASSANDRA-10769: - I suspect this is a duplicate of CASSANDRA-9935, fixed in upcoming 2.1.15. > "received out of order wrt DecoratedKey" after scrub > > > Key: CASSANDRA-10769 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10769 > Project: Cassandra > Issue Type: Bug > Environment: C* 2.1.11, Debian Wheezy >Reporter: mlowicki > > After running scrub and cleanup on all nodes in single data center I'm > getting: > {code} > ERROR [ValidationExecutor:103] 2015-11-25 06:28:21,530 Validator.java:245 - > Failed creating a merkle tree for [repair > #89fa2b70-933d-11e5-b036-75bb514ae072 on sync/entity_by_id2, > (-5867793819051725444,-5865919628027816979]], /10.210.3.221 (see log for > details) > ERROR [ValidationExecutor:103] 2015-11-25 06:28:21,531 > CassandraDaemon.java:227 - Exception in thread > Thread[ValidationExecutor:103,1,main] > java.lang.AssertionError: row DecoratedKey(-5867787467868737053, > 000932373633313036313204808800) received out of order wrt > DecoratedKey(-5865937851627253360, 000933313230313737333204c3c700) > at org.apache.cassandra.repair.Validator.add(Validator.java:127) > ~[apache-cassandra-2.1.11.jar:2.1.11] > at > org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1010) > ~[apache-cassandra-2.1.11.jar:2.1.11] > at > org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:94) > ~[apache-cassandra-2.1.11.jar:2.1.11] > at > org.apache.cassandra.db.compaction.CompactionManager$9.call(CompactionManager.java:622) > ~[apache-cassandra-2.1.11.jar:2.1.11] > at java.util.concurrent.FutureTask.run(FutureTask.java:262) > ~[na:1.7.0_80] > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > ~[na:1.7.0_80] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > [na:1.7.0_80] > at java.lang.Thread.run(Thread.java:745) [na:1.7.0_80] > {code} > What I did is to run repair on other node: > {code} > time nodetool repair --in-local-dc > {code} > Corresponding log on the node where repair has been started: > {code} > ERROR [AntiEntropySessions:414] 2015-11-25 06:28:21,533 > RepairSession.java:303 - [repair #89fa2b70-933d-11e5-b036-75bb514ae072] > session completed with the following error > org.apache.cassandra.exceptions.RepairException: [repair > #89fa2b70-933d-11e5-b036-75bb514ae072 on sync/entity_by_id2, > (-5867793819051725444,-5865919628027816979]] Validation failed in > /10.210.3.117 > at > org.apache.cassandra.repair.RepairSession.validationComplete(RepairSession.java:166) > ~[apache-cassandra-2.1.11.jar:2.1.11] > at > org.apache.cassandra.service.ActiveRepairService.handleMessage(ActiveRepairService.java:406) > ~[apache-cassandra-2.1.11.jar:2.1.11] > at > org.apache.cassandra.repair.RepairMessageVerbHandler.doVerb(RepairMessageVerbHandler.java:134) > ~[apache-cassandra-2.1.11.jar:2.1.11] > at > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:64) > ~[apache-cassandra-2.1.11.jar:2.1.11] > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > [na:1.7.0_80] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > [na:1.7.0_80] > at java.lang.Thread.run(Thread.java:745) [na:1.7.0_80] > INFO [AntiEntropySessions:415] 2015-11-25 06:28:21,533 > RepairSession.java:260 - [repair #b9458fa0-933d-11e5-b036-75bb514ae072] new > session: will sync /10.210.3.221, /10.210.3.118, /10.210.3.117 on range > (7119703141488009983,7129744584776466802] for sync.[device_token, entity2, > user_stats, user_device, user_quota, user_store, user_device_progress, > entity_by_id2] > ERROR [AntiEntropySessions:414] 2015-11-25 06:28:21,533 > CassandraDaemon.java:227 - Exception in thread > Thread[AntiEntropySessions:414,5,RMI Runtime] > java.lang.RuntimeException: org.apache.cassandra.exceptions.RepairException: > [repair #89fa2b70-933d-11e5-b036-75bb514ae072 on sync/entity_by_id2, > (-5867793819051725444,-5865919628027816979]] Validation failed in > /10.210.3.117 > at com.google.common.base.Throwables.propagate(Throwables.java:160) > ~[guava-16.0.jar:na] > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) > ~[apache-cassandra-2.1.11.jar:2.1.11] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) > ~[na:1.7.0_80] > at java.util.concurrent.FutureTask.run(FutureTask.java:262)
cassandra git commit: close streams
Repository: cassandra Updated Branches: refs/heads/trunk eb82861c8 -> 79a2875a7 close streams Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/79a2875a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/79a2875a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/79a2875a Branch: refs/heads/trunk Commit: 79a2875a70efc98f2cda32dcb7c439d07dc7480d Parents: eb82861 Author: Dave BrosiusAuthored: Sun Jun 26 14:40:45 2016 -0400 Committer: Dave Brosius Committed: Sun Jun 26 14:41:12 2016 -0400 -- src/java/org/apache/cassandra/utils/HeapUtils.java | 15 --- 1 file changed, 8 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/79a2875a/src/java/org/apache/cassandra/utils/HeapUtils.java -- diff --git a/src/java/org/apache/cassandra/utils/HeapUtils.java b/src/java/org/apache/cassandra/utils/HeapUtils.java index bfc8a0b..65364d8 100644 --- a/src/java/org/apache/cassandra/utils/HeapUtils.java +++ b/src/java/org/apache/cassandra/utils/HeapUtils.java @@ -125,15 +125,16 @@ public final class HeapUtils */ private static void logProcessOutput(Process p) throws IOException { -BufferedReader input = new BufferedReader(new InputStreamReader(p.getInputStream())); - -StrBuilder builder = new StrBuilder(); -String line; -while ((line = input.readLine()) != null) +try (BufferedReader input = new BufferedReader(new InputStreamReader(p.getInputStream( { -builder.appendln(line); +StrBuilder builder = new StrBuilder(); +String line; +while ((line = input.readLine()) != null) +{ +builder.appendln(line); +} +logger.info(builder.toString()); } -logger.info(builder.toString()); } /**
[jira] [Commented] (CASSANDRA-12073) [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial results
[ https://issues.apache.org/jira/browse/CASSANDRA-12073?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350182#comment-15350182 ] Jeremiah Jordan commented on CASSANDRA-12073: - We don't set the fix version to a specific version until the patch has been applied. Thanks. > [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial > results > --- > > Key: CASSANDRA-12073 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12073 > Project: Cassandra > Issue Type: Bug > Components: CQL > Environment: Cassandra 3.7 >Reporter: DOAN DuyHai >Assignee: DOAN DuyHai > Fix For: 3.x > > Attachments: patch_PREFIX_search_with_CONTAINS_mode.txt > > > {noformat} > cqlsh:music> CREATE TABLE music.albums ( > id uuid PRIMARY KEY, > artist text, > country text, > quality text, > status text, > title text, > year int > ); > cqlsh:music> CREATE CUSTOM INDEX albums_artist_idx ON music.albums (artist) > USING 'org.apache.cassandra.index.sasi.SASIIndex' WITH OPTIONS = {'mode': > 'CONTAINS', 'analyzer_class': > 'org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer', > 'case_sensitive': 'false'}; > cqlsh:music> SELECT * FROM albums WHERE artist like 'lady%' LIMIT 100; > id | artist| country| quality > | status| title | year > --+---++-+---+---+-- > 372bb0ab-3263-41bc-baad-bb520ddfa787 | Lady Gaga |USA | normal > | Official | Red and Blue EP | 2006 > 1a4abbcd-b5de-4c69-a578-31231e01ff09 | Lady Gaga |Unknown | normal > | Promotion |Poker Face | 2008 > 31f4a0dc-9efc-48bf-9f5e-bfc09af42b82 | Lady Gaga |USA | normal > | Official | The Cherrytree Sessions | 2009 > 8ebfaebd-28d0-477d-b735-469661ce6873 | Lady Gaga |Unknown | normal > | Official |Poker Face | 2009 > 98107d82-e0dd-46bc-a273-1577578984c7 | Lady Gaga |USA | normal > | Official | Just Dance: The Remixes | 2008 > a76af0f2-f5c5-4306-974a-e3c17158e6c6 | Lady Gaga | Italy | normal > | Official | The Fame | 2008 > 849ee019-8b15-4767-8660-537ab9710459 | Lady Gaga |USA | normal > | Official |Christmas Tree | 2008 > 4bad59ac-913f-43da-9d48-89adc65453d2 | Lady Gaga | Australia | normal > | Official | Eh Eh | 2009 > 80327731-c450-457f-bc12-0a8c21fd9c5d | Lady Gaga |USA | normal > | Official | Just Dance Remixes Part 2 | 2008 > 3ad33659-e932-4d31-a040-acab0e23c3d4 | Lady Gaga |Unknown | normal > | null |Just Dance | 2008 > 9adce7f6-6a1d-49fd-b8bd-8f6fac73558b | Lady Gaga | United Kingdom | normal > | Official |Just Dance | 2009 > (11 rows) > {noformat} > *SASI* says that there are only 11 artists whose name starts with {{lady}}. > However, in the data set, there are: > * Lady Pank > * Lady Saw > * Lady Saw > * Ladyhawke > * Ladytron > * Ladysmith Black Mambazo > * Lady Gaga > * Lady Sovereign > etc ... > By debugging the source code, the issue is in > {{OnDiskIndex.TermIterator::computeNext()}} > {code:java} > for (;;) > { > if (currentBlock == null) > return endOfData(); > if (offset >= 0 && offset < currentBlock.termCount()) > { > DataTerm currentTerm = currentBlock.getTerm(nextOffset()); > if (checkLower && !e.isLowerSatisfiedBy(currentTerm)) > continue; > // flip the flag right on the first bounds match > // to avoid expensive comparisons > checkLower = false; > if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) > return endOfData(); > return currentTerm; > } > nextBlock(); > } > {code} > So the {{endOfData()}} conditions are: > * currentBlock == null > * checkUpper && !e.isUpperSatisfiedBy(currentTerm) > The problem is that {{e::isUpperSatisfiedBy}} is checking not only whether > the term match but also returns *false* when it's a *partial term* ! > {code:java} > public boolean isUpperSatisfiedBy(OnDiskIndex.DataTerm term) > { > if (!hasUpper()) > return true; > if (nonMatchingPartial(term)) > return false; > int cmp = term.compareTo(validator, upper.value, false); > return cmp < 0 || cmp == 0 && upper.inclusive; > } > {code} > By debugging
[jira] [Updated] (CASSANDRA-12073) [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial results
[ https://issues.apache.org/jira/browse/CASSANDRA-12073?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeremiah Jordan updated CASSANDRA-12073: Fix Version/s: (was: 3.8) 3.x > [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial > results > --- > > Key: CASSANDRA-12073 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12073 > Project: Cassandra > Issue Type: Bug > Components: CQL > Environment: Cassandra 3.7 >Reporter: DOAN DuyHai >Assignee: DOAN DuyHai > Fix For: 3.x > > Attachments: patch_PREFIX_search_with_CONTAINS_mode.txt > > > {noformat} > cqlsh:music> CREATE TABLE music.albums ( > id uuid PRIMARY KEY, > artist text, > country text, > quality text, > status text, > title text, > year int > ); > cqlsh:music> CREATE CUSTOM INDEX albums_artist_idx ON music.albums (artist) > USING 'org.apache.cassandra.index.sasi.SASIIndex' WITH OPTIONS = {'mode': > 'CONTAINS', 'analyzer_class': > 'org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer', > 'case_sensitive': 'false'}; > cqlsh:music> SELECT * FROM albums WHERE artist like 'lady%' LIMIT 100; > id | artist| country| quality > | status| title | year > --+---++-+---+---+-- > 372bb0ab-3263-41bc-baad-bb520ddfa787 | Lady Gaga |USA | normal > | Official | Red and Blue EP | 2006 > 1a4abbcd-b5de-4c69-a578-31231e01ff09 | Lady Gaga |Unknown | normal > | Promotion |Poker Face | 2008 > 31f4a0dc-9efc-48bf-9f5e-bfc09af42b82 | Lady Gaga |USA | normal > | Official | The Cherrytree Sessions | 2009 > 8ebfaebd-28d0-477d-b735-469661ce6873 | Lady Gaga |Unknown | normal > | Official |Poker Face | 2009 > 98107d82-e0dd-46bc-a273-1577578984c7 | Lady Gaga |USA | normal > | Official | Just Dance: The Remixes | 2008 > a76af0f2-f5c5-4306-974a-e3c17158e6c6 | Lady Gaga | Italy | normal > | Official | The Fame | 2008 > 849ee019-8b15-4767-8660-537ab9710459 | Lady Gaga |USA | normal > | Official |Christmas Tree | 2008 > 4bad59ac-913f-43da-9d48-89adc65453d2 | Lady Gaga | Australia | normal > | Official | Eh Eh | 2009 > 80327731-c450-457f-bc12-0a8c21fd9c5d | Lady Gaga |USA | normal > | Official | Just Dance Remixes Part 2 | 2008 > 3ad33659-e932-4d31-a040-acab0e23c3d4 | Lady Gaga |Unknown | normal > | null |Just Dance | 2008 > 9adce7f6-6a1d-49fd-b8bd-8f6fac73558b | Lady Gaga | United Kingdom | normal > | Official |Just Dance | 2009 > (11 rows) > {noformat} > *SASI* says that there are only 11 artists whose name starts with {{lady}}. > However, in the data set, there are: > * Lady Pank > * Lady Saw > * Lady Saw > * Ladyhawke > * Ladytron > * Ladysmith Black Mambazo > * Lady Gaga > * Lady Sovereign > etc ... > By debugging the source code, the issue is in > {{OnDiskIndex.TermIterator::computeNext()}} > {code:java} > for (;;) > { > if (currentBlock == null) > return endOfData(); > if (offset >= 0 && offset < currentBlock.termCount()) > { > DataTerm currentTerm = currentBlock.getTerm(nextOffset()); > if (checkLower && !e.isLowerSatisfiedBy(currentTerm)) > continue; > // flip the flag right on the first bounds match > // to avoid expensive comparisons > checkLower = false; > if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) > return endOfData(); > return currentTerm; > } > nextBlock(); > } > {code} > So the {{endOfData()}} conditions are: > * currentBlock == null > * checkUpper && !e.isUpperSatisfiedBy(currentTerm) > The problem is that {{e::isUpperSatisfiedBy}} is checking not only whether > the term match but also returns *false* when it's a *partial term* ! > {code:java} > public boolean isUpperSatisfiedBy(OnDiskIndex.DataTerm term) > { > if (!hasUpper()) > return true; > if (nonMatchingPartial(term)) > return false; > int cmp = term.compareTo(validator, upper.value, false); > return cmp < 0 || cmp == 0 && upper.inclusive; > } > {code} > By debugging the OnDiskIndex data, I've found: > {noformat} > ... > Data Term (partial ?
[jira] [Updated] (CASSANDRA-12073) [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial results
[ https://issues.apache.org/jira/browse/CASSANDRA-12073?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] DOAN DuyHai updated CASSANDRA-12073: Attachment: patch_PREFIX_search_with_CONTAINS_mode.txt Patch attached > [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial > results > --- > > Key: CASSANDRA-12073 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12073 > Project: Cassandra > Issue Type: Bug > Components: CQL > Environment: Cassandra 3.7 >Reporter: DOAN DuyHai >Assignee: DOAN DuyHai > Fix For: 3.8 > > Attachments: patch_PREFIX_search_with_CONTAINS_mode.txt > > > {noformat} > cqlsh:music> CREATE TABLE music.albums ( > id uuid PRIMARY KEY, > artist text, > country text, > quality text, > status text, > title text, > year int > ); > cqlsh:music> CREATE CUSTOM INDEX albums_artist_idx ON music.albums (artist) > USING 'org.apache.cassandra.index.sasi.SASIIndex' WITH OPTIONS = {'mode': > 'CONTAINS', 'analyzer_class': > 'org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer', > 'case_sensitive': 'false'}; > cqlsh:music> SELECT * FROM albums WHERE artist like 'lady%' LIMIT 100; > id | artist| country| quality > | status| title | year > --+---++-+---+---+-- > 372bb0ab-3263-41bc-baad-bb520ddfa787 | Lady Gaga |USA | normal > | Official | Red and Blue EP | 2006 > 1a4abbcd-b5de-4c69-a578-31231e01ff09 | Lady Gaga |Unknown | normal > | Promotion |Poker Face | 2008 > 31f4a0dc-9efc-48bf-9f5e-bfc09af42b82 | Lady Gaga |USA | normal > | Official | The Cherrytree Sessions | 2009 > 8ebfaebd-28d0-477d-b735-469661ce6873 | Lady Gaga |Unknown | normal > | Official |Poker Face | 2009 > 98107d82-e0dd-46bc-a273-1577578984c7 | Lady Gaga |USA | normal > | Official | Just Dance: The Remixes | 2008 > a76af0f2-f5c5-4306-974a-e3c17158e6c6 | Lady Gaga | Italy | normal > | Official | The Fame | 2008 > 849ee019-8b15-4767-8660-537ab9710459 | Lady Gaga |USA | normal > | Official |Christmas Tree | 2008 > 4bad59ac-913f-43da-9d48-89adc65453d2 | Lady Gaga | Australia | normal > | Official | Eh Eh | 2009 > 80327731-c450-457f-bc12-0a8c21fd9c5d | Lady Gaga |USA | normal > | Official | Just Dance Remixes Part 2 | 2008 > 3ad33659-e932-4d31-a040-acab0e23c3d4 | Lady Gaga |Unknown | normal > | null |Just Dance | 2008 > 9adce7f6-6a1d-49fd-b8bd-8f6fac73558b | Lady Gaga | United Kingdom | normal > | Official |Just Dance | 2009 > (11 rows) > {noformat} > *SASI* says that there are only 11 artists whose name starts with {{lady}}. > However, in the data set, there are: > * Lady Pank > * Lady Saw > * Lady Saw > * Ladyhawke > * Ladytron > * Ladysmith Black Mambazo > * Lady Gaga > * Lady Sovereign > etc ... > By debugging the source code, the issue is in > {{OnDiskIndex.TermIterator::computeNext()}} > {code:java} > for (;;) > { > if (currentBlock == null) > return endOfData(); > if (offset >= 0 && offset < currentBlock.termCount()) > { > DataTerm currentTerm = currentBlock.getTerm(nextOffset()); > if (checkLower && !e.isLowerSatisfiedBy(currentTerm)) > continue; > // flip the flag right on the first bounds match > // to avoid expensive comparisons > checkLower = false; > if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) > return endOfData(); > return currentTerm; > } > nextBlock(); > } > {code} > So the {{endOfData()}} conditions are: > * currentBlock == null > * checkUpper && !e.isUpperSatisfiedBy(currentTerm) > The problem is that {{e::isUpperSatisfiedBy}} is checking not only whether > the term match but also returns *false* when it's a *partial term* ! > {code:java} > public boolean isUpperSatisfiedBy(OnDiskIndex.DataTerm term) > { > if (!hasUpper()) > return true; > if (nonMatchingPartial(term)) > return false; > int cmp = term.compareTo(validator, upper.value, false); > return cmp < 0 || cmp == 0 && upper.inclusive; > } > {code} > By debugging the OnDiskIndex data, I've found: > {noformat} > ... > Data Term
[jira] [Updated] (CASSANDRA-12073) [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial results
[ https://issues.apache.org/jira/browse/CASSANDRA-12073?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] DOAN DuyHai updated CASSANDRA-12073: Fix Version/s: (was: 3.x) 3.8 Reproduced In: 3.7, 3.8 (was: 3.7) Status: Patch Available (was: Open) > [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial > results > --- > > Key: CASSANDRA-12073 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12073 > Project: Cassandra > Issue Type: Bug > Components: CQL > Environment: Cassandra 3.7 >Reporter: DOAN DuyHai >Assignee: DOAN DuyHai > Fix For: 3.8 > > > {noformat} > cqlsh:music> CREATE TABLE music.albums ( > id uuid PRIMARY KEY, > artist text, > country text, > quality text, > status text, > title text, > year int > ); > cqlsh:music> CREATE CUSTOM INDEX albums_artist_idx ON music.albums (artist) > USING 'org.apache.cassandra.index.sasi.SASIIndex' WITH OPTIONS = {'mode': > 'CONTAINS', 'analyzer_class': > 'org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer', > 'case_sensitive': 'false'}; > cqlsh:music> SELECT * FROM albums WHERE artist like 'lady%' LIMIT 100; > id | artist| country| quality > | status| title | year > --+---++-+---+---+-- > 372bb0ab-3263-41bc-baad-bb520ddfa787 | Lady Gaga |USA | normal > | Official | Red and Blue EP | 2006 > 1a4abbcd-b5de-4c69-a578-31231e01ff09 | Lady Gaga |Unknown | normal > | Promotion |Poker Face | 2008 > 31f4a0dc-9efc-48bf-9f5e-bfc09af42b82 | Lady Gaga |USA | normal > | Official | The Cherrytree Sessions | 2009 > 8ebfaebd-28d0-477d-b735-469661ce6873 | Lady Gaga |Unknown | normal > | Official |Poker Face | 2009 > 98107d82-e0dd-46bc-a273-1577578984c7 | Lady Gaga |USA | normal > | Official | Just Dance: The Remixes | 2008 > a76af0f2-f5c5-4306-974a-e3c17158e6c6 | Lady Gaga | Italy | normal > | Official | The Fame | 2008 > 849ee019-8b15-4767-8660-537ab9710459 | Lady Gaga |USA | normal > | Official |Christmas Tree | 2008 > 4bad59ac-913f-43da-9d48-89adc65453d2 | Lady Gaga | Australia | normal > | Official | Eh Eh | 2009 > 80327731-c450-457f-bc12-0a8c21fd9c5d | Lady Gaga |USA | normal > | Official | Just Dance Remixes Part 2 | 2008 > 3ad33659-e932-4d31-a040-acab0e23c3d4 | Lady Gaga |Unknown | normal > | null |Just Dance | 2008 > 9adce7f6-6a1d-49fd-b8bd-8f6fac73558b | Lady Gaga | United Kingdom | normal > | Official |Just Dance | 2009 > (11 rows) > {noformat} > *SASI* says that there are only 11 artists whose name starts with {{lady}}. > However, in the data set, there are: > * Lady Pank > * Lady Saw > * Lady Saw > * Ladyhawke > * Ladytron > * Ladysmith Black Mambazo > * Lady Gaga > * Lady Sovereign > etc ... > By debugging the source code, the issue is in > {{OnDiskIndex.TermIterator::computeNext()}} > {code:java} > for (;;) > { > if (currentBlock == null) > return endOfData(); > if (offset >= 0 && offset < currentBlock.termCount()) > { > DataTerm currentTerm = currentBlock.getTerm(nextOffset()); > if (checkLower && !e.isLowerSatisfiedBy(currentTerm)) > continue; > // flip the flag right on the first bounds match > // to avoid expensive comparisons > checkLower = false; > if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) > return endOfData(); > return currentTerm; > } > nextBlock(); > } > {code} > So the {{endOfData()}} conditions are: > * currentBlock == null > * checkUpper && !e.isUpperSatisfiedBy(currentTerm) > The problem is that {{e::isUpperSatisfiedBy}} is checking not only whether > the term match but also returns *false* when it's a *partial term* ! > {code:java} > public boolean isUpperSatisfiedBy(OnDiskIndex.DataTerm term) > { > if (!hasUpper()) > return true; > if (nonMatchingPartial(term)) > return false; > int cmp = term.compareTo(validator, upper.value, false); > return cmp < 0 || cmp == 0 && upper.inclusive; > } > {code} > By debugging the OnDiskIndex data, I've found: > {noformat} > ... > Data Term
[jira] [Comment Edited] (CASSANDRA-11990) Address rows rather than partitions in SASI
[ https://issues.apache.org/jira/browse/CASSANDRA-11990?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350152#comment-15350152 ] Alex Petrov edited comment on CASSANDRA-11990 at 6/26/16 3:21 PM: -- I've done a small investigation of what it'd take and talked to several people about potential scenarios. First of all, I'd indicate that it'll be a rather big change, which will require a new format for writing the SASI {{TokenTree}}. I'll list several steps that would need to be taken: * tl;dr version: we have to extend TokenTree to fit row offset along with partition key. More elaborate version: currently SASI is a highly optimised tree that aims to encode a tree of {{long token}}/{{short + int}} entries. Since the max offset size does not exceed 48 bytes, there are more optimisations involved. It takes several optimisation steps to improve read performance and storage overhead. Short description of current format can be found [here|https://gist.github.com/ifesdjeen/0436faf9a66b401ace0ad0947d256317]. Since we'll have to hold two offsets (partition and row offset, Partition offset is required to read the PK and static rows etc), on the first step, for the proof of concept, we'll reduce the number of distinction to more simple cases (single and multiple offsets). The rest of possible combinations of optimisation (with the most obvious being when both items fit into the single long, and possibly adding more distinctions if they're flexibly skippable/addressable). * currently, we can only read the decorated partition key, so we need to extend storage to address a single row as well * we need to extend TokenTree to support other partitioners (whether or not it's going to be done in scope of this ticket, we'll have to make sure we're not making it harder to extend it this way. * there might be a need to store the order-preserving hash of clustering keys for queries where row is split across multiple SSTables, although I have to gather more information on that one, as we might be able to resolve rows after reading them from sstables. * we'll need to find migration/upgrade paths from current format, which may involve re-indexing and failing queries while upgrade is in process or supporting two format versions at read time, to support reads from old format while indexes are rebuilt. cc [~xedin] [~beobal] [~jrwest] was (Author: ifesdjeen): I've done a small investigation of what it'd take and talked to several people about potential scenarios. First of all, I'd indicate that it'll be a rather big change, which will require a new format for writing the SASI {{TokenTree}}. I'll list several steps that would need to be taken: * tl;dr version: we have to extend TokenTree to fit row offset along with partition key. More elaborate version: currently SASI is a highly optimised tree that aims to encode a tree of {{long token}}/{{short + int}} entries. Since the max offset size does not exceed 48 bytes, there are more optimisations involved. It takes several optimisation steps to improve read performance and storage overhead. Short description of current format can be found [here|https://gist.github.com/ifesdjeen/0436faf9a66b401ace0ad0947d256317]. Since we'll have to hold two offsets (partition and row offset, Partition offset is required to read the PK and static rows etc), on the first step, for the proof of concept, we'll reduce the number of distinction to more simple cases (single and multiple offsets). The rest of possible combinations of optimisation (with the most obvious being when both items fit into the single long, and possibly adding more distinctions if they're flexibly skippable/addressable). * we need to extend TokenTree to support other partitioners (whether or not it's going to be done in scope of this ticket, we'll have to make sure we're not making it harder to extend it this way. * there might be a need to store the order-preserving hash of clustering keys for queries where row is split across multiple SSTables, although I have to gather more information on that one, as we might be able to resolve rows after reading them from sstables. * we'll need to find migration/upgrade paths from current format, which may involve re-indexing and failing queries while upgrade is in process or supporting two format versions at read time, to support reads from old format while indexes are rebuilt. cc [~xedin] [~beobal] [~jrwest] > Address rows rather than partitions in SASI > --- > > Key: CASSANDRA-11990 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11990 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Alex Petrov >Assignee: Alex Petrov > > Currently, the lookup in SASI index
[jira] [Commented] (CASSANDRA-11990) Address rows rather than partitions in SASI
[ https://issues.apache.org/jira/browse/CASSANDRA-11990?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350152#comment-15350152 ] Alex Petrov commented on CASSANDRA-11990: - I've done a small investigation of what it'd take and talked to several people about potential scenarios. First of all, I'd indicate that it'll be a rather big change, which will require a new format for writing the SASI {{TokenTree}}. I'll list several steps that would need to be taken: * tl;dr version: we have to extend TokenTree to fit row offset along with partition key. More elaborate version: currently SASI is a highly optimised tree that aims to encode a tree of {{long token}}/{{short + int}} entries. Since the max offset size does not exceed 48 bytes, there are more optimisations involved. It takes several optimisation steps to improve read performance and storage overhead. Short description of current format can be found [here|https://gist.github.com/ifesdjeen/0436faf9a66b401ace0ad0947d256317]. Since we'll have to hold two offsets (partition and row offset, Partition offset is required to read the PK and static rows etc), on the first step, for the proof of concept, we'll reduce the number of distinction to more simple cases (single and multiple offsets). The rest of possible combinations of optimisation (with the most obvious being when both items fit into the single long, and possibly adding more distinctions if they're flexibly skippable/addressable). * we need to extend TokenTree to support other partitioners (whether or not it's going to be done in scope of this ticket, we'll have to make sure we're not making it harder to extend it this way. * there might be a need to store the order-preserving hash of clustering keys for queries where row is split across multiple SSTables, although I have to gather more information on that one, as we might be able to resolve rows after reading them from sstables. * we'll need to find migration/upgrade paths from current format, which may involve re-indexing and failing queries while upgrade is in process or supporting two format versions at read time, to support reads from old format while indexes are rebuilt. cc [~xedin] [~beobal] [~jrwest] > Address rows rather than partitions in SASI > --- > > Key: CASSANDRA-11990 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11990 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Alex Petrov >Assignee: Alex Petrov > > Currently, the lookup in SASI index would return the key position of the > partition. After the partition lookup, the rows are iterated and the > operators are applied in order to filter out ones that do not match. > bq. TokenTree which accepts variable size keys (such would enable different > partitioners, collections support, primary key indexing etc.), -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-12073) [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial results
[ https://issues.apache.org/jira/browse/CASSANDRA-12073?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350113#comment-15350113 ] DOAN DuyHai edited comment on CASSANDRA-12073 at 6/26/16 1:32 PM: -- Ok I've dug further into the code and the bug has wider implication 1) Premature termination by calling {{endOfData()}} as mentioned above in {code:java} if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) return endOfData(); {code} 2) Incorrect next block skipping in {{OnDiskIndex.nextBlock()}} {code:java} protected void nextBlock() { currentBlock = null; if (blockIndex < 0 || blockIndex >= dataLevel.blockCount) return; currentBlock = dataLevel.getBlock(nextBlockIndex()); offset = checkLower ? order.startAt(currentBlock, e) : currentBlock.minOffset(order); // let's check the last term of the new block right away // if expression's upper bound is satisfied by it such means that we can avoid // doing any expensive upper bound checks for that block. checkUpper = e.hasUpper() && !e.isUpperSatisfiedBy(currentBlock.getTerm(currentBlock.maxOffset(order))); } {code} Above, if we are unlucky and the last term of next block is partial, the entire next block will be skipped incorrectly Proposed change: * remove the {{if (nonMatchingPartial(term))}} check from current {{e::isLowerSatisfiedBy}} and {{e::isUpperSatisfiedBy}} * rename the method {{Expression::nonMatchingPartial}} to {{Expression::partialTermAndPrefixOperation}} which describes exactly which condition it checks, much clearer name from my pov. Also make the method access *public* * add an extra condition in {{OnDiskIndex::computeNext()}} to skip the current term and process the next one {code:java} if (offset >= 0 && offset < currentBlock.termCount()) { DataTerm currentTerm = currentBlock.getTerm(nextOffset()); if (e.partialTermAndPrefixOperation(currentTerm) || (checkLower && !e.isLowerSatisfiedBy(currentTerm))) continue; // flip the flag right on the first bounds match // to avoid expensive comparisons checkLower = false; if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) return endOfData(); return currentTerm; } {code} I'll also add an extra unit-test to cover the premature {{endOfData()}} case. To test the incorrect next block skipping we'll need to generate enough data so that the last term of next block is partial. I'm not sure how feasible it is. What do you think @xedin, [~jrwest] ? was (Author: doanduyhai): Ok I've dug further into the code and the bug has wider implication 1) Premature termination by calling {{endOfData()}} as mentioned above in {code:java} if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) return endOfData(); {code} 2) Incorrect next block skipping in {{OnDiskIndex.nextBlock()}} {code:java} protected void nextBlock() { currentBlock = null; if (blockIndex < 0 || blockIndex >= dataLevel.blockCount) return; currentBlock = dataLevel.getBlock(nextBlockIndex()); offset = checkLower ? order.startAt(currentBlock, e) : currentBlock.minOffset(order); // let's check the last term of the new block right away // if expression's upper bound is satisfied by it such means that we can avoid // doing any expensive upper bound checks for that block. checkUpper = e.hasUpper() && !e.isUpperSatisfiedBy(currentBlock.getTerm(currentBlock.maxOffset(order))); } {code} Above, if we are unlucky and the last term of next block is partial, the entire next block will be skipped Proposed change: * remove the {{if (nonMatchingPartial(term))}} check from current {{e::isLowerSatisfiedBy}} and {{e::isUpperSatisfiedBy}} * rename the method {{Expression::nonMatchingPartial}} to {{Expression::partialTermAndPrefixOperation}} which describes exactly which condition it checks, much clearer name from my pov. Also make the method access *public* * add an extra condition in {{OnDiskIndex::computeNext()}} to skip the current term and process the next one {code:java} if (offset >= 0 && offset < currentBlock.termCount()) { DataTerm currentTerm = currentBlock.getTerm(nextOffset()); if (e.partialTermAndPrefixOperation(currentTerm) || (checkLower && !e.isLowerSatisfiedBy(currentTerm))) continue; // flip the flag right on the first bounds match // to avoid expensive comparisons checkLower = false; if (checkUpper && !e.isUpperSatisfiedBy(currentTerm))
[jira] [Comment Edited] (CASSANDRA-12073) [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial results
[ https://issues.apache.org/jira/browse/CASSANDRA-12073?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350113#comment-15350113 ] DOAN DuyHai edited comment on CASSANDRA-12073 at 6/26/16 1:33 PM: -- Ok I've dug further into the code and the bug has wider implication 1) Premature termination by calling {{endOfData()}} as mentioned above in {code:java} if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) return endOfData(); {code} 2) Incorrect next block skipping in {{OnDiskIndex.nextBlock()}} {code:java} protected void nextBlock() { currentBlock = null; if (blockIndex < 0 || blockIndex >= dataLevel.blockCount) return; currentBlock = dataLevel.getBlock(nextBlockIndex()); offset = checkLower ? order.startAt(currentBlock, e) : currentBlock.minOffset(order); // let's check the last term of the new block right away // if expression's upper bound is satisfied by it such means that we can avoid // doing any expensive upper bound checks for that block. checkUpper = e.hasUpper() && !e.isUpperSatisfiedBy(currentBlock.getTerm(currentBlock.maxOffset(order))); } {code} Above, if we are unlucky and the last term of next block is partial, the entire next block will be skipped incorrectly Proposed change: * remove the {{if (nonMatchingPartial(term))}} check from current {{Expression::isLowerSatisfiedBy}} and {{Expression::isUpperSatisfiedBy}} * rename the method {{Expression::nonMatchingPartial}} to {{Expression::partialTermAndPrefixOperation}} which describes exactly which condition it checks, much clearer name from my pov. Also make the method access *public* * add an extra condition in {{OnDiskIndex::computeNext()}} to skip the current term and process the next one {code:java} if (offset >= 0 && offset < currentBlock.termCount()) { DataTerm currentTerm = currentBlock.getTerm(nextOffset()); if (e.partialTermAndPrefixOperation(currentTerm) || (checkLower && !e.isLowerSatisfiedBy(currentTerm))) continue; // flip the flag right on the first bounds match // to avoid expensive comparisons checkLower = false; if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) return endOfData(); return currentTerm; } {code} I'll also add an extra unit-test to cover the premature {{endOfData()}} case. To test the incorrect next block skipping we'll need to generate enough data so that the last term of next block is partial. I'm not sure how feasible it is. What do you think @xedin, [~jrwest] ? was (Author: doanduyhai): Ok I've dug further into the code and the bug has wider implication 1) Premature termination by calling {{endOfData()}} as mentioned above in {code:java} if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) return endOfData(); {code} 2) Incorrect next block skipping in {{OnDiskIndex.nextBlock()}} {code:java} protected void nextBlock() { currentBlock = null; if (blockIndex < 0 || blockIndex >= dataLevel.blockCount) return; currentBlock = dataLevel.getBlock(nextBlockIndex()); offset = checkLower ? order.startAt(currentBlock, e) : currentBlock.minOffset(order); // let's check the last term of the new block right away // if expression's upper bound is satisfied by it such means that we can avoid // doing any expensive upper bound checks for that block. checkUpper = e.hasUpper() && !e.isUpperSatisfiedBy(currentBlock.getTerm(currentBlock.maxOffset(order))); } {code} Above, if we are unlucky and the last term of next block is partial, the entire next block will be skipped incorrectly Proposed change: * remove the {{if (nonMatchingPartial(term))}} check from current {{e::isLowerSatisfiedBy}} and {{e::isUpperSatisfiedBy}} * rename the method {{Expression::nonMatchingPartial}} to {{Expression::partialTermAndPrefixOperation}} which describes exactly which condition it checks, much clearer name from my pov. Also make the method access *public* * add an extra condition in {{OnDiskIndex::computeNext()}} to skip the current term and process the next one {code:java} if (offset >= 0 && offset < currentBlock.termCount()) { DataTerm currentTerm = currentBlock.getTerm(nextOffset()); if (e.partialTermAndPrefixOperation(currentTerm) || (checkLower && !e.isLowerSatisfiedBy(currentTerm))) continue; // flip the flag right on the first bounds match // to avoid expensive comparisons checkLower = false; if (checkUpper &&
[jira] [Commented] (CASSANDRA-12073) [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial results
[ https://issues.apache.org/jira/browse/CASSANDRA-12073?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350113#comment-15350113 ] DOAN DuyHai commented on CASSANDRA-12073: - Ok I've dug further into the code and the bug has wider implication 1) Premature termination by calling {{endOfData()}} as mentioned above in {code:java} if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) return endOfData(); {code} 2) Incorrect next block skipping in {{OnDiskIndex.nextBlock()}} {code:java} protected void nextBlock() { currentBlock = null; if (blockIndex < 0 || blockIndex >= dataLevel.blockCount) return; currentBlock = dataLevel.getBlock(nextBlockIndex()); offset = checkLower ? order.startAt(currentBlock, e) : currentBlock.minOffset(order); // let's check the last term of the new block right away // if expression's upper bound is satisfied by it such means that we can avoid // doing any expensive upper bound checks for that block. checkUpper = e.hasUpper() && !e.isUpperSatisfiedBy(currentBlock.getTerm(currentBlock.maxOffset(order))); } {code} Above, if we are unlucky and the last term of next block is partial, the entire next block will be skipped Proposed change: * remove the {{if (nonMatchingPartial(term))}} check from current {{e::isLowerSatisfiedBy}} and {{e::isUpperSatisfiedBy}} * rename the method {{Expression::nonMatchingPartial}} to {{Expression::partialTermAndPrefixOperation}} which describes exactly which condition it checks, much clearer name from my pov. Also make the method access *public* * add an extra condition in {{OnDiskIndex::computeNext()}} to skip the current term and process the next one {code:java} if (offset >= 0 && offset < currentBlock.termCount()) { DataTerm currentTerm = currentBlock.getTerm(nextOffset()); if (e.partialTermAndPrefixOperation(currentTerm) || (checkLower && !e.isLowerSatisfiedBy(currentTerm))) continue; // flip the flag right on the first bounds match // to avoid expensive comparisons checkLower = false; if (checkUpper && !e.isUpperSatisfiedBy(currentTerm)) return endOfData(); return currentTerm; } {code} I'll also add an extra unit-test to cover the premature {{endOfData()}} case. To test the incorrect next block skipping we'll need to generate enough data so that the last term of next block is partial. I'm not sure how feasible it is. What do you think @xedin, [~jrwest] ? > [SASI] PREFIX search on CONTAINS/NonTokenizer mode returns only partial > results > --- > > Key: CASSANDRA-12073 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12073 > Project: Cassandra > Issue Type: Bug > Components: CQL > Environment: Cassandra 3.7 >Reporter: DOAN DuyHai >Assignee: DOAN DuyHai > Fix For: 3.x > > > {noformat} > cqlsh:music> CREATE TABLE music.albums ( > id uuid PRIMARY KEY, > artist text, > country text, > quality text, > status text, > title text, > year int > ); > cqlsh:music> CREATE CUSTOM INDEX albums_artist_idx ON music.albums (artist) > USING 'org.apache.cassandra.index.sasi.SASIIndex' WITH OPTIONS = {'mode': > 'CONTAINS', 'analyzer_class': > 'org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer', > 'case_sensitive': 'false'}; > cqlsh:music> SELECT * FROM albums WHERE artist like 'lady%' LIMIT 100; > id | artist| country| quality > | status| title | year > --+---++-+---+---+-- > 372bb0ab-3263-41bc-baad-bb520ddfa787 | Lady Gaga |USA | normal > | Official | Red and Blue EP | 2006 > 1a4abbcd-b5de-4c69-a578-31231e01ff09 | Lady Gaga |Unknown | normal > | Promotion |Poker Face | 2008 > 31f4a0dc-9efc-48bf-9f5e-bfc09af42b82 | Lady Gaga |USA | normal > | Official | The Cherrytree Sessions | 2009 > 8ebfaebd-28d0-477d-b735-469661ce6873 | Lady Gaga |Unknown | normal > | Official |Poker Face | 2009 > 98107d82-e0dd-46bc-a273-1577578984c7 | Lady Gaga |USA | normal > | Official | Just Dance: The Remixes | 2008 > a76af0f2-f5c5-4306-974a-e3c17158e6c6 | Lady Gaga | Italy | normal > | Official | The Fame | 2008 > 849ee019-8b15-4767-8660-537ab9710459 | Lady Gaga |USA | normal > | Official |Christmas Tree | 2008 >
[jira] [Updated] (CASSANDRA-12078) [SASI] Move skip_stop_words filter BEFORE stemming
[ https://issues.apache.org/jira/browse/CASSANDRA-12078?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pavel Yaskevich updated CASSANDRA-12078: Resolution: Fixed Status: Resolved (was: Patch Available) Committed, thanks [~doanduyhai]! I've removed already committed part from the patch and included only change for {{StemmingFilters}} and tests. > [SASI] Move skip_stop_words filter BEFORE stemming > -- > > Key: CASSANDRA-12078 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12078 > Project: Cassandra > Issue Type: Bug > Components: sasi > Environment: Cassandra 3.7, Cassandra 3.8 >Reporter: DOAN DuyHai >Assignee: DOAN DuyHai > Fix For: 3.8 > > Attachments: patch.txt, patch_V2.txt > > > Right now, if skip stop words and stemming are enabled, SASI will put > stemming in the filter pipeline BEFORE skip_stop_words: > {code:java} > private FilterPipelineTask getFilterPipeline() > { > FilterPipelineBuilder builder = new FilterPipelineBuilder(new > BasicResultFilters.NoOperation()); > ... > 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(); > } > {code} > The problem is that stemming before removing stop words can yield wrong > results. > I have an example: > {code:sql} > SELECT * FROM music.albums WHERE country='France' AND title LIKE 'danse' > ALLOW FILTERING; > {code} > Because of stemming *danse* ( *dance* in English) becomes *dans* (the final > vowel is removed). Then skip stop words is applied. Unfortunately *dans* > (*in* in English) is a stop word in French so it is removed completely. > In the end the query is equivalent to {{SELECT * FROM music.albums WHERE > country='France'}} and of course the results are wrong. > Attached is a trivial patch to move the skip_stop_words filter BEFORE > stemming filter > /cc [~xedin] [~jrwest] [~beobal] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
cassandra git commit: Correct english word stemming test and add a test for french
Repository: cassandra Updated Branches: refs/heads/trunk f1cabcade -> eb82861c8 Correct english word stemming test and add a test for french patch by doanduyhai; reviewed by xedin for CASSANDRA-12078 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/eb82861c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/eb82861c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/eb82861c Branch: refs/heads/trunk Commit: eb82861c8d4c497d64b5e61a1606bdd270e8e109 Parents: f1cabca Author: Pavel YaskevichAuthored: Sun Jun 26 01:48:23 2016 -0700 Committer: Pavel Yaskevich Committed: Sun Jun 26 01:50:32 2016 -0700 -- .../sasi/analyzer/filter/StemmingFilters.java | 2 +- .../french_skip_stop_words_before_stemming.txt | 1 + .../sasi/analyzer/StandardAnalyzerTest.java | 33 +++- 3 files changed, 34 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/eb82861c/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 index 9e098d1..cb840a8 100644 --- a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StemmingFilters.java +++ b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StemmingFilters.java @@ -37,7 +37,7 @@ public class StemmingFilters public String process(String input) throws Exception { -if (stemmer == null) +if (input == null || stemmer == null) return input; stemmer.setCurrent(input); return (stemmer.stem()) ? stemmer.getCurrent() : input; http://git-wip-us.apache.org/repos/asf/cassandra/blob/eb82861c/test/resources/tokenization/french_skip_stop_words_before_stemming.txt -- diff --git a/test/resources/tokenization/french_skip_stop_words_before_stemming.txt b/test/resources/tokenization/french_skip_stop_words_before_stemming.txt new file mode 100644 index 000..59a1c23 --- /dev/null +++ b/test/resources/tokenization/french_skip_stop_words_before_stemming.txt @@ -0,0 +1 @@ +"La danse sous la pluie" est une chanson connue \ No newline at end of file http://git-wip-us.apache.org/repos/asf/cassandra/blob/eb82861c/test/unit/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzerTest.java -- diff --git a/test/unit/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzerTest.java b/test/unit/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzerTest.java index e307512..7a88a3d 100644 --- a/test/unit/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzerTest.java +++ b/test/unit/org/apache/cassandra/index/sasi/analyzer/StandardAnalyzerTest.java @@ -25,6 +25,8 @@ import java.util.Locale; import org.junit.Test; +import org.apache.cassandra.serializers.UTF8Serializer; + import static org.junit.Assert.assertEquals; public class StandardAnalyzerTest @@ -151,7 +153,36 @@ public class StandardAnalyzerTest while (tokenizer.hasNext()) tokens.add(tokenizer.next()); -assertEquals(40249, tokens.size()); +assertEquals(37739, tokens.size()); +} + +@Test +public void testSkipStopWordBeforeStemmingFrench() throws Exception +{ +InputStream is = StandardAnalyzerTest.class.getClassLoader() + .getResourceAsStream("tokenization/french_skip_stop_words_before_stemming.txt"); + +StandardTokenizerOptions options = new StandardTokenizerOptions.OptionsBuilder().stemTerms(true) +.ignoreStopTerms(true).useLocale(Locale.FRENCH) +.alwaysLowerCaseTerms(true).build(); +StandardAnalyzer tokenizer = new StandardAnalyzer(); +tokenizer.init(options); + +List tokens = new ArrayList<>(); +List words = new ArrayList<>(); +tokenizer.reset(is); +while (tokenizer.hasNext()) +{ +final ByteBuffer nextToken = tokenizer.next(); +tokens.add(nextToken); + words.add(UTF8Serializer.instance.deserialize(nextToken.duplicate())); +} + +assertEquals(4, tokens.size()); +assertEquals("dans", words.get(0)); +assertEquals("plui", words.get(1)); +assertEquals("chanson", words.get(2)); +assertEquals("connu", words.get(3)); } @Test
[jira] [Updated] (CASSANDRA-12078) [SASI] Move skip_stop_words filter BEFORE stemming
[ https://issues.apache.org/jira/browse/CASSANDRA-12078?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] DOAN DuyHai updated CASSANDRA-12078: Attachment: patch_V2.txt Attached is {{patch_V2.txt}} > [SASI] Move skip_stop_words filter BEFORE stemming > -- > > Key: CASSANDRA-12078 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12078 > Project: Cassandra > Issue Type: Bug > Components: sasi > Environment: Cassandra 3.7, Cassandra 3.8 >Reporter: DOAN DuyHai >Assignee: DOAN DuyHai > Fix For: 3.8 > > Attachments: patch.txt, patch_V2.txt > > > Right now, if skip stop words and stemming are enabled, SASI will put > stemming in the filter pipeline BEFORE skip_stop_words: > {code:java} > private FilterPipelineTask getFilterPipeline() > { > FilterPipelineBuilder builder = new FilterPipelineBuilder(new > BasicResultFilters.NoOperation()); > ... > 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(); > } > {code} > The problem is that stemming before removing stop words can yield wrong > results. > I have an example: > {code:sql} > SELECT * FROM music.albums WHERE country='France' AND title LIKE 'danse' > ALLOW FILTERING; > {code} > Because of stemming *danse* ( *dance* in English) becomes *dans* (the final > vowel is removed). Then skip stop words is applied. Unfortunately *dans* > (*in* in English) is a stop word in French so it is removed completely. > In the end the query is equivalent to {{SELECT * FROM music.albums WHERE > country='France'}} and of course the results are wrong. > Attached is a trivial patch to move the skip_stop_words filter BEFORE > stemming filter > /cc [~xedin] [~jrwest] [~beobal] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-12078) [SASI] Move skip_stop_words filter BEFORE stemming
[ https://issues.apache.org/jira/browse/CASSANDRA-12078?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] DOAN DuyHai updated CASSANDRA-12078: Status: Patch Available (was: Reopened) > [SASI] Move skip_stop_words filter BEFORE stemming > -- > > Key: CASSANDRA-12078 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12078 > Project: Cassandra > Issue Type: Bug > Components: sasi > Environment: Cassandra 3.7, Cassandra 3.8 >Reporter: DOAN DuyHai >Assignee: DOAN DuyHai > Fix For: 3.8 > > Attachments: patch.txt > > > Right now, if skip stop words and stemming are enabled, SASI will put > stemming in the filter pipeline BEFORE skip_stop_words: > {code:java} > private FilterPipelineTask getFilterPipeline() > { > FilterPipelineBuilder builder = new FilterPipelineBuilder(new > BasicResultFilters.NoOperation()); > ... > 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(); > } > {code} > The problem is that stemming before removing stop words can yield wrong > results. > I have an example: > {code:sql} > SELECT * FROM music.albums WHERE country='France' AND title LIKE 'danse' > ALLOW FILTERING; > {code} > Because of stemming *danse* ( *dance* in English) becomes *dans* (the final > vowel is removed). Then skip stop words is applied. Unfortunately *dans* > (*in* in English) is a stop word in French so it is removed completely. > In the end the query is equivalent to {{SELECT * FROM music.albums WHERE > country='France'}} and of course the results are wrong. > Attached is a trivial patch to move the skip_stop_words filter BEFORE > stemming filter > /cc [~xedin] [~jrwest] [~beobal] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-12078) [SASI] Move skip_stop_words filter BEFORE stemming
[ https://issues.apache.org/jira/browse/CASSANDRA-12078?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15350028#comment-15350028 ] DOAN DuyHai commented on CASSANDRA-12078: - [~xedin] I have been able to reproduce the unit test failing locally. The error comes from test {{testTokenizationAdventuresOfHuckFinn}}. After switching skip stop words before stemming, the expected tokens count is *37739* and not *40249* There is also a {{NullPointerException}} when switching skip stop words before stemming. Indeed in some case, the token is removed by stop words filter so the input of the stemming filter is null. I've added an extra null check in the {{DefaultStemmingFilter}} {code:java} public String process(String input) throws Exception { if (input == null || stemmer == null) return input; stemmer.setCurrent(input); return (stemmer.stem()) ? stemmer.getCurrent() : input; } {code} I have also added a new unit test in {{StandardAnalyzerTest}} to cover the french issue mentioned above: {code:java} @Test public void testSkipStopWordBeforeStemmingFrench() throws Exception { InputStream is = StandardAnalyzerTest.class.getClassLoader() .getResourceAsStream("tokenization/french_skip_stop_words_before_stemming.txt"); StandardTokenizerOptions options = new StandardTokenizerOptions.OptionsBuilder().stemTerms(true) .ignoreStopTerms(true).useLocale(Locale.FRENCH) .alwaysLowerCaseTerms(true).build(); StandardAnalyzer tokenizer = new StandardAnalyzer(); tokenizer.init(options); List tokens = new ArrayList<>(); List words = new ArrayList<>(); tokenizer.reset(is); while (tokenizer.hasNext()) { final ByteBuffer nextToken = tokenizer.next(); tokens.add(nextToken); words.add(UTF8Serializer.instance.deserialize(nextToken.duplicate())); } assertEquals(4, tokens.size()); assertEquals("dans", words.get(0)); assertEquals("plui", words.get(1)); assertEquals("chanson", words.get(2)); assertEquals("connu", words.get(3)); } {code} > [SASI] Move skip_stop_words filter BEFORE stemming > -- > > Key: CASSANDRA-12078 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12078 > Project: Cassandra > Issue Type: Bug > Components: sasi > Environment: Cassandra 3.7, Cassandra 3.8 >Reporter: DOAN DuyHai >Assignee: DOAN DuyHai > Fix For: 3.8 > > Attachments: patch.txt > > > Right now, if skip stop words and stemming are enabled, SASI will put > stemming in the filter pipeline BEFORE skip_stop_words: > {code:java} > private FilterPipelineTask getFilterPipeline() > { > FilterPipelineBuilder builder = new FilterPipelineBuilder(new > BasicResultFilters.NoOperation()); > ... > 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(); > } > {code} > The problem is that stemming before removing stop words can yield wrong > results. > I have an example: > {code:sql} > SELECT * FROM music.albums WHERE country='France' AND title LIKE 'danse' > ALLOW FILTERING; > {code} > Because of stemming *danse* ( *dance* in English) becomes *dans* (the final > vowel is removed). Then skip stop words is applied. Unfortunately *dans* > (*in* in English) is a stop word in French so it is removed completely. > In the end the query is equivalent to {{SELECT * FROM music.albums WHERE > country='France'}} and of course the results are wrong. > Attached is a trivial patch to move the skip_stop_words filter BEFORE > stemming filter > /cc [~xedin] [~jrwest] [~beobal] -- This message was sent by Atlassian JIRA (v6.3.4#6332)