You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by st...@apache.org on 2016/09/20 01:29:42 UTC

[1/3] cassandra git commit: Extend ColumnIdentifier.internedInstances key to include the type that generated the byte buffer

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 85ed48a1d -> 4bc3aa933
  refs/heads/trunk d43b9ce50 -> 105054471


Extend ColumnIdentifier.internedInstances key to include the type that generated the byte buffer

patch by Stefania Alborghetti; reviewed by Aleksey Yeschenko for CASSANDRA-12516


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

Branch: refs/heads/cassandra-3.0
Commit: 4bc3aa93337474a29bc4ad43ff4438755c9c7ab8
Parents: 85ed48a
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Fri Sep 2 16:28:35 2016 +0800
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Tue Sep 20 09:26:38 2016 +0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/cql3/ColumnIdentifier.java | 46 +++++++++++++++++---
 .../apache/cassandra/schema/SchemaKeyspace.java |  6 ++-
 .../utils/NativeSSTableLoaderClient.java        |  6 ++-
 .../cassandra/cql3/ColumnIdentifierTest.java    | 24 ++++++++++
 5 files changed, 72 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4bc3aa93/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5fd8e5e..b61c76c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.10
+ * Extend ColumnIdentifier.internedInstances key to include the type that generated the byte buffer (CASSANDRA-12516)
  * Backport CASSANDRA-10756 (race condition in NativeTransportService shutdown) (CASSANDRA-12472)
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4bc3aa93/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index afb65e1..1e25b23 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -60,7 +60,38 @@ public class ColumnIdentifier extends Selectable implements IMeasurableMemory, C
 
     private static final long EMPTY_SIZE = ObjectSizes.measure(new ColumnIdentifier(ByteBufferUtil.EMPTY_BYTE_BUFFER, "", false));
 
-    private static final ConcurrentMap<ByteBuffer, ColumnIdentifier> internedInstances = new MapMaker().weakValues().makeMap();
+    private static final ConcurrentMap<InternedKey, ColumnIdentifier> internedInstances = new MapMaker().weakValues().makeMap();
+
+    private static final class InternedKey
+    {
+        private final AbstractType<?> type;
+        private final ByteBuffer bytes;
+
+        InternedKey(AbstractType<?> type, ByteBuffer bytes)
+        {
+            this.type = type;
+            this.bytes = bytes;
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            InternedKey that = (InternedKey) o;
+            return bytes.equals(that.bytes) && type.equals(that.type);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return bytes.hashCode() + 31 * type.hashCode();
+        }
+    }
 
     private static long prefixComparison(ByteBuffer bytes)
     {
@@ -103,24 +134,25 @@ public class ColumnIdentifier extends Selectable implements IMeasurableMemory, C
 
     public static ColumnIdentifier getInterned(ByteBuffer bytes, AbstractType<?> type)
     {
-        return getInterned(bytes, type.getString(bytes));
+        return getInterned(type, bytes, type.getString(bytes));
     }
 
     public static ColumnIdentifier getInterned(String rawText, boolean keepCase)
     {
         String text = keepCase ? rawText : rawText.toLowerCase(Locale.US);
         ByteBuffer bytes = ByteBufferUtil.bytes(text);
-        return getInterned(bytes, text);
+        return getInterned(UTF8Type.instance, bytes, text);
     }
 
-    public static ColumnIdentifier getInterned(ByteBuffer bytes, String text)
+    public static ColumnIdentifier getInterned(AbstractType<?> type, ByteBuffer bytes, String text)
     {
-        ColumnIdentifier id = internedInstances.get(bytes);
+        InternedKey key = new InternedKey(type, bytes);
+        ColumnIdentifier id = internedInstances.get(key);
         if (id != null)
             return id;
 
         ColumnIdentifier created = new ColumnIdentifier(bytes, text, true);
-        ColumnIdentifier previous = internedInstances.putIfAbsent(bytes, created);
+        ColumnIdentifier previous = internedInstances.putIfAbsent(key, created);
         return previous == null ? created : previous;
     }
 
@@ -246,7 +278,7 @@ public class ColumnIdentifier extends Selectable implements IMeasurableMemory, C
                 if (def.name.bytes.equals(bufferName))
                     return def.name;
             }
