Skip to content

Commit

Permalink
Jamm update - dependency update
Browse files Browse the repository at this point in the history
  • Loading branch information
jacek-lewandowski committed Oct 3, 2024
1 parent 24b842f commit 5696964
Show file tree
Hide file tree
Showing 12 changed files with 198 additions and 162 deletions.
2 changes: 1 addition & 1 deletion build.xml
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,7 @@
<property name="jacoco.version" value="0.8.7"/>

<property name="byteman.version" value="4.0.23"/>
<property name="jamm.version" value="0.3.2"/>
<property name="jamm.version" value="0.4.0"/>
<property name="ecj.version" value="3.33.0"/>
<property name="ohc.version" value="0.5.1"/>
<property name="asm.version" value="9.7"/>
Expand Down
2 changes: 1 addition & 1 deletion src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
Original file line number Diff line number Diff line change
Expand Up @@ -221,7 +221,7 @@ public long unsharedHeapSize()
public long unsharedHeapSizeExcludingData()
{
return EMPTY_SIZE
+ ObjectSizes.sizeOnHeapExcludingData(bytes)
+ ObjectSizes.sizeOnHeapExcludingDataOf(bytes)
+ ObjectSizes.sizeOf(text);
}

Expand Down
2 changes: 1 addition & 1 deletion src/java/org/apache/cassandra/db/BufferClustering.java
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ public long unsharedHeapSizeExcludingData()
{
if (this == Clustering.EMPTY || this == Clustering.STATIC_CLUSTERING)
return 0;
return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(values);
return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingDataOf(values);
}

public static BufferClustering make(ByteBuffer... values)
Expand Down
2 changes: 1 addition & 1 deletion src/java/org/apache/cassandra/db/rows/BufferCell.java
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,6 @@ public Cell<?> clone(ByteBufferCloner cloner)

public long unsharedHeapSizeExcludingData()
{
return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(value) + (path == null ? 0 : path.unsharedHeapSizeExcludingData());
return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingDataOf(value) + (path == null ? 0 : path.unsharedHeapSizeExcludingData());
}
}
2 changes: 1 addition & 1 deletion src/java/org/apache/cassandra/db/rows/CellPath.java
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ public CellPath clone(ByteBufferCloner cloner)

public long unsharedHeapSizeExcludingData()
{
return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(value);
return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingDataOf(value);
}
}

Expand Down
4 changes: 3 additions & 1 deletion src/java/org/apache/cassandra/db/tries/MemtableTrie.java
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,8 @@
import org.apache.cassandra.utils.ObjectSizes;
import org.github.jamm.MemoryLayoutSpecification;

import static org.github.jamm.MemoryMeterStrategy.MEMORY_LAYOUT;

/**
* Memtable trie, i.e. an in-memory trie built for fast modification and reads executing concurrently with writes from
* a single mutator thread.
Expand Down Expand Up @@ -982,7 +984,7 @@ public long sizeOffHeap()
/** Returns the on heap size of the memtable trie itself, not counting any space taken by referenced content. */
public long sizeOnHeap()
{
return contentCount * MemoryLayoutSpecification.SPEC.getReferenceSize() +
return contentCount * (long) MEMORY_LAYOUT.getReferenceSize() +
(bufferType == BufferType.ON_HEAP ? allocatedPos + EMPTY_SIZE_ON_HEAP : EMPTY_SIZE_OFF_HEAP);
}

Expand Down
5 changes: 3 additions & 2 deletions src/java/org/apache/cassandra/fql/FullQueryLogger.java
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@
import org.github.jamm.MemoryLayoutSpecification;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.github.jamm.MemoryMeterStrategy.MEMORY_LAYOUT;

/**
* A logger that logs entire query contents after the query finishes (or times out).
Expand Down Expand Up @@ -84,8 +85,8 @@ public class FullQueryLogger implements QueryEvents.Listener
private static final int EMPTY_LIST_SIZE = Ints.checkedCast(ObjectSizes.measureDeep(new ArrayList<>(0)));
private static final int EMPTY_BYTEBUF_SIZE;

private static final int OBJECT_HEADER_SIZE = MemoryLayoutSpecification.SPEC.getObjectHeaderSize();
private static final int OBJECT_REFERENCE_SIZE = MemoryLayoutSpecification.SPEC.getReferenceSize();
private static final int OBJECT_HEADER_SIZE = MEMORY_LAYOUT.getObjectHeaderSize();
private static final int OBJECT_REFERENCE_SIZE = MEMORY_LAYOUT.getReferenceSize();

public static final FullQueryLogger instance = new FullQueryLogger();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,13 +57,6 @@ class BKDPostingsIndex
}
}

public long memoryUsage()
{
// IntLongHashMap uses two arrays: one for keys, one for values.
return MemoryLayoutSpecification.sizeOfArray(index.size(), 4L)
+ MemoryLayoutSpecification.sizeOfArray(index.size(), 8L);
}

/**
* Returns <tt>true</tt> if given node ID has an auxiliary posting list.
*/
Expand Down
162 changes: 103 additions & 59 deletions src/java/org/apache/cassandra/utils/ObjectSizes.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,25 +19,33 @@

package org.apache.cassandra.utils;

import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.nio.ByteBuffer;

import org.github.jamm.MemoryLayoutSpecification;
import org.github.jamm.MemoryMeter;
import org.github.jamm.MemoryMeter.ByteBufferMode;
import org.github.jamm.MemoryMeter.Guess;

import static org.github.jamm.MemoryMeterStrategy.MEMORY_LAYOUT;
import static org.github.jamm.utils.ArrayMeasurementUtils.computeArraySize;

