From 50c1f7ace2843216fb4e11dce59c4bee78b477a8 Mon Sep 17 00:00:00 2001
From: asdf2014
Date: Mon, 13 Jan 2020 00:33:18 +0800
Subject: [PATCH 01/43] * Fix huge number of watches in zk
* Tear down nodeAnnouncer
* Remove useless Logger and ExecutorService
* Init CuratorListener by lambda
* Improve explicit type
* Using CuratorMultiTransaction instead of CuratorTransaction
* Add @GuardedBy("toAnnounce") for toUpdate field
* Improve docs
---
.../worker/WorkerCuratorCoordinator.java | 7 +-
.../druid/curator/announcement/Announcer.java | 2 +
.../curator/announcement/NodeAnnouncer.java | 350 ++++++++++++++++++
.../discovery/CuratorDruidNodeAnnouncer.java | 6 +-
.../apache/druid/guice/AnnouncerModule.java | 8 +
.../LookupResourceListenerAnnouncer.java | 4 +-
.../org/apache/druid/server/ZKPathsUtils.java | 30 ++
.../CuratorDataSegmentServerAnnouncer.java | 6 +-
.../announcer/ListenerResourceAnnouncer.java | 8 +-
.../client/BatchServerInventoryViewTest.java | 8 +-
.../announcement/NodeAnnouncerTest.java | 285 ++++++++++++++
...torDruidNodeAnnouncerAndDiscoveryTest.java | 8 +-
.../ListenerResourceAnnouncerTest.java | 23 +-
13 files changed, 702 insertions(+), 43 deletions(-)
create mode 100644 server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
create mode 100644 server/src/main/java/org/apache/druid/server/ZKPathsUtils.java
create mode 100644 server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
index 7e7c09893b9d..7ed8aea24f0b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
@@ -25,11 +25,10 @@
import com.google.inject.Inject;
import org.apache.curator.framework.CuratorFramework;
import org.apache.druid.curator.CuratorUtils;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.common.logger.Logger;
@@ -54,7 +53,7 @@ public class WorkerCuratorCoordinator
private final ObjectMapper jsonMapper;
private final RemoteTaskRunnerConfig config;
private final CuratorFramework curatorFramework;
- private final Announcer announcer;
+ private final NodeAnnouncer announcer;
private final String baseAnnouncementsPath;
private final String baseTaskPath;
@@ -77,7 +76,7 @@ public WorkerCuratorCoordinator(
this.curatorFramework = curatorFramework;
this.worker = worker;
- this.announcer = new Announcer(curatorFramework, Execs.directExecutor());
+ this.announcer = new NodeAnnouncer(curatorFramework);
this.baseAnnouncementsPath = getPath(Arrays.asList(indexerZkConfig.getAnnouncementsPath(), worker.getHost()));
this.baseTaskPath = getPath(Arrays.asList(indexerZkConfig.getTasksPath(), worker.getHost()));
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index 533389a02c3e..89694d67c557 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -40,6 +40,7 @@
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
+import javax.annotation.concurrent.GuardedBy;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
@@ -64,6 +65,7 @@ public class Announcer
private final ExecutorService pathChildrenCacheExecutor;
private final List toAnnounce = new ArrayList<>();
+ @GuardedBy("toAnnounce")
private final List toUpdate = new ArrayList<>();
private final ConcurrentMap listeners = new ConcurrentHashMap<>();
private final ConcurrentMap> announcements = new ConcurrentHashMap<>();
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
new file mode 100644
index 000000000000..a68b426e5402
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -0,0 +1,350 @@
+/*
+ * 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.druid.curator.announcement;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.transaction.CuratorOp;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.CloseQuietly;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.ZKPathsUtils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+/**
+ * NodeAnnouncer announces single node on Zookeeper and only watches this node,
+ * while {@link Announcer} watches all child paths, not only this node
+ */
+public class NodeAnnouncer
+{
+ private static final Logger log = new Logger(NodeAnnouncer.class);
+
+ private final CuratorFramework curator;
+
+ /**
+ * In case a path is added to this collection in {@link #announce} before zk is connected,
+ * should remember the path and do announce in {@link #start} later.
+ */
+ private final List toAnnounce = new ArrayList<>();
+ /**
+ * In case a path is added to this collection in {@link #update} before zk is connected,
+ * should remember the path and do update in {@link #start} later.
+ */
+ @GuardedBy("toAnnounce")
+ private final List toUpdate = new ArrayList<>();
+ private final ConcurrentMap listeners = new ConcurrentHashMap<>();
+ private final ConcurrentMap announcedPaths = new ConcurrentHashMap<>();
+ /**
+ * Only the one created the parent path can drop the parent path, so should remember these created parents.
+ */
+ private final List pathsCreatedInThisAnnouncer = new CopyOnWriteArrayList<>();
+
+ private boolean started = false;
+
+ public NodeAnnouncer(CuratorFramework curator)
+ {
+ this.curator = curator;
+ }
+
+ @VisibleForTesting
+ Set getAddedPaths()
+ {
+ return announcedPaths.keySet();
+ }
+
+ @LifecycleStart
+ @SuppressWarnings("DuplicatedCode")
+ public void start()
+ {
+ log.info("Starting announcer");
+ synchronized (toAnnounce) {
+ if (started) {
+ return;
+ }
+
+ started = true;
+
+ for (Announceable announceable : toAnnounce) {
+ announce(announceable.path, announceable.bytes, announceable.removeParentsIfCreated);
+ }
+ toAnnounce.clear();
+
+ for (Announceable announceable : toUpdate) {
+ update(announceable.path, announceable.bytes);
+ }
+ toUpdate.clear();
+ }
+ }
+
+ @LifecycleStop
+ public void stop()
+ {
+ log.info("Stopping announcer");
+ synchronized (toAnnounce) {
+ if (!started) {
+ return;
+ }
+
+ started = false;
+
+ Closer closer = Closer.create();
+ for (NodeCache cache : listeners.values()) {
+ closer.register(cache);
+ }
+ CloseQuietly.close(closer);
+
+ for (String announcementPath : announcedPaths.keySet()) {
+ unannounce(announcementPath);
+ }
+
+ if (!pathsCreatedInThisAnnouncer.isEmpty()) {
+ final List deleteOps = new ArrayList<>(pathsCreatedInThisAnnouncer.size());
+ for (String parent : pathsCreatedInThisAnnouncer) {
+ try {
+ deleteOps.add(curator.transactionOp().delete().forPath(parent));
+ }
+ catch (Exception e) {
+ log.error(e, "Unable to delete parent[%s].", parent);
+ }
+ }
+ try {
+ curator.transaction().forOperations(deleteOps);
+ }
+ catch (Exception e) {
+ log.error(e, "Unable to commit transaction.");
+ }
+ }
+ }
+ }
+
+ /**
+ * Like announce(path, bytes, true).
+ */
+ public void announce(String path, byte[] bytes)
+ {
+ announce(path, bytes, true);
+ }
+
+ /**
+ * Announces the provided bytes at the given path. Announcement means that it will create an ephemeral node
+ * and monitor it to make sure that it always exists until it is unannounced or this object is closed.
+ *
+ * @param path The path to announce at
+ * @param bytes The payload to announce
+ * @param removeParentIfCreated remove parent of "path" if we had created that parent
+ */
+ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
+ {
+ synchronized (toAnnounce) {
+ if (!started) {
+ toAnnounce.add(new Announceable(path, bytes, removeParentIfCreated));
+ return;
+ }
+ }
+
+ final String parentPath = ZKPathsUtils.getParentPath(path);
+ boolean buildParentPath = false;
+
+ byte[] value = announcedPaths.get(path);
+
+ if (value == null) {
+ try {
+ if (curator.checkExists().forPath(parentPath) == null) {
+ buildParentPath = true;
+ }
+ }
+ catch (Exception e) {
+ log.debug(e, "Problem checking if the parent existed, ignoring.");
+ }
+
+ // Synchronize to make sure that I only create a listener once.
+ synchronized (toAnnounce) {
+ if (!listeners.containsKey(path)) {
+ final NodeCache cache = new NodeCache(curator, path, true);
+ cache.getListenable().addListener(
+ () -> {
+ ChildData currentData = cache.getCurrentData();
+ if (currentData == null) {
+ final byte[] value1 = announcedPaths.get(path);
+ if (value1 != null) {
+ log.info("Node[%s] dropped, reinstating.", path);
+ createAnnouncement(path, value1);
+ }
+ }
+ }
+ );
+
+ if (started) {
+ if (buildParentPath) {
+ createPath(parentPath, removeParentIfCreated);
+ }
+ startCache(cache);
+ listeners.put(path, cache);
+ }
+ }
+ }
+ }
+
+ boolean created = false;
+ synchronized (toAnnounce) {
+ if (started) {
+ byte[] oldBytes = announcedPaths.putIfAbsent(path, bytes);
+
+ if (oldBytes == null) {
+ created = true;
+ } else if (!Arrays.equals(oldBytes, bytes)) {
+ throw new IAE("Cannot reannounce different values under the same path");
+ }
+ }
+ }
+
+ if (created) {
+ try {
+ createAnnouncement(path, bytes);
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ public void update(final String path, final byte[] bytes)
+ {
+ synchronized (toAnnounce) {
+ if (!started) {
+ // removeParentsIfCreated is not relevant for updates; use dummy value "false".
+ toUpdate.add(new Announceable(path, bytes, false));
+ return;
+ }
+ }
+
+ byte[] oldBytes = announcedPaths.get(path);
+
+ if (oldBytes == null) {
+ throw new ISE("Cannot update a path[%s] that hasn't been announced!", path);
+ }
+
+ synchronized (toAnnounce) {
+ try {
+ if (!Arrays.equals(oldBytes, bytes)) {
+ announcedPaths.put(path, bytes);
+ updateAnnouncement(path, bytes);
+ }
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ private void createAnnouncement(final String path, byte[] value) throws Exception
+ {
+ curator.create().compressed().withMode(CreateMode.EPHEMERAL).inBackground().forPath(path, value);
+ }
+
+ private void updateAnnouncement(final String path, final byte[] value) throws Exception
+ {
+ curator.setData().compressed().inBackground().forPath(path, value);
+ }
+
+ /**
+ * Unannounces an announcement created at path. Note that if all announcements get removed, the Announcer
+ * will continue to have ZK watches on paths because clearing them out is a source of ugly race conditions.
+ *
+ * If you need to completely clear all the state of what is being watched and announced, stop() the Announcer.
+ *
+ * @param path the path to unannounce
+ */
+ public void unannounce(String path)
+ {
+ log.info("unannouncing [%s]", path);
+ final byte[] value = announcedPaths.remove(path);
+
+ if (value == null) {
+ log.error("Path[%s] not announced, cannot unannounce.", path);
+ return;
+ }
+
+ try {
+ curator.transaction().forOperations(curator.transactionOp().delete().forPath(path));
+ }
+ catch (KeeperException.NoNodeException e) {
+ log.info("node[%s] didn't exist anyway...", path);
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void startCache(NodeCache cache)
+ {
+ try {
+ cache.start();
+ }
+ catch (Exception e) {
+ CloseQuietly.close(cache);
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void createPath(String parentPath, boolean removeParentsIfCreated)
+ {
+ try {
+ curator.create().creatingParentsIfNeeded().forPath(parentPath);
+ if (removeParentsIfCreated) {
+ pathsCreatedInThisAnnouncer.add(parentPath);
+ }
+ log.debug("Created parentPath[%s], %s remove on stop() called.", parentPath, removeParentsIfCreated ? "will" : "will not");
+ }
+ catch (Exception e) {
+ log.error(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
+ }
+ }
+
+ private static class Announceable
+ {
+ final String path;
+ final byte[] bytes;
+ final boolean removeParentsIfCreated;
+
+ public Announceable(String path, byte[] bytes, boolean removeParentsIfCreated)
+ {
+ this.path = path;
+ this.bytes = bytes;
+ this.removeParentsIfCreated = removeParentsIfCreated;
+ }
+ }
+}
diff --git a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java
index 5b536c65d39d..0dc4d85ec8ea 100644
--- a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java
@@ -23,7 +23,7 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import org.apache.curator.utils.ZKPaths;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.discovery.DiscoveryDruidNode;
import org.apache.druid.discovery.DruidNodeAnnouncer;
import org.apache.druid.discovery.NodeRole;
@@ -42,12 +42,12 @@ static String makeNodeAnnouncementPath(ZkPathsConfig config, NodeRole nodeRole,
private static final Logger log = new Logger(CuratorDruidNodeAnnouncer.class);
- private final Announcer announcer;
+ private final NodeAnnouncer announcer;
private final ZkPathsConfig config;
private final ObjectMapper jsonMapper;
@Inject
- public CuratorDruidNodeAnnouncer(Announcer announcer, ZkPathsConfig config, @Json ObjectMapper jsonMapper)
+ public CuratorDruidNodeAnnouncer(NodeAnnouncer announcer, ZkPathsConfig config, @Json ObjectMapper jsonMapper)
{
this.announcer = announcer;
this.config = config;
diff --git a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
index 3a48183bdc1a..13a2372b86c9 100644
--- a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
+++ b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
@@ -24,6 +24,7 @@
import com.google.inject.Provides;
import org.apache.curator.framework.CuratorFramework;
import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer;
import org.apache.druid.server.coordination.CuratorDataSegmentServerAnnouncer;
@@ -52,4 +53,11 @@ public Announcer getAnnouncer(CuratorFramework curator)
{
return new Announcer(curator, Execs.singleThreaded("Announcer-%s"));
}
+
+ @Provides
+ @ManageLifecycle
+ public NodeAnnouncer getNodeAnnouncer(CuratorFramework curator)
+ {
+ return new NodeAnnouncer(curator);
+ }
}
diff --git a/server/src/main/java/org/apache/druid/query/lookup/LookupResourceListenerAnnouncer.java b/server/src/main/java/org/apache/druid/query/lookup/LookupResourceListenerAnnouncer.java
index d58e2b69d7bc..07a8c4d80250 100644
--- a/server/src/main/java/org/apache/druid/query/lookup/LookupResourceListenerAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/query/lookup/LookupResourceListenerAnnouncer.java
@@ -20,7 +20,7 @@
package org.apache.druid.query.lookup;
import com.google.inject.Inject;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.guice.annotations.Self;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.http.HostAndPortWithScheme;
@@ -31,7 +31,7 @@ class LookupResourceListenerAnnouncer extends ListenerResourceAnnouncer
{
@Inject
public LookupResourceListenerAnnouncer(
- Announcer announcer,
+ NodeAnnouncer announcer,
LookupListeningAnnouncerConfig lookupListeningAnnouncerConfig,
@Self DruidNode node
)
diff --git a/server/src/main/java/org/apache/druid/server/ZKPathsUtils.java b/server/src/main/java/org/apache/druid/server/ZKPathsUtils.java
new file mode 100644
index 000000000000..9832abde27bf
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/server/ZKPathsUtils.java
@@ -0,0 +1,30 @@
+/*
+ * 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.druid.server;
+
+import org.apache.curator.utils.ZKPaths;
+
+public class ZKPathsUtils
+{
+ public static String getParentPath(String path)
+ {
+ return ZKPaths.getPathAndNode(path).getPath();
+ }
+}
diff --git a/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java
index 1f456bd3445d..a337c78b948d 100644
--- a/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java
@@ -23,7 +23,7 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import org.apache.curator.utils.ZKPaths;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.initialization.ZkPathsConfig;
@@ -37,7 +37,7 @@ public class CuratorDataSegmentServerAnnouncer implements DataSegmentServerAnnou
private final DruidServerMetadata server;
private final ZkPathsConfig config;
- private final Announcer announcer;
+ private final NodeAnnouncer announcer;
private final ObjectMapper jsonMapper;
private final Object lock = new Object();
@@ -48,7 +48,7 @@ public class CuratorDataSegmentServerAnnouncer implements DataSegmentServerAnnou
public CuratorDataSegmentServerAnnouncer(
DruidServerMetadata server,
ZkPathsConfig config,
- Announcer announcer,
+ NodeAnnouncer announcer,
ObjectMapper jsonMapper
)
{
diff --git a/server/src/main/java/org/apache/druid/server/listener/announcer/ListenerResourceAnnouncer.java b/server/src/main/java/org/apache/druid/server/listener/announcer/ListenerResourceAnnouncer.java
index 04d51df0b41d..f0d84d81fff7 100644
--- a/server/src/main/java/org/apache/druid/server/listener/announcer/ListenerResourceAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/server/listener/announcer/ListenerResourceAnnouncer.java
@@ -20,7 +20,7 @@
package org.apache.druid.server.listener.announcer;
import org.apache.curator.utils.ZKPaths;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.common.logger.Logger;
@@ -41,11 +41,11 @@ public abstract class ListenerResourceAnnouncer
private static final Logger LOG = new Logger(ListenerResourceAnnouncer.class);
private final Object startStopSync = new Object();
private volatile boolean started = false;
- private final Announcer announcer;
+ private final NodeAnnouncer announcer;
private final String announcePath;
public ListenerResourceAnnouncer(
- Announcer announcer,
+ NodeAnnouncer announcer,
ListeningAnnouncerConfig listeningAnnouncerConfig,
String listener_key,
HostAndPortWithScheme node
@@ -59,7 +59,7 @@ public ListenerResourceAnnouncer(
}
ListenerResourceAnnouncer(
- Announcer announcer,
+ NodeAnnouncer announcer,
String announceBasePath,
HostAndPortWithScheme node
)
diff --git a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java
index 9ba446659633..76d959e5d2f7 100644
--- a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java
+++ b/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java
@@ -39,6 +39,7 @@
import org.apache.druid.client.ServerView;
import org.apache.druid.curator.PotentiallyGzippedCompressionProvider;
import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
@@ -90,6 +91,7 @@ public class BatchServerInventoryViewTest
private CuratorFramework cf;
private ObjectMapper jsonMapper;
private Announcer announcer;
+ private NodeAnnouncer nodeAnnouncer;
private BatchDataSegmentAnnouncer segmentAnnouncer;
private DataSegmentServerAnnouncer serverAnnouncer;
private Set testSegments;
@@ -123,6 +125,9 @@ public void setUp() throws Exception
);
announcer.start();
+ nodeAnnouncer = new NodeAnnouncer(cf);
+ nodeAnnouncer.start();
+
DruidServerMetadata serverMetadata = new DruidServerMetadata(
"id",
"host",
@@ -145,7 +150,7 @@ public String getBase()
serverAnnouncer = new CuratorDataSegmentServerAnnouncer(
serverMetadata,
zkPathsConfig,
- announcer,
+ nodeAnnouncer,
jsonMapper
);
serverAnnouncer.announce();
@@ -225,6 +230,7 @@ public void tearDown() throws Exception
filteredBatchServerInventoryView.stop();
serverAnnouncer.unannounce();
announcer.stop();
+ nodeAnnouncer.stop();
cf.close();
testingCluster.stop();
}
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
new file mode 100644
index 000000000000..220c7e40e160
--- /dev/null
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -0,0 +1,285 @@
+/*
+ * 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.druid.curator.announcement;
+
+import com.google.common.collect.Sets;
+import org.apache.curator.framework.api.CuratorEventType;
+import org.apache.curator.framework.api.transaction.CuratorOp;
+import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
+import org.apache.curator.test.KillSession;
+import org.apache.curator.utils.ZKPaths;
+import org.apache.druid.curator.CuratorTestBase;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.zookeeper.KeeperException.Code;
+import org.apache.zookeeper.data.Stat;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+
+/**
+ */
+public class NodeAnnouncerTest extends CuratorTestBase
+{
+
+ @Before
+ public void setUp() throws Exception
+ {
+ setupServerAndCurator();
+ }
+
+ @After
+ public void tearDown()
+ {
+ tearDownServerAndCurator();
+ }
+
+ @Test(timeout = 60_000L)
+ public void testSanity() throws Exception
+ {
+ curator.start();
+ curator.blockUntilConnected();
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
+
+ final byte[] billy = StringUtils.toUtf8("billy");
+ final String testPath1 = "/test1";
+ final String testPath2 = "/somewhere/test2";
+ announcer.announce(testPath1, billy);
+
+ Assert.assertNull("/test1 does not exists", curator.checkExists().forPath(testPath1));
+ Assert.assertNull("/somewhere/test2 does not exists", curator.checkExists().forPath(testPath2));
+
+ announcer.start();
+ while (!announcer.getAddedPaths().contains("/test1")) {
+ Thread.sleep(100);
+ }
+
+ try {
+ Assert.assertArrayEquals("/test1 has data", billy, curator.getData().decompressed().forPath(testPath1));
+ Assert.assertNull("/somewhere/test2 still does not exist", curator.checkExists().forPath(testPath2));
+
+ announcer.announce(testPath2, billy);
+
+ Assert.assertArrayEquals("/test1 still has data", billy, curator.getData().decompressed().forPath(testPath1));
+ Assert.assertArrayEquals(
+ "/somewhere/test2 has data",
+ billy,
+ curator.getData().decompressed().forPath(testPath2)
+ );
+
+ final CountDownLatch latch = new CountDownLatch(1);
+ curator.getCuratorListenable().addListener(
+ (client, event) -> {
+ if (event.getType() == CuratorEventType.CREATE && event.getPath().equals(testPath1)) {
+ latch.countDown();
+ }
+ }
+ );
+ final CuratorOp deleteOp = curator.transactionOp().delete().forPath(testPath1);
+ final Collection results = curator.transaction().forOperations(deleteOp);
+ Assert.assertEquals(1, results.size());
+ final CuratorTransactionResult result = results.iterator().next();
+ Assert.assertEquals(Code.OK.intValue(), result.getError()); // assert delete
+
+ Assert.assertTrue("Wait for /test1 to be created", timing.forWaiting().awaitLatch(latch));
+
+ Assert.assertArrayEquals(
+ "expect /test1 data is restored",
+ billy,
+ curator.getData().decompressed().forPath(testPath1)
+ );
+ Assert.assertArrayEquals(
+ "expect /somewhere/test2 is still there",
+ billy,
+ curator.getData().decompressed().forPath(testPath2)
+ );
+
+ announcer.unannounce(testPath1);
+ Assert.assertNull("expect /test1 unannounced", curator.checkExists().forPath(testPath1));
+ Assert.assertArrayEquals(
+ "expect /somewhere/test2 is still still there",
+ billy,
+ curator.getData().decompressed().forPath(testPath2)
+ );
+ }
+ finally {
+ announcer.stop();
+ }
+
+ Assert.assertNull("expect /test1 remains unannounced", curator.checkExists().forPath(testPath1));
+ Assert.assertNull("expect /somewhere/test2 unannounced", curator.checkExists().forPath(testPath2));
+ }
+
+ @Test(timeout = 60_000L)
+ public void testSessionKilled() throws Exception
+ {
+ curator.start();
+ curator.blockUntilConnected();
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ try {
+ curator.inTransaction().create().forPath("/somewhere").and().commit();
+ announcer.start();
+
+ final byte[] billy = StringUtils.toUtf8("billy");
+ final String testPath1 = "/test1";
+ final String testPath2 = "/somewhere/test2";
+ final Set paths = Sets.newHashSet(testPath1, testPath2);
+ announcer.announce(testPath1, billy);
+ announcer.announce(testPath2, billy);
+
+ Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
+ Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
+
+ final CountDownLatch latch = new CountDownLatch(1);
+ curator.getCuratorListenable().addListener(
+ (client, event) -> {
+ if (event.getType() == CuratorEventType.CREATE) {
+ paths.remove(event.getPath());
+ if (paths.isEmpty()) {
+ latch.countDown();
+ }
+ }
+ }
+ );
+ KillSession.kill(curator.getZookeeperClient().getZooKeeper(), server.getConnectString());
+
+ Assert.assertTrue(timing.forWaiting().awaitLatch(latch));
+
+ Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
+ Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
+
+ announcer.stop();
+
+ while ((curator.checkExists().forPath(testPath1) != null) || (curator.checkExists().forPath(testPath2) != null)) {
+ Thread.sleep(100);
+ }
+
+ Assert.assertNull(curator.checkExists().forPath(testPath1));
+ Assert.assertNull(curator.checkExists().forPath(testPath2));
+ }
+ finally {
+ announcer.stop();
+ }
+ }
+
+ @Test
+ public void testCleansUpItsLittleTurdlings() throws Exception
+ {
+ curator.start();
+ curator.blockUntilConnected();
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
+
+ final byte[] billy = StringUtils.toUtf8("billy");
+ final String testPath = "/somewhere/test2";
+ final String parent = ZKPaths.getPathAndNode(testPath).getPath();
+
+ announcer.start();
+ try {
+ Assert.assertNull(curator.checkExists().forPath(parent));
+
+ awaitAnnounce(announcer, testPath, billy, true);
+
+ Assert.assertNotNull(curator.checkExists().forPath(parent));
+ }
+ finally {
+ announcer.stop();
+ }
+
+ Assert.assertNull(curator.checkExists().forPath(parent));
+ }
+
+ @Test
+ public void testLeavesBehindTurdlingsThatAlreadyExisted() throws Exception
+ {
+ curator.start();
+ curator.blockUntilConnected();
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
+
+ final byte[] billy = StringUtils.toUtf8("billy");
+ final String testPath = "/somewhere/test2";
+ final String parent = ZKPaths.getPathAndNode(testPath).getPath();
+
+ curator.create().forPath(parent);
+ final Stat initialStat = curator.checkExists().forPath(parent);
+
+ announcer.start();
+ try {
+ Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
+
+ awaitAnnounce(announcer, testPath, billy, true);
+
+ Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
+ }
+ finally {
+ announcer.stop();
+ }
+
+ Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
+ }
+
+ @Test
+ public void testLeavesBehindTurdlingsWhenToldTo() throws Exception
+ {
+ curator.start();
+ curator.blockUntilConnected();
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
+
+ final byte[] billy = StringUtils.toUtf8("billy");
+ final String testPath = "/somewhere/test2";
+ final String parent = ZKPaths.getPathAndNode(testPath).getPath();
+
+ announcer.start();
+ try {
+ Assert.assertNull(curator.checkExists().forPath(parent));
+
+ awaitAnnounce(announcer, testPath, billy, false);
+
+ Assert.assertNotNull(curator.checkExists().forPath(parent));
+ }
+ finally {
+ announcer.stop();
+ }
+
+ Assert.assertNotNull(curator.checkExists().forPath(parent));
+ }
+
+ private void awaitAnnounce(
+ final NodeAnnouncer announcer,
+ final String path,
+ final byte[] bytes,
+ boolean removeParentsIfCreated
+ ) throws InterruptedException
+ {
+ final CountDownLatch latch = new CountDownLatch(1);
+ curator.getCuratorListenable().addListener(
+ (client, event) -> {
+ if (event.getType() == CuratorEventType.CREATE && event.getPath().equals(path)) {
+ latch.countDown();
+ }
+ }
+ );
+ announcer.announce(path, bytes, removeParentsIfCreated);
+ latch.await();
+ }
+}
diff --git a/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java
index 1851cc749614..12a87fc5e3d3 100644
--- a/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java
+++ b/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java
@@ -24,12 +24,11 @@
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.curator.CuratorTestBase;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.discovery.DiscoveryDruidNode;
import org.apache.druid.discovery.DruidNodeDiscovery;
import org.apache.druid.discovery.NodeRole;
import org.apache.druid.jackson.DefaultObjectMapper;
-import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.initialization.ServerConfig;
import org.apache.druid.server.initialization.ZkPathsConfig;
@@ -69,10 +68,7 @@ public void testAnnouncementAndDiscovery() throws Exception
curator.start();
curator.blockUntilConnected();
- Announcer announcer = new Announcer(
- curator,
- Execs.directExecutor()
- );
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
announcer.start();
CuratorDruidNodeAnnouncer druidNodeAnnouncer = new CuratorDruidNodeAnnouncer(
diff --git a/server/src/test/java/org/apache/druid/server/listener/announcer/ListenerResourceAnnouncerTest.java b/server/src/test/java/org/apache/druid/server/listener/announcer/ListenerResourceAnnouncerTest.java
index 96e2a18bde7b..5d86e2e7e1c9 100644
--- a/server/src/test/java/org/apache/druid/server/listener/announcer/ListenerResourceAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/server/listener/announcer/ListenerResourceAnnouncerTest.java
@@ -21,21 +21,17 @@
import org.apache.curator.utils.ZKPaths;
import org.apache.druid.curator.CuratorTestBase;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.segment.CloserRule;
import org.apache.druid.server.http.HostAndPortWithScheme;
import org.apache.druid.server.initialization.ZkPathsConfig;
import org.easymock.EasyMock;
-import org.junit.After;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import java.io.Closeable;
-import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
public class ListenerResourceAnnouncerTest extends CuratorTestBase
@@ -45,19 +41,6 @@ public class ListenerResourceAnnouncerTest extends CuratorTestBase
private final String announcePath = listeningAnnouncerConfig.getAnnouncementPath(listenerKey);
@Rule
public CloserRule closerRule = new CloserRule(true);
- private ExecutorService executorService;
-
- @Before
- public void setUp()
- {
- executorService = Execs.singleThreaded("listener-resource--%d");
- }
-
- @After
- public void tearDown()
- {
- executorService.shutdownNow();
- }
@Test
public void testAnnouncerBehaves() throws Exception
@@ -68,7 +51,7 @@ public void testAnnouncerBehaves() throws Exception
closerRule.closeLater(curator);
Assert.assertNotNull(curator.create().forPath("/druid"));
Assert.assertTrue(curator.blockUntilConnected(10, TimeUnit.SECONDS));
- final Announcer announcer = new Announcer(curator, executorService);
+ final NodeAnnouncer announcer = new NodeAnnouncer(curator);
final HostAndPortWithScheme node = HostAndPortWithScheme.fromString("localhost");
final ListenerResourceAnnouncer listenerResourceAnnouncer = new ListenerResourceAnnouncer(
announcer,
@@ -109,7 +92,7 @@ public void close()
@Test
public void testStartCorrect()
{
- final Announcer announcer = EasyMock.createStrictMock(Announcer.class);
+ final NodeAnnouncer announcer = EasyMock.createStrictMock(NodeAnnouncer.class);
final HostAndPortWithScheme node = HostAndPortWithScheme.fromString("some_host");
final ListenerResourceAnnouncer resourceAnnouncer = new ListenerResourceAnnouncer(
From 8420d73df63250d0a1dcd7d5b1acb72365d601dd Mon Sep 17 00:00:00 2001
From: asdf2014
Date: Mon, 20 Jan 2020 00:02:34 +0800
Subject: [PATCH 02/43] Patch comments
---
.../curator/announcement/Announceable.java | 15 ++++++++++
.../druid/curator/announcement/Announcer.java | 21 +++-----------
.../curator/announcement/NodeAnnouncer.java | 29 ++++++-------------
.../org/apache/druid/server/ZKPathsUtils.java | 5 ++++
.../curator/announcement/AnnouncerTest.java | 8 ++---
.../announcement/NodeAnnouncerTest.java | 8 ++---
6 files changed, 41 insertions(+), 45 deletions(-)
create mode 100644 server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java b/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
new file mode 100644
index 000000000000..2520cb3012bc
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
@@ -0,0 +1,15 @@
+package org.apache.druid.curator.announcement;
+
+class Announceable
+{
+ final String path;
+ final byte[] bytes;
+ final boolean removeParentsIfCreated;
+
+ public Announceable(String path, byte[] bytes, boolean removeParentsIfCreated)
+ {
+ this.path = path;
+ this.bytes = bytes;
+ this.removeParentsIfCreated = removeParentsIfCreated;
+ }
+}
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index 89694d67c557..a58e35430997 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -36,6 +36,7 @@
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.ZKPathsUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
@@ -54,7 +55,8 @@
import java.util.concurrent.atomic.AtomicReference;
/**
- * Announces things on Zookeeper.
+ * {@link NodeAnnouncer} announces single node on Zookeeper and only watches this node,
+ * while {@link Announcer} watches all child paths, not only this node.
*/
public class Announcer
{
@@ -242,8 +244,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th
switch (event.getType()) {
case CHILD_REMOVED:
final ChildData child = event.getData();
- final ZKPaths.PathAndNode childPath = ZKPaths.getPathAndNode(child.getPath());
- final byte[] value = finalSubPaths.get(childPath.getNode());
+ final byte[] value = finalSubPaths.get(ZKPathsUtils.getParentNode(child.getPath()));
if (value != null) {
log.info("Node[%s] dropped, reinstating.", child.getPath());
createAnnouncement(child.getPath(), value);
@@ -434,18 +435,4 @@ private void createPath(String parentPath, boolean removeParentsIfCreated)
log.info(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
}
}
-
- private static class Announceable
- {
- final String path;
- final byte[] bytes;
- final boolean removeParentsIfCreated;
-
- public Announceable(String path, byte[] bytes, boolean removeParentsIfCreated)
- {
- this.path = path;
- this.bytes = bytes;
- this.removeParentsIfCreated = removeParentsIfCreated;
- }
- }
}
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index a68b426e5402..3f05fce84dd3 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -42,11 +42,10 @@
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArrayList;
/**
- * NodeAnnouncer announces single node on Zookeeper and only watches this node,
- * while {@link Announcer} watches all child paths, not only this node
+ * {@link NodeAnnouncer} announces single node on Zookeeper and only watches this node,
+ * while {@link Announcer} watches all child paths, not only this node.
*/
public class NodeAnnouncer
{
@@ -68,10 +67,13 @@ public class NodeAnnouncer
private final ConcurrentMap listeners = new ConcurrentHashMap<>();
private final ConcurrentMap announcedPaths = new ConcurrentHashMap<>();
/**
- * Only the one created the parent path can drop the parent path, so should remember these created parents.
+ * Only the one created the parent path can drop it, so should remember these created parents.
+ * This comment sounds confusing, shouldn't one of "parent path" occurrences in it be something else?
*/
- private final List pathsCreatedInThisAnnouncer = new CopyOnWriteArrayList<>();
+ @GuardedBy("toAnnounce")
+ private final List pathsCreatedInThisAnnouncer = new ArrayList<>();
+ @GuardedBy("toAnnounce")
private boolean started = false;
public NodeAnnouncer(CuratorFramework curator)
@@ -187,7 +189,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
}
}
catch (Exception e) {
- log.debug(e, "Problem checking if the parent existed, ignoring.");
+ log.debug(e, "Problem checking if the parent path doesn't exist, ignoring.");
}
// Synchronize to make sure that I only create a listener once.
@@ -320,6 +322,7 @@ private void startCache(NodeCache cache)
}
}
+ @GuardedBy("toAnnounce")
private void createPath(String parentPath, boolean removeParentsIfCreated)
{
try {
@@ -333,18 +336,4 @@ private void createPath(String parentPath, boolean removeParentsIfCreated)
log.error(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
}
}
-
- private static class Announceable
- {
- final String path;
- final byte[] bytes;
- final boolean removeParentsIfCreated;
-
- public Announceable(String path, byte[] bytes, boolean removeParentsIfCreated)
- {
- this.path = path;
- this.bytes = bytes;
- this.removeParentsIfCreated = removeParentsIfCreated;
- }
- }
}
diff --git a/server/src/main/java/org/apache/druid/server/ZKPathsUtils.java b/server/src/main/java/org/apache/druid/server/ZKPathsUtils.java
index 9832abde27bf..59a9c8c2aeee 100644
--- a/server/src/main/java/org/apache/druid/server/ZKPathsUtils.java
+++ b/server/src/main/java/org/apache/druid/server/ZKPathsUtils.java
@@ -27,4 +27,9 @@ public static String getParentPath(String path)
{
return ZKPaths.getPathAndNode(path).getPath();
}
+
+ public static String getParentNode(String path)
+ {
+ return ZKPaths.getPathAndNode(path).getNode();
+ }
}
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
index e12d8bc47bb7..3f0c7e545e42 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
@@ -27,11 +27,11 @@
import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
import org.apache.curator.test.KillSession;
-import org.apache.curator.utils.ZKPaths;
import org.apache.druid.curator.CuratorTestBase;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.ZKPathsUtils;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.data.Stat;
import org.junit.After;
@@ -212,7 +212,7 @@ public void testCleansUpItsLittleTurdlings() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
- final String parent = ZKPaths.getPathAndNode(testPath).getPath();
+ final String parent = ZKPathsUtils.getParentPath(testPath);
announcer.start();
try {
@@ -238,7 +238,7 @@ public void testLeavesBehindTurdlingsThatAlreadyExisted() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
- final String parent = ZKPaths.getPathAndNode(testPath).getPath();
+ final String parent = ZKPathsUtils.getParentPath(testPath);
curator.create().forPath(parent);
final Stat initialStat = curator.checkExists().forPath(parent);
@@ -267,7 +267,7 @@ public void testLeavesBehindTurdlingsWhenToldTo() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
- final String parent = ZKPaths.getPathAndNode(testPath).getPath();
+ final String parent = ZKPathsUtils.getParentPath(testPath);
announcer.start();
try {
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
index 220c7e40e160..42fe943349d6 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -24,9 +24,9 @@
import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
import org.apache.curator.test.KillSession;
-import org.apache.curator.utils.ZKPaths;
import org.apache.druid.curator.CuratorTestBase;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.server.ZKPathsUtils;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.data.Stat;
import org.junit.After;
@@ -192,7 +192,7 @@ public void testCleansUpItsLittleTurdlings() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
- final String parent = ZKPaths.getPathAndNode(testPath).getPath();
+ final String parent = ZKPathsUtils.getParentPath(testPath);
announcer.start();
try {
@@ -218,7 +218,7 @@ public void testLeavesBehindTurdlingsThatAlreadyExisted() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
- final String parent = ZKPaths.getPathAndNode(testPath).getPath();
+ final String parent = ZKPathsUtils.getParentPath(testPath);
curator.create().forPath(parent);
final Stat initialStat = curator.checkExists().forPath(parent);
@@ -247,7 +247,7 @@ public void testLeavesBehindTurdlingsWhenToldTo() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
- final String parent = ZKPaths.getPathAndNode(testPath).getPath();
+ final String parent = ZKPathsUtils.getParentPath(testPath);
announcer.start();
try {
From da8538b709cc8b617a1923d74b5758896d3a3e65 Mon Sep 17 00:00:00 2001
From: asdf2014
Date: Mon, 20 Jan 2020 00:10:06 +0800
Subject: [PATCH 03/43] Add license
---
.../curator/announcement/Announceable.java | 19 +++++++++++++++++++
1 file changed, 19 insertions(+)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java b/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
index 2520cb3012bc..59f26d2f902e 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
@@ -1,3 +1,22 @@
+/*
+ * 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.druid.curator.announcement;
class Announceable
From 67b5064d31a072605c32c6fe0abc1d06326a68d9 Mon Sep 17 00:00:00 2001
From: asdf2014
Date: Thu, 23 Jan 2020 21:27:45 +0800
Subject: [PATCH 04/43] Patch comments
---
.../apache/druid/curator/announcement/Announcer.java | 7 ++++---
.../druid/curator/announcement/NodeAnnouncer.java | 12 ++++++------
2 files changed, 10 insertions(+), 9 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index a58e35430997..e47916814cd4 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -55,7 +55,7 @@
import java.util.concurrent.atomic.AtomicReference;
/**
- * {@link NodeAnnouncer} announces single node on Zookeeper and only watches this node,
+ * {@link NodeAnnouncer} announces a single node on Zookeeper and only watches this node,
* while {@link Announcer} watches all child paths, not only this node.
*/
public class Announcer
@@ -66,12 +66,13 @@ public class Announcer
private final PathChildrenCacheFactory factory;
private final ExecutorService pathChildrenCacheExecutor;
+ @GuardedBy("toAnnounce")
private final List toAnnounce = new ArrayList<>();
@GuardedBy("toAnnounce")
private final List toUpdate = new ArrayList<>();
private final ConcurrentMap listeners = new ConcurrentHashMap<>();
private final ConcurrentMap> announcements = new ConcurrentHashMap<>();
- private final List parentsIBuilt = new CopyOnWriteArrayList();
+ private final List parentsIBuilt = new CopyOnWriteArrayList<>();
// Used for testing
private Set addedChildren;
@@ -233,7 +234,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
cache.getListenable().addListener(
new PathChildrenCacheListener()
{
- private final AtomicReference> pathsLost = new AtomicReference>(null);
+ private final AtomicReference> pathsLost = new AtomicReference<>(null);
@Override
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 3f05fce84dd3..0648d2a7d48d 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -44,7 +44,7 @@
import java.util.concurrent.ConcurrentMap;
/**
- * {@link NodeAnnouncer} announces single node on Zookeeper and only watches this node,
+ * {@link NodeAnnouncer} announces a single node on Zookeeper and only watches this node,
* while {@link Announcer} watches all child paths, not only this node.
*/
public class NodeAnnouncer
@@ -57,6 +57,7 @@ public class NodeAnnouncer
* In case a path is added to this collection in {@link #announce} before zk is connected,
* should remember the path and do announce in {@link #start} later.
*/
+ @GuardedBy("toAnnounce")
private final List toAnnounce = new ArrayList<>();
/**
* In case a path is added to this collection in {@link #update} before zk is connected,
@@ -67,8 +68,8 @@ public class NodeAnnouncer
private final ConcurrentMap listeners = new ConcurrentHashMap<>();
private final ConcurrentMap announcedPaths = new ConcurrentHashMap<>();
/**
- * Only the one created the parent path can drop it, so should remember these created parents.
- * This comment sounds confusing, shouldn't one of "parent path" occurrences in it be something else?
+ * This list is to remember all paths this node announcer has created.
+ * On {@link #stop}, the node announcer is responsible for deleting all paths in this list.
*/
@GuardedBy("toAnnounce")
private final List pathsCreatedInThisAnnouncer = new ArrayList<>();
@@ -126,11 +127,10 @@ public void stop()
for (NodeCache cache : listeners.values()) {
closer.register(cache);
}
- CloseQuietly.close(closer);
-
for (String announcementPath : announcedPaths.keySet()) {
- unannounce(announcementPath);
+ closer.register(() -> unannounce(announcementPath));
}
+ CloseQuietly.close(closer);
if (!pathsCreatedInThisAnnouncer.isEmpty()) {
final List deleteOps = new ArrayList<>(pathsCreatedInThisAnnouncer.size());
From da2f21283acd88e768ed89fad85d2a11a0da8b81 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 11 Nov 2024 15:46:33 +0800
Subject: [PATCH 05/43] Update to use CloseableUtils instead of CloseQuietly.
---
.../apache/druid/curator/announcement/NodeAnnouncer.java | 7 +++----
1 file changed, 3 insertions(+), 4 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 0648d2a7d48d..3ba68f28b8db 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -26,12 +26,12 @@
import org.apache.curator.framework.recipes.cache.NodeCache;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.guava.CloseQuietly;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.ZKPathsUtils;
+import org.apache.druid.utils.CloseableUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -130,7 +130,7 @@ public void stop()
for (String announcementPath : announcedPaths.keySet()) {
closer.register(() -> unannounce(announcementPath));
}
- CloseQuietly.close(closer);
+ CloseableUtils.closeAndWrapExceptions(closer);
if (!pathsCreatedInThisAnnouncer.isEmpty()) {
final List deleteOps = new ArrayList<>(pathsCreatedInThisAnnouncer.size());
@@ -317,8 +317,7 @@ private void startCache(NodeCache cache)
cache.start();
}
catch (Exception e) {
- CloseQuietly.close(cache);
- throw new RuntimeException(e);
+ throw CloseableUtils.closeInCatch(new RuntimeException(e), cache);
}
}
From a5d0669ab7205f158e10600d9b9ae8d8a57e991b Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 11 Nov 2024 18:11:10 +0800
Subject: [PATCH 06/43] Add Javadocs for Announceable class
---
.../druid/curator/announcement/Announceable.java | 15 +++++++++++++++
1 file changed, 15 insertions(+)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java b/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
index 59f26d2f902e..c2ad7c588163 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
@@ -19,10 +19,25 @@
package org.apache.druid.curator.announcement;
+/**
+ * The {@link Announceable} is a representation of an announcement to be made in ZooKeeper.
+ */
class Announceable
{
+ /**
+ * Represents the path in ZooKeeper where the announcement will be made.
+ */
final String path;
+
+ /**
+ * Holds the actual data to be announced.
+ */
final byte[] bytes;
+
+ /**
+ * Indicates whether parent nodes should be removed if the announcement is created successfully.
+ * This can be useful for cleaning up unused paths in ZooKeeper.
+ */
final boolean removeParentsIfCreated;
public Announceable(String path, byte[] bytes, boolean removeParentsIfCreated)
From f3b21a4ae68c006aea87f6964e5cee315adaf81d Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 11 Nov 2024 18:16:13 +0800
Subject: [PATCH 07/43] Javadocs on Announcers
---
.../druid/curator/announcement/Announcer.java | 9 +-
.../curator/announcement/NodeAnnouncer.java | 88 +++++++++++++------
2 files changed, 69 insertions(+), 28 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index a25d89f38a3a..828ee3a6c793 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -54,8 +54,13 @@
import java.util.concurrent.atomic.AtomicReference;
/**
- * {@link NodeAnnouncer} announces a single node on Zookeeper and only watches this node,
- * while {@link Announcer} watches all child paths, not only this node.
+ * The {@link Announcer} class manages the announcement of multiple child nodes
+ * under a specified parent path in a ZooKeeper ensemble. It monitors these nodes
+ * to ensure their existence and manage their lifecycle collectively.
+ *
+ * Utilize this class when you need to handle complex node structures,
+ * including relationships between multiple child nodes. Should your use case
+ * involve the management of a standalone node instead, see {@link NodeAnnouncer}.
*/
public class Announcer
{
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 3ba68f28b8db..b7eaeca371a9 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -44,39 +44,54 @@
import java.util.concurrent.ConcurrentMap;
/**
- * {@link NodeAnnouncer} announces a single node on Zookeeper and only watches this node,
- * while {@link Announcer} watches all child paths, not only this node.
+ * The {@link NodeAnnouncer} class is responsible for announcing a single node
+ * in a ZooKeeper ensemble. It creates an ephemeral node at a specified path
+ * and monitors its existence to ensure that it remains active until it is
+ * explicitly unannounced or the object is closed.
+ *
+ * This class provides methods to announce and update the content of the
+ * node as well as handle path creation if required.
+ *
+ * Use this class when you need to manage the lifecycle of a standalone
+ * node without concerns about its children or siblings. Should your use case
+ * involve the management of child nodes under a specific parent path in a
+ * ZooKeeper ensemble, see {@link Announcer}.
*/
public class NodeAnnouncer
{
private static final Logger log = new Logger(NodeAnnouncer.class);
private final CuratorFramework curator;
+ private final ConcurrentMap listeners = new ConcurrentHashMap<>();
+ private final ConcurrentMap announcedPaths = new ConcurrentHashMap<>();
+
+ @GuardedBy("toAnnounce")
+ private boolean started = false;
/**
- * In case a path is added to this collection in {@link #announce} before zk is connected,
- * should remember the path and do announce in {@link #start} later.
+ * This list holds paths that need to be announced. If a path is added to this list
+ * in the {@link #announce} method before the connection to ZooKeeper is established,
+ * it will be stored here and announced later during the {@link #start} method.
*/
@GuardedBy("toAnnounce")
private final List toAnnounce = new ArrayList<>();
+
/**
- * In case a path is added to this collection in {@link #update} before zk is connected,
- * should remember the path and do update in {@link #start} later.
+ * This list holds paths that need to be updated. If a path is added to this list
+ * in the {@link #update} method before the connection to ZooKeeper is established,
+ * it will be stored here and updated later during the {@link #start} method.
*/
@GuardedBy("toAnnounce")
private final List toUpdate = new ArrayList<>();
- private final ConcurrentMap listeners = new ConcurrentHashMap<>();
- private final ConcurrentMap announcedPaths = new ConcurrentHashMap<>();
+
/**
- * This list is to remember all paths this node announcer has created.
- * On {@link #stop}, the node announcer is responsible for deleting all paths in this list.
+ * This list keeps track of all the paths created by this node announcer.
+ * When the {@link #stop} method is called,
+ * the node announcer is responsible for deleting all paths stored in this list.
*/
@GuardedBy("toAnnounce")
private final List pathsCreatedInThisAnnouncer = new ArrayList<>();
- @GuardedBy("toAnnounce")
- private boolean started = false;
-
public NodeAnnouncer(CuratorFramework curator)
{
this.curator = curator;
@@ -89,12 +104,12 @@ Set getAddedPaths()
}
@LifecycleStart
- @SuppressWarnings("DuplicatedCode")
public void start()
{
- log.info("Starting announcer");
+ log.info("Starting NodeAnnouncer");
synchronized (toAnnounce) {
if (started) {
+ log.debug("Called start() but NodeAnnouncer have already started.");
return;
}
@@ -115,23 +130,37 @@ public void start()
@LifecycleStop
public void stop()
{
- log.info("Stopping announcer");
+ log.info("Stopping NodeAnnouncer");
synchronized (toAnnounce) {
if (!started) {
+ log.debug("Called stop() but NodeAnnouncer have not started.");
return;
}
started = false;
+ closeResources();
+ deletePaths();
+ }
+ }
- Closer closer = Closer.create();
- for (NodeCache cache : listeners.values()) {
- closer.register(cache);
- }
- for (String announcementPath : announcedPaths.keySet()) {
- closer.register(() -> unannounce(announcementPath));
- }
- CloseableUtils.closeAndWrapExceptions(closer);
+ private void closeResources()
+ {
+ Closer closer = Closer.create();
+ for (NodeCache cache : listeners.values()) {
+ closer.register(cache);
+ }
+ for (String announcementPath : announcedPaths.keySet()) {
+ closer.register(() -> unannounce(announcementPath));
+ }
+ CloseableUtils.closeAndWrapExceptions(closer);
+ }
+ private void deletePaths()
+ {
+ // deletePaths method is only used in stop(), which already has synchronized(toAnnounce),
+ // this line is here just to prevent the static analysis from throwing
+ // "Access to field 'pathsCreatedInThisAnnouncer' outside declared guards".
+ synchronized (toAnnounce) {
if (!pathsCreatedInThisAnnouncer.isEmpty()) {
final List deleteOps = new ArrayList<>(pathsCreatedInThisAnnouncer.size());
for (String parent : pathsCreatedInThisAnnouncer) {
@@ -142,6 +171,7 @@ public void stop()
log.error(e, "Unable to delete parent[%s].", parent);
}
}
+
try {
curator.transaction().forOperations(deleteOps);
}
@@ -152,8 +182,9 @@ public void stop()
}
}
+
/**
- * Like announce(path, bytes, true).
+ * Overload of {@link #announce(String, byte[],boolean)}, but removes parent node of path after announcement.
*/
public void announce(String path, byte[] bytes)
{
@@ -172,6 +203,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
{
synchronized (toAnnounce) {
if (!started) {
+ log.debug("NodeAnnouncer has not started yet, queuing announcement for later processing...");
toAnnounce.add(new Announceable(path, bytes, removeParentIfCreated));
return;
}
@@ -329,7 +361,11 @@ private void createPath(String parentPath, boolean removeParentsIfCreated)
if (removeParentsIfCreated) {
pathsCreatedInThisAnnouncer.add(parentPath);
}
- log.debug("Created parentPath[%s], %s remove on stop() called.", parentPath, removeParentsIfCreated ? "will" : "will not");
+ log.debug(
+ "Created parentPath[%s], %s remove on stop() called.",
+ parentPath,
+ removeParentsIfCreated ? "will" : "will not"
+ );
}
catch (Exception e) {
log.error(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
From 10530420b6763739f44b3c8c2c9e0d2fcb3f385e Mon Sep 17 00:00:00 2001
From: GWphua
Date: Tue, 12 Nov 2024 11:07:29 +0800
Subject: [PATCH 08/43] Refactor announce method
---
.../curator/announcement/NodeAnnouncer.java | 182 ++++++++++--------
1 file changed, 102 insertions(+), 80 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index b7eaeca371a9..72df62fd828a 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -70,7 +70,7 @@ public class NodeAnnouncer
/**
* This list holds paths that need to be announced. If a path is added to this list
- * in the {@link #announce} method before the connection to ZooKeeper is established,
+ * in the {@link #announce(String, byte[], boolean)} method before the connection to ZooKeeper is established,
* it will be stored here and announced later during the {@link #start} method.
*/
@GuardedBy("toAnnounce")
@@ -139,10 +139,11 @@ public void stop()
started = false;
closeResources();
- deletePaths();
+ dropPathsCreatedInThisAnnouncer();
}
}
+ @GuardedBy("toAnnounce")
private void closeResources()
{
Closer closer = Closer.create();
@@ -155,36 +156,32 @@ private void closeResources()
CloseableUtils.closeAndWrapExceptions(closer);
}
- private void deletePaths()
+ @GuardedBy("toAnnounce")
+ private void dropPathsCreatedInThisAnnouncer()
{
- // deletePaths method is only used in stop(), which already has synchronized(toAnnounce),
- // this line is here just to prevent the static analysis from throwing
- // "Access to field 'pathsCreatedInThisAnnouncer' outside declared guards".
- synchronized (toAnnounce) {
- if (!pathsCreatedInThisAnnouncer.isEmpty()) {
- final List deleteOps = new ArrayList<>(pathsCreatedInThisAnnouncer.size());
- for (String parent : pathsCreatedInThisAnnouncer) {
- try {
- deleteOps.add(curator.transactionOp().delete().forPath(parent));
- }
- catch (Exception e) {
- log.error(e, "Unable to delete parent[%s].", parent);
- }
- }
-
+ if (!pathsCreatedInThisAnnouncer.isEmpty()) {
+ final List deleteOps = new ArrayList<>(pathsCreatedInThisAnnouncer.size());
+ for (String parent : pathsCreatedInThisAnnouncer) {
try {
- curator.transaction().forOperations(deleteOps);
+ deleteOps.add(curator.transactionOp().delete().forPath(parent));
}
catch (Exception e) {
- log.error(e, "Unable to commit transaction.");
+ log.error(e, "Unable to delete parent[%s].", parent);
}
}
+
+ try {
+ curator.transaction().forOperations(deleteOps);
+ }
+ catch (Exception e) {
+ log.error(e, "Unable to commit transaction.");
+ }
}
}
/**
- * Overload of {@link #announce(String, byte[],boolean)}, but removes parent node of path after announcement.
+ * Overload of {@link #announce(String, byte[], boolean)}, but removes parent node of path after announcement.
*/
public void announce(String path, byte[] bytes)
{
@@ -202,6 +199,8 @@ public void announce(String path, byte[] bytes)
public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
{
synchronized (toAnnounce) {
+ // In the case that this method is called by other components or thread that assumes the NodeAnnouncer
+ // is ready when NodeAnnouncer has not started, we will queue the announcement request.
if (!started) {
log.debug("NodeAnnouncer has not started yet, queuing announcement for later processing...");
toAnnounce.add(new Announceable(path, bytes, removeParentIfCreated));
@@ -210,39 +209,19 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
}
final String parentPath = ZKPathsUtils.getParentPath(path);
- boolean buildParentPath = false;
-
byte[] value = announcedPaths.get(path);
+ // We have yet to announce this path. Check if we need to build a parent path.
if (value == null) {
- try {
- if (curator.checkExists().forPath(parentPath) == null) {
- buildParentPath = true;
- }
- }
- catch (Exception e) {
- log.debug(e, "Problem checking if the parent path doesn't exist, ignoring.");
- }
+ boolean shouldBuildParentPath = canBuildParentPath(parentPath);
// Synchronize to make sure that I only create a listener once.
synchronized (toAnnounce) {
if (!listeners.containsKey(path)) {
- final NodeCache cache = new NodeCache(curator, path, true);
- cache.getListenable().addListener(
- () -> {
- ChildData currentData = cache.getCurrentData();
- if (currentData == null) {
- final byte[] value1 = announcedPaths.get(path);
- if (value1 != null) {
- log.info("Node[%s] dropped, reinstating.", path);
- createAnnouncement(path, value1);
- }
- }
- }
- );
+ final NodeCache cache = setupNodeCache(path);
if (started) {
- if (buildParentPath) {
+ if (shouldBuildParentPath) {
createPath(parentPath, removeParentIfCreated);
}
startCache(cache);
@@ -252,6 +231,57 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
}
}
+ final boolean readyToCreateAnnouncement = updateAnnouncedPaths(path, bytes);
+
+ if (readyToCreateAnnouncement) {
+ try {
+ createAnnouncement(path, bytes);
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ private boolean canBuildParentPath(String parentPath)
+ {
+ try {
+ return (curator.checkExists().forPath(parentPath) == null);
+ }
+ catch (Exception e) {
+ log.debug(e, "Failed to check existence of parent path. Proceeding without creating parent path.");
+ return false;
+ }
+ }
+
+ @GuardedBy("toAnnounce")
+ private NodeCache setupNodeCache(String path)
+ {
+ final NodeCache cache = new NodeCache(curator, path, true);
+ cache.getListenable().addListener(
+ () -> {
+ ChildData currentData = cache.getCurrentData();
+
+ if (currentData == null) {
+ // If currentData is null, and we record having announced the data,
+ // this means that the ephemeral node was unexpectedly removed.
+ // We will recreate the node again using the previous data.
+ final byte[] previouslyAnnouncedData = announcedPaths.get(path);
+ if (previouslyAnnouncedData != null) {
+ log.info(
+ "Ephemeral node at path [%s] was unexpectedly removed. Recreating node with previous data.",
+ path
+ );
+ createAnnouncement(path, previouslyAnnouncedData);
+ }
+ }
+ }
+ );
+ return cache;
+ }
+
+ private boolean updateAnnouncedPaths(String path, byte[] bytes)
+ {
boolean created = false;
synchronized (toAnnounce) {
if (started) {
@@ -264,14 +294,35 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
}
}
}
+ return created;
+ }
- if (created) {
- try {
- createAnnouncement(path, bytes);
- }
- catch (Exception e) {
- throw new RuntimeException(e);
+ @GuardedBy("toAnnounce")
+ private void createPath(String parentPath, boolean removeParentsIfCreated)
+ {
+ try {
+ curator.create().creatingParentsIfNeeded().forPath(parentPath);
+ if (removeParentsIfCreated) {
+ pathsCreatedInThisAnnouncer.add(parentPath);
}
+ log.debug(
+ "Created parentPath[%s], %s remove on stop() called.",
+ parentPath,
+ removeParentsIfCreated ? "will" : "will not"
+ );
+ }
+ catch (Exception e) {
+ log.error(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
+ }
+ }
+
+ private void startCache(NodeCache cache)
+ {
+ try {
+ cache.start();
+ }
+ catch (Exception e) {
+ throw CloseableUtils.closeInCatch(new RuntimeException(e), cache);
}
}
@@ -342,33 +393,4 @@ public void unannounce(String path)
throw new RuntimeException(e);
}
}
-
- private void startCache(NodeCache cache)
- {
- try {
- cache.start();
- }
- catch (Exception e) {
- throw CloseableUtils.closeInCatch(new RuntimeException(e), cache);
- }
- }
-
- @GuardedBy("toAnnounce")
- private void createPath(String parentPath, boolean removeParentsIfCreated)
- {
- try {
- curator.create().creatingParentsIfNeeded().forPath(parentPath);
- if (removeParentsIfCreated) {
- pathsCreatedInThisAnnouncer.add(parentPath);
- }
- log.debug(
- "Created parentPath[%s], %s remove on stop() called.",
- parentPath,
- removeParentsIfCreated ? "will" : "will not"
- );
- }
- catch (Exception e) {
- log.error(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
- }
- }
}
From 903db88262c1d270038672c960bc56d24d8a1d28 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Tue, 12 Nov 2024 15:47:06 +0800
Subject: [PATCH 09/43] Refactor NodeAnnouncer
---
.../curator/announcement/NodeAnnouncer.java | 64 +++++++++++--------
1 file changed, 39 insertions(+), 25 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 72df62fd828a..99cf82c29910 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -63,7 +63,7 @@ public class NodeAnnouncer
private final CuratorFramework curator;
private final ConcurrentMap listeners = new ConcurrentHashMap<>();
- private final ConcurrentMap announcedPaths = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap announcedPaths = new ConcurrentHashMap<>();
@GuardedBy("toAnnounce")
private boolean started = false;
@@ -282,19 +282,23 @@ private NodeCache setupNodeCache(String path)
private boolean updateAnnouncedPaths(String path, byte[] bytes)
{
- boolean created = false;
synchronized (toAnnounce) {
- if (started) {
- byte[] oldBytes = announcedPaths.putIfAbsent(path, bytes);
-
- if (oldBytes == null) {
- created = true;
- } else if (!Arrays.equals(oldBytes, bytes)) {
- throw new IAE("Cannot reannounce different values under the same path");
- }
+ if (!started) {
+ return false; // Do nothing if not started
}
}
- return created;
+
+ final byte[] updatedAnnouncementData = announcedPaths.compute(path, (key, oldBytes) -> {
+ if (oldBytes == null) {
+ return bytes; // Insert the new value
+ } else if (!Arrays.equals(oldBytes, bytes)) {
+ throw new IAE("Cannot reannounce different values under the same path");
+ }
+ return oldBytes; // No change if values are equal
+ });
+
+ // Return true if we have updated the paths.
+ return Arrays.equals(updatedAnnouncementData, bytes);
}
@GuardedBy("toAnnounce")
@@ -306,13 +310,16 @@ private void createPath(String parentPath, boolean removeParentsIfCreated)
pathsCreatedInThisAnnouncer.add(parentPath);
}
log.debug(
- "Created parentPath[%s], %s remove on stop() called.",
+ "Created parentPath[%s], %s remove when stop() is called.",
parentPath,
removeParentsIfCreated ? "will" : "will not"
);
}
+ catch (KeeperException.NodeExistsException e) {
+ log.error(e, "The parentPath[%s] already exists.", parentPath);
+ }
catch (Exception e) {
- log.error(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
+ log.error(e, "Failed to create parentPath[%s].", parentPath);
}
}
@@ -342,17 +349,22 @@ public void update(final String path, final byte[] bytes)
throw new ISE("Cannot update a path[%s] that hasn't been announced!", path);
}
+ boolean canUpdate = false;
synchronized (toAnnounce) {
- try {
- if (!Arrays.equals(oldBytes, bytes)) {
- announcedPaths.put(path, bytes);
- updateAnnouncement(path, bytes);
- }
+ if (!Arrays.equals(oldBytes, bytes)) {
+ announcedPaths.put(path, bytes);
+ canUpdate = true;
}
- catch (Exception e) {
- throw new RuntimeException(e);
+ }
+
+ try {
+ if (canUpdate) {
+ updateAnnouncement(path, bytes);
}
}
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
}
private void createAnnouncement(final String path, byte[] value) throws Exception
@@ -375,12 +387,14 @@ private void updateAnnouncement(final String path, final byte[] value) throws Ex
*/
public void unannounce(String path)
{
- log.info("unannouncing [%s]", path);
- final byte[] value = announcedPaths.remove(path);
+ synchronized (toAnnounce) {
+ log.info("unannouncing [%s]", path);
+ final byte[] value = announcedPaths.remove(path);
- if (value == null) {
- log.error("Path[%s] not announced, cannot unannounce.", path);
- return;
+ if (value == null) {
+ log.error("Path[%s] not announced, cannot unannounce.", path);
+ return;
+ }
}
try {
From 954fa534d607d81a08f9ed2d197e3420d695ab17 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Tue, 12 Nov 2024 18:38:31 +0800
Subject: [PATCH 10/43] Add Unit Tests
---
.../curator/announcement/NodeAnnouncer.java | 1 -
.../announcement/NodeAnnouncerTest.java | 89 +++++++++++++++++++
2 files changed, 89 insertions(+), 1 deletion(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 99cf82c29910..4c17993631ba 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -179,7 +179,6 @@ private void dropPathsCreatedInThisAnnouncer()
}
}
-
/**
* Overload of {@link #announce(String, byte[], boolean)}, but removes parent node of path after announcement.
*/
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
index 42fe943349d6..c50bdef41dfd 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -25,6 +25,7 @@
import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
import org.apache.curator.test.KillSession;
import org.apache.druid.curator.CuratorTestBase;
+import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.server.ZKPathsUtils;
import org.apache.zookeeper.KeeperException.Code;
@@ -39,6 +40,7 @@
import java.util.concurrent.CountDownLatch;
/**
+ *
*/
public class NodeAnnouncerTest extends CuratorTestBase
{
@@ -55,6 +57,93 @@ public void tearDown()
tearDownServerAndCurator();
}
+ @Test
+ public void testAnnounceBeforeStartingNodeAnnouncer() throws Exception
+ {
+ curator.start();
+ curator.blockUntilConnected();
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ final byte[] billy = StringUtils.toUtf8("billy");
+ final String testPath = "/testAnnounce";
+
+ announcer.announce(testPath, billy);
+ announcer.start();
+
+ // Verify that the path was announced
+ Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+ announcer.stop();
+ }
+
+ @Test
+ public void testCreateParentPath() throws Exception
+ {
+ curator.start();
+ curator.blockUntilConnected();
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ final byte[] billy = StringUtils.toUtf8("billy");
+ final String testPath = "/newParent/testPath";
+ final String parentPath = ZKPathsUtils.getParentPath(testPath);
+
+ announcer.start();
+
+ Assert.assertNull("Parent path should not exist before announcement", curator.checkExists().forPath(parentPath));
+
+ // Announce with parent creation
+ announcer.announce(testPath, billy);
+
+ // Wait for the announcement to be processed
+ while (curator.checkExists().forPath(testPath) == null) {
+ Thread.sleep(100);
+ }
+
+ // Verify the parent path has been created
+ Assert.assertNotNull("Parent path should be created", curator.checkExists().forPath(parentPath));
+ Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+
+ announcer.stop();
+ }
+
+ @Test
+ public void testUpdateSuccessfully() throws Exception
+ {
+ curator.start();
+ curator.blockUntilConnected();
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ final byte[] billy = StringUtils.toUtf8("billy");
+ final byte[] tilly = StringUtils.toUtf8("tilly");
+ final String testPath = "/testUpdate";
+
+ announcer.start();
+ announcer.announce(testPath, billy);
+ Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+
+ announcer.update(testPath, billy);
+ Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+
+ announcer.update(testPath, tilly);
+ Assert.assertArrayEquals(tilly, curator.getData().decompressed().forPath(testPath));
+ announcer.stop();
+ }
+
+ @Test
+ public void testUpdateWithNonExistentPath() throws Exception
+ {
+ curator.start();
+ curator.blockUntilConnected();
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ final byte[] billy = StringUtils.toUtf8("billy");
+ final String testPath = "/testUpdate";
+
+ announcer.start();
+
+ Exception exception = Assert.assertThrows(ISE.class, () -> {
+ announcer.update(testPath, billy);
+ });
+ Assert.assertTrue(exception.getMessage().contains("Cannot update a path"));
+
+ announcer.stop();
+ }
+
@Test(timeout = 60_000L)
public void testSanity() throws Exception
{
From 25de238871d0345162ddbf9eaa5fd6dcb82818b8 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Wed, 13 Nov 2024 10:19:34 +0800
Subject: [PATCH 11/43] Remove humor from logs
---
.../druid/indexing/worker/WorkerCuratorCoordinator.java | 2 +-
.../org/apache/druid/curator/announcement/Announcer.java | 6 +++---
.../apache/druid/curator/announcement/NodeAnnouncer.java | 2 +-
.../druid/curator/announcement/NodeAnnouncerTest.java | 3 +--
4 files changed, 6 insertions(+), 7 deletions(-)
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
index 7ed8aea24f0b..6e09217e958e 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
@@ -86,7 +86,7 @@ public WorkerCuratorCoordinator(
@LifecycleStart
public void start() throws Exception
{
- log.info("WorkerCuratorCoordinator good to go sir. Server[%s]", worker.getHost());
+ log.info("WorkerCuratorCoordinator good to go. Server[%s]", worker.getHost());
synchronized (lock) {
if (started) {
return;
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index 828ee3a6c793..2c12a73488cd 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -169,14 +169,14 @@ public void stop()
transaction = transaction.delete().forPath(parent).and();
}
catch (Exception e) {
- log.info(e, "Unable to delete parent[%s], boooo.", parent);
+ log.info(e, "Unable to delete parent[%s].", parent);
}
}
try {
((CuratorTransactionFinal) transaction).commit();
}
catch (Exception e) {
- log.info(e, "Unable to commit transaction. Please feed the hamsters");
+ log.info(e, "Unable to commit transaction.");
}
}
}
@@ -356,7 +356,7 @@ public void update(final String path, final byte[] bytes)
ConcurrentMap subPaths = announcements.get(parentPath);
if (subPaths == null || subPaths.get(nodePath) == null) {
- throw new ISE("Cannot update a path[%s] that hasn't been announced!", path);
+ throw new ISE("Cannot update path[%s] that hasn't been announced!", path);
}
synchronized (toAnnounce) {
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 4c17993631ba..4eae8a31915b 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -345,7 +345,7 @@ public void update(final String path, final byte[] bytes)
byte[] oldBytes = announcedPaths.get(path);
if (oldBytes == null) {
- throw new ISE("Cannot update a path[%s] that hasn't been announced!", path);
+ throw new ISE("Cannot update path[%s] that hasn't been announced!", path);
}
boolean canUpdate = false;
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
index c50bdef41dfd..16beef3ccf67 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -139,8 +139,7 @@ public void testUpdateWithNonExistentPath() throws Exception
Exception exception = Assert.assertThrows(ISE.class, () -> {
announcer.update(testPath, billy);
});
- Assert.assertTrue(exception.getMessage().contains("Cannot update a path"));
-
+ Assert.assertEquals(exception.getMessage(), "Cannot update path[/testUpdate] that hasn't been announced!");
announcer.stop();
}
From a7733e3a73b556b8784fd23100ac03633adfe8cf Mon Sep 17 00:00:00 2001
From: GWphua
Date: Wed, 13 Nov 2024 15:28:48 +0800
Subject: [PATCH 12/43] Add unit test for NodeAnnouncer announcing same path
different payload.
---
.../curator/announcement/NodeAnnouncer.java | 8 +--
.../announcement/NodeAnnouncerTest.java | 58 ++++++++++++++++---
2 files changed, 54 insertions(+), 12 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 4eae8a31915b..e85822916f9a 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -208,10 +208,10 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
}
final String parentPath = ZKPathsUtils.getParentPath(path);
- byte[] value = announcedPaths.get(path);
+ byte[] announcedPayload = announcedPaths.get(path);
- // We have yet to announce this path. Check if we need to build a parent path.
- if (value == null) {
+ if (announcedPayload == null) {
+ // Payload does not exist. We have yet to announce this path. Check if we need to build a parent path.
boolean shouldBuildParentPath = canBuildParentPath(parentPath);
// Synchronize to make sure that I only create a listener once.
@@ -291,7 +291,7 @@ private boolean updateAnnouncedPaths(String path, byte[] bytes)
if (oldBytes == null) {
return bytes; // Insert the new value
} else if (!Arrays.equals(oldBytes, bytes)) {
- throw new IAE("Cannot reannounce different values under the same path");
+ throw new IAE("Cannot reannounce different values under the same path.");
}
return oldBytes; // No change if values are equal
});
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
index 16beef3ccf67..bcf0a2d14e33 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -25,6 +25,7 @@
import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
import org.apache.curator.test.KillSession;
import org.apache.druid.curator.CuratorTestBase;
+import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.server.ZKPathsUtils;
@@ -74,7 +75,7 @@ public void testAnnounceBeforeStartingNodeAnnouncer() throws Exception
announcer.stop();
}
- @Test
+ @Test(timeout = 60_000L)
public void testCreateParentPath() throws Exception
{
curator.start();
@@ -85,10 +86,7 @@ public void testCreateParentPath() throws Exception
final String parentPath = ZKPathsUtils.getParentPath(testPath);
announcer.start();
-
Assert.assertNull("Parent path should not exist before announcement", curator.checkExists().forPath(parentPath));
-
- // Announce with parent creation
announcer.announce(testPath, billy);
// Wait for the announcement to be processed
@@ -96,10 +94,56 @@ public void testCreateParentPath() throws Exception
Thread.sleep(100);
}
- // Verify the parent path has been created
Assert.assertNotNull("Parent path should be created", curator.checkExists().forPath(parentPath));
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+ announcer.stop();
+ }
+
+ @Test(timeout = 60_000L)
+ public void testAnnounceSamePathWithDifferentPayloadThrowsIAE() throws Exception
+ {
+ curator.start();
+ curator.blockUntilConnected();
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ final byte[] billy = StringUtils.toUtf8("billy");
+ final byte[] tilly = StringUtils.toUtf8("tilly");
+ final String testPath = "/testPath";
+
+ announcer.start();
+ announcer.announce(testPath, billy);
+ while (curator.checkExists().forPath(testPath) == null) {
+ Thread.sleep(100);
+ }
+ Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+ // Nothing wrong when we announce same path.
+ announcer.announce(testPath, billy);
+
+ // Something wrong when we announce different path.
+ Exception exception = Assert.assertThrows(IAE.class, () -> announcer.announce(testPath, tilly));
+ Assert.assertEquals(exception.getMessage(), "Cannot reannounce different values under the same path.");
+
+ // Confirm that the new announcement is invalidated, and we still have payload from previous announcement.
+ Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+ announcer.stop();
+ }
+
+ @Test
+ public void testUpdateBeforeStartingNodeAnnouncer() throws Exception
+ {
+ curator.start();
+ curator.blockUntilConnected();
+ NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ final byte[] billy = StringUtils.toUtf8("billy");
+ final byte[] tilly = StringUtils.toUtf8("tilly");
+ final String testPath = "/testAnnounce";
+
+ announcer.update(testPath, tilly);
+ announcer.announce(testPath, billy);
+ announcer.start();
+
+ // Verify that the path was announced
+ Assert.assertArrayEquals(tilly, curator.getData().decompressed().forPath(testPath));
announcer.stop();
}
@@ -136,9 +180,7 @@ public void testUpdateWithNonExistentPath() throws Exception
announcer.start();
- Exception exception = Assert.assertThrows(ISE.class, () -> {
- announcer.update(testPath, billy);
- });
+ Exception exception = Assert.assertThrows(ISE.class, () -> announcer.update(testPath, billy));
Assert.assertEquals(exception.getMessage(), "Cannot update path[/testUpdate] that hasn't been announced!");
announcer.stop();
}
From f541b84a44d7d2458bb349e859da926b05146a24 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Fri, 15 Nov 2024 16:20:20 +0800
Subject: [PATCH 13/43] Tweak import and change Javadocs
---
.../apache/druid/curator/announcement/Announcer.java | 12 ++++++------
.../druid/curator/announcement/NodeAnnouncer.java | 10 +++-------
2 files changed, 9 insertions(+), 13 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index 2c12a73488cd..ba87ec07e258 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -20,6 +20,7 @@
package org.apache.druid.curator.announcement;
import com.google.common.annotations.VisibleForTesting;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.transaction.CuratorTransaction;
import org.apache.curator.framework.api.transaction.CuratorTransactionFinal;
@@ -39,7 +40,6 @@
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
-import javax.annotation.concurrent.GuardedBy;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@@ -54,13 +54,13 @@
import java.util.concurrent.atomic.AtomicReference;
/**
- * The {@link Announcer} class manages the announcement of multiple child nodes
- * under a specified parent path in a ZooKeeper ensemble. It monitors these nodes
+ * The {@link Announcer} class manages the announcement of a node, and watches all child
+ * nodes under the specified path in a ZooKeeper ensemble. It monitors these nodes
* to ensure their existence and manage their lifecycle collectively.
*
- * Utilize this class when you need to handle complex node structures,
- * including relationships between multiple child nodes. Should your use case
- * involve the management of a standalone node instead, see {@link NodeAnnouncer}.
+ * Use this class when you need to manage the lifecycle of all child nodes under the
+ * specified path. Should your use case involve the management of a standalone node
+ * instead, see {@link NodeAnnouncer}.
*/
public class Announcer
{
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index e85822916f9a..35d7e27b0640 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -20,6 +20,7 @@
package org.apache.druid.curator.announcement;
import com.google.common.annotations.VisibleForTesting;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.recipes.cache.ChildData;
@@ -35,7 +36,6 @@
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
-import javax.annotation.concurrent.GuardedBy;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@@ -49,13 +49,9 @@
* and monitors its existence to ensure that it remains active until it is
* explicitly unannounced or the object is closed.
*
- * This class provides methods to announce and update the content of the
- * node as well as handle path creation if required.
- *
* Use this class when you need to manage the lifecycle of a standalone
- * node without concerns about its children or siblings. Should your use case
- * involve the management of child nodes under a specific parent path in a
- * ZooKeeper ensemble, see {@link Announcer}.
+ * node. Should your use case involve watching all child nodes of your specified
+ * path in a ZooKeeper ensemble, see {@link Announcer}.
*/
public class NodeAnnouncer
{
From 20cfb0f03d56d340f5aecc072b70e472698eb15c Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 18 Nov 2024 09:54:40 +0800
Subject: [PATCH 14/43] Update Druid unannouncement logging to make sense
---
.../java/org/apache/druid/curator/announcement/Announcer.java | 2 +-
.../org/apache/druid/curator/announcement/NodeAnnouncer.java | 2 +-
2 files changed, 2 insertions(+), 2 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index ba87ec07e258..20583255d295 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -409,7 +409,7 @@ public void unannounce(String path)
curator.inTransaction().delete().forPath(path).and().commit();
}
catch (KeeperException.NoNodeException e) {
- log.info("Node[%s] didn't exist anyway...", path);
+ log.info("Unannounced node[%s] that does not exist.", path);
}
catch (Exception e) {
throw new RuntimeException(e);
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 35d7e27b0640..833bb4ab0689 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -396,7 +396,7 @@ public void unannounce(String path)
curator.transaction().forOperations(curator.transactionOp().delete().forPath(path));
}
catch (KeeperException.NoNodeException e) {
- log.info("node[%s] didn't exist anyway...", path);
+ log.info("Unannounced node[%s] that does not exist.", path);
}
catch (Exception e) {
throw new RuntimeException(e);
From 2a1f803d04ce990ff79a7b83287031091d89bd79 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 18 Nov 2024 10:21:07 +0800
Subject: [PATCH 15/43] Replace deprecated inTransaction() with transaction()
---
.../druid/curator/announcement/Announcer.java | 18 +++++++++++-------
1 file changed, 11 insertions(+), 7 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index 20583255d295..da86d175d77a 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -22,8 +22,8 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.errorprone.annotations.concurrent.GuardedBy;
import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.api.transaction.CuratorTransaction;
-import org.apache.curator.framework.api.transaction.CuratorTransactionFinal;
+import org.apache.curator.framework.api.transaction.CuratorMultiTransaction;
+import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
@@ -163,17 +163,20 @@ public void stop()
}
if (!parentsIBuilt.isEmpty()) {
- CuratorTransaction transaction = curator.inTransaction();
+ CuratorMultiTransaction transaction = curator.transaction();
+
+ ArrayList operations = new ArrayList<>();
for (String parent : parentsIBuilt) {
try {
- transaction = transaction.delete().forPath(parent).and();
+ operations.add(curator.transactionOp().delete().forPath(parent));
}
catch (Exception e) {
log.info(e, "Unable to delete parent[%s].", parent);
}
}
+
try {
- ((CuratorTransactionFinal) transaction).commit();
+ transaction.forOperations(operations);
}
catch (Exception e) {
log.info(e, "Unable to commit transaction.");
@@ -227,7 +230,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
// I don't have a watcher on this path yet, create a Map and start watching.
announcements.putIfAbsent(parentPath, new ConcurrentHashMap<>());
- // Guaranteed to be non-null, but might be a map put in there by another thread.
+ // Guaranteed to be non-null, but might be a map put in here by another thread.
final ConcurrentMap finalSubPaths = announcements.get(parentPath);
// Synchronize to make sure that I only create a listener once.
@@ -406,7 +409,8 @@ public void unannounce(String path)
log.info("Unannouncing [%s]", path);
try {
- curator.inTransaction().delete().forPath(path).and().commit();
+ CuratorOp deleteOp = curator.transactionOp().delete().forPath(path);
+ curator.transaction().forOperations(deleteOp);
}
catch (KeeperException.NoNodeException e) {
log.info("Unannounced node[%s] that does not exist.", path);
From a857bfcb713e3dd49fe86874f79d3135da273a37 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Tue, 19 Nov 2024 16:40:20 +0800
Subject: [PATCH 16/43] Refactor buildParentPath
---
.../curator/announcement/NodeAnnouncer.java | 21 +++++++------------
1 file changed, 8 insertions(+), 13 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 833bb4ab0689..955a3baa7a4e 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -207,8 +207,14 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
byte[] announcedPayload = announcedPaths.get(path);
if (announcedPayload == null) {
+ boolean buildParentPath = false;
// Payload does not exist. We have yet to announce this path. Check if we need to build a parent path.
- boolean shouldBuildParentPath = canBuildParentPath(parentPath);
+ try {
+ buildParentPath = curator.checkExists().forPath(parentPath) == null;
+ }
+ catch (Exception e) {
+ log.debug(e, "Failed to check existence of parent path. Proceeding without creating parent path.");
+ }
// Synchronize to make sure that I only create a listener once.
synchronized (toAnnounce) {
@@ -216,7 +222,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
final NodeCache cache = setupNodeCache(path);
if (started) {
- if (shouldBuildParentPath) {
+ if (buildParentPath) {
createPath(parentPath, removeParentIfCreated);
}
startCache(cache);
@@ -238,17 +244,6 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
}
}
- private boolean canBuildParentPath(String parentPath)
- {
- try {
- return (curator.checkExists().forPath(parentPath) == null);
- }
- catch (Exception e) {
- log.debug(e, "Failed to check existence of parent path. Proceeding without creating parent path.");
- return false;
- }
- }
-
@GuardedBy("toAnnounce")
private NodeCache setupNodeCache(String path)
{
From 582691e088697f9c1dbd3e74cd6abe9a88dce118 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Fri, 29 Nov 2024 16:22:13 +0800
Subject: [PATCH 17/43] Add executorService for NodeCache
---
.../worker/WorkerCuratorCoordinator.java | 3 +-
.../curator/announcement/NodeAnnouncer.java | 17 ++++++++---
.../apache/druid/guice/AnnouncerModule.java | 2 +-
.../client/BatchServerInventoryViewTest.java | 7 ++---
.../announcement/NodeAnnouncerTest.java | 28 +++++++++++--------
...torDruidNodeAnnouncerAndDiscoveryTest.java | 3 +-
6 files changed, 37 insertions(+), 23 deletions(-)
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
index 6e09217e958e..1d903d2bd0e8 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
@@ -29,6 +29,7 @@
import org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.common.logger.Logger;
@@ -76,7 +77,7 @@ public WorkerCuratorCoordinator(
this.curatorFramework = curatorFramework;
this.worker = worker;
- this.announcer = new NodeAnnouncer(curatorFramework);
+ this.announcer = new NodeAnnouncer(curatorFramework, Execs.directExecutor());
this.baseAnnouncementsPath = getPath(Arrays.asList(indexerZkConfig.getAnnouncementsPath(), worker.getHost()));
this.baseTaskPath = getPath(Arrays.asList(indexerZkConfig.getTasksPath(), worker.getHost()));
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 955a3baa7a4e..469b028f614c 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -42,6 +42,7 @@
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
/**
* The {@link NodeAnnouncer} class is responsible for announcing a single node
@@ -58,6 +59,8 @@ public class NodeAnnouncer
private static final Logger log = new Logger(NodeAnnouncer.class);
private final CuratorFramework curator;
+ private final ExecutorService nodeCacheExecutor;
+
private final ConcurrentMap listeners = new ConcurrentHashMap<>();
private final ConcurrentHashMap announcedPaths = new ConcurrentHashMap<>();
@@ -88,9 +91,10 @@ public class NodeAnnouncer
@GuardedBy("toAnnounce")
private final List pathsCreatedInThisAnnouncer = new ArrayList<>();
- public NodeAnnouncer(CuratorFramework curator)
+ public NodeAnnouncer(CuratorFramework curator, ExecutorService exec)
{
this.curator = curator;
+ this.nodeCacheExecutor = exec;
}
@VisibleForTesting
@@ -249,7 +253,7 @@ private NodeCache setupNodeCache(String path)
{
final NodeCache cache = new NodeCache(curator, path, true);
cache.getListenable().addListener(
- () -> {
+ () -> nodeCacheExecutor.submit(() -> {
ChildData currentData = cache.getCurrentData();
if (currentData == null) {
@@ -262,10 +266,15 @@ private NodeCache setupNodeCache(String path)
"Ephemeral node at path [%s] was unexpectedly removed. Recreating node with previous data.",
path
);
- createAnnouncement(path, previouslyAnnouncedData);
+ try {
+ createAnnouncement(path, previouslyAnnouncedData);
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
}
}
- }
+ })
);
return cache;
}
diff --git a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
index 3f76028f5b65..dd447fbf6c3a 100644
--- a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
+++ b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
@@ -75,6 +75,6 @@ public Announcer getAnnouncer(CuratorFramework curator)
@ManageLifecycle
public NodeAnnouncer getNodeAnnouncer(CuratorFramework curator)
{
- return new NodeAnnouncer(curator);
+ return new NodeAnnouncer(curator, Execs.singleThreaded("Announcer-%s"));
}
}
diff --git a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java
index a74636faaf1c..30e62c9983f5 100644
--- a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java
+++ b/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java
@@ -118,13 +118,10 @@ public void setUp() throws Exception
jsonMapper = TestHelper.makeJsonMapper();
- announcer = new Announcer(
- cf,
- Execs.directExecutor()
- );
+ announcer = new Announcer(cf, Execs.directExecutor());
announcer.start();
- nodeAnnouncer = new NodeAnnouncer(cf);
+ nodeAnnouncer = new NodeAnnouncer(cf, Execs.directExecutor());
nodeAnnouncer.start();
DruidServerMetadata serverMetadata = new DruidServerMetadata(
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
index bcf0a2d14e33..a2c846cd5ba3 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -28,6 +28,8 @@
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.ZKPathsUtils;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.data.Stat;
@@ -39,17 +41,21 @@
import java.util.Collection;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
/**
*
*/
public class NodeAnnouncerTest extends CuratorTestBase
{
+ private static final Logger log = new Logger(NodeAnnouncerTest.class);
+ private ExecutorService exec;
@Before
public void setUp() throws Exception
{
setupServerAndCurator();
+ exec = Execs.singleThreaded("test-node-announcer-sanity-%s");
}
@After
@@ -63,7 +69,7 @@ public void testAnnounceBeforeStartingNodeAnnouncer() throws Exception
{
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/testAnnounce";
@@ -80,7 +86,7 @@ public void testCreateParentPath() throws Exception
{
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/newParent/testPath";
final String parentPath = ZKPathsUtils.getParentPath(testPath);
@@ -104,7 +110,7 @@ public void testAnnounceSamePathWithDifferentPayloadThrowsIAE() throws Exception
{
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final byte[] tilly = StringUtils.toUtf8("tilly");
final String testPath = "/testPath";
@@ -133,7 +139,7 @@ public void testUpdateBeforeStartingNodeAnnouncer() throws Exception
{
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final byte[] tilly = StringUtils.toUtf8("tilly");
final String testPath = "/testAnnounce";
@@ -152,7 +158,7 @@ public void testUpdateSuccessfully() throws Exception
{
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final byte[] tilly = StringUtils.toUtf8("tilly");
final String testPath = "/testUpdate";
@@ -174,7 +180,7 @@ public void testUpdateWithNonExistentPath() throws Exception
{
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/testUpdate";
@@ -190,7 +196,7 @@ public void testSanity() throws Exception
{
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath1 = "/test1";
@@ -266,7 +272,7 @@ public void testSessionKilled() throws Exception
{
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
try {
curator.inTransaction().create().forPath("/somewhere").and().commit();
announcer.start();
@@ -318,7 +324,7 @@ public void testCleansUpItsLittleTurdlings() throws Exception
{
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
@@ -344,7 +350,7 @@ public void testLeavesBehindTurdlingsThatAlreadyExisted() throws Exception
{
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
@@ -373,7 +379,7 @@ public void testLeavesBehindTurdlingsWhenToldTo() throws Exception
{
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
diff --git a/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java
index f2755d760dc8..7d9368a8e1e5 100644
--- a/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java
+++ b/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java
@@ -29,6 +29,7 @@
import org.apache.druid.discovery.DruidNodeDiscovery;
import org.apache.druid.discovery.NodeRole;
import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.initialization.ServerConfig;
import org.apache.druid.server.initialization.ZkPathsConfig;
@@ -69,7 +70,7 @@ public void testAnnouncementAndDiscovery() throws Exception
curator.start();
curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator);
+ NodeAnnouncer announcer = new NodeAnnouncer(curator, Execs.directExecutor());
announcer.start();
CuratorDruidNodeAnnouncer druidNodeAnnouncer = new CuratorDruidNodeAnnouncer(
From a305748405e15caeeb9bf4c33ad1318f64977941 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Fri, 29 Nov 2024 18:43:48 +0800
Subject: [PATCH 18/43] Try replace all Announcer
---
.../BatchDataSegmentAnnouncer.java | 8 +++----
.../client/BatchServerInventoryViewTest.java | 15 ++++++-------
.../curator/announcement/AnnouncerTest.java | 1 +
.../BatchDataSegmentAnnouncerTest.java | 21 +++++++++++++------
4 files changed, 26 insertions(+), 19 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java
index cf407102c5a7..4b1784d726ac 100644
--- a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java
@@ -32,7 +32,7 @@
import org.apache.curator.utils.ZKPaths;
import org.apache.druid.common.utils.UUIDUtils;
import org.apache.druid.curator.ZkEnablementConfig;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
@@ -64,7 +64,7 @@ public class BatchDataSegmentAnnouncer implements DataSegmentAnnouncer
private final BatchDataSegmentAnnouncerConfig config;
@Nullable //Null if zk is disabled or isSkipSegmentAnnouncementOnZk = true
- private final Announcer announcer;
+ private final NodeAnnouncer announcer;
private final ObjectMapper jsonMapper;
private final String liveSegmentLocation;
@@ -91,7 +91,7 @@ public BatchDataSegmentAnnouncer(
DruidServerMetadata server,
final BatchDataSegmentAnnouncerConfig config,
ZkPathsConfig zkPaths,
- Provider announcerProvider,
+ Provider announcerProvider,
ObjectMapper jsonMapper,
ZkEnablementConfig zkEnablementConfig
)
@@ -127,7 +127,7 @@ public BatchDataSegmentAnnouncer(
DruidServerMetadata server,
final BatchDataSegmentAnnouncerConfig config,
ZkPathsConfig zkPaths,
- Announcer announcer,
+ NodeAnnouncer announcer,
ObjectMapper jsonMapper
)
{
diff --git a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java
index 30e62c9983f5..6a21eac963ad 100644
--- a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java
+++ b/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java
@@ -38,7 +38,6 @@
import org.apache.druid.client.DruidServer;
import org.apache.druid.client.ServerView;
import org.apache.druid.curator.PotentiallyGzippedCompressionProvider;
-import org.apache.druid.curator.announcement.Announcer;
import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
@@ -89,7 +88,6 @@ public class BatchServerInventoryViewTest
private TestingCluster testingCluster;
private CuratorFramework cf;
private ObjectMapper jsonMapper;
- private Announcer announcer;
private NodeAnnouncer nodeAnnouncer;
private BatchDataSegmentAnnouncer segmentAnnouncer;
private DataSegmentServerAnnouncer serverAnnouncer;
@@ -118,9 +116,6 @@ public void setUp() throws Exception
jsonMapper = TestHelper.makeJsonMapper();
- announcer = new Announcer(cf, Execs.directExecutor());
- announcer.start();
-
nodeAnnouncer = new NodeAnnouncer(cf, Execs.directExecutor());
nodeAnnouncer.start();
@@ -162,7 +157,7 @@ public int getSegmentsPerNode()
}
},
zkPathsConfig,
- announcer,
+ nodeAnnouncer,
jsonMapper
);
@@ -227,7 +222,6 @@ public void tearDown() throws Exception
batchServerInventoryView.stop();
filteredBatchServerInventoryView.stop();
serverAnnouncer.unannounce();
- announcer.stop();
nodeAnnouncer.stop();
cf.close();
testingCluster.stop();
@@ -428,7 +422,10 @@ private void waitForUpdateEvents(int count)
public void testSameTimeZnode() throws Exception
{
final int numThreads = INITIAL_SEGMENTS / 10;
- final ListeningExecutorService executor = MoreExecutors.listeningDecorator(Execs.multiThreaded(numThreads, "BatchServerInventoryViewTest-%d"));
+ final ListeningExecutorService executor = MoreExecutors.listeningDecorator(Execs.multiThreaded(
+ numThreads,
+ "BatchServerInventoryViewTest-%d"
+ ));
segmentAnnouncer.announceSegments(testSegments);
@@ -477,7 +474,7 @@ public String getBase()
return TEST_BASE_PATH;
}
},
- announcer,
+ nodeAnnouncer,
jsonMapper
);
List segments = new ArrayList();
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
index 3f0c7e545e42..27c05a1e4deb 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
@@ -45,6 +45,7 @@
import java.util.concurrent.ExecutorService;
/**
+ *
*/
public class AnnouncerTest extends CuratorTestBase
{
diff --git a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java
index 986252d75502..087a2f3080bd 100644
--- a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java
@@ -31,7 +31,7 @@
import org.apache.curator.retry.ExponentialBackoffRetry;
import org.apache.curator.test.TestingCluster;
import org.apache.druid.curator.PotentiallyGzippedCompressionProvider;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.segment.TestHelper;
@@ -284,7 +284,10 @@ public void testSkipDimensions() throws Exception
List zNodes = cf.getChildren().forPath(TEST_SEGMENTS_PATH);
for (String zNode : zNodes) {
- DataSegment announcedSegment = Iterables.getOnlyElement(segmentReader.read(JOINER.join(TEST_SEGMENTS_PATH, zNode)));
+ DataSegment announcedSegment = Iterables.getOnlyElement(segmentReader.read(JOINER.join(
+ TEST_SEGMENTS_PATH,
+ zNode
+ )));
Assert.assertEquals(announcedSegment, firstSegment);
Assert.assertTrue(announcedSegment.getDimensions().isEmpty());
Assert.assertTrue(announcedSegment.getMetrics().isEmpty());
@@ -307,7 +310,10 @@ public void testSkipLoadSpec() throws Exception
List zNodes = cf.getChildren().forPath(TEST_SEGMENTS_PATH);
for (String zNode : zNodes) {
- DataSegment announcedSegment = Iterables.getOnlyElement(segmentReader.read(JOINER.join(TEST_SEGMENTS_PATH, zNode)));
+ DataSegment announcedSegment = Iterables.getOnlyElement(segmentReader.read(JOINER.join(
+ TEST_SEGMENTS_PATH,
+ zNode
+ )));
Assert.assertEquals(announcedSegment, firstSegment);
Assert.assertNull(announcedSegment.getLoadSpec());
}
@@ -402,7 +408,8 @@ public void testSchemaAnnounce() throws Exception
segmentAnnouncer.announceSegmentSchemas(
taskId,
new SegmentSchemas(Collections.singletonList(absoluteSchema1)),
- new SegmentSchemas(Collections.singletonList(absoluteSchema1)));
+ new SegmentSchemas(Collections.singletonList(absoluteSchema1))
+ );
ChangeRequestsSnapshot snapshot;
@@ -620,7 +627,7 @@ public Set read(String path)
}
}
- private static class TestAnnouncer extends Announcer
+ private static class TestAnnouncer extends NodeAnnouncer
{
private final ConcurrentHashMap> numPathAnnounced = new ConcurrentHashMap<>();
@@ -632,7 +639,9 @@ private TestAnnouncer(CuratorFramework curator, ExecutorService exec)
@Override
public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
{
- numPathAnnounced.computeIfAbsent(path, k -> new ConcurrentHashMap<>()).computeIfAbsent(bytes, k -> new AtomicInteger(0)).incrementAndGet();
+ numPathAnnounced.computeIfAbsent(path, k -> new ConcurrentHashMap<>())
+ .computeIfAbsent(bytes, k -> new AtomicInteger(0))
+ .incrementAndGet();
super.announce(path, bytes, removeParentIfCreated);
}
}
From 438bfd91dcd5b27fce27bb8f78b49bdc4499ce4d Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 2 Dec 2024 10:44:40 +0800
Subject: [PATCH 19/43] Add JavaDocs
---
.../druid/curator/announcement/Announcer.java | 18 +++++++----
.../curator/announcement/NodeAnnouncer.java | 30 +++++++++++--------
.../apache/druid/guice/AnnouncerModule.java | 2 +-
3 files changed, 31 insertions(+), 19 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index da86d175d77a..b5e5fcdc6173 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -55,10 +55,11 @@
/**
* The {@link Announcer} class manages the announcement of a node, and watches all child
- * nodes under the specified path in a ZooKeeper ensemble. It monitors these nodes
+ * and sibling nodes under the specified path in a ZooKeeper ensemble. It monitors these nodes
* to ensure their existence and manage their lifecycle collectively.
*
- * Use this class when you need to manage the lifecycle of all child nodes under the
+ *
+ * Use this class when you need to manage the lifecycle of all child nodes under the parent of your
* specified path. Should your use case involve the management of a standalone node
* instead, see {@link NodeAnnouncer}.
*/
@@ -186,7 +187,7 @@ public void stop()
}
/**
- * Like announce(path, bytes, true).
+ * Overload of {@link #announce(String, byte[], boolean)}, but removes parent node of path after announcement.
*/
public void announce(String path, byte[] bytes)
{
@@ -194,12 +195,17 @@ public void announce(String path, byte[] bytes)
}
/**
- * Announces the provided bytes at the given path. Announcement means that it will create an ephemeral node
- * and monitor it to make sure that it always exists until it is unannounced or this object is closed.
+ * Announces the provided bytes at the given path.
+ *
+ *
+ * Announcement using {@link Announcer} will create an ephemeral znode at the specified path, and uses its parent
+ * path to watch all the siblings and children znodes of your specified path. The watched nodes will always exist
+ * until it is unannounced, or until {@link #stop()} is called.
+ *
*
* @param path The path to announce at
* @param bytes The payload to announce
- * @param removeParentIfCreated remove parent of "path" if we had created that parent
+ * @param removeParentIfCreated remove parent of "path" if we had created that parent during announcement
*/
public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
{
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 469b028f614c..2cf6532826f4 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -25,6 +25,7 @@
import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.curator.utils.CloseableExecutorService;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.io.Closer;
@@ -51,15 +52,15 @@
* explicitly unannounced or the object is closed.
*
* Use this class when you need to manage the lifecycle of a standalone
- * node. Should your use case involve watching all child nodes of your specified
- * path in a ZooKeeper ensemble, see {@link Announcer}.
+ * node. Should your use case involve watching all child and sibling nodes of your
+ * specified path in a ZooKeeper ensemble, see {@link Announcer}.
*/
public class NodeAnnouncer
{
private static final Logger log = new Logger(NodeAnnouncer.class);
private final CuratorFramework curator;
- private final ExecutorService nodeCacheExecutor;
+ private final CloseableExecutorService nodeCacheExecutor;
private final ConcurrentMap listeners = new ConcurrentHashMap<>();
private final ConcurrentHashMap announcedPaths = new ConcurrentHashMap<>();
@@ -94,7 +95,7 @@ public class NodeAnnouncer
public NodeAnnouncer(CuratorFramework curator, ExecutorService exec)
{
this.curator = curator;
- this.nodeCacheExecutor = exec;
+ this.nodeCacheExecutor = new CloseableExecutorService(exec);
}
@VisibleForTesting
@@ -153,6 +154,8 @@ private void closeResources()
for (String announcementPath : announcedPaths.keySet()) {
closer.register(() -> unannounce(announcementPath));
}
+ closer.register(nodeCacheExecutor);
+
CloseableUtils.closeAndWrapExceptions(closer);
}
@@ -188,18 +191,20 @@ public void announce(String path, byte[] bytes)
}
/**
- * Announces the provided bytes at the given path. Announcement means that it will create an ephemeral node
- * and monitor it to make sure that it always exists until it is unannounced or this object is closed.
+ * Announces the provided bytes at the given path.
+ *
+ *
+ * Announcement using {@link NodeAnnouncer} will create an ephemeral znode at the specified path, and listens for
+ * changes on your znode. Your znode will exist until it is unannounced, or until {@link #stop()} is called.
+ *
*
* @param path The path to announce at
* @param bytes The payload to announce
- * @param removeParentIfCreated remove parent of "path" if we had created that parent
+ * @param removeParentIfCreated remove parent of "path" if we had created that parent during announcement
*/
public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
{
synchronized (toAnnounce) {
- // In the case that this method is called by other components or thread that assumes the NodeAnnouncer
- // is ready when NodeAnnouncer has not started, we will queue the announcement request.
if (!started) {
log.debug("NodeAnnouncer has not started yet, queuing announcement for later processing...");
toAnnounce.add(new Announceable(path, bytes, removeParentIfCreated));
@@ -210,9 +215,10 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
final String parentPath = ZKPathsUtils.getParentPath(path);
byte[] announcedPayload = announcedPaths.get(path);
+ // If announcedPayload is null, this means that we have yet to announce this path.
+ // There is a possibility that the parent paths do not exist, so we check if we need to create the parent path first.
if (announcedPayload == null) {
boolean buildParentPath = false;
- // Payload does not exist. We have yet to announce this path. Check if we need to build a parent path.
try {
buildParentPath = curator.checkExists().forPath(parentPath) == null;
}
@@ -327,8 +333,8 @@ private void startCache(NodeCache cache)
try {
cache.start();
}
- catch (Exception e) {
- throw CloseableUtils.closeInCatch(new RuntimeException(e), cache);
+ catch (Throwable e) {
+ throw CloseableUtils.closeAndWrapInCatch(e, cache);
}
}
diff --git a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
index dd447fbf6c3a..46a5c5a0a7ff 100644
--- a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
+++ b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
@@ -72,7 +72,7 @@ public Announcer getAnnouncer(CuratorFramework curator)
}
@Provides
- @ManageLifecycle
+ @ManageLifecycleAnnouncements
public NodeAnnouncer getNodeAnnouncer(CuratorFramework curator)
{
return new NodeAnnouncer(curator, Execs.singleThreaded("Announcer-%s"));
From c6bfbd6bca08b278a036c0c27730a5ebd0d748ec Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 2 Dec 2024 14:12:58 +0800
Subject: [PATCH 20/43] Refactor Test classes
---
.../curator/announcement/AnnouncerTest.java | 77 +++++----------
.../announcement/NodeAnnouncerTest.java | 94 +++++++------------
2 files changed, 57 insertions(+), 114 deletions(-)
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
index 27c05a1e4deb..d78d20a62b03 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
@@ -19,7 +19,6 @@
package org.apache.druid.curator.announcement;
-import com.google.common.collect.Sets;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.CuratorEvent;
import org.apache.curator.framework.api.CuratorEventType;
@@ -39,8 +38,8 @@
import org.junit.Before;
import org.junit.Test;
+import java.util.Arrays;
import java.util.Collection;
-import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
@@ -57,6 +56,8 @@ public void setUp() throws Exception
{
setupServerAndCurator();
exec = Execs.singleThreaded("test-announcer-sanity-%s");
+ curator.start();
+ curator.blockUntilConnected();
}
@After
@@ -68,8 +69,6 @@ public void tearDown()
@Test(timeout = 60_000L)
public void testSanity() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
Announcer announcer = new Announcer(curator, exec);
announcer.initializeAddedChildren();
@@ -99,19 +98,8 @@ public void testSanity() throws Exception
curator.getData().decompressed().forPath(testPath2)
);
- final CountDownLatch latch = new CountDownLatch(1);
- curator.getCuratorListenable().addListener(
- new CuratorListener()
- {
- @Override
- public void eventReceived(CuratorFramework client, CuratorEvent event)
- {
- if (event.getType() == CuratorEventType.CREATE && event.getPath().equals(testPath1)) {
- latch.countDown();
- }
- }
- }
- );
+ final CountDownLatch latch = createCountdownLatchForPaths(testPath1);
+
final CuratorOp deleteOp = curator.transactionOp().delete().forPath(testPath1);
final Collection results = curator.transaction().forOperations(deleteOp);
Assert.assertEquals(1, results.size());
@@ -150,39 +138,24 @@ public void eventReceived(CuratorFramework client, CuratorEvent event)
@Test(timeout = 60_000L)
public void testSessionKilled() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
Announcer announcer = new Announcer(curator, exec);
try {
- curator.inTransaction().create().forPath("/somewhere").and().commit();
+ CuratorOp createOp = curator.transactionOp().create().forPath("/somewhere");
+ curator.transaction().forOperations(createOp);
announcer.start();
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath1 = "/test1";
final String testPath2 = "/somewhere/test2";
- final Set paths = Sets.newHashSet(testPath1, testPath2);
+ final String[] paths = new String[]{testPath1, testPath2};
announcer.announce(testPath1, billy);
announcer.announce(testPath2, billy);
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
- final CountDownLatch latch = new CountDownLatch(1);
- curator.getCuratorListenable().addListener(
- new CuratorListener()
- {
- @Override
- public void eventReceived(CuratorFramework client, CuratorEvent event)
- {
- if (event.getType() == CuratorEventType.CREATE) {
- paths.remove(event.getPath());
- if (paths.isEmpty()) {
- latch.countDown();
- }
- }
- }
- }
- );
+ final CountDownLatch latch = createCountdownLatchForPaths(paths);
+
KillSession.kill(curator.getZookeeperClient().getZooKeeper(), server.getConnectString());
Assert.assertTrue(timing.forWaiting().awaitLatch(latch));
@@ -205,10 +178,8 @@ public void eventReceived(CuratorFramework client, CuratorEvent event)
}
@Test
- public void testCleansUpItsLittleTurdlings() throws Exception
+ public void testRemovesParentIfCreated() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
Announcer announcer = new Announcer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
@@ -231,14 +202,12 @@ public void testCleansUpItsLittleTurdlings() throws Exception
}
@Test
- public void testLeavesBehindTurdlingsThatAlreadyExisted() throws Exception
+ public void testLeavesBehindParentPathIfAlreadyExists() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
Announcer announcer = new Announcer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
- final String testPath = "/somewhere/test2";
+ final String testPath = "/somewhere/test";
final String parent = ZKPathsUtils.getParentPath(testPath);
curator.create().forPath(parent);
@@ -260,14 +229,12 @@ public void testLeavesBehindTurdlingsThatAlreadyExisted() throws Exception
}
@Test
- public void testLeavesBehindTurdlingsWhenToldTo() throws Exception
+ public void testLeavesParentPathsUntouchedWhenInstructed() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
Announcer announcer = new Announcer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
- final String testPath = "/somewhere/test2";
+ final String testPath = "/somewhere/test";
final String parent = ZKPathsUtils.getParentPath(testPath);
announcer.start();
@@ -292,20 +259,26 @@ private void awaitAnnounce(
boolean removeParentsIfCreated
) throws InterruptedException
{
- final CountDownLatch latch = new CountDownLatch(1);
+ CountDownLatch latch = createCountdownLatchForPaths(path);
+ announcer.announce(path, bytes, removeParentsIfCreated);
+ latch.await();
+ }
+
+ private CountDownLatch createCountdownLatchForPaths(String ...path) {
+ final CountDownLatch latch = new CountDownLatch(path.length);
curator.getCuratorListenable().addListener(
new CuratorListener()
{
@Override
public void eventReceived(CuratorFramework client, CuratorEvent event)
{
- if (event.getType() == CuratorEventType.CREATE && event.getPath().equals(path)) {
+ if (event.getType() == CuratorEventType.CREATE && Arrays.asList(path).contains(event.getPath())) {
latch.countDown();
}
}
}
);
- announcer.announce(path, bytes, removeParentsIfCreated);
- latch.await();
+
+ return latch;
}
}
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
index a2c846cd5ba3..4c02b3f1b70e 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -19,8 +19,10 @@
package org.apache.druid.curator.announcement;
-import com.google.common.collect.Sets;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.CuratorEvent;
import org.apache.curator.framework.api.CuratorEventType;
+import org.apache.curator.framework.api.CuratorListener;
import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
import org.apache.curator.test.KillSession;
@@ -38,8 +40,8 @@
import org.junit.Before;
import org.junit.Test;
+import java.util.Arrays;
import java.util.Collection;
-import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
@@ -56,6 +58,8 @@ public void setUp() throws Exception
{
setupServerAndCurator();
exec = Execs.singleThreaded("test-node-announcer-sanity-%s");
+ curator.start();
+ curator.blockUntilConnected();
}
@After
@@ -64,28 +68,9 @@ public void tearDown()
tearDownServerAndCurator();
}
- @Test
- public void testAnnounceBeforeStartingNodeAnnouncer() throws Exception
- {
- curator.start();
- curator.blockUntilConnected();
- NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
- final byte[] billy = StringUtils.toUtf8("billy");
- final String testPath = "/testAnnounce";
-
- announcer.announce(testPath, billy);
- announcer.start();
-
- // Verify that the path was announced
- Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
- announcer.stop();
- }
-
@Test(timeout = 60_000L)
public void testCreateParentPath() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/newParent/testPath";
@@ -108,8 +93,6 @@ public void testCreateParentPath() throws Exception
@Test(timeout = 60_000L)
public void testAnnounceSamePathWithDifferentPayloadThrowsIAE() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final byte[] tilly = StringUtils.toUtf8("tilly");
@@ -137,8 +120,6 @@ public void testAnnounceSamePathWithDifferentPayloadThrowsIAE() throws Exception
@Test
public void testUpdateBeforeStartingNodeAnnouncer() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final byte[] tilly = StringUtils.toUtf8("tilly");
@@ -156,8 +137,6 @@ public void testUpdateBeforeStartingNodeAnnouncer() throws Exception
@Test
public void testUpdateSuccessfully() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final byte[] tilly = StringUtils.toUtf8("tilly");
@@ -176,10 +155,8 @@ public void testUpdateSuccessfully() throws Exception
}
@Test
- public void testUpdateWithNonExistentPath() throws Exception
+ public void testUpdateNonExistentPath()
{
- curator.start();
- curator.blockUntilConnected();
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/testUpdate";
@@ -194,8 +171,6 @@ public void testUpdateWithNonExistentPath() throws Exception
@Test(timeout = 60_000L)
public void testSanity() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
@@ -270,34 +245,23 @@ public void testSanity() throws Exception
@Test(timeout = 60_000L)
public void testSessionKilled() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
try {
- curator.inTransaction().create().forPath("/somewhere").and().commit();
+ CuratorOp createOp = curator.transactionOp().create().forPath("/somewhere");
+ curator.transaction().forOperations(createOp);
announcer.start();
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath1 = "/test1";
final String testPath2 = "/somewhere/test2";
- final Set paths = Sets.newHashSet(testPath1, testPath2);
+ final String[] paths = new String[]{testPath1, testPath2};
announcer.announce(testPath1, billy);
announcer.announce(testPath2, billy);
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
- final CountDownLatch latch = new CountDownLatch(1);
- curator.getCuratorListenable().addListener(
- (client, event) -> {
- if (event.getType() == CuratorEventType.CREATE) {
- paths.remove(event.getPath());
- if (paths.isEmpty()) {
- latch.countDown();
- }
- }
- }
- );
+ final CountDownLatch latch = createCountdownLatchForPaths(paths);
KillSession.kill(curator.getZookeeperClient().getZooKeeper(), server.getConnectString());
Assert.assertTrue(timing.forWaiting().awaitLatch(latch));
@@ -320,10 +284,8 @@ public void testSessionKilled() throws Exception
}
@Test
- public void testCleansUpItsLittleTurdlings() throws Exception
+ public void testRemovesParentIfCreated() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
@@ -346,10 +308,8 @@ public void testCleansUpItsLittleTurdlings() throws Exception
}
@Test
- public void testLeavesBehindTurdlingsThatAlreadyExisted() throws Exception
+ public void testLeavesBehindParentPathIfAlreadyExists() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
@@ -375,10 +335,8 @@ public void testLeavesBehindTurdlingsThatAlreadyExisted() throws Exception
}
@Test
- public void testLeavesBehindTurdlingsWhenToldTo() throws Exception
+ public void testLeavesParentPathsUntouchedWhenInstructed() throws Exception
{
- curator.start();
- curator.blockUntilConnected();
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
@@ -407,15 +365,27 @@ private void awaitAnnounce(
boolean removeParentsIfCreated
) throws InterruptedException
{
- final CountDownLatch latch = new CountDownLatch(1);
+ final CountDownLatch latch = createCountdownLatchForPaths(path);
+ announcer.announce(path, bytes, removeParentsIfCreated);
+ latch.await();
+ }
+
+ private CountDownLatch createCountdownLatchForPaths(String... path)
+ {
+ final CountDownLatch latch = new CountDownLatch(path.length);
curator.getCuratorListenable().addListener(
- (client, event) -> {
- if (event.getType() == CuratorEventType.CREATE && event.getPath().equals(path)) {
- latch.countDown();
+ new CuratorListener()
+ {
+ @Override
+ public void eventReceived(CuratorFramework client, CuratorEvent event)
+ {
+ if (event.getType() == CuratorEventType.CREATE && Arrays.asList(path).contains(event.getPath())) {
+ latch.countDown();
+ }
}
}
);
- announcer.announce(path, bytes, removeParentsIfCreated);
- latch.await();
+
+ return latch;
}
}
From 1aea3210c78ad5f56ae942bda09e6e955697389b Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 2 Dec 2024 15:54:10 +0800
Subject: [PATCH 21/43] Add JavaDocs and logging messages
---
.../druid/curator/announcement/Announcer.java | 22 ++-
.../curator/announcement/NodeAnnouncer.java | 164 +++++++++---------
2 files changed, 96 insertions(+), 90 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index b5e5fcdc6173..8d19afb28e02 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -59,9 +59,10 @@
* to ensure their existence and manage their lifecycle collectively.
*
*
- * Use this class when you need to manage the lifecycle of all child nodes under the parent of your
- * specified path. Should your use case involve the management of a standalone node
- * instead, see {@link NodeAnnouncer}.
+ * This class uses Apache Curator's PathChildrenCache recipe under the hood to track all znodes
+ * under the specified node's parent. See {@link NodeAnnouncer} for an announcer that
+ * uses the NodeCache recipe instead.
+ *
*/
public class Announcer
{
@@ -172,7 +173,7 @@ public void stop()
operations.add(curator.transactionOp().delete().forPath(parent));
}
catch (Exception e) {
- log.info(e, "Unable to delete parent[%s].", parent);
+ log.info(e, "Unable to delete parent[%s] when closing Announcer.", parent);
}
}
@@ -180,7 +181,7 @@ public void stop()
transaction.forOperations(operations);
}
catch (Exception e) {
- log.info(e, "Unable to commit transaction.");
+ log.info(e, "Unable to commit transaction when closing Announcer.");
}
}
}
@@ -211,6 +212,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
{
synchronized (toAnnounce) {
if (!started) {
+ log.debug("Announcer has not started yet, queuing announcement for later processing...");
toAnnounce.add(new Announceable(path, bytes, removeParentIfCreated));
return;
}
@@ -230,7 +232,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
}
}
catch (Exception e) {
- log.debug(e, "Problem checking if the parent existed, ignoring.");
+ log.warn(e, "Failed to check existence of parent path. Proceeding without creating parent path.");
}
// I don't have a watcher on this path yet, create a Map and start watching.
@@ -421,6 +423,9 @@ public void unannounce(String path)
catch (KeeperException.NoNodeException e) {
log.info("Unannounced node[%s] that does not exist.", path);
}
+ catch (KeeperException.NotEmptyException e) {
+ log.warn("Unannouncing non-empty path[%s]", path);
+ }
catch (Exception e) {
throw new RuntimeException(e);
}
@@ -445,8 +450,11 @@ private void createPath(String parentPath, boolean removeParentsIfCreated)
}
log.debug("Created parentPath[%s], %s remove on stop.", parentPath, removeParentsIfCreated ? "will" : "will not");
}
- catch (Exception e) {
+ catch (KeeperException.NodeExistsException e) {
log.info(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
}
+ catch (Exception e) {
+ log.error(e, "Unhandled exception when creating parentPath[%s].", parentPath);
+ }
}
}
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 2cf6532826f4..c678eaa5002b 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -22,13 +22,13 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.errorprone.annotations.concurrent.GuardedBy;
import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.transaction.CuratorMultiTransaction;
import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.NodeCache;
import org.apache.curator.utils.CloseableExecutorService;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.common.logger.Logger;
@@ -37,12 +37,14 @@
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutorService;
/**
@@ -51,9 +53,11 @@
* and monitors its existence to ensure that it remains active until it is
* explicitly unannounced or the object is closed.
*
- * Use this class when you need to manage the lifecycle of a standalone
- * node. Should your use case involve watching all child and sibling nodes of your
- * specified path in a ZooKeeper ensemble, see {@link Announcer}.
+ *
+ * This class uses Apache Curator's NodeCache recipe under the hood to track a single
+ * node, along with all of its parent's status. See {@link Announcer} for an announcer that
+ * uses the PathChildrenCache recipe instead.
+ *
*/
public class NodeAnnouncer
{
@@ -90,7 +94,7 @@ public class NodeAnnouncer
* the node announcer is responsible for deleting all paths stored in this list.
*/
@GuardedBy("toAnnounce")
- private final List pathsCreatedInThisAnnouncer = new ArrayList<>();
+ private final List parentsIBuilt = new CopyOnWriteArrayList<>();
public NodeAnnouncer(CuratorFramework curator, ExecutorService exec)
{
@@ -107,7 +111,7 @@ Set getAddedPaths()
@LifecycleStart
public void start()
{
- log.info("Starting NodeAnnouncer");
+ log.debug("Starting NodeAnnouncer");
synchronized (toAnnounce) {
if (started) {
log.debug("Called start() but NodeAnnouncer have already started.");
@@ -131,7 +135,7 @@ public void start()
@LifecycleStop
public void stop()
{
- log.info("Stopping NodeAnnouncer");
+ log.debug("Stopping NodeAnnouncer");
synchronized (toAnnounce) {
if (!started) {
log.debug("Called stop() but NodeAnnouncer have not started.");
@@ -140,44 +144,45 @@ public void stop()
started = false;
closeResources();
- dropPathsCreatedInThisAnnouncer();
}
}
@GuardedBy("toAnnounce")
private void closeResources()
{
- Closer closer = Closer.create();
- for (NodeCache cache : listeners.values()) {
- closer.register(cache);
+ try {
+ CloseableUtils.closeAll(listeners.values());
+
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ finally {
+ nodeCacheExecutor.close();
}
+
for (String announcementPath : announcedPaths.keySet()) {
- closer.register(() -> unannounce(announcementPath));
+ unannounce(announcementPath);
}
- closer.register(nodeCacheExecutor);
- CloseableUtils.closeAndWrapExceptions(closer);
- }
+ if (!parentsIBuilt.isEmpty()) {
+ CuratorMultiTransaction transaction = curator.transaction();
- @GuardedBy("toAnnounce")
- private void dropPathsCreatedInThisAnnouncer()
- {
- if (!pathsCreatedInThisAnnouncer.isEmpty()) {
- final List deleteOps = new ArrayList<>(pathsCreatedInThisAnnouncer.size());
- for (String parent : pathsCreatedInThisAnnouncer) {
+ ArrayList operations = new ArrayList<>();
+ for (String parent : parentsIBuilt) {
try {
- deleteOps.add(curator.transactionOp().delete().forPath(parent));
+ operations.add(curator.transactionOp().delete().forPath(parent));
}
catch (Exception e) {
- log.error(e, "Unable to delete parent[%s].", parent);
+ log.info(e, "Unable to delete parent[%s] when closing NodeAnnouncer.", parent);
}
}
try {
- curator.transaction().forOperations(deleteOps);
+ transaction.forOperations(operations);
}
catch (Exception e) {
- log.error(e, "Unable to commit transaction.");
+ log.info(e, "Unable to commit transaction when closing NodeAnnouncer.");
}
}
}
@@ -223,7 +228,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
buildParentPath = curator.checkExists().forPath(parentPath) == null;
}
catch (Exception e) {
- log.debug(e, "Failed to check existence of parent path. Proceeding without creating parent path.");
+ log.warn(e, "Failed to check existence of parent path. Proceeding without creating parent path.");
}
// Synchronize to make sure that I only create a listener once.
@@ -268,10 +273,7 @@ private NodeCache setupNodeCache(String path)
// We will recreate the node again using the previous data.
final byte[] previouslyAnnouncedData = announcedPaths.get(path);
if (previouslyAnnouncedData != null) {
- log.info(
- "Ephemeral node at path [%s] was unexpectedly removed. Recreating node with previous data.",
- path
- );
+ log.info("Node[%s] dropped, reinstating.", path);
try {
createAnnouncement(path, previouslyAnnouncedData);
}
@@ -306,38 +308,6 @@ private boolean updateAnnouncedPaths(String path, byte[] bytes)
return Arrays.equals(updatedAnnouncementData, bytes);
}
- @GuardedBy("toAnnounce")
- private void createPath(String parentPath, boolean removeParentsIfCreated)
- {
- try {
- curator.create().creatingParentsIfNeeded().forPath(parentPath);
- if (removeParentsIfCreated) {
- pathsCreatedInThisAnnouncer.add(parentPath);
- }
- log.debug(
- "Created parentPath[%s], %s remove when stop() is called.",
- parentPath,
- removeParentsIfCreated ? "will" : "will not"
- );
- }
- catch (KeeperException.NodeExistsException e) {
- log.error(e, "The parentPath[%s] already exists.", parentPath);
- }
- catch (Exception e) {
- log.error(e, "Failed to create parentPath[%s].", parentPath);
- }
- }
-
- private void startCache(NodeCache cache)
- {
- try {
- cache.start();
- }
- catch (Throwable e) {
- throw CloseableUtils.closeAndWrapInCatch(e, cache);
- }
- }
-
public void update(final String path, final byte[] bytes)
{
synchronized (toAnnounce) {
@@ -346,29 +316,22 @@ public void update(final String path, final byte[] bytes)
toUpdate.add(new Announceable(path, bytes, false));
return;
}
- }
- byte[] oldBytes = announcedPaths.get(path);
+ byte[] oldBytes = announcedPaths.get(path);
- if (oldBytes == null) {
- throw new ISE("Cannot update path[%s] that hasn't been announced!", path);
- }
-
- boolean canUpdate = false;
- synchronized (toAnnounce) {
- if (!Arrays.equals(oldBytes, bytes)) {
- announcedPaths.put(path, bytes);
- canUpdate = true;
+ if (oldBytes == null) {
+ throw new ISE("Cannot update path[%s] that hasn't been announced!", path);
}
- }
- try {
- if (canUpdate) {
- updateAnnouncement(path, bytes);
+ try {
+ if (!Arrays.equals(oldBytes, bytes)) {
+ announcedPaths.put(path, bytes);
+ updateAnnouncement(path, bytes);
+ }
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
}
- }
- catch (Exception e) {
- throw new RuntimeException(e);
}
}
@@ -393,23 +356,58 @@ private void updateAnnouncement(final String path, final byte[] value) throws Ex
public void unannounce(String path)
{
synchronized (toAnnounce) {
- log.info("unannouncing [%s]", path);
final byte[] value = announcedPaths.remove(path);
if (value == null) {
- log.error("Path[%s] not announced, cannot unannounce.", path);
+ log.debug("Path[%s] not announced, cannot unannounce.", path);
return;
}
}
+ log.info("unannouncing [%s]", path);
+
try {
- curator.transaction().forOperations(curator.transactionOp().delete().forPath(path));
+ CuratorOp deleteOp = curator.transactionOp().delete().forPath(path);
+ curator.transaction().forOperations(deleteOp);
}
catch (KeeperException.NoNodeException e) {
log.info("Unannounced node[%s] that does not exist.", path);
}
+ catch (KeeperException.NotEmptyException e) {
+ log.warn("Unannouncing non-empty path[%s]", path);
+ }
catch (Exception e) {
throw new RuntimeException(e);
}
}
+
+ private void startCache(NodeCache cache)
+ {
+ try {
+ cache.start();
+ }
+ catch (Throwable e) {
+ throw CloseableUtils.closeAndWrapInCatch(e, cache);
+ }
+ }
+
+ @GuardedBy("toAnnounce")
+ private void createPath(String parentPath, boolean removeParentsIfCreated)
+ {
+ try {
+ curator.create().creatingParentsIfNeeded().forPath(parentPath);
+ if (removeParentsIfCreated) {
+ // We keep track of all parents we have built, so we can delete them later on when needed.
+ parentsIBuilt.add(parentPath);
+ }
+
+ log.debug("Created parentPath[%s], %s remove on stop.", parentPath, removeParentsIfCreated ? "will" : "will not");
+ }
+ catch (KeeperException.NodeExistsException e) {
+ log.info(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
+ }
+ catch (Exception e) {
+ log.error(e, "Unhandled exception when creating parentPath[%s].", parentPath);
+ }
+ }
}
From 6e262f8c15212042acd644fa0973f84830e91123 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 2 Dec 2024 16:33:13 +0800
Subject: [PATCH 22/43] Fix Checkstyle
---
.../org/apache/druid/curator/announcement/AnnouncerTest.java | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
index d78d20a62b03..21f06650056f 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
@@ -264,7 +264,8 @@ private void awaitAnnounce(
latch.await();
}
- private CountDownLatch createCountdownLatchForPaths(String ...path) {
+ private CountDownLatch createCountdownLatchForPaths(String... path)
+ {
final CountDownLatch latch = new CountDownLatch(path.length);
curator.getCuratorListenable().addListener(
new CuratorListener()
From b41241518ff3be117262228461e916037f3cbe13 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 2 Dec 2024 18:33:34 +0800
Subject: [PATCH 23/43] Add debug logs for update
---
.../org/apache/druid/curator/announcement/NodeAnnouncer.java | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index c678eaa5002b..5936b739af6f 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -268,7 +268,7 @@ private NodeCache setupNodeCache(String path)
ChildData currentData = cache.getCurrentData();
if (currentData == null) {
- // If currentData is null, and we record having announced the data,
+ // If currentData is null, and we know we have already announced the data,
// this means that the ephemeral node was unexpectedly removed.
// We will recreate the node again using the previous data.
final byte[] previouslyAnnouncedData = announcedPaths.get(path);
@@ -312,6 +312,7 @@ public void update(final String path, final byte[] bytes)
{
synchronized (toAnnounce) {
if (!started) {
+ log.debug("NodeAnnouncer has not started yet, queuing updates for later processing...");
// removeParentsIfCreated is not relevant for updates; use dummy value "false".
toUpdate.add(new Announceable(path, bytes, false));
return;
From 6c9bb94e1d2f9ef22906024468af372d42b0bbff Mon Sep 17 00:00:00 2001
From: GWphua
Date: Tue, 3 Dec 2024 10:28:54 +0800
Subject: [PATCH 24/43] Replace CloseableExecutorService within NodeCaches with
ExecutorService
---
.../apache/druid/curator/announcement/NodeAnnouncer.java | 9 ++++-----
1 file changed, 4 insertions(+), 5 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 5936b739af6f..d5fee329970d 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -26,7 +26,6 @@
import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.NodeCache;
-import org.apache.curator.utils.CloseableExecutorService;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
@@ -64,7 +63,7 @@ public class NodeAnnouncer
private static final Logger log = new Logger(NodeAnnouncer.class);
private final CuratorFramework curator;
- private final CloseableExecutorService nodeCacheExecutor;
+ private final ExecutorService nodeCacheExecutor;
private final ConcurrentMap listeners = new ConcurrentHashMap<>();
private final ConcurrentHashMap announcedPaths = new ConcurrentHashMap<>();
@@ -99,7 +98,7 @@ public class NodeAnnouncer
public NodeAnnouncer(CuratorFramework curator, ExecutorService exec)
{
this.curator = curator;
- this.nodeCacheExecutor = new CloseableExecutorService(exec);
+ this.nodeCacheExecutor = exec;
}
@VisibleForTesting
@@ -151,14 +150,14 @@ public void stop()
private void closeResources()
{
try {
+ // Close all caches...
CloseableUtils.closeAll(listeners.values());
-
}
catch (IOException e) {
throw new RuntimeException(e);
}
finally {
- nodeCacheExecutor.close();
+ nodeCacheExecutor.shutdown();
}
for (String announcementPath : announcedPaths.keySet()) {
From c8636d228d88f20c5f5cdabfef7b7f44e21eb65d Mon Sep 17 00:00:00 2001
From: GWphua
Date: Tue, 3 Dec 2024 14:59:39 +0800
Subject: [PATCH 25/43] Debug logs for NodeAnnouncer starting and stopping
---
.../org/apache/druid/curator/announcement/NodeAnnouncer.java | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index d5fee329970d..1b82f7940a69 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -113,7 +113,7 @@ public void start()
log.debug("Starting NodeAnnouncer");
synchronized (toAnnounce) {
if (started) {
- log.debug("Called start() but NodeAnnouncer have already started.");
+ log.debug("Cannot start NodeAnnouncer that has already started.");
return;
}
@@ -137,7 +137,7 @@ public void stop()
log.debug("Stopping NodeAnnouncer");
synchronized (toAnnounce) {
if (!started) {
- log.debug("Called stop() but NodeAnnouncer have not started.");
+ log.debug("Cannot stop NodeAnnouncer has not started.");
return;
}
From f12e09d0eb73cc6a8c8a8211eb576d3c5665e107 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Tue, 3 Dec 2024 16:33:19 +0800
Subject: [PATCH 26/43] Add debug logs for Announcer.java
---
.../java/org/apache/druid/curator/announcement/Announcer.java | 2 ++
.../org/apache/druid/curator/announcement/NodeAnnouncer.java | 2 +-
2 files changed, 3 insertions(+), 1 deletion(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index 8d19afb28e02..9c5c3d15c358 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -118,6 +118,7 @@ public void start()
log.debug("Starting Announcer.");
synchronized (toAnnounce) {
if (started) {
+ log.debug("Cannot start Announcer that has already started.");
return;
}
@@ -141,6 +142,7 @@ public void stop()
log.debug("Stopping Announcer.");
synchronized (toAnnounce) {
if (!started) {
+ log.debug("Cannot stop Announcer that has not started.");
return;
}
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 1b82f7940a69..e10448afc344 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -137,7 +137,7 @@ public void stop()
log.debug("Stopping NodeAnnouncer");
synchronized (toAnnounce) {
if (!started) {
- log.debug("Cannot stop NodeAnnouncer has not started.");
+ log.debug("Cannot stop NodeAnnouncer that has not started.");
return;
}
From 64bce5ccc378fb377172ec81334ccd41f2bf4a77 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Thu, 27 Feb 2025 15:57:40 +0800
Subject: [PATCH 27/43] Remove ZKPathsUtils
---
.../curator/announcement/NodeAnnouncer.java | 1 -
.../org/apache/druid/server/ZKPathsUtils.java | 35 -------------------
2 files changed, 36 deletions(-)
delete mode 100644 server/src/main/java/org/apache/druid/server/ZKPathsUtils.java
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index e10448afc344..6f1955380fb6 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -31,7 +31,6 @@
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.server.ZKPathsUtils;
import org.apache.druid.utils.CloseableUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
diff --git a/server/src/main/java/org/apache/druid/server/ZKPathsUtils.java b/server/src/main/java/org/apache/druid/server/ZKPathsUtils.java
deleted file mode 100644
index 59a9c8c2aeee..000000000000
--- a/server/src/main/java/org/apache/druid/server/ZKPathsUtils.java
+++ /dev/null
@@ -1,35 +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.druid.server;
-
-import org.apache.curator.utils.ZKPaths;
-
-public class ZKPathsUtils
-{
- public static String getParentPath(String path)
- {
- return ZKPaths.getPathAndNode(path).getPath();
- }
-
- public static String getParentNode(String path)
- {
- return ZKPaths.getPathAndNode(path).getNode();
- }
-}
From 431f7b50c0431fe97e06c93045485096fd949d1e Mon Sep 17 00:00:00 2001
From: GWphua
Date: Thu, 27 Feb 2025 15:58:47 +0800
Subject: [PATCH 28/43] Remove usage of ZKPathUtils in NodeAnnouncer.java
---
.../org/apache/druid/curator/announcement/NodeAnnouncer.java | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 6f1955380fb6..d3c3c78094d9 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -26,6 +26,7 @@
import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.curator.utils.ZKPaths;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
@@ -215,7 +216,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
}
}
- final String parentPath = ZKPathsUtils.getParentPath(path);
+ final String parentPath = ZKPaths.getPathAndNode(path).getPath();
byte[] announcedPayload = announcedPaths.get(path);
// If announcedPayload is null, this means that we have yet to announce this path.
From c58d8a583943a517aef70e56be17c134e94303bf Mon Sep 17 00:00:00 2001
From: GWphua
Date: Thu, 27 Feb 2025 16:01:21 +0800
Subject: [PATCH 29/43] Remove ZKPathUtils usage in test classes
---
.../apache/druid/curator/announcement/AnnouncerTest.java | 7 +++----
.../druid/curator/announcement/NodeAnnouncerTest.java | 7 +++----
2 files changed, 6 insertions(+), 8 deletions(-)
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
index 21f06650056f..0c993f4ab882 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
@@ -30,7 +30,6 @@
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.server.ZKPathsUtils;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.data.Stat;
import org.junit.After;
@@ -184,7 +183,7 @@ public void testRemovesParentIfCreated() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
- final String parent = ZKPathsUtils.getParentPath(testPath);
+ final String parent = ZKPaths.getPathAndNode(testPath).getPath();
announcer.start();
try {
@@ -208,7 +207,7 @@ public void testLeavesBehindParentPathIfAlreadyExists() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test";
- final String parent = ZKPathsUtils.getParentPath(testPath);
+ final String parent = ZKPaths.getPathAndNode(testPath).getPath();
curator.create().forPath(parent);
final Stat initialStat = curator.checkExists().forPath(parent);
@@ -235,7 +234,7 @@ public void testLeavesParentPathsUntouchedWhenInstructed() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test";
- final String parent = ZKPathsUtils.getParentPath(testPath);
+ final String parent = ZKPaths.getPathAndNode(testPath).getPath();
announcer.start();
try {
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
index 4c02b3f1b70e..14badf936081 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -32,7 +32,6 @@
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.server.ZKPathsUtils;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.data.Stat;
import org.junit.After;
@@ -74,7 +73,7 @@ public void testCreateParentPath() throws Exception
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/newParent/testPath";
- final String parentPath = ZKPathsUtils.getParentPath(testPath);
+ final String parent = ZKPaths.getPathAndNode(testPath).getPath();
announcer.start();
Assert.assertNull("Parent path should not exist before announcement", curator.checkExists().forPath(parentPath));
@@ -290,7 +289,7 @@ public void testRemovesParentIfCreated() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
- final String parent = ZKPathsUtils.getParentPath(testPath);
+ final String parent = ZKPaths.getPathAndNode(testPath).getPath();
announcer.start();
try {
@@ -314,7 +313,7 @@ public void testLeavesBehindParentPathIfAlreadyExists() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
- final String parent = ZKPathsUtils.getParentPath(testPath);
+ final String parent = ZKPaths.getPathAndNode(testPath).getPath();
curator.create().forPath(parent);
final Stat initialStat = curator.checkExists().forPath(parent);
From 1a5f663d7a85319492acb10021b43e7fd5e93358 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Thu, 27 Feb 2025 16:08:14 +0800
Subject: [PATCH 30/43] Migrate unit tests away from ZkPathUtils
---
.../org/apache/druid/curator/announcement/AnnouncerTest.java | 1 +
.../apache/druid/curator/announcement/NodeAnnouncerTest.java | 5 +++--
2 files changed, 4 insertions(+), 2 deletions(-)
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
index 0c993f4ab882..8a66919bca6b 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
@@ -26,6 +26,7 @@
import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
import org.apache.curator.test.KillSession;
+import org.apache.curator.utils.ZKPaths;
import org.apache.druid.curator.CuratorTestBase;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
index 14badf936081..a554431a5a34 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -26,6 +26,7 @@
import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
import org.apache.curator.test.KillSession;
+import org.apache.curator.utils.ZKPaths;
import org.apache.druid.curator.CuratorTestBase;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
@@ -73,7 +74,7 @@ public void testCreateParentPath() throws Exception
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/newParent/testPath";
- final String parent = ZKPaths.getPathAndNode(testPath).getPath();
+ final String parentPath = ZKPaths.getPathAndNode(testPath).getPath();
announcer.start();
Assert.assertNull("Parent path should not exist before announcement", curator.checkExists().forPath(parentPath));
@@ -340,7 +341,7 @@ public void testLeavesParentPathsUntouchedWhenInstructed() throws Exception
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
- final String parent = ZKPathsUtils.getParentPath(testPath);
+ final String parent = ZKPaths.getPathAndNode(testPath).getPath();
announcer.start();
try {
From 948d721ba631352ee57496b9d14705f543e55991 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Thu, 27 Feb 2025 16:31:17 +0800
Subject: [PATCH 31/43] Use computeIfAbsent where necessary
---
.../druid/curator/announcement/Announcer.java | 26 ++++++++-----------
.../curator/announcement/NodeAnnouncer.java | 19 +++++++-------
2 files changed, 21 insertions(+), 24 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index 9c5c3d15c358..3daa0794c1df 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -38,7 +38,6 @@
import org.apache.druid.utils.CloseableUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
import java.io.IOException;
import java.util.ArrayList;
@@ -76,8 +75,8 @@ public class Announcer
private final List toAnnounce = new ArrayList<>();
@GuardedBy("toAnnounce")
private final List toUpdate = new ArrayList<>();
- private final ConcurrentMap listeners = new ConcurrentHashMap<>();
- private final ConcurrentMap> announcements = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap listeners = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap> announcements = new ConcurrentHashMap<>();
private final List parentsIBuilt = new CopyOnWriteArrayList<>();
// Used for testing
@@ -118,7 +117,7 @@ public void start()
log.debug("Starting Announcer.");
synchronized (toAnnounce) {
if (started) {
- log.debug("Cannot start Announcer that has already started.");
+ log.debug("Announcer has already been started by another thread, ignoring start request.");
return;
}
@@ -142,7 +141,7 @@ public void stop()
log.debug("Stopping Announcer.");
synchronized (toAnnounce) {
if (!started) {
- log.debug("Cannot stop Announcer that has not started.");
+ log.debug("Announcer has already been stopped by another thread, ignoring stop request.");
return;
}
@@ -158,7 +157,7 @@ public void stop()
pathChildrenCacheExecutor.shutdown();
}
- for (Map.Entry> entry : announcements.entrySet()) {
+ for (Map.Entry> entry : announcements.entrySet()) {
String basePath = entry.getKey();
for (String announcementPath : entry.getValue().keySet()) {
@@ -237,11 +236,8 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
log.warn(e, "Failed to check existence of parent path. Proceeding without creating parent path.");
}
- // I don't have a watcher on this path yet, create a Map and start watching.
- announcements.putIfAbsent(parentPath, new ConcurrentHashMap<>());
-
- // Guaranteed to be non-null, but might be a map put in here by another thread.
- final ConcurrentMap finalSubPaths = announcements.get(parentPath);
+ final ConcurrentHashMap finalSubPaths = announcements.computeIfAbsent(parentPath,
+ key -> new ConcurrentHashMap<>());
// Synchronize to make sure that I only create a listener once.
synchronized (finalSubPaths) {
@@ -387,14 +383,14 @@ public void update(final String path, final byte[] bytes)
}
}
- private String createAnnouncement(final String path, byte[] value) throws Exception
+ private void createAnnouncement(final String path, byte[] value) throws Exception
{
- return curator.create().compressed().withMode(CreateMode.EPHEMERAL).inBackground().forPath(path, value);
+ curator.create().compressed().withMode(CreateMode.EPHEMERAL).inBackground().forPath(path, value);
}
- private Stat updateAnnouncement(final String path, final byte[] value) throws Exception
+ private void updateAnnouncement(final String path, final byte[] value) throws Exception
{
- return curator.setData().compressed().inBackground().forPath(path, value);
+ curator.setData().compressed().inBackground().forPath(path, value);
}
/**
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index d3c3c78094d9..283c6509bafa 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -42,7 +42,6 @@
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutorService;
@@ -65,7 +64,7 @@ public class NodeAnnouncer
private final CuratorFramework curator;
private final ExecutorService nodeCacheExecutor;
- private final ConcurrentMap listeners = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap listeners = new ConcurrentHashMap<>();
private final ConcurrentHashMap announcedPaths = new ConcurrentHashMap<>();
@GuardedBy("toAnnounce")
@@ -113,7 +112,7 @@ public void start()
log.debug("Starting NodeAnnouncer");
synchronized (toAnnounce) {
if (started) {
- log.debug("Cannot start NodeAnnouncer that has already started.");
+ log.debug("NodeAnnouncer has already been started by another thread, ignoring start request.");
return;
}
@@ -137,7 +136,7 @@ public void stop()
log.debug("Stopping NodeAnnouncer");
synchronized (toAnnounce) {
if (!started) {
- log.debug("Cannot stop NodeAnnouncer that has not started.");
+ log.debug("NodeAnnouncer has already been stopped by another thread, ignoring stop request.");
return;
}
@@ -232,6 +231,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
// Synchronize to make sure that I only create a listener once.
synchronized (toAnnounce) {
+
if (!listeners.containsKey(path)) {
final NodeCache cache = setupNodeCache(path);
@@ -269,17 +269,18 @@ private NodeCache setupNodeCache(String path)
if (currentData == null) {
// If currentData is null, and we know we have already announced the data,
// this means that the ephemeral node was unexpectedly removed.
+
// We will recreate the node again using the previous data.
- final byte[] previouslyAnnouncedData = announcedPaths.get(path);
- if (previouslyAnnouncedData != null) {
- log.info("Node[%s] dropped, reinstating.", path);
+ announcedPaths.computeIfPresent(path, (key, data) -> {
+ log.info("Node[%s] dropped, reinstating.", key);
try {
- createAnnouncement(path, previouslyAnnouncedData);
+ createAnnouncement(key, data);
}
catch (Exception e) {
throw new RuntimeException(e);
}
- }
+ return data;
+ });
}
})
);
From f964497cba088eff638bc8a1b812c963bc87bfab Mon Sep 17 00:00:00 2001
From: GWphua
Date: Thu, 27 Feb 2025 16:56:08 +0800
Subject: [PATCH 32/43] Use JUnit5 for new Test class
---
.../druid/curator/announcement/Announcer.java | 3 +-
.../announcement/NodeAnnouncerTest.java | 194 ++++++++----------
2 files changed, 92 insertions(+), 105 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index 3daa0794c1df..0df92d5799f1 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -236,8 +236,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
log.warn(e, "Failed to check existence of parent path. Proceeding without creating parent path.");
}
- final ConcurrentHashMap finalSubPaths = announcements.computeIfAbsent(parentPath,
- key -> new ConcurrentHashMap<>());
+ final ConcurrentHashMap finalSubPaths = announcements.computeIfAbsent(parentPath, key -> new ConcurrentHashMap<>());
// Synchronize to make sure that I only create a listener once.
synchronized (finalSubPaths) {
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
index a554431a5a34..fc8383f7e9da 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -35,25 +35,25 @@
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.data.Stat;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
import java.util.Arrays;
import java.util.Collection;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
-/**
- *
- */
+
+
public class NodeAnnouncerTest extends CuratorTestBase
{
private static final Logger log = new Logger(NodeAnnouncerTest.class);
private ExecutorService exec;
- @Before
+ @BeforeEach
public void setUp() throws Exception
{
setupServerAndCurator();
@@ -62,13 +62,14 @@ public void setUp() throws Exception
curator.blockUntilConnected();
}
- @After
+ @AfterEach
public void tearDown()
{
tearDownServerAndCurator();
}
- @Test(timeout = 60_000L)
+ @Test
+ @Timeout(60_000)
public void testCreateParentPath() throws Exception
{
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
@@ -77,7 +78,7 @@ public void testCreateParentPath() throws Exception
final String parentPath = ZKPaths.getPathAndNode(testPath).getPath();
announcer.start();
- Assert.assertNull("Parent path should not exist before announcement", curator.checkExists().forPath(parentPath));
+ Assertions.assertNull(curator.checkExists().forPath(parentPath), "Parent path should not exist before announcement");
announcer.announce(testPath, billy);
// Wait for the announcement to be processed
@@ -85,12 +86,13 @@ public void testCreateParentPath() throws Exception
Thread.sleep(100);
}
- Assert.assertNotNull("Parent path should be created", curator.checkExists().forPath(parentPath));
- Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+ Assertions.assertNotNull(curator.checkExists().forPath(parentPath), "Parent path should be created");
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
announcer.stop();
}
- @Test(timeout = 60_000L)
+ @Test
+ @Timeout(60_000)
public void testAnnounceSamePathWithDifferentPayloadThrowsIAE() throws Exception
{
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
@@ -103,17 +105,17 @@ public void testAnnounceSamePathWithDifferentPayloadThrowsIAE() throws Exception
while (curator.checkExists().forPath(testPath) == null) {
Thread.sleep(100);
}
- Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
- // Nothing wrong when we announce same path.
+ // Nothing wrong when we announce same payload on the same path.
announcer.announce(testPath, billy);
- // Something wrong when we announce different path.
- Exception exception = Assert.assertThrows(IAE.class, () -> announcer.announce(testPath, tilly));
- Assert.assertEquals(exception.getMessage(), "Cannot reannounce different values under the same path.");
+ // Expect an exception when announcing a different payload
+ IAE exception = Assertions.assertThrows(IAE.class, () -> announcer.announce(testPath, tilly));
+ Assertions.assertEquals("Cannot reannounce different values under the same path.", exception.getMessage());
- // Confirm that the new announcement is invalidated, and we still have payload from previous announcement.
- Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+ // Confirm that the announcement remains unchanged.
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
announcer.stop();
}
@@ -125,12 +127,13 @@ public void testUpdateBeforeStartingNodeAnnouncer() throws Exception
final byte[] tilly = StringUtils.toUtf8("tilly");
final String testPath = "/testAnnounce";
+ // Queue update before the announcer is started
announcer.update(testPath, tilly);
announcer.announce(testPath, billy);
announcer.start();
- // Verify that the path was announced
- Assert.assertArrayEquals(tilly, curator.getData().decompressed().forPath(testPath));
+ // Verify that the update took precedence
+ Assertions.assertArrayEquals(tilly, curator.getData().decompressed().forPath(testPath));
announcer.stop();
}
@@ -144,13 +147,15 @@ public void testUpdateSuccessfully() throws Exception
announcer.start();
announcer.announce(testPath, billy);
- Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+ // Update with the same payload: nothing should change.
announcer.update(testPath, billy);
- Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath));
+ // Update with a new payload.
announcer.update(testPath, tilly);
- Assert.assertArrayEquals(tilly, curator.getData().decompressed().forPath(testPath));
+ Assertions.assertArrayEquals(tilly, curator.getData().decompressed().forPath(testPath));
announcer.stop();
}
@@ -163,12 +168,13 @@ public void testUpdateNonExistentPath()
announcer.start();
- Exception exception = Assert.assertThrows(ISE.class, () -> announcer.update(testPath, billy));
- Assert.assertEquals(exception.getMessage(), "Cannot update path[/testUpdate] that hasn't been announced!");
+ ISE exception = Assertions.assertThrows(ISE.class, () -> announcer.update(testPath, billy));
+ Assertions.assertEquals("Cannot update path[/testUpdate] that hasn't been announced!", exception.getMessage());
announcer.stop();
}
- @Test(timeout = 60_000L)
+ @Test
+ @Timeout(60_000)
public void testSanity() throws Exception
{
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
@@ -178,8 +184,8 @@ public void testSanity() throws Exception
final String testPath2 = "/somewhere/test2";
announcer.announce(testPath1, billy);
- Assert.assertNull("/test1 does not exists", curator.checkExists().forPath(testPath1));
- Assert.assertNull("/somewhere/test2 does not exists", curator.checkExists().forPath(testPath2));
+ Assertions.assertNull(curator.checkExists().forPath(testPath1), "/test1 does not exist before announcer start");
+ Assertions.assertNull(curator.checkExists().forPath(testPath2), "/somewhere/test2 does not exist before announcer start");
announcer.start();
while (!announcer.getAddedPaths().contains("/test1")) {
@@ -187,62 +193,45 @@ public void testSanity() throws Exception
}
try {
- Assert.assertArrayEquals("/test1 has data", billy, curator.getData().decompressed().forPath(testPath1));
- Assert.assertNull("/somewhere/test2 still does not exist", curator.checkExists().forPath(testPath2));
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1), "/test1 has data");
+ Assertions.assertNull(curator.checkExists().forPath(testPath2), "/somewhere/test2 still does not exist");
announcer.announce(testPath2, billy);
- Assert.assertArrayEquals("/test1 still has data", billy, curator.getData().decompressed().forPath(testPath1));
- Assert.assertArrayEquals(
- "/somewhere/test2 has data",
- billy,
- curator.getData().decompressed().forPath(testPath2)
- );
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1), "/test1 still has data");
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2), "/somewhere/test2 has data");
final CountDownLatch latch = new CountDownLatch(1);
- curator.getCuratorListenable().addListener(
- (client, event) -> {
- if (event.getType() == CuratorEventType.CREATE && event.getPath().equals(testPath1)) {
- latch.countDown();
- }
- }
- );
+ curator.getCuratorListenable().addListener((client, event) -> {
+ if (event.getType() == CuratorEventType.CREATE && event.getPath().equals(testPath1)) {
+ latch.countDown();
+ }
+ });
final CuratorOp deleteOp = curator.transactionOp().delete().forPath(testPath1);
final Collection results = curator.transaction().forOperations(deleteOp);
- Assert.assertEquals(1, results.size());
+ Assertions.assertEquals(1, results.size(), "Expected one result from the delete op");
final CuratorTransactionResult result = results.iterator().next();
- Assert.assertEquals(Code.OK.intValue(), result.getError()); // assert delete
+ Assertions.assertEquals(Code.OK.intValue(), result.getError(), "Expected OK code on delete");
- Assert.assertTrue("Wait for /test1 to be created", timing.forWaiting().awaitLatch(latch));
+ Assertions.assertTrue(timing.forWaiting().awaitLatch(latch), "Wait for /test1 to be recreated");
- Assert.assertArrayEquals(
- "expect /test1 data is restored",
- billy,
- curator.getData().decompressed().forPath(testPath1)
- );
- Assert.assertArrayEquals(
- "expect /somewhere/test2 is still there",
- billy,
- curator.getData().decompressed().forPath(testPath2)
- );
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1), "Expected /test1 data to be restored");
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2), "Expected /somewhere/test2 data to remain");
announcer.unannounce(testPath1);
- Assert.assertNull("expect /test1 unannounced", curator.checkExists().forPath(testPath1));
- Assert.assertArrayEquals(
- "expect /somewhere/test2 is still still there",
- billy,
- curator.getData().decompressed().forPath(testPath2)
- );
+ Assertions.assertNull(curator.checkExists().forPath(testPath1), "Expected /test1 to be unannounced");
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2), "Expected /somewhere/test2 to remain");
}
finally {
announcer.stop();
}
- Assert.assertNull("expect /test1 remains unannounced", curator.checkExists().forPath(testPath1));
- Assert.assertNull("expect /somewhere/test2 unannounced", curator.checkExists().forPath(testPath2));
+ Assertions.assertNull(curator.checkExists().forPath(testPath1), "Expected /test1 to remain unannounced");
+ Assertions.assertNull(curator.checkExists().forPath(testPath2), "Expected /somewhere/test2 to be unannounced");
}
- @Test(timeout = 60_000L)
+ @Test
+ @Timeout(60_000)
public void testSessionKilled() throws Exception
{
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
@@ -258,25 +247,26 @@ public void testSessionKilled() throws Exception
announcer.announce(testPath1, billy);
announcer.announce(testPath2, billy);
- Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
- Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
final CountDownLatch latch = createCountdownLatchForPaths(paths);
KillSession.kill(curator.getZookeeperClient().getZooKeeper(), server.getConnectString());
- Assert.assertTrue(timing.forWaiting().awaitLatch(latch));
+ Assertions.assertTrue(timing.forWaiting().awaitLatch(latch), "Await latch after killing session");
- Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
- Assert.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath1));
+ Assertions.assertArrayEquals(billy, curator.getData().decompressed().forPath(testPath2));
announcer.stop();
- while ((curator.checkExists().forPath(testPath1) != null) || (curator.checkExists().forPath(testPath2) != null)) {
+ while ((curator.checkExists().forPath(testPath1) != null) ||
+ (curator.checkExists().forPath(testPath2) != null)) {
Thread.sleep(100);
}
- Assert.assertNull(curator.checkExists().forPath(testPath1));
- Assert.assertNull(curator.checkExists().forPath(testPath2));
+ Assertions.assertNull(curator.checkExists().forPath(testPath1));
+ Assertions.assertNull(curator.checkExists().forPath(testPath2));
}
finally {
announcer.stop();
@@ -294,17 +284,17 @@ public void testRemovesParentIfCreated() throws Exception
announcer.start();
try {
- Assert.assertNull(curator.checkExists().forPath(parent));
+ Assertions.assertNull(curator.checkExists().forPath(parent));
awaitAnnounce(announcer, testPath, billy, true);
- Assert.assertNotNull(curator.checkExists().forPath(parent));
+ Assertions.assertNotNull(curator.checkExists().forPath(parent));
}
finally {
announcer.stop();
}
- Assert.assertNull(curator.checkExists().forPath(parent));
+ Assertions.assertNull(curator.checkExists().forPath(parent));
}
@Test
@@ -321,17 +311,17 @@ public void testLeavesBehindParentPathIfAlreadyExists() throws Exception
announcer.start();
try {
- Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
+ Assertions.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
awaitAnnounce(announcer, testPath, billy, true);
- Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
+ Assertions.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
}
finally {
announcer.stop();
}
- Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
+ Assertions.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid());
}
@Test
@@ -345,24 +335,24 @@ public void testLeavesParentPathsUntouchedWhenInstructed() throws Exception
announcer.start();
try {
- Assert.assertNull(curator.checkExists().forPath(parent));
+ Assertions.assertNull(curator.checkExists().forPath(parent));
awaitAnnounce(announcer, testPath, billy, false);
- Assert.assertNotNull(curator.checkExists().forPath(parent));
+ Assertions.assertNotNull(curator.checkExists().forPath(parent));
}
finally {
announcer.stop();
}
- Assert.assertNotNull(curator.checkExists().forPath(parent));
+ Assertions.assertNotNull(curator.checkExists().forPath(parent));
}
private void awaitAnnounce(
- final NodeAnnouncer announcer,
- final String path,
- final byte[] bytes,
- boolean removeParentsIfCreated
+ final NodeAnnouncer announcer,
+ final String path,
+ final byte[] bytes,
+ boolean removeParentsIfCreated
) throws InterruptedException
{
final CountDownLatch latch = createCountdownLatchForPaths(path);
@@ -370,22 +360,20 @@ private void awaitAnnounce(
latch.await();
}
- private CountDownLatch createCountdownLatchForPaths(String... path)
+ private CountDownLatch createCountdownLatchForPaths(String... paths)
{
- final CountDownLatch latch = new CountDownLatch(path.length);
- curator.getCuratorListenable().addListener(
- new CuratorListener()
- {
- @Override
- public void eventReceived(CuratorFramework client, CuratorEvent event)
- {
- if (event.getType() == CuratorEventType.CREATE && Arrays.asList(path).contains(event.getPath())) {
- latch.countDown();
- }
- }
+ final CountDownLatch latch = new CountDownLatch(paths.length);
+ curator.getCuratorListenable().addListener(new CuratorListener()
+ {
+ @Override
+ public void eventReceived(CuratorFramework client, CuratorEvent event)
+ {
+ if (event.getType() == CuratorEventType.CREATE && Arrays.asList(paths).contains(event.getPath())) {
+ latch.countDown();
}
- );
+ }
+ });
return latch;
}
-}
+}
\ No newline at end of file
From 6f6b12d6c72290f90b500b575f06355404598831 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Thu, 27 Feb 2025 17:08:22 +0800
Subject: [PATCH 33/43] Fix checkstyle
---
.../druid/curator/announcement/NodeAnnouncerTest.java | 6 ++----
1 file changed, 2 insertions(+), 4 deletions(-)
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
index fc8383f7e9da..532512da12d8 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -35,8 +35,8 @@
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.data.Stat;
-import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
@@ -46,8 +46,6 @@
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
-
-
public class NodeAnnouncerTest extends CuratorTestBase
{
private static final Logger log = new Logger(NodeAnnouncerTest.class);
@@ -376,4 +374,4 @@ public void eventReceived(CuratorFramework client, CuratorEvent event)
return latch;
}
-}
\ No newline at end of file
+}
From d27e27e4d24087ea2c20e7037011f90deea04d55 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Fri, 28 Feb 2025 15:06:37 +0800
Subject: [PATCH 34/43] Revert NodeAnnouncer changes
---
.../druid/curator/announcement/NodeAnnouncer.java | 15 +++++++--------
1 file changed, 7 insertions(+), 8 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 283c6509bafa..578ee5acf575 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -1,4 +1,5 @@
-/*
+/Users/SP13272/code/druid/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+ /*
* 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
@@ -231,7 +232,6 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
// Synchronize to make sure that I only create a listener once.
synchronized (toAnnounce) {
-
if (!listeners.containsKey(path)) {
final NodeCache cache = setupNodeCache(path);
@@ -269,18 +269,17 @@ private NodeCache setupNodeCache(String path)
if (currentData == null) {
// If currentData is null, and we know we have already announced the data,
// this means that the ephemeral node was unexpectedly removed.
-
// We will recreate the node again using the previous data.
- announcedPaths.computeIfPresent(path, (key, data) -> {
- log.info("Node[%s] dropped, reinstating.", key);
+ final byte[] previouslyAnnouncedData = announcedPaths.get(path);
+ if (previouslyAnnouncedData != null) {
+ log.info("Node[%s] dropped, reinstating.", path);
try {
- createAnnouncement(key, data);
+ createAnnouncement(path, previouslyAnnouncedData);
}
catch (Exception e) {
throw new RuntimeException(e);
}
- return data;
- });
+ }
}
})
);
From 96481553a6e91217d0321dd23f8acdf07da397c7 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Fri, 28 Feb 2025 15:18:02 +0800
Subject: [PATCH 35/43] Fix compile
---
.../org/apache/druid/curator/announcement/NodeAnnouncer.java | 3 +--
1 file changed, 1 insertion(+), 2 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 578ee5acf575..b34061865c21 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -1,5 +1,4 @@
-/Users/SP13272/code/druid/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
- /*
+/*
* 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
From a0513b988a4a7eda3feee46ce9bea1c5ae2f48f8 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Tue, 11 Mar 2025 11:53:03 +0800
Subject: [PATCH 36/43] Upgrade to CuratorCache
---
.../curator/announcement/Announceable.java | 7 ++++
.../druid/curator/announcement/Announcer.java | 3 +-
.../curator/announcement/NodeAnnouncer.java | 34 ++++++++-----------
.../announcement/NodeAnnouncerTest.java | 18 +++-------
4 files changed, 27 insertions(+), 35 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java b/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
index c2ad7c588163..cecdb8dbcd53 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announceable.java
@@ -46,4 +46,11 @@ public Announceable(String path, byte[] bytes, boolean removeParentsIfCreated)
this.bytes = bytes;
this.removeParentsIfCreated = removeParentsIfCreated;
}
+
+ // This should be used for updates only, where removeParentsIfCreated is not relevant.
+ public Announceable(String path, byte[] bytes)
+ {
+ // removeParentsIfCreated is irrelevant, so we can use dummy value "false".
+ this(path, bytes, false);
+ }
}
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index 0df92d5799f1..8e5382fec8b6 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -350,8 +350,7 @@ public void update(final String path, final byte[] bytes)
{
synchronized (toAnnounce) {
if (!started) {
- // removeParentsIfCreated is not relevant for updates; use dummy value "false".
- toUpdate.add(new Announceable(path, bytes, false));
+ toUpdate.add(new Announceable(path, bytes));
return;
}
}
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index b34061865c21..04e77bf0cd71 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -24,8 +24,8 @@
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.transaction.CuratorMultiTransaction;
import org.apache.curator.framework.api.transaction.CuratorOp;
-import org.apache.curator.framework.recipes.cache.ChildData;
-import org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.curator.framework.recipes.cache.CuratorCache;
+import org.apache.curator.framework.recipes.cache.CuratorCacheListener;
import org.apache.curator.utils.ZKPaths;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
@@ -64,7 +64,7 @@ public class NodeAnnouncer
private final CuratorFramework curator;
private final ExecutorService nodeCacheExecutor;
- private final ConcurrentHashMap listeners = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap listeners = new ConcurrentHashMap<>();
private final ConcurrentHashMap announcedPaths = new ConcurrentHashMap<>();
@GuardedBy("toAnnounce")
@@ -232,7 +232,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
// Synchronize to make sure that I only create a listener once.
synchronized (toAnnounce) {
if (!listeners.containsKey(path)) {
- final NodeCache cache = setupNodeCache(path);
+ final CuratorCache cache = createCacheForPath(path);
if (started) {
if (buildParentPath) {
@@ -258,21 +258,17 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
}
@GuardedBy("toAnnounce")
- private NodeCache setupNodeCache(String path)
+ private CuratorCache createCacheForPath(String path)
{
- final NodeCache cache = new NodeCache(curator, path, true);
- cache.getListenable().addListener(
- () -> nodeCacheExecutor.submit(() -> {
- ChildData currentData = cache.getCurrentData();
-
- if (currentData == null) {
- // If currentData is null, and we know we have already announced the data,
- // this means that the ephemeral node was unexpectedly removed.
- // We will recreate the node again using the previous data.
+ final CuratorCache cache = CuratorCache.build(curator, path, CuratorCache.Options.SINGLE_NODE_CACHE);
+
+ cache.listenable().addListener(
+ (type, oldData, data) -> {
+ if (type == CuratorCacheListener.Type.NODE_DELETED) {
final byte[] previouslyAnnouncedData = announcedPaths.get(path);
if (previouslyAnnouncedData != null) {
- log.info("Node[%s] dropped, reinstating.", path);
try {
+ log.info("ZooKeeper Node[%s] dropped, reinstating...", path);
createAnnouncement(path, previouslyAnnouncedData);
}
catch (Exception e) {
@@ -280,8 +276,9 @@ private NodeCache setupNodeCache(String path)
}
}
}
- })
+ }, nodeCacheExecutor
);
+
return cache;
}
@@ -311,8 +308,7 @@ public void update(final String path, final byte[] bytes)
synchronized (toAnnounce) {
if (!started) {
log.debug("NodeAnnouncer has not started yet, queuing updates for later processing...");
- // removeParentsIfCreated is not relevant for updates; use dummy value "false".
- toUpdate.add(new Announceable(path, bytes, false));
+ toUpdate.add(new Announceable(path, bytes));
return;
}
@@ -380,7 +376,7 @@ public void unannounce(String path)
}
}
- private void startCache(NodeCache cache)
+ private void startCache(CuratorCache cache)
{
try {
cache.start();
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
index 532512da12d8..6841f87b3142 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/NodeAnnouncerTest.java
@@ -19,10 +19,7 @@
package org.apache.druid.curator.announcement;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.api.CuratorEvent;
import org.apache.curator.framework.api.CuratorEventType;
-import org.apache.curator.framework.api.CuratorListener;
import org.apache.curator.framework.api.transaction.CuratorOp;
import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
import org.apache.curator.test.KillSession;
@@ -32,7 +29,6 @@
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
-import org.apache.druid.java.util.common.logger.Logger;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.data.Stat;
import org.junit.jupiter.api.AfterEach;
@@ -48,7 +44,6 @@
public class NodeAnnouncerTest extends CuratorTestBase
{
- private static final Logger log = new Logger(NodeAnnouncerTest.class);
private ExecutorService exec;
@BeforeEach
@@ -277,7 +272,7 @@ public void testRemovesParentIfCreated() throws Exception
NodeAnnouncer announcer = new NodeAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
- final String testPath = "/somewhere/test2";
+ final String testPath = "/somewhere/test";
final String parent = ZKPaths.getPathAndNode(testPath).getPath();
announcer.start();
@@ -361,14 +356,9 @@ private void awaitAnnounce(
private CountDownLatch createCountdownLatchForPaths(String... paths)
{
final CountDownLatch latch = new CountDownLatch(paths.length);
- curator.getCuratorListenable().addListener(new CuratorListener()
- {
- @Override
- public void eventReceived(CuratorFramework client, CuratorEvent event)
- {
- if (event.getType() == CuratorEventType.CREATE && Arrays.asList(paths).contains(event.getPath())) {
- latch.countDown();
- }
+ curator.getCuratorListenable().addListener((client, event) -> {
+ if (event.getType() == CuratorEventType.CREATE && Arrays.asList(paths).contains(event.getPath())) {
+ latch.countDown();
}
});
From c3d89fcf0d344795c75bacb36853ca22d8e81f55 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 17 Mar 2025 15:57:02 +0800
Subject: [PATCH 37/43] Revert logging of exceptions
---
.../java/org/apache/druid/curator/announcement/Announcer.java | 2 +-
.../org/apache/druid/curator/announcement/NodeAnnouncer.java | 2 +-
2 files changed, 2 insertions(+), 2 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index 8e5382fec8b6..3bae2f1b2503 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -233,7 +233,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
}
}
catch (Exception e) {
- log.warn(e, "Failed to check existence of parent path. Proceeding without creating parent path.");
+ log.debug(e, "Problem checking if the parent existed, ignoring.");
}
final ConcurrentHashMap finalSubPaths = announcements.computeIfAbsent(parentPath, key -> new ConcurrentHashMap<>());
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 04e77bf0cd71..ef8eeaef45ba 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -226,7 +226,7 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
buildParentPath = curator.checkExists().forPath(parentPath) == null;
}
catch (Exception e) {
- log.warn(e, "Failed to check existence of parent path. Proceeding without creating parent path.");
+ log.debug(e, "Problem checking if the parent existed, ignoring.");
}
// Synchronize to make sure that I only create a listener once.
From 384849caa910a66728cccde12ba7254b847f5a73 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 17 Mar 2025 16:43:38 +0800
Subject: [PATCH 38/43] Changes to avoid race conditions when announcing
---
.../curator/announcement/NodeAnnouncer.java | 36 +++++++------------
1 file changed, 13 insertions(+), 23 deletions(-)
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index ef8eeaef45ba..935eb9217d9a 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -245,9 +245,20 @@ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
}
}
- final boolean readyToCreateAnnouncement = updateAnnouncedPaths(path, bytes);
+ boolean created = false;
+ synchronized (toAnnounce) {
+ if (started) {
+ byte[] oldBytes = announcedPaths.putIfAbsent(path, bytes);
- if (readyToCreateAnnouncement) {
+ if (oldBytes == null) {
+ created = true;
+ } else if (!Arrays.equals(oldBytes, bytes)) {
+ throw new IAE("Cannot reannounce different values under the same path.");
+ }
+ }
+ }
+
+ if (created) {
try {
createAnnouncement(path, bytes);
}
@@ -282,27 +293,6 @@ private CuratorCache createCacheForPath(String path)
return cache;
}
- private boolean updateAnnouncedPaths(String path, byte[] bytes)
- {
- synchronized (toAnnounce) {
- if (!started) {
- return false; // Do nothing if not started
- }
- }
-
- final byte[] updatedAnnouncementData = announcedPaths.compute(path, (key, oldBytes) -> {
- if (oldBytes == null) {
- return bytes; // Insert the new value
- } else if (!Arrays.equals(oldBytes, bytes)) {
- throw new IAE("Cannot reannounce different values under the same path.");
- }
- return oldBytes; // No change if values are equal
- });
-
- // Return true if we have updated the paths.
- return Arrays.equals(updatedAnnouncementData, bytes);
- }
-
public void update(final String path, final byte[] bytes)
{
synchronized (toAnnounce) {
From 7a9e09fc76a884a5dc292df9c7643c977b319e11 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 17 Mar 2025 17:29:07 +0800
Subject: [PATCH 39/43] Feature flag for announcers
---
docs/api-reference/tasks-api.md | 4 +-
docs/configuration/index.md | 1 +
.../apache/druid/curator/CuratorConfig.java | 8 +
.../druid/curator/announcement/Announcer.java | 435 +----------------
.../curator/announcement/NodeAnnouncer.java | 26 +-
.../announcement/PathChildrenAnnouncer.java | 462 ++++++++++++++++++
.../apache/druid/guice/AnnouncerModule.java | 18 +-
...st.java => PathChildrenAnnouncerTest.java} | 16 +-
8 files changed, 513 insertions(+), 457 deletions(-)
create mode 100644 server/src/main/java/org/apache/druid/curator/announcement/PathChildrenAnnouncer.java
rename server/src/test/java/org/apache/druid/curator/announcement/{AnnouncerTest.java => PathChildrenAnnouncerTest.java} (93%)
diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md
index d94be5b0c5fd..69a0a0153619 100644
--- a/docs/api-reference/tasks-api.md
+++ b/docs/api-reference/tasks-api.md
@@ -1059,9 +1059,9 @@ Host: http://ROUTER_IP:ROUTER_PORT
2023-07-03T22:11:17,933 INFO [task-runner-0-priority-0] org.apache.kafka.common.metrics.Metrics - Closing reporter org.apache.kafka.common.metrics.JmxReporter
2023-07-03T22:11:17,933 INFO [task-runner-0-priority-0] org.apache.kafka.common.metrics.Metrics - Metrics reporters closed
2023-07-03T22:11:17,935 INFO [task-runner-0-priority-0] org.apache.kafka.common.utils.AppInfoParser - App info kafka.consumer for consumer-kafka-supervisor-dcanhmig-1 unregistered
- 2023-07-03T22:11:17,936 INFO [task-runner-0-priority-0] org.apache.druid.curator.announcement.Announcer - Unannouncing [/druid/internal-discovery/PEON/localhost:8100]
+ 2023-07-03T22:11:17,936 INFO [task-runner-0-priority-0] org.apache.druid.curator.announcement.PathChildrenAnnouncer - Unannouncing [/druid/internal-discovery/PEON/localhost:8100]
2023-07-03T22:11:17,972 INFO [task-runner-0-priority-0] org.apache.druid.curator.discovery.CuratorDruidNodeAnnouncer - Unannounced self [{"druidNode":{"service":"druid/middleManager","host":"localhost","bindOnHost":false,"plaintextPort":8100,"port":-1,"tlsPort":-1,"enablePlaintextPort":true,"enableTlsPort":false},"nodeType":"peon","services":{"dataNodeService":{"type":"dataNodeService","tier":"_default_tier","maxSize":0,"type":"indexer-executor","serverType":"indexer-executor","priority":0},"lookupNodeService":{"type":"lookupNodeService","lookupTier":"__default"}}}].
- 2023-07-03T22:11:17,972 INFO [task-runner-0-priority-0] org.apache.druid.curator.announcement.Announcer - Unannouncing [/druid/announcements/localhost:8100]
+ 2023-07-03T22:11:17,972 INFO [task-runner-0-priority-0] org.apache.druid.curator.announcement.PathChildrenAnnouncer - Unannouncing [/druid/announcements/localhost:8100]
2023-07-03T22:11:17,996 INFO [task-runner-0-priority-0] org.apache.druid.indexing.worker.executor.ExecutorLifecycle - Task completed with status: {
"id" : "index_kafka_social_media_0e905aa31037879_nommnaeg",
"status" : "SUCCESS",
diff --git a/docs/configuration/index.md b/docs/configuration/index.md
index cd1b8dfc1f9e..ed8612d9d629 100644
--- a/docs/configuration/index.md
+++ b/docs/configuration/index.md
@@ -148,6 +148,7 @@ We recommend just setting the base ZK path and the ZK service host, but all ZK p
|`druid.zk.service.connectionTimeoutMs`|ZooKeeper connection timeout, in milliseconds.|`15000`|
|`druid.zk.service.compress`|Boolean flag for whether or not created Znodes should be compressed.|`true`|
|`druid.zk.service.acl`|Boolean flag for whether or not to enable ACL security for ZooKeeper. If ACL is enabled, zNode creators will have all permissions.|`false`|
+|`druid.zk.service.pathChildrenCacheStrategy`|Dictates the underlying caching strategy for service announcements. Set true to let announcers to use Apache Curator's PathChildrenCache strategy, otherwise NodeCache strategy. Consider using NodeCache strategy when you are dealing with huge number of ZooKeeper watches in your cluster.|`true`|
#### Path configuration
diff --git a/server/src/main/java/org/apache/druid/curator/CuratorConfig.java b/server/src/main/java/org/apache/druid/curator/CuratorConfig.java
index 7a53ee941d7d..e98a457b0f37 100644
--- a/server/src/main/java/org/apache/druid/curator/CuratorConfig.java
+++ b/server/src/main/java/org/apache/druid/curator/CuratorConfig.java
@@ -63,6 +63,9 @@ public class CuratorConfig
@JsonProperty("maxZkRetries")
private int maxZkRetries = 29;
+ @JsonProperty("pathChildrenCacheStrategy")
+ private boolean pathChildrenCacheStrategy = true;
+
public static CuratorConfig create(String hosts)
{
CuratorConfig config = new CuratorConfig();
@@ -141,4 +144,9 @@ public int getMaxZkRetries()
{
return maxZkRetries;
}
+
+ public boolean getPathChildrenCacheStrategy()
+ {
+ return pathChildrenCacheStrategy;
+ }
}
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
index 3bae2f1b2503..8852920071b4 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
@@ -19,438 +19,17 @@
package org.apache.druid.curator.announcement;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.errorprone.annotations.concurrent.GuardedBy;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.api.transaction.CuratorMultiTransaction;
-import org.apache.curator.framework.api.transaction.CuratorOp;
-import org.apache.curator.framework.recipes.cache.ChildData;
-import org.apache.curator.framework.recipes.cache.PathChildrenCache;
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
-import org.apache.curator.utils.ZKPaths;
-import org.apache.druid.curator.cache.PathChildrenCacheFactory;
-import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
-import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.utils.CloseableUtils;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * The {@link Announcer} class manages the announcement of a node, and watches all child
- * and sibling nodes under the specified path in a ZooKeeper ensemble. It monitors these nodes
- * to ensure their existence and manage their lifecycle collectively.
- *
- *
- * This class uses Apache Curator's PathChildrenCache recipe under the hood to track all znodes
- * under the specified node's parent. See {@link NodeAnnouncer} for an announcer that
- * uses the NodeCache recipe instead.
- *
- */
-public class Announcer
+public interface Announcer
{
- private static final Logger log = new Logger(Announcer.class);
-
- private final CuratorFramework curator;
- private final PathChildrenCacheFactory factory;
- private final ExecutorService pathChildrenCacheExecutor;
-
- @GuardedBy("toAnnounce")
- private final List toAnnounce = new ArrayList<>();
- @GuardedBy("toAnnounce")
- private final List toUpdate = new ArrayList<>();
- private final ConcurrentHashMap listeners = new ConcurrentHashMap<>();
- private final ConcurrentHashMap> announcements = new ConcurrentHashMap<>();
- private final List parentsIBuilt = new CopyOnWriteArrayList<>();
-
- // Used for testing
- private Set addedChildren;
-
- private boolean started = false;
-
- public Announcer(
- CuratorFramework curator,
- ExecutorService exec
- )
- {
- this.curator = curator;
- this.pathChildrenCacheExecutor = exec;
- this.factory = new PathChildrenCacheFactory.Builder()
- .withCacheData(false)
- .withCompressed(true)
- .withExecutorService(exec)
- .withShutdownExecutorOnClose(false)
- .build();
- }
-
- @VisibleForTesting
- void initializeAddedChildren()
- {
- addedChildren = new HashSet<>();
- }
-
- @VisibleForTesting
- Set getAddedChildren()
- {
- return addedChildren;
- }
-
- @LifecycleStart
- public void start()
- {
- log.debug("Starting Announcer.");
- synchronized (toAnnounce) {
- if (started) {
- log.debug("Announcer has already been started by another thread, ignoring start request.");
- return;
- }
-
- started = true;
-
- for (Announceable announceable : toAnnounce) {
- announce(announceable.path, announceable.bytes, announceable.removeParentsIfCreated);
- }
- toAnnounce.clear();
-
- for (Announceable announceable : toUpdate) {
- update(announceable.path, announceable.bytes);
- }
- toUpdate.clear();
- }
- }
-
- @LifecycleStop
- public void stop()
- {
- log.debug("Stopping Announcer.");
- synchronized (toAnnounce) {
- if (!started) {
- log.debug("Announcer has already been stopped by another thread, ignoring stop request.");
- return;
- }
-
- started = false;
-
- try {
- CloseableUtils.closeAll(listeners.values());
- }
- catch (IOException e) {
- throw new RuntimeException(e);
- }
- finally {
- pathChildrenCacheExecutor.shutdown();
- }
-
- for (Map.Entry> entry : announcements.entrySet()) {
- String basePath = entry.getKey();
-
- for (String announcementPath : entry.getValue().keySet()) {
- unannounce(ZKPaths.makePath(basePath, announcementPath));
- }
- }
-
- if (!parentsIBuilt.isEmpty()) {
- CuratorMultiTransaction transaction = curator.transaction();
-
- ArrayList operations = new ArrayList<>();
- for (String parent : parentsIBuilt) {
- try {
- operations.add(curator.transactionOp().delete().forPath(parent));
- }
- catch (Exception e) {
- log.info(e, "Unable to delete parent[%s] when closing Announcer.", parent);
- }
- }
-
- try {
- transaction.forOperations(operations);
- }
- catch (Exception e) {
- log.info(e, "Unable to commit transaction when closing Announcer.");
- }
- }
- }
- }
-
- /**
- * Overload of {@link #announce(String, byte[], boolean)}, but removes parent node of path after announcement.
- */
- public void announce(String path, byte[] bytes)
- {
- announce(path, bytes, true);
- }
-
- /**
- * Announces the provided bytes at the given path.
- *
- *
- * Announcement using {@link Announcer} will create an ephemeral znode at the specified path, and uses its parent
- * path to watch all the siblings and children znodes of your specified path. The watched nodes will always exist
- * until it is unannounced, or until {@link #stop()} is called.
- *
- *
- * @param path The path to announce at
- * @param bytes The payload to announce
- * @param removeParentIfCreated remove parent of "path" if we had created that parent during announcement
- */
- public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
- {
- synchronized (toAnnounce) {
- if (!started) {
- log.debug("Announcer has not started yet, queuing announcement for later processing...");
- toAnnounce.add(new Announceable(path, bytes, removeParentIfCreated));
- return;
- }
- }
-
- final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
-
- final String parentPath = pathAndNode.getPath();
- boolean buildParentPath = false;
-
- ConcurrentMap subPaths = announcements.get(parentPath);
-
- if (subPaths == null) {
- try {
- if (curator.checkExists().forPath(parentPath) == null) {
- buildParentPath = true;
- }
- }
- catch (Exception e) {
- log.debug(e, "Problem checking if the parent existed, ignoring.");
- }
-
- final ConcurrentHashMap finalSubPaths = announcements.computeIfAbsent(parentPath, key -> new ConcurrentHashMap<>());
-
- // Synchronize to make sure that I only create a listener once.
- synchronized (finalSubPaths) {
- if (!listeners.containsKey(parentPath)) {
- final PathChildrenCache cache = factory.make(curator, parentPath);
- cache.getListenable().addListener(
- new PathChildrenCacheListener()
- {
- private final AtomicReference> pathsLost = new AtomicReference<>(null);
-
- @Override
- public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
- {
- // NOTE: ZooKeeper does not guarantee that we will get every event, and thus PathChildrenCache doesn't
- // as well. If one of the below events are missed, Announcer might not work properly.
- log.debug("Path[%s] got event[%s]", parentPath, event);
- switch (event.getType()) {
- case CHILD_REMOVED:
- final ChildData child = event.getData();
- final ZKPaths.PathAndNode childPath = ZKPaths.getPathAndNode(child.getPath());
- final byte[] value = finalSubPaths.get(childPath.getNode());
- if (value != null) {
- log.info("Node[%s] dropped, reinstating.", child.getPath());
- createAnnouncement(child.getPath(), value);
- }
- break;
- case CONNECTION_LOST:
- // Lost connection, which means session is broken, take inventory of what has been seen.
- // This is to protect from a race condition in which the ephemeral node could have been
- // created but not actually seen by the PathChildrenCache, which means that it won't know
- // that it disappeared and thus will not generate a CHILD_REMOVED event for us. Under normal
- // circumstances, this can only happen upon connection loss; but technically if you have
- // an adversary in the system, they could also delete the ephemeral node before the cache sees
- // it. This does not protect from that case, so don't have adversaries.
-
- Set pathsToReinstate = new HashSet<>();
- for (String node : finalSubPaths.keySet()) {
- String path = ZKPaths.makePath(parentPath, node);
- log.info("Node[%s] is added to reinstate.", path);
- pathsToReinstate.add(path);
- }
-
- if (!pathsToReinstate.isEmpty() && !pathsLost.compareAndSet(null, pathsToReinstate)) {
- log.info("Already had a pathsLost set!?[%s]", parentPath);
- }
- break;
- case CONNECTION_RECONNECTED:
- final Set thePathsLost = pathsLost.getAndSet(null);
-
- if (thePathsLost != null) {
- for (String path : thePathsLost) {
- log.info("Reinstating [%s]", path);
- final ZKPaths.PathAndNode split = ZKPaths.getPathAndNode(path);
- createAnnouncement(path, announcements.get(split.getPath()).get(split.getNode()));
- }
- }
- break;
- case CHILD_ADDED:
- if (addedChildren != null) {
- addedChildren.add(event.getData().getPath());
- }
- // fall through
- case INITIALIZED:
- case CHILD_UPDATED:
- case CONNECTION_SUSPENDED:
- // do nothing
- }
- }
- }
- );
-
- synchronized (toAnnounce) {
- if (started) {
- if (buildParentPath) {
- createPath(parentPath, removeParentIfCreated);
- }
- startCache(cache);
- listeners.put(parentPath, cache);
- }
- }
- }
- }
-
- subPaths = finalSubPaths;
- }
-
- boolean created = false;
- synchronized (toAnnounce) {
- if (started) {
- byte[] oldBytes = subPaths.putIfAbsent(pathAndNode.getNode(), bytes);
-
- if (oldBytes == null) {
- created = true;
- } else if (!Arrays.equals(oldBytes, bytes)) {
- throw new IAE("Cannot reannounce different values under the same path");
- }
- }
- }
-
- if (created) {
- try {
- createAnnouncement(path, bytes);
- }
- catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
- }
-
- public void update(final String path, final byte[] bytes)
- {
- synchronized (toAnnounce) {
- if (!started) {
- toUpdate.add(new Announceable(path, bytes));
- return;
- }
- }
-
- final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
-
- final String parentPath = pathAndNode.getPath();
- final String nodePath = pathAndNode.getNode();
-
- ConcurrentMap subPaths = announcements.get(parentPath);
-
- if (subPaths == null || subPaths.get(nodePath) == null) {
- throw new ISE("Cannot update path[%s] that hasn't been announced!", path);
- }
-
- synchronized (toAnnounce) {
- try {
- byte[] oldBytes = subPaths.get(nodePath);
-
- if (!Arrays.equals(oldBytes, bytes)) {
- subPaths.put(nodePath, bytes);
- updateAnnouncement(path, bytes);
- }
- }
- catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
- }
-
- private void createAnnouncement(final String path, byte[] value) throws Exception
- {
- curator.create().compressed().withMode(CreateMode.EPHEMERAL).inBackground().forPath(path, value);
- }
-
- private void updateAnnouncement(final String path, final byte[] value) throws Exception
- {
- curator.setData().compressed().inBackground().forPath(path, value);
- }
-
- /**
- * Unannounces an announcement created at path. Note that if all announcements get removed, the Announcer
- * will continue to have ZK watches on paths because clearing them out is a source of ugly race conditions.
- *
- * If you need to completely clear all the state of what is being watched and announced, stop() the Announcer.
- *
- * @param path the path to unannounce
- */
- public void unannounce(String path)
- {
- final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
- final String parentPath = pathAndNode.getPath();
+ void start();
- final ConcurrentMap subPaths = announcements.get(parentPath);
+ void stop();
- if (subPaths == null || subPaths.remove(pathAndNode.getNode()) == null) {
- log.debug("Path[%s] not announced, cannot unannounce.", path);
- return;
- }
- log.info("Unannouncing [%s]", path);
+ void announce(String path, byte[] bytes);
- try {
- CuratorOp deleteOp = curator.transactionOp().delete().forPath(path);
- curator.transaction().forOperations(deleteOp);
- }
- catch (KeeperException.NoNodeException e) {
- log.info("Unannounced node[%s] that does not exist.", path);
- }
- catch (KeeperException.NotEmptyException e) {
- log.warn("Unannouncing non-empty path[%s]", path);
- }
- catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
+ void announce(String path, byte[] bytes, boolean removeParentIfCreated);
- private void startCache(PathChildrenCache cache)
- {
- try {
- cache.start();
- }
- catch (Throwable e) {
- throw CloseableUtils.closeAndWrapInCatch(e, cache);
- }
- }
+ void update(String path, byte[] bytes);
- private void createPath(String parentPath, boolean removeParentsIfCreated)
- {
- try {
- curator.create().creatingParentsIfNeeded().forPath(parentPath);
- if (removeParentsIfCreated) {
- parentsIBuilt.add(parentPath);
- }
- log.debug("Created parentPath[%s], %s remove on stop.", parentPath, removeParentsIfCreated ? "will" : "will not");
- }
- catch (KeeperException.NodeExistsException e) {
- log.info(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
- }
- catch (Exception e) {
- log.error(e, "Unhandled exception when creating parentPath[%s].", parentPath);
- }
- }
+ void unannounce(String path);
}
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index 935eb9217d9a..bff7c36c85aa 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -53,11 +53,11 @@
*
*
* This class uses Apache Curator's NodeCache recipe under the hood to track a single
- * node, along with all of its parent's status. See {@link Announcer} for an announcer that
+ * node, along with all of its parent's status. See {@link PathChildrenAnnouncer} for an announcer that
* uses the PathChildrenCache recipe instead.
*
*/
-public class NodeAnnouncer
+public class NodeAnnouncer implements Announcer
{
private static final Logger log = new Logger(NodeAnnouncer.class);
@@ -107,12 +107,13 @@ Set getAddedPaths()
}
@LifecycleStart
+ @Override
public void start()
{
- log.debug("Starting NodeAnnouncer");
+ log.debug("Starting Announcer");
synchronized (toAnnounce) {
if (started) {
- log.debug("NodeAnnouncer has already been started by another thread, ignoring start request.");
+ log.debug("Announcer has already been started by another thread, ignoring start request.");
return;
}
@@ -131,12 +132,13 @@ public void start()
}
@LifecycleStop
+ @Override
public void stop()
{
- log.debug("Stopping NodeAnnouncer");
+ log.debug("Stopping Announcer");
synchronized (toAnnounce) {
if (!started) {
- log.debug("NodeAnnouncer has already been stopped by another thread, ignoring stop request.");
+ log.debug("Announcer has already been stopped by another thread, ignoring stop request.");
return;
}
@@ -172,7 +174,7 @@ private void closeResources()
operations.add(curator.transactionOp().delete().forPath(parent));
}
catch (Exception e) {
- log.info(e, "Unable to delete parent[%s] when closing NodeAnnouncer.", parent);
+ log.info(e, "Unable to delete parent[%s] when closing Announcer.", parent);
}
}
@@ -180,7 +182,7 @@ private void closeResources()
transaction.forOperations(operations);
}
catch (Exception e) {
- log.info(e, "Unable to commit transaction when closing NodeAnnouncer.");
+ log.info(e, "Unable to commit transaction when closing Announcer.");
}
}
}
@@ -188,6 +190,7 @@ private void closeResources()
/**
* Overload of {@link #announce(String, byte[], boolean)}, but removes parent node of path after announcement.
*/
+ @Override
public void announce(String path, byte[] bytes)
{
announce(path, bytes, true);
@@ -205,11 +208,12 @@ public void announce(String path, byte[] bytes)
* @param bytes The payload to announce
* @param removeParentIfCreated remove parent of "path" if we had created that parent during announcement
*/
+ @Override
public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
{
synchronized (toAnnounce) {
if (!started) {
- log.debug("NodeAnnouncer has not started yet, queuing announcement for later processing...");
+ log.debug("Announcer has not started yet, queuing announcement for later processing...");
toAnnounce.add(new Announceable(path, bytes, removeParentIfCreated));
return;
}
@@ -293,11 +297,12 @@ private CuratorCache createCacheForPath(String path)
return cache;
}
+ @Override
public void update(final String path, final byte[] bytes)
{
synchronized (toAnnounce) {
if (!started) {
- log.debug("NodeAnnouncer has not started yet, queuing updates for later processing...");
+ log.debug("Announcer has not started yet, queuing updates for later processing...");
toUpdate.add(new Announceable(path, bytes));
return;
}
@@ -338,6 +343,7 @@ private void updateAnnouncement(final String path, final byte[] value) throws Ex
*
* @param path the path to unannounce
*/
+ @Override
public void unannounce(String path)
{
synchronized (toAnnounce) {
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/PathChildrenAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/PathChildrenAnnouncer.java
new file mode 100644
index 000000000000..9ce965551114
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/curator/announcement/PathChildrenAnnouncer.java
@@ -0,0 +1,462 @@
+/*
+ * 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.druid.curator.announcement;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.transaction.CuratorMultiTransaction;
+import org.apache.curator.framework.api.transaction.CuratorOp;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.PathChildrenCache;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
+import org.apache.curator.utils.ZKPaths;
+import org.apache.druid.curator.cache.PathChildrenCacheFactory;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.utils.CloseableUtils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * The {@link PathChildrenAnnouncer} class manages the announcement of a node, and watches all child
+ * and sibling nodes under the specified path in a ZooKeeper ensemble. It monitors these nodes
+ * to ensure their existence and manage their lifecycle collectively.
+ *
+ *
+ * This class uses Apache Curator's PathChildrenCache recipe under the hood to track all znodes
+ * under the specified node's parent. See {@link NodeAnnouncer} for an announcer that
+ * uses the NodeCache recipe instead.
+ *
+ */
+public class PathChildrenAnnouncer implements Announcer
+{
+ private static final Logger log = new Logger(PathChildrenAnnouncer.class);
+
+ private final CuratorFramework curator;
+ private final PathChildrenCacheFactory factory;
+ private final ExecutorService pathChildrenCacheExecutor;
+
+ @GuardedBy("toAnnounce")
+ private final List toAnnounce = new ArrayList<>();
+ @GuardedBy("toAnnounce")
+ private final List toUpdate = new ArrayList<>();
+ private final ConcurrentHashMap listeners = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap> announcements = new ConcurrentHashMap<>();
+ private final List parentsIBuilt = new CopyOnWriteArrayList<>();
+
+ // Used for testing
+ private Set addedChildren;
+
+ private boolean started = false;
+
+ public PathChildrenAnnouncer(
+ CuratorFramework curator,
+ ExecutorService exec
+ )
+ {
+ this.curator = curator;
+ this.pathChildrenCacheExecutor = exec;
+ this.factory = new PathChildrenCacheFactory.Builder()
+ .withCacheData(false)
+ .withCompressed(true)
+ .withExecutorService(exec)
+ .withShutdownExecutorOnClose(false)
+ .build();
+ }
+
+ @VisibleForTesting
+ void initializeAddedChildren()
+ {
+ addedChildren = new HashSet<>();
+ }
+
+ @VisibleForTesting
+ Set getAddedChildren()
+ {
+ return addedChildren;
+ }
+
+ @LifecycleStart
+ @Override
+ public void start()
+ {
+ log.debug("Starting Announcer.");
+ synchronized (toAnnounce) {
+ if (started) {
+ log.debug("Announcer has already been started by another thread, ignoring start request.");
+ return;
+ }
+
+ started = true;
+
+ for (Announceable announceable : toAnnounce) {
+ announce(announceable.path, announceable.bytes, announceable.removeParentsIfCreated);
+ }
+ toAnnounce.clear();
+
+ for (Announceable announceable : toUpdate) {
+ update(announceable.path, announceable.bytes);
+ }
+ toUpdate.clear();
+ }
+ }
+
+ @LifecycleStop
+ @Override
+ public void stop()
+ {
+ log.debug("Stopping Announcer.");
+ synchronized (toAnnounce) {
+ if (!started) {
+ log.debug("Announcer has already been stopped by another thread, ignoring stop request.");
+ return;
+ }
+
+ started = false;
+
+ try {
+ CloseableUtils.closeAll(listeners.values());
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ finally {
+ pathChildrenCacheExecutor.shutdown();
+ }
+
+ for (Map.Entry> entry : announcements.entrySet()) {
+ String basePath = entry.getKey();
+
+ for (String announcementPath : entry.getValue().keySet()) {
+ unannounce(ZKPaths.makePath(basePath, announcementPath));
+ }
+ }
+
+ if (!parentsIBuilt.isEmpty()) {
+ CuratorMultiTransaction transaction = curator.transaction();
+
+ ArrayList operations = new ArrayList<>();
+ for (String parent : parentsIBuilt) {
+ try {
+ operations.add(curator.transactionOp().delete().forPath(parent));
+ }
+ catch (Exception e) {
+ log.info(e, "Unable to delete parent[%s] when closing Announcer.", parent);
+ }
+ }
+
+ try {
+ transaction.forOperations(operations);
+ }
+ catch (Exception e) {
+ log.info(e, "Unable to commit transaction when closing Announcer.");
+ }
+ }
+ }
+ }
+
+ /**
+ * Overload of {@link #announce(String, byte[], boolean)}, but removes parent node of path after announcement.
+ */
+ @Override
+ public void announce(String path, byte[] bytes)
+ {
+ announce(path, bytes, true);
+ }
+
+ /**
+ * Announces the provided bytes at the given path.
+ *
+ *
+ * Announcement using {@link PathChildrenAnnouncer} will create an ephemeral znode at the specified path, and uses its parent
+ * path to watch all the siblings and children znodes of your specified path. The watched nodes will always exist
+ * until it is unannounced, or until {@link #stop()} is called.
+ *
+ *
+ * @param path The path to announce at
+ * @param bytes The payload to announce
+ * @param removeParentIfCreated remove parent of "path" if we had created that parent during announcement
+ */
+ @Override
+ public void announce(String path, byte[] bytes, boolean removeParentIfCreated)
+ {
+ synchronized (toAnnounce) {
+ if (!started) {
+ log.debug("Announcer has not started yet, queuing announcement for later processing...");
+ toAnnounce.add(new Announceable(path, bytes, removeParentIfCreated));
+ return;
+ }
+ }
+
+ final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
+
+ final String parentPath = pathAndNode.getPath();
+ boolean buildParentPath = false;
+
+ ConcurrentMap subPaths = announcements.get(parentPath);
+
+ if (subPaths == null) {
+ try {
+ if (curator.checkExists().forPath(parentPath) == null) {
+ buildParentPath = true;
+ }
+ }
+ catch (Exception e) {
+ log.debug(e, "Problem checking if the parent existed, ignoring.");
+ }
+
+ final ConcurrentHashMap finalSubPaths = announcements.computeIfAbsent(parentPath, key -> new ConcurrentHashMap<>());
+
+ // Synchronize to make sure that I only create a listener once.
+ synchronized (finalSubPaths) {
+ if (!listeners.containsKey(parentPath)) {
+ final PathChildrenCache cache = factory.make(curator, parentPath);
+ cache.getListenable().addListener(
+ new PathChildrenCacheListener()
+ {
+ private final AtomicReference> pathsLost = new AtomicReference<>(null);
+
+ @Override
+ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
+ {
+ // NOTE: ZooKeeper does not guarantee that we will get every event, and thus PathChildrenCache doesn't
+ // as well. If one of the below events are missed, Announcer might not work properly.
+ log.debug("Path[%s] got event[%s]", parentPath, event);
+ switch (event.getType()) {
+ case CHILD_REMOVED:
+ final ChildData child = event.getData();
+ final ZKPaths.PathAndNode childPath = ZKPaths.getPathAndNode(child.getPath());
+ final byte[] value = finalSubPaths.get(childPath.getNode());
+ if (value != null) {
+ log.info("Node[%s] dropped, reinstating.", child.getPath());
+ createAnnouncement(child.getPath(), value);
+ }
+ break;
+ case CONNECTION_LOST:
+ // Lost connection, which means session is broken, take inventory of what has been seen.
+ // This is to protect from a race condition in which the ephemeral node could have been
+ // created but not actually seen by the PathChildrenCache, which means that it won't know
+ // that it disappeared and thus will not generate a CHILD_REMOVED event for us. Under normal
+ // circumstances, this can only happen upon connection loss; but technically if you have
+ // an adversary in the system, they could also delete the ephemeral node before the cache sees
+ // it. This does not protect from that case, so don't have adversaries.
+
+ Set pathsToReinstate = new HashSet<>();
+ for (String node : finalSubPaths.keySet()) {
+ String path = ZKPaths.makePath(parentPath, node);
+ log.info("Node[%s] is added to reinstate.", path);
+ pathsToReinstate.add(path);
+ }
+
+ if (!pathsToReinstate.isEmpty() && !pathsLost.compareAndSet(null, pathsToReinstate)) {
+ log.info("Already had a pathsLost set!?[%s]", parentPath);
+ }
+ break;
+ case CONNECTION_RECONNECTED:
+ final Set thePathsLost = pathsLost.getAndSet(null);
+
+ if (thePathsLost != null) {
+ for (String path : thePathsLost) {
+ log.info("Reinstating [%s]", path);
+ final ZKPaths.PathAndNode split = ZKPaths.getPathAndNode(path);
+ createAnnouncement(path, announcements.get(split.getPath()).get(split.getNode()));
+ }
+ }
+ break;
+ case CHILD_ADDED:
+ if (addedChildren != null) {
+ addedChildren.add(event.getData().getPath());
+ }
+ // fall through
+ case INITIALIZED:
+ case CHILD_UPDATED:
+ case CONNECTION_SUSPENDED:
+ // do nothing
+ }
+ }
+ }
+ );
+
+ synchronized (toAnnounce) {
+ if (started) {
+ if (buildParentPath) {
+ createPath(parentPath, removeParentIfCreated);
+ }
+ startCache(cache);
+ listeners.put(parentPath, cache);
+ }
+ }
+ }
+ }
+
+ subPaths = finalSubPaths;
+ }
+
+ boolean created = false;
+ synchronized (toAnnounce) {
+ if (started) {
+ byte[] oldBytes = subPaths.putIfAbsent(pathAndNode.getNode(), bytes);
+
+ if (oldBytes == null) {
+ created = true;
+ } else if (!Arrays.equals(oldBytes, bytes)) {
+ throw new IAE("Cannot reannounce different values under the same path");
+ }
+ }
+ }
+
+ if (created) {
+ try {
+ createAnnouncement(path, bytes);
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ @Override
+ public void update(final String path, final byte[] bytes)
+ {
+ synchronized (toAnnounce) {
+ if (!started) {
+ toUpdate.add(new Announceable(path, bytes));
+ return;
+ }
+ }
+
+ final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
+
+ final String parentPath = pathAndNode.getPath();
+ final String nodePath = pathAndNode.getNode();
+
+ ConcurrentMap subPaths = announcements.get(parentPath);
+
+ if (subPaths == null || subPaths.get(nodePath) == null) {
+ throw new ISE("Cannot update path[%s] that hasn't been announced!", path);
+ }
+
+ synchronized (toAnnounce) {
+ try {
+ byte[] oldBytes = subPaths.get(nodePath);
+
+ if (!Arrays.equals(oldBytes, bytes)) {
+ subPaths.put(nodePath, bytes);
+ updateAnnouncement(path, bytes);
+ }
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ private void createAnnouncement(final String path, byte[] value) throws Exception
+ {
+ curator.create().compressed().withMode(CreateMode.EPHEMERAL).inBackground().forPath(path, value);
+ }
+
+ private void updateAnnouncement(final String path, final byte[] value) throws Exception
+ {
+ curator.setData().compressed().inBackground().forPath(path, value);
+ }
+
+ /**
+ * Unannounces an announcement created at path. Note that if all announcements get removed, the Announcer
+ * will continue to have ZK watches on paths because clearing them out is a source of ugly race conditions.
+ *
+ * If you need to completely clear all the state of what is being watched and announced, stop() the Announcer.
+ *
+ * @param path the path to unannounce
+ */
+ @Override
+ public void unannounce(String path)
+ {
+ final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path);
+ final String parentPath = pathAndNode.getPath();
+
+ final ConcurrentMap subPaths = announcements.get(parentPath);
+
+ if (subPaths == null || subPaths.remove(pathAndNode.getNode()) == null) {
+ log.debug("Path[%s] not announced, cannot unannounce.", path);
+ return;
+ }
+ log.info("Unannouncing [%s]", path);
+
+ try {
+ CuratorOp deleteOp = curator.transactionOp().delete().forPath(path);
+ curator.transaction().forOperations(deleteOp);
+ }
+ catch (KeeperException.NoNodeException e) {
+ log.info("Unannounced node[%s] that does not exist.", path);
+ }
+ catch (KeeperException.NotEmptyException e) {
+ log.warn("Unannouncing non-empty path[%s]", path);
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void startCache(PathChildrenCache cache)
+ {
+ try {
+ cache.start();
+ }
+ catch (Throwable e) {
+ throw CloseableUtils.closeAndWrapInCatch(e, cache);
+ }
+ }
+
+ private void createPath(String parentPath, boolean removeParentsIfCreated)
+ {
+ try {
+ curator.create().creatingParentsIfNeeded().forPath(parentPath);
+ if (removeParentsIfCreated) {
+ parentsIBuilt.add(parentPath);
+ }
+ log.debug("Created parentPath[%s], %s remove on stop.", parentPath, removeParentsIfCreated ? "will" : "will not");
+ }
+ catch (KeeperException.NodeExistsException e) {
+ log.info(e, "Problem creating parentPath[%s], someone else created it first?", parentPath);
+ }
+ catch (Exception e) {
+ log.error(e, "Unhandled exception when creating parentPath[%s].", parentPath);
+ }
+ }
+}
diff --git a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
index 46a5c5a0a7ff..f80d2a2f268c 100644
--- a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
+++ b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
@@ -24,8 +24,10 @@
import com.google.inject.Module;
import com.google.inject.Provides;
import org.apache.curator.framework.CuratorFramework;
+import org.apache.druid.curator.CuratorConfig;
import org.apache.druid.curator.ZkEnablementConfig;
import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.PathChildrenAnnouncer;
import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer;
@@ -66,15 +68,13 @@ public void configure(Binder binder)
@Provides
@ManageLifecycleAnnouncements
- public Announcer getAnnouncer(CuratorFramework curator)
+ public Announcer getAnnouncer(CuratorFramework curator, CuratorConfig config)
{
- return new Announcer(curator, Execs.singleThreaded("Announcer-%s"));
- }
-
- @Provides
- @ManageLifecycleAnnouncements
- public NodeAnnouncer getNodeAnnouncer(CuratorFramework curator)
- {
- return new NodeAnnouncer(curator, Execs.singleThreaded("Announcer-%s"));
+ boolean usingPathChildrenCacheAnnouncer = config.getPathChildrenCacheStrategy();
+ if (usingPathChildrenCacheAnnouncer) {
+ return new PathChildrenAnnouncer(curator, Execs.singleThreaded("Announcer-%s"));
+ } else {
+ return new NodeAnnouncer(curator, Execs.singleThreaded("Announcer-%s"));
+ }
}
}
diff --git a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/announcement/PathChildrenAnnouncerTest.java
similarity index 93%
rename from server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
rename to server/src/test/java/org/apache/druid/curator/announcement/PathChildrenAnnouncerTest.java
index 8a66919bca6b..095a38f69ce0 100644
--- a/server/src/test/java/org/apache/druid/curator/announcement/AnnouncerTest.java
+++ b/server/src/test/java/org/apache/druid/curator/announcement/PathChildrenAnnouncerTest.java
@@ -46,9 +46,9 @@
/**
*
*/
-public class AnnouncerTest extends CuratorTestBase
+public class PathChildrenAnnouncerTest extends CuratorTestBase
{
- private static final Logger log = new Logger(AnnouncerTest.class);
+ private static final Logger log = new Logger(PathChildrenAnnouncerTest.class);
private ExecutorService exec;
@Before
@@ -69,7 +69,7 @@ public void tearDown()
@Test(timeout = 60_000L)
public void testSanity() throws Exception
{
- Announcer announcer = new Announcer(curator, exec);
+ PathChildrenAnnouncer announcer = new PathChildrenAnnouncer(curator, exec);
announcer.initializeAddedChildren();
final byte[] billy = StringUtils.toUtf8("billy");
@@ -138,7 +138,7 @@ public void testSanity() throws Exception
@Test(timeout = 60_000L)
public void testSessionKilled() throws Exception
{
- Announcer announcer = new Announcer(curator, exec);
+ PathChildrenAnnouncer announcer = new PathChildrenAnnouncer(curator, exec);
try {
CuratorOp createOp = curator.transactionOp().create().forPath("/somewhere");
curator.transaction().forOperations(createOp);
@@ -180,7 +180,7 @@ public void testSessionKilled() throws Exception
@Test
public void testRemovesParentIfCreated() throws Exception
{
- Announcer announcer = new Announcer(curator, exec);
+ PathChildrenAnnouncer announcer = new PathChildrenAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test2";
@@ -204,7 +204,7 @@ public void testRemovesParentIfCreated() throws Exception
@Test
public void testLeavesBehindParentPathIfAlreadyExists() throws Exception
{
- Announcer announcer = new Announcer(curator, exec);
+ PathChildrenAnnouncer announcer = new PathChildrenAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test";
@@ -231,7 +231,7 @@ public void testLeavesBehindParentPathIfAlreadyExists() throws Exception
@Test
public void testLeavesParentPathsUntouchedWhenInstructed() throws Exception
{
- Announcer announcer = new Announcer(curator, exec);
+ PathChildrenAnnouncer announcer = new PathChildrenAnnouncer(curator, exec);
final byte[] billy = StringUtils.toUtf8("billy");
final String testPath = "/somewhere/test";
@@ -253,7 +253,7 @@ public void testLeavesParentPathsUntouchedWhenInstructed() throws Exception
}
private void awaitAnnounce(
- final Announcer announcer,
+ final PathChildrenAnnouncer announcer,
final String path,
final byte[] bytes,
boolean removeParentsIfCreated
From 5e67ed2112e045b184f1b35584702956bad7b7c2 Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 17 Mar 2025 17:49:56 +0800
Subject: [PATCH 40/43] Checkstyle
---
.../src/main/java/org/apache/druid/guice/AnnouncerModule.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
index f80d2a2f268c..e90bd8676e5a 100644
--- a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
+++ b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
@@ -27,8 +27,8 @@
import org.apache.druid.curator.CuratorConfig;
import org.apache.druid.curator.ZkEnablementConfig;
import org.apache.druid.curator.announcement.Announcer;
-import org.apache.druid.curator.announcement.PathChildrenAnnouncer;
import org.apache.druid.curator.announcement.NodeAnnouncer;
+import org.apache.druid.curator.announcement.PathChildrenAnnouncer;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer;
import org.apache.druid.server.coordination.CuratorDataSegmentServerAnnouncer;
From c0fc4752a79ac32a1cf42a28ab83adfad326ae3b Mon Sep 17 00:00:00 2001
From: GWphua
Date: Tue, 18 Mar 2025 10:28:38 +0800
Subject: [PATCH 41/43] Dependency injection for Announcers
---
.../worker/WorkerCuratorCoordinator.java | 10 +++---
.../worker/WorkerTaskMonitorTest.java | 3 ++
.../worker/http/WorkerResourceTest.java | 3 ++
.../discovery/CuratorDruidNodeAnnouncer.java | 11 ++++--
.../apache/druid/guice/AnnouncerModule.java | 18 +++++++++-
.../annotations/DirectExecutorAnnouncer.java | 34 +++++++++++++++++++
.../annotations/SingleThreadedAnnouncer.java | 34 +++++++++++++++++++
.../BatchDataSegmentAnnouncer.java | 9 ++---
.../CuratorDataSegmentServerAnnouncer.java | 7 ++--
website/.spelling | 2 ++
10 files changed, 115 insertions(+), 16 deletions(-)
create mode 100644 server/src/main/java/org/apache/druid/guice/annotations/DirectExecutorAnnouncer.java
create mode 100644 server/src/main/java/org/apache/druid/guice/annotations/SingleThreadedAnnouncer.java
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
index 1d903d2bd0e8..669a5acf4cbc 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
@@ -25,11 +25,11 @@
import com.google.inject.Inject;
import org.apache.curator.framework.CuratorFramework;
import org.apache.druid.curator.CuratorUtils;
-import org.apache.druid.curator.announcement.NodeAnnouncer;
+import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.guice.annotations.DirectExecutorAnnouncer;
import org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.common.logger.Logger;
@@ -54,7 +54,7 @@ public class WorkerCuratorCoordinator
private final ObjectMapper jsonMapper;
private final RemoteTaskRunnerConfig config;
private final CuratorFramework curatorFramework;
- private final NodeAnnouncer announcer;
+ private final Announcer announcer;
private final String baseAnnouncementsPath;
private final String baseTaskPath;
@@ -69,6 +69,7 @@ public WorkerCuratorCoordinator(
IndexerZkConfig indexerZkConfig,
RemoteTaskRunnerConfig config,
CuratorFramework curatorFramework,
+ @DirectExecutorAnnouncer Announcer announcer,
Worker worker
)
{
@@ -76,8 +77,7 @@ public WorkerCuratorCoordinator(
this.config = config;
this.curatorFramework = curatorFramework;
this.worker = worker;
-
- this.announcer = new NodeAnnouncer(curatorFramework, Execs.directExecutor());
+ this.announcer = announcer;
this.baseAnnouncementsPath = getPath(Arrays.asList(indexerZkConfig.getAnnouncementsPath(), worker.getHost()));
this.baseTaskPath = getPath(Arrays.asList(indexerZkConfig.getTasksPath(), worker.getHost()));
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java
index c8a754625358..3a3dc2d5333f 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java
@@ -28,6 +28,7 @@
import org.apache.curator.test.TestingCluster;
import org.apache.druid.client.coordinator.NoopCoordinatorClient;
import org.apache.druid.curator.PotentiallyGzippedCompressionProvider;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexing.common.IndexingServiceCondition;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
@@ -47,6 +48,7 @@
import org.apache.druid.indexing.worker.config.WorkerConfig;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.rpc.indexing.NoopOverlordClient;
import org.apache.druid.rpc.indexing.OverlordClient;
import org.apache.druid.segment.IndexIO;
@@ -141,6 +143,7 @@ public String getBase()
),
new TestRemoteTaskRunnerConfig(new Period("PT1S")),
cf,
+ new NodeAnnouncer(cf, Execs.directExecutor()),
worker
);
workerCuratorCoordinator.start();
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/http/WorkerResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/http/WorkerResourceTest.java
index 0ad900dd8c44..df8e34ab1111 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/http/WorkerResourceTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/http/WorkerResourceTest.java
@@ -26,6 +26,7 @@
import org.apache.curator.test.TestingCluster;
import org.apache.druid.curator.PotentiallyGzippedCompressionProvider;
import org.apache.druid.curator.ZkEnablementConfig;
+import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
import org.apache.druid.indexing.worker.Worker;
import org.apache.druid.indexing.worker.WorkerCuratorCoordinator;
@@ -33,6 +34,7 @@
import org.apache.druid.indexing.worker.config.WorkerConfig;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.server.initialization.IndexerZkConfig;
import org.apache.druid.server.initialization.ZkPathsConfig;
import org.easymock.EasyMock;
@@ -95,6 +97,7 @@ public String getBase()
}, null, null, null, null),
new RemoteTaskRunnerConfig(),
cf,
+ new NodeAnnouncer(cf, Execs.directExecutor()),
worker
);
curatorCoordinator.start();
diff --git a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java
index 0dc4d85ec8ea..34d54dbb56ca 100644
--- a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java
@@ -23,11 +23,12 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import org.apache.curator.utils.ZKPaths;
-import org.apache.druid.curator.announcement.NodeAnnouncer;
+import org.apache.druid.curator.announcement.Announcer;
import org.apache.druid.discovery.DiscoveryDruidNode;
import org.apache.druid.discovery.DruidNodeAnnouncer;
import org.apache.druid.discovery.NodeRole;
import org.apache.druid.guice.annotations.Json;
+import org.apache.druid.guice.annotations.SingleThreadedAnnouncer;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.DruidNode;
@@ -42,12 +43,16 @@ static String makeNodeAnnouncementPath(ZkPathsConfig config, NodeRole nodeRole,
private static final Logger log = new Logger(CuratorDruidNodeAnnouncer.class);
- private final NodeAnnouncer announcer;
+ private final Announcer announcer;
private final ZkPathsConfig config;
private final ObjectMapper jsonMapper;
@Inject
- public CuratorDruidNodeAnnouncer(NodeAnnouncer announcer, ZkPathsConfig config, @Json ObjectMapper jsonMapper)
+ public CuratorDruidNodeAnnouncer(
+ @SingleThreadedAnnouncer Announcer announcer,
+ ZkPathsConfig config,
+ @Json ObjectMapper jsonMapper
+ )
{
this.announcer = announcer;
this.config = config;
diff --git a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
index e90bd8676e5a..da2d38f64c9f 100644
--- a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
+++ b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
@@ -29,6 +29,8 @@
import org.apache.druid.curator.announcement.Announcer;
import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.curator.announcement.PathChildrenAnnouncer;
+import org.apache.druid.guice.annotations.DirectExecutorAnnouncer;
+import org.apache.druid.guice.annotations.SingleThreadedAnnouncer;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer;
import org.apache.druid.server.coordination.CuratorDataSegmentServerAnnouncer;
@@ -67,8 +69,9 @@ public void configure(Binder binder)
}
@Provides
+ @SingleThreadedAnnouncer
@ManageLifecycleAnnouncements
- public Announcer getAnnouncer(CuratorFramework curator, CuratorConfig config)
+ public Announcer getAnnouncerWithSingleThreadedExecutorService(CuratorFramework curator, CuratorConfig config)
{
boolean usingPathChildrenCacheAnnouncer = config.getPathChildrenCacheStrategy();
if (usingPathChildrenCacheAnnouncer) {
@@ -77,4 +80,17 @@ public Announcer getAnnouncer(CuratorFramework curator, CuratorConfig config)
return new NodeAnnouncer(curator, Execs.singleThreaded("Announcer-%s"));
}
}
+
+ @Provides
+ @DirectExecutorAnnouncer
+ @ManageLifecycleAnnouncements
+ public Announcer getAnnouncerWithDirectExecutorService(CuratorFramework curator, CuratorConfig config)
+ {
+ boolean usingPathChildrenCacheAnnouncer = config.getPathChildrenCacheStrategy();
+ if (usingPathChildrenCacheAnnouncer) {
+ return new PathChildrenAnnouncer(curator, Execs.directExecutor());
+ } else {
+ return new NodeAnnouncer(curator, Execs.directExecutor());
+ }
+ }
}
diff --git a/server/src/main/java/org/apache/druid/guice/annotations/DirectExecutorAnnouncer.java b/server/src/main/java/org/apache/druid/guice/annotations/DirectExecutorAnnouncer.java
new file mode 100644
index 000000000000..0d675469222b
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/guice/annotations/DirectExecutorAnnouncer.java
@@ -0,0 +1,34 @@
+/*
+ * 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.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@BindingAnnotation
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface DirectExecutorAnnouncer
+{
+}
diff --git a/server/src/main/java/org/apache/druid/guice/annotations/SingleThreadedAnnouncer.java b/server/src/main/java/org/apache/druid/guice/annotations/SingleThreadedAnnouncer.java
new file mode 100644
index 000000000000..8f815301ecfa
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/guice/annotations/SingleThreadedAnnouncer.java
@@ -0,0 +1,34 @@
+/*
+ * 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.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@BindingAnnotation
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface SingleThreadedAnnouncer
+{
+}
diff --git a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java
index 9d111b4a85dd..d75b788205eb 100644
--- a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java
@@ -32,7 +32,8 @@
import org.apache.curator.utils.ZKPaths;
import org.apache.druid.common.utils.UUIDUtils;
import org.apache.druid.curator.ZkEnablementConfig;
-import org.apache.druid.curator.announcement.NodeAnnouncer;
+import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.guice.annotations.SingleThreadedAnnouncer;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
@@ -64,7 +65,7 @@ public class BatchDataSegmentAnnouncer implements DataSegmentAnnouncer
private final BatchDataSegmentAnnouncerConfig config;
@Nullable //Null if zk is disabled or isSkipSegmentAnnouncementOnZk = true
- private final NodeAnnouncer announcer;
+ private final Announcer announcer;
private final ObjectMapper jsonMapper;
private final String liveSegmentLocation;
@@ -91,7 +92,7 @@ public BatchDataSegmentAnnouncer(
DruidServerMetadata server,
final BatchDataSegmentAnnouncerConfig config,
ZkPathsConfig zkPaths,
- Provider announcerProvider,
+ @SingleThreadedAnnouncer Provider announcerProvider,
ObjectMapper jsonMapper,
ZkEnablementConfig zkEnablementConfig
)
@@ -127,7 +128,7 @@ public BatchDataSegmentAnnouncer(
DruidServerMetadata server,
final BatchDataSegmentAnnouncerConfig config,
ZkPathsConfig zkPaths,
- NodeAnnouncer announcer,
+ Announcer announcer,
ObjectMapper jsonMapper
)
{
diff --git a/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java
index a337c78b948d..dea15d840aad 100644
--- a/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java
@@ -23,7 +23,8 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import org.apache.curator.utils.ZKPaths;
-import org.apache.druid.curator.announcement.NodeAnnouncer;
+import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.guice.annotations.SingleThreadedAnnouncer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.initialization.ZkPathsConfig;
@@ -37,7 +38,7 @@ public class CuratorDataSegmentServerAnnouncer implements DataSegmentServerAnnou
private final DruidServerMetadata server;
private final ZkPathsConfig config;
- private final NodeAnnouncer announcer;
+ private final Announcer announcer;
private final ObjectMapper jsonMapper;
private final Object lock = new Object();
@@ -48,7 +49,7 @@ public class CuratorDataSegmentServerAnnouncer implements DataSegmentServerAnnou
public CuratorDataSegmentServerAnnouncer(
DruidServerMetadata server,
ZkPathsConfig config,
- NodeAnnouncer announcer,
+ @SingleThreadedAnnouncer Announcer announcer,
ObjectMapper jsonMapper
)
{
diff --git a/website/.spelling b/website/.spelling
index 8d9be4dfd7ee..c484813d8da7 100644
--- a/website/.spelling
+++ b/website/.spelling
@@ -196,6 +196,7 @@ Murmur3
MVCC
MV_TO_ARRAY
NFS
+NodeCache
OCF
OIDC
OLAP
@@ -207,6 +208,7 @@ OutputStream
ParAccel
ParseSpec
ParseSpecs
+PathChildrenCache
Protobuf
protobuf
pull-deps
From 5c0845a24109bbcd48c9ef98a709049f814ebf1e Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 24 Mar 2025 10:29:35 +0800
Subject: [PATCH 42/43] Rename announcer to serviceAnnouncer
---
.../druid/indexing/worker/WorkerCuratorCoordinator.java | 6 +++---
.../apache/druid/curator/announcement/NodeAnnouncer.java | 2 +-
.../druid/curator/announcement/PathChildrenAnnouncer.java | 2 +-
.../{Announcer.java => ServiceAnnouncer.java} | 2 +-
.../curator/discovery/CuratorDruidNodeAnnouncer.java | 6 +++---
.../main/java/org/apache/druid/guice/AnnouncerModule.java | 6 +++---
.../server/coordination/BatchDataSegmentAnnouncer.java | 8 ++++----
.../coordination/CuratorDataSegmentServerAnnouncer.java | 6 +++---
8 files changed, 19 insertions(+), 19 deletions(-)
rename server/src/main/java/org/apache/druid/curator/announcement/{Announcer.java => ServiceAnnouncer.java} (96%)
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
index 669a5acf4cbc..c018c6a1a63b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerCuratorCoordinator.java
@@ -25,7 +25,7 @@
import com.google.inject.Inject;
import org.apache.curator.framework.CuratorFramework;
import org.apache.druid.curator.CuratorUtils;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.ServiceAnnouncer;
import org.apache.druid.guice.annotations.DirectExecutorAnnouncer;
import org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
import org.apache.druid.java.util.common.DateTimes;
@@ -54,7 +54,7 @@ public class WorkerCuratorCoordinator
private final ObjectMapper jsonMapper;
private final RemoteTaskRunnerConfig config;
private final CuratorFramework curatorFramework;
- private final Announcer announcer;
+ private final ServiceAnnouncer announcer;
private final String baseAnnouncementsPath;
private final String baseTaskPath;
@@ -69,7 +69,7 @@ public WorkerCuratorCoordinator(
IndexerZkConfig indexerZkConfig,
RemoteTaskRunnerConfig config,
CuratorFramework curatorFramework,
- @DirectExecutorAnnouncer Announcer announcer,
+ @DirectExecutorAnnouncer ServiceAnnouncer announcer,
Worker worker
)
{
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
index bff7c36c85aa..d38bf79fa385 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/NodeAnnouncer.java
@@ -57,7 +57,7 @@
* uses the PathChildrenCache recipe instead.
*
*/
-public class NodeAnnouncer implements Announcer
+public class NodeAnnouncer implements ServiceAnnouncer
{
private static final Logger log = new Logger(NodeAnnouncer.class);
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/PathChildrenAnnouncer.java b/server/src/main/java/org/apache/druid/curator/announcement/PathChildrenAnnouncer.java
index 9ce965551114..ec754d96be2c 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/PathChildrenAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/PathChildrenAnnouncer.java
@@ -63,7 +63,7 @@
* uses the NodeCache recipe instead.
*
*/
-public class PathChildrenAnnouncer implements Announcer
+public class PathChildrenAnnouncer implements ServiceAnnouncer
{
private static final Logger log = new Logger(PathChildrenAnnouncer.class);
diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/ServiceAnnouncer.java
similarity index 96%
rename from server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
rename to server/src/main/java/org/apache/druid/curator/announcement/ServiceAnnouncer.java
index 8852920071b4..e079e1340e97 100644
--- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java
+++ b/server/src/main/java/org/apache/druid/curator/announcement/ServiceAnnouncer.java
@@ -19,7 +19,7 @@
package org.apache.druid.curator.announcement;
-public interface Announcer
+public interface ServiceAnnouncer
{
void start();
diff --git a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java
index 34d54dbb56ca..662706578ba8 100644
--- a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java
@@ -23,7 +23,7 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import org.apache.curator.utils.ZKPaths;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.ServiceAnnouncer;
import org.apache.druid.discovery.DiscoveryDruidNode;
import org.apache.druid.discovery.DruidNodeAnnouncer;
import org.apache.druid.discovery.NodeRole;
@@ -43,13 +43,13 @@ static String makeNodeAnnouncementPath(ZkPathsConfig config, NodeRole nodeRole,
private static final Logger log = new Logger(CuratorDruidNodeAnnouncer.class);
- private final Announcer announcer;
+ private final ServiceAnnouncer announcer;
private final ZkPathsConfig config;
private final ObjectMapper jsonMapper;
@Inject
public CuratorDruidNodeAnnouncer(
- @SingleThreadedAnnouncer Announcer announcer,
+ @SingleThreadedAnnouncer ServiceAnnouncer announcer,
ZkPathsConfig config,
@Json ObjectMapper jsonMapper
)
diff --git a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
index da2d38f64c9f..5b109b975d73 100644
--- a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
+++ b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
@@ -26,7 +26,7 @@
import org.apache.curator.framework.CuratorFramework;
import org.apache.druid.curator.CuratorConfig;
import org.apache.druid.curator.ZkEnablementConfig;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.ServiceAnnouncer;
import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.curator.announcement.PathChildrenAnnouncer;
import org.apache.druid.guice.annotations.DirectExecutorAnnouncer;
@@ -71,7 +71,7 @@ public void configure(Binder binder)
@Provides
@SingleThreadedAnnouncer
@ManageLifecycleAnnouncements
- public Announcer getAnnouncerWithSingleThreadedExecutorService(CuratorFramework curator, CuratorConfig config)
+ public ServiceAnnouncer getAnnouncerWithSingleThreadedExecutorService(CuratorFramework curator, CuratorConfig config)
{
boolean usingPathChildrenCacheAnnouncer = config.getPathChildrenCacheStrategy();
if (usingPathChildrenCacheAnnouncer) {
@@ -84,7 +84,7 @@ public Announcer getAnnouncerWithSingleThreadedExecutorService(CuratorFramework
@Provides
@DirectExecutorAnnouncer
@ManageLifecycleAnnouncements
- public Announcer getAnnouncerWithDirectExecutorService(CuratorFramework curator, CuratorConfig config)
+ public ServiceAnnouncer getAnnouncerWithDirectExecutorService(CuratorFramework curator, CuratorConfig config)
{
boolean usingPathChildrenCacheAnnouncer = config.getPathChildrenCacheStrategy();
if (usingPathChildrenCacheAnnouncer) {
diff --git a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java
index d75b788205eb..e88768de4b9c 100644
--- a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java
@@ -32,7 +32,7 @@
import org.apache.curator.utils.ZKPaths;
import org.apache.druid.common.utils.UUIDUtils;
import org.apache.druid.curator.ZkEnablementConfig;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.ServiceAnnouncer;
import org.apache.druid.guice.annotations.SingleThreadedAnnouncer;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
@@ -65,7 +65,7 @@ public class BatchDataSegmentAnnouncer implements DataSegmentAnnouncer
private final BatchDataSegmentAnnouncerConfig config;
@Nullable //Null if zk is disabled or isSkipSegmentAnnouncementOnZk = true
- private final Announcer announcer;
+ private final ServiceAnnouncer announcer;
private final ObjectMapper jsonMapper;
private final String liveSegmentLocation;
@@ -92,7 +92,7 @@ public BatchDataSegmentAnnouncer(
DruidServerMetadata server,
final BatchDataSegmentAnnouncerConfig config,
ZkPathsConfig zkPaths,
- @SingleThreadedAnnouncer Provider announcerProvider,
+ @SingleThreadedAnnouncer Provider announcerProvider,
ObjectMapper jsonMapper,
ZkEnablementConfig zkEnablementConfig
)
@@ -128,7 +128,7 @@ public BatchDataSegmentAnnouncer(
DruidServerMetadata server,
final BatchDataSegmentAnnouncerConfig config,
ZkPathsConfig zkPaths,
- Announcer announcer,
+ ServiceAnnouncer announcer,
ObjectMapper jsonMapper
)
{
diff --git a/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java
index dea15d840aad..ec029ba6c9a2 100644
--- a/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java
+++ b/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java
@@ -23,7 +23,7 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import org.apache.curator.utils.ZKPaths;
-import org.apache.druid.curator.announcement.Announcer;
+import org.apache.druid.curator.announcement.ServiceAnnouncer;
import org.apache.druid.guice.annotations.SingleThreadedAnnouncer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.initialization.ZkPathsConfig;
@@ -38,7 +38,7 @@ public class CuratorDataSegmentServerAnnouncer implements DataSegmentServerAnnou
private final DruidServerMetadata server;
private final ZkPathsConfig config;
- private final Announcer announcer;
+ private final ServiceAnnouncer announcer;
private final ObjectMapper jsonMapper;
private final Object lock = new Object();
@@ -49,7 +49,7 @@ public class CuratorDataSegmentServerAnnouncer implements DataSegmentServerAnnou
public CuratorDataSegmentServerAnnouncer(
DruidServerMetadata server,
ZkPathsConfig config,
- @SingleThreadedAnnouncer Announcer announcer,
+ @SingleThreadedAnnouncer ServiceAnnouncer announcer,
ObjectMapper jsonMapper
)
{
From ea5d9050f3f573ce13826b499a174dd93f27f42f Mon Sep 17 00:00:00 2001
From: GWphua
Date: Mon, 24 Mar 2025 11:01:13 +0800
Subject: [PATCH 43/43] Fix checkstyle for import order
---
.../src/main/java/org/apache/druid/guice/AnnouncerModule.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
index 5b109b975d73..b6a4d472283b 100644
--- a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
+++ b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java
@@ -26,9 +26,9 @@
import org.apache.curator.framework.CuratorFramework;
import org.apache.druid.curator.CuratorConfig;
import org.apache.druid.curator.ZkEnablementConfig;
-import org.apache.druid.curator.announcement.ServiceAnnouncer;
import org.apache.druid.curator.announcement.NodeAnnouncer;
import org.apache.druid.curator.announcement.PathChildrenAnnouncer;
+import org.apache.druid.curator.announcement.ServiceAnnouncer;
import org.apache.druid.guice.annotations.DirectExecutorAnnouncer;
import org.apache.druid.guice.annotations.SingleThreadedAnnouncer;
import org.apache.druid.java.util.common.concurrent.Execs;