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

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

[~yukim] I had tried using CompactionExecutor earlier and I had issues with 
that because it's submit method returns Future instead of the ListenableFuture. 
In the "repair"'s code flow they use the ListeningExecutorService and the 
equivalent for the cleanup will be something like 
{code}
ListeningExecutorService service = MoreExecutors.listeningDecorator(new 
JMXConfigurableThreadPoolExecutor(1,
                        Integer.MAX_VALUE,
                        TimeUnit.SECONDS,
                        new LinkedBlockingQueue<Runnable>(),
                        new NamedThreadFactory("Cleanup#" + 
cfs.keyspace.getName() + " " + cfs.name + " " + 
Integer.toString(sstable.hashCode())),
                        "internal"));                   
                        ListenableFuture<Object> sstableFuture = 
service.submit(new Callable<Object>()
{code}

Now in the {{parallelAsyncAllSSTableOperation}} method of the 
CompactionManager.java, the reason why I had used {{Future#get}} is because 
there is a {{Iterable<SSTableReader> compactingSSTables = 
cfs.markAllCompacting();}} in the beginning and one 
{{cfs.getDataTracker().unmarkCompacting(compactingSSTables);}} at the end. I 
guess you cannot call the unmark operation without completing the job. During 
my testing, I had found that if we remove the {{Future#get}}, the cleanup tests 
are all failing. 

Please advice. Thanks

> 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