[ 
https://issues.apache.org/jira/browse/CASSANDRA-7124?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14219946#comment-14219946
 ] 

Rajanarayanan Thottuvaikkatumana commented on CASSANDRA-7124:
-------------------------------------------------------------

[~yukim], Thanks for the response. Some clarifications please.
1) Can you please tell me how did you get the exception captured? I searched in 
the system.log and could not find it. Instead of casting the executor, is that 
OK if I cast the Future<Object> to ListenableFuture<Object> which is the result 
of the executer.submit(....) ? When I tried executing the command 
"./bin/nodetool -h localhost cleanup" that resulted in the output of the 
following lines in the console and the system.log

INFO  20:07:08 No sstables for system_traces.sessions
INFO  20:07:08 No sstables for system_traces.events

The code that generates the above output is below from the StorageService.java
{code}
    private FutureTask<Object> createCleanupTask(final int cmd, final String 
keyspace, final ColumnFamilyStore cfStore)
    {
        return new FutureTask<>(new WrappedRunnable()
        {
            protected void runMayThrow() throws Exception
            {
                Iterable<SSTableReader> compactingSSTables = 
cfStore.markAllCompacting();
                if (compactingSSTables == null){
                        logger.info("Aborting operation on {}.{} after failing 
to interrupt other compaction operations", cfStore.keyspace.getName(), 
cfStore.name);
                        return;
                }
                if (Iterables.isEmpty(compactingSSTables))
                {
                    logger.info("No sstables for {}.{}", 
cfStore.keyspace.getName(), cfStore.name);
                    return;
                }               
                String message = String.format("Starting cleanup command #%d, 
cleaning up keyspace %s with column family store %s", cmd, keyspace, 
cfStore.name);
                logger.info(message);
                sendNotification("cleanup", message, new int[]{cmd, 
ActiveRepairService.Status.STARTED.ordinal()});
                                List<ListenableFuture<Object>> futures = 
cfStore.forceAsyncCleanup();
                                for(final ListenableFuture<Object> future: 
futures)
                                {
                                        Futures.addCallback(future, new 
FutureCallback<Object>()
                                        {
                                          public void onFailure(Throwable 
thrown) 
                                          {
                                                String message = "Failed 
cleanup job " + future.toString() + "with exception: " + thrown.getMessage();
                                                logger.info(message);  
                                            sendNotification("cleanup", 
message, new int[]{cmd, ActiveRepairService.Status.SESSION_FAILED.ordinal()});
                                          }     
                                          public void onSuccess(Object future) 
                                          {     
                                                String message = "Cleanup 
Session: " + future.toString() ;
                                                logger.info(message);
                                                sendNotification("cleanup", 
message, new int[]{cmd, ActiveRepairService.Status.SESSION_SUCCESS.ordinal()});
                                          }                                     
          
                                        });
                                        future.get();
                                }
                                
cfStore.getDataTracker().unmarkCompacting(compactingSSTables);
                                message = String.format("Ending cleanup command 
#%d, cleaning up keyspace %s with column family store %s", cmd, keyspace, 
cfStore.name);
                logger.info(message);
                sendNotification("cleanup", message, new int[]{cmd, 
ActiveRepairService.Status.FINISHED.ordinal()});                            
            }
        },null);
    }
{code}

2) Where are the current tests for the cleanup located? 
3) Regarding your comment "Make original sync method to use async method and 
block there. Those two have duplicate codes". Did you mean to change the 
original "forceKeyspaceCleanup" method in the StorageService.java. Please 
clarify. 

Thanks a lot

> Use JMX Notifications to Indicate Success/Failure of Long-Running Operations
> ----------------------------------------------------------------------------
>
>                 Key: CASSANDRA-7124
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7124
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Tools
>            Reporter: Tyler Hobbs
>            Assignee: Rajanarayanan Thottuvaikkatumana
>            Priority: Minor
>              Labels: lhf
>             Fix For: 3.0
>
>         Attachments: cassandra-trunk-cleanup-7124.txt
>
>
> If {{nodetool cleanup}} or some other long-running operation takes too long 
> to complete, you'll see an error like the one in CASSANDRA-2126, so you can't 
> tell if the operation completed successfully or not.  CASSANDRA-4767 fixed 
> this for repairs with JMX notifications.  We should do something similar for 
> nodetool cleanup, compact, decommission, move, relocate, etc.



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

Reply via email to