-            return getInterned(thriftColumnNameType.fromString(rawText), text);
+            return getInterned(thriftColumnNameType, thriftColumnNameType.fromString(rawText), text);
         }
 
         public boolean processesSelection()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4bc3aa93/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index e3756ec..84a5e13 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -1001,8 +1001,6 @@ public final class SchemaKeyspace
         String keyspace = row.getString("keyspace_name");
         String table = row.getString("table_name");
 
-        ColumnIdentifier name = ColumnIdentifier.getInterned(row.getBytes("column_name_bytes"), row.getString("column_name"));
-
         ColumnDefinition.Kind kind = ColumnDefinition.Kind.valueOf(row.getString("kind").toUpperCase());
 
         int position = row.getInt("position");
@@ -1012,6 +1010,10 @@ public final class SchemaKeyspace
         if (order == ClusteringOrder.DESC)
             type = ReversedType.getInstance(type);
 
+        ColumnIdentifier name = ColumnIdentifier.getInterned(type,
+                                                             row.getBytes("column_name_bytes"),
+                                                             row.getString("column_name"));
+
         return new ColumnDefinition(keyspace, table, name, type, position, kind);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4bc3aa93/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
index 5bcbcf7..4c6b12e 100644
--- a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
+++ b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
@@ -198,13 +198,15 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
 
     private static ColumnDefinition createDefinitionFromRow(Row row, String keyspace, String table, Types types)
     {
-        ColumnIdentifier name = ColumnIdentifier.getInterned(row.getBytes("column_name_bytes"), row.getString("column_name"));
-
         ClusteringOrder order = ClusteringOrder.valueOf(row.getString("clustering_order").toUpperCase());
         AbstractType<?> type = CQLTypeParser.parse(keyspace, row.getString("type"), types);
         if (order == ClusteringOrder.DESC)
             type = ReversedType.getInstance(type);
 
+        ColumnIdentifier name = ColumnIdentifier.getInterned(type,
+                                                             row.getBytes("column_name_bytes"),
+                                                             row.getString("column_name"));
+
         int position = row.getInt("position");
         ColumnDefinition.Kind kind = ColumnDefinition.Kind.valueOf(row.getString("kind").toUpperCase());
         return new ColumnDefinition(keyspace, table, name, type, position, kind);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4bc3aa93/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java b/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
index c287883..c4b43b8 100644
--- a/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
@@ -24,7 +24,9 @@ import java.util.concurrent.ThreadLocalRandom;
 import org.junit.Test;
 
 import junit.framework.Assert;
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class ColumnIdentifierTest
@@ -58,4 +60,26 @@ public class ColumnIdentifierTest
         return v < 0 ? -1 : v > 0 ? 1 : 0;
     }
 
+    @Test
+    public void testInternedCache()
+    {
+        AbstractType<?> utf8Type = UTF8Type.instance;
+        AbstractType<?> bytesType = BytesType.instance;
+
+        byte[] bytes = new byte [] { 0x63, (byte) 0x32 };
+        String text = "c2"; // the UTF-8 encoding of this string is the same as bytes, 0x630x32
+
+        ColumnIdentifier c1 = ColumnIdentifier.getInterned(ByteBuffer.wrap(bytes), bytesType);
+        ColumnIdentifier c2 = ColumnIdentifier.getInterned(utf8Type, utf8Type.fromString(text), text);
+        ColumnIdentifier c3 = ColumnIdentifier.getInterned(text, true);
+
+        Assert.assertTrue(c1.isInterned());
+        Assert.assertTrue(c2.isInterned());
+        Assert.assertTrue(c3.isInterned());
+
+        Assert.assertEquals("6332", c1.toString());
+        Assert.assertEquals(text, c2.toString());
+        Assert.assertEquals(text, c3.toString());
+    }
+
 }


[3/3] cassandra git commit: Merge branch 'cassandra-3.0' into trunk

Posted by st...@apache.org.
Merge branch 'cassandra-3.0' into trunk


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

