Yeah, a weak map would’ve made things easier but the map itself is unnecessary. When I wrote it I wasn’t sure how ZK was implemented internally. Of course, I’m now taking advantage of internal knowledge of ZK but there’s a lot of that in Curator and I feel pretty confident it won’t change anytime soon.
NamespaceWatcher is a package protected internal class and is only ever used to wrap passed in Watchers/CuratorWatchers and then passed into ZK. So, the missing comparisons don’t concern me. > The only part that bugs me is having NamespaceWatcher.equals(raw Watcher). This is required and is the “magic” that makes removing the Map possible. This way, I can pass in new NamespaceWatcher instances each time but have them compare equal to the wrapped Watcher. This is vital. What this is doing is creating a proxy that allows a passed in Watcher to be wrapped but appear as equal inside of ZK. -Jordan > On Feb 10, 2016, at 11:30 AM, Scott Blum <[email protected]> wrote: > > Here's where I am right this second. I looked back over commit > ff8a795e61d0d44622bdbaf2144c25c70e31e864, and I think I understand it about > 90%. I suspect the issue might have been solved by simply having the > original NamespaceWatcherMap have weak keys and weak values-- it only had > weak values, but again I don't have the 100% view on this. > > That said, the new code seems much cleaner to me. And in general, having > NamespaceWatcher.equals(NamespaceWatcher) seems 100% legit to me. If we're > only ever passing NamespaceWatcher instances to the ZK layer to add and > remove, that seems great. > > The only part that bugs me is having NamespaceWatcher.equals(raw Watcher). > If we're relying on this behavior anywhere, it's a recipe for problems. If > we're NOT relying on this behavior, then we should rip some code out of > NamespaceWatcher and have it so that a NamespaceWatcher can only equals > another NamespaceWatcher. > > What do you think? > > > On Wed, Feb 10, 2016 at 8:48 AM, Jordan Zimmerman <[email protected] > <mailto:[email protected]>> wrote: > Scott - are you OK with a release or should I wait for more discussion on > this issue? > > -Jordan > >> On Feb 9, 2016, at 1:50 PM, Scott Blum <[email protected] >> <mailto:[email protected]>> wrote: >> >> Sounds like a job for weak hash map. Will follow up later with more >> >> On Feb 9, 2016 12:01 PM, "Jordan Zimmerman" <[email protected] >> <mailto:[email protected]>> wrote: >> > So.... taking a step back, what was underlying motivation for the hashCode >> > / equality changes? IE, what's the bigger problem we were trying to solve? >> >> Before this change, we were maintaining a map from Watcher to >> NamespaceWatcher so that we could track/remove the wrapped watcher. This is >> necessary due to this guarantee of ZooKeeper: >> >> http://zookeeper.apache.org/doc/trunk/zookeeperProgrammers.html#sc_WatchGuarantees >> >> <http://zookeeper.apache.org/doc/trunk/zookeeperProgrammers.html#sc_WatchGuarantees> >> >> "if the same watch object is registered for an exists and a getData call for >> the same file and that file is then deleted, the watch object would only be >> invoked once with the deletion notification for the file.” >> >> Given that NamespaceWatcher is an internal wrapper, Curator needs to >> generate the same NamespaceWatcher for a given client’s >> Watcher/CuratorWatcher. The map handled this. In the past, this was >> difficult to manage and had potential memory leaks if the map wasn’t managed >> correctly. It occurred to me that the map isn’t needed if NamespaceWatcher >> could have equality/hash values the same as the Watcher that it wraps. My >> testing proved this. >> >> Thoughts? >> >> -Jordan >> >> >> > On Feb 9, 2016, at 11:49 AM, Scott Blum <[email protected] >> > <mailto:[email protected]>> wrote: >> > >> > Hi guys, >> > >> > I'm a practical guy, not a purist, but the 3.0 implementations of >> > NamespaceWatcher.hashCode() and equals() are bothering me. The reason I >> > care is that I want to avoid subtle bugs cropping up. >> > >> > So here's the problem. >> > >> > 1) equals() is not reflexive between NamespaceWatcher and Watcher >> > >> > Assuming you have a NamespaceWatcher nw wrapping a Watcher w, the >> > following code might or might not work: >> > >> > container.add(nw) >> > container.remove(w) >> > >> > It depends on whether the underlying container ultimately does >> > "nw.equals(w)" or "w.equals(nw)". Set.contains() would have the same >> > problem. >> > >> > 2) hashCode() and equals() inconsistent with each other >> > >> > Because nw.hashCode() != w.hashCode(), lookups in a hashSet or hashMap >> > will practically never work except by luck. >> > >> > hashSet.put(nw) >> > hashSet.contains(w) >> > >> > Most of the time this will return false, except in the exact case where nw >> > and w happen to have hashCodes that map into the same bucket, and the >> > equality check is done the "right" order. >> > >> > >> > So.... taking a step back, what was underlying motivation for the hashCode >> > / equality changes? IE, what's the bigger problem we were trying to solve? >> > >> > Scott >> > >> > >
