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 2020/08/25 18:17:15 UTC

[cassandra] branch cassandra-3.11 updated: Correctly interpret SASI's `max_compaction_flush_memory_in_mb` setting in megabytes not bytes

This is an automated email from the ASF dual-hosted git repository.

mck pushed a commit to branch cassandra-3.11
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/cassandra-3.11 by this push:
     new 116e9c1  Correctly interpret SASI's `max_compaction_flush_memory_in_mb` setting in megabytes not bytes
116e9c1 is described below

commit 116e9c1678a1e96748236962e71319f337e07f8d
Author: Mick Semb Wever <mc...@apache.org>
AuthorDate: Sat Aug 22 15:15:19 2020 +0200

    Correctly interpret SASI's `max_compaction_flush_memory_in_mb` setting in megabytes not bytes
    
     patch by Mick Semb Wever; reviewed by Zhao Yang for CASSANDRA-16071
---
 CHANGES.txt                                        |   1 +
 .../cassandra/index/sasi/conf/IndexMode.java       |  12 +-
 .../index/sasi/disk/PerSSTableIndexWriter.java     |   2 +-
 .../cassandra/index/sasi/conf/IndexModeTest.java   | 222 +++++++++++++++++++++
 4 files changed, 230 insertions(+), 7 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 714b104..1ba44f5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.8
+ * Correctly interpret SASI's `max_compaction_flush_memory_in_mb` setting in megabytes not bytes (CASSANDRA-16071)
  * Fix short read protection for GROUP BY queries (CASSANDRA-15459)
  * Frozen RawTuple is not annotated with frozen in the toString method (CASSANDRA-15857)
 Merged from 3.0:
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java b/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
index 5709a0f..8d76bb0 100644
--- a/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
+++ b/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
@@ -60,15 +60,15 @@ public class IndexMode
     public final Mode mode;
     public final boolean isAnalyzed, isLiteral;
     public final Class analyzerClass;
-    public final long maxCompactionFlushMemoryInMb;
+    public final long maxCompactionFlushMemoryInBytes;
 
-    private IndexMode(Mode mode, boolean isLiteral, boolean isAnalyzed, Class analyzerClass, long maxFlushMemMb)
+    private IndexMode(Mode mode, boolean isLiteral, boolean isAnalyzed, Class analyzerClass, long maxMemBytes)
     {
         this.mode = mode;
         this.isLiteral = isLiteral;
         this.isAnalyzed = isAnalyzed;
         this.analyzerClass = analyzerClass;
-        this.maxCompactionFlushMemoryInMb = maxFlushMemMb;
+        this.maxCompactionFlushMemoryInBytes = maxMemBytes;
     }
 
     public AbstractAnalyzer getAnalyzer(AbstractType<?> validator)
@@ -186,11 +186,11 @@ public class IndexMode
             logger.error("failed to parse {} option, defaulting to 'false'.", INDEX_IS_LITERAL_OPTION);
         }
 
-        Long maxMemMb = indexOptions.get(INDEX_MAX_FLUSH_MEMORY_OPTION) == null
+        long maxMemBytes = indexOptions.get(INDEX_MAX_FLUSH_MEMORY_OPTION) == null
                 ? (long) (1073741824 * INDEX_MAX_FLUSH_DEFAULT_MULTIPLIER) // 1G default for memtable
-                : Long.parseLong(indexOptions.get(INDEX_MAX_FLUSH_MEMORY_OPTION));
+                : 1048576L * Long.parseLong(indexOptions.get(INDEX_MAX_FLUSH_MEMORY_OPTION));
 
-        return new IndexMode(mode, isLiteral, isAnalyzed, analyzerClass, maxMemMb);
+        return new IndexMode(mode, isLiteral, isAnalyzed, analyzerClass, maxMemBytes);
     }
 
     public boolean supports(Op operator)
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
index d5ecaf7..527deab 100644
--- a/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
+++ b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
@@ -360,7 +360,7 @@ public class PerSSTableIndexWriter implements SSTableFlushObserver
     protected long maxMemorySize(ColumnIndex columnIndex)
     {
         // 1G for memtable and configuration for compaction
-        return source == OperationType.FLUSH ? 1073741824L : columnIndex.getMode().maxCompactionFlushMemoryInMb;
+        return source == OperationType.FLUSH ? 1073741824L : columnIndex.getMode().maxCompactionFlushMemoryInBytes;
     }
 
     public int hashCode()
