You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "David Capwell (Jira)" <ji...@apache.org> on 2022/07/28 00:49:00 UTC
[jira] [Commented] (CASSANDRA-17103) CEP-15 (C*): Messaging and storage engine integration
[ https://issues.apache.org/jira/browse/CASSANDRA-17103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17572183#comment-17572183 ]
David Capwell commented on CASSANDRA-17103:
-------------------------------------------
working on patches for the branch, but publishing my findings
* Attempting to read a Command while it is still active yields (this node didn't see Commit and Apply)
The following error hangs the tx (relies on timeout), so likely unexpected throws or unchecked future
{code}
ERROR [node2_ReadStage-2] node2 2022-07-27 13:04:37,293 JVMStabilityInspector.java:68 - Exception in thread Thread[node2_ReadStage-2,5,SharedPool]
java.lang.IllegalStateException: Cannot load into a field with unsaved changes
at org.apache.cassandra.service.accord.store.AbstractStoredField.preLoad(AbstractStoredField.java:84)
at org.apache.cassandra.service.accord.store.StoredNavigableMap.load(StoredNavigableMap.java:95)
at org.apache.cassandra.service.accord.AccordKeyspace.loadCommand(AccordKeyspace.java:629)
at org.apache.cassandra.service.accord.async.AsyncLoader.lambda$referenceAndDispatchReads$0(AsyncLoader.java:114)
at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:81)
at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:47)
at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:57)
at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:120)
at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
{code}
* We are modifying the context while populating it (same thread)
The following error hangs the tx (relies on timeout), so likely unexpected throws or unchecked future
{code}
ERROR [node2_AccordStage-3] node2 2022-07-27 12:55:52,825 JVMStabilityInspector.java:68 - Exception in thread Thread[node2_AccordStage-3,5,SharedPool]
java.lang.RuntimeException: java.util.ConcurrentModificationException
at accord.local.CommandStores.reduce(CommandStores.java:223)
at accord.local.CommandStores.mapReduce(CommandStores.java:238)
at accord.local.CommandStores.mapReduce(CommandStores.java:243)
at accord.local.CommandStores.mapReduce(CommandStores.java:268)
at accord.local.Node.mapReduceLocal(Node.java:202)
at accord.messages.Apply.process(Apply.java:80)
at org.apache.cassandra.service.accord.AccordVerbHandler.doVerb(AccordVerbHandler.java:46)
at org.apache.cassandra.net.InboundSink.lambda$new$0(InboundSink.java:78)
at org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:64)
at org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:50)
at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:97)
at org.apache.cassandra.distributed.impl.Instance.lambda$receiveMessageRunnable$6(Instance.java:485)
at org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:124)
at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:120)
at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: java.util.ConcurrentModificationException: null
at java.base/java.util.HashMap$Values.forEach(HashMap.java:980)
at org.apache.cassandra.service.accord.async.AsyncWriter.denormalize(AsyncWriter.java:230)
at org.apache.cassandra.service.accord.async.AsyncWriter.save(AsyncWriter.java:251)
at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:100)
at org.apache.cassandra.service.accord.async.AsyncOperation.callback(AsyncOperation.java:77)
at org.apache.cassandra.utils.concurrent.ListenerList$CallbackBiConsumerListener.run(ListenerList.java:247)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
... 1 common frames omitted
{code}
The following error hangs the tx (relies on timeout), so likely unexpected throws or unchecked future
{code}
ERROR [node1_AccordScheduled:1] node1 2022-07-27 17:22:40,508 JVMStabilityInspector.java:68 - Exception in thread Thread[node1_AccordScheduled:1,5,AccordScheduled]
java.lang.RuntimeException: java.lang.IllegalArgumentException
at accord.local.CommandStores.reduce(CommandStores.java:223)
at accord.local.CommandStores.mapReduce(CommandStores.java:376)
at accord.messages.ReadData$LocalRead$ReportWaiting.run(ReadData.java:86)
at org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:124)
at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
at java.base/java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:264)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java)
at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: java.lang.IllegalArgumentException: null
at com.google.common.base.Preconditions.checkArgument(Preconditions.java:127)
at org.apache.cassandra.service.accord.AccordCommandStore.command(AccordCommandStore.java:164)
at accord.messages.ReadData$LocalRead$ReportWaiting.lambda$run$0(ReadData.java:87)
at org.apache.cassandra.service.accord.async.AsyncOperation$ForFunction.apply(AsyncOperation.java:140)
at org.apache.cassandra.service.accord.async.AsyncOperation$ForFunction.apply(AsyncOperation.java:127)
at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:96)
at org.apache.cassandra.service.accord.async.AsyncOperation.callback(AsyncOperation.java:77)
at org.apache.cassandra.utils.concurrent.ListenerList$CallbackBiConsumerListener.run(ListenerList.java:247)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
... 1 common frames omitted
{code}
> CEP-15 (C*): Messaging and storage engine integration
> -----------------------------------------------------
>
> Key: CASSANDRA-17103
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17103
> Project: Cassandra
> Issue Type: New Feature
> Components: Accord
> Reporter: Benedict Elliott Smith
> Priority: Normal
>
> This work encompasses implementing Accord’s storage and networking interfaces within Cassandra, so that messages may be sent around the cluster and exectuted
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org