From af26c563c6a2726cf26c46a52a3c03678512e4b4 Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Mon, 18 Feb 2019 16:22:31 -0800 Subject: [PATCH 01/22] V1 - improve parallelism of zookeeper based segment change processing --- .../segment/loading/SegmentLoaderConfig.java | 3 +- .../server/coordination/ZkCoordinator.java | 120 ++--- .../coordinator/CuratorLoadQueuePeonV2.java | 418 ++++++++++++++++++ .../coordinator/DruidCoordinatorConfig.java | 4 + .../coordinator/LoadQueueTaskMaster.java | 10 +- .../coordination/ZkCoordinatorTest.java | 3 +- .../CuratorDruidCoordinatorTest.java | 3 +- .../coordinator/DruidCoordinatorTest.java | 3 +- .../coordinator/HttpLoadQueuePeonTest.java | 3 +- .../server/coordinator/LoadQueuePeonTest.java | 5 +- .../TestDruidCoordinatorConfig.java | 11 +- .../DruidCoordinatorSegmentKillerTest.java | 3 +- 12 files changed, 520 insertions(+), 66 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeonV2.java diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java index b09bd0c5bea8..80f0fbc9fdf4 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.utils.JvmUtils; import org.hibernate.validator.constraints.NotEmpty; import java.io.File; @@ -46,7 +47,7 @@ public class SegmentLoaderConfig private int announceIntervalMillis = 0; // do not background announce @JsonProperty("numLoadingThreads") - private int numLoadingThreads = 10; + private int numLoadingThreads = JvmUtils.getRuntimeInfo().getAvailableProcessors(); @JsonProperty("numBootstrapThreads") private Integer numBootstrapThreads = null; diff --git a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java index fcf86b322e67..98883d3508ed 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Inject; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.ChildData; @@ -32,9 +33,12 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.server.initialization.ZkPathsConfig; import java.io.IOException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; /** * Use {@link org.apache.druid.server.coordinator.HttpLoadQueuePeon} for segment load/drops. @@ -54,6 +58,7 @@ public class ZkCoordinator private volatile PathChildrenCache loadQueueCache; private volatile boolean started = false; + private final ExecutorService loadSegmentService; @Inject public ZkCoordinator( @@ -61,7 +66,8 @@ public ZkCoordinator( ObjectMapper jsonMapper, ZkPathsConfig zkPaths, DruidServerMetadata me, - CuratorFramework curator + CuratorFramework curator, + SegmentLoaderConfig config ) { this.dataSegmentChangeHandler = loadDropHandler; @@ -69,6 +75,11 @@ public ZkCoordinator( this.zkPaths = zkPaths; this.me = me; this.curator = curator; + this.loadSegmentService = Executors.newFixedThreadPool( + config.getNumLoadingThreads(), + new ThreadFactoryBuilder().setNameFormat( + "ZKCoordinator-LoadSegment--%d").build() + ); } @LifecycleStart @@ -102,72 +113,71 @@ public void start() throws IOException new PathChildrenCacheListener() { @Override - public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception + public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) { - final ChildData child = event.getData(); - switch (event.getType()) { - case CHILD_ADDED: - final String path = child.getPath(); - final DataSegmentChangeRequest request = jsonMapper.readValue( - child.getData(), DataSegmentChangeRequest.class - ); - - log.info("New request[%s] with zNode[%s].", request.asString(), path); - - try { - request.go( - dataSegmentChangeHandler, - new DataSegmentChangeCallback() - { - boolean hasRun = false; - - @Override - public void execute() + loadSegmentService.submit(() -> { + final ChildData child = event.getData(); + switch (event.getType()) { + case CHILD_ADDED: + final String path = child.getPath(); + DataSegmentChangeRequest request = new SegmentChangeRequestNoop(); + try { + request = jsonMapper.readValue( + child.getData(), DataSegmentChangeRequest.class + ); + final DataSegmentChangeRequest finalRequest = request; + log.info("New request[%s] with zNode[%s].", request.asString(), path); + + finalRequest.go( + dataSegmentChangeHandler, + new DataSegmentChangeCallback() { - try { - if (!hasRun) { - curator.delete().guaranteed().forPath(path); - log.info("Completed request [%s]", request.asString()); - hasRun = true; - } - } - catch (Exception e) { + @Override + public void execute() + { try { curator.delete().guaranteed().forPath(path); + log.info("Completed request [%s]", finalRequest.asString()); } - catch (Exception e1) { - log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path); - } - log.error(e, "Exception while removing zNode[%s]", path); - throw new RuntimeException(e); + catch (Exception e) { + try { + curator.delete().guaranteed().forPath(path); + } + catch (Exception e1) { + log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path); + } + log.error(e, "Exception while removing zNode[%s]", path); + throw Throwables.propagate(e); + } } } - } - ); - } - catch (Exception e) { - try { - curator.delete().guaranteed().forPath(path); + ); } - catch (Exception e1) { - log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path); + catch (Exception e) { + try { + curator.delete().guaranteed().forPath(path); + } + catch (Exception e1) { + log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path); + } + + log.makeAlert(e, "Segment load/unload: uncaught exception.") + .addData("node", path) + .addData("nodeProperties", request) + .emit(); } - log.makeAlert(e, "Segment load/unload: uncaught exception.") - .addData("node", path) - .addData("nodeProperties", request) - .emit(); - } - - break; - case CHILD_REMOVED: - log.info("zNode[%s] was removed", event.getData().getPath()); - break; - default: - log.info("Ignoring event[%s]", event); - } + break; + case CHILD_REMOVED: + log.info("zNode[%s] was removed", event.getData().getPath()); + break; + default: + log.info("Ignoring event[%s]", event); + } + }); } } + ); loadQueueCache.start(); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeonV2.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeonV2.java new file mode 100644 index 000000000000..332caa489ca1 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeonV2.java @@ -0,0 +1,418 @@ +/* + * 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.coordinator; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.api.CuratorWatcher; +import org.apache.curator.utils.ZKPaths; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordination.DataSegmentChangeRequest; +import org.apache.druid.server.coordination.SegmentChangeRequestDrop; +import org.apache.druid.server.coordination.SegmentChangeRequestLoad; +import org.apache.druid.server.coordination.SegmentChangeRequestNoop; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.data.Stat; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Use {@link HttpLoadQueuePeon} instead. + */ +@Deprecated +public class CuratorLoadQueuePeonV2 extends LoadQueuePeon +{ + private static final EmittingLogger log = new EmittingLogger(CuratorLoadQueuePeon.class); + private static final int DROP = 0; + private static final int LOAD = 1; + private final CuratorFramework curator; + private final String basePath; + private final ObjectMapper jsonMapper; + private final ExecutorService processingExecutor; + private final ScheduledExecutorService checkNodeRemovedExecutor; + private final ExecutorService callBackExecutor; + private final DruidCoordinatorConfig config; + + private final AtomicLong queuedSize = new AtomicLong(0); + private final AtomicInteger failedAssignCount = new AtomicInteger(0); + + private final ConcurrentSkipListMap segmentsToLoad = new ConcurrentSkipListMap<>( + DruidCoordinator.SEGMENT_COMPARATOR + ); + private final ConcurrentSkipListMap segmentsToDrop = new ConcurrentSkipListMap<>( + DruidCoordinator.SEGMENT_COMPARATOR + ); + private final ConcurrentSkipListSet segmentsMarkedToDrop = new ConcurrentSkipListSet<>( + DruidCoordinator.SEGMENT_COMPARATOR + ); + + private static final int MAX_TASK_WAIT = 100; + + CuratorLoadQueuePeonV2( + CuratorFramework curator, + String basePath, + ObjectMapper jsonMapper, + ExecutorService processingExecutor, + ExecutorService callbackExecutor, + DruidCoordinatorConfig config + ) + { + this.curator = curator; + this.basePath = basePath; + this.jsonMapper = jsonMapper; + this.callBackExecutor = callbackExecutor; + this.processingExecutor = processingExecutor; + this.config = config; + //TODO: samarth configure the size of this pool + this.checkNodeRemovedExecutor = Executors.newScheduledThreadPool(5, new ThreadFactoryBuilder().setNameFormat( + "ZKNodeDeletionChecker--%d").build()); + } + + @JsonProperty + @Override + public Set getSegmentsToLoad() + { + return segmentsToLoad.keySet(); + } + + @JsonProperty + @Override + public Set getSegmentsToDrop() + { + return segmentsToDrop.keySet(); + } + + @JsonProperty + @Override + public Set getSegmentsMarkedToDrop() + { + return segmentsMarkedToDrop; + } + + @Override + public long getLoadQueueSize() + { + return queuedSize.get(); + } + + @Override + public int getAndResetFailedAssignCount() + { + return failedAssignCount.getAndSet(0); + } + + @Override + public int getNumberOfSegmentsInQueue() + { + return segmentsToLoad.size(); + } + + @Override + public void loadSegment( + final DataSegment segment, + final LoadPeonCallback callback + ) + { + + final SegmentHolder existingHolder = segmentsToLoad.get(segment); + if (existingHolder != null) { + if ((callback != null)) { + existingHolder.addCallback(callback); + } + return; + } + log.info("Asking server peon[%s] to load segment[%s]", basePath, segment.getId()); + queuedSize.addAndGet(segment.getSize()); + SegmentHolder holder = new SegmentHolder(segment, LOAD, Collections.singletonList(callback)); + segmentsToLoad.put(segment, holder); + processingExecutor.submit((Callable) () -> { + processSegmentChangeRequest(holder); + return null; + }); + } + + @Override + public void dropSegment( + final DataSegment segment, + final LoadPeonCallback callback + ) + { + final SegmentHolder existingHolder = segmentsToDrop.get(segment); + if (existingHolder != null) { + if (callback != null) { + existingHolder.addCallback(callback); + } + return; + } + log.info("Asking server peon[%s] to drop segment[%s]", basePath, segment.getId()); + SegmentHolder segmentHolder = new SegmentHolder(segment, DROP, Collections.singletonList(callback)); + segmentsToDrop.put(segment, segmentHolder); + processingExecutor.submit((Callable) () -> { + processSegmentChangeRequest(segmentHolder); + return null; + }); + } + + @Override + public void markSegmentToDrop(DataSegment dataSegment) + { + segmentsMarkedToDrop.add(dataSegment); + } + + @Override + public void unmarkSegmentToDrop(DataSegment dataSegment) + { + segmentsMarkedToDrop.remove(dataSegment); + } + + private void processSegmentChangeRequest(SegmentHolder segmentHolder) + { + try { + // Wait for a random duration since we do not want zookeeper getting pounded with create node requests + TimeUnit.MILLISECONDS.sleep(ThreadLocalRandom.current().nextInt(MAX_TASK_WAIT)); + log.info("Server[%s] %s [%s]", basePath, segmentHolder.getType() == LOAD ? + "loading" : "dropping", segmentHolder.getSegmentId().toString()); + final String path = ZKPaths.makePath(basePath, segmentHolder.getSegmentId().toString()); + final byte[] payload = jsonMapper.writeValueAsBytes(segmentHolder.getChangeRequest()); + curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, payload); + + checkNodeRemovedExecutor.schedule( + () -> { + try { + if (curator.checkExists().forPath(path) != null) { + failAssign(segmentHolder, new ISE("%s was never removed! Failing this operation!", path)); + } else { + log.info("%s detected to be removed. ", path); + } + } + catch (Exception e) { + failAssign(segmentHolder, e); + } + }, + config.getLoadTimeoutDelay().getMillis(), + TimeUnit.MILLISECONDS + ); + + final Stat stat = curator.checkExists().usingWatcher( + (CuratorWatcher) watchedEvent -> { + switch (watchedEvent.getType()) { + case NodeDeleted: + entryRemoved(segmentHolder, watchedEvent.getPath()); + break; + default: + // do nothing + } + } + ).forPath(path); + + if (stat == null) { + final byte[] noopPayload = jsonMapper.writeValueAsBytes(new SegmentChangeRequestNoop()); + + // Create a node and then delete it to remove the registered watcher. This is a work-around for + // a zookeeper race condition. Specifically, when you set a watcher, it fires on the next event + // that happens for that node. If no events happen, the watcher stays registered foreverz. + // Couple that with the fact that you cannot set a watcher when you create a node, but what we + // want is to create a node and then watch for it to get deleted. The solution is that you *can* + // set a watcher when you check to see if it exists so, we first create the node and then set a + // watcher on its existence. However, if already does not exist by the time the existence check + // returns, then the watcher that was set will never fire (nobody will ever create the node + // again) and thus lead to a slow, but real, memory leak. So, we create another node to cause + // that watcher to fire and delete it right away. + // + // We do not create the existence watcher first, because then it will fire when we create the + // node and we'll have the same race when trying to refresh that watcher. + curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, noopPayload); + entryRemoved(segmentHolder, path); + } + } + catch (Exception e) { + failAssign(segmentHolder, e); + } + } + + private void actionCompleted(SegmentHolder segmentHolder) + { + log.info("Action completed: " + segmentHolder.getType()); + switch (segmentHolder.getType()) { + case LOAD: + segmentsToLoad.remove(segmentHolder.getSegment()); + queuedSize.addAndGet(-segmentHolder.getSegmentSize()); + break; + case DROP: + segmentsToDrop.remove(segmentHolder.getSegment()); + break; + default: + throw new UnsupportedOperationException(); + } + + final List callbacks = segmentHolder.getCallbacks(); + callBackExecutor.execute( + () -> executeCallbacks(callbacks) + ); + } + + + @Override + public void start() + {} + + @Override + public void stop() + { + for (SegmentHolder holder : segmentsToDrop.values()) { + executeCallbacks(holder.getCallbacks()); + } + segmentsToDrop.clear(); + + for (SegmentHolder holder : segmentsToLoad.values()) { + executeCallbacks(holder.getCallbacks()); + } + segmentsToLoad.clear(); + + queuedSize.set(0L); + failedAssignCount.set(0); + processingExecutor.shutdown(); + callBackExecutor.shutdown(); + checkNodeRemovedExecutor.shutdown(); + } + + private void entryRemoved(SegmentHolder segmentHolder, String path) + { + if (!ZKPaths.getNodeFromPath(path).equals(segmentHolder.getSegmentId().toString())) { + log.warn( + "Server[%s] entry [%s] was removed even though it's not what is currently loading[%s]", + basePath, path, segmentHolder + ); + return; + } + actionCompleted(segmentHolder); + log.info("Server[%s] done processing [%s]", basePath, path); + } + + private void failAssign(SegmentHolder segmentHolder, Exception e) + { + log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, segmentHolder); + failedAssignCount.getAndIncrement(); + // Act like it was completed so that the coordinator gives it to someone else + actionCompleted(segmentHolder); + } + + + private static class SegmentHolder + { + private final DataSegment segment; + private final DataSegmentChangeRequest changeRequest; + private final int type; + private final List callbacks = new ArrayList<>(); + + private SegmentHolder(DataSegment segment, int type, Collection callbacks) + { + this.segment = segment; + this.type = type; + this.changeRequest = (type == LOAD) + ? new SegmentChangeRequestLoad(segment) + : new SegmentChangeRequestDrop(segment); + this.callbacks.addAll(callbacks); + } + + public DataSegment getSegment() + { + return segment; + } + + public int getType() + { + return type; + } + + public SegmentId getSegmentId() + { + return segment.getId(); + } + + public long getSegmentSize() + { + return segment.getSize(); + } + + public void addCallbacks(Collection newCallbacks) + { + synchronized (callbacks) { + callbacks.addAll(newCallbacks); + } + } + + public void addCallback(LoadPeonCallback newCallback) + { + synchronized (callbacks) { + callbacks.add(newCallback); + } + } + + public List getCallbacks() + { + synchronized (callbacks) { + return callbacks; + } + } + + public DataSegmentChangeRequest getChangeRequest() + { + return changeRequest; + } + + @Override + public String toString() + { + return changeRequest.toString(); + } + } + + private void executeCallbacks(List callbacks) + { + // Create a copy of the passed list since it can be concurrently modified by another thread when + // segmentHolder.addCallBack() is called + for (LoadPeonCallback callback : ImmutableList.copyOf(callbacks)) { + if (callback != null) { + callBackExecutor.submit(() -> callback.execute()); + } + } + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java index 32eb87c1f675..d9bd1cc890ab 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java @@ -75,6 +75,10 @@ public String getLoadQueuePeonType() return "curator"; } + @Config("druid.coordinator.curator.create.zknode.numThreads") + @Default("20") + public abstract int getCreateZkNodeNumThreads(); + @Config("druid.coordinator.loadqueuepeon.http.repeatDelay") public Duration getHttpLoadQueuePeonRepeatDelay() { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/LoadQueueTaskMaster.java b/server/src/main/java/org/apache/druid/server/coordinator/LoadQueueTaskMaster.java index 30263094a3d8..a20c3a700806 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/LoadQueueTaskMaster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/LoadQueueTaskMaster.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.utils.ZKPaths; import org.apache.druid.client.ImmutableDruidServer; @@ -27,6 +28,7 @@ import org.apache.druid.server.initialization.ZkPathsConfig; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; /** @@ -66,11 +68,15 @@ public LoadQueuePeon giveMePeon(ImmutableDruidServer server) if ("http".equalsIgnoreCase(config.getLoadQueuePeonType())) { return new HttpLoadQueuePeon(server.getURL(), jsonMapper, httpClient, config, peonExec, callbackExec); } else { - return new CuratorLoadQueuePeon( + return new CuratorLoadQueuePeonV2( curator, ZKPaths.makePath(zkPaths.getLoadQueuePath(), server.getName()), jsonMapper, - peonExec, + Executors.newFixedThreadPool( + config.getCreateZkNodeNumThreads(), + new ThreadFactoryBuilder().setNameFormat( + "CuratorLoadQueuePeon--%d").build() + ), callbackExec, config ); diff --git a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java index 1be65cb86ac8..9bdd84ed70fb 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java @@ -133,7 +133,8 @@ public void removeSegment(DataSegment s, DataSegmentChangeCallback callback) jsonMapper, zkPaths, me, - curator + curator, + new SegmentLoaderConfig() ); zkCoordinator.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java index 25dce8cf577f..1ab02665f6a8 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java @@ -162,7 +162,8 @@ public void setUp() throws Exception null, false, false, - new Duration("PT0s") + new Duration("PT0s"), + 1 ); sourceLoadQueueChildrenCache = new PathChildrenCache( curator, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 1f4f7e2844b2..04581a13c06b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -139,7 +139,8 @@ public void setUp() throws Exception null, false, false, - new Duration("PT0s") + new Duration("PT0s"), + 1 ); pathChildrenCache = new PathChildrenCache( curator, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java index 0f73695c167f..6a81121db1fe 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java @@ -84,7 +84,8 @@ public class HttpLoadQueuePeonTest null, false, false, - Duration.ZERO + Duration.ZERO, + 1 ) { @Override diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java index 437b2e47cea7..e97b97d90017 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java @@ -88,7 +88,7 @@ public void testMultipleLoadDropSegments() throws Exception jsonMapper, Execs.scheduledSingleThreaded("test_load_queue_peon_scheduled-%d"), Execs.singleThreaded("test_load_queue_peon-%d"), - new TestDruidCoordinatorConfig(null, null, null, null, null, null, 10, null, false, false, Duration.ZERO) + new TestDruidCoordinatorConfig(null, null, null, null, null, null, 10, null, false, false, Duration.ZERO, 1) ); loadQueuePeon.start(); @@ -294,7 +294,8 @@ public void testFailAssign() throws Exception Execs.scheduledSingleThreaded("test_load_queue_peon_scheduled-%d"), Execs.singleThreaded("test_load_queue_peon-%d"), // set time-out to 1 ms so that LoadQueuePeon will fail the assignment quickly - new TestDruidCoordinatorConfig(null, null, null, new Duration(1), null, null, 10, null, false, false, new Duration("PT1s")) + new TestDruidCoordinatorConfig(null, null, null, new Duration(1), null, null, 10, null, false, false, + new Duration("PT1s"), 1) ); loadQueuePeon.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java index 317f2fe67118..ddab290daa1d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java @@ -34,6 +34,7 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig private final int coordinatorKillMaxSegments; private final String consoleStatic; + private final int createZkNodeNumThreads; public TestDruidCoordinatorConfig( Duration coordinatorStartDelay, @@ -46,7 +47,8 @@ public TestDruidCoordinatorConfig( String consoleStatic, boolean mergeSegments, boolean convertSegments, - Duration getLoadQueuePeonRepeatDelay + Duration getLoadQueuePeonRepeatDelay, + int createZkNodeNumThreads ) { this.coordinatorStartDelay = coordinatorStartDelay; @@ -58,6 +60,7 @@ public TestDruidCoordinatorConfig( this.coordinatorKillMaxSegments = coordinatorKillMaxSegments; this.consoleStatic = consoleStatic; this.getLoadQueuePeonRepeatDelay = getLoadQueuePeonRepeatDelay; + this.createZkNodeNumThreads = createZkNodeNumThreads; } @Override @@ -112,4 +115,10 @@ public String getConsoleStatic() { return getLoadQueuePeonRepeatDelay; } + + @Override + public int getCreateZkNodeNumThreads() + { + return createZkNodeNumThreads; + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java index 8593e301b972..8c3bbfa4defe 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java @@ -113,7 +113,8 @@ private void testFindIntervalForKillTask(List segmentManagerResult, In null, false, false, - Duration.ZERO + Duration.ZERO, + 1 ) ); From e220ed882a84d93b1fcda789635c991d494475ce Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Sat, 23 Feb 2019 13:10:50 -0800 Subject: [PATCH 02/22] Create zk nodes in batches. Address code review comments. Introduce various configs. --- .../server/coordination/ZkCoordinator.java | 42 +- .../coordinator/CuratorLoadQueuePeon.java | 353 ++++++++------- .../coordinator/CuratorLoadQueuePeonV2.java | 418 ------------------ .../coordinator/DruidCoordinatorConfig.java | 32 +- .../coordinator/LoadQueueTaskMaster.java | 10 +- .../CuratorDruidCoordinatorTest.java | 3 +- .../coordinator/DruidCoordinatorTest.java | 3 +- .../coordinator/HttpLoadQueuePeonTest.java | 3 +- .../server/coordinator/LoadQueuePeonTest.java | 4 +- .../TestDruidCoordinatorConfig.java | 10 +- .../DruidCoordinatorSegmentKillerTest.java | 3 +- .../org/apache/druid/cli/CliCoordinator.java | 16 +- 12 files changed, 250 insertions(+), 647 deletions(-) delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeonV2.java diff --git a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java index 98883d3508ed..d08f0c14a71a 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Inject; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.ChildData; @@ -38,7 +37,6 @@ import java.io.IOException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; /** * Use {@link org.apache.druid.server.coordinator.HttpLoadQueuePeon} for segment load/drops. @@ -58,7 +56,7 @@ public class ZkCoordinator private volatile PathChildrenCache loadQueueCache; private volatile boolean started = false; - private final ExecutorService loadSegmentService; + private final ExecutorService segmentLoadUnloadService; @Inject public ZkCoordinator( @@ -75,11 +73,8 @@ public ZkCoordinator( this.zkPaths = zkPaths; this.me = me; this.curator = curator; - this.loadSegmentService = Executors.newFixedThreadPool( - config.getNumLoadingThreads(), - new ThreadFactoryBuilder().setNameFormat( - "ZKCoordinator-LoadSegment--%d").build() - ); + this.segmentLoadUnloadService = Execs.multiThreaded( + config.getNumLoadingThreads(), "ZKCoordinator--%d"); } @LifecycleStart @@ -115,18 +110,18 @@ public void start() throws IOException @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) { - loadSegmentService.submit(() -> { - final ChildData child = event.getData(); - switch (event.getType()) { - case CHILD_ADDED: + final ChildData child = event.getData(); + switch (event.getType()) { + case CHILD_ADDED: + log.info("Child zNode added at [%s]", event.getData().getPath()); + segmentLoadUnloadService.submit(() -> { final String path = child.getPath(); DataSegmentChangeRequest request = new SegmentChangeRequestNoop(); try { - request = jsonMapper.readValue( + final DataSegmentChangeRequest finalRequest = jsonMapper.readValue( child.getData(), DataSegmentChangeRequest.class ); - final DataSegmentChangeRequest finalRequest = request; - log.info("New request[%s] with zNode[%s].", request.asString(), path); + log.info("Starting request[%s] with zNode[%s]", finalRequest.asString(), path); finalRequest.go( dataSegmentChangeHandler, @@ -166,15 +161,14 @@ public void execute() .addData("nodeProperties", request) .emit(); } - - break; - case CHILD_REMOVED: - log.info("zNode[%s] was removed", event.getData().getPath()); - break; - default: - log.info("Ignoring event[%s]", event); - } - }); + }); + break; + case CHILD_REMOVED: + log.info("zNode[%s] was removed", event.getData().getPath()); + break; + default: + log.info("Ignoring event[%s]", event); + } } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index a6b10bc345c4..d79faa9199ae 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -21,19 +21,20 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.api.CuratorWatcher; import org.apache.curator.utils.ZKPaths; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.SegmentChangeRequestDrop; import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordination.SegmentChangeRequestNoop; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; import java.util.ArrayList; @@ -41,10 +42,15 @@ import java.util.Collections; import java.util.List; import java.util.Set; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -59,19 +65,11 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon private static final int DROP = 0; private static final int LOAD = 1; - private static void executeCallbacks(List callbacks) - { - for (LoadPeonCallback callback : callbacks) { - if (callback != null) { - callback.execute(); - } - } - } - private final CuratorFramework curator; private final String basePath; private final ObjectMapper jsonMapper; private final ScheduledExecutorService processingExecutor; + private final ScheduledExecutorService checkNodeRemovedExecutor; private final ExecutorService callBackExecutor; private final DruidCoordinatorConfig config; @@ -88,10 +86,10 @@ private static void executeCallbacks(List callbacks) DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST ); - private final Object lock = new Object(); - - private volatile SegmentHolder currentlyProcessing = null; - private boolean stopped = false; + private final int numProcessingQueues; + private final LinkedBlockingQueue[] segmentProcessingQueues; + // Max number of segments to process in every round + private final int batchSizePerTask; CuratorLoadQueuePeon( CuratorFramework curator, @@ -108,6 +106,28 @@ private static void executeCallbacks(List callbacks) this.callBackExecutor = callbackExecutor; this.processingExecutor = processingExecutor; this.config = config; + this.checkNodeRemovedExecutor = + Executors.newScheduledThreadPool( + config.getCuratorNumMonitorThreads(), + Execs.makeThreadFactory("LoadQueuePeon") + ); + // One processing queue per thread + Preconditions.checkArgument(config.getCreateZkNodeNumThreads() > 0); + this.numProcessingQueues = config.getCreateZkNodeNumThreads(); + + Preconditions.checkArgument(config.getCuratorCreateZkNodeBatchSize() > 0); + this.batchSizePerTask = (config.getCuratorCreateZkNodeBatchSize() / numProcessingQueues); + + this.segmentProcessingQueues = new LinkedBlockingQueue[numProcessingQueues]; + for (int i = 0; i < numProcessingQueues; i++) { + segmentProcessingQueues[i] = new LinkedBlockingQueue<>(); + processingExecutor.scheduleAtFixedRate( + new SegmentChangeProcessor(segmentProcessingQueues[i]), + 0, + config.getCuratorCreateZkNodesRepeatDelay().getMillis(), + TimeUnit.MILLISECONDS + ); + } } @JsonProperty @@ -152,29 +172,19 @@ public int getNumberOfSegmentsInQueue() @Override public void loadSegment(final DataSegment segment, final LoadPeonCallback callback) { - synchronized (lock) { - if ((currentlyProcessing != null) && - currentlyProcessing.getSegmentId().equals(segment.getId())) { - if (callback != null) { - currentlyProcessing.addCallback(callback); - } - return; - } - } - synchronized (lock) { - final SegmentHolder existingHolder = segmentsToLoad.get(segment); - if (existingHolder != null) { - if ((callback != null)) { - existingHolder.addCallback(callback); - } - return; + final SegmentHolder existingHolder = segmentsToLoad.get(segment); + if (existingHolder != null) { + if ((callback != null)) { + existingHolder.addCallback(callback); } + return; } - log.debug("Asking server peon[%s] to load segment[%s]", basePath, segment.getId()); queuedSize.addAndGet(segment.getSize()); - segmentsToLoad.put(segment, new SegmentHolder(segment, LOAD, Collections.singletonList(callback))); + SegmentHolder segmentHolder = new SegmentHolder(segment, LOAD, Collections.singletonList(callback)); + segmentsToLoad.put(segment, segmentHolder); + segmentProcessingQueues[getSegmentQueueIndex()].offer(segmentHolder); } @Override @@ -183,28 +193,17 @@ public void dropSegment( final LoadPeonCallback callback ) { - synchronized (lock) { - if ((currentlyProcessing != null) && - currentlyProcessing.getSegmentId().equals(segment.getId())) { - if (callback != null) { - currentlyProcessing.addCallback(callback); - } - return; - } - } - - synchronized (lock) { - final SegmentHolder existingHolder = segmentsToDrop.get(segment); - if (existingHolder != null) { - if (callback != null) { - existingHolder.addCallback(callback); - } - return; + final SegmentHolder existingHolder = segmentsToDrop.get(segment); + if (existingHolder != null) { + if (callback != null) { + existingHolder.addCallback(callback); } + return; } - log.debug("Asking server peon[%s] to drop segment[%s]", basePath, segment.getId()); - segmentsToDrop.put(segment, new SegmentHolder(segment, DROP, Collections.singletonList(callback))); + SegmentHolder segmentHolder = new SegmentHolder(segment, DROP, Collections.singletonList(callback)); + segmentsToDrop.put(segment, segmentHolder); + segmentProcessingQueues[getSegmentQueueIndex()].offer(segmentHolder); } @Override @@ -219,42 +218,64 @@ public void unmarkSegmentToDrop(DataSegment dataSegment) segmentsMarkedToDrop.remove(dataSegment); } - private void processSegmentChangeRequest() + private int getSegmentQueueIndex() { - if (currentlyProcessing != null) { - log.debug( - "Server[%s] skipping processSegmentChangeRequest because something is currently loading[%s].", - basePath, - currentlyProcessing.getSegmentId() - ); + return ThreadLocalRandom.current().nextInt(0, numProcessingQueues); + } - return; + private class SegmentChangeProcessor implements Runnable + { + private final BlockingQueue segmentQueue; + + private SegmentChangeProcessor(BlockingQueue segmentQueue) + { + this.segmentQueue = segmentQueue; } - if (!segmentsToDrop.isEmpty()) { - currentlyProcessing = segmentsToDrop.firstEntry().getValue(); - log.debug("Server[%s] dropping [%s]", basePath, currentlyProcessing.getSegmentId()); - } else if (!segmentsToLoad.isEmpty()) { - currentlyProcessing = segmentsToLoad.firstEntry().getValue(); - log.debug("Server[%s] loading [%s]", basePath, currentlyProcessing.getSegmentId()); - } else { - return; + @Override + public void run() + { + try { + int numProcessed = 0; + while (numProcessed++ < batchSizePerTask && !segmentQueue.isEmpty()) { + // Instead of calling poll for every element, drain the batch to a list. + List batch = new ArrayList<>(batchSizePerTask); + segmentQueue.drainTo(batch, batchSizePerTask); + for (SegmentHolder s : batch) { + processSegmentChangeRequest(s); + } + } + } + catch (Throwable e) { + // Swallow all errors so that the executor thread doesn't die + log.error(e, "Throwable caught and ignored when processing segments from the queue"); + } } + } + private void processSegmentChangeRequest(SegmentHolder segmentHolder) + { try { - final String path = ZKPaths.makePath(basePath, currentlyProcessing.getSegmentId().toString()); - final byte[] payload = jsonMapper.writeValueAsBytes(currentlyProcessing.getChangeRequest()); + final String path = ZKPaths.makePath(basePath, segmentHolder.getSegmentIdentifier()); + final byte[] payload = jsonMapper.writeValueAsBytes(segmentHolder.getChangeRequest()); curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, payload); - - processingExecutor.schedule( + log.debug( + "ZKNode created for server to [%s] %s [%s]", + basePath, + segmentHolder.getType() == LOAD ? "load" : "drop", + segmentHolder.getSegmentIdentifier() + ); + final ScheduledFuture future = checkNodeRemovedExecutor.schedule( () -> { try { if (curator.checkExists().forPath(path) != null) { - failAssign(new ISE("%s was never removed! Failing this operation!", path)); + failAssign(segmentHolder, new ISE("%s was never removed! Failing this operation!", path)); + } else { + log.debug("%s detected to be removed. ", path); } } catch (Exception e) { - failAssign(e); + failAssign(segmentHolder, e); } }, config.getLoadTimeoutDelay().getMillis(), @@ -265,7 +286,10 @@ private void processSegmentChangeRequest() (CuratorWatcher) watchedEvent -> { switch (watchedEvent.getType()) { case NodeDeleted: - entryRemoved(watchedEvent.getPath()); + // Cancel the check node deleted task since we have already + // been notified by the zk watcher + future.cancel(true); + entryRemoved(segmentHolder, watchedEvent.getPath()); break; default: // do nothing @@ -290,120 +314,98 @@ private void processSegmentChangeRequest() // We do not create the existence watcher first, because then it will fire when we create the // node and we'll have the same race when trying to refresh that watcher. curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, noopPayload); - - entryRemoved(path); + entryRemoved(segmentHolder, path); } } + catch (KeeperException.NodeExistsException ne) { + // This is expected when historicals haven't yet picked up processing this segment and coordinator + // tries reassigning it to the same node. + log.warn(ne, "ZK node already exists because segment change request hasn't yet been processed"); + failAssign(segmentHolder); + } catch (Exception e) { - failAssign(e); + failAssign(segmentHolder, e); } } - private void actionCompleted() + private void actionCompleted(SegmentHolder segmentHolder) { - if (currentlyProcessing != null) { - switch (currentlyProcessing.getType()) { - case LOAD: - segmentsToLoad.remove(currentlyProcessing.getSegment()); - queuedSize.addAndGet(-currentlyProcessing.getSegmentSize()); - break; - case DROP: - segmentsToDrop.remove(currentlyProcessing.getSegment()); - break; - default: - throw new UnsupportedOperationException(); - } - - final List callbacks = currentlyProcessing.getCallbacks(); - currentlyProcessing = null; - callBackExecutor.execute( - () -> executeCallbacks(callbacks) - ); + switch (segmentHolder.getType()) { + case LOAD: + segmentsToLoad.remove(segmentHolder.getSegment()); + queuedSize.addAndGet(-segmentHolder.getSegmentSize()); + break; + case DROP: + segmentsToDrop.remove(segmentHolder.getSegment()); + break; + default: + throw new UnsupportedOperationException(); } + + callBackExecutor.execute( + () -> executeCallbacks(segmentHolder) + ); } + @Override public void start() - { - ScheduledExecutors.scheduleAtFixedRate( - processingExecutor, - config.getLoadQueuePeonRepeatDelay(), - config.getLoadQueuePeonRepeatDelay(), - () -> { - processSegmentChangeRequest(); - - if (stopped) { - return ScheduledExecutors.Signal.STOP; - } else { - return ScheduledExecutors.Signal.REPEAT; - } - } - ); - } + { } @Override public void stop() { - synchronized (lock) { - if (currentlyProcessing != null) { - executeCallbacks(currentlyProcessing.getCallbacks()); - currentlyProcessing = null; - } - - if (!segmentsToDrop.isEmpty()) { - for (SegmentHolder holder : segmentsToDrop.values()) { - executeCallbacks(holder.getCallbacks()); - } - } - segmentsToDrop.clear(); - - if (!segmentsToLoad.isEmpty()) { - for (SegmentHolder holder : segmentsToLoad.values()) { - executeCallbacks(holder.getCallbacks()); - } - } - segmentsToLoad.clear(); + for (SegmentHolder holder : segmentsToDrop.values()) { + executeCallbacks(holder); + } + segmentsToDrop.clear(); - queuedSize.set(0L); - failedAssignCount.set(0); - stopped = true; + for (SegmentHolder holder : segmentsToLoad.values()) { + executeCallbacks(holder); } + segmentsToLoad.clear(); + + queuedSize.set(0L); + failedAssignCount.set(0); + processingExecutor.shutdown(); + callBackExecutor.shutdown(); + checkNodeRemovedExecutor.shutdown(); } - private void entryRemoved(String path) + private void entryRemoved(SegmentHolder segmentHolder, String path) { - synchronized (lock) { - if (currentlyProcessing == null) { - log.warn("Server[%s] an entry[%s] was removed even though it wasn't loading!?", basePath, path); - return; - } - if (!ZKPaths.getNodeFromPath(path).equals(currentlyProcessing.getSegmentId().toString())) { - log.warn( - "Server[%s] entry [%s] was removed even though it's not what is currently loading[%s]", - basePath, path, currentlyProcessing - ); - return; - } - log.debug( - "Server[%s] done processing %s of segment [%s]", - basePath, - currentlyProcessing.getType() == LOAD ? "load" : "drop", - path + if (!ZKPaths.getNodeFromPath(path).equals(segmentHolder.getSegmentIdentifier())) { + log.warn( + "Server[%s] entry [%s] was removed even though it's not what is currently loading[%s]", + basePath, path, segmentHolder ); - actionCompleted(); + return; } + actionCompleted(segmentHolder); + log.debug( + "Server[%s] done processing %s of segment [%s]", + basePath, + segmentHolder.getType() == LOAD ? "load" : "drop", + path + ); } - private void failAssign(Exception e) + private void failAssign(SegmentHolder segmentHolder) { - synchronized (lock) { - log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, currentlyProcessing); - failedAssignCount.getAndIncrement(); - // Act like it was completed so that the coordinator gives it to someone else - actionCompleted(); + failAssign(segmentHolder, null); + } + + private void failAssign(SegmentHolder segmentHolder, Exception e) + { + if (e != null) { + log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, segmentHolder); } + failedAssignCount.getAndIncrement(); + // Act like it was completed so that the coordinator gives it to someone else + actionCompleted(segmentHolder); } + private static class SegmentHolder { private final DataSegment segment; @@ -411,7 +413,11 @@ private static class SegmentHolder private final int type; private final List callbacks = new ArrayList<>(); - private SegmentHolder(DataSegment segment, int type, Collection callbacks) + private SegmentHolder( + DataSegment segment, + int type, + Collection callbacks + ) { this.segment = segment; this.type = type; @@ -431,9 +437,9 @@ public int getType() return type; } - public SegmentId getSegmentId() + public String getSegmentIdentifier() { - return segment.getId(); + return segment.getId().toString(); } public long getSegmentSize() @@ -441,13 +447,6 @@ public long getSegmentSize() return segment.getSize(); } - public void addCallbacks(Collection newCallbacks) - { - synchronized (callbacks) { - callbacks.addAll(newCallbacks); - } - } - public void addCallback(LoadPeonCallback newCallback) { synchronized (callbacks) { @@ -455,10 +454,11 @@ public void addCallback(LoadPeonCallback newCallback) } } - public List getCallbacks() + LoadPeonCallback[] getCallbacks() { synchronized (callbacks) { - return callbacks; + // Return a copy so that callers get a consistent view + return callbacks.toArray(new LoadPeonCallback[0]); } } @@ -473,4 +473,13 @@ public String toString() return changeRequest.toString(); } } + + private void executeCallbacks(SegmentHolder holder) + { + for (LoadPeonCallback callback : holder.getCallbacks()) { + if (callback != null) { + callBackExecutor.submit(() -> callback.execute()); + } + } + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeonV2.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeonV2.java deleted file mode 100644 index 332caa489ca1..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeonV2.java +++ /dev/null @@ -1,418 +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.coordinator; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.api.CuratorWatcher; -import org.apache.curator.utils.ZKPaths; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.coordination.DataSegmentChangeRequest; -import org.apache.druid.server.coordination.SegmentChangeRequestDrop; -import org.apache.druid.server.coordination.SegmentChangeRequestLoad; -import org.apache.druid.server.coordination.SegmentChangeRequestNoop; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.data.Stat; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; - -/** - * Use {@link HttpLoadQueuePeon} instead. - */ -@Deprecated -public class CuratorLoadQueuePeonV2 extends LoadQueuePeon -{ - private static final EmittingLogger log = new EmittingLogger(CuratorLoadQueuePeon.class); - private static final int DROP = 0; - private static final int LOAD = 1; - private final CuratorFramework curator; - private final String basePath; - private final ObjectMapper jsonMapper; - private final ExecutorService processingExecutor; - private final ScheduledExecutorService checkNodeRemovedExecutor; - private final ExecutorService callBackExecutor; - private final DruidCoordinatorConfig config; - - private final AtomicLong queuedSize = new AtomicLong(0); - private final AtomicInteger failedAssignCount = new AtomicInteger(0); - - private final ConcurrentSkipListMap segmentsToLoad = new ConcurrentSkipListMap<>( - DruidCoordinator.SEGMENT_COMPARATOR - ); - private final ConcurrentSkipListMap segmentsToDrop = new ConcurrentSkipListMap<>( - DruidCoordinator.SEGMENT_COMPARATOR - ); - private final ConcurrentSkipListSet segmentsMarkedToDrop = new ConcurrentSkipListSet<>( - DruidCoordinator.SEGMENT_COMPARATOR - ); - - private static final int MAX_TASK_WAIT = 100; - - CuratorLoadQueuePeonV2( - CuratorFramework curator, - String basePath, - ObjectMapper jsonMapper, - ExecutorService processingExecutor, - ExecutorService callbackExecutor, - DruidCoordinatorConfig config - ) - { - this.curator = curator; - this.basePath = basePath; - this.jsonMapper = jsonMapper; - this.callBackExecutor = callbackExecutor; - this.processingExecutor = processingExecutor; - this.config = config; - //TODO: samarth configure the size of this pool - this.checkNodeRemovedExecutor = Executors.newScheduledThreadPool(5, new ThreadFactoryBuilder().setNameFormat( - "ZKNodeDeletionChecker--%d").build()); - } - - @JsonProperty - @Override - public Set getSegmentsToLoad() - { - return segmentsToLoad.keySet(); - } - - @JsonProperty - @Override - public Set getSegmentsToDrop() - { - return segmentsToDrop.keySet(); - } - - @JsonProperty - @Override - public Set getSegmentsMarkedToDrop() - { - return segmentsMarkedToDrop; - } - - @Override - public long getLoadQueueSize() - { - return queuedSize.get(); - } - - @Override - public int getAndResetFailedAssignCount() - { - return failedAssignCount.getAndSet(0); - } - - @Override - public int getNumberOfSegmentsInQueue() - { - return segmentsToLoad.size(); - } - - @Override - public void loadSegment( - final DataSegment segment, - final LoadPeonCallback callback - ) - { - - final SegmentHolder existingHolder = segmentsToLoad.get(segment); - if (existingHolder != null) { - if ((callback != null)) { - existingHolder.addCallback(callback); - } - return; - } - log.info("Asking server peon[%s] to load segment[%s]", basePath, segment.getId()); - queuedSize.addAndGet(segment.getSize()); - SegmentHolder holder = new SegmentHolder(segment, LOAD, Collections.singletonList(callback)); - segmentsToLoad.put(segment, holder); - processingExecutor.submit((Callable) () -> { - processSegmentChangeRequest(holder); - return null; - }); - } - - @Override - public void dropSegment( - final DataSegment segment, - final LoadPeonCallback callback - ) - { - final SegmentHolder existingHolder = segmentsToDrop.get(segment); - if (existingHolder != null) { - if (callback != null) { - existingHolder.addCallback(callback); - } - return; - } - log.info("Asking server peon[%s] to drop segment[%s]", basePath, segment.getId()); - SegmentHolder segmentHolder = new SegmentHolder(segment, DROP, Collections.singletonList(callback)); - segmentsToDrop.put(segment, segmentHolder); - processingExecutor.submit((Callable) () -> { - processSegmentChangeRequest(segmentHolder); - return null; - }); - } - - @Override - public void markSegmentToDrop(DataSegment dataSegment) - { - segmentsMarkedToDrop.add(dataSegment); - } - - @Override - public void unmarkSegmentToDrop(DataSegment dataSegment) - { - segmentsMarkedToDrop.remove(dataSegment); - } - - private void processSegmentChangeRequest(SegmentHolder segmentHolder) - { - try { - // Wait for a random duration since we do not want zookeeper getting pounded with create node requests - TimeUnit.MILLISECONDS.sleep(ThreadLocalRandom.current().nextInt(MAX_TASK_WAIT)); - log.info("Server[%s] %s [%s]", basePath, segmentHolder.getType() == LOAD ? - "loading" : "dropping", segmentHolder.getSegmentId().toString()); - final String path = ZKPaths.makePath(basePath, segmentHolder.getSegmentId().toString()); - final byte[] payload = jsonMapper.writeValueAsBytes(segmentHolder.getChangeRequest()); - curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, payload); - - checkNodeRemovedExecutor.schedule( - () -> { - try { - if (curator.checkExists().forPath(path) != null) { - failAssign(segmentHolder, new ISE("%s was never removed! Failing this operation!", path)); - } else { - log.info("%s detected to be removed. ", path); - } - } - catch (Exception e) { - failAssign(segmentHolder, e); - } - }, - config.getLoadTimeoutDelay().getMillis(), - TimeUnit.MILLISECONDS - ); - - final Stat stat = curator.checkExists().usingWatcher( - (CuratorWatcher) watchedEvent -> { - switch (watchedEvent.getType()) { - case NodeDeleted: - entryRemoved(segmentHolder, watchedEvent.getPath()); - break; - default: - // do nothing - } - } - ).forPath(path); - - if (stat == null) { - final byte[] noopPayload = jsonMapper.writeValueAsBytes(new SegmentChangeRequestNoop()); - - // Create a node and then delete it to remove the registered watcher. This is a work-around for - // a zookeeper race condition. Specifically, when you set a watcher, it fires on the next event - // that happens for that node. If no events happen, the watcher stays registered foreverz. - // Couple that with the fact that you cannot set a watcher when you create a node, but what we - // want is to create a node and then watch for it to get deleted. The solution is that you *can* - // set a watcher when you check to see if it exists so, we first create the node and then set a - // watcher on its existence. However, if already does not exist by the time the existence check - // returns, then the watcher that was set will never fire (nobody will ever create the node - // again) and thus lead to a slow, but real, memory leak. So, we create another node to cause - // that watcher to fire and delete it right away. - // - // We do not create the existence watcher first, because then it will fire when we create the - // node and we'll have the same race when trying to refresh that watcher. - curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, noopPayload); - entryRemoved(segmentHolder, path); - } - } - catch (Exception e) { - failAssign(segmentHolder, e); - } - } - - private void actionCompleted(SegmentHolder segmentHolder) - { - log.info("Action completed: " + segmentHolder.getType()); - switch (segmentHolder.getType()) { - case LOAD: - segmentsToLoad.remove(segmentHolder.getSegment()); - queuedSize.addAndGet(-segmentHolder.getSegmentSize()); - break; - case DROP: - segmentsToDrop.remove(segmentHolder.getSegment()); - break; - default: - throw new UnsupportedOperationException(); - } - - final List callbacks = segmentHolder.getCallbacks(); - callBackExecutor.execute( - () -> executeCallbacks(callbacks) - ); - } - - - @Override - public void start() - {} - - @Override - public void stop() - { - for (SegmentHolder holder : segmentsToDrop.values()) { - executeCallbacks(holder.getCallbacks()); - } - segmentsToDrop.clear(); - - for (SegmentHolder holder : segmentsToLoad.values()) { - executeCallbacks(holder.getCallbacks()); - } - segmentsToLoad.clear(); - - queuedSize.set(0L); - failedAssignCount.set(0); - processingExecutor.shutdown(); - callBackExecutor.shutdown(); - checkNodeRemovedExecutor.shutdown(); - } - - private void entryRemoved(SegmentHolder segmentHolder, String path) - { - if (!ZKPaths.getNodeFromPath(path).equals(segmentHolder.getSegmentId().toString())) { - log.warn( - "Server[%s] entry [%s] was removed even though it's not what is currently loading[%s]", - basePath, path, segmentHolder - ); - return; - } - actionCompleted(segmentHolder); - log.info("Server[%s] done processing [%s]", basePath, path); - } - - private void failAssign(SegmentHolder segmentHolder, Exception e) - { - log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, segmentHolder); - failedAssignCount.getAndIncrement(); - // Act like it was completed so that the coordinator gives it to someone else - actionCompleted(segmentHolder); - } - - - private static class SegmentHolder - { - private final DataSegment segment; - private final DataSegmentChangeRequest changeRequest; - private final int type; - private final List callbacks = new ArrayList<>(); - - private SegmentHolder(DataSegment segment, int type, Collection callbacks) - { - this.segment = segment; - this.type = type; - this.changeRequest = (type == LOAD) - ? new SegmentChangeRequestLoad(segment) - : new SegmentChangeRequestDrop(segment); - this.callbacks.addAll(callbacks); - } - - public DataSegment getSegment() - { - return segment; - } - - public int getType() - { - return type; - } - - public SegmentId getSegmentId() - { - return segment.getId(); - } - - public long getSegmentSize() - { - return segment.getSize(); - } - - public void addCallbacks(Collection newCallbacks) - { - synchronized (callbacks) { - callbacks.addAll(newCallbacks); - } - } - - public void addCallback(LoadPeonCallback newCallback) - { - synchronized (callbacks) { - callbacks.add(newCallback); - } - } - - public List getCallbacks() - { - synchronized (callbacks) { - return callbacks; - } - } - - public DataSegmentChangeRequest getChangeRequest() - { - return changeRequest; - } - - @Override - public String toString() - { - return changeRequest.toString(); - } - } - - private void executeCallbacks(List callbacks) - { - // Create a copy of the passed list since it can be concurrently modified by another thread when - // segmentHolder.addCallBack() is called - for (LoadPeonCallback callback : ImmutableList.copyOf(callbacks)) { - if (callback != null) { - callBackExecutor.submit(() -> callback.execute()); - } - } - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java index d9bd1cc890ab..34b73e849de3 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java @@ -75,9 +75,35 @@ public String getLoadQueuePeonType() return "curator"; } - @Config("druid.coordinator.curator.create.zknode.numThreads") - @Default("20") - public abstract int getCreateZkNodeNumThreads(); + @Config("druid.coordinator.loadqueuepeon.curator.numCreateThreads") + public int getCreateZkNodeNumThreads() + { + return 10; + } + + @Config("druid.coordinator.loadqueuepeon.curator.numCallbackThreads") + public int getCuratorNumCallBackThreads() + { + return 10; + } + + @Config("druid.coordinator.loadqueuepeon.curator.numMonitorThreads") + public int getCuratorNumMonitorThreads() + { + return 1; + } + + @Config("druid.coordinator.curator.create.zknode.batchSize") + public int getCuratorCreateZkNodeBatchSize() + { + return 5000; + } + + @Config("druid.coordinator.curator.create.zknode.repeatDelay") + public Duration getCuratorCreateZkNodesRepeatDelay() + { + return Duration.millis(60000); + } @Config("druid.coordinator.loadqueuepeon.http.repeatDelay") public Duration getHttpLoadQueuePeonRepeatDelay() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/LoadQueueTaskMaster.java b/server/src/main/java/org/apache/druid/server/coordinator/LoadQueueTaskMaster.java index a20c3a700806..30263094a3d8 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/LoadQueueTaskMaster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/LoadQueueTaskMaster.java @@ -20,7 +20,6 @@ package org.apache.druid.server.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.utils.ZKPaths; import org.apache.druid.client.ImmutableDruidServer; @@ -28,7 +27,6 @@ import org.apache.druid.server.initialization.ZkPathsConfig; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; /** @@ -68,15 +66,11 @@ public LoadQueuePeon giveMePeon(ImmutableDruidServer server) if ("http".equalsIgnoreCase(config.getLoadQueuePeonType())) { return new HttpLoadQueuePeon(server.getURL(), jsonMapper, httpClient, config, peonExec, callbackExec); } else { - return new CuratorLoadQueuePeonV2( + return new CuratorLoadQueuePeon( curator, ZKPaths.makePath(zkPaths.getLoadQueuePath(), server.getName()), jsonMapper, - Executors.newFixedThreadPool( - config.getCreateZkNodeNumThreads(), - new ThreadFactoryBuilder().setNameFormat( - "CuratorLoadQueuePeon--%d").build() - ), + peonExec, callbackExec, config ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java index 1ab02665f6a8..25dce8cf577f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java @@ -162,8 +162,7 @@ public void setUp() throws Exception null, false, false, - new Duration("PT0s"), - 1 + new Duration("PT0s") ); sourceLoadQueueChildrenCache = new PathChildrenCache( curator, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 04581a13c06b..1f4f7e2844b2 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -139,8 +139,7 @@ public void setUp() throws Exception null, false, false, - new Duration("PT0s"), - 1 + new Duration("PT0s") ); pathChildrenCache = new PathChildrenCache( curator, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java index 6a81121db1fe..0f73695c167f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java @@ -84,8 +84,7 @@ public class HttpLoadQueuePeonTest null, false, false, - Duration.ZERO, - 1 + Duration.ZERO ) { @Override diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java index e97b97d90017..544105aca7b8 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java @@ -88,7 +88,7 @@ public void testMultipleLoadDropSegments() throws Exception jsonMapper, Execs.scheduledSingleThreaded("test_load_queue_peon_scheduled-%d"), Execs.singleThreaded("test_load_queue_peon-%d"), - new TestDruidCoordinatorConfig(null, null, null, null, null, null, 10, null, false, false, Duration.ZERO, 1) + new TestDruidCoordinatorConfig(null, null, null, null, null, null, 10, null, false, false, Duration.ZERO) ); loadQueuePeon.start(); @@ -295,7 +295,7 @@ public void testFailAssign() throws Exception Execs.singleThreaded("test_load_queue_peon-%d"), // set time-out to 1 ms so that LoadQueuePeon will fail the assignment quickly new TestDruidCoordinatorConfig(null, null, null, new Duration(1), null, null, 10, null, false, false, - new Duration("PT1s"), 1) + new Duration("PT1s")) ); loadQueuePeon.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java index ddab290daa1d..07d6be4050fb 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java @@ -34,7 +34,6 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig private final int coordinatorKillMaxSegments; private final String consoleStatic; - private final int createZkNodeNumThreads; public TestDruidCoordinatorConfig( Duration coordinatorStartDelay, @@ -47,8 +46,7 @@ public TestDruidCoordinatorConfig( String consoleStatic, boolean mergeSegments, boolean convertSegments, - Duration getLoadQueuePeonRepeatDelay, - int createZkNodeNumThreads + Duration getLoadQueuePeonRepeatDelay ) { this.coordinatorStartDelay = coordinatorStartDelay; @@ -60,7 +58,6 @@ public TestDruidCoordinatorConfig( this.coordinatorKillMaxSegments = coordinatorKillMaxSegments; this.consoleStatic = consoleStatic; this.getLoadQueuePeonRepeatDelay = getLoadQueuePeonRepeatDelay; - this.createZkNodeNumThreads = createZkNodeNumThreads; } @Override @@ -116,9 +113,4 @@ public String getConsoleStatic() return getLoadQueuePeonRepeatDelay; } - @Override - public int getCreateZkNodeNumThreads() - { - return createZkNodeNumThreads; - } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java index 8c3bbfa4defe..8593e301b972 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java @@ -113,8 +113,7 @@ private void testFindIntervalForKillTask(List segmentManagerResult, In null, false, false, - Duration.ZERO, - 1 + Duration.ZERO ) ); diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 4097e952096a..2d52d52bacab 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -46,6 +46,7 @@ import org.apache.druid.guice.annotations.CoordinatorIndexingServiceHelper; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.http.JettyHttpClientModule; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.HttpClient; @@ -90,7 +91,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Properties; -import java.util.concurrent.Executors; +import java.util.concurrent.ExecutorService; /** */ @@ -249,11 +250,20 @@ public LoadQueueTaskMaster getLoadQueueTaskMaster( ZkPathsConfig zkPaths ) { + boolean useHttpLoadQueuePeon = "http".equalsIgnoreCase(config.getLoadQueuePeonType()); + int poolSize = useHttpLoadQueuePeon ? 1 : config.getCreateZkNodeNumThreads(); + ExecutorService callBackExec; + if (useHttpLoadQueuePeon) { + callBackExec = Execs.singleThreaded("LoadQueuePeon-callbackexec--%d"); + } else { + callBackExec = Execs.multiThreaded(config.getCuratorNumCallBackThreads(), "LoadQueuePeon" + + "-callbackexec--%d"); + } return new LoadQueueTaskMaster( curator, jsonMapper, - factory.create(1, "Master-PeonExec--%d"), - Executors.newSingleThreadExecutor(), + factory.create(poolSize, "Master-PeonExec--%d"), + callBackExec, config, httpClient, zkPaths From 6be694072c590ab6923db011b6dfefaf8c7f4db0 Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Sat, 23 Feb 2019 13:40:20 -0800 Subject: [PATCH 03/22] Add documentation for the newly added configs --- docs/content/configuration/index.md | 6 +++++- .../druid/server/coordinator/DruidCoordinatorConfig.java | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index cda566996cfc..c0bb6df3de40 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -1255,7 +1255,11 @@ These Historical configurations can be defined in the `historical/runtime.proper |`druid.segmentCache.infoDir`|Historical processes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| |`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)| |`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from from deep storage.|10| -|`druid.segmentCache.numBootstrapThreads`|How many segments to load concurrently from local storage at startup.|Same as numLoadingThreads| +|`druid.coordinator.loadqueuepeon.curator.numCreateThreads`|How many threads to use for concurrently creating zk nodes corresponding to segments that need to be loaded or dropped.|10| +|`druid.coordinator.loadqueuepeon.curator.numCallbackThreads`|How many threads to use for concurrently executing callback actions associated with loading or dropping segments.|2| +|`druid.coordinator.loadqueuepeon.curator.numMonitorThreads`|How many threads to use for monitoring deletion of zk nodes corresponding to segments that need to loaded or dropped|1| +|`druid.coordinator.curator.create.zknode.batchSize`|How many zk nodes to create in one iteration.|5000| +|`druid.coordinator.curator.create.zknode.repeatDelay`|How long to wait for creating next batch of zk nodes|PT1M| In `druid.segmentCache.locations`, *freeSpacePercent* was added because *maxSize* setting is only a theoretical limit and assumes that much space will always be available for storing segments. In case of any druid bug leading to unaccounted segment files left alone on disk or some other process writing stuff to disk, This check can start failing segment loading early before filling up the disk completely and leaving the host usable otherwise. diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java index 34b73e849de3..2e1abbdbad16 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java @@ -84,7 +84,7 @@ public int getCreateZkNodeNumThreads() @Config("druid.coordinator.loadqueuepeon.curator.numCallbackThreads") public int getCuratorNumCallBackThreads() { - return 10; + return 2; } @Config("druid.coordinator.loadqueuepeon.curator.numMonitorThreads") From dcd288606d4c9055e92987db9cbd04e4227fdeae Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Mon, 25 Feb 2019 12:53:27 -0800 Subject: [PATCH 04/22] Fix test failures --- .../coordinator/CuratorLoadQueuePeon.java | 28 ++++++++++--------- .../coordinator/LoadQueuePeonTester.java | 22 ++++++++++++++- 2 files changed, 36 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index d79faa9199ae..a711c39605d2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -69,7 +69,7 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon private final String basePath; private final ObjectMapper jsonMapper; private final ScheduledExecutorService processingExecutor; - private final ScheduledExecutorService checkNodeRemovedExecutor; + private final ScheduledExecutorService monitorNodeRemovedExecutor; private final ExecutorService callBackExecutor; private final DruidCoordinatorConfig config; @@ -106,10 +106,10 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon this.callBackExecutor = callbackExecutor; this.processingExecutor = processingExecutor; this.config = config; - this.checkNodeRemovedExecutor = + this.monitorNodeRemovedExecutor = Executors.newScheduledThreadPool( config.getCuratorNumMonitorThreads(), - Execs.makeThreadFactory("LoadQueuePeon") + Execs.makeThreadFactory("LoadQueuePeon-NodeRemovedMonitor--%d") ); // One processing queue per thread Preconditions.checkArgument(config.getCreateZkNodeNumThreads() > 0); @@ -117,16 +117,9 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon Preconditions.checkArgument(config.getCuratorCreateZkNodeBatchSize() > 0); this.batchSizePerTask = (config.getCuratorCreateZkNodeBatchSize() / numProcessingQueues); - this.segmentProcessingQueues = new LinkedBlockingQueue[numProcessingQueues]; for (int i = 0; i < numProcessingQueues; i++) { segmentProcessingQueues[i] = new LinkedBlockingQueue<>(); - processingExecutor.scheduleAtFixedRate( - new SegmentChangeProcessor(segmentProcessingQueues[i]), - 0, - config.getCuratorCreateZkNodesRepeatDelay().getMillis(), - TimeUnit.MILLISECONDS - ); } } @@ -265,7 +258,7 @@ private void processSegmentChangeRequest(SegmentHolder segmentHolder) segmentHolder.getType() == LOAD ? "load" : "drop", segmentHolder.getSegmentIdentifier() ); - final ScheduledFuture future = checkNodeRemovedExecutor.schedule( + final ScheduledFuture future = monitorNodeRemovedExecutor.schedule( () -> { try { if (curator.checkExists().forPath(path) != null) { @@ -350,7 +343,16 @@ private void actionCompleted(SegmentHolder segmentHolder) @Override public void start() - { } + { + for (int i = 0; i < numProcessingQueues; i++) { + processingExecutor.scheduleAtFixedRate( + new SegmentChangeProcessor(segmentProcessingQueues[i]), + 0, + config.getCuratorCreateZkNodesRepeatDelay().getMillis(), + TimeUnit.MILLISECONDS + ); + } + } @Override public void stop() @@ -369,7 +371,7 @@ public void stop() failedAssignCount.set(0); processingExecutor.shutdown(); callBackExecutor.shutdown(); - checkNodeRemovedExecutor.shutdown(); + monitorNodeRemovedExecutor.shutdown(); } private void entryRemoved(SegmentHolder segmentHolder, String path) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java index e07f363e78a4..cae7c938589b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator; import org.apache.druid.timeline.DataSegment; +import org.joda.time.Duration; import java.util.concurrent.ConcurrentSkipListSet; @@ -29,7 +30,26 @@ public class LoadQueuePeonTester extends CuratorLoadQueuePeon public LoadQueuePeonTester() { - super(null, null, null, null, null, null); + super( + null, + null, + null, + null, + null, + new TestDruidCoordinatorConfig( + null, + null, + null, + new Duration(1), + null, + null, + 10, + null, + false, + false, + new Duration("PT1s") + ) + ); } @Override From 5e18302c6e6728670361dd3469ef778032bb7439 Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Fri, 8 Mar 2019 03:07:12 -0800 Subject: [PATCH 05/22] Fix more test failures --- .../CuratorDruidCoordinatorTest.java | 3 +- .../coordinator/DruidCoordinatorTest.java | 3 +- .../coordinator/HttpLoadQueuePeonTest.java | 3 +- .../server/coordinator/LoadQueuePeonTest.java | 191 +++++++++--------- .../coordinator/LoadQueuePeonTester.java | 3 +- .../TestDruidCoordinatorConfig.java | 11 +- .../DruidCoordinatorSegmentKillerTest.java | 3 +- 7 files changed, 114 insertions(+), 103 deletions(-) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java index 25dce8cf577f..3b2223514f59 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java @@ -162,7 +162,8 @@ public void setUp() throws Exception null, false, false, - new Duration("PT0s") + new Duration("PT0s"), + Duration.millis(10) ); sourceLoadQueueChildrenCache = new PathChildrenCache( curator, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 1f4f7e2844b2..793bd287b950 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -139,7 +139,8 @@ public void setUp() throws Exception null, false, false, - new Duration("PT0s") + new Duration("PT0s"), + Duration.millis(10) ); pathChildrenCache = new PathChildrenCache( curator, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java index 0f73695c167f..894472a2cbcc 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java @@ -84,7 +84,8 @@ public class HttpLoadQueuePeonTest null, false, false, - Duration.ZERO + Duration.ZERO, + Duration.millis(10) ) { @Override diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java index 544105aca7b8..ffa8d4fbee7a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java @@ -39,6 +39,7 @@ import org.apache.druid.server.coordination.SegmentChangeRequestDrop; import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NoneShardSpec; import org.joda.time.Duration; import org.junit.After; @@ -47,8 +48,10 @@ import org.junit.Test; import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.TimeUnit; public class LoadQueuePeonTest extends CuratorTestBase { @@ -79,46 +82,23 @@ public void setUp() throws Exception @Test public void testMultipleLoadDropSegments() throws Exception { - final AtomicInteger requestSignalIdx = new AtomicInteger(0); - final AtomicInteger segmentSignalIdx = new AtomicInteger(0); - loadQueuePeon = new CuratorLoadQueuePeon( curator, LOAD_QUEUE_PATH, jsonMapper, Execs.scheduledSingleThreaded("test_load_queue_peon_scheduled-%d"), Execs.singleThreaded("test_load_queue_peon-%d"), - new TestDruidCoordinatorConfig(null, null, null, null, null, null, 10, null, false, false, Duration.ZERO) + new TestDruidCoordinatorConfig(null, null, null, null, null, null, 10, null, false, false, + Duration.millis(0), Duration.millis(10) + ) ); loadQueuePeon.start(); - final CountDownLatch[] loadRequestSignal = new CountDownLatch[5]; - final CountDownLatch[] dropRequestSignal = new CountDownLatch[5]; - final CountDownLatch[] segmentLoadedSignal = new CountDownLatch[5]; - final CountDownLatch[] segmentDroppedSignal = new CountDownLatch[5]; - - for (int i = 0; i < 5; ++i) { - loadRequestSignal[i] = new CountDownLatch(1); - dropRequestSignal[i] = new CountDownLatch(1); - segmentLoadedSignal[i] = new CountDownLatch(1); - segmentDroppedSignal[i] = new CountDownLatch(1); - } - - final DataSegmentChangeHandler handler = new DataSegmentChangeHandler() - { - @Override - public void addSegment(DataSegment segment, DataSegmentChangeCallback callback) - { - loadRequestSignal[requestSignalIdx.get()].countDown(); - } - - @Override - public void removeSegment(DataSegment segment, DataSegmentChangeCallback callback) - { - dropRequestSignal[requestSignalIdx.get()].countDown(); - } - }; + ConcurrentMap loadRequestSignals = new ConcurrentHashMap<>(5); + ConcurrentMap dropRequestSignals = new ConcurrentHashMap<>(5); + ConcurrentMap segmentLoadedSignals = new ConcurrentHashMap<>(5); + ConcurrentMap segmentDroppedSignals = new ConcurrentHashMap<>(5); final List segmentToDrop = Lists.transform( ImmutableList.of( @@ -132,11 +112,24 @@ public void removeSegment(DataSegment segment, DataSegmentChangeCallback callbac @Override public DataSegment apply(String intervalStr) { - return dataSegmentWithInterval(intervalStr); + DataSegment dataSegment = dataSegmentWithInterval(intervalStr); + return dataSegment; } } ); + final CountDownLatch[] dropRequestLatches = new CountDownLatch[5]; + final CountDownLatch[] dropSegmentLatches = new CountDownLatch[5]; + for (int i = 0; i < 5; i++) { + dropRequestLatches[i] = new CountDownLatch(1); + dropSegmentLatches[i] = new CountDownLatch(1); + } + int i = 0; + for (DataSegment s : segmentToDrop) { + dropRequestSignals.put(s.getId(), dropRequestLatches[i]); + segmentDroppedSignals.put(s.getId(), dropSegmentLatches[i++]); + } + final List segmentToLoad = Lists.transform( ImmutableList.of( "2014-10-27T00:00:00Z/P1D", @@ -149,11 +142,26 @@ public DataSegment apply(String intervalStr) @Override public DataSegment apply(String intervalStr) { - return dataSegmentWithInterval(intervalStr); + DataSegment dataSegment = dataSegmentWithInterval(intervalStr); + loadRequestSignals.put(dataSegment.getId(), new CountDownLatch(1)); + segmentLoadedSignals.put(dataSegment.getId(), new CountDownLatch(1)); + return dataSegment; } } ); + final CountDownLatch[] loadRequestLatches = new CountDownLatch[5]; + final CountDownLatch[] segmentLoadedLatches = new CountDownLatch[5]; + for (i = 0; i < 5; i++) { + loadRequestLatches[i] = new CountDownLatch(1); + segmentLoadedLatches[i] = new CountDownLatch(1); + } + i = 0; + for (DataSegment s : segmentToDrop) { + loadRequestSignals.put(s.getId(), loadRequestLatches[i]); + segmentLoadedSignals.put(s.getId(), segmentLoadedLatches[i++]); + } + // segment with latest interval should be loaded first final List expectedLoadOrder = Lists.transform( ImmutableList.of( @@ -162,59 +170,64 @@ public DataSegment apply(String intervalStr) "2014-10-30T00:00:00Z/P1D", "2014-10-28T00:00:00Z/P1D", "2014-10-27T00:00:00Z/P1D" - ), new Function() - { - @Override - public DataSegment apply(String intervalStr) - { - return dataSegmentWithInterval(intervalStr); - } - } + ), intervalStr -> dataSegmentWithInterval(intervalStr) ); + final DataSegmentChangeHandler handler = new DataSegmentChangeHandler() + { + @Override + public void addSegment(DataSegment segment, DataSegmentChangeCallback callback) + { + try { + loadRequestSignals.get(segment.getId()).countDown(); + } + catch (Exception e) { + e.printStackTrace(); + } + } + + @Override + public void removeSegment(DataSegment segment, DataSegmentChangeCallback callback) + { + try { + CountDownLatch latch = dropRequestSignals.get(segment.getId()); + latch.countDown(); + } + catch (Exception e) { + e.printStackTrace(); + } + } + }; + loadQueueCache.getListenable().addListener( - new PathChildrenCacheListener() - { - @Override - public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception - { - if (event.getType() == PathChildrenCacheEvent.Type.CHILD_ADDED) { - DataSegmentChangeRequest request = jsonMapper.readValue( - event.getData().getData(), - DataSegmentChangeRequest.class - ); + (client, event) -> { + if (event.getType() == PathChildrenCacheEvent.Type.CHILD_ADDED) { + DataSegmentChangeRequest request = jsonMapper.readValue( + event.getData().getData(), + DataSegmentChangeRequest.class + ); + try { request.go(handler, null); } + catch (Exception e) { + e.printStackTrace(); + } } } ); loadQueueCache.start(); - for (DataSegment segment : segmentToDrop) { + for (final DataSegment segment : segmentToDrop) { loadQueuePeon.dropSegment( segment, - new LoadPeonCallback() - { - @Override - public void execute() - { - segmentDroppedSignal[segmentSignalIdx.get()].countDown(); - } - } + () -> segmentDroppedSignals.get(segment.getId()).countDown() ); } - for (DataSegment segment : segmentToLoad) { + for (final DataSegment segment : segmentToLoad) { loadQueuePeon.loadSegment( segment, - new LoadPeonCallback() - { - @Override - public void execute() - { - segmentLoadedSignal[segmentSignalIdx.get()].countDown(); - } - } + () -> segmentLoadedSignals.get(segment.getId()).countDown() ); } @@ -224,8 +237,14 @@ public void execute() for (DataSegment segment : segmentToDrop) { String dropRequestPath = ZKPaths.makePath(LOAD_QUEUE_PATH, segment.getId().toString()); - Assert.assertTrue(timing.forWaiting().awaitLatch(dropRequestSignal[requestSignalIdx.get()])); - Assert.assertNotNull(curator.checkExists().forPath(dropRequestPath)); + Assert.assertTrue( + "Latch not counted down for " + dropRequestSignals.get(segment.getId()), + dropRequestSignals.get(segment.getId()).await(10, TimeUnit.SECONDS) + ); + Assert.assertNotNull( + "Path " + dropRequestPath + " doesn't exist", + curator.checkExists().forPath(dropRequestPath) + ); Assert.assertEquals( segment, ((SegmentChangeRequestDrop) jsonMapper.readValue( @@ -235,29 +254,14 @@ public void execute() )).getSegment() ); - if (requestSignalIdx.get() == 4) { - requestSignalIdx.set(0); - } else { - requestSignalIdx.incrementAndGet(); - } - // simulate completion of drop request by historical curator.delete().guaranteed().forPath(dropRequestPath); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentDroppedSignal[segmentSignalIdx.get()])); - - int expectedNumSegmentToDrop = 5 - segmentSignalIdx.get() - 1; - Assert.assertEquals(expectedNumSegmentToDrop, loadQueuePeon.getSegmentsToDrop().size()); - - if (segmentSignalIdx.get() == 4) { - segmentSignalIdx.set(0); - } else { - segmentSignalIdx.incrementAndGet(); - } + Assert.assertTrue(timing.forWaiting().awaitLatch(segmentDroppedSignals.get(segment.getId()))); } for (DataSegment segment : expectedLoadOrder) { String loadRequestPath = ZKPaths.makePath(LOAD_QUEUE_PATH, segment.getId().toString()); - Assert.assertTrue(timing.forWaiting().awaitLatch(loadRequestSignal[requestSignalIdx.get()])); + Assert.assertTrue(timing.forWaiting().awaitLatch(loadRequestSignals.get(segment.getId()))); Assert.assertNotNull(curator.checkExists().forPath(loadRequestPath)); Assert.assertEquals( segment, @@ -266,16 +270,9 @@ public void execute() .getSegment() ); - requestSignalIdx.incrementAndGet(); - // simulate completion of load request by historical curator.delete().guaranteed().forPath(loadRequestPath); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentLoadedSignal[segmentSignalIdx.get()])); - - int expectedNumSegmentToLoad = 5 - segmentSignalIdx.get() - 1; - Assert.assertEquals(1200 * expectedNumSegmentToLoad, loadQueuePeon.getLoadQueueSize()); - Assert.assertEquals(expectedNumSegmentToLoad, loadQueuePeon.getSegmentsToLoad().size()); - segmentSignalIdx.incrementAndGet(); + Assert.assertTrue(timing.forWaiting().awaitLatch(segmentLoadedSignals.get(segment.getId()))); } } @@ -295,7 +292,7 @@ public void testFailAssign() throws Exception Execs.singleThreaded("test_load_queue_peon-%d"), // set time-out to 1 ms so that LoadQueuePeon will fail the assignment quickly new TestDruidCoordinatorConfig(null, null, null, new Duration(1), null, null, 10, null, false, false, - new Duration("PT1s")) + new Duration("PT1s"), Duration.millis(10)) ); loadQueuePeon.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java index cae7c938589b..841049172863 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java @@ -47,7 +47,8 @@ public LoadQueuePeonTester() null, false, false, - new Duration("PT1s") + new Duration("PT1s"), + Duration.millis(10) ) ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java index 07d6be4050fb..e51466cb99d5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java @@ -32,6 +32,7 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig private final Duration coordinatorKillDurationToRetain; private final Duration getLoadQueuePeonRepeatDelay; private final int coordinatorKillMaxSegments; + private final Duration curatorCreateZkNodesRepeatDelay; private final String consoleStatic; @@ -46,7 +47,8 @@ public TestDruidCoordinatorConfig( String consoleStatic, boolean mergeSegments, boolean convertSegments, - Duration getLoadQueuePeonRepeatDelay + Duration getLoadQueuePeonRepeatDelay, + Duration CuratorCreateZkNodesRepeatDelay ) { this.coordinatorStartDelay = coordinatorStartDelay; @@ -58,6 +60,7 @@ public TestDruidCoordinatorConfig( this.coordinatorKillMaxSegments = coordinatorKillMaxSegments; this.consoleStatic = consoleStatic; this.getLoadQueuePeonRepeatDelay = getLoadQueuePeonRepeatDelay; + this.curatorCreateZkNodesRepeatDelay = CuratorCreateZkNodesRepeatDelay; } @Override @@ -113,4 +116,10 @@ public String getConsoleStatic() return getLoadQueuePeonRepeatDelay; } + @Override + public Duration getCuratorCreateZkNodesRepeatDelay() + { + return curatorCreateZkNodesRepeatDelay; + } + } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java index 8593e301b972..7ee58a2a8f19 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java @@ -113,7 +113,8 @@ private void testFindIntervalForKillTask(List segmentManagerResult, In null, false, false, - Duration.ZERO + Duration.ZERO, + Duration.millis(10) ) ); From f020d84b2f4513acdd3dc62429fc0f2cd62451ac Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Fri, 8 Mar 2019 12:25:11 -0800 Subject: [PATCH 06/22] Remove prinstacktrace statements --- .../server/coordinator/LoadQueuePeonTest.java | 22 +++---------------- 1 file changed, 3 insertions(+), 19 deletions(-) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java index ffa8d4fbee7a..7622d8fdeb74 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java @@ -178,24 +178,13 @@ public DataSegment apply(String intervalStr) @Override public void addSegment(DataSegment segment, DataSegmentChangeCallback callback) { - try { - loadRequestSignals.get(segment.getId()).countDown(); - } - catch (Exception e) { - e.printStackTrace(); - } + loadRequestSignals.get(segment.getId()).countDown(); } @Override public void removeSegment(DataSegment segment, DataSegmentChangeCallback callback) { - try { - CountDownLatch latch = dropRequestSignals.get(segment.getId()); - latch.countDown(); - } - catch (Exception e) { - e.printStackTrace(); - } + dropRequestSignals.get(segment.getId()).countDown(); } }; @@ -206,12 +195,7 @@ public void removeSegment(DataSegment segment, DataSegmentChangeCallback callbac event.getData().getData(), DataSegmentChangeRequest.class ); - try { - request.go(handler, null); - } - catch (Exception e) { - e.printStackTrace(); - } + request.go(handler, null); } } ); From 55c3711cc2c1227e4ee349bfd3843fd5a2d7fbc1 Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Tue, 12 Mar 2019 00:48:11 -0700 Subject: [PATCH 07/22] Address code review comments --- docs/content/configuration/index.md | 10 +++++----- .../server/coordinator/CuratorLoadQueuePeon.java | 6 +++--- .../server/coordinator/DruidCoordinatorConfig.java | 12 ++++++------ .../java/org/apache/druid/cli/CliCoordinator.java | 4 ++-- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index c0bb6df3de40..cab0362b910b 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -1255,11 +1255,11 @@ These Historical configurations can be defined in the `historical/runtime.proper |`druid.segmentCache.infoDir`|Historical processes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| |`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)| |`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from from deep storage.|10| -|`druid.coordinator.loadqueuepeon.curator.numCreateThreads`|How many threads to use for concurrently creating zk nodes corresponding to segments that need to be loaded or dropped.|10| -|`druid.coordinator.loadqueuepeon.curator.numCallbackThreads`|How many threads to use for concurrently executing callback actions associated with loading or dropping segments.|2| -|`druid.coordinator.loadqueuepeon.curator.numMonitorThreads`|How many threads to use for monitoring deletion of zk nodes corresponding to segments that need to loaded or dropped|1| -|`druid.coordinator.curator.create.zknode.batchSize`|How many zk nodes to create in one iteration.|5000| -|`druid.coordinator.curator.create.zknode.repeatDelay`|How long to wait for creating next batch of zk nodes|PT1M| +|`druid.coordinator.loadqueuepeon.curator.numCreateThreads`|Number of threads creating zk nodes corresponding to segments that need to be loaded or dropped.|10| +|`druid.coordinator.loadqueuepeon.curator.numCallbackThreads`|Number of threads for executing callback actions associated with loading or dropping of segments.|2| +|`druid.coordinator.loadqueuepeon.curator.numMonitorThreads`|Number of threads to use for monitoring deletion of zk nodes|1| +|`druid.coordinator.curator.create.zknode.batchSize`|Number of zk nodes to create in one iteration.|5000| +|`druid.coordinator.curator.create.zknode.repeatDelay`|Delay before creating next batch of zk nodes|PT1M| In `druid.segmentCache.locations`, *freeSpacePercent* was added because *maxSize* setting is only a theoretical limit and assumes that much space will always be available for storing segments. In case of any druid bug leading to unaccounted segment files left alone on disk or some other process writing stuff to disk, This check can start failing segment loading early before filling up the disk completely and leaving the host usable otherwise. diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index a711c39605d2..d7bb533decec 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -108,12 +108,12 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon this.config = config; this.monitorNodeRemovedExecutor = Executors.newScheduledThreadPool( - config.getCuratorNumMonitorThreads(), + config.getNumZookeeperMonitorThreads(), Execs.makeThreadFactory("LoadQueuePeon-NodeRemovedMonitor--%d") ); // One processing queue per thread - Preconditions.checkArgument(config.getCreateZkNodeNumThreads() > 0); - this.numProcessingQueues = config.getCreateZkNodeNumThreads(); + Preconditions.checkArgument(config.getNumZookeeperNodeCreatingThreads() > 0); + this.numProcessingQueues = config.getNumZookeeperNodeCreatingThreads(); Preconditions.checkArgument(config.getCuratorCreateZkNodeBatchSize() > 0); this.batchSizePerTask = (config.getCuratorCreateZkNodeBatchSize() / numProcessingQueues); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java index 2e1abbdbad16..90035ae7889e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java @@ -75,20 +75,20 @@ public String getLoadQueuePeonType() return "curator"; } - @Config("druid.coordinator.loadqueuepeon.curator.numCreateThreads") - public int getCreateZkNodeNumThreads() + @Config("druid.coordinator.curator.loadqueuepeon.numCreateThreads") + public int getNumZookeeperNodeCreatingThreads() { return 10; } - @Config("druid.coordinator.loadqueuepeon.curator.numCallbackThreads") - public int getCuratorNumCallBackThreads() + @Config("druid.coordinator.curator.loadqueuepeon.numCallbackThreads") + public int getNumCuratorCallBackThreads() { return 2; } - @Config("druid.coordinator.loadqueuepeon.curator.numMonitorThreads") - public int getCuratorNumMonitorThreads() + @Config("druid.coordinator.curator.loadqueuepeon.numMonitorThreads") + public int getNumZookeeperMonitorThreads() { return 1; } diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 2d52d52bacab..622617c531cd 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -251,12 +251,12 @@ public LoadQueueTaskMaster getLoadQueueTaskMaster( ) { boolean useHttpLoadQueuePeon = "http".equalsIgnoreCase(config.getLoadQueuePeonType()); - int poolSize = useHttpLoadQueuePeon ? 1 : config.getCreateZkNodeNumThreads(); + int poolSize = useHttpLoadQueuePeon ? 1 : config.getNumZookeeperNodeCreatingThreads(); ExecutorService callBackExec; if (useHttpLoadQueuePeon) { callBackExec = Execs.singleThreaded("LoadQueuePeon-callbackexec--%d"); } else { - callBackExec = Execs.multiThreaded(config.getCuratorNumCallBackThreads(), "LoadQueuePeon" + callBackExec = Execs.multiThreaded(config.getNumCuratorCallBackThreads(), "LoadQueuePeon" + "-callbackexec--%d"); } return new LoadQueueTaskMaster( From 1fa8c1033b34289dea947887c753b42adc3af49b Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Wed, 13 Mar 2019 13:43:46 -0700 Subject: [PATCH 08/22] Use a single queue --- .../coordinator/CuratorLoadQueuePeon.java | 39 ++++++------------- 1 file changed, 11 insertions(+), 28 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index d7bb533decec..cae8eba0451f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -42,7 +42,6 @@ import java.util.Collections; import java.util.List; import java.util.Set; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutorService; @@ -50,7 +49,6 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -86,8 +84,8 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST ); - private final int numProcessingQueues; - private final LinkedBlockingQueue[] segmentProcessingQueues; + private final int numTasks; + private final LinkedBlockingQueue segmentProcessingQueue; // Max number of segments to process in every round private final int batchSizePerTask; @@ -113,14 +111,11 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon ); // One processing queue per thread Preconditions.checkArgument(config.getNumZookeeperNodeCreatingThreads() > 0); - this.numProcessingQueues = config.getNumZookeeperNodeCreatingThreads(); + this.numTasks = config.getNumZookeeperNodeCreatingThreads(); Preconditions.checkArgument(config.getCuratorCreateZkNodeBatchSize() > 0); - this.batchSizePerTask = (config.getCuratorCreateZkNodeBatchSize() / numProcessingQueues); - this.segmentProcessingQueues = new LinkedBlockingQueue[numProcessingQueues]; - for (int i = 0; i < numProcessingQueues; i++) { - segmentProcessingQueues[i] = new LinkedBlockingQueue<>(); - } + this.batchSizePerTask = (config.getCuratorCreateZkNodeBatchSize() / numTasks); + this.segmentProcessingQueue = new LinkedBlockingQueue(); } @JsonProperty @@ -177,7 +172,7 @@ public void loadSegment(final DataSegment segment, final LoadPeonCallback callba queuedSize.addAndGet(segment.getSize()); SegmentHolder segmentHolder = new SegmentHolder(segment, LOAD, Collections.singletonList(callback)); segmentsToLoad.put(segment, segmentHolder); - segmentProcessingQueues[getSegmentQueueIndex()].offer(segmentHolder); + segmentProcessingQueue.offer(segmentHolder); } @Override @@ -196,7 +191,7 @@ public void dropSegment( log.debug("Asking server peon[%s] to drop segment[%s]", basePath, segment.getId()); SegmentHolder segmentHolder = new SegmentHolder(segment, DROP, Collections.singletonList(callback)); segmentsToDrop.put(segment, segmentHolder); - segmentProcessingQueues[getSegmentQueueIndex()].offer(segmentHolder); + segmentProcessingQueue.offer(segmentHolder); } @Override @@ -211,29 +206,17 @@ public void unmarkSegmentToDrop(DataSegment dataSegment) segmentsMarkedToDrop.remove(dataSegment); } - private int getSegmentQueueIndex() - { - return ThreadLocalRandom.current().nextInt(0, numProcessingQueues); - } - private class SegmentChangeProcessor implements Runnable { - private final BlockingQueue segmentQueue; - - private SegmentChangeProcessor(BlockingQueue segmentQueue) - { - this.segmentQueue = segmentQueue; - } - @Override public void run() { try { int numProcessed = 0; - while (numProcessed++ < batchSizePerTask && !segmentQueue.isEmpty()) { + while (numProcessed++ < batchSizePerTask && !segmentProcessingQueue.isEmpty()) { // Instead of calling poll for every element, drain the batch to a list. List batch = new ArrayList<>(batchSizePerTask); - segmentQueue.drainTo(batch, batchSizePerTask); + segmentProcessingQueue.drainTo(batch, batchSizePerTask); for (SegmentHolder s : batch) { processSegmentChangeRequest(s); } @@ -344,9 +327,9 @@ private void actionCompleted(SegmentHolder segmentHolder) @Override public void start() { - for (int i = 0; i < numProcessingQueues; i++) { + for (int i = 0; i < numTasks; i++) { processingExecutor.scheduleAtFixedRate( - new SegmentChangeProcessor(segmentProcessingQueues[i]), + new SegmentChangeProcessor(), 0, config.getCuratorCreateZkNodesRepeatDelay().getMillis(), TimeUnit.MILLISECONDS From 13d3c0b677c8ffca99260ba8e92e57ab3de544d6 Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Fri, 22 Mar 2019 01:06:11 -0700 Subject: [PATCH 09/22] Address code review comments Since we have a separate load peon for every historical, just having a single SegmentChangeProcessor task per historical is enough. This commit also gets rid of the associated config druid.coordinator.loadqueuepeon.curator.numCreateThreads --- docs/content/configuration/index.md | 1 - .../server/coordination/ZkCoordinator.java | 10 +++--- .../coordinator/CuratorLoadQueuePeon.java | 32 ++++++++----------- .../coordinator/DruidCoordinatorConfig.java | 6 ---- 4 files changed, 19 insertions(+), 30 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index cab0362b910b..387dfb21ce4e 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -1255,7 +1255,6 @@ These Historical configurations can be defined in the `historical/runtime.proper |`druid.segmentCache.infoDir`|Historical processes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| |`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)| |`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from from deep storage.|10| -|`druid.coordinator.loadqueuepeon.curator.numCreateThreads`|Number of threads creating zk nodes corresponding to segments that need to be loaded or dropped.|10| |`druid.coordinator.loadqueuepeon.curator.numCallbackThreads`|Number of threads for executing callback actions associated with loading or dropping of segments.|2| |`druid.coordinator.loadqueuepeon.curator.numMonitorThreads`|Number of threads to use for monitoring deletion of zk nodes|1| |`druid.coordinator.curator.create.zknode.batchSize`|Number of zk nodes to create in one iteration.|5000| diff --git a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java index d08f0c14a71a..f4e952d83077 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java @@ -74,7 +74,9 @@ public ZkCoordinator( this.me = me; this.curator = curator; this.segmentLoadUnloadService = Execs.multiThreaded( - config.getNumLoadingThreads(), "ZKCoordinator--%d"); + config.getNumLoadingThreads(), + "ZKCoordinator--%d" + ); } @LifecycleStart @@ -113,15 +115,14 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) final ChildData child = event.getData(); switch (event.getType()) { case CHILD_ADDED: - log.info("Child zNode added at [%s]", event.getData().getPath()); segmentLoadUnloadService.submit(() -> { final String path = child.getPath(); DataSegmentChangeRequest request = new SegmentChangeRequestNoop(); try { final DataSegmentChangeRequest finalRequest = jsonMapper.readValue( - child.getData(), DataSegmentChangeRequest.class + child.getData(), + DataSegmentChangeRequest.class ); - log.info("Starting request[%s] with zNode[%s]", finalRequest.asString(), path); finalRequest.go( dataSegmentChangeHandler, @@ -149,6 +150,7 @@ public void execute() ); } catch (Exception e) { + // Something went wrong in either deserializing the request using jsonMapper or when invoking it try { curator.delete().guaranteed().forPath(path); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index cae8eba0451f..00d9d2159ba3 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -84,10 +84,7 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST ); - private final int numTasks; private final LinkedBlockingQueue segmentProcessingQueue; - // Max number of segments to process in every round - private final int batchSizePerTask; CuratorLoadQueuePeon( CuratorFramework curator, @@ -104,17 +101,15 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon this.callBackExecutor = callbackExecutor; this.processingExecutor = processingExecutor; this.config = config; + // Threadpool with daemon threads running scheduled tasks that monitor whether + // the zk nodes created for segment processing are deleted after a certain period + // determined by config.getLoadTimeoutDelay() this.monitorNodeRemovedExecutor = Executors.newScheduledThreadPool( config.getNumZookeeperMonitorThreads(), Execs.makeThreadFactory("LoadQueuePeon-NodeRemovedMonitor--%d") ); - // One processing queue per thread - Preconditions.checkArgument(config.getNumZookeeperNodeCreatingThreads() > 0); - this.numTasks = config.getNumZookeeperNodeCreatingThreads(); - Preconditions.checkArgument(config.getCuratorCreateZkNodeBatchSize() > 0); - this.batchSizePerTask = (config.getCuratorCreateZkNodeBatchSize() / numTasks); this.segmentProcessingQueue = new LinkedBlockingQueue(); } @@ -213,10 +208,11 @@ public void run() { try { int numProcessed = 0; - while (numProcessed++ < batchSizePerTask && !segmentProcessingQueue.isEmpty()) { + int batchSize = config.getCuratorCreateZkNodeBatchSize(); + while (numProcessed++ < batchSize && !segmentProcessingQueue.isEmpty()) { // Instead of calling poll for every element, drain the batch to a list. - List batch = new ArrayList<>(batchSizePerTask); - segmentProcessingQueue.drainTo(batch, batchSizePerTask); + List batch = new ArrayList<>(batchSize); + segmentProcessingQueue.drainTo(batch, batchSize); for (SegmentHolder s : batch) { processSegmentChangeRequest(s); } @@ -327,14 +323,12 @@ private void actionCompleted(SegmentHolder segmentHolder) @Override public void start() { - for (int i = 0; i < numTasks; i++) { - processingExecutor.scheduleAtFixedRate( - new SegmentChangeProcessor(), - 0, - config.getCuratorCreateZkNodesRepeatDelay().getMillis(), - TimeUnit.MILLISECONDS - ); - } + processingExecutor.scheduleAtFixedRate( + new SegmentChangeProcessor(), + 0, + config.getCuratorCreateZkNodesRepeatDelay().getMillis(), + TimeUnit.MILLISECONDS + ); } @Override diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java index 90035ae7889e..7c8413f2b495 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java @@ -75,12 +75,6 @@ public String getLoadQueuePeonType() return "curator"; } - @Config("druid.coordinator.curator.loadqueuepeon.numCreateThreads") - public int getNumZookeeperNodeCreatingThreads() - { - return 10; - } - @Config("druid.coordinator.curator.loadqueuepeon.numCallbackThreads") public int getNumCuratorCallBackThreads() { From 5cc9e632b3add8b920125dab0d478673c95ebf4a Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Fri, 22 Mar 2019 01:18:26 -0700 Subject: [PATCH 10/22] Resolve merge conflict --- .../org/apache/druid/server/coordination/ZkCoordinator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java index f4e952d83077..4ab0a182e961 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java @@ -143,7 +143,7 @@ public void execute() log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path); } log.error(e, "Exception while removing zNode[%s]", path); - throw Throwables.propagate(e); + throw new RuntimeException(e); } } } From 7b2341944d1739f3f1d777bcafa158c11f252f4c Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Fri, 22 Mar 2019 15:50:07 -0700 Subject: [PATCH 11/22] Fix compilation failure --- .../src/main/java/org/apache/druid/cli/CliCoordinator.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 622617c531cd..88245624093d 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -251,7 +251,6 @@ public LoadQueueTaskMaster getLoadQueueTaskMaster( ) { boolean useHttpLoadQueuePeon = "http".equalsIgnoreCase(config.getLoadQueuePeonType()); - int poolSize = useHttpLoadQueuePeon ? 1 : config.getNumZookeeperNodeCreatingThreads(); ExecutorService callBackExec; if (useHttpLoadQueuePeon) { callBackExec = Execs.singleThreaded("LoadQueuePeon-callbackexec--%d"); @@ -262,7 +261,7 @@ public LoadQueueTaskMaster getLoadQueueTaskMaster( return new LoadQueueTaskMaster( curator, jsonMapper, - factory.create(poolSize, "Master-PeonExec--%d"), + factory.create(1, "Master-PeonExec--%d"), callBackExec, config, httpClient, From af59c24b865514d6b23d8b22fed1a00ad792e65e Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Fri, 12 Apr 2019 12:17:40 -0700 Subject: [PATCH 12/22] Remove batching since we already have a dynamic config maxSegmentsInNodeLoadingQueue that provides that control --- .../coordinator/CuratorLoadQueuePeon.java | 181 ++++++++---------- .../coordinator/DruidCoordinatorConfig.java | 12 -- .../TestDruidCoordinatorConfig.java | 11 +- 3 files changed, 79 insertions(+), 125 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index 00d9d2159ba3..11c68ee1df12 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.api.CuratorWatcher; import org.apache.curator.utils.ZKPaths; @@ -46,7 +45,6 @@ import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -84,8 +82,6 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST ); - private final LinkedBlockingQueue segmentProcessingQueue; - CuratorLoadQueuePeon( CuratorFramework curator, String basePath, @@ -102,15 +98,12 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon this.processingExecutor = processingExecutor; this.config = config; // Threadpool with daemon threads running scheduled tasks that monitor whether - // the zk nodes created for segment processing are deleted after a certain period - // determined by config.getLoadTimeoutDelay() + // the zk nodes created for segment processing are removed this.monitorNodeRemovedExecutor = Executors.newScheduledThreadPool( config.getNumZookeeperMonitorThreads(), Execs.makeThreadFactory("LoadQueuePeon-NodeRemovedMonitor--%d") ); - Preconditions.checkArgument(config.getCuratorCreateZkNodeBatchSize() > 0); - this.segmentProcessingQueue = new LinkedBlockingQueue(); } @JsonProperty @@ -167,7 +160,7 @@ public void loadSegment(final DataSegment segment, final LoadPeonCallback callba queuedSize.addAndGet(segment.getSize()); SegmentHolder segmentHolder = new SegmentHolder(segment, LOAD, Collections.singletonList(callback)); segmentsToLoad.put(segment, segmentHolder); - segmentProcessingQueue.offer(segmentHolder); + processingExecutor.submit(new SegmentChangeProcessor(segmentHolder)); } @Override @@ -186,7 +179,7 @@ public void dropSegment( log.debug("Asking server peon[%s] to drop segment[%s]", basePath, segment.getId()); SegmentHolder segmentHolder = new SegmentHolder(segment, DROP, Collections.singletonList(callback)); segmentsToDrop.put(segment, segmentHolder); - segmentProcessingQueue.offer(segmentHolder); + processingExecutor.submit(new SegmentChangeProcessor(segmentHolder)); } @Override @@ -203,100 +196,87 @@ public void unmarkSegmentToDrop(DataSegment dataSegment) private class SegmentChangeProcessor implements Runnable { + private final SegmentHolder segmentHolder; + + private SegmentChangeProcessor(SegmentHolder segmentHolder) + { + this.segmentHolder = segmentHolder; + } + @Override public void run() { try { - int numProcessed = 0; - int batchSize = config.getCuratorCreateZkNodeBatchSize(); - while (numProcessed++ < batchSize && !segmentProcessingQueue.isEmpty()) { - // Instead of calling poll for every element, drain the batch to a list. - List batch = new ArrayList<>(batchSize); - segmentProcessingQueue.drainTo(batch, batchSize); - for (SegmentHolder s : batch) { - processSegmentChangeRequest(s); - } - } - } - catch (Throwable e) { - // Swallow all errors so that the executor thread doesn't die - log.error(e, "Throwable caught and ignored when processing segments from the queue"); - } - } - } - - private void processSegmentChangeRequest(SegmentHolder segmentHolder) - { - try { - final String path = ZKPaths.makePath(basePath, segmentHolder.getSegmentIdentifier()); - final byte[] payload = jsonMapper.writeValueAsBytes(segmentHolder.getChangeRequest()); - curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, payload); - log.debug( - "ZKNode created for server to [%s] %s [%s]", - basePath, - segmentHolder.getType() == LOAD ? "load" : "drop", - segmentHolder.getSegmentIdentifier() - ); - final ScheduledFuture future = monitorNodeRemovedExecutor.schedule( - () -> { - try { - if (curator.checkExists().forPath(path) != null) { - failAssign(segmentHolder, new ISE("%s was never removed! Failing this operation!", path)); - } else { - log.debug("%s detected to be removed. ", path); + final String path = ZKPaths.makePath(basePath, segmentHolder.getSegmentIdentifier()); + final byte[] payload = jsonMapper.writeValueAsBytes(segmentHolder.getChangeRequest()); + curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, payload); + log.debug( + "ZKNode created for server to [%s] %s [%s]", + basePath, + segmentHolder.getType() == LOAD ? "load" : "drop", + segmentHolder.getSegmentIdentifier() + ); + final ScheduledFuture future = monitorNodeRemovedExecutor.schedule( + () -> { + try { + if (curator.checkExists().forPath(path) != null) { + failAssign(segmentHolder, new ISE("%s was never removed! Failing this operation!", path)); + } else { + log.debug("%s detected to be removed. ", path); + } } - } - catch (Exception e) { - failAssign(segmentHolder, e); - } - }, - config.getLoadTimeoutDelay().getMillis(), - TimeUnit.MILLISECONDS - ); + catch (Exception e) { + failAssign(segmentHolder, e); + } + }, + config.getLoadTimeoutDelay().getMillis(), + TimeUnit.MILLISECONDS + ); - final Stat stat = curator.checkExists().usingWatcher( - (CuratorWatcher) watchedEvent -> { - switch (watchedEvent.getType()) { - case NodeDeleted: - // Cancel the check node deleted task since we have already - // been notified by the zk watcher - future.cancel(true); - entryRemoved(segmentHolder, watchedEvent.getPath()); - break; - default: - // do nothing + final Stat stat = curator.checkExists().usingWatcher( + (CuratorWatcher) watchedEvent -> { + switch (watchedEvent.getType()) { + case NodeDeleted: + // Cancel the check node deleted task since we have already + // been notified by the zk watcher + future.cancel(true); + entryRemoved(segmentHolder, watchedEvent.getPath()); + break; + default: + // do nothing + } } - } - ).forPath(path); - - if (stat == null) { - final byte[] noopPayload = jsonMapper.writeValueAsBytes(new SegmentChangeRequestNoop()); - - // Create a node and then delete it to remove the registered watcher. This is a work-around for - // a zookeeper race condition. Specifically, when you set a watcher, it fires on the next event - // that happens for that node. If no events happen, the watcher stays registered foreverz. - // Couple that with the fact that you cannot set a watcher when you create a node, but what we - // want is to create a node and then watch for it to get deleted. The solution is that you *can* - // set a watcher when you check to see if it exists so, we first create the node and then set a - // watcher on its existence. However, if already does not exist by the time the existence check - // returns, then the watcher that was set will never fire (nobody will ever create the node - // again) and thus lead to a slow, but real, memory leak. So, we create another node to cause - // that watcher to fire and delete it right away. - // - // We do not create the existence watcher first, because then it will fire when we create the - // node and we'll have the same race when trying to refresh that watcher. - curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, noopPayload); - entryRemoved(segmentHolder, path); + ).forPath(path); + + if (stat == null) { + final byte[] noopPayload = jsonMapper.writeValueAsBytes(new SegmentChangeRequestNoop()); + + // Create a node and then delete it to remove the registered watcher. This is a work-around for + // a zookeeper race condition. Specifically, when you set a watcher, it fires on the next event + // that happens for that node. If no events happen, the watcher stays registered foreverz. + // Couple that with the fact that you cannot set a watcher when you create a node, but what we + // want is to create a node and then watch for it to get deleted. The solution is that you *can* + // set a watcher when you check to see if it exists so, we first create the node and then set a + // watcher on its existence. However, if already does not exist by the time the existence check + // returns, then the watcher that was set will never fire (nobody will ever create the node + // again) and thus lead to a slow, but real, memory leak. So, we create another node to cause + // that watcher to fire and delete it right away. + // + // We do not create the existence watcher first, because then it will fire when we create the + // node and we'll have the same race when trying to refresh that watcher. + curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, noopPayload); + entryRemoved(segmentHolder, path); + } + } + catch (KeeperException.NodeExistsException ne) { + // This is expected when historicals haven't yet picked up processing this segment and coordinator + // tries reassigning it to the same node. + log.warn(ne, "ZK node already exists because segment change request hasn't yet been processed"); + failAssign(segmentHolder); + } + catch (Exception e) { + failAssign(segmentHolder, e); } - } - catch (KeeperException.NodeExistsException ne) { - // This is expected when historicals haven't yet picked up processing this segment and coordinator - // tries reassigning it to the same node. - log.warn(ne, "ZK node already exists because segment change request hasn't yet been processed"); - failAssign(segmentHolder); - } - catch (Exception e) { - failAssign(segmentHolder, e); } } @@ -322,14 +302,7 @@ private void actionCompleted(SegmentHolder segmentHolder) @Override public void start() - { - processingExecutor.scheduleAtFixedRate( - new SegmentChangeProcessor(), - 0, - config.getCuratorCreateZkNodesRepeatDelay().getMillis(), - TimeUnit.MILLISECONDS - ); - } + { } @Override public void stop() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java index 7c8413f2b495..0a643da12835 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java @@ -87,18 +87,6 @@ public int getNumZookeeperMonitorThreads() return 1; } - @Config("druid.coordinator.curator.create.zknode.batchSize") - public int getCuratorCreateZkNodeBatchSize() - { - return 5000; - } - - @Config("druid.coordinator.curator.create.zknode.repeatDelay") - public Duration getCuratorCreateZkNodesRepeatDelay() - { - return Duration.millis(60000); - } - @Config("druid.coordinator.loadqueuepeon.http.repeatDelay") public Duration getHttpLoadQueuePeonRepeatDelay() { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java index e51466cb99d5..e1b91354de70 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java @@ -32,7 +32,6 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig private final Duration coordinatorKillDurationToRetain; private final Duration getLoadQueuePeonRepeatDelay; private final int coordinatorKillMaxSegments; - private final Duration curatorCreateZkNodesRepeatDelay; private final String consoleStatic; @@ -60,7 +59,6 @@ public TestDruidCoordinatorConfig( this.coordinatorKillMaxSegments = coordinatorKillMaxSegments; this.consoleStatic = consoleStatic; this.getLoadQueuePeonRepeatDelay = getLoadQueuePeonRepeatDelay; - this.curatorCreateZkNodesRepeatDelay = CuratorCreateZkNodesRepeatDelay; } @Override @@ -111,15 +109,10 @@ public String getConsoleStatic() return consoleStatic; } - @Override public Duration getLoadQueuePeonRepeatDelay() - { - return getLoadQueuePeonRepeatDelay; - } - @Override - public Duration getCuratorCreateZkNodesRepeatDelay() + public Duration getLoadQueuePeonRepeatDelay() { - return curatorCreateZkNodesRepeatDelay; + return getLoadQueuePeonRepeatDelay; } } From e3b4519847869862a4b80344de0bc275f30c982d Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Tue, 16 Apr 2019 11:20:39 -0700 Subject: [PATCH 13/22] Fix NPE in test --- .../apache/druid/server/coordinator/LoadQueuePeonTester.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java index 841049172863..c979671ff847 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java @@ -19,6 +19,7 @@ package org.apache.druid.server.coordinator; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.timeline.DataSegment; import org.joda.time.Duration; @@ -34,7 +35,7 @@ public LoadQueuePeonTester() null, null, null, - null, + Execs.scheduledSingleThreaded("LoadQueuePeonTester--%d"), null, new TestDruidCoordinatorConfig( null, From fc0ef0cd950552c5c8faaea81783fc8f9d9cfa65 Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Tue, 16 Apr 2019 11:25:07 -0700 Subject: [PATCH 14/22] Remove documentation for configs that are no longer needed --- docs/content/configuration/index.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 387dfb21ce4e..4ec8a2395312 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -1257,8 +1257,6 @@ These Historical configurations can be defined in the `historical/runtime.proper |`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from from deep storage.|10| |`druid.coordinator.loadqueuepeon.curator.numCallbackThreads`|Number of threads for executing callback actions associated with loading or dropping of segments.|2| |`druid.coordinator.loadqueuepeon.curator.numMonitorThreads`|Number of threads to use for monitoring deletion of zk nodes|1| -|`druid.coordinator.curator.create.zknode.batchSize`|Number of zk nodes to create in one iteration.|5000| -|`druid.coordinator.curator.create.zknode.repeatDelay`|Delay before creating next batch of zk nodes|PT1M| In `druid.segmentCache.locations`, *freeSpacePercent* was added because *maxSize* setting is only a theoretical limit and assumes that much space will always be available for storing segments. In case of any druid bug leading to unaccounted segment files left alone on disk or some other process writing stuff to disk, This check can start failing segment loading early before filling up the disk completely and leaving the host usable otherwise. From 2c4b80a8b3394102638a685629e8fe285ac59a0a Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Wed, 17 Apr 2019 14:53:44 -0700 Subject: [PATCH 15/22] Address code review comments --- docs/content/configuration/index.md | 6 +- .../server/coordination/ZkCoordinator.java | 103 +++++++++--------- .../coordinator/CuratorLoadQueuePeon.java | 13 ++- .../server/coordinator/LoadQueuePeonTest.java | 31 +++++- 4 files changed, 92 insertions(+), 61 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 4ec8a2395312..3a97a42dc143 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -1254,9 +1254,9 @@ These Historical configurations can be defined in the `historical/runtime.proper |`druid.segmentCache.dropSegmentDelayMillis`|How long a process delays before completely dropping segment.|30000 (30 seconds)| |`druid.segmentCache.infoDir`|Historical processes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| |`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)| -|`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from from deep storage.|10| -|`druid.coordinator.loadqueuepeon.curator.numCallbackThreads`|Number of threads for executing callback actions associated with loading or dropping of segments.|2| -|`druid.coordinator.loadqueuepeon.curator.numMonitorThreads`|Number of threads to use for monitoring deletion of zk nodes|1| +|`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from deep storage. Note that the work of loading segments involves downloading segments from deep storage, decompressing them and loading them to a memory mapped location. So the work is not all I/O Bound. Depending on CPU and network load, one could possibly increase this config to a higher value.|Number of cores| +|`druid.coordinator.loadqueuepeon.curator.numCallbackThreads`|Number of threads for executing callback actions associated with loading or dropping of segments. One might want to increase this number when noticing clusters are lagging behind w.r.t. balancing segments across historical nodes.|2| +|`druid.coordinator.loadqueuepeon.curator.numMonitorThreads`|Number of threads to use for monitoring deletion of zk nodes. Tasks in this pool get scheduled to run at time `druid.coordinator.load.timeout` after a segment is added to the queue. Increase this number if you see segments not getting loaded or dropped even after `druid.coordinator.load.timeout` since it is possible they are not getting re-assigned to the queue of other historicals soon enough.|1| In `druid.segmentCache.locations`, *freeSpacePercent* was added because *maxSize* setting is only a theoretical limit and assumes that much space will always be available for storing segments. In case of any druid bug leading to unaccounted segment files left alone on disk or some other process writing stuff to disk, This check can start failing segment loading early before filling up the disk completely and leaving the host usable otherwise. diff --git a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java index 4ab0a182e961..ca56b10024e7 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java @@ -115,55 +115,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) final ChildData child = event.getData(); switch (event.getType()) { case CHILD_ADDED: - segmentLoadUnloadService.submit(() -> { - final String path = child.getPath(); - DataSegmentChangeRequest request = new SegmentChangeRequestNoop(); - try { - final DataSegmentChangeRequest finalRequest = jsonMapper.readValue( - child.getData(), - DataSegmentChangeRequest.class - ); - - finalRequest.go( - dataSegmentChangeHandler, - new DataSegmentChangeCallback() - { - @Override - public void execute() - { - try { - curator.delete().guaranteed().forPath(path); - log.info("Completed request [%s]", finalRequest.asString()); - } - catch (Exception e) { - try { - curator.delete().guaranteed().forPath(path); - } - catch (Exception e1) { - log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path); - } - log.error(e, "Exception while removing zNode[%s]", path); - throw new RuntimeException(e); - } - } - } - ); - } - catch (Exception e) { - // Something went wrong in either deserializing the request using jsonMapper or when invoking it - try { - curator.delete().guaranteed().forPath(path); - } - catch (Exception e1) { - log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path); - } - - log.makeAlert(e, "Segment load/unload: uncaught exception.") - .addData("node", path) - .addData("nodeProperties", request) - .emit(); - } - }); + childAdded(child); break; case CHILD_REMOVED: log.info("zNode[%s] was removed", event.getData().getPath()); @@ -186,6 +138,59 @@ public void execute() } } + private void childAdded(ChildData child) + { + segmentLoadUnloadService.submit(() -> { + final String path = child.getPath(); + DataSegmentChangeRequest request = new SegmentChangeRequestNoop(); + try { + final DataSegmentChangeRequest finalRequest = jsonMapper.readValue( + child.getData(), + DataSegmentChangeRequest.class + ); + + finalRequest.go( + dataSegmentChangeHandler, + new DataSegmentChangeCallback() + { + @Override + public void execute() + { + try { + curator.delete().guaranteed().forPath(path); + log.info("Completed request [%s]", finalRequest.asString()); + } + catch (Exception e) { + try { + curator.delete().guaranteed().forPath(path); + } + catch (Exception e1) { + log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path); + } + log.error(e, "Exception while removing zNode[%s]", path); + throw new RuntimeException(e); + } + } + } + ); + } + catch (Exception e) { + // Something went wrong in either deserializing the request using jsonMapper or when invoking it + try { + curator.delete().guaranteed().forPath(path); + } + catch (Exception e1) { + log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path); + } + + log.makeAlert(e, "Segment load/unload: uncaught exception.") + .addData("node", path) + .addData("nodeProperties", request) + .emit(); + } + }); + } + @LifecycleStop public void stop() { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index 11c68ee1df12..34d5fe8ce10d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.api.CuratorWatcher; import org.apache.curator.utils.ZKPaths; @@ -65,6 +66,10 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon private final String basePath; private final ObjectMapper jsonMapper; private final ScheduledExecutorService processingExecutor; + /** + * Threadpool with daemon threads running scheduled tasks that monitor whether + * the zk nodes created for segment processing are removed + */ private final ScheduledExecutorService monitorNodeRemovedExecutor; private final ExecutorService callBackExecutor; private final DruidCoordinatorConfig config; @@ -97,8 +102,6 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon this.callBackExecutor = callbackExecutor; this.processingExecutor = processingExecutor; this.config = config; - // Threadpool with daemon threads running scheduled tasks that monitor whether - // the zk nodes created for segment processing are removed this.monitorNodeRemovedExecutor = Executors.newScheduledThreadPool( config.getNumZookeeperMonitorThreads(), @@ -406,11 +409,11 @@ public void addCallback(LoadPeonCallback newCallback) } } - LoadPeonCallback[] getCallbacks() + List snapshotCallbacks() { synchronized (callbacks) { // Return a copy so that callers get a consistent view - return callbacks.toArray(new LoadPeonCallback[0]); + return ImmutableList.copyOf(callbacks); } } @@ -428,7 +431,7 @@ public String toString() private void executeCallbacks(SegmentHolder holder) { - for (LoadPeonCallback callback : holder.getCallbacks()) { + for (LoadPeonCallback callback : holder.snapshotCallbacks()) { if (callback != null) { callBackExecutor.submit(() -> callback.execute()); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java index 7622d8fdeb74..8d8271dab1dc 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java @@ -88,8 +88,19 @@ public void testMultipleLoadDropSegments() throws Exception jsonMapper, Execs.scheduledSingleThreaded("test_load_queue_peon_scheduled-%d"), Execs.singleThreaded("test_load_queue_peon-%d"), - new TestDruidCoordinatorConfig(null, null, null, null, null, null, 10, null, false, false, - Duration.millis(0), Duration.millis(10) + new TestDruidCoordinatorConfig( + null, + null, + null, + null, + null, + null, + 10, + null, + false, + false, + Duration.millis(0), + Duration.millis(10) ) ); @@ -275,8 +286,20 @@ public void testFailAssign() throws Exception Execs.scheduledSingleThreaded("test_load_queue_peon_scheduled-%d"), Execs.singleThreaded("test_load_queue_peon-%d"), // set time-out to 1 ms so that LoadQueuePeon will fail the assignment quickly - new TestDruidCoordinatorConfig(null, null, null, new Duration(1), null, null, 10, null, false, false, - new Duration("PT1s"), Duration.millis(10)) + new TestDruidCoordinatorConfig( + null, + null, + null, + new Duration(1), + null, + null, + 10, + null, + false, + false, + new Duration("PT1s"), + Duration.millis(10) + ) ); loadQueuePeon.start(); From 83ad6a18d128907e77575c43155ca4a020065aa7 Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Sat, 20 Apr 2019 00:55:08 -0700 Subject: [PATCH 16/22] Address more code review comments --- docs/content/configuration/index.md | 2 +- .../server/coordinator/CuratorLoadQueuePeon.java | 13 +++++-------- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 3a97a42dc143..9590cb1c181e 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -1256,7 +1256,7 @@ These Historical configurations can be defined in the `historical/runtime.proper |`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)| |`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from deep storage. Note that the work of loading segments involves downloading segments from deep storage, decompressing them and loading them to a memory mapped location. So the work is not all I/O Bound. Depending on CPU and network load, one could possibly increase this config to a higher value.|Number of cores| |`druid.coordinator.loadqueuepeon.curator.numCallbackThreads`|Number of threads for executing callback actions associated with loading or dropping of segments. One might want to increase this number when noticing clusters are lagging behind w.r.t. balancing segments across historical nodes.|2| -|`druid.coordinator.loadqueuepeon.curator.numMonitorThreads`|Number of threads to use for monitoring deletion of zk nodes. Tasks in this pool get scheduled to run at time `druid.coordinator.load.timeout` after a segment is added to the queue. Increase this number if you see segments not getting loaded or dropped even after `druid.coordinator.load.timeout` since it is possible they are not getting re-assigned to the queue of other historicals soon enough.|1| +|`druid.coordinator.loadqueuepeon.curator.numMonitorThreads`|Number of threads in the pool that monitors when the zk nodes are getting deleted. The zookeeper node created for processing a segment load/drop should be deleted within druid.coordinator.load.timeout. If the node doesn't get deleted, then it means the historical failed to process the request. When such a timeout happens, queue peon needs to mark the change request as failed and effectively tell the coordinator to assign this change request to another historical. With several concurrent change requests in flight, it is possible that such reassignment may not run fast enough. Such a scenario though is very unlikely which is why the default number of threads is 1.|1| In `druid.segmentCache.locations`, *freeSpacePercent* was added because *maxSize* setting is only a theoretical limit and assumes that much space will always be available for storing segments. In case of any druid bug leading to unaccounted segment files left alone on disk or some other process writing stuff to disk, This check can start failing segment loading early before filling up the disk completely and leaving the host usable otherwise. diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index 34d5fe8ce10d..e788c9c6ef3b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -151,9 +151,9 @@ public int getNumberOfSegmentsInQueue() @Override public void loadSegment(final DataSegment segment, final LoadPeonCallback callback) { - - final SegmentHolder existingHolder = segmentsToLoad.get(segment); - if (existingHolder != null) { + SegmentHolder segmentHolder = new SegmentHolder(segment, LOAD, Collections.singletonList(callback)); + final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder); + if (existingHolder != null) { if ((callback != null)) { existingHolder.addCallback(callback); } @@ -161,8 +161,6 @@ public void loadSegment(final DataSegment segment, final LoadPeonCallback callba } log.debug("Asking server peon[%s] to load segment[%s]", basePath, segment.getId()); queuedSize.addAndGet(segment.getSize()); - SegmentHolder segmentHolder = new SegmentHolder(segment, LOAD, Collections.singletonList(callback)); - segmentsToLoad.put(segment, segmentHolder); processingExecutor.submit(new SegmentChangeProcessor(segmentHolder)); } @@ -172,7 +170,8 @@ public void dropSegment( final LoadPeonCallback callback ) { - final SegmentHolder existingHolder = segmentsToDrop.get(segment); + SegmentHolder segmentHolder = new SegmentHolder(segment, DROP, Collections.singletonList(callback)); + final SegmentHolder existingHolder = segmentsToDrop.put(segment, segmentHolder); if (existingHolder != null) { if (callback != null) { existingHolder.addCallback(callback); @@ -180,8 +179,6 @@ public void dropSegment( return; } log.debug("Asking server peon[%s] to drop segment[%s]", basePath, segment.getId()); - SegmentHolder segmentHolder = new SegmentHolder(segment, DROP, Collections.singletonList(callback)); - segmentsToDrop.put(segment, segmentHolder); processingExecutor.submit(new SegmentChangeProcessor(segmentHolder)); } From edba89e8c502fe7756fed274991880be11907a9a Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Sat, 20 Apr 2019 16:08:54 -0700 Subject: [PATCH 17/22] Fix checkstyle issue --- .../apache/druid/server/coordinator/CuratorLoadQueuePeon.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index e788c9c6ef3b..8cbf2edd4ea3 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -153,7 +153,7 @@ public void loadSegment(final DataSegment segment, final LoadPeonCallback callba { SegmentHolder segmentHolder = new SegmentHolder(segment, LOAD, Collections.singletonList(callback)); final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder); - if (existingHolder != null) { + if (existingHolder != null) { if ((callback != null)) { existingHolder.addCallback(callback); } From d1eafe8d4cd0b4380a22a14392aaa3c1be93d43c Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Mon, 22 Apr 2019 13:54:26 -0700 Subject: [PATCH 18/22] Address code review comments --- .../coordinator/CuratorLoadQueuePeon.java | 30 +++++++++++++++++-- 1 file changed, 28 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index 8cbf2edd4ea3..99c43114d172 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -54,6 +54,12 @@ /** * Use {@link HttpLoadQueuePeon} instead. + *

