cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Brandon Williams (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (CASSANDRA-8741) Running a drain before a decommission apparently the wrong thing to do
Date Wed, 26 Aug 2015 18:42:46 GMT

     [ https://issues.apache.org/jira/browse/CASSANDRA-8741?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Brandon Williams updated CASSANDRA-8741:
----------------------------------------
    Reviewer: Stefania

> Running a drain before a decommission apparently the wrong thing to do
> ----------------------------------------------------------------------
>
>                 Key: CASSANDRA-8741
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8741
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Ubuntu 14.04; Cassandra 2.0.11.82 (Datastax Enterprise 4.5.3)
>            Reporter: Casey Marshall
>            Assignee: Jan Karlsson
>            Priority: Trivial
>              Labels: lhf
>             Fix For: 2.1.x, 2.0.x
>
>         Attachments: 8741.txt
>
>
> This might simply be a documentation issue. It appears that running "nodetool drain"
is a very wrong thing to do before running a "nodetool decommission".
> The idea was that I was going to safely shut off writes and flush everything to disk
before beginning the decommission. What happens is the "decommission" call appears to fail
very early on after starting, and afterwards, the node in question is stuck in state LEAVING,
but all other nodes in the ring see that node as NORMAL, but down. No streams are ever sent
from the node being decommissioned to other nodes.
> The drain command does indeed shut down the "BatchlogTasks" executor (org/apache/cassandra/service/StorageService.java,
line 3445 in git tag "cassandra-2.0.11") but the decommission process tries using that executor
when calling the "startBatchlogReplay" function (org/apache/cassandra/db/BatchlogManager.java,
line 123) called through org.apache.cassandra.service.StorageService.unbootstrap (see the
stack trace pasted below).
> This also failed in a similar way on Cassandra 1.2.13-ish (DSE 3.2.4).
> So, either something is wrong with the drain/decommission commands, or it's very wrong
to run a drain before a decommission. What's worse, there seems to be no way to recover this
node once it is in this state; you need to shut it down and run "removenode".
> My terminal output:
> {code}
> ubuntu@x:~$ nodetool drain
> ubuntu@x:~$ tail /var/log/^C
> ubuntu@x:~$ nodetool decommission
> Exception in thread "main" java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@3008fa33
rejected from org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor@1d6242e8[Terminated,
pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 52]
>         at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2048)
>         at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:821)
>         at java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:325)
>         at java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:530)
>         at java.util.concurrent.ScheduledThreadPoolExecutor.submit(ScheduledThreadPoolExecutor.java:629)
>         at org.apache.cassandra.db.BatchlogManager.startBatchlogReplay(BatchlogManager.java:123)
>         at org.apache.cassandra.service.StorageService.unbootstrap(StorageService.java:2966)
>         at org.apache.cassandra.service.StorageService.decommission(StorageService.java:2934)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:606)
>         at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:75)
>         at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:606)
>         at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:279)
>         at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112)
>         at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46)
>         at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237)
>         at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138)
>         at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252)
>         at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819)
>         at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:801)
>         at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1487)
>         at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:97)
>         at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1328)
>         at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1420)
>         at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:848)
>         at sun.reflect.GeneratedMethodAccessor59.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:606)
>         at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322)
>         at sun.rmi.transport.Transport$1.run(Transport.java:177)
>         at sun.rmi.transport.Transport$1.run(Transport.java:174)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at sun.rmi.transport.Transport.serviceCall(Transport.java:173)
>         at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:556)
>         at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:811)
>         at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:670)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:745){code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message