/**
* A convenience class for wrapping access to MemoryMeter
* A convenience class for wrapping access to MemoryMeter. Should be used instead of using a {@code MemoryMeter} directly.
* {@code MemoryMeter} can be used directly for testing as it allow a more fine tuned configuration for comparison.
*/
public class ObjectSizes
{
private static final MemoryMeter meter = new MemoryMeter().omitSharedBufferOverhead()
.withGuessing(MemoryMeter.Guess.FALLBACK_UNSAFE)
.ignoreKnownSingletons();
private static final MemoryMeter meter = MemoryMeter.builder().withGuessing(Guess.INSTRUMENTATION_AND_SPECIFICATION,
Guess.UNSAFE)
.build();

private static final long EMPTY_HEAP_BUFFER_SIZE = measure(ByteBufferUtil.EMPTY_BYTE_BUFFER);
private static final long EMPTY_BYTE_ARRAY_SIZE = measure(new byte[0]);
private static final long EMPTY_STRING_SIZE = measure("");
private static final long HEAP_BUFFER_SHALLOW_SIZE = measure(ByteBufferUtil.EMPTY_BYTE_BUFFER);
private static final long DIRECT_BUFFER_SHALLOW_SIZE = measure(ByteBuffer.allocateDirect(0));
private static final long DIRECT_BUFFER_DEEP_SIZE = measureDeep(ByteBuffer.allocateDirect(0));

private static final long DIRECT_BUFFER_HEAP_SIZE = measure(ByteBuffer.allocateDirect(0));
public static final long IPV6_SOCKET_ADDRESS_SIZE = ObjectSizes.measureDeep(new InetSocketAddress(getIpvAddress(16), 42));

/**
* Memory a byte array consumes
Expand All @@ -47,10 +55,7 @@ public class ObjectSizes
*/
public static long sizeOfArray(byte[] bytes)
{
if (bytes == null)
return 0;

return sizeOfArray(bytes.length, 1);
return meter.measureArray(bytes);
}

/**
Expand All @@ -61,10 +66,7 @@ public static long sizeOfArray(byte[] bytes)
*/
public static long sizeOfArray(long[] longs)
{
if (longs == null)
return 0;

return sizeOfArray(longs.length, 8);
return meter.measureArray(longs);
}

/**
Expand All @@ -75,10 +77,7 @@ public static long sizeOfArray(long[] longs)
*/
public static long sizeOfArray(int[] ints)
{
if (ints == null)
return 0;

return sizeOfArray(ints.length, 4);
return meter.measureArray(ints);
}

/**
Expand All @@ -89,7 +88,7 @@ public static long sizeOfArray(int[] ints)
*/
public static long sizeOfReferenceArray(int length)
{
return sizeOfArray(length, MemoryLayoutSpecification.SPEC.getReferenceSize());
return sizeOfArray(length, MEMORY_LAYOUT.getReferenceSize());
}

/**
Expand All @@ -100,15 +99,12 @@ public static long sizeOfReferenceArray(int length)
*/
public static long sizeOfArray(Object[] objects)
{
if (objects == null)
return 0;

return sizeOfReferenceArray(objects.length);
return meter.measureArray(objects);
}

private static long sizeOfArray(int length, long elementSize)
private static long sizeOfArray(int length, int elementSize)
{
return MemoryLayoutSpecification.sizeOfArray(length, elementSize);
return computeArraySize(MEMORY_LAYOUT.getArrayHeaderSize(), length, elementSize, MEMORY_LAYOUT.getObjectAlignment());
}

/**
Expand All @@ -129,65 +125,89 @@ public static long sizeOnHeapOf(ByteBuffer[] array)

/**
* Amount of non-data heap memory consumed by the array of byte buffers. It sums memory consumed
* by the array itself and for each included byte buffer using {@link #sizeOnHeapExcludingData(ByteBuffer)}.
* by the array itself and for each included byte buffer using {@link #sizeOnHeapExcludingDataOf(ByteBuffer)}.
*/
public static long sizeOnHeapExcludingData(ByteBuffer[] array)
public static long sizeOnHeapExcludingDataOf(ByteBuffer[] array)
{
if (array == null)
return 0;

long sum = sizeOfArray(array);
for (ByteBuffer b : array)
sum += sizeOnHeapExcludingData(b);
sum += sizeOnHeapExcludingDataOf(b);

return sum;
}

/**
* @return heap memory consumed by the byte buffer. If it is a slice, it counts the data size, but it does not
* include the internal array overhead.
* Measures the heap memory used by the specified byte buffer. If the buffer is a slab only the data size will be
* counted but not the internal overhead. A SLAB is assumed to be created by: {@code buffer.duplicate().position(start).limit(end)} without the use of {@code slice()}.
* <p>This method makes a certain amount of assumptions:
* <ul>
* <li>That slabs are always created using: {@code buffer.duplicate().position(start).limit(end)} and not through slice</li>
* <li>That the input buffers are not read-only buffers</li>
* <li>That the direct buffers that are not slab are not duplicates</li>
* </ul>
* Non-respect of those assumptions can lead to an invalid value being returned.
* @param buffer the buffer to measure
* @return the heap memory used by the specified byte buffer.
*/
public static long sizeOnHeapOf(ByteBuffer buffer)
{
if (buffer == null)
return 0;

if (buffer.isDirect())
return DIRECT_BUFFER_HEAP_SIZE;
assert !buffer.isReadOnly();

int arrayLen = buffer.array().length;
int bufLen = buffer.remaining();
// We assume here that slabs are always created using: buffer.duplicate().position(start).limit(end) and not through slice
if (ByteBufferMode.SLAB_ALLOCATION_NO_SLICE.isSlab(buffer))
{
if (buffer.isDirect())
return DIRECT_BUFFER_SHALLOW_SIZE; // We ignore the underlying buffer

// if we're only referencing a sub-portion of the ByteBuffer, don't count the array overhead (assume it is SLAB
// allocated - the overhead amortized over all the allocations is negligible and better to undercount than over)
if (arrayLen > bufLen)
return EMPTY_HEAP_BUFFER_SIZE + bufLen;
return HEAP_BUFFER_SHALLOW_SIZE + buffer.remaining(); // We ignore the array overhead
}

return EMPTY_HEAP_BUFFER_SIZE + (arrayLen == 0 ? EMPTY_BYTE_ARRAY_SIZE : sizeOfArray(arrayLen, 1));
if (buffer.isDirect())
return DIRECT_BUFFER_DEEP_SIZE; // That might not be true if the buffer is a view of another buffer so we could undercount

return HEAP_BUFFER_SHALLOW_SIZE + meter.measureArray(buffer.array());
}

/**
* @return non-data heap memory consumed by the byte buffer. If it is a slice, it does not include the internal
* array overhead.
* Measures the heap memory used by the specified byte buffer excluding the data. If the buffer shallow size will be counted.
* A SLAB is assumed to be created by: {@code buffer.duplicate().position(start).limit(end)} without the use of {@code slice()}.
* <p>This method makes a certain amount of assumptions:
* <ul>
* <li>That slabs are always created using: {@code buffer.duplicate().position(start).limit(end)} and not through slice</li>
* <li>That the input buffers are not read-only buffers</li>
* <li>That the direct buffers that are not slab are not duplicates</li>
* </ul>
* Non-respect of those assumptions can lead to an invalid value being returned. T
* @param buffer the buffer to measure
* @return the heap memory used by the specified byte buffer excluding the data..
*/
public static long sizeOnHeapExcludingData(ByteBuffer buffer)
public static long sizeOnHeapExcludingDataOf(ByteBuffer buffer)
{
if (buffer == null)
return 0;

assert !buffer.isReadOnly();

// We assume here that slabs are always created using: buffer.duplicate().position(start).limit(end) and not through slice
if (ByteBufferMode.SLAB_ALLOCATION_NO_SLICE.isSlab(buffer))
{
if (buffer.isDirect())
return DIRECT_BUFFER_HEAP_SIZE;
return DIRECT_BUFFER_SHALLOW_SIZE; // We ignore the underlying buffer

int arrayLen = buffer.array().length;
int bufLen = buffer.remaining();
return HEAP_BUFFER_SHALLOW_SIZE; // We ignore the array overhead
}

// if we're only referencing a sub-portion of the ByteBuffer, don't count the array overhead (assume it is SLAB
// allocated - the overhead amortized over all the allocations is negligible and better to undercount than over)
if (arrayLen > bufLen)
return EMPTY_HEAP_BUFFER_SIZE;
if (buffer.isDirect())
return DIRECT_BUFFER_DEEP_SIZE; // That might not be true if the buffer is a view of another buffer so we could undercount

// If buffers are dedicated, account for byte array size and any padding overhead
return EMPTY_HEAP_BUFFER_SIZE + (arrayLen == 0 ? EMPTY_BYTE_ARRAY_SIZE : (sizeOfArray(arrayLen, 1) - arrayLen));
byte[] bytes = buffer.array();
return HEAP_BUFFER_SHALLOW_SIZE + meter.measureArray(bytes) - bytes.length;
}

/**
Expand All @@ -196,13 +216,9 @@ public static long sizeOnHeapExcludingData(ByteBuffer buffer)
* @param str String to calculate memory size of
* @return Total in-memory size of the String
*/
// TODO hard coding this to 2 isn't necessarily correct in Java 11
public static long sizeOf(String str)
{
if (str == null)
return 0;

return EMPTY_STRING_SIZE + sizeOfArray(str.length(), Character.BYTES);
return meter.measureStringDeep(str);
}

/**
Expand All @@ -216,6 +232,18 @@ public static long measureDeep(Object pojo)
return meter.measureDeep(pojo);
}

/**
* @param pojo the object to measure
* @return The size on the heap of the instance and all retained heap referenced by it, excluding portions of
* ByteBuffer that are not directly referenced by it but including any other referenced that may also be retained
* by other objects. This also includes bytes referenced in direct byte buffers, and may double-count memory if
* it is referenced by multiple ByteBuffer copies.
*/
public static long measureDeepOmitShared(Object pojo)
{
return meter.measureDeep(pojo, ByteBufferMode.SLAB_ALLOCATION_NO_SLICE);
}

/**
* @param pojo the object to measure
* @return the size on the heap of the instance only, excluding any referenced objects
Expand All @@ -224,4 +252,20 @@ public static long measure(Object pojo)
{
return meter.measure(pojo);
}

private static InetAddress getIpvAddress(int size)
{
if (size == 16 || size ==4)
{
try
{
return InetAddress.getByAddress(new byte[size]);
}
catch (UnknownHostException e)
{
throw new IllegalArgumentException("Invalid size of a byte array when getting and ipv address: " + size);
}
}
else throw new IllegalArgumentException("Excpected a byte array size of 4 or 16 for an ipv address but got: " + size);
}
}
2 changes: 1 addition & 1 deletion test/unit/org/apache/cassandra/db/CellSpecTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ public void unsharedHeapSizeExcludingData()
private static long valuePtrSize(Object value)
{
if (value instanceof ByteBuffer)
return ObjectSizes.sizeOnHeapExcludingData((ByteBuffer) value);
return ObjectSizes.sizeOnHeapExcludingDataOf((ByteBuffer) value);
else if (value instanceof byte[])
return ObjectSizes.sizeOfArray((byte[]) value) - ((byte[]) value).length;
throw new IllegalArgumentException("Unsupported type by valuePtrSize: " + value.getClass());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,8 +47,9 @@ public abstract class MemtableSizeTestBase extends CQLTester
// The meter in ObjectSizes uses omitSharedBufferOverhead which counts off-heap data too
// Note: To see a printout of the usage for each object, add .enableDebug() here (most useful with smaller number of
// partitions).
static final MemoryMeter meter = new MemoryMeter().ignoreKnownSingletons()
.withGuessing(MemoryMeter.Guess.FALLBACK_UNSAFE);
private static final MemoryMeter meter = MemoryMeter.builder()
.withGuessing(MemoryMeter.Guess.INSTRUMENTATION, MemoryMeter.Guess.UNSAFE)
.build();

static String keyspace;
String table;
Expand Down
Loading

1 comment on commit 5696964

@cassci-bot
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Build rejected: 147 NEW test failure(s) in 1 builds., Build 1: ran 17720 tests with 160 failures and 127 skipped.
Butler analysis done on ds-cassandra-pr-gate/CNDB-11144 vs last 16 runs of ds-cassandra-build-nightly/main.
Showing only first 13 NEW test failures
test_cqlsh_output.TestCqlshOutput.test_boolean_output: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++]; [NEW]
test_cqlsh_output.TestCqlshOutput.test_blob_output: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++]; [NEW]
cqlsh_tests.test_cqlsh.TestCqlshSmoke.test_copy_stdout: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [+++++++++++++]; [NEW]
test_cqlsh_output.TestCqlshOutput.test_printing_cql_error: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++]; [NEW]
test_cqlsh_output.TestCqlshOutput.test_warn_py2: test is constantly failing. No results on upstream;
branch story: [F] vs upstream: [SSSSSSSSSSSSSSSSSSSSSSSSSSSSSS]; [NEW]
test_cqlsh_output.TestCqlshOutput.test_expanded_output_counts_past_page: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++]; [NEW]
replica_side_filtering_test.TestAllowFiltering.test_update_on_collection: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [+++++++++++++]; [NEW]
org.apache.cassandra.tools.ClearSnapshotTest.testClearSnapshot_RemoveMultiple: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.tools.nodetool.SetGetInterDCStreamThroughputTest.testMaxValue: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
test_cqlsh_output.TestCqlshOutput.test_no_prompt_or_colors_output: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++]; [NEW]
test_cqlsh_output.TestCqlshOutput.test_user_types_with_collections: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.tools.NodeToolTPStatsTest.testMaybeChangeDocs: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
test_cqlsh_output.TestCqlshOutput.test_numeric_output: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++]; [NEW]
butler comparison

Please sign in to comment.