Branch: refs/heads/trunk
Commit: 10505447182a9df6c475e4ce0327f3eec217e7a5
Parents: d43b9ce 4bc3aa9
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Tue Sep 20 09:27:21 2016 +0800
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Tue Sep 20 09:28:26 2016 +0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/config/ColumnDefinition.java      |  2 +-
 .../apache/cassandra/cql3/ColumnIdentifier.java | 45 +++++++++++++++++---
 .../apache/cassandra/schema/SchemaKeyspace.java |  6 ++-
 .../utils/NativeSSTableLoaderClient.java        |  6 ++-
 .../cassandra/cql3/ColumnIdentifierTest.java    | 24 +++++++++++
 6 files changed, 73 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/10505447/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index e9e8ccf,b61c76c..979618d
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,69 -1,11 +1,70 @@@
 -3.0.10
 +3.10
 + * Support optional backpressure strategies at the coordinator (CASSANDRA-9318)
 + * Make randompartitioner work with new vnode allocation (CASSANDRA-12647)
 + * Fix cassandra-stress graphing (CASSANDRA-12237)
 + * Allow filtering on partition key columns for queries without secondary indexes (CASSANDRA-11031)
 + * Fix Cassandra Stress reporting thread model and precision (CASSANDRA-12585)
 + * Add JMH benchmarks.jar (CASSANDRA-12586)
 + * Add row offset support to SASI (CASSANDRA-11990)
 + * Cleanup uses of AlterTableStatementColumn (CASSANDRA-12567)
 + * Add keep-alive to streaming (CASSANDRA-11841)
 + * Tracing payload is passed through newSession(..) (CASSANDRA-11706)
 + * avoid deleting non existing sstable files and improve related log messages (CASSANDRA-12261)
 + * json/yaml output format for nodetool compactionhistory (CASSANDRA-12486)
 + * Retry all internode messages once after a connection is
 +   closed and reopened (CASSANDRA-12192)
 + * Add support to rebuild from targeted replica (CASSANDRA-9875)
 + * Add sequence distribution type to cassandra stress (CASSANDRA-12490)
 + * "SELECT * FROM foo LIMIT ;" does not error out (CASSANDRA-12154)
 + * Define executeLocally() at the ReadQuery Level (CASSANDRA-12474)
 + * Extend read/write failure messages with a map of replica addresses
 +   to error codes in the v5 native protocol (CASSANDRA-12311)
 + * Fix rebuild of SASI indexes with existing index files (CASSANDRA-12374)
 + * Let DatabaseDescriptor not implicitly startup services (CASSANDRA-9054, 12550)
 + * Fix clustering indexes in presence of static columns in SASI (CASSANDRA-12378)
 + * Fix queries on columns with reversed type on SASI indexes (CASSANDRA-12223)
 + * Added slow query log (CASSANDRA-12403)
 + * Count full coordinated request against timeout (CASSANDRA-12256)
 + * Allow TTL with null value on insert and update (CASSANDRA-12216)
 + * Make decommission operation resumable (CASSANDRA-12008)
 + * Add support to one-way targeted repair (CASSANDRA-9876)
 + * Remove clientutil jar (CASSANDRA-11635)
 + * Fix compaction throughput throttle (CASSANDRA-12366)
 + * Delay releasing Memtable memory on flush until PostFlush has finished running (CASSANDRA-12358)
 + * Cassandra stress should dump all setting on startup (CASSANDRA-11914)
 + * Make it possible to compact a given token range (CASSANDRA-10643)
 + * Allow updating DynamicEndpointSnitch properties via JMX (CASSANDRA-12179)
 + * Collect metrics on queries by consistency level (CASSANDRA-7384)
 + * Add support for GROUP BY to SELECT statement (CASSANDRA-10707)
 + * Deprecate memtable_cleanup_threshold and update default for memtable_flush_writers (CASSANDRA-12228)
 + * Upgrade to OHC 0.4.4 (CASSANDRA-12133)
 + * Add version command to cassandra-stress (CASSANDRA-12258)
 + * Create compaction-stress tool (CASSANDRA-11844)
 + * Garbage-collecting compaction operation and schema option (CASSANDRA-7019)
 + * Add beta protocol flag for v5 native protocol (CASSANDRA-12142)
 + * Support filtering on non-PRIMARY KEY columns in the CREATE
 +   MATERIALIZED VIEW statement's WHERE clause (CASSANDRA-10368)
 + * Unify STDOUT and SYSTEMLOG logback format (CASSANDRA-12004)
 + * COPY FROM should raise error for non-existing input files (CASSANDRA-12174)
 + * Faster write path (CASSANDRA-12269)
 + * Option to leave omitted columns in INSERT JSON unset (CASSANDRA-11424)
 + * Support json/yaml output in nodetool tpstats (CASSANDRA-12035)
 + * Expose metrics for successful/failed authentication attempts (CASSANDRA-10635)
 + * Prepend snapshot name with "truncated" or "dropped" when a snapshot
 +   is taken before truncating or dropping a table (CASSANDRA-12178)
 + * Optimize RestrictionSet (CASSANDRA-12153)
 + * cqlsh does not automatically downgrade CQL version (CASSANDRA-12150)
 + * Omit (de)serialization of state variable in UDAs (CASSANDRA-9613)
 + * Create a system table to expose prepared statements (CASSANDRA-8831)
 + * Reuse DataOutputBuffer from ColumnIndex (CASSANDRA-11970)
 + * Remove DatabaseDescriptor dependency from SegmentedFile (CASSANDRA-11580)
 + * Add supplied username to authentication error messages (CASSANDRA-12076)
 + * Remove pre-startup check for open JMX port (CASSANDRA-12074)
 + * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738)
 + * Restore resumable hints delivery (CASSANDRA-11960)
 +Merged from 3.0:
