[ https://issues.apache.org/jira/browse/CASSANDRA-16304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17585634#comment-17585634 ]
Ekaterina Dimitrova edited comment on CASSANDRA-16304 at 8/26/22 11:17 PM: --------------------------------------------------------------------------- So seems to me the particular issue here in this ticket is when we do not use Instrumentation but fall back to unsafe and use objectFieldOffset method which was changed in the unsupported package version of Unsafe to not support hidden classes and records. Why I mention "in the unsupported package", because it calls the internal old version after that. I do not advocate to switch to that one and open internals... One idea was to explore JOL. (thanks [~dcapwell] for bringing it up) Otherwise, I guess for jamm we can explore the Variable Handles which are supposed to substitute Unsafe as per [JEP193|https://openjdk.org/jeps/193] In theory their goal is to provide same or similar performance, but definitely we would need to do some testing. Also, I noticed the [latest jamm work|https://github.com/jbellis/jamm/pull/41] from [~snazy] was actually supposed to deprecate the *Unsafe* version (unreleased) It seems he was planning to leave two *GUESS* options in the future - *BEST* (I{_}nstrumentation --> Unsafe -> Specification,{_} depending on what Is available) and *ALWAYS_INSTRUMENTATION* and deprecate the old ones. Around jamm we actually also hit this on startup which we can workaround with opening internals but I guess we need to look for solution to that too: {code:java} ERROR [ScheduledTasks:1] 2022-08-23 12:33:23,630 JVMStabilityInspector.java:68 - Exception in thread Thread[ScheduledTasks:1,5,ScheduledTasks] java.lang.ExceptionInInitializerError: null at org.apache.cassandra.streaming.StreamManager.<init>(StreamManager.java:257) at org.apache.cassandra.streaming.StreamManager.<clinit>(StreamManager.java:58) at org.apache.cassandra.service.StorageService.<init>(StorageService.java:376) at org.apache.cassandra.service.StorageService.<clinit>(StorageService.java:226) at org.apache.cassandra.locator.DynamicEndpointSnitch.updateScores(DynamicEndpointSnitch.java:274) at org.apache.cassandra.locator.DynamicEndpointSnitch$1.run(DynamicEndpointSnitch.java:91) at org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:133) at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539) at java.base/java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305) at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.base/java.lang.Thread.run(Thread.java:833) Caused by: java.lang.reflect.InaccessibleObjectException: Unable to make field transient volatile java.util.concurrent.ConcurrentHashMap$Node[] java.util.concurrent.ConcurrentHashMap.table accessible: module java.base does not "opens java.util.concurrent" to unnamed module @4c012563 at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:354) at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:297) at java.base/java.lang.reflect.Field.checkCanSetAccessible(Field.java:178) at java.base/java.lang.reflect.Field.setAccessible(Field.java:172) at org.github.jamm.MemoryMeter.addFieldChildren(MemoryMeter.java:330) at org.github.jamm.MemoryMeter.measureDeep(MemoryMeter.java:269) at org.apache.cassandra.utils.ObjectSizes.measureDeep(ObjectSizes.java:215) at org.apache.cassandra.streaming.StreamingState.<clinit>(StreamingState.java:50) ... 14 common frames omitted ERROR [ScheduledTasks:1] 2022-08-23 12:35:20,677 JVMStabilityInspector.java:68 - Exception in thread Thread[ScheduledTasks:1,5,ScheduledTasks] java.lang.ExceptionInInitializerError: null at org.apache.cassandra.streaming.StreamManager.<init>(StreamManager.java:257) at org.apache.cassandra.streaming.StreamManager.<clinit>(StreamManager.java:58) at org.apache.cassandra.service.StorageService.<init>(StorageService.java:376) at org.apache.cassandra.service.StorageService.<clinit>(StorageService.java:226) at org.apache.cassandra.locator.DynamicEndpointSnitch.updateScores(DynamicEndpointSnitch.java:274) at org.apache.cassandra.locator.DynamicEndpointSnitch$1.run(DynamicEndpointSnitch.java:91) at org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:133) at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539) at java.base/java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305) at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.base/java.lang.Thread.run(Thread.java:833) Caused by: java.lang.reflect.InaccessibleObjectException: Unable to make field transient java.util.Set java.util.AbstractMap.keySet accessible: module java.base does not "opens java.util" to unnamed module @4c012563 at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:354) at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:297) at java.base/java.lang.reflect.Field.checkCanSetAccessible(Field.java:178) at java.base/java.lang.reflect.Field.setAccessible(Field.java:172) at org.github.jamm.MemoryMeter.addFieldChildren(MemoryMeter.java:330) at org.github.jamm.MemoryMeter.measureDeep(MemoryMeter.java:269) at org.apache.cassandra.utils.ObjectSizes.measureDeep(ObjectSizes.java:215) at org.apache.cassandra.streaming.StreamingState.<clinit>(StreamingState.java:50) ... 14 common frames omitted Exception (java.lang.ExceptionInInitializerError) encountered during startup: null java.lang.ExceptionInInitializerError at org.apache.cassandra.db.memtable.SkipListMemtableFactory.create(SkipListMemtableFactory.java:44) at org.apache.cassandra.db.ColumnFamilyStore.createMemtable(ColumnFamilyStore.java:1364) at org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:492) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:751) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:738) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:729) at org.apache.cassandra.db.Keyspace.initCf(Keyspace.java:460) at org.apache.cassandra.db.Keyspace.<init>(Keyspace.java:350) at org.apache.cassandra.db.Keyspace.lambda$open$0(Keyspace.java:163) at org.apache.cassandra.utils.concurrent.LoadingMap.blockingLoadIfAbsent(LoadingMap.java:105) at org.apache.cassandra.schema.Schema.maybeAddKeyspaceInstance(Schema.java:228) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:163) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:152) at org.apache.cassandra.db.SystemKeyspace.checkHealth(SystemKeyspace.java:1069) at org.apache.cassandra.service.StartupChecks$13.execute(StartupChecks.java:665) at org.apache.cassandra.service.StartupChecks.verify(StartupChecks.java:175) at org.apache.cassandra.service.CassandraDaemon.runStartupChecks(CassandraDaemon.java:490) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:257) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:755) at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:879) Caused by: java.lang.reflect.InaccessibleObjectException: Unable to make field transient volatile java.util.concurrent.atomic.Striped64$Cell[] java.util.concurrent.atomic.Striped64.cells accessible: module java.base does not "opens java.util.concurrent.atomic" to unnamed module @4c012563 at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:354) at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:297) at java.base/java.lang.reflect.Field.checkCanSetAccessible(Field.java:178) at java.base/java.lang.reflect.Field.setAccessible(Field.java:172) at org.github.jamm.MemoryMeter.addFieldChildren(MemoryMeter.java:330) at org.github.jamm.MemoryMeter.measureDeep(MemoryMeter.java:269) at org.apache.cassandra.utils.ObjectSizes.measureDeep(ObjectSizes.java:215) at org.apache.cassandra.db.memtable.SkipListMemtable.estimateRowOverhead(SkipListMemtable.java:232) at org.apache.cassandra.db.memtable.SkipListMemtable.<clinit>(SkipListMemtable.java:77) ... 20 more {code} Also, I think this is just an excerpt of what we will need to open If someone suggests to go into that direction...I remember while testing before C* with Java 17 there were tests where jamm got quite "hungry" for internals so we will need some better solution, definitely was (Author: e.dimitrova): So seems to me the issue here is when we do not use Instrumentation but fall back to unsafe and use objectFieldOffset method which was changed in the unsupported package version of Unsafe to not support hidden classes and records. Why I mention "in the unsupported package", because it calls the internal old version after that. I do not advocate to switch to that one and open internals... One idea was to explore JOL. (thanks [~dcapwell] for bringing it up) Otherwise, I guess for jamm we can explore the Variable Handles which are supposed to substitute Unsafe as per [JEP193|https://openjdk.org/jeps/193] In theory their goal is to provide same or similar performance, but definitely we would need to do some testing. Also, I noticed the [latest jamm work|https://github.com/jbellis/jamm/pull/41] from [~snazy] was actually supposed to deprecate the *Unsafe* version (unreleased) It seems he was planning to leave two *GUESS* options in the future - *BEST* (I{_}nstrumentation --> Unsafe -> Specification,{_} depending on what Is available) and *ALWAYS_INSTRUMENTATION* and deprecate the old ones. Around jamm we actually also hit this on startup which we can workaround with opening internals but I guess we need to look for solution to that too: {code:java} ERROR [ScheduledTasks:1] 2022-08-23 12:33:23,630 JVMStabilityInspector.java:68 - Exception in thread Thread[ScheduledTasks:1,5,ScheduledTasks] java.lang.ExceptionInInitializerError: null at org.apache.cassandra.streaming.StreamManager.<init>(StreamManager.java:257) at org.apache.cassandra.streaming.StreamManager.<clinit>(StreamManager.java:58) at org.apache.cassandra.service.StorageService.<init>(StorageService.java:376) at org.apache.cassandra.service.StorageService.<clinit>(StorageService.java:226) at org.apache.cassandra.locator.DynamicEndpointSnitch.updateScores(DynamicEndpointSnitch.java:274) at org.apache.cassandra.locator.DynamicEndpointSnitch$1.run(DynamicEndpointSnitch.java:91) at org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:133) at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539) at java.base/java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305) at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.base/java.lang.Thread.run(Thread.java:833) Caused by: java.lang.reflect.InaccessibleObjectException: Unable to make field transient volatile java.util.concurrent.ConcurrentHashMap$Node[] java.util.concurrent.ConcurrentHashMap.table accessible: module java.base does not "opens java.util.concurrent" to unnamed module @4c012563 at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:354) at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:297) at java.base/java.lang.reflect.Field.checkCanSetAccessible(Field.java:178) at java.base/java.lang.reflect.Field.setAccessible(Field.java:172) at org.github.jamm.MemoryMeter.addFieldChildren(MemoryMeter.java:330) at org.github.jamm.MemoryMeter.measureDeep(MemoryMeter.java:269) at org.apache.cassandra.utils.ObjectSizes.measureDeep(ObjectSizes.java:215) at org.apache.cassandra.streaming.StreamingState.<clinit>(StreamingState.java:50) ... 14 common frames omitted ERROR [ScheduledTasks:1] 2022-08-23 12:35:20,677 JVMStabilityInspector.java:68 - Exception in thread Thread[ScheduledTasks:1,5,ScheduledTasks] java.lang.ExceptionInInitializerError: null at org.apache.cassandra.streaming.StreamManager.<init>(StreamManager.java:257) at org.apache.cassandra.streaming.StreamManager.<clinit>(StreamManager.java:58) at org.apache.cassandra.service.StorageService.<init>(StorageService.java:376) at org.apache.cassandra.service.StorageService.<clinit>(StorageService.java:226) at org.apache.cassandra.locator.DynamicEndpointSnitch.updateScores(DynamicEndpointSnitch.java:274) at org.apache.cassandra.locator.DynamicEndpointSnitch$1.run(DynamicEndpointSnitch.java:91) at org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:133) at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539) at java.base/java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305) at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.base/java.lang.Thread.run(Thread.java:833) Caused by: java.lang.reflect.InaccessibleObjectException: Unable to make field transient java.util.Set java.util.AbstractMap.keySet accessible: module java.base does not "opens java.util" to unnamed module @4c012563 at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:354) at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:297) at java.base/java.lang.reflect.Field.checkCanSetAccessible(Field.java:178) at java.base/java.lang.reflect.Field.setAccessible(Field.java:172) at org.github.jamm.MemoryMeter.addFieldChildren(MemoryMeter.java:330) at org.github.jamm.MemoryMeter.measureDeep(MemoryMeter.java:269) at org.apache.cassandra.utils.ObjectSizes.measureDeep(ObjectSizes.java:215) at org.apache.cassandra.streaming.StreamingState.<clinit>(StreamingState.java:50) ... 14 common frames omitted Exception (java.lang.ExceptionInInitializerError) encountered during startup: null java.lang.ExceptionInInitializerError at org.apache.cassandra.db.memtable.SkipListMemtableFactory.create(SkipListMemtableFactory.java:44) at org.apache.cassandra.db.ColumnFamilyStore.createMemtable(ColumnFamilyStore.java:1364) at org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:492) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:751) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:738) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:729) at org.apache.cassandra.db.Keyspace.initCf(Keyspace.java:460) at org.apache.cassandra.db.Keyspace.<init>(Keyspace.java:350) at org.apache.cassandra.db.Keyspace.lambda$open$0(Keyspace.java:163) at org.apache.cassandra.utils.concurrent.LoadingMap.blockingLoadIfAbsent(LoadingMap.java:105) at org.apache.cassandra.schema.Schema.maybeAddKeyspaceInstance(Schema.java:228) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:163) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:152) at org.apache.cassandra.db.SystemKeyspace.checkHealth(SystemKeyspace.java:1069) at org.apache.cassandra.service.StartupChecks$13.execute(StartupChecks.java:665) at org.apache.cassandra.service.StartupChecks.verify(StartupChecks.java:175) at org.apache.cassandra.service.CassandraDaemon.runStartupChecks(CassandraDaemon.java:490) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:257) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:755) at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:879) Caused by: java.lang.reflect.InaccessibleObjectException: Unable to make field transient volatile java.util.concurrent.atomic.Striped64$Cell[] java.util.concurrent.atomic.Striped64.cells accessible: module java.base does not "opens java.util.concurrent.atomic" to unnamed module @4c012563 at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:354) at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:297) at java.base/java.lang.reflect.Field.checkCanSetAccessible(Field.java:178) at java.base/java.lang.reflect.Field.setAccessible(Field.java:172) at org.github.jamm.MemoryMeter.addFieldChildren(MemoryMeter.java:330) at org.github.jamm.MemoryMeter.measureDeep(MemoryMeter.java:269) at org.apache.cassandra.utils.ObjectSizes.measureDeep(ObjectSizes.java:215) at org.apache.cassandra.db.memtable.SkipListMemtable.estimateRowOverhead(SkipListMemtable.java:232) at org.apache.cassandra.db.memtable.SkipListMemtable.<clinit>(SkipListMemtable.java:77) ... 20 more {code} Also, I think this is just an excerpt of what we will need to open If someone suggests to go into that direction...I remember while testing before C* with Java 17 there were tests where jamm got quite "hungry" for internals so we will need some better solution, definitely > Consider implementing ClusteringComparator without a lambda > ----------------------------------------------------------- > > Key: CASSANDRA-16304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-16304 > Project: Cassandra > Issue Type: Improvement > Components: Build > Reporter: Adrian Cole > Priority: Normal > Fix For: 4.x > > > Using lambdas forces jamm to do things that can easily break. It might be > safer to implement things like ClusteringComparator directly as classes or as > an enum > {noformat} > Unexpected exception during request > (org.apache.cassandra.transport.messages.ErrorMessage) > java.lang.UnsupportedOperationException: can't get field offset on a hidden > class: private final org.apache.cassandra.db.ClusteringComparator > org.apache.cassandra.db.ClusteringComparator$$Lambda$165/0x000000010028ab60.arg$1 > at jdk.unsupported/sun.misc.Unsafe.objectFieldOffset(Unknown Source) > at > org.github.jamm.MemoryLayoutSpecification.sizeOfInstanceWithUnsafe(MemoryLayoutSpecification.java:108) > at > org.github.jamm.MemoryLayoutSpecification.sizeOfWithUnsafe(MemoryLayoutSpecification.java:89) > at org.github.jamm.MemoryMeter.measure(MemoryMeter.java:217) > at org.github.jamm.MemoryMeter.measureDeep(MemoryMeter.java:259) > at > org.apache.cassandra.utils.ObjectSizes.measureDeep(ObjectSizes.java:155) > at > org.apache.cassandra.cql3.QueryProcessor.storePreparedStatement(QueryProcessor.java:454) > at > org.apache.cassandra.cql3.QueryProcessor.prepare(QueryProcessor.java:424) > at > org.apache.cassandra.cql3.QueryProcessor.prepare(QueryProcessor.java:408) > at > org.apache.cassandra.transport.messages.PrepareMessage.execute(PrepareMessage.java:114) > at > org.apache.cassandra.transport.Message$Request.execute(Message.java:253) > at > org.apache.cassandra.transport.Message$Dispatcher.processRequest(Message.java:725) > at > org.apache.cassandra.transport.Message$Dispatcher.lambda$channelRead0$0(Message.java:630) > at > java.base/java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162) > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:119) > at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > at java.base/java.lang.Thread.run(Unknown Source) > {noformat} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org