You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@curator.apache.org by ra...@apache.org on 2017/05/02 03:57:03 UTC

[19/20] curator git commit: Major rework of caching. Having the wrapped caches adds little value. Focus on the integrated caching in the modeled client instance

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledPathChildrenCache.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledPathChildrenCache.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledPathChildrenCache.java
deleted file mode 100644
index 7b54fb7..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledPathChildrenCache.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.curator.x.async.modeled.recipes;
-
-import org.apache.curator.framework.listen.Listenable;
-import org.apache.curator.framework.recipes.cache.PathChildrenCache;
-import org.apache.curator.x.async.modeled.ModelSerializer;
-import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.details.recipes.ModeledPathChildrenCacheImpl;
-import java.io.Closeable;
-import java.util.List;
-import java.util.Optional;
-
-/**
- * Wraps a {@link org.apache.curator.framework.recipes.cache.PathChildrenCache} so that
- * node data can be viewed as strongly typed models.
- */
-public interface ModeledPathChildrenCache<T> extends ModeledCache<T>, Closeable
-{
-    /**
-     * Return a newly wrapped cache
-     *
-     * @param cache the cache to wrap
-     * @param serializer model serializer
-     * @return new wrapped cache
-     */
-    static <T> ModeledPathChildrenCache<T> wrap(PathChildrenCache cache, ModelSerializer<T> serializer)
-    {
-        return new ModeledPathChildrenCacheImpl<>(cache, serializer);
-    }
-
-    /**
-     * Return the original cache that was wrapped
-     *
-     * @return cache
-     */
-    PathChildrenCache unwrap();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#start()}
-     */
-    void start();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#start(org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode)}
-     */
-    void start(PathChildrenCache.StartMode mode);
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#rebuild()}
-     */
-    void rebuild();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#rebuildNode(String)}
-     */
-    void rebuildNode(ZPath fullPath);
-
-    /**
-     * Return the listener container so that you can add/remove listeners
-     *
-     * @return listener container
-     */
-    Listenable<ModeledCacheListener<T>> getListenable();
-
-    /**
-     * Return the modeled current data. There are no guarantees of accuracy. This is
-     * merely the most recent view of the data. The data is returned in sorted order.
-     *
-     * @return list of children and data
-     */
-    List<ModeledCachedNode<T>> getCurrentData();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#clearDataBytes(String)}
-     */
-    void clearDataBytes(ZPath fullPath);
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#clearDataBytes(String, int)}
-     */
-    boolean clearDataBytes(ZPath fullPath, int ifVersion);
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#clearAndRefresh()}
-     */
-    void clearAndRefresh();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#clear()}
-     */
-    void clear();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#rebuildNode(String)}
-     */
-    void close();
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledTreeCache.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledTreeCache.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledTreeCache.java
deleted file mode 100644
index 07eb191..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledTreeCache.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.curator.x.async.modeled.recipes;
-
-import org.apache.curator.framework.listen.Listenable;
-import org.apache.curator.framework.recipes.cache.TreeCache;
-import org.apache.curator.x.async.modeled.ModelSerializer;
-import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.details.recipes.ModeledTreeCacheImpl;
-import java.io.Closeable;
-import java.util.Map;
-
-/**
- * Wraps a {@link org.apache.curator.framework.recipes.cache.TreeCache} so that
- * node data can be viewed as strongly typed models.
- */
-public interface ModeledTreeCache<T> extends ModeledCache<T>, Closeable
-{
-    /**
-     * Return a newly wrapped cache
-     *
-     * @param cache the cache to wrap
-     * @param serializer model serializer
-     * @return new wrapped cache
-     */
-    static <T> ModeledTreeCache<T> wrap(TreeCache cache, ModelSerializer<T> serializer)
-    {
-        return new ModeledTreeCacheImpl<>(cache, serializer);
-    }
-
-    /**
-     * Return the original cache that was wrapped
-     *
-     * @return cache
-     */
-    TreeCache unwrap();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.TreeCache#start()}
-     */
-    void start();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.TreeCache#close()}
-     */
-    void close();
-
-    /**
-     * Return the listener container so that you can add/remove listeners
-     *
-     * @return listener container
-     */
-    Listenable<ModeledCacheListener<T>> getListenable();
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/TestCachedModeledCuratorFramework.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/TestCachedModeledCuratorFramework.java b/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/TestCachedModeledCuratorFramework.java
index 4a0bed1..e4f0158 100644
--- a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/TestCachedModeledCuratorFramework.java
+++ b/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/TestCachedModeledCuratorFramework.java
@@ -23,7 +23,7 @@ import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.x.async.CompletableBaseClassForTests;
-import org.apache.curator.x.async.modeled.CachedModeledCuratorFramework;
+import org.apache.curator.x.async.modeled.cached.CachedModeledCuratorFramework;
 import org.apache.curator.x.async.modeled.CuratorModelSpec;
 import org.apache.curator.x.async.modeled.JacksonModelSerializer;
 import org.apache.curator.x.async.modeled.ModelSerializer;
@@ -69,7 +69,7 @@ public class TestCachedModeledCuratorFramework extends CompletableBaseClassForTe
         client.start();
 
         AtomicInteger counter = new AtomicInteger();
-        ((CachedModeledCuratorFrameworkImpl)client).debugCachedReadCount = counter;
+//        ((CachedModeledCuratorFrameworkImpl)client).debugCachedReadCount = counter;
 
         complete(client.read());
         Assert.assertEquals(counter.get(), 0);

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/recipes/TestEventTypeMappings.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/recipes/TestEventTypeMappings.java b/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/recipes/TestEventTypeMappings.java
deleted file mode 100644
index 73c634f..0000000
--- a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/recipes/TestEventTypeMappings.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.curator.x.async.modeled.details.recipes;
-
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
-import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
-import org.testng.annotations.Test;
-
-public class TestEventTypeMappings
-{
-    @Test
-    public void testPathChildrenCacheTypes()
-    {
-        for ( PathChildrenCacheEvent.Type type : PathChildrenCacheEvent.Type.values() )
-        {
-            ModeledPathChildrenCacheImpl.toType(type);  // throws an exception on unknown types
-        }
-    }
-
-    @Test
-    public void testTreeCacheTypes()
-    {
-        for ( TreeCacheEvent.Type type : TreeCacheEvent.Type.values() )
-        {
-            ModeledTreeCacheImpl.toType(type);  // throws an exception on unknown types
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/recipes/TestModeledCaches.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/recipes/TestModeledCaches.java b/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/recipes/TestModeledCaches.java
deleted file mode 100644
index 0a32e9a..0000000
--- a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/recipes/TestModeledCaches.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.curator.x.async.modeled.recipes;
-
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.framework.recipes.cache.NodeCache;
-import org.apache.curator.framework.recipes.cache.PathChildrenCache;
-import org.apache.curator.framework.recipes.cache.TreeCache;
-import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.utils.CloseableUtils;
-import org.apache.curator.x.async.CompletableBaseClassForTests;
-import org.apache.curator.x.async.modeled.CuratorModelSpec;
-import org.apache.curator.x.async.modeled.JacksonModelSerializer;
-import org.apache.curator.x.async.modeled.ModeledCuratorFramework;
-import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.models.TestModel;
-import org.apache.zookeeper.data.Stat;
-import org.testng.Assert;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-import java.math.BigInteger;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-public class TestModeledCaches extends CompletableBaseClassForTests
-{
-    private CuratorFramework client;
-    private JacksonModelSerializer<TestModel> serializer;
-    private ZPath path;
-    private ModeledCuratorFramework<TestModel> modeled;
-
-    @BeforeMethod
-    @Override
-    public void setup() throws Exception
-    {
-        super.setup();
-
-        client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
-        client.start();
-
-        serializer = JacksonModelSerializer.build(TestModel.class);
-
-        path = ZPath.parse("/test/path");
-        modeled = ModeledCuratorFramework.wrap(client, CuratorModelSpec.builder(path, serializer).build());
-    }
-
-    @AfterMethod
-    @Override
-    public void teardown() throws Exception
-    {
-        CloseableUtils.closeQuietly(client);
-
-        super.teardown();
-    }
-
-    @Test
-    public void testModeledNodeCache() throws InterruptedException
-    {
-        try ( ModeledNodeCache<TestModel> cache = ModeledNodeCache.wrap(new NodeCache(client, path.fullPath()), serializer) )
-        {
-            cache.start(true);
-
-            BlockingQueue<ModeledCacheEvent<TestModel>> events = new LinkedBlockingQueue<>();
-            ModeledCacheListener<TestModel> listener = events::add;
-            cache.getListenable().addListener(listener);
-
-            TestModel model1 = new TestModel("a", "b", "c", 1, BigInteger.TEN);
-            TestModel model2 = new TestModel("d", "e", "f", 10, BigInteger.ONE);
-
-            Stat stat = new Stat();
-            modeled.set(model1, stat);
-            ModeledCacheEvent<TestModel> event = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event);
-            Assert.assertEquals(event.getType(), ModeledCacheEventType.NODE_UPDATED);
-            Assert.assertNotNull(event.getNode());
-            Assert.assertEquals(event.getNode().getPath(), path);
-            Assert.assertEquals(event.getNode().getModel(), model1);
-            Assert.assertEquals(event.getNode().getStat(), stat);
-
-            timing.sleepABit();
-            Assert.assertEquals(events.size(), 0);
-
-            modeled.update(model2);
-            event = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event.getNode());
-            Assert.assertEquals(event.getNode().getPath(), path);
-            Assert.assertEquals(event.getNode().getModel(), model2);
-
-            modeled.delete();
-            event = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNull(event.getNode());
-        }
-    }
-
-    @Test
-    public void testModeledPathChildrenCache() throws InterruptedException
-    {
-        try ( ModeledPathChildrenCache<TestModel> cache = ModeledPathChildrenCache.wrap(new PathChildrenCache(client, path.fullPath(), true), serializer) )
-        {
-            cache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE);
-
-            BlockingQueue<ModeledCacheEvent<TestModel>> events = new LinkedBlockingQueue<>();
-            ModeledCacheListener<TestModel> listener = events::add;
-            cache.getListenable().addListener(listener);
-
-            TestModel model1 = new TestModel("a", "b", "c", 1, BigInteger.TEN);
-            TestModel model2 = new TestModel("d", "e", "f", 10, BigInteger.ONE);
-            TestModel model3 = new TestModel("g", "h", "i", 100, BigInteger.ZERO);
-
-            modeled.at("1").set(model1).thenApply(__ -> modeled.at("2").set(model2));
-            ModeledCacheEvent<TestModel> event1 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            ModeledCacheEvent<TestModel> event2 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event1);
-            Assert.assertNotNull(event2);
-            Assert.assertEquals(event1.getType(), ModeledCacheEventType.NODE_ADDED);
-            Assert.assertEquals(event2.getType(), ModeledCacheEventType.NODE_ADDED);
-            Assert.assertEquals((event1.getNode() != null) ? event1.getNode().getModel() : null, model1);
-            Assert.assertEquals((event2.getNode() != null) ? event2.getNode().getModel() : null, model2);
-            Assert.assertEquals(event1.getNode().getPath(), path.at("1"));
-            Assert.assertEquals(event2.getNode().getPath(), path.at("2"));
-
-            modeled.at("1").delete();
-            event1 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event1);
-            Assert.assertEquals(event1.getType(), ModeledCacheEventType.NODE_REMOVED);
-            Assert.assertEquals(event1.getNode().getPath(), path.at("1"));
-
-            modeled.at("2").update(model3);
-            event1 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event1);
-            Assert.assertEquals(event1.getType(), ModeledCacheEventType.NODE_UPDATED);
-            Assert.assertEquals(event1.getNode().getPath(), path.at("2"));
-            Assert.assertEquals((event1.getNode() != null) ? event1.getNode().getModel() : null, model3);
-
-            cache.getListenable().removeListener(listener);
-            modeled.at("2").delete();
-            Assert.assertNull(events.poll(timing.forSleepingABit().milliseconds(), TimeUnit.MILLISECONDS));  // listener is removed - shouldn't get an event
-        }
-    }
-
-    @Test
-    public void testModeledTreeCache() throws Exception
-    {
-        try (ModeledTreeCache<TestModel> cache = ModeledTreeCache.wrap(TreeCache.newBuilder(client, path.fullPath()).build(), serializer) )
-        {
-            BlockingQueue<ModeledCacheEvent<TestModel>> events = new LinkedBlockingQueue<>();
-            ModeledCacheListener<TestModel> listener = ModeledCacheListener.filtered(events::add, ModeledCacheListener.<TestModel>nodeRemovedFilter().or(ModeledCacheListener.hasModelFilter()));
-            cache.getListenable().addListener(listener);
-
-            cache.start();
-
-            TestModel model1 = new TestModel("a", "b", "c", 1, BigInteger.TEN);
-            TestModel model2 = new TestModel("d", "e", "f", 10, BigInteger.ONE);
-            TestModel model3 = new TestModel("g", "h", "i", 100, BigInteger.ZERO);
-
-            modeled.at("1").set(model1).thenApply(__ -> modeled.at("1").at("2").set(model2).thenApply(___ -> modeled.at("1").at("2").at("3").set(model3)));
-            ModeledCacheEvent<TestModel> event1 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            ModeledCacheEvent<TestModel> event2 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            ModeledCacheEvent<TestModel> event3 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event1);
-            Assert.assertNotNull(event2);
-            Assert.assertNotNull(event3);
-            Assert.assertEquals(event1.getType(), ModeledCacheEventType.NODE_ADDED);
-            Assert.assertEquals(event2.getType(), ModeledCacheEventType.NODE_ADDED);
-            Assert.assertEquals(event3.getType(), ModeledCacheEventType.NODE_ADDED);
-            Assert.assertEquals((event1.getNode() != null) ? event1.getNode().getModel() : null, model1);
-            Assert.assertEquals((event2.getNode() != null) ? event2.getNode().getModel() : null, model2);
-            Assert.assertEquals((event3.getNode() != null) ? event3.getNode().getModel() : null, model3);
-            Assert.assertEquals(event1.getNode().getPath(), path.at("1"));
-            Assert.assertEquals(event2.getNode().getPath(), path.at("1").at("2"));
-            Assert.assertEquals(event3.getNode().getPath(), path.at("1").at("2").at("3"));
-        }
-    }
-}