+  * Extend ColumnIdentifier.internedInstances key to include the type that generated the byte buffer (CASSANDRA-12516)
 - * Backport CASSANDRA-10756 (race condition in NativeTransportService shutdown) (CASSANDRA-12472)
 -
 -
 -3.0.9
   * Handle composite prefixes with final EOC=0 as in 2.x and refactor LegacyLayout.decodeBound (CASSANDRA-12423)
 - * Fix paging for 2.x to 3.x upgrades (CASSANDRA-11195)
   * select_distinct_with_deletions_test failing on non-vnode environments (CASSANDRA-11126)
   * Stack Overflow returned to queries while upgrading (CASSANDRA-12527)
   * Fix legacy regex for temporary files from 2.2 (CASSANDRA-12565)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/10505447/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/ColumnDefinition.java
index 713d684,6bcc2e0..6044ee9
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@@ -430,193 -398,4 +430,193 @@@ public class ColumnDefinition extends C
               ? ((CollectionType)type).valueComparator()
               : type;
      }
 +
 +    public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames) throws InvalidRequestException
 +    {
 +        return SimpleSelector.newFactory(this, addAndGetIndex(this, defs));
 +    }
 +
 +    public AbstractType<?> getExactTypeIfKnown(String keyspace)
 +    {
 +        return type;
 +    }
 +
 +    /**
 +     * Because Thrift-created tables may have a non-text comparator, we cannot determine the proper 'key' until
 +     * we know the comparator. ColumnDefinition.Raw is a placeholder that can be converted to a real ColumnIdentifier
 +     * once the comparator is known with prepare(). This should only be used with identifiers that are actual
 +     * column names. See CASSANDRA-8178 for more background.
 +     */
 +    public static abstract class Raw extends Selectable.Raw
 +    {
 +        /**
 +         * Creates a {@code ColumnDefinition.Raw} from an unquoted identifier string.
 +         */
 +        public static Raw forUnquoted(String text)
 +        {
 +            return new Literal(text, false);
 +        }
 +
 +        /**
 +         * Creates a {@code ColumnDefinition.Raw} from a quoted identifier string.
 +         */
 +        public static Raw forQuoted(String text)
 +        {
 +            return new Literal(text, true);
 +        }
 +
 +        /**
 +         * Creates a {@code ColumnDefinition.Raw} from a pre-existing {@code ColumnDefinition}
 +         * (useful in the rare cases where we already have the column but need
 +         * a {@code ColumnDefinition.Raw} for typing purposes).
 +         */
 +        public static Raw forColumn(ColumnDefinition column)
 +        {
 +            return new ForColumn(column);
 +        }
 +
 +        /**
 +         * Get the identifier corresponding to this raw column, without assuming this is an
 +         * existing column (unlike {@link #prepare}).
 +         */
 +        public abstract ColumnIdentifier getIdentifier(CFMetaData cfm);
 +
 +        public abstract String rawText();
 +
 +        @Override
 +        public abstract ColumnDefinition prepare(CFMetaData cfm);
 +
 +        @Override
 +        public boolean processesSelection()
 +        {
 +            return false;
 +        }
 +
 +        @Override
 +        public final int hashCode()
 +        {
 +            return toString().hashCode();
 +        }
 +
 +        @Override
 +        public final boolean equals(Object o)
 +        {
 +            if(!(o instanceof Raw))
 +                return false;
 +
 +            Raw that = (Raw)o;
 +            return this.toString().equals(that.toString());
 +        }
 +
 +        private static class Literal extends Raw
 +        {
 +            private final String text;
 +
 +            public Literal(String rawText, boolean keepCase)
 +            {
 +                this.text =  keepCase ? rawText : rawText.toLowerCase(Locale.US);
 +            }
 +
 +            public ColumnIdentifier getIdentifier(CFMetaData cfm)
 +            {
 +                if (!cfm.isStaticCompactTable())
 +                    return ColumnIdentifier.getInterned(text, true);
 +
 +                AbstractType<?> thriftColumnNameType = cfm.thriftColumnNameType();
 +                if (thriftColumnNameType instanceof UTF8Type)
 +                    return ColumnIdentifier.getInterned(text, true);
 +
 +                // We have a Thrift-created table with a non-text comparator. Check if we have a match column, otherwise assume we should use
 +                // thriftColumnNameType
 +                ByteBuffer bufferName = ByteBufferUtil.bytes(text);
 +                for (ColumnDefinition def : cfm.allColumns())
 +                {
 +                    if (def.name.bytes.equals(bufferName))
 +                        return def.name;
 +                }
-                 return ColumnIdentifier.getInterned(thriftColumnNameType.fromString(text), text);
++                return ColumnIdentifier.getInterned(thriftColumnNameType, thriftColumnNameType.fromString(text), text);
 +            }
 +
 +            public ColumnDefinition prepare(CFMetaData cfm)
 +            {
 +                if (!cfm.isStaticCompactTable())
 +                    return find(cfm);
 +
 +                AbstractType<?> thriftColumnNameType = cfm.thriftColumnNameType();
 +                if (thriftColumnNameType instanceof UTF8Type)
 +                    return find(cfm);
 +
 +                // We have a Thrift-created table with a non-text comparator. Check if we have a match column, otherwise assume we should use
 +                // thriftColumnNameType
 +                ByteBuffer bufferName = ByteBufferUtil.bytes(text);
 +                for (ColumnDefinition def : cfm.allColumns())
 +                {
 +                    if (def.name.bytes.equals(bufferName))
 +                        return def;
 +                }
 +                return find(thriftColumnNameType.fromString(text), cfm);
 +            }
 +
 +            private ColumnDefinition find(CFMetaData cfm)
 +            {
 +                return find(ByteBufferUtil.bytes(text), cfm);
 +            }
 +
 +            private ColumnDefinition find(ByteBuffer id, CFMetaData cfm)
 +            {
 +                ColumnDefinition def = cfm.getColumnDefinition(id);
 +                if (def == null)
 +                    throw new InvalidRequestException(String.format("Undefined column name %s", toString()));
 +                return def;
 +            }
 +
 +            public String rawText()
 +            {
 +                return text;
 +            }
 +
 +            @Override
 +            public String toString()
 +            {
 +                return ColumnIdentifier.maybeQuote(text);
 +            }
 +        }
 +
 +        // Use internally in the rare case where we need a ColumnDefinition.Raw for type-checking but
 +        // actually already have the column itself.
 +        private static class ForColumn extends Raw
 +        {
 +            private final ColumnDefinition column;
 +
 +            private ForColumn(ColumnDefinition column)
 +            {
 +                this.column = column;
 +            }
 +
 +            public ColumnIdentifier getIdentifier(CFMetaData cfm)
 +            {
 +                return column.name;
 +            }
 +
 +            public ColumnDefinition prepare(CFMetaData cfm)
 +            {
 +                assert cfm.getColumnDefinition(column.name) != null; // Sanity check that we're not doing something crazy
 +                return column;
 +            }
 +
 +            public String rawText()
 +            {
 +                return column.name.toString();
 +            }
 +
 +            @Override
 +            public String toString()
 +            {
 +                return column.name.toCQLString();
 +            }
 +        }
 +    }
 +
 +
 +
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/10505447/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index a17b0e1,1e25b23..7a23231
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@@ -27,7 -27,14 +27,8 @@@ import com.google.common.annotations.Vi
  import com.google.common.collect.MapMaker;
  
  import org.apache.cassandra.cache.IMeasurableMemory;
 -import org.apache.cassandra.config.CFMetaData;
 -import org.apache.cassandra.config.ColumnDefinition;
 -import org.apache.cassandra.cql3.selection.Selectable;
 -import org.apache.cassandra.cql3.selection.Selector;
 -import org.apache.cassandra.cql3.selection.SimpleSelector;
  import org.apache.cassandra.db.marshal.AbstractType;
 -import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.db.marshal.UTF8Type;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.ObjectSizes;
  import org.apache.cassandra.utils.memory.AbstractAllocator;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/10505447/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/10505447/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/10505447/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
