[
https://issues.apache.org/jira/browse/CASSANDRA-5725?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sergio Bossa updated CASSANDRA-5725:
------------------------------------
Attachment: 5725-0001.patch
> Silently failing messages in case of schema not fully propagated
> ----------------------------------------------------------------
>
> Key: CASSANDRA-5725
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5725
> Project: Cassandra
> Issue Type: Bug
> Affects Versions: 1.2.6
> Reporter: Sergio Bossa
> Attachments: 5725-0001.patch
>
>
> When a new keyspace and/or column family is created on a multi nodes cluster
> (at least three), and then a mutation is executed on such new column family,
> the operations sometimes silently fails by timing out.
> I tracked this down to the schema not being fully propagated to all nodes.
> Here's what happens:
> 1) Node 1 receives the create keyspace/column family request.
> 2) The same node receives a mutation request at CL.QUORUM and sends to other
> nodes too.
> 3) Upon receiving the mutation request, other nodes try to deserialize it and
> fail in doing so if the schema is not fully propagated, i.e. because they
> don't find the mutated column family.
> 4) The connection between node 1 and the failed node is dropped, and the
> request on the former hangs until timing out.
> Here is the underlying exception, I had to tweak several log levels to get
> it:
> {noformat}
> INFO 13:11:39,441 IOException reading from socket; closing
> org.apache.cassandra.db.UnknownColumnFamilyException: Couldn't find
> cfId=a31c7604-0e40-393b-82d7-ba3d910ad50a
> at
> org.apache.cassandra.db.ColumnFamilySerializer.deserializeCfId(ColumnFamilySerializer.java:184)
> at
> org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:94)
> at
> org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:397)
> at
> org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:407)
> at
> org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:367)
> at org.apache.cassandra.net.MessageIn.read(MessageIn.java:94)
> at
> org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:207)
> at
> org.apache.cassandra.net.IncomingTcpConnection.handleModernVersion(IncomingTcpConnection.java:139)
> at
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:82)
> {noformat}
> Finally, there's probably a correlated failure happening during repairs of
> newly created/mutated column family, causing the repair process to hang
> forever as follows:
> {noformat}
> "AntiEntropySessions:1" daemon prio=5 tid=7fe981148000 nid=0x11abea000 in
> Object.wait() [11abe9000]
> java.lang.Thread.State: WAITING (on object monitor)
> at java.lang.Object.wait(Native Method)
> - waiting on <7c6200840> (a org.apache.cassandra.utils.SimpleCondition)
> at java.lang.Object.wait(Object.java:485)
> at
> org.apache.cassandra.utils.SimpleCondition.await(SimpleCondition.java:34)
> - locked <7c6200840> (a org.apache.cassandra.utils.SimpleCondition)
> at
> org.apache.cassandra.service.AntiEntropyService$RepairSession.runMayThrow(AntiEntropyService.java:695)
> at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:439)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
> at java.util.concurrent.FutureTask.run(FutureTask.java:138)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
> at java.lang.Thread.run(Thread.java:680)
> "http-8983-1" daemon prio=5 tid=7fe97d24d000 nid=0x11a5c8000 in Object.wait()
> [11a5c6000]
> java.lang.Thread.State: WAITING (on object monitor)
> at java.lang.Object.wait(Native Method)
> - waiting on <7c620db58> (a org.apache.cassandra.utils.SimpleCondition)
> at java.lang.Object.wait(Object.java:485)
> at
> org.apache.cassandra.utils.SimpleCondition.await(SimpleCondition.java:34)
> - locked <7c620db58> (a org.apache.cassandra.utils.SimpleCondition)
> at
> org.apache.cassandra.service.StorageService$4.runMayThrow(StorageService.java:2442)
> at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:439)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
> at java.util.concurrent.FutureTask.run(FutureTask.java:138)
> at
> org.apache.cassandra.service.StorageService.forceTableRepairRange(StorageService.java:2409)
> at
> org.apache.cassandra.service.StorageService.forceTableRepair(StorageService.java:2387)
> at
> com.datastax.bdp.cassandra.index.solr.SolrCoreResourceManager.repairResources(SolrCoreResourceManager.java:693)
> at
> com.datastax.bdp.cassandra.index.solr.SolrCoreResourceManager.createCore(SolrCoreResourceManager.java:255)
> at
> com.datastax.bdp.cassandra.index.solr.CassandraCoreAdminHandler.handleCreateAction(CassandraCoreAdminHandler.java:121)
> at
> org.apache.solr.handler.admin.CoreAdminHandler.handleRequestBody(CoreAdminHandler.java:144)
> at
> org.apache.solr.handler.RequestHandlerBase.handleRequest(RequestHandlerBase.java:135)
> at
> org.apache.solr.servlet.SolrDispatchFilter.handleAdminRequest(SolrDispatchFilter.java:615)
> at
> org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:206)
> {noformat}
> I wasn't able to track any exception as I can't reproduce it reliably enough,
> but I believe it's correlated to schema propagation as based on log messages
> the merkle tree request on node 1 happens concurrently to schema installation
> on other nodes.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira