diff --git a/src/java/org/apache/cassandra/db/NativeClustering.java b/src/java/org/apache/cassandra/db/NativeClustering.java index e7c7e8893a17..e83ded06c6e0 100644 --- a/src/java/org/apache/cassandra/db/NativeClustering.java +++ b/src/java/org/apache/cassandra/db/NativeClustering.java @@ -28,6 +28,7 @@ import org.apache.cassandra.utils.concurrent.OpOrder; import org.apache.cassandra.utils.memory.HeapCloner; import org.apache.cassandra.utils.memory.MemoryUtil; +import org.apache.cassandra.utils.memory.NativeEndianMemoryUtil; import org.apache.cassandra.utils.memory.NativeAllocator; public class NativeClustering implements Clustering @@ -50,30 +51,30 @@ public NativeClustering(NativeAllocator allocator, OpOrder.Group writeOp, Cluste peer = allocator.allocate(metadataSize + dataSize + bitmapSize, writeOp); long bitmapStart = peer + metadataSize; - MemoryUtil.setShort(peer, (short) count); - MemoryUtil.setShort(peer + (metadataSize - 2), (short) dataSize); // goes at the end of the other offsets + NativeEndianMemoryUtil.setShort(peer, (short) count); + NativeEndianMemoryUtil.setShort(peer + (metadataSize - 2), (short) dataSize); // goes at the end of the other offsets - MemoryUtil.setByte(bitmapStart, bitmapSize, (byte) 0); + NativeEndianMemoryUtil.setByte(bitmapStart, bitmapSize, (byte) 0); long dataStart = peer + metadataSize + bitmapSize; int dataOffset = 0; for (int i = 0 ; i < count ; i++) { - MemoryUtil.setShort(peer + 2 + i * 2, (short) dataOffset); + NativeEndianMemoryUtil.setShort(peer + 2 + i * 2, (short) dataOffset); ByteBuffer value = clustering.bufferAt(i); if (value == null) { long boffset = bitmapStart + (i >>> 3); - int b = MemoryUtil.getByte(boffset); + int b = NativeEndianMemoryUtil.getByte(boffset); b |= 1 << (i & 7); - MemoryUtil.setByte(boffset, (byte) b); + NativeEndianMemoryUtil.setByte(boffset, (byte) b); continue; } assert value.order() == ByteOrder.BIG_ENDIAN; int size = value.remaining(); - MemoryUtil.setBytes(dataStart + dataOffset, value); + NativeEndianMemoryUtil.setBytes(dataStart + dataOffset, value); dataOffset += size; } } @@ -90,13 +91,13 @@ public ClusteringPrefix clustering() public int size() { - return MemoryUtil.getShort(peer); + return NativeEndianMemoryUtil.getUnsignedShort(peer); } public int dataSize() { int dataSizeOffset = (size() * 2) + 2; // metadataSize - 2 - return MemoryUtil.getShort(peer + dataSizeOffset); + return NativeEndianMemoryUtil.getUnsignedShort(peer + dataSizeOffset); } public ByteBuffer get(int i) @@ -109,12 +110,12 @@ public ByteBuffer get(int i) int metadataSize = (size * 2) + 4; int bitmapSize = ((size + 7) >>> 3); long bitmapStart = peer + metadataSize; - int b = MemoryUtil.getByte(bitmapStart + (i >>> 3)); + int b = NativeEndianMemoryUtil.getByte(bitmapStart + (i >>> 3)); if ((b & (1 << (i & 7))) != 0) return null; - int startOffset = MemoryUtil.getShort(peer + 2 + i * 2); - int endOffset = MemoryUtil.getShort(peer + 4 + i * 2); + int startOffset = NativeEndianMemoryUtil.getUnsignedShort(peer + 2 + i * 2); + int endOffset = NativeEndianMemoryUtil.getUnsignedShort(peer + 4 + i * 2); return MemoryUtil.getByteBuffer(bitmapStart + bitmapSize + startOffset, endOffset - startOffset, ByteOrder.BIG_ENDIAN); diff --git a/src/java/org/apache/cassandra/db/NativeDecoratedKey.java b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java index bc149084d852..76b2367ae210 100644 --- a/src/java/org/apache/cassandra/db/NativeDecoratedKey.java +++ b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java @@ -26,6 +26,7 @@ import org.apache.cassandra.utils.concurrent.OpOrder; import org.apache.cassandra.utils.memory.MemoryUtil; import org.apache.cassandra.utils.memory.NativeAllocator; +import org.apache.cassandra.utils.memory.NativeEndianMemoryUtil; public class NativeDecoratedKey extends DecoratedKey { @@ -39,7 +40,7 @@ public NativeDecoratedKey(Token token, NativeAllocator allocator, OpOrder.Group int size = key.remaining(); this.peer = allocator.allocate(4 + size, writeOp); - MemoryUtil.setInt(peer, size); + NativeEndianMemoryUtil.setInt(peer, size); MemoryUtil.setBytes(peer + 4, key); } @@ -50,14 +51,14 @@ public NativeDecoratedKey(Token token, NativeAllocator allocator, OpOrder.Group int size = keyBytes.length; this.peer = allocator.allocate(4 + size, writeOp); - MemoryUtil.setInt(peer, size); + NativeEndianMemoryUtil.setInt(peer, size); MemoryUtil.setBytes(peer + 4, keyBytes, 0, size); } @Inline int length() { - return MemoryUtil.getInt(peer); + return NativeEndianMemoryUtil.getInt(peer); } @Inline @@ -75,7 +76,7 @@ public ByteBuffer getKey() @Override public int getKeyLength() { - return MemoryUtil.getInt(peer); + return NativeEndianMemoryUtil.getInt(peer); } @Override diff --git a/src/java/org/apache/cassandra/db/rows/NativeCell.java b/src/java/org/apache/cassandra/db/rows/NativeCell.java index b8eaa557ce39..9a191ebab923 100644 --- a/src/java/org/apache/cassandra/db/rows/NativeCell.java +++ b/src/java/org/apache/cassandra/db/rows/NativeCell.java @@ -28,6 +28,7 @@ import org.apache.cassandra.utils.concurrent.OpOrder; import org.apache.cassandra.utils.memory.MemoryUtil; import org.apache.cassandra.utils.memory.NativeAllocator; +import org.apache.cassandra.utils.memory.NativeEndianMemoryUtil; public class NativeCell extends AbstractCell { @@ -101,11 +102,11 @@ public NativeCell(NativeAllocator allocator, // cellpath? : timestamp : ttl : localDeletionTime : length : : [cell path length] : [] peer = allocator.allocate((int) size, writeOp); - MemoryUtil.setByte(peer + HAS_CELLPATH, (byte)(path == null ? 0 : 1)); - MemoryUtil.setLong(peer + TIMESTAMP, timestamp); - MemoryUtil.setInt(peer + TTL, ttl); - MemoryUtil.setInt(peer + DELETION, localDeletionTimeUnsignedInteger); - MemoryUtil.setInt(peer + LENGTH, value.remaining()); + NativeEndianMemoryUtil.setByte(peer + HAS_CELLPATH, (byte)(path == null ? 0 : 1)); + NativeEndianMemoryUtil.setLong(peer + TIMESTAMP, timestamp); + NativeEndianMemoryUtil.setInt(peer + TTL, ttl); + NativeEndianMemoryUtil.setInt(peer + DELETION, localDeletionTimeUnsignedInteger); + NativeEndianMemoryUtil.setInt(peer + LENGTH, value.remaining()); MemoryUtil.setBytes(peer + VALUE, value); if (path != null) @@ -114,7 +115,7 @@ public NativeCell(NativeAllocator allocator, assert pathbuffer.order() == ByteOrder.BIG_ENDIAN; long offset = peer + VALUE + value.remaining(); - MemoryUtil.setInt(offset, pathbuffer.remaining()); + NativeEndianMemoryUtil.setInt(offset, pathbuffer.remaining()); MemoryUtil.setBytes(offset + 4, pathbuffer); } } @@ -126,17 +127,17 @@ private static long offHeapSizeWithoutPath(int length) public long timestamp() { - return MemoryUtil.getLong(peer + TIMESTAMP); + return NativeEndianMemoryUtil.getLong(peer + TIMESTAMP); } public int ttl() { - return MemoryUtil.getInt(peer + TTL); + return NativeEndianMemoryUtil.getInt(peer + TTL); } public ByteBuffer value()// FIXME: add native accessor { - int length = MemoryUtil.getInt(peer + LENGTH); + int length = NativeEndianMemoryUtil.getInt(peer + LENGTH); return MemoryUtil.getByteBuffer(peer + VALUE, length, ByteOrder.BIG_ENDIAN); } @@ -147,7 +148,7 @@ public ValueAccessor accessor() public int valueSize() { - return MemoryUtil.getInt(peer + LENGTH); + return NativeEndianMemoryUtil.getInt(peer + LENGTH); } public CellPath path() @@ -155,8 +156,8 @@ public CellPath path() if (!hasPath()) return null; - long offset = peer + VALUE + MemoryUtil.getInt(peer + LENGTH); - int size = MemoryUtil.getInt(offset); + long offset = peer + VALUE + NativeEndianMemoryUtil.getInt(peer + LENGTH); + int size = NativeEndianMemoryUtil.getInt(offset); return CellPath.create(MemoryUtil.getByteBuffer(offset + 4, size, ByteOrder.BIG_ENDIAN)); } @@ -200,20 +201,20 @@ public long unsharedHeapSizeExcludingData() public long offHeapSize() { - long size = offHeapSizeWithoutPath(MemoryUtil.getInt(peer + LENGTH)); + long size = offHeapSizeWithoutPath(NativeEndianMemoryUtil.getInt(peer + LENGTH)); if (hasPath()) - size += 4 + MemoryUtil.getInt(peer + size); + size += 4 + NativeEndianMemoryUtil.getInt(peer + size); return size; } private boolean hasPath() { - return MemoryUtil.getByte(peer+ HAS_CELLPATH) != 0; + return NativeEndianMemoryUtil.getByte(peer + HAS_CELLPATH) != 0; } @Override protected int localDeletionTimeAsUnsignedInt() { - return MemoryUtil.getInt(peer + DELETION); + return NativeEndianMemoryUtil.getInt(peer + DELETION); } } diff --git a/src/java/org/apache/cassandra/io/sstable/indexsummary/IndexSummary.java b/src/java/org/apache/cassandra/io/sstable/indexsummary/IndexSummary.java index ec5248c70f02..39377cc01fe2 100644 --- a/src/java/org/apache/cassandra/io/sstable/indexsummary/IndexSummary.java +++ b/src/java/org/apache/cassandra/io/sstable/indexsummary/IndexSummary.java @@ -456,6 +456,18 @@ public IndexSummary deserialize(T in, IP entries.free(); throw ioe; } + + // Before 5.0 offsets were written using Native Endian, now they are stored as Little Endian, + // so we apply a heuristic here to detect + // if the loading index summary was created on a Big Endian machine using Native Endian format + if (offsets.size() > 0) + { + int offset = offsets.getInt(0); + int offsetReversed = Integer.reverseBytes(offset); + if (offsetReversed > 0 && offset > offsetReversed || offset - offsets.size() < 0) + throw new IOException(String.format("Rebuilding index summary because offset value (%d) at position: %d " + + "is Big Endian while Little Endian is expected", offset, 0)); + } // our on-disk representation treats the offsets and the summary data as one contiguous structure, // in which the offsets are based from the start of the structure. i.e., if the offsets occupy // X bytes, the value of the first offset will be X. In memory we split the two regions up, so that diff --git a/src/java/org/apache/cassandra/io/util/Memory.java b/src/java/org/apache/cassandra/io/util/Memory.java index 6913caf4cf2e..512acd59d7a7 100644 --- a/src/java/org/apache/cassandra/io/util/Memory.java +++ b/src/java/org/apache/cassandra/io/util/Memory.java @@ -22,14 +22,15 @@ import net.nicoulaj.compilecommand.annotations.Inline; -import org.apache.cassandra.utils.Architecture; import org.apache.cassandra.utils.FastByteOperations; import org.apache.cassandra.utils.concurrent.Ref; +import org.apache.cassandra.utils.memory.LittleEndianMemoryUtil; import org.apache.cassandra.utils.memory.MemoryUtil; import sun.misc.Unsafe; /** * An off-heap region of memory that must be manually free'd when no longer needed. + * It uses Little Endian (LE). */ public class Memory implements AutoCloseable, ReadableMemory { @@ -90,7 +91,7 @@ public static Memory allocate(long bytes) public void setByte(long offset, byte b) { checkBounds(offset, offset + 1); - unsafe.putByte(peer + offset, b); + LittleEndianMemoryUtil.setByte(peer + offset, b); } public void setMemory(long offset, long bytes, byte b) @@ -103,86 +104,13 @@ public void setMemory(long offset, long bytes, byte b) public void setLong(long offset, long l) { checkBounds(offset, offset + 8); - if (Architecture.IS_UNALIGNED) - unsafe.putLong(peer + offset, Architecture.BIG_ENDIAN ? Long.reverseBytes(l) : l); - else - putLongByByte(peer + offset, l); - } - - private void putLongByByte(long address, long value) - { - if (Architecture.BIG_ENDIAN) - { - unsafe.putByte(address, (byte) (value >> 56)); - unsafe.putByte(address + 1, (byte) (value >> 48)); - unsafe.putByte(address + 2, (byte) (value >> 40)); - unsafe.putByte(address + 3, (byte) (value >> 32)); - unsafe.putByte(address + 4, (byte) (value >> 24)); - unsafe.putByte(address + 5, (byte) (value >> 16)); - unsafe.putByte(address + 6, (byte) (value >> 8)); - unsafe.putByte(address + 7, (byte) (value)); - } - else - { - unsafe.putByte(address + 7, (byte) (value >> 56)); - unsafe.putByte(address + 6, (byte) (value >> 48)); - unsafe.putByte(address + 5, (byte) (value >> 40)); - unsafe.putByte(address + 4, (byte) (value >> 32)); - unsafe.putByte(address + 3, (byte) (value >> 24)); - unsafe.putByte(address + 2, (byte) (value >> 16)); - unsafe.putByte(address + 1, (byte) (value >> 8)); - unsafe.putByte(address, (byte) (value)); - } + LittleEndianMemoryUtil.setLong(peer + offset, l); } public void setInt(long offset, int l) { checkBounds(offset, offset + 4); - if (Architecture.IS_UNALIGNED) - unsafe.putInt(peer + offset, Architecture.BIG_ENDIAN ? Integer.reverseBytes(l) : l); - else - putIntByByte(peer + offset, l); - } - - private void putIntByByte(long address, int value) - { - if (Architecture.BIG_ENDIAN) - { - unsafe.putByte(address, (byte) (value >> 24)); - unsafe.putByte(address + 1, (byte) (value >> 16)); - unsafe.putByte(address + 2, (byte) (value >> 8)); - unsafe.putByte(address + 3, (byte) (value)); - } - else - { - unsafe.putByte(address + 3, (byte) (value >> 24)); - unsafe.putByte(address + 2, (byte) (value >> 16)); - unsafe.putByte(address + 1, (byte) (value >> 8)); - unsafe.putByte(address, (byte) (value)); - } - } - - public void setShort(long offset, short l) - { - checkBounds(offset, offset + 2); - if (Architecture.IS_UNALIGNED) - unsafe.putShort(peer + offset, Architecture.BIG_ENDIAN ? Short.reverseBytes(l) : l); - else - putShortByByte(peer + offset, l); - } - - private void putShortByByte(long address, short value) - { - if (Architecture.BIG_ENDIAN) - { - unsafe.putByte(address, (byte) (value >> 8)); - unsafe.putByte(address + 1, (byte) (value)); - } - else - { - unsafe.putByte(address + 1, (byte) (value >> 8)); - unsafe.putByte(address, (byte) (value)); - } + LittleEndianMemoryUtil.setInt(peer + offset, l); } public void setBytes(long memoryOffset, ByteBuffer buffer) @@ -230,69 +158,19 @@ else if (count == 0) public byte getByte(long offset) { checkBounds(offset, offset + 1); - return unsafe.getByte(peer + offset); + return LittleEndianMemoryUtil.getByte(peer + offset); } public long getLong(long offset) { checkBounds(offset, offset + 8); - if (Architecture.IS_UNALIGNED) - return Architecture.BIG_ENDIAN ? Long.reverseBytes(unsafe.getLong(peer+offset)) : unsafe.getLong(peer+offset); - else - return getLongByByte(peer + offset); - } - - private long getLongByByte(long address) - { - if (Architecture.BIG_ENDIAN) - { - return (((long) unsafe.getByte(address ) ) << 56) | - (((long) unsafe.getByte(address + 1) & 0xff) << 48) | - (((long) unsafe.getByte(address + 2) & 0xff) << 40) | - (((long) unsafe.getByte(address + 3) & 0xff) << 32) | - (((long) unsafe.getByte(address + 4) & 0xff) << 24) | - (((long) unsafe.getByte(address + 5) & 0xff) << 16) | - (((long) unsafe.getByte(address + 6) & 0xff) << 8) | - (((long) unsafe.getByte(address + 7) & 0xff) ); - } - else - { - return (((long) unsafe.getByte(address + 7) ) << 56) | - (((long) unsafe.getByte(address + 6) & 0xff) << 48) | - (((long) unsafe.getByte(address + 5) & 0xff) << 40) | - (((long) unsafe.getByte(address + 4) & 0xff) << 32) | - (((long) unsafe.getByte(address + 3) & 0xff) << 24) | - (((long) unsafe.getByte(address + 2) & 0xff) << 16) | - (((long) unsafe.getByte(address + 1) & 0xff) << 8) | - (((long) unsafe.getByte(address ) & 0xff) ); - } + return LittleEndianMemoryUtil.getLong(peer + offset); } public int getInt(long offset) { checkBounds(offset, offset + 4); - if (Architecture.IS_UNALIGNED) - return Architecture.BIG_ENDIAN ? Integer.reverseBytes(unsafe.getInt(peer+offset)) : unsafe.getInt(peer+offset); - else - return getIntByByte(peer + offset); - } - - private int getIntByByte(long address) - { - if (Architecture.BIG_ENDIAN) - { - return ((unsafe.getByte(address ) ) << 24) | - ((unsafe.getByte(address + 1) & 0xff) << 16) | - ((unsafe.getByte(address + 2) & 0xff) << 8 ) | - ((unsafe.getByte(address + 3) & 0xff) ); - } - else - { - return ((unsafe.getByte(address + 3) ) << 24) | - ((unsafe.getByte(address + 2) & 0xff) << 16) | - ((unsafe.getByte(address + 1) & 0xff) << 8) | - ((unsafe.getByte(address ) & 0xff) ); - } + return LittleEndianMemoryUtil.getInt(peer + offset); } /** @@ -378,18 +256,18 @@ public ByteBuffer[] asByteBuffers(long offset, long length) int size = (int) (size() / result.length); for (int i = 0 ; i < result.length - 1 ; i++) { - result[i] = MemoryUtil.getByteBuffer(peer + offset, size); + result[i] = LittleEndianMemoryUtil.getByteBuffer(peer + offset, size); offset += size; length -= size; } - result[result.length - 1] = MemoryUtil.getByteBuffer(peer + offset, (int) length); + result[result.length - 1] = LittleEndianMemoryUtil.getByteBuffer(peer + offset, (int) length); return result; } public ByteBuffer asByteBuffer(long offset, int length) { checkBounds(offset, offset + length); - return MemoryUtil.getByteBuffer(peer + offset, length); + return LittleEndianMemoryUtil.getByteBuffer(peer + offset, length); } // MUST provide a buffer created via MemoryUtil.getHollowDirectByteBuffer() diff --git a/src/java/org/apache/cassandra/utils/memory/LittleEndianMemoryUtil.java b/src/java/org/apache/cassandra/utils/memory/LittleEndianMemoryUtil.java new file mode 100644 index 000000000000..2553b9314984 --- /dev/null +++ b/src/java/org/apache/cassandra/utils/memory/LittleEndianMemoryUtil.java @@ -0,0 +1,146 @@ +/* + * 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.utils.memory; + + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.utils.Architecture; + +public class LittleEndianMemoryUtil extends MemoryUtil +{ + public static int getUnsignedShort(long address) + { + if (Architecture.IS_UNALIGNED || (address & 0b1) == 0L) + return (Architecture.BIG_ENDIAN ? Short.reverseBytes(unsafe.getShort(address)) : unsafe.getShort(address)) & 0xffff; + else + return getShortByByte(address) & 0xffff; + } + + public static int getInt(long address) + { + if (Architecture.IS_UNALIGNED || (address & 0b11) == 0L) + return Architecture.BIG_ENDIAN ? Integer.reverseBytes(unsafe.getInt(address)) : unsafe.getInt(address); + else + return getIntByByte(address); + } + + public static long getLong(long address) + { + if (Architecture.IS_UNALIGNED || (address & 0b111) == 0L) + return Architecture.BIG_ENDIAN ? Long.reverseBytes(unsafe.getLong(address)) : unsafe.getLong(address); + else + return getLongByByte(address); + } + + public static void setShort(long address, short s) + { + if (Architecture.IS_UNALIGNED || (address & 0b1) == 0L) + unsafe.putShort(address, Architecture.BIG_ENDIAN ? Short.reverseBytes(s) : s); + else + putShortByByte(address, s); + } + + public static void setInt(long address, int l) + { + if (Architecture.IS_UNALIGNED || (address & 0b11) == 0L) + unsafe.putInt(address, Architecture.BIG_ENDIAN ? Integer.reverseBytes(l) : l); + else + putIntByByte(address, l); + } + + public static void setLong(long address, long l) + { + if (Architecture.IS_UNALIGNED || (address & 0b111) == 0L) + unsafe.putLong(address, Architecture.BIG_ENDIAN ? Long.reverseBytes(l) : l); + else + putLongByByte(address, l); + } + + @VisibleForTesting + static long getLongByByte(long address) + { + return (((long) unsafe.getByte(address + 7) ) << 56) | + (((long) unsafe.getByte(address + 6) & 0xff) << 48) | + (((long) unsafe.getByte(address + 5) & 0xff) << 40) | + (((long) unsafe.getByte(address + 4) & 0xff) << 32) | + (((long) unsafe.getByte(address + 3) & 0xff) << 24) | + (((long) unsafe.getByte(address + 2) & 0xff) << 16) | + (((long) unsafe.getByte(address + 1) & 0xff) << 8) | + (((long) unsafe.getByte(address ) & 0xff) ); + } + + @VisibleForTesting + static int getIntByByte(long address) + { + return (((int) unsafe.getByte(address + 3) ) << 24) | + (((int) unsafe.getByte(address + 2) & 0xff) << 16) | + (((int) unsafe.getByte(address + 1) & 0xff) << 8) | + (((int) unsafe.getByte(address ) & 0xff) ); + } + + @VisibleForTesting + static int getShortByByte(long address) + { + return (((int) unsafe.getByte(address + 1) ) << 8) | + (((int) unsafe.getByte(address ) & 0xff) ); + } + + @VisibleForTesting + static void putLongByByte(long address, long value) + { + unsafe.putByte(address + 7, (byte) (value >> 56)); + unsafe.putByte(address + 6, (byte) (value >> 48)); + unsafe.putByte(address + 5, (byte) (value >> 40)); + unsafe.putByte(address + 4, (byte) (value >> 32)); + unsafe.putByte(address + 3, (byte) (value >> 24)); + unsafe.putByte(address + 2, (byte) (value >> 16)); + unsafe.putByte(address + 1, (byte) (value >> 8)); + unsafe.putByte(address , (byte) (value )); + } + + @VisibleForTesting + static void putIntByByte(long address, int value) + { + unsafe.putByte(address + 3, (byte) (value >> 24)); + unsafe.putByte(address + 2, (byte) (value >> 16)); + unsafe.putByte(address + 1, (byte) (value >> 8)); + unsafe.putByte(address , (byte) (value )); + } + + @VisibleForTesting + static void putShortByByte(long address, short value) + { + unsafe.putByte(address + 1, (byte) (value >> 8)); + unsafe.putByte(address , (byte) (value )); + } + + public static ByteBuffer getByteBuffer(long address, int length) + { + return getByteBuffer(address, length, ByteOrder.LITTLE_ENDIAN); + } + + public static ByteBuffer getHollowDirectByteBuffer() + { + return getHollowDirectByteBuffer(ByteOrder.LITTLE_ENDIAN); + } +} diff --git a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java index 453f3eda1ba3..86416c49a703 100644 --- a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java +++ b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java @@ -24,24 +24,20 @@ import com.sun.jna.Native; -import org.apache.cassandra.utils.Architecture; - import sun.misc.Unsafe; public abstract class MemoryUtil { private static final long UNSAFE_COPY_THRESHOLD = 1024 * 1024L; // copied from java.nio.Bits - private static final Unsafe unsafe; + protected static final Unsafe unsafe; private static final Class DIRECT_BYTE_BUFFER_CLASS, RO_DIRECT_BYTE_BUFFER_CLASS; private static final long DIRECT_BYTE_BUFFER_ADDRESS_OFFSET; private static final long DIRECT_BYTE_BUFFER_CAPACITY_OFFSET; private static final long DIRECT_BYTE_BUFFER_LIMIT_OFFSET; private static final long DIRECT_BYTE_BUFFER_POSITION_OFFSET; private static final long DIRECT_BYTE_BUFFER_ATTACHMENT_OFFSET; - private static final Class BYTE_BUFFER_CLASS; - private static final long BYTE_BUFFER_OFFSET_OFFSET; - private static final long BYTE_BUFFER_HB_OFFSET; + protected static final Class BYTE_BUFFER_CLASS; private static final long BYTE_ARRAY_BASE_OFFSET; static @@ -61,8 +57,6 @@ public abstract class MemoryUtil RO_DIRECT_BYTE_BUFFER_CLASS = ByteBuffer.allocateDirect(0).asReadOnlyBuffer().getClass(); clazz = ByteBuffer.allocate(0).getClass(); - BYTE_BUFFER_OFFSET_OFFSET = unsafe.objectFieldOffset(ByteBuffer.class.getDeclaredField("offset")); - BYTE_BUFFER_HB_OFFSET = unsafe.objectFieldOffset(ByteBuffer.class.getDeclaredField("hb")); BYTE_BUFFER_CLASS = clazz; BYTE_ARRAY_BASE_OFFSET = unsafe.arrayBaseOffset(byte[].class); @@ -104,56 +98,11 @@ public static void setByte(long address, int count, byte b) unsafe.setMemory(address, count, b); } - public static void setShort(long address, short s) - { - unsafe.putShort(address, Architecture.BIG_ENDIAN ? Short.reverseBytes(s) : s); - } - - public static void setInt(long address, int l) - { - if (Architecture.IS_UNALIGNED) - unsafe.putInt(address, Architecture.BIG_ENDIAN ? Integer.reverseBytes(l) : l); - else - putIntByByte(address, l); - } - - public static void setLong(long address, long l) - { - if (Architecture.IS_UNALIGNED) - unsafe.putLong(address, Architecture.BIG_ENDIAN ? Long.reverseBytes(l) : l); - else - putLongByByte(address, l); - } - public static byte getByte(long address) { return unsafe.getByte(address); } - public static int getShort(long address) - { - if (Architecture.IS_UNALIGNED) - return (Architecture.BIG_ENDIAN ? Short.reverseBytes(unsafe.getShort(address)) : unsafe.getShort(address)) & 0xffff; - else - return getShortByByte(address) & 0xffff; - } - - public static int getInt(long address) - { - if (Architecture.IS_UNALIGNED) - return Architecture.BIG_ENDIAN ? Integer.reverseBytes(unsafe.getInt(address)) : unsafe.getInt(address); - else - return getIntByByte(address); - } - - public static long getLong(long address) - { - if (Architecture.IS_UNALIGNED) - return Architecture.BIG_ENDIAN ? Long.reverseBytes(unsafe.getLong(address)) : unsafe.getLong(address); - else - return getLongByByte(address); - } - public static ByteBuffer getByteBuffer(long address, int length) { return getByteBuffer(address, length, ByteOrder.nativeOrder()); @@ -186,21 +135,6 @@ public static ByteBuffer getHollowDirectByteBuffer(ByteOrder order) return instance; } - public static ByteBuffer getHollowByteBuffer() - { - ByteBuffer instance; - try - { - instance = (ByteBuffer) unsafe.allocateInstance(BYTE_BUFFER_CLASS); - } - catch (InstantiationException e) - { - throw new AssertionError(e); - } - instance.order(ByteOrder.nativeOrder()); - return instance; - } - public static boolean isExactlyDirect(ByteBuffer buffer) { return buffer.getClass() == DIRECT_BYTE_BUFFER_CLASS; @@ -250,109 +184,6 @@ public static void setByteBufferCapacity(ByteBuffer instance, int capacity) unsafe.putInt(instance, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET, capacity); } - public static long getLongByByte(long address) - { - if (Architecture.BIG_ENDIAN) - { - return (((long) unsafe.getByte(address ) ) << 56) | - (((long) unsafe.getByte(address + 1) & 0xff) << 48) | - (((long) unsafe.getByte(address + 2) & 0xff) << 40) | - (((long) unsafe.getByte(address + 3) & 0xff) << 32) | - (((long) unsafe.getByte(address + 4) & 0xff) << 24) | - (((long) unsafe.getByte(address + 5) & 0xff) << 16) | - (((long) unsafe.getByte(address + 6) & 0xff) << 8) | - (((long) unsafe.getByte(address + 7) & 0xff) ); - } - else - { - return (((long) unsafe.getByte(address + 7) ) << 56) | - (((long) unsafe.getByte(address + 6) & 0xff) << 48) | - (((long) unsafe.getByte(address + 5) & 0xff) << 40) | - (((long) unsafe.getByte(address + 4) & 0xff) << 32) | - (((long) unsafe.getByte(address + 3) & 0xff) << 24) | - (((long) unsafe.getByte(address + 2) & 0xff) << 16) | - (((long) unsafe.getByte(address + 1) & 0xff) << 8) | - (((long) unsafe.getByte(address ) & 0xff) ); - } - } - - public static int getIntByByte(long address) - { - if (Architecture.BIG_ENDIAN) - { - return (((int) unsafe.getByte(address ) ) << 24) | - (((int) unsafe.getByte(address + 1) & 0xff) << 16) | - (((int) unsafe.getByte(address + 2) & 0xff) << 8 ) | - (((int) unsafe.getByte(address + 3) & 0xff) ); - } - else - { - return (((int) unsafe.getByte(address + 3) ) << 24) | - (((int) unsafe.getByte(address + 2) & 0xff) << 16) | - (((int) unsafe.getByte(address + 1) & 0xff) << 8) | - (((int) unsafe.getByte(address ) & 0xff) ); - } - } - - - public static int getShortByByte(long address) - { - if (Architecture.BIG_ENDIAN) - { - return (((int) unsafe.getByte(address ) ) << 8) | - (((int) unsafe.getByte(address + 1) & 0xff) ); - } - else - { - return (((int) unsafe.getByte(address + 1) ) << 8) | - (((int) unsafe.getByte(address ) & 0xff) ); - } - } - - public static void putLongByByte(long address, long value) - { - if (Architecture.BIG_ENDIAN) - { - unsafe.putByte(address, (byte) (value >> 56)); - unsafe.putByte(address + 1, (byte) (value >> 48)); - unsafe.putByte(address + 2, (byte) (value >> 40)); - unsafe.putByte(address + 3, (byte) (value >> 32)); - unsafe.putByte(address + 4, (byte) (value >> 24)); - unsafe.putByte(address + 5, (byte) (value >> 16)); - unsafe.putByte(address + 6, (byte) (value >> 8)); - unsafe.putByte(address + 7, (byte) (value)); - } - else - { - unsafe.putByte(address + 7, (byte) (value >> 56)); - unsafe.putByte(address + 6, (byte) (value >> 48)); - unsafe.putByte(address + 5, (byte) (value >> 40)); - unsafe.putByte(address + 4, (byte) (value >> 32)); - unsafe.putByte(address + 3, (byte) (value >> 24)); - unsafe.putByte(address + 2, (byte) (value >> 16)); - unsafe.putByte(address + 1, (byte) (value >> 8)); - unsafe.putByte(address, (byte) (value)); - } - } - - public static void putIntByByte(long address, int value) - { - if (Architecture.BIG_ENDIAN) - { - unsafe.putByte(address, (byte) (value >> 24)); - unsafe.putByte(address + 1, (byte) (value >> 16)); - unsafe.putByte(address + 2, (byte) (value >> 8)); - unsafe.putByte(address + 3, (byte) (value)); - } - else - { - unsafe.putByte(address + 3, (byte) (value >> 24)); - unsafe.putByte(address + 2, (byte) (value >> 16)); - unsafe.putByte(address + 1, (byte) (value >> 8)); - unsafe.putByte(address, (byte) (value)); - } - } - public static void setBytes(long address, ByteBuffer buffer) { int start = buffer.position(); diff --git a/src/java/org/apache/cassandra/utils/memory/NativeEndianMemoryUtil.java b/src/java/org/apache/cassandra/utils/memory/NativeEndianMemoryUtil.java new file mode 100644 index 000000000000..3cb5edb28b98 --- /dev/null +++ b/src/java/org/apache/cassandra/utils/memory/NativeEndianMemoryUtil.java @@ -0,0 +1,214 @@ +/* + * 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.utils.memory; + + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.utils.Architecture; + +/** + * Use this API only for data which are stored in-memory + * and not serialized directly (without converting to Java primitives) to disk and network + */ +public class NativeEndianMemoryUtil extends MemoryUtil +{ + public static int getUnsignedShort(long address) + { + if (Architecture.IS_UNALIGNED || (address & 0b1) == 0L) + return unsafe.getShort(address) & 0xffff; + else + return getShortByByte(address) & 0xffff; + } + + public static int getInt(long address) + { + if (Architecture.IS_UNALIGNED || (address & 0b11) == 0L) + return unsafe.getInt(address); + else + return getIntByByte(address); + } + + public static long getLong(long address) + { + if (Architecture.IS_UNALIGNED || (address & 0b111) == 0L) + return unsafe.getLong(address); + else + return getLongByByte(address); + } + + public static void setShort(long address, short s) + { + if (Architecture.IS_UNALIGNED || (address & 0b1) == 0L) + unsafe.putShort(address, s); + else + putShortByByte(address, s); + } + + public static void setInt(long address, int l) + { + if (Architecture.IS_UNALIGNED || (address & 0b11) == 0L) + unsafe.putInt(address, l); + else + putIntByByte(address, l); + } + + public static void setLong(long address, long l) + { + if (Architecture.IS_UNALIGNED || (address & 0b111) == 0L) + unsafe.putLong(address, l); + else + putLongByByte(address, l); + } + + @VisibleForTesting + static long getLongByByte(long address) + { + if (Architecture.BIG_ENDIAN) + { + return (((long) unsafe.getByte(address ) ) << 56) | + (((long) unsafe.getByte(address + 1) & 0xff) << 48) | + (((long) unsafe.getByte(address + 2) & 0xff) << 40) | + (((long) unsafe.getByte(address + 3) & 0xff) << 32) | + (((long) unsafe.getByte(address + 4) & 0xff) << 24) | + (((long) unsafe.getByte(address + 5) & 0xff) << 16) | + (((long) unsafe.getByte(address + 6) & 0xff) << 8) | + (((long) unsafe.getByte(address + 7) & 0xff) ); + } + else + { + return (((long) unsafe.getByte(address + 7) ) << 56) | + (((long) unsafe.getByte(address + 6) & 0xff) << 48) | + (((long) unsafe.getByte(address + 5) & 0xff) << 40) | + (((long) unsafe.getByte(address + 4) & 0xff) << 32) | + (((long) unsafe.getByte(address + 3) & 0xff) << 24) | + (((long) unsafe.getByte(address + 2) & 0xff) << 16) | + (((long) unsafe.getByte(address + 1) & 0xff) << 8) | + (((long) unsafe.getByte(address ) & 0xff) ); + } + } + + @VisibleForTesting + static int getIntByByte(long address) + { + if (Architecture.BIG_ENDIAN) + { + return (((int) unsafe.getByte(address ) ) << 24) | + (((int) unsafe.getByte(address + 1) & 0xff) << 16) | + (((int) unsafe.getByte(address + 2) & 0xff) << 8) | + (((int) unsafe.getByte(address + 3) & 0xff) ); + } + else + { + return (((int) unsafe.getByte(address + 3) ) << 24) | + (((int) unsafe.getByte(address + 2) & 0xff) << 16) | + (((int) unsafe.getByte(address + 1) & 0xff) << 8) | + (((int) unsafe.getByte(address ) & 0xff) ); + } + } + + @VisibleForTesting + static int getShortByByte(long address) + { + if (Architecture.BIG_ENDIAN) + { + return (((int) unsafe.getByte(address ) ) << 8) | + (((int) unsafe.getByte(address + 1) & 0xff) ); + } + else + { + return (((int) unsafe.getByte(address + 1) ) << 8) | + (((int) unsafe.getByte(address ) & 0xff) ); + } + } + + @VisibleForTesting + static void putLongByByte(long address, long value) + { + if (Architecture.BIG_ENDIAN) + { + unsafe.putByte(address , (byte) (value >> 56)); + unsafe.putByte(address + 1, (byte) (value >> 48)); + unsafe.putByte(address + 2, (byte) (value >> 40)); + unsafe.putByte(address + 3, (byte) (value >> 32)); + unsafe.putByte(address + 4, (byte) (value >> 24)); + unsafe.putByte(address + 5, (byte) (value >> 16)); + unsafe.putByte(address + 6, (byte) (value >> 8)); + unsafe.putByte(address + 7, (byte) (value )); + } + else + { + unsafe.putByte(address + 7, (byte) (value >> 56)); + unsafe.putByte(address + 6, (byte) (value >> 48)); + unsafe.putByte(address + 5, (byte) (value >> 40)); + unsafe.putByte(address + 4, (byte) (value >> 32)); + unsafe.putByte(address + 3, (byte) (value >> 24)); + unsafe.putByte(address + 2, (byte) (value >> 16)); + unsafe.putByte(address + 1, (byte) (value >> 8)); + unsafe.putByte(address , (byte) (value )); + } + } + + @VisibleForTesting + static void putIntByByte(long address, int value) + { + if (Architecture.BIG_ENDIAN) + { + unsafe.putByte(address , (byte) (value >> 24)); + unsafe.putByte(address + 1, (byte) (value >> 16)); + unsafe.putByte(address + 2, (byte) (value >> 8)); + unsafe.putByte(address + 3, (byte) (value )); + } + else + { + unsafe.putByte(address + 3, (byte) (value >> 24)); + unsafe.putByte(address + 2, (byte) (value >> 16)); + unsafe.putByte(address + 1, (byte) (value >> 8)); + unsafe.putByte(address , (byte) (value )); + } + } + + @VisibleForTesting + static void putShortByByte(long address, short value) + { + if (Architecture.BIG_ENDIAN) + { + unsafe.putByte(address , (byte) (value >> 8)); + unsafe.putByte(address + 1, (byte) (value )); + } + else + { + unsafe.putByte(address + 1, (byte) (value >> 8)); + unsafe.putByte(address , (byte) (value )); + } + } + + public static ByteBuffer getByteBuffer(long address, int length) + { + return getByteBuffer(address, length, ByteOrder.nativeOrder()); + } + + public static ByteBuffer getHollowDirectByteBuffer() + { + return getHollowDirectByteBuffer(ByteOrder.nativeOrder()); + } +} diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-CompressionInfo.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-CompressionInfo.db new file mode 100644 index 000000000000..aa6d6e6a102d Binary files /dev/null and b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Data.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Data.db new file mode 100644 index 000000000000..c24c3fae61a4 Binary files /dev/null and b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Data.db differ diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Digest.crc32 b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Digest.crc32 new file mode 100644 index 000000000000..c84b9595a002 --- /dev/null +++ b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Digest.crc32 @@ -0,0 +1 @@ +1026070592 \ No newline at end of file diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Filter.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Filter.db new file mode 100644 index 000000000000..8868e5c18008 Binary files /dev/null and b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Filter.db differ diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Partitions.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Partitions.db new file mode 100644 index 000000000000..6e4273411ff0 Binary files /dev/null and b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Partitions.db differ diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Rows.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Rows.db new file mode 100644 index 000000000000..d2ba4d639b61 Binary files /dev/null and b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Rows.db differ diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Statistics.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Statistics.db new file mode 100644 index 000000000000..fb646831d00a Binary files /dev/null and b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-Statistics.db differ diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-TOC.txt b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-TOC.txt new file mode 100644 index 000000000000..298910cfdc58 --- /dev/null +++ b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_be_index_summary/da-500-bti-TOC.txt @@ -0,0 +1,8 @@ +Data.db +Statistics.db +Digest.crc32 +TOC.txt +CompressionInfo.db +Filter.db +Partitions.db +Rows.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-CompressionInfo.db new file mode 100644 index 000000000000..49ebec126f4a Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Data.db new file mode 100644 index 000000000000..4b0fa7ef3507 Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Data.db differ diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Digest.crc32 new file mode 100644 index 000000000000..5f0b313cc8a5 --- /dev/null +++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Digest.crc32 @@ -0,0 +1 @@ +1371588035 \ No newline at end of file diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Filter.db new file mode 100644 index 000000000000..2e1d5d29ca06 Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Filter.db differ diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Index.db new file mode 100644 index 000000000000..ad88ef6efc77 Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Index.db differ diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Statistics.db new file mode 100644 index 000000000000..2c3a57350c2c Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Statistics.db differ diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Summary.db new file mode 100644 index 000000000000..0c575b7c1bf6 Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-Summary.db differ diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-TOC.txt new file mode 100644 index 000000000000..dde00207af51 --- /dev/null +++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_be_index_summary/ma-306-big-TOC.txt @@ -0,0 +1,8 @@ +CompressionInfo.db +Data.db +Digest.crc32 +Summary.db +Index.db +Statistics.db +TOC.txt +Filter.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-CompressionInfo.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-CompressionInfo.db new file mode 100644 index 000000000000..589a4160f5ff Binary files /dev/null and b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Data.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Data.db new file mode 100644 index 000000000000..4b4fe735f53f Binary files /dev/null and b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Data.db differ diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Digest.crc32 b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Digest.crc32 new file mode 100644 index 000000000000..cc0ff47a6beb --- /dev/null +++ b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Digest.crc32 @@ -0,0 +1 @@ +849406636 \ No newline at end of file diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Filter.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Filter.db new file mode 100644 index 000000000000..2e1d5d29ca06 Binary files /dev/null and b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Filter.db differ diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Index.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Index.db new file mode 100644 index 000000000000..f1bccb20c571 Binary files /dev/null and b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Index.db differ diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Statistics.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Statistics.db new file mode 100644 index 000000000000..7510a11124cd Binary files /dev/null and b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Statistics.db differ diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Summary.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Summary.db new file mode 100644 index 000000000000..0c575b7c1bf6 Binary files /dev/null and b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-Summary.db differ diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-TOC.txt b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-TOC.txt new file mode 100644 index 000000000000..6cd09785abec --- /dev/null +++ b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_be_index_summary/mb-307-big-TOC.txt @@ -0,0 +1,8 @@ +Index.db +Filter.db +TOC.txt +Digest.crc32 +Summary.db +Data.db +Statistics.db +CompressionInfo.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-CompressionInfo.db new file mode 100644 index 000000000000..8a0a148b64c0 Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Data.db new file mode 100644 index 000000000000..5208946c1d29 Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Data.db differ diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Digest.crc32 new file mode 100644 index 000000000000..3f9e06243713 --- /dev/null +++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Digest.crc32 @@ -0,0 +1 @@ +1373250029 \ No newline at end of file diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Filter.db new file mode 100644 index 000000000000..2e1d5d29ca06 Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Filter.db differ diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Index.db new file mode 100644 index 000000000000..f1bccb20c571 Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Index.db differ diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Statistics.db new file mode 100644 index 000000000000..dba6ce739c22 Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Statistics.db differ diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Summary.db new file mode 100644 index 000000000000..0c575b7c1bf6 Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-Summary.db differ diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-TOC.txt new file mode 100644 index 000000000000..19154fb1404d --- /dev/null +++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_be_index_summary/mc-3113-big-TOC.txt @@ -0,0 +1,8 @@ +Summary.db +CompressionInfo.db +Digest.crc32 +Statistics.db +Index.db +Data.db +Filter.db +TOC.txt diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-CompressionInfo.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-CompressionInfo.db new file mode 100644 index 000000000000..e79d9784589a Binary files /dev/null and b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Data.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Data.db new file mode 100644 index 000000000000..a6a94e6b5ec3 Binary files /dev/null and b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Data.db differ diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Digest.crc32 b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Digest.crc32 new file mode 100644 index 000000000000..44c7c2710e14 --- /dev/null +++ b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Digest.crc32 @@ -0,0 +1 @@ +2461228597 \ No newline at end of file diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Filter.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Filter.db new file mode 100644 index 000000000000..2e1d5d29ca06 Binary files /dev/null and b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Filter.db differ diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Index.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Index.db new file mode 100644 index 000000000000..0860005fb9d8 Binary files /dev/null and b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Index.db differ diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Statistics.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Statistics.db new file mode 100644 index 000000000000..9288af336e4c Binary files /dev/null and b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Statistics.db differ diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Summary.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Summary.db new file mode 100644 index 000000000000..0c575b7c1bf6 Binary files /dev/null and b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-Summary.db differ diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-TOC.txt b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-TOC.txt new file mode 100644 index 000000000000..fa38f567c160 --- /dev/null +++ b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_be_index_summary/md-31110-big-TOC.txt @@ -0,0 +1,8 @@ +Data.db +Statistics.db +Filter.db +Summary.db +CompressionInfo.db +Index.db +TOC.txt +Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-CompressionInfo.db new file mode 100644 index 000000000000..f807a43f3e96 Binary files /dev/null and b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Data.db new file mode 100644 index 000000000000..6fb056218cc9 Binary files /dev/null and b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Data.db differ diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Digest.crc32 new file mode 100644 index 000000000000..4d40edafc7ad --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Digest.crc32 @@ -0,0 +1 @@ +3525076442 \ No newline at end of file diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Filter.db new file mode 100644 index 000000000000..2e1d5d29ca06 Binary files /dev/null and b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Filter.db differ diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Index.db new file mode 100644 index 000000000000..2fd4d9b85fe5 Binary files /dev/null and b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Index.db differ diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Statistics.db new file mode 100644 index 000000000000..2482df5a8151 Binary files /dev/null and b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Statistics.db differ diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Summary.db new file mode 100644 index 000000000000..0c575b7c1bf6 Binary files /dev/null and b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-Summary.db differ diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-TOC.txt new file mode 100644 index 000000000000..b91824d28955 --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_be_index_summary/me-31111-big-TOC.txt @@ -0,0 +1,8 @@ +Index.db +CompressionInfo.db +Statistics.db +TOC.txt +Summary.db +Digest.crc32 +Data.db +Filter.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-CompressionInfo.db new file mode 100644 index 000000000000..8ded333178f7 Binary files /dev/null and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Data.db new file mode 100644 index 000000000000..0bbfe1acd569 Binary files /dev/null and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Data.db differ diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Digest.crc32 new file mode 100644 index 000000000000..7ffa3bc3c73b --- /dev/null +++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Digest.crc32 @@ -0,0 +1 @@ +309317098 \ No newline at end of file diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Filter.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Filter.db new file mode 100644 index 000000000000..8868e5c18008 Binary files /dev/null and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Filter.db differ diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Index.db new file mode 100644 index 000000000000..e04d3dbda92e Binary files /dev/null and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Index.db differ diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Statistics.db new file mode 100644 index 000000000000..804ccf248f87 Binary files /dev/null and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Statistics.db differ diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Summary.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Summary.db new file mode 100644 index 000000000000..0c575b7c1bf6 Binary files /dev/null and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-Summary.db differ diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-TOC.txt new file mode 100644 index 000000000000..203013437a0b --- /dev/null +++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_be_index_summary/na-40-big-TOC.txt @@ -0,0 +1,8 @@ +TOC.txt +Data.db +Index.db +Statistics.db +Digest.crc32 +CompressionInfo.db +Filter.db +Summary.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-CompressionInfo.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-CompressionInfo.db new file mode 100644 index 000000000000..c34e6728465e Binary files /dev/null and b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Data.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Data.db new file mode 100644 index 000000000000..becb2d28fbd6 Binary files /dev/null and b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Data.db differ diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Digest.crc32 b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Digest.crc32 new file mode 100644 index 000000000000..b0e4cbb28e6d --- /dev/null +++ b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Digest.crc32 @@ -0,0 +1 @@ +3874015080 \ No newline at end of file diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Filter.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Filter.db new file mode 100644 index 000000000000..8868e5c18008 Binary files /dev/null and b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Filter.db differ diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Index.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Index.db new file mode 100644 index 000000000000..d3b366dd185c Binary files /dev/null and b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Index.db differ diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Statistics.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Statistics.db new file mode 100644 index 000000000000..f49e76531f64 Binary files /dev/null and b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Statistics.db differ diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Summary.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Summary.db new file mode 100644 index 000000000000..0c575b7c1bf6 Binary files /dev/null and b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-Summary.db differ diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-TOC.txt b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-TOC.txt new file mode 100644 index 000000000000..dc0664391ebb --- /dev/null +++ b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_be_index_summary/nb-400-big-TOC.txt @@ -0,0 +1,8 @@ +Data.db +TOC.txt +Digest.crc32 +Summary.db +Index.db +Filter.db +CompressionInfo.db +Statistics.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-CompressionInfo.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-CompressionInfo.db new file mode 100644 index 000000000000..3d328b087561 Binary files /dev/null and b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Data.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Data.db new file mode 100644 index 000000000000..7d422681cb7a Binary files /dev/null and b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Data.db differ diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Digest.crc32 b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Digest.crc32 new file mode 100644 index 000000000000..e86ec54b54e0 --- /dev/null +++ b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Digest.crc32 @@ -0,0 +1 @@ +1158768921 \ No newline at end of file diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Filter.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Filter.db new file mode 100644 index 000000000000..8868e5c18008 Binary files /dev/null and b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Filter.db differ diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Index.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Index.db new file mode 100644 index 000000000000..46ab61e61f37 Binary files /dev/null and b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Index.db differ diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Statistics.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Statistics.db new file mode 100644 index 000000000000..a548e130d2e9 Binary files /dev/null and b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Statistics.db differ diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Summary.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Summary.db new file mode 100644 index 000000000000..0c575b7c1bf6 Binary files /dev/null and b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-Summary.db differ diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-TOC.txt b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-TOC.txt new file mode 100644 index 000000000000..576c5e0598c5 --- /dev/null +++ b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_be_index_summary/oa-500-big-TOC.txt @@ -0,0 +1,8 @@ +Data.db +Statistics.db +Digest.crc32 +TOC.txt +CompressionInfo.db +Filter.db +Index.db +Summary.db diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java index ce0422217281..34b91e151857 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java @@ -921,7 +921,7 @@ private void loadTestSStables(ColumnFamilyStore cfs, File ksDir) throws IOExcept File tableDir = new File(ksDir, cfs.name); Assert.assertTrue("The table directory " + tableDir + " was not found", tableDir.isDirectory()); for (File file : tableDir.tryList()) - LegacySSTableTest.copyFile(cfDir, file); + LegacySSTableTest.copyFileToDir(file, cfDir); } cfs.loadNewSSTables(); } diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java index aa80f0e25c53..eb7480e309bb 100644 --- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java @@ -34,8 +34,10 @@ import org.junit.After; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -90,6 +92,9 @@ public class LegacySSTableTest { private static final Logger logger = LoggerFactory.getLogger(LegacySSTableTest.class); + @ClassRule + public static TemporaryFolder tempFolder = new TemporaryFolder(); + public static File LEGACY_SSTABLE_ROOT; private static final String LEGACY_TABLES_KEYSPACE = "legacy_tables"; @@ -164,11 +169,11 @@ public void tearDown() /** * Get a descriptor for the legacy sstable at the given version. */ - protected Descriptor getDescriptor(String legacyVersion, String table) throws IOException + protected Descriptor getDescriptor(File dir) throws IOException { - Path file = Files.list(getTableDir(legacyVersion, table).toPath()) + Path file = Files.list(dir.toPath()) .findFirst() - .orElseThrow(() -> new RuntimeException(String.format("No files for verion=%s and table=%s", legacyVersion, table))); + .orElseThrow(() -> new RuntimeException(String.format("No files for path=%s", dir.absolutePath()))); return Descriptor.fromFile(new File(file)); } @@ -494,15 +499,19 @@ private void streamLegacyTables(String legacyVersion) throws Exception streamLegacyTable("legacy_%s_clust", legacyVersion); streamLegacyTable("legacy_%s_clust_counter", legacyVersion); streamLegacyTable("legacy_%s_tuple", legacyVersion); + streamLegacyTable("legacy_%s_clust_be_index_summary", legacyVersion); } private void streamLegacyTable(String tablePattern, String legacyVersion) throws Exception { String table = String.format(tablePattern, legacyVersion); - Descriptor descriptor = getDescriptor(legacyVersion, table); + // streaming can mutate test data (rewrite IndexSummary, so we have to copy them) + File testDataDir = new File(tempFolder.newFolder(LEGACY_TABLES_KEYSPACE, table)); + copySstablesToTestData(legacyVersion, table, testDataDir); + Descriptor descriptor = getDescriptor(testDataDir); if (null != descriptor) { - SSTableReader sstable = SSTableReader.open(null, getDescriptor(legacyVersion, table)); + SSTableReader sstable = SSTableReader.open(null, descriptor); IPartitioner p = sstable.getPartitioner(); List> ranges = new ArrayList<>(); ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("100")))); @@ -526,6 +535,7 @@ public static void truncateLegacyTables(String legacyVersion) throws Exception Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_clust", legacyVersion)).truncateBlocking(); Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_clust_counter", legacyVersion)).truncateBlocking(); Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_tuple", legacyVersion)).truncateBlocking(); + Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_clust_be_index_summary", legacyVersion)).truncateBlocking(); CacheService.instance.invalidateCounterCache(); CacheService.instance.invalidateKeyCache(); } @@ -538,6 +548,7 @@ private static void compactLegacyTables(String legacyVersion) throws Exception Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_clust", legacyVersion)).forceMajorCompaction(); Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_clust_counter", legacyVersion)).forceMajorCompaction(); Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_tuple", legacyVersion)).forceMajorCompaction(); + Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_clust_be_index_summary", legacyVersion)).forceMajorCompaction(); } public static void loadLegacyTables(String legacyVersion) throws Exception @@ -548,6 +559,7 @@ public static void loadLegacyTables(String legacyVersion) throws Exception loadLegacyTable(legacyVersion, "clust"); loadLegacyTable(legacyVersion, "clust_counter"); loadLegacyTable(legacyVersion, "tuple"); + loadLegacyTable(legacyVersion, "clust_be_index_summary"); } private static void verifyCache(String legacyVersion, long startCount) throws InterruptedException, java.util.concurrent.ExecutionException @@ -585,7 +597,8 @@ private static void verifyReads(String legacyVersion) readSimpleCounterTable(legacyVersion, pkValue); } - readClusteringTable(legacyVersion, ck, ckValue, pkValue); + readClusteringTable("legacy_%s_clust", legacyVersion, ck, ckValue, pkValue); + readClusteringTable("legacy_%s_clust_be_index_summary", legacyVersion, ck, ckValue, pkValue); readClusteringCounterTable(legacyVersion, ckValue, pkValue); } } @@ -601,16 +614,16 @@ private static void readClusteringCounterTable(String legacyVersion, String ckVa Assert.assertEquals(1L, rs.one().getLong("val")); } - private static void readClusteringTable(String legacyVersion, int ck, String ckValue, String pkValue) + private static void readClusteringTable(String tableName, String legacyVersion, int ck, String ckValue, String pkValue) { logger.debug("Read legacy_{}_clust", legacyVersion); UntypedResultSet rs; - rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust WHERE pk=? AND ck=?", legacyVersion), pkValue, ckValue); + rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables." + tableName + " WHERE pk=? AND ck=?", legacyVersion), pkValue, ckValue); assertLegacyClustRows(1, rs); String ckValue2 = Integer.toString(ck < 10 ? 40 : ck - 1) + longString; String ckValue3 = Integer.toString(ck > 39 ? 10 : ck + 1) + longString; - rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust WHERE pk=? AND ck IN (?, ?, ?)", legacyVersion), pkValue, ckValue, ckValue2, ckValue3); + rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables." + tableName + " WHERE pk=? AND ck IN (?, ?, ?)", legacyVersion), pkValue, ckValue, ckValue2, ckValue3); assertLegacyClustRows(3, rs); } @@ -645,7 +658,7 @@ private static void createTables(String legacyVersion) QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple_counter (pk text PRIMARY KEY, val counter)", legacyVersion)); QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust (pk text, ck text, val text, PRIMARY KEY (pk, ck))", legacyVersion)); QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust_counter (pk text, ck text, val counter, PRIMARY KEY (pk, ck))", legacyVersion)); - + QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust_be_index_summary (pk text, ck text, val text, PRIMARY KEY (pk, ck))", legacyVersion)); QueryProcessor.executeInternal(String.format("CREATE TYPE legacy_tables.legacy_%s_tuple_udt (name tuple)", legacyVersion)); @@ -668,6 +681,7 @@ private static void truncateTables(String legacyVersion) QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_simple_counter", legacyVersion)); QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust", legacyVersion)); QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust_counter", legacyVersion)); + QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust_be_index_summary", legacyVersion)); CacheService.instance.invalidateCounterCache(); CacheService.instance.invalidateKeyCache(); } @@ -747,6 +761,13 @@ public void testGenerateSstables() throws Throwable QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_clust_counter SET val = val + 1 WHERE pk = '%s' AND ck='%s'", format.getLatestVersion(), valPk, valCk + longString)); + + // note: to emulate BE for offsets in Summary you can comment temporary the following line: + // offset = Integer.reverseBytes(offset); + // in org.apache.cassandra.io.sstable.indexsummary.IndexSummary.IndexSummarySerializer.serialize + QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_clust_be_index_summary (pk, ck, val) VALUES ('%s', '%s', '%s')", + format.getLatestVersion(), valPk, valCk + longString, randomString)); + } } @@ -759,6 +780,7 @@ public void testGenerateSstables() throws Throwable copySstablesFromTestData(format.getLatestVersion(), "legacy_%s_clust", ksDir); copySstablesFromTestData(format.getLatestVersion(), "legacy_%s_clust_counter", ksDir); copySstablesFromTestData(format.getLatestVersion(), "legacy_%s_tuple", ksDir); + copySstablesFromTestData(format.getLatestVersion(), "legacy_%s_clust_be_index_summary", ksDir); } public static void copySstablesFromTestData(Version legacyVersion, String tablePattern, File ksDir) throws IOException @@ -774,42 +796,47 @@ public static void copySstablesFromTestData(Version legacyVersion, String tableP for (File srcDir : Keyspace.open(ks).getColumnFamilyStore(table).getDirectories().getCFDirectories()) { - for (File file : srcDir.tryList()) + for (File sourceFile : srcDir.tryList()) { // Sequence IDs represent the C* version used when creating the SSTable, i.e. with #testGenerateSstables() (if not uuid based) String newSeqId = FBUtilities.getReleaseVersionString().split("-")[0].replaceAll("[^0-9]", ""); - File target = new File(cfDir, file.name().replace(legacyVersion + "-1-", legacyVersion + "-" + newSeqId + "-")); - copyFile(cfDir, file, target); + File target = new File(cfDir, sourceFile.name().replace(legacyVersion + "-1-", legacyVersion + "-" + newSeqId + "-")); + copyFile(sourceFile, target); } } } - private static void copySstablesToTestData(String legacyVersion, String table, File cfDir) throws IOException + private static void copySstablesToTestData(String legacyVersion, String table, File targetDir) throws IOException + { + File testDataTableDir = getTestDataTableDir(legacyVersion, table); + Assert.assertTrue("The table directory " + testDataTableDir + " was not found", testDataTableDir.isDirectory()); + for (File sourceTestFile : testDataTableDir.tryList()) + copyFileToDir(sourceTestFile, targetDir); + } + + private static File getTestDataTableDir(File parentDir, String legacyVersion, String table) { - File tableDir = getTableDir(legacyVersion, table); - Assert.assertTrue("The table directory " + tableDir + " was not found", tableDir.isDirectory()); - for (File file : tableDir.tryList()) - copyFile(cfDir, file); + return new File(parentDir, String.format("%s/legacy_tables/%s", legacyVersion, table)); } - private static File getTableDir(String legacyVersion, String table) + private static File getTestDataTableDir(String legacyVersion, String table) { - return new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables/%s", legacyVersion, table)); + return getTestDataTableDir(LEGACY_SSTABLE_ROOT, legacyVersion, table); } - public static void copyFile(File cfDir, File file) throws IOException + public static void copyFileToDir(File sourceFile, File targetDir) throws IOException { - copyFile(cfDir, file, new File(cfDir, file.name())); + copyFile(sourceFile, new File(targetDir, sourceFile.name())); } - public static void copyFile(File cfDir, File file, File target) throws IOException + public static void copyFile(File sourceFile, File targetFile) throws IOException { byte[] buf = new byte[65536]; - if (file.isFile()) + if (sourceFile.isFile()) { int rd; - try (FileInputStreamPlus is = new FileInputStreamPlus(file); - FileOutputStreamPlus os = new FileOutputStreamPlus(target);) + try (FileInputStreamPlus is = new FileInputStreamPlus(sourceFile); + FileOutputStreamPlus os = new FileOutputStreamPlus(targetFile);) { while ((rd = is.read(buf)) >= 0) os.write(buf, 0, rd); diff --git a/test/unit/org/apache/cassandra/utils/memory/LittleEndianMemoryUtilTest.java b/test/unit/org/apache/cassandra/utils/memory/LittleEndianMemoryUtilTest.java new file mode 100644 index 000000000000..592d01906bcc --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/memory/LittleEndianMemoryUtilTest.java @@ -0,0 +1,148 @@ +/* + * 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.utils.memory; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +import org.junit.Assert; +import org.junit.Test; + +public class LittleEndianMemoryUtilTest +{ + private static final int TEST_BUFFER_LENGTH = 8; + private final ByteBuffer directBuffer = ByteBuffer.allocateDirect(TEST_BUFFER_LENGTH); + { + directBuffer.order(ByteOrder.LITTLE_ENDIAN); + } + private final long address = LittleEndianMemoryUtil.getAddress(directBuffer); + + @Test + public void testGetSetLong() + { + long originalValue = 0xAB_CD_EF_12_34_56_78_90L; + directBuffer.putLong(originalValue); + Assert.assertEquals(originalValue, LittleEndianMemoryUtil.getLong(address)); + + directBuffer.rewind(); + directBuffer.putLong(0); + LittleEndianMemoryUtil.setLong(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getLong(0)); + Assert.assertEquals(originalValue, LittleEndianMemoryUtil.getLong(address)); + + } + + @Test + public void testGetSetInt() + { + int originalValue = 0xAB_CD_EF_12; + directBuffer.putInt(originalValue); + Assert.assertEquals(originalValue, LittleEndianMemoryUtil.getInt(address)); + + directBuffer.rewind(); + directBuffer.putInt(0); + LittleEndianMemoryUtil.setInt(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getInt(0)); + Assert.assertEquals(originalValue, LittleEndianMemoryUtil.getInt(address)); + + } + + @Test + public void testGetSetUnsighedShort() + { + short originalValue = (short) 0xAB_CD; + directBuffer.putShort(originalValue); + Assert.assertEquals(originalValue & 0xffff, LittleEndianMemoryUtil.getUnsignedShort(address)); + + directBuffer.rewind(); + directBuffer.putShort((short) 0); + LittleEndianMemoryUtil.setShort(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getShort(0)); + Assert.assertEquals(originalValue & 0xffff, LittleEndianMemoryUtil.getUnsignedShort(address)); + } + + @Test + public void testGetSetLongByBytes() + { + long originalValue = 0xAB_CD_EF_12_34_56_78_90L; + directBuffer.putLong(originalValue); + Assert.assertEquals(originalValue, LittleEndianMemoryUtil.getLongByByte(address)); + + directBuffer.rewind(); + directBuffer.putLong(0); + LittleEndianMemoryUtil.putLongByByte(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getLong(0)); + Assert.assertEquals(originalValue, LittleEndianMemoryUtil.getLongByByte(address)); + + } + + @Test + public void testGetSetIntByBytes() + { + int originalValue = 0xAB_CD_EF_12; + directBuffer.putInt(originalValue); + Assert.assertEquals(originalValue, LittleEndianMemoryUtil.getIntByByte(address)); + + directBuffer.rewind(); + directBuffer.putInt(0); + LittleEndianMemoryUtil.putIntByByte(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getInt(0)); + Assert.assertEquals(originalValue, LittleEndianMemoryUtil.getIntByByte(address)); + + } + + @Test + public void testGetSetShortByBytes() + { + short originalValue = (short) 0xAB_CD; + directBuffer.putShort(originalValue); + Assert.assertEquals(originalValue, LittleEndianMemoryUtil.getShortByByte(address)); + + directBuffer.rewind(); + directBuffer.putShort((short) 0); + LittleEndianMemoryUtil.putShortByByte(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getShort(0)); + Assert.assertEquals(originalValue, LittleEndianMemoryUtil.getShortByByte(address)); + } + + + @Test + public void testGetHollowDirectByteBuffer() + { + ByteBuffer byteBuffer = LittleEndianMemoryUtil.getHollowDirectByteBuffer(); + Assert.assertEquals(directBuffer.getClass(), byteBuffer.getClass()); + Assert.assertEquals(ByteOrder.LITTLE_ENDIAN, byteBuffer.order()); + } + + @Test + public void testGetByteBuffer() + { + ByteBuffer byteBuffer = LittleEndianMemoryUtil.getByteBuffer(address, TEST_BUFFER_LENGTH); + Assert.assertEquals(directBuffer.getClass(), byteBuffer.getClass()); + Assert.assertEquals(ByteOrder.LITTLE_ENDIAN, byteBuffer.order()); + Assert.assertEquals(TEST_BUFFER_LENGTH, byteBuffer.capacity()); + Assert.assertEquals(0, byteBuffer.position()); + } +} diff --git a/test/unit/org/apache/cassandra/utils/memory/NativeEndianMemoryUtilTest.java b/test/unit/org/apache/cassandra/utils/memory/NativeEndianMemoryUtilTest.java new file mode 100644 index 000000000000..ba0527b18a0e --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/memory/NativeEndianMemoryUtilTest.java @@ -0,0 +1,148 @@ +/* + * 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.utils.memory; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +import org.junit.Assert; +import org.junit.Test; + +public class NativeEndianMemoryUtilTest +{ + private static final int TEST_BUFFER_LENGTH = 8; + private final ByteBuffer directBuffer = ByteBuffer.allocateDirect(TEST_BUFFER_LENGTH); + { + directBuffer.order(ByteOrder.nativeOrder()); + } + private final long address = NativeEndianMemoryUtil.getAddress(directBuffer); + + @Test + public void testGetSetLong() + { + long originalValue = 0xAB_CD_EF_12_34_56_78_90L; + directBuffer.putLong(originalValue); + Assert.assertEquals(originalValue, NativeEndianMemoryUtil.getLong(address)); + + directBuffer.rewind(); + directBuffer.putLong(0); + NativeEndianMemoryUtil.setLong(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getLong(0)); + Assert.assertEquals(originalValue, NativeEndianMemoryUtil.getLong(address)); + + } + + @Test + public void testGetSetInt() + { + int originalValue = 0xAB_CD_EF_12; + directBuffer.putInt(originalValue); + Assert.assertEquals(originalValue, NativeEndianMemoryUtil.getInt(address)); + + directBuffer.rewind(); + directBuffer.putInt(0); + NativeEndianMemoryUtil.setInt(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getInt(0)); + Assert.assertEquals(originalValue, NativeEndianMemoryUtil.getInt(address)); + + } + + @Test + public void testGetSetUnsighedShort() + { + short originalValue = (short) 0xAB_CD; + directBuffer.putShort(originalValue); + Assert.assertEquals(originalValue & 0xffff, NativeEndianMemoryUtil.getUnsignedShort(address)); + + directBuffer.rewind(); + directBuffer.putShort((short) 0); + NativeEndianMemoryUtil.setShort(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getShort(0)); + Assert.assertEquals(originalValue & 0xffff, NativeEndianMemoryUtil.getUnsignedShort(address)); + } + + @Test + public void testGetSetLongByBytes() + { + long originalValue = 0xAB_CD_EF_12_34_56_78_90L; + directBuffer.putLong(originalValue); + Assert.assertEquals(originalValue, NativeEndianMemoryUtil.getLongByByte(address)); + + directBuffer.rewind(); + directBuffer.putLong(0); + NativeEndianMemoryUtil.putLongByByte(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getLong(0)); + Assert.assertEquals(originalValue, NativeEndianMemoryUtil.getLongByByte(address)); + + } + + @Test + public void testGetSetIntByBytes() + { + int originalValue = 0xAB_CD_EF_12; + directBuffer.putInt(originalValue); + Assert.assertEquals(originalValue, NativeEndianMemoryUtil.getIntByByte(address)); + + directBuffer.rewind(); + directBuffer.putInt(0); + NativeEndianMemoryUtil.putIntByByte(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getInt(0)); + Assert.assertEquals(originalValue, NativeEndianMemoryUtil.getIntByByte(address)); + + } + + @Test + public void testGetSetShortByBytes() + { + short originalValue = (short) 0xAB_CD; + directBuffer.putShort(originalValue); + Assert.assertEquals(originalValue, NativeEndianMemoryUtil.getShortByByte(address)); + + directBuffer.rewind(); + directBuffer.putShort((short) 0); + NativeEndianMemoryUtil.putShortByByte(address, originalValue); + + Assert.assertEquals(originalValue, directBuffer.getShort(0)); + Assert.assertEquals(originalValue, NativeEndianMemoryUtil.getShortByByte(address)); + } + + + @Test + public void testGetHollowDirectByteBuffer() + { + ByteBuffer byteBuffer = NativeEndianMemoryUtil.getHollowDirectByteBuffer(); + Assert.assertEquals(directBuffer.getClass(), byteBuffer.getClass()); + Assert.assertEquals(ByteOrder.nativeOrder(), byteBuffer.order()); + } + + @Test + public void testGetByteBuffer() + { + ByteBuffer byteBuffer = NativeEndianMemoryUtil.getByteBuffer(address, TEST_BUFFER_LENGTH); + Assert.assertEquals(directBuffer.getClass(), byteBuffer.getClass()); + Assert.assertEquals(ByteOrder.nativeOrder(), byteBuffer.order()); + Assert.assertEquals(TEST_BUFFER_LENGTH, byteBuffer.capacity()); + Assert.assertEquals(0, byteBuffer.position()); + } +}