index 158110c,c4b43b8..48fa40b
--- a/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
@@@ -24,9 -24,10 +24,11 @@@ import java.util.concurrent.ThreadLocal
  import org.junit.Test;
  
  import junit.framework.Assert;
+ import org.apache.cassandra.db.marshal.AbstractType;
  import org.apache.cassandra.db.marshal.BytesType;
+ import org.apache.cassandra.db.marshal.UTF8Type;
  import org.apache.cassandra.utils.ByteBufferUtil;
 +import static org.junit.Assert.assertEquals;
  
  public class ColumnIdentifierTest
  {
@@@ -58,23 -59,27 +60,45 @@@
      {
          return v < 0 ? -1 : v > 0 ? 1 : 0;
      }
 +    
 +    @Test
 +    public void testMaybeQuote()
 +    {
 +        String unquotable = "a";
 +        assertEquals(unquotable, ColumnIdentifier.maybeQuote(unquotable));
 +        unquotable = "z4";
 +        assertEquals(unquotable, ColumnIdentifier.maybeQuote(unquotable));
 +        unquotable = "m_4_";
 +        assertEquals(unquotable, ColumnIdentifier.maybeQuote(unquotable));
 +        unquotable = "f__";
 +        assertEquals(unquotable, ColumnIdentifier.maybeQuote(unquotable));
 +        
 +        assertEquals("\"A\"", ColumnIdentifier.maybeQuote("A"));
 +        assertEquals("\"4b\"", ColumnIdentifier.maybeQuote("4b"));
 +        assertEquals("\"\"\"\"", ColumnIdentifier.maybeQuote("\""));
 +        assertEquals("\"\"\"a\"\"b\"\"\"", ColumnIdentifier.maybeQuote("\"a\"b\""));
 +    }
  
+     @Test
+     public void testInternedCache()
+     {
+         AbstractType<?> utf8Type = UTF8Type.instance;
+         AbstractType<?> bytesType = BytesType.instance;
+ 
+         byte[] bytes = new byte [] { 0x63, (byte) 0x32 };
+         String text = "c2"; // the UTF-8 encoding of this string is the same as bytes, 0x630x32
+ 
+         ColumnIdentifier c1 = ColumnIdentifier.getInterned(ByteBuffer.wrap(bytes), bytesType);
+         ColumnIdentifier c2 = ColumnIdentifier.getInterned(utf8Type, utf8Type.fromString(text), text);
+         ColumnIdentifier c3 = ColumnIdentifier.getInterned(text, true);
+ 
+         Assert.assertTrue(c1.isInterned());
+         Assert.assertTrue(c2.isInterned());
+         Assert.assertTrue(c3.isInterned());
+ 
+         Assert.assertEquals("6332", c1.toString());
+         Assert.assertEquals(text, c2.toString());
+         Assert.assertEquals(text, c3.toString());
+     }
+ 
  }


