update caches to use byte[] keys to reduce memory overhead patch by vijay; reviewed by jbellis for CASSANDRA-3966
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/63f6c843 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/63f6c843 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/63f6c843 Branch: refs/heads/cassandra-1.1 Commit: 63f6c8431918e39a50023613102bf8dc3c9768da Parents: 0a8e978 Author: Jonathan Ellis <[email protected]> Authored: Wed Apr 4 14:27:18 2012 -0500 Committer: Jonathan Ellis <[email protected]> Committed: Wed Apr 4 14:27:18 2012 -0500 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../apache/cassandra/cache/AutoSavingCache.java | 6 +- src/java/org/apache/cassandra/cache/CacheKey.java | 5 +- .../org/apache/cassandra/cache/KeyCacheKey.java | 59 ++++++++++++--- .../org/apache/cassandra/cache/RowCacheKey.java | 57 +++++++------- .../org/apache/cassandra/utils/ByteBufferUtil.java | 6 ++ .../apache/cassandra/cache/CacheProviderTest.java | 17 ++++ 7 files changed, 105 insertions(+), 46 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/63f6c843/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 6fd478a..631c774 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 1.1.1-dev + * update caches to use byte[] keys to reduce memory overhead (CASSANDRA-3966) * add column limit to cli (CASSANDRA-3012, 4098) * clean up and optimize DataOutputBuffer, used by CQL compression and CompositeType (CASSANDRA-4072) http://git-wip-us.apache.org/repos/asf/cassandra/blob/63f6c843/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 e2901a7..aff7aa8 100644 --- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java +++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java @@ -39,7 +39,6 @@ import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.io.util.SequentialWriter; import org.apache.cassandra.service.CacheService; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.WrappedRunnable; import org.apache.cassandra.utils.Pair; @@ -237,9 +236,8 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K writers.put(path, writer); } - ByteBuffer bytes = key.serializeForStorage(); - ByteBufferUtil.writeWithLength(bytes, writer.stream); - bytesWritten += bytes.remaining(); + key.write(writer.stream); + bytesWritten += key.serializedSize(); } } finally http://git-wip-us.apache.org/repos/asf/cassandra/blob/63f6c843/src/java/org/apache/cassandra/cache/CacheKey.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/cache/CacheKey.java b/src/java/org/apache/cassandra/cache/CacheKey.java index e8f0caf..579ab03 100644 --- a/src/java/org/apache/cassandra/cache/CacheKey.java +++ b/src/java/org/apache/cassandra/cache/CacheKey.java @@ -19,7 +19,8 @@ */ package org.apache.cassandra.cache; -import java.nio.ByteBuffer; +import java.io.DataOutputStream; +import java.io.IOException; import org.apache.cassandra.utils.Pair; @@ -28,7 +29,7 @@ public interface CacheKey /** * @return Serialized part of the key which should be persisted */ - public ByteBuffer serializeForStorage(); + public void write(DataOutputStream out) throws IOException; /** * @return The size of the serialized key http://git-wip-us.apache.org/repos/asf/cassandra/blob/63f6c843/src/java/org/apache/cassandra/cache/KeyCacheKey.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/cache/KeyCacheKey.java b/src/java/org/apache/cassandra/cache/KeyCacheKey.java index 7f6523d..0c75edb 100644 --- a/src/java/org/apache/cassandra/cache/KeyCacheKey.java +++ b/src/java/org/apache/cassandra/cache/KeyCacheKey.java @@ -19,40 +19,75 @@ */ package org.apache.cassandra.cache; +import java.io.DataOutputStream; +import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.charset.CharacterCodingException; +import java.util.Arrays; +import org.apache.cassandra.db.DBConstants; import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.Pair; -public class KeyCacheKey extends Pair<Descriptor, ByteBuffer> implements CacheKey +import com.google.common.base.Objects; + +public class KeyCacheKey implements CacheKey { + private final Descriptor desc; + private final byte[] key; + public KeyCacheKey(Descriptor desc, ByteBuffer key) { - super(desc, key); + this.desc = desc; + this.key = ByteBufferUtil.getArray(key); + assert this.key != null; } - public ByteBuffer serializeForStorage() + public void write(DataOutputStream out) throws IOException { - ByteBuffer bytes = ByteBuffer.allocate(serializedSize()); - - bytes.put(right.slice()); - bytes.rewind(); - - return bytes; + ByteBufferUtil.writeWithLength(key, out); } public Pair<String, String> getPathInfo() { - return new Pair<String, String>(left.ksname, left.cfname); + return new Pair<String, String>(desc.ksname, desc.cfname); } public int serializedSize() { - return right.remaining(); + return key.length + DBConstants.intSize; } public String toString() { - return String.format("KeyCacheKey(descriptor:%s, key:%s)", left, right); + try + { + return String.format("KeyCacheKey(descriptor:%s, key:%s)", desc, ByteBufferUtil.string(ByteBuffer.wrap(key))); + } + catch (CharacterCodingException e) + { + throw new AssertionError(e); + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + KeyCacheKey that = (KeyCacheKey) o; + + if (desc != null ? !desc.equals(that.desc) : that.desc != null) return false; + return Arrays.equals(key, that.key); + } + + @Override + public int hashCode() + { + int result = desc != null ? desc.hashCode() : 0; + result = 31 * result + (key != null ? Arrays.hashCode(key) : 0); + return result; } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/63f6c843/src/java/org/apache/cassandra/cache/RowCacheKey.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/cache/RowCacheKey.java b/src/java/org/apache/cassandra/cache/RowCacheKey.java index 9bfc40b..ad2d2e1 100644 --- a/src/java/org/apache/cassandra/cache/RowCacheKey.java +++ b/src/java/org/apache/cassandra/cache/RowCacheKey.java @@ -19,33 +19,38 @@ */ package org.apache.cassandra.cache; +import java.io.DataOutputStream; +import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; import org.apache.cassandra.config.Schema; +import org.apache.cassandra.db.DBConstants; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Pair; -import org.apache.commons.lang.builder.HashCodeBuilder; public class RowCacheKey implements CacheKey, Comparable<RowCacheKey> { public final int cfId; - public final ByteBuffer key; + public final byte[] key; public RowCacheKey(int cfId, DecoratedKey key) { - this.cfId = cfId; - this.key = key.key; + this(cfId, key.key); } - public ByteBuffer serializeForStorage() + public RowCacheKey(int cfId, ByteBuffer key) { - ByteBuffer bytes = ByteBuffer.allocate(serializedSize()); - - bytes.put(key.slice()); - bytes.rewind(); + this.cfId = cfId; + this.key = ByteBufferUtil.getArray(key); + assert this.key != null; + } - return bytes; + public void write(DataOutputStream out) throws IOException + { + ByteBufferUtil.writeWithLength(key, out); } public Pair<String, String> getPathInfo() @@ -55,36 +60,32 @@ public class RowCacheKey implements CacheKey, Comparable<RowCacheKey> public int serializedSize() { - return key.remaining(); + return key.length + DBConstants.intSize; } @Override - public int hashCode() + public boolean equals(Object o) { - return new HashCodeBuilder(131, 56337) - .append(cfId) - .append(key).toHashCode(); + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + RowCacheKey that = (RowCacheKey) o; + + if (cfId != that.cfId) return false; + return Arrays.equals(key, that.key); } @Override - public boolean equals(Object obj) + public int hashCode() { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - - RowCacheKey otherKey = (RowCacheKey) obj; - - return cfId == otherKey.cfId && key.equals(otherKey.key); + int result = cfId; + result = 31 * result + (key != null ? Arrays.hashCode(key) : 0); + return result; } - @Override public int compareTo(RowCacheKey otherKey) { - return (cfId < otherKey.cfId) ? -1 : ((cfId == otherKey.cfId) ? ByteBufferUtil.compareUnsigned(key, otherKey.key) : 1); + return (cfId < otherKey.cfId) ? -1 : ((cfId == otherKey.cfId) ? FBUtilities.compareUnsigned(key, otherKey.key, 0, 0, key.length, otherKey.key.length) : 1); } @Override http://git-wip-us.apache.org/repos/asf/cassandra/blob/63f6c843/src/java/org/apache/cassandra/utils/ByteBufferUtil.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java index 4dc11d8..93f28df 100644 --- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java +++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java @@ -311,6 +311,12 @@ public class ByteBufferUtil write(bytes, out); // writing data bytes to output source } + public static void writeWithLength(byte[] bytes, DataOutput out) throws IOException + { + out.writeInt(bytes.length); + out.write(bytes); + } + public static void write(ByteBuffer buffer, DataOutput out) throws IOException { if (buffer.hasArray()) http://git-wip-us.apache.org/repos/asf/cassandra/blob/63f6c843/test/unit/org/apache/cassandra/cache/CacheProviderTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java index 5ff1411..bbecdcd 100644 --- a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java +++ b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java @@ -21,6 +21,7 @@ package org.apache.cassandra.cache; */ +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import org.junit.Test; @@ -115,4 +116,20 @@ public class CacheProviderTest extends SchemaLoader simpleCase(cf, cache); concurrentCase(cf, cache); } + + @Test + public void testKeys() + { + byte[] b1 = {1, 2, 3, 4}; + RowCacheKey key1 = new RowCacheKey(123, ByteBuffer.wrap(b1)); + byte[] b2 = {1, 2, 3, 4}; + RowCacheKey key2 = new RowCacheKey(123, ByteBuffer.wrap(b2)); + assertEquals(key1, key2); + assertEquals(key1.hashCode(), key2.hashCode()); + + byte[] b3 = {1, 2, 3, 5}; + RowCacheKey key3 = new RowCacheKey(123, ByteBuffer.wrap(b3)); + assertNotSame(key1, key3); + assertNotSame(key1.hashCode(), key3.hashCode()); + } }