+ * Objects of this class can be accessed by multiple threads. State wise, this class + * is thread safe and callers of the public methods can expect thread safe behavior. + * Though, like a typical object being accessed by multiple threads, + * callers shouldn't expect strict consistency in results between two calls + * of the same or different methods. */ @Deprecated public class CuratorLoadQueuePeon extends LoadQueuePeon @@ -71,18 +77,37 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon * the zk nodes created for segment processing are removed */ private final ScheduledExecutorService monitorNodeRemovedExecutor; + + /** + * Threadpool with daemon threads that execute callback actions associated + * with loading or dropping segments. + */ private final ExecutorService callBackExecutor; private final DruidCoordinatorConfig config; private final AtomicLong queuedSize = new AtomicLong(0); private final AtomicInteger failedAssignCount = new AtomicInteger(0); + /** + * Needs to be thread safe since it can be concurrently accessed via + * loadSegment(), actionCompleted(), getSegmentsToLoad() and stop() + */ private final ConcurrentSkipListMap segmentsToLoad = new ConcurrentSkipListMap<>( DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST ); + + /** + * Needs to be thread safe since it can be concurrently accessed via + * dropSegment(), actionCompleted(), getSegmentsToDrop() and stop() + */ private final ConcurrentSkipListMap segmentsToDrop = new ConcurrentSkipListMap<>( DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST ); + + /** + * Needs to be thread safe since it can be concurrently accessed via + * markSegmentToDrop(), unmarkSegmentToDrop() and getSegmentsMarkedToDrop() + */ private final ConcurrentSkipListSet segmentsMarkedToDrop = new ConcurrentSkipListSet<>( DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST ); @@ -152,6 +177,7 @@ public int getNumberOfSegmentsInQueue() public void loadSegment(final DataSegment segment, final LoadPeonCallback callback) { SegmentHolder segmentHolder = new SegmentHolder(segment, LOAD, Collections.singletonList(callback)); + final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder); if (existingHolder != null) { if ((callback != null)) { @@ -171,7 +197,7 @@ public void dropSegment( ) { SegmentHolder segmentHolder = new SegmentHolder(segment, DROP, Collections.singletonList(callback)); - final SegmentHolder existingHolder = segmentsToDrop.put(segment, segmentHolder); + final SegmentHolder existingHolder = segmentsToDrop.putIfAbsent(segment, segmentHolder); if (existingHolder != null) { if (callback != null) { existingHolder.addCallback(callback); @@ -409,7 +435,7 @@ public void addCallback(LoadPeonCallback newCallback) List snapshotCallbacks() { synchronized (callbacks) { - // Return a copy so that callers get a consistent view + // Return an immutable copy so that callers don't have to worry about concurrent modification return ImmutableList.copyOf(callbacks); } } From f648b126b87a1a59eda49dc476471e7c81776e02 Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Mon, 29 Apr 2019 11:13:54 -0700 Subject: [PATCH 19/22] Code review comments --- docs/content/configuration/index.md | 1 - .../coordinator/CuratorLoadQueuePeon.java | 40 ++++++------------- .../coordinator/DruidCoordinatorConfig.java | 6 --- 3 files changed, 12 insertions(+), 35 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 9590cb1c181e..5a91d556a897 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -1256,7 +1256,6 @@ These Historical configurations can be defined in the `historical/runtime.proper |`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)| |`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from deep storage. Note that the work of loading segments involves downloading segments from deep storage, decompressing them and loading them to a memory mapped location. So the work is not all I/O Bound. Depending on CPU and network load, one could possibly increase this config to a higher value.|Number of cores| |`druid.coordinator.loadqueuepeon.curator.numCallbackThreads`|Number of threads for executing callback actions associated with loading or dropping of segments. One might want to increase this number when noticing clusters are lagging behind w.r.t. balancing segments across historical nodes.|2| -|`druid.coordinator.loadqueuepeon.curator.numMonitorThreads`|Number of threads in the pool that monitors when the zk nodes are getting deleted. The zookeeper node created for processing a segment load/drop should be deleted within druid.coordinator.load.timeout. If the node doesn't get deleted, then it means the historical failed to process the request. When such a timeout happens, queue peon needs to mark the change request as failed and effectively tell the coordinator to assign this change request to another historical. With several concurrent change requests in flight, it is possible that such reassignment may not run fast enough. Such a scenario though is very unlikely which is why the default number of threads is 1.|1| In `druid.segmentCache.locations`, *freeSpacePercent* was added because *maxSize* setting is only a theoretical limit and assumes that much space will always be available for storing segments. In case of any druid bug leading to unaccounted segment files left alone on disk or some other process writing stuff to disk, This check can start failing segment loading early before filling up the disk completely and leaving the host usable otherwise. diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index 99c43114d172..5d3f5ac07709 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -26,7 +26,6 @@ import org.apache.curator.framework.api.CuratorWatcher; import org.apache.curator.utils.ZKPaths; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.SegmentChangeRequestDrop; @@ -37,6 +36,7 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -45,7 +45,6 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -72,11 +71,6 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon private final String basePath; private final ObjectMapper jsonMapper; private final ScheduledExecutorService processingExecutor; - /** - * Threadpool with daemon threads running scheduled tasks that monitor whether - * the zk nodes created for segment processing are removed - */ - private final ScheduledExecutorService monitorNodeRemovedExecutor; /** * Threadpool with daemon threads that execute callback actions associated @@ -90,7 +84,8 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon /** * Needs to be thread safe since it can be concurrently accessed via - * loadSegment(), actionCompleted(), getSegmentsToLoad() and stop() + * {@link #loadSegment(DataSegment, LoadPeonCallback)}, {@link #actionCompleted(SegmentHolder)}, + * {@link #getSegmentsToLoad()} and {@link #stop()} */ private final ConcurrentSkipListMap segmentsToLoad = new ConcurrentSkipListMap<>( DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST @@ -98,7 +93,8 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon /** * Needs to be thread safe since it can be concurrently accessed via - * dropSegment(), actionCompleted(), getSegmentsToDrop() and stop() + * {@link #dropSegment(DataSegment, LoadPeonCallback)}, {@link #actionCompleted(SegmentHolder)}, + * {@link #getSegmentsToDrop()} and {@link #stop()} */ private final ConcurrentSkipListMap segmentsToDrop = new ConcurrentSkipListMap<>( DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST @@ -106,7 +102,8 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon /** * Needs to be thread safe since it can be concurrently accessed via - * markSegmentToDrop(), unmarkSegmentToDrop() and getSegmentsMarkedToDrop() + * {@link #markSegmentToDrop(DataSegment)}}, {@link #unmarkSegmentToDrop(DataSegment)}} + * and {@link #getSegmentsToDrop()} */ private final ConcurrentSkipListSet segmentsMarkedToDrop = new ConcurrentSkipListSet<>( DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST @@ -127,11 +124,6 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon this.callBackExecutor = callbackExecutor; this.processingExecutor = processingExecutor; this.config = config; - this.monitorNodeRemovedExecutor = - Executors.newScheduledThreadPool( - config.getNumZookeeperMonitorThreads(), - Execs.makeThreadFactory("LoadQueuePeon-NodeRemovedMonitor--%d") - ); } @JsonProperty @@ -174,13 +166,12 @@ public int getNumberOfSegmentsInQueue() } @Override - public void loadSegment(final DataSegment segment, final LoadPeonCallback callback) + public void loadSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback) { SegmentHolder segmentHolder = new SegmentHolder(segment, LOAD, Collections.singletonList(callback)); - final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder); if (existingHolder != null) { - if ((callback != null)) { + if (callback != null) { existingHolder.addCallback(callback); } return; @@ -191,10 +182,7 @@ public void loadSegment(final DataSegment segment, final LoadPeonCallback callba } @Override - public void dropSegment( - final DataSegment segment, - final LoadPeonCallback callback - ) + public void dropSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback) { SegmentHolder segmentHolder = new SegmentHolder(segment, DROP, Collections.singletonList(callback)); final SegmentHolder existingHolder = segmentsToDrop.putIfAbsent(segment, segmentHolder); @@ -242,7 +230,7 @@ public void run() segmentHolder.getType() == LOAD ? "load" : "drop", segmentHolder.getSegmentIdentifier() ); - final ScheduledFuture future = monitorNodeRemovedExecutor.schedule( + final ScheduledFuture future = processingExecutor.schedule( () -> { try { if (curator.checkExists().forPath(path) != null) { @@ -319,10 +307,7 @@ private void actionCompleted(SegmentHolder segmentHolder) default: throw new UnsupportedOperationException(); } - - callBackExecutor.execute( - () -> executeCallbacks(segmentHolder) - ); + executeCallbacks(segmentHolder); } @@ -347,7 +332,6 @@ public void stop() failedAssignCount.set(0); processingExecutor.shutdown(); callBackExecutor.shutdown(); - monitorNodeRemovedExecutor.shutdown(); } private void entryRemoved(SegmentHolder segmentHolder, String path) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java index 0a643da12835..b4adebe851c9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java @@ -81,12 +81,6 @@ public int getNumCuratorCallBackThreads() return 2; } - @Config("druid.coordinator.curator.loadqueuepeon.numMonitorThreads") - public int getNumZookeeperMonitorThreads() - { - return 1; - } - @Config("druid.coordinator.loadqueuepeon.http.repeatDelay") public Duration getHttpLoadQueuePeonRepeatDelay() { From 97cd80436d94516e7d53533a83739c41094f0328 Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Mon, 29 Apr 2019 23:36:27 -0700 Subject: [PATCH 20/22] Add back monitor node remove executor --- .../server/coordinator/CuratorLoadQueuePeon.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index 5d3f5ac07709..49a7175df9b0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -26,6 +26,7 @@ import org.apache.curator.framework.api.CuratorWatcher; import org.apache.curator.utils.ZKPaths; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.SegmentChangeRequestDrop; @@ -109,6 +110,12 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST ); + /** + * Threadpool with daemon threads running scheduled tasks that monitor whether + * the zk nodes created for segment processing are removed + */ + private final ScheduledExecutorService monitorNodeRemovedExecutor; + CuratorLoadQueuePeon( CuratorFramework curator, String basePath, @@ -124,6 +131,7 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon this.callBackExecutor = callbackExecutor; this.processingExecutor = processingExecutor; this.config = config; + this.monitorNodeRemovedExecutor = Execs.scheduledSingleThreaded("LoadQueuePeon-NodeRemovedMonitor--%d"); } @JsonProperty @@ -230,7 +238,7 @@ public void run() segmentHolder.getType() == LOAD ? "load" : "drop", segmentHolder.getSegmentIdentifier() ); - final ScheduledFuture future = processingExecutor.schedule( + final ScheduledFuture future = monitorNodeRemovedExecutor.schedule( () -> { try { if (curator.checkExists().forPath(path) != null) { @@ -240,6 +248,7 @@ public void run() } } catch (Exception e) { + log.error(e, "Exception caught and ignored when checking whether zk node was deleted"); failAssign(segmentHolder, e); } }, @@ -332,6 +341,7 @@ public void stop() failedAssignCount.set(0); processingExecutor.shutdown(); callBackExecutor.shutdown(); + monitorNodeRemovedExecutor.shutdown(); } private void entryRemoved(SegmentHolder segmentHolder, String path) From f77942dd93e5e1e40cdbac67cd0052e932bce80d Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Tue, 30 Apr 2019 15:15:19 -0700 Subject: [PATCH 21/22] Cleanup code to isolate null checks and minor refactoring --- .../coordinator/CuratorLoadQueuePeon.java | 83 ++++++++++--------- 1 file changed, 42 insertions(+), 41 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index 49a7175df9b0..7549f7daa497 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -26,7 +26,6 @@ import org.apache.curator.framework.api.CuratorWatcher; import org.apache.curator.utils.ZKPaths; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.SegmentChangeRequestDrop; @@ -37,10 +36,12 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentSkipListMap; @@ -110,12 +111,6 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST ); - /** - * Threadpool with daemon threads running scheduled tasks that monitor whether - * the zk nodes created for segment processing are removed - */ - private final ScheduledExecutorService monitorNodeRemovedExecutor; - CuratorLoadQueuePeon( CuratorFramework curator, String basePath, @@ -131,7 +126,6 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon this.callBackExecutor = callbackExecutor; this.processingExecutor = processingExecutor; this.config = config; - this.monitorNodeRemovedExecutor = Execs.scheduledSingleThreaded("LoadQueuePeon-NodeRemovedMonitor--%d"); } @JsonProperty @@ -179,9 +173,7 @@ public void loadSegment(final DataSegment segment, @Nullable final LoadPeonCallb SegmentHolder segmentHolder = new SegmentHolder(segment, LOAD, Collections.singletonList(callback)); final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder); if (existingHolder != null) { - if (callback != null) { - existingHolder.addCallback(callback); - } + existingHolder.addCallback(callback); return; } log.debug("Asking server peon[%s] to load segment[%s]", basePath, segment.getId()); @@ -195,9 +187,7 @@ public void dropSegment(final DataSegment segment, @Nullable final LoadPeonCallb SegmentHolder segmentHolder = new SegmentHolder(segment, DROP, Collections.singletonList(callback)); final SegmentHolder existingHolder = segmentsToDrop.putIfAbsent(segment, segmentHolder); if (existingHolder != null) { - if (callback != null) { - existingHolder.addCallback(callback); - } + existingHolder.addCallback(callback); return; } log.debug("Asking server peon[%s] to drop segment[%s]", basePath, segment.getId()); @@ -238,31 +228,14 @@ public void run() segmentHolder.getType() == LOAD ? "load" : "drop", segmentHolder.getSegmentIdentifier() ); - final ScheduledFuture future = monitorNodeRemovedExecutor.schedule( - () -> { - try { - if (curator.checkExists().forPath(path) != null) { - failAssign(segmentHolder, new ISE("%s was never removed! Failing this operation!", path)); - } else { - log.debug("%s detected to be removed. ", path); - } - } - catch (Exception e) { - log.error(e, "Exception caught and ignored when checking whether zk node was deleted"); - failAssign(segmentHolder, e); - } - }, - config.getLoadTimeoutDelay().getMillis(), - TimeUnit.MILLISECONDS - ); - + final ScheduledFuture nodeDeletedCheck = scheduleNodeDeletedCheck(path); final Stat stat = curator.checkExists().usingWatcher( (CuratorWatcher) watchedEvent -> { switch (watchedEvent.getType()) { case NodeDeleted: // Cancel the check node deleted task since we have already // been notified by the zk watcher - future.cancel(true); + nodeDeletedCheck.cancel(true); entryRemoved(segmentHolder, watchedEvent.getPath()); break; default: @@ -301,6 +274,28 @@ public void run() failAssign(segmentHolder, e); } } + + @Nonnull + private ScheduledFuture scheduleNodeDeletedCheck(String path) + { + return processingExecutor.schedule( + () -> { + try { + if (curator.checkExists().forPath(path) != null) { + failAssign(segmentHolder, new ISE("%s was never removed! Failing this operation!", path)); + } else { + log.debug("%s detected to be removed. ", path); + } + } + catch (Exception e) { + log.error(e, "Exception caught and ignored when checking whether zk node was deleted"); + failAssign(segmentHolder, e); + } + }, + config.getLoadTimeoutDelay().getMillis(), + TimeUnit.MILLISECONDS + ); + } } private void actionCompleted(SegmentHolder segmentHolder) @@ -341,7 +336,6 @@ public void stop() failedAssignCount.set(0); processingExecutor.shutdown(); callBackExecutor.shutdown(); - monitorNodeRemovedExecutor.shutdown(); } private void entryRemoved(SegmentHolder segmentHolder, String path) @@ -383,6 +377,7 @@ private static class SegmentHolder private final DataSegment segment; private final DataSegmentChangeRequest changeRequest; private final int type; + // Guaranteed to store only non-null elements private final List callbacks = new ArrayList<>(); private SegmentHolder( @@ -396,7 +391,13 @@ private SegmentHolder( this.changeRequest = (type == LOAD) ? new SegmentChangeRequestLoad(segment) : new SegmentChangeRequestDrop(segment); - this.callbacks.addAll(callbacks); + Iterator itr = callbacks.iterator(); + while (itr.hasNext()) { + LoadPeonCallback c = itr.next(); + if (c != null) { + callbacks.add(c); + } + } } public DataSegment getSegment() @@ -419,10 +420,12 @@ public long getSegmentSize() return segment.getSize(); } - public void addCallback(LoadPeonCallback newCallback) + public void addCallback(@Nullable LoadPeonCallback newCallback) { - synchronized (callbacks) { - callbacks.add(newCallback); + if (newCallback != null) { + synchronized (callbacks) { + callbacks.add(newCallback); + } } } @@ -449,9 +452,7 @@ public String toString() private void executeCallbacks(SegmentHolder holder) { for (LoadPeonCallback callback : holder.snapshotCallbacks()) { - if (callback != null) { - callBackExecutor.submit(() -> callback.execute()); - } + callBackExecutor.submit(() -> callback.execute()); } } } From c762c48a86a9b096423267dd2a981e20eb20d5ea Mon Sep 17 00:00:00 2001 From: Samarth Jain Date: Tue, 30 Apr 2019 16:12:41 -0700 Subject: [PATCH 22/22] Change param name since it conflicts with member variable name --- .../apache/druid/server/coordinator/CuratorLoadQueuePeon.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java index 7549f7daa497..a4d5d948cf07 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java @@ -383,7 +383,7 @@ private static class SegmentHolder private SegmentHolder( DataSegment segment, int type, - Collection callbacks + Collection callbacksParam ) { this.segment = segment; @@ -391,7 +391,7 @@ private SegmentHolder( this.changeRequest = (type == LOAD) ? new SegmentChangeRequestLoad(segment) : new SegmentChangeRequestDrop(segment); - Iterator itr = callbacks.iterator(); + Iterator itr = callbacksParam.iterator(); while (itr.hasNext()) { LoadPeonCallback c = itr.next(); if (c != null) {