[2/3] cassandra git commit: Extend ColumnIdentifier.internedInstances key to include the type that generated the byte buffer

Posted by st...@apache.org.
Extend ColumnIdentifier.internedInstances key to include the type that generated the byte buffer

patch by Stefania Alborghetti; reviewed by Aleksey Yeschenko for CASSANDRA-12516


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

Branch: refs/heads/trunk
Commit: 4bc3aa93337474a29bc4ad43ff4438755c9c7ab8
Parents: 85ed48a
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Fri Sep 2 16:28:35 2016 +0800
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Tue Sep 20 09:26:38 2016 +0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/cql3/ColumnIdentifier.java | 46 +++++++++++++++++---
 .../apache/cassandra/schema/SchemaKeyspace.java |  6 ++-
 .../utils/NativeSSTableLoaderClient.java        |  6 ++-
 .../cassandra/cql3/ColumnIdentifierTest.java    | 24 ++++++++++
 5 files changed, 72 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4bc3aa93/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5fd8e5e..b61c76c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.10
+ * Extend ColumnIdentifier.internedInstances key to include the type that generated the byte buffer (CASSANDRA-12516)
  * Backport CASSANDRA-10756 (race condition in NativeTransportService shutdown) (CASSANDRA-12472)
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4bc3aa93/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index afb65e1..1e25b23 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -60,7 +60,38 @@ public class ColumnIdentifier extends Selectable implements IMeasurableMemory, C
 
     private static final long EMPTY_SIZE = ObjectSizes.measure(new ColumnIdentifier(ByteBufferUtil.EMPTY_BYTE_BUFFER, "", false));
 
