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

Gary Dusbabek commented on CASSANDRA-700:
-----------------------------------------

Same comment from 693 regarding MessagingService:

This patch makes it so that MessageService cannot be reused after shutdown(). 
There is a significant amount of work that goes on in the constructor that was 
executed by instance() if shutdown() had been called previously.

One approach would be to replace shutdown() with a reset() that does what 
shutdown() does now but takes the additional step of re-initializing things 
(the work done in the constructor). Kind of lame, but it would make the code 
functionally similar to the way it was before the patch.

But then again, it might be wasted work since other services limit how 
reentrant cassandra can be overall. Perhaps a better approach would be to state 
unequivocally that the cassandra services *are not* reentrant and make sure 
that Bad Things happen if MessagingService is used after it's shutdown() has 
been called.

> Replacing the many forms of singleton instance methods
> ------------------------------------------------------
>
>                 Key: CASSANDRA-700
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-700
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Jeff Hodges
>         Attachments: 
> 0001-MessagingService.instance-MessagingService.instance.patch, 
> 0001-squashed-singleton-clean-up.patch, 
> 0002-HintedHandOffManager.instance-HintedHandOffManager.i.patch, 
> 0003-StorageService.instance-StorageService.instance.patch, 
> 0004-AntiEntropyService.instance-AntiEntropyService.insta.patch, 
> 0005-Gossiper.instance-Gossiper.instance.patch, 
> 0006-StorageLoadBalancer.instance-StorageLoadBalancer.ins.patch, 
> 0007-FailureDetector.instance-FailureDetector.instance.patch, 
> 0008-ReadRepairManager.instance-ReadRepairManager.instanc.patch
>
>
> In many places, the Cassandra codebase attempts to allow for only one 
> instance of a particular class per process. It does so in a variety of ways, 
> some of which are attempts at being thread-safe (with mixed results) while 
> others seem to just delay instantiation.
> This issue is to track the changes necessary to consolidate these many forms 
> of singleton into one form. 
> What's interesting is that Java has a nice way of providing this facility in 
> a very Java-y way.  We can create a public static variable (called, say, 
> instance) and, in the class definition, set it to a new instance of the very 
> class we are in. We then can protect the class's constructor to prevent 
> others from trying to use it. This allows us to have only one instance of the 
> variable accessible only through the a static variable on the class itself.
> This is thread-safe because class definitions in Java is thread-safe. This is 
> lazily loaded because Java doesn't do class definition until the first time 
> the class is referenced in any way. Basically, this is everything that was 
> attempted with the various static instance() methods but with complete 
> success.
> The current classes I was able to find that can benefit from this are:
> AntiEntropyService
> FailureDetector
> Gossiper
> HintedHandOffManager
> MessagingService
> ReadRepairManager
> StorageLoadBalancer
> StorageService
> Other classes with similar constructions would be nice to remove, too.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to