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

Jonathan Ellis updated CASSANDRA-3727:
--------------------------------------

    Attachment: 3727.txt

So, I was over-optimistic in CASSANDRA-3335 when I thought I could get by 
without a MessagingService shutdown method.  The problem is that although my 
changes there do work to prevent accepting new connections, and to stop work on 
existing connections *after the first command post-shutdown*, that's not good 
enough in this case since the client is just sitting on its connection and 
never sends another command.

So, this patch renames MS.waitForCallbacks() back to shutdown(), and refuses to 
add new callbacks after that.  However, the analysis on 3335 that there's no 
good way to deal with an exception here, so we do this instead in ExpiringMap:

{code}
.   public V put(K key, V value, long timeout)
    {
        if (shutdown)
        {
            // StorageProxy isn't equipped to deal with "I'm nominally alive, 
but I can't send any messages out."
            // So we'll just sit on this thread for until the rest of the 
server shutdown completes.
            //
            // See comments in CustomTThreadPoolServer.serve, CASSANDRA-3335, 
and CASSANDRA-3727.
            try
            {
                Thread.sleep(Long.MAX_VALUE);
            }
            catch (InterruptedException e)
            {
                throw new AssertionError(e);
            }
        }
        CacheableObject<V> previous = cache.put(key, new 
CacheableObject<V>(value, timeout));
        return (previous == null) ? null : previous.getValue();
    }
{code}

Then, we switch the Thrift executor (and all DTPE instances) to use daemon 
threads, and remove the wait-for-WorkerProcess threads code from 
CustomTThreadPoolServer.serve:

{code}
.       // Thrift's default shutdown waits for the WorkerProcess threads to 
complete.  We do not,
        // because doing that allows a client to hold our shutdown "hostage" by 
simply not sending
        // another message after stop is called (since process will block 
indefinitely trying to read
        // the next meessage header).
        //
        // The "right" fix would be to update thrift to set a socket timeout on 
client connections
        // (and tolerate unintentional timeouts until stopped_ is set).  But 
this requires deep
        // changes to the code generator, so simply setting these threads to 
daemon (in our custom
        // CleaningThreadPool) and ignoring them after shutdown is good enough.
        //
        // Remember, our goal on shutdown is not necessarily that each client 
request we receive
        // gets answered first [to do that, you should redirect clients to a 
different coordinator
        // first], but rather (1) to make sure that for each update we ack as 
successful, we generate
        // hints for any non-responsive replicas, and (2) to make sure that we 
quickly stop
        // accepting client connections so shutdown can continue.  Not waiting 
for the WorkerProcess
        // threads here accomplishes (2); MessagingService's shutdown method 
takes care of (1).
        //
        // See CASSANDRA-3335 and CASSANDRA-3727.
{code}

Finally, this patch also updates Memtable's memorymeter thread to use the newly 
daemonized DTPE for good measure, since there's no reason to ever block 
shutdown for that either.
                
> Fix unit tests failure
> ----------------------
>
>                 Key: CASSANDRA-3727
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3727
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Tests
>    Affects Versions: 1.0.7
>            Reporter: Sylvain Lebresne
>            Priority: Blocker
>             Fix For: 1.0.7
>
>         Attachments: 3727.txt, CASSANDRA-3727-CliTest-timeout-fix.patch
>
>
> On current 1.0 branch (and on my machine: Linux), I have the following unit 
> test failures:
> * CliTest and EmbeddedCassandraTest: they both first kind of pass (JUnit 
> first prints a message with no failures in it), then hang until JUnit timeout 
> and fails with a 'Timeout occurred'. In other word, the tests themselves are 
> passing, but something they do prevents the process to exit cleanly leading 
> to a JUnit timeout. I don't want to discard that as not a problem, because if 
> something can make the process not exit cleanly, this can be a pain for 
> restarts (and in particular upgrades) and hence would be basically a 
> regression. I'm marking the ticket as blocker (for the release of 1.0.7) 
> mostly because of this one.
> * SystemTableTest: throws an assertionError. I haven't checked yet, so that 
> could be an easy one to fix.
> * RemoveTest: it fails, saying that '/127.0.0.1:7010 is in use by another 
> process' (consistently). But I have no other process running on port 7010. 
> It's likely just of problem of the test, but it's new and in the meantime 
> removes are not tested.
> * I also see a bunch of stack trace with errors like:
> {noformat}
>     [junit] ERROR 10:01:59,007 Fatal exception in thread 
> Thread[NonPeriodicTasks:1,5,main]
>     [junit] java.lang.RuntimeException: java.io.IOException: Unable to create 
> hard link from build/test/cassandra/data/Keyspace1/Indexed1-hc-1-Index.db to 
> /home/mcmanus/Git/cassandra/build/test/cassandra/data/Keyspace1/backups/Indexed1-hc-1-Index.db
>  (errno 17)
> {noformat}
> (with SSTableReaderTest). This does not make the tests fail, but it is still 
> worth investigating. It may be due to CASSANDRA-3101.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to