-    private static final ConcurrentMap<ByteBuffer, ColumnIdentifier> internedInstances = new MapMaker().weakValues().makeMap();
+    private static final ConcurrentMap<InternedKey, ColumnIdentifier> internedInstances = new MapMaker().weakValues().makeMap();
+
+    private static final class InternedKey
+    {
+        private final AbstractType<?> type;
+        private final ByteBuffer bytes;
+
+        InternedKey(AbstractType<?> type, ByteBuffer bytes)
+        {
+            this.type = type;
+            this.bytes = bytes;
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            InternedKey that = (InternedKey) o;
+            return bytes.equals(that.bytes) && type.equals(that.type);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return bytes.hashCode() + 31 * type.hashCode();
+        }
+    }
 
     private static long prefixComparison(ByteBuffer bytes)
     {
@@ -103,24 +134,25 @@ public class ColumnIdentifier extends Selectable implements IMeasurableMemory, C
 
     public static ColumnIdentifier getInterned(ByteBuffer bytes, AbstractType<?> type)
     {
-        return getInterned(bytes, type.getString(bytes));
+        return getInterned(type, bytes, type.getString(bytes));
     }
 
     public static ColumnIdentifier getInterned(String rawText, boolean keepCase)
     {
         String text = keepCase ? rawText : rawText.toLowerCase(Locale.US);
         ByteBuffer bytes = ByteBufferUtil.bytes(text);
-        return getInterned(bytes, text);
+        return getInterned(UTF8Type.instance, bytes, text);
     }
 
-    public static ColumnIdentifier getInterned(ByteBuffer bytes, String text)
+    public static ColumnIdentifier getInterned(AbstractType<?> type, ByteBuffer bytes, String text)
     {
-        ColumnIdentifier id = internedInstances.get(bytes);
+        InternedKey key = new InternedKey(type, bytes);
+        ColumnIdentifier id = internedInstances.get(key);
         if (id != null)
             return id;
 
         ColumnIdentifier created = new ColumnIdentifier(bytes, text, true);
-        ColumnIdentifier previous = internedInstances.putIfAbsent(bytes, created);
+        ColumnIdentifier previous = internedInstances.putIfAbsent(key, created);
         return previous == null ? created : previous;
     }
 
@@ -246,7 +278,7 @@ public class ColumnIdentifier extends Selectable implements IMeasurableMemory, C
                 if (def.name.bytes.equals(bufferName))
                     return def.name;
             }