diff --git a/test/unit/org/apache/cassandra/index/sasi/conf/IndexModeTest.java b/test/unit/org/apache/cassandra/index/sasi/conf/IndexModeTest.java
new file mode 100644
index 0000000..9f93f27
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sasi/conf/IndexModeTest.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.index.sasi.conf;
+
+import java.util.Collections;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder.Mode;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+
+public class IndexModeTest
+{
+
+    private final CFMetaData cfm = CFMetaData.Builder.create("ks", "cf")
+                        .addPartitionKey("pkey", AsciiType.instance)
+                        .build();
+
+    @BeforeClass
+    public static void setupDD()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @Test
+    public void test_notIndexed()
+    {
+        Assert.assertEquals(IndexMode.NOT_INDEXED, IndexMode.getMode(null, (Map)null));
+        Assert.assertEquals(IndexMode.NOT_INDEXED, IndexMode.getMode(null, Collections.EMPTY_MAP));
+    }
+
+    @Test(expected = ConfigurationException.class)
+    public void test_bad_mode_option()
+    {
+        IndexMode.getMode(null, Collections.singletonMap("mode", "invalid"));
+    }
+
+    @Test
+    public void test_asciiType()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), AsciiType.instance);
+
+        IndexMode result = IndexMode.getMode(cd, Collections.singletonMap("something", "nothing"));
+        Assert.assertNull(result.analyzerClass);
+        Assert.assertFalse(result.isAnalyzed);
+        Assert.assertTrue(result.isLiteral);
+        Assert.assertEquals((long)(1073741824 * 0.15), result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+
+    @Test
+    public void test_asciiType_notLiteral()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), AsciiType.instance);
+
+        IndexMode result = IndexMode.getMode(cd, Collections.singletonMap("is_literal", "false"));
+        Assert.assertNull(result.analyzerClass);
+        Assert.assertFalse(result.isAnalyzed);
+        Assert.assertFalse(result.isLiteral);
+        Assert.assertEquals((long)(1073741824 * 0.15), result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+
+    @Test
+    public void test_asciiType_errLiteral()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), AsciiType.instance);
+
+        IndexMode result = IndexMode.getMode(cd, Collections.singletonMap("is_literal", "junk"));
+        Assert.assertNull(result.analyzerClass);
+        Assert.assertFalse(result.isAnalyzed);
+        Assert.assertFalse(result.isLiteral);
+        Assert.assertEquals((long)(1073741824 * 0.15), result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+
+    @Test
+    public void test_asciiType_analyzed()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), AsciiType.instance);
+
+        IndexMode result = IndexMode.getMode(cd, Collections.singletonMap("analyzed", "true"));
+        Assert.assertNull(result.analyzerClass);
+        Assert.assertTrue(result.isAnalyzed);
+        Assert.assertTrue(result.isLiteral);
+        Assert.assertEquals((long)(1073741824 * 0.15), result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+
+    @Test
+    public void test_utf8Type()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), UTF8Type.instance);
+
+        IndexMode result = IndexMode.getMode(cd, Collections.singletonMap("something", "nothing"));
+        Assert.assertNull(result.analyzerClass);
+        Assert.assertFalse(result.isAnalyzed);
+        Assert.assertTrue(result.isLiteral);
+        Assert.assertEquals((long)(1073741824 * 0.15), result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+
+    @Test
+    public void test_bytesType()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), BytesType.instance);
+
+        IndexMode result = IndexMode.getMode(cd, Collections.singletonMap("something", "nothing"));
+        Assert.assertNull(result.analyzerClass);
+        Assert.assertFalse(result.isAnalyzed);
+        Assert.assertFalse(result.isLiteral);
+        Assert.assertEquals((long)(1073741824 * 0.15), result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+
+    @Test
+    public void test_bytesType_isLiteral()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), BytesType.instance);
+
+        IndexMode result = IndexMode.getMode(cd, Collections.singletonMap("is_literal", "true"));
+        Assert.assertNull(result.analyzerClass);
+        Assert.assertFalse(result.isAnalyzed);
+        Assert.assertTrue(result.isLiteral);
+        Assert.assertEquals((long)(1073741824 * 0.15), result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+
+    @Test
+    public void test_bytesType_errLiteral()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), BytesType.instance);
+
+        IndexMode result = IndexMode.getMode(cd, Collections.singletonMap("is_literal", "junk"));
+        Assert.assertNull(result.analyzerClass);
+        Assert.assertFalse(result.isAnalyzed);
+        Assert.assertFalse(result.isLiteral);
+        Assert.assertEquals((long)(1073741824 * 0.15), result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+
+    @Test
+    public void test_bytesType_analyzed()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), BytesType.instance);
+
+        IndexMode result = IndexMode.getMode(cd, Collections.singletonMap("analyzed", "true"));
+        Assert.assertNull(result.analyzerClass);
+        Assert.assertTrue(result.isAnalyzed);
+        Assert.assertFalse(result.isLiteral);
+        Assert.assertEquals((long)(1073741824 * 0.15), result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+
+    @Test
+    public void test_bytesType_analyzer()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), BytesType.instance);
+
+        IndexMode result = IndexMode.getMode(cd, Collections.singletonMap("analyzer_class", "java.lang.Object"));
+        Assert.assertEquals(Object.class, result.analyzerClass);
+        Assert.assertTrue(result.isAnalyzed);
+        Assert.assertFalse(result.isLiteral);
+        Assert.assertEquals((long)(1073741824 * 0.15), result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+
+    @Test
+    public void test_bytesType_analyzer_unanalyzed()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), BytesType.instance);
+
+        IndexMode result = IndexMode.getMode(cd, ImmutableMap.of("analyzer_class", "java.lang.Object",
+                                                                 "analyzed", "false"));
+
+        Assert.assertEquals(Object.class, result.analyzerClass);
+        Assert.assertFalse(result.isAnalyzed);
+        Assert.assertFalse(result.isLiteral);
+        Assert.assertEquals((long)(1073741824 * 0.15), result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+
+    @Test
+    public void test_bytesType_maxCompactionFlushMemoryInBytes()
+    {
+        ColumnDefinition cd = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition"), BytesType.instance);
+
+        IndexMode result = IndexMode.getMode(cd, Collections.singletonMap("max_compaction_flush_memory_in_mb", "1"));
+        Assert.assertNull(result.analyzerClass);
+        Assert.assertFalse(result.isAnalyzed);
+        Assert.assertFalse(result.isLiteral);
+        Assert.assertEquals(1048576L, result.maxCompactionFlushMemoryInBytes);
+        Assert.assertEquals(Mode.PREFIX, result.mode);
+    }
+}


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