-            return getInterned(thriftColumnNameType.fromString(rawText), text);
+            return getInterned(thriftColumnNameType, thriftColumnNameType.fromString(rawText), text);
         }
 
         public boolean processesSelection()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4bc3aa93/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index e3756ec..84a5e13 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -1001,8 +1001,6 @@ public final class SchemaKeyspace
         String keyspace = row.getString("keyspace_name");
         String table = row.getString("table_name");
 
-        ColumnIdentifier name = ColumnIdentifier.getInterned(row.getBytes("column_name_bytes"), row.getString("column_name"));
-
         ColumnDefinition.Kind kind = ColumnDefinition.Kind.valueOf(row.getString("kind").toUpperCase());
 
         int position = row.getInt("position");
@@ -1012,6 +1010,10 @@ public final class SchemaKeyspace
         if (order == ClusteringOrder.DESC)
             type = ReversedType.getInstance(type);
 
+        ColumnIdentifier name = ColumnIdentifier.getInterned(type,
+                                                             row.getBytes("column_name_bytes"),
+                                                             row.getString("column_name"));
+
         return new ColumnDefinition(keyspace, table, name, type, position, kind);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4bc3aa93/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
index 5bcbcf7..4c6b12e 100644
--- a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
+++ b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
@@ -198,13 +198,15 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
 
     private static ColumnDefinition createDefinitionFromRow(Row row, String keyspace, String table, Types types)
     {
-        ColumnIdentifier name = ColumnIdentifier.getInterned(row.getBytes("column_name_bytes"), row.getString("column_name"));
-
         ClusteringOrder order = ClusteringOrder.valueOf(row.getString("clustering_order").toUpperCase());
         AbstractType<?> type = CQLTypeParser.parse(keyspace, row.getString("type"), types);
         if (order == ClusteringOrder.DESC)
             type = ReversedType.getInstance(type);
 
+        ColumnIdentifier name = ColumnIdentifier.getInterned(type,
+                                                             row.getBytes("column_name_bytes"),
+                                                             row.getString("column_name"));
+
         int position = row.getInt("position");
         ColumnDefinition.Kind kind = ColumnDefinition.Kind.valueOf(row.getString("kind").toUpperCase());
         return new ColumnDefinition(keyspace, table, name, type, position, kind);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4bc3aa93/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java b/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
index c287883..c4b43b8 100644
--- a/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
@@ -24,7 +24,9 @@ import java.util.concurrent.ThreadLocalRandom;
 import org.junit.Test;
 
 import junit.framework.Assert;
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class ColumnIdentifierTest
@@ -58,4 +60,26 @@ public class ColumnIdentifierTest
         return v < 0 ? -1 : v > 0 ? 1 : 0;
     }
 
+    @Test
+    public void testInternedCache()
+    {
+        AbstractType<?> utf8Type = UTF8Type.instance;
+        AbstractType<?> bytesType = BytesType.instance;
+
+        byte[] bytes = new byte [] { 0x63, (byte) 0x32 };
+        String text = "c2"; // the UTF-8 encoding of this string is the same as bytes, 0x630x32
+
+        ColumnIdentifier c1 = ColumnIdentifier.getInterned(ByteBuffer.wrap(bytes), bytesType);
+        ColumnIdentifier c2 = ColumnIdentifier.getInterned(utf8Type, utf8Type.fromString(text), text);
+        ColumnIdentifier c3 = ColumnIdentifier.getInterned(text, true);
+
+        Assert.assertTrue(c1.isInterned());
+        Assert.assertTrue(c2.isInterned());
+        Assert.assertTrue(c3.isInterned());
+
+        Assert.assertEquals("6332", c1.toString());
+        Assert.assertEquals(text, c2.toString());
+        Assert.assertEquals(text, c3.toString());
+    }
+
 }