You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by ji...@apache.org on 2019/02/20 19:56:28 UTC

[incubator-druid] branch master updated: Coordinator await initialization before finishing startup (#6847)

This is an automated email from the ASF dual-hosted git repository.

jihoonson pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/master by this push:
     new dd34691  Coordinator await initialization before finishing startup (#6847)
dd34691 is described below

commit dd34691004ca87128e775136ac99be404191f897
Author: Mingming Qiu <cs...@gmail.com>
AuthorDate: Thu Feb 21 03:56:23 2019 +0800

    Coordinator await initialization before finishing startup (#6847)
    
    * Curator server inventory await initialization
    
    * address comments
    
    * print exception object in log
    
    * remove throws ISE
    
    * cachingCost awaitInitialization default to false
---
 docs/content/configuration/index.md                |  6 ++--
 .../client/CoordinatorSegmentWatcherConfig.java    | 33 +++++++++++++++++++
 .../apache/druid/client/CoordinatorServerView.java | 23 ++++++++++++-
 .../druid/client/HttpServerInventoryView.java      | 13 +-------
 .../CachingCostBalancerStrategyConfig.java         | 33 +++++++++++++++++++
 .../CachingCostBalancerStrategyFactory.java        | 38 ++++++++++++++++------
 .../druid/client/CoordinatorServerViewTest.java    |  3 +-
 .../coordinator/CuratorDruidCoordinatorTest.java   |  3 +-
 .../java/org/apache/druid/cli/CliCoordinator.java  | 11 ++++++-
 9 files changed, 135 insertions(+), 28 deletions(-)

diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md
index 639bb45..22dd1ae 100644
--- a/docs/content/configuration/index.md
+++ b/docs/content/configuration/index.md
@@ -726,7 +726,8 @@ These Coordinator static configurations can be defined in the `coordinator/runti
 |`druid.coordinator.kill.period`|How often to send kill tasks to the indexing service. Value must be greater than `druid.coordinator.period.indexingPeriod`. Only applies if kill is turned on.|P1D (1 Day)|
 |`druid.coordinator.kill.durationToRetain`| Do not kill segments in last `durationToRetain`, must be greater or equal to 0. Only applies and MUST be specified if kill is turned on. Note that default value is invalid.|PT-1S (-1 seconds)|
 |`druid.coordinator.kill.maxSegments`|Kill at most n segments per kill task submission, must be greater than 0. Only applies and MUST be specified if kill is turned on. Note that default value is invalid.|0|
-|`druid.coordinator.balancer.strategy`|Specify the type of balancing strategy that the Coordinator should use to distribute segments among the Historicals. `cachingCost` is logically equivalent to `cost` but is more CPU-efficient on large clusters and will replace `cost` in the future versions, users are invited to try it. Use `diskNormalized` to distribute segments among nodes so that the disks fill up uniformly and use `random` to randomly pick nodes to distribute segments.|`cost`|
+|`druid.coordinator.balancer.strategy`|Specify the type of balancing strategy that the coordinator should use to distribute segments among the historicals. `cachingCost` is logically equivalent to `cost` but is more CPU-efficient on large clusters and will replace `cost` in the future versions, users are invited to try it. Use `diskNormalized` to distribute segments among nodes so that the disks fill up uniformly and use `random` to randomly pick nodes to distribute segments.|`cost`|
+|`druid.coordinator.balancer.cachingCost.awaitInitialization`|Whether to wait for segment view initialization before creating the `cachingCost` balancing strategy. This property is enabled only when `druid.coordinator.balancer.strategy` is `cachingCost`. If set to 'true', the Coordinator will not start to assign segments, until the segment view is initialized. If set to 'false', the Coordinator will fallback to use the `cost` balancing strategy only if the segment view is not initialized [...]
 |`druid.coordinator.loadqueuepeon.repeatDelay`|The start and repeat delay for the loadqueuepeon , which manages the load and drop of segments.|PT0.050S (50 ms)|
 |`druid.coordinator.asOverlord.enabled`|Boolean value for whether this Coordinator node should act like an Overlord as well. This configuration allows users to simplify a druid cluster by not having to deploy any standalone Overlord nodes. If set to true, then Overlord console is available at `http://coordinator-host:port/console.html` and be sure to set `druid.coordinator.asOverlord.overlordService` also. See next.|false|
 |`druid.coordinator.asOverlord.overlordService`| Required, if `druid.coordinator.asOverlord.enabled` is `true`. This must be same value as `druid.service` on standalone Overlord nodes and `druid.selectors.indexing.serviceName` on Middle Managers.|NULL|
@@ -735,7 +736,8 @@ These Coordinator static configurations can be defined in the `coordinator/runti
 |Property|Possible Values|Description|Default|
 |--------|---------------|-----------|-------|
 |`druid.serverview.type`|batch or http|Segment discovery method to use. "http" enables discovering segments using HTTP instead of zookeeper.|batch|
-|`druid.coordinator.loadqueuepeon.type`|curator or http|Whether to use "http" or "curator" implementation to assign segment loads/drops to Historical|curator|
+|`druid.coordinator.loadqueuepeon.type`|curator or http|Whether to use "http" or "curator" implementation to assign segment loads/drops to historical|curator|
+|`druid.coordinator.segment.awaitInitializationOnStart`|true or false|Whether the the Coordinator will wait for its view of segments to fully initialize before starting up. If set to 'true', the Coordinator's HTTP server will not start up, and the Coordinator will not announce itself as available, until the server view is initialized.|true|
 
 ###### Additional config when "http" loadqueuepeon is used
 |Property|Description|Default|
diff --git a/server/src/main/java/org/apache/druid/client/CoordinatorSegmentWatcherConfig.java b/server/src/main/java/org/apache/druid/client/CoordinatorSegmentWatcherConfig.java
new file mode 100644
index 0000000..ed51012
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/client/CoordinatorSegmentWatcherConfig.java
@@ -0,0 +1,33 @@
+/*
+ * 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.client;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class CoordinatorSegmentWatcherConfig
+{
+  @JsonProperty
+  private boolean awaitInitializationOnStart = true;
+
+  public boolean isAwaitInitializationOnStart()
+  {
+    return awaitInitializationOnStart;
+  }
+}
diff --git a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java
index 3263beb..91ebe38 100644
--- a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java
+++ b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java
@@ -22,7 +22,9 @@ package org.apache.druid.client;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Ordering;
 import com.google.inject.Inject;
+import org.apache.druid.guice.ManageLifecycle;
 import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.query.DataSource;
 import org.apache.druid.server.coordination.DruidServerMetadata;
@@ -34,11 +36,13 @@ import org.apache.druid.timeline.partition.PartitionChunk;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 
 /**
  * ServerView of coordinator for the state of segments being loaded in the cluster.
  */
+@ManageLifecycle
 public class CoordinatorServerView implements InventoryView
 {
   private static final Logger log = new Logger(CoordinatorServerView.class);
@@ -49,13 +53,18 @@ public class CoordinatorServerView implements InventoryView
   private final Map<String, VersionedIntervalTimeline<String, SegmentLoadInfo>> timelines;
 
   private final ServerInventoryView baseView;
+  private final CoordinatorSegmentWatcherConfig segmentWatcherConfig;
+
+  private final CountDownLatch initialized = new CountDownLatch(1);
 
   @Inject
   public CoordinatorServerView(
-      ServerInventoryView baseView
+      ServerInventoryView baseView,
+      CoordinatorSegmentWatcherConfig segmentWatcherConfig
   )
   {
     this.baseView = baseView;
+    this.segmentWatcherConfig = segmentWatcherConfig;
     this.segmentLoadInfos = new HashMap<>();
     this.timelines = new HashMap<>();
 
@@ -81,6 +90,7 @@ public class CoordinatorServerView implements InventoryView
           @Override
           public ServerView.CallbackAction segmentViewInitialized()
           {
+            initialized.countDown();
             return ServerView.CallbackAction.CONTINUE;
           }
         }
@@ -100,6 +110,17 @@ public class CoordinatorServerView implements InventoryView
     );
   }
 
+  @LifecycleStart
+  public void start() throws InterruptedException
+  {
+    if (segmentWatcherConfig.isAwaitInitializationOnStart()) {
+      final long startMillis = System.currentTimeMillis();
+      log.info("%s waiting for initialization.", getClass().getSimpleName());
+      initialized.await();
+      log.info("%s initialized in [%,d] ms.", getClass().getSimpleName(), System.currentTimeMillis() - startMillis);
+    }
+  }
+
   private void removeServer(DruidServer server)
   {
     for (DataSegment segment : server.getSegments()) {
diff --git a/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java b/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java
index be08117..1279b3a 100644
--- a/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java
+++ b/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java
@@ -64,7 +64,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
@@ -111,8 +110,6 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
   private final ObjectMapper smileMapper;
   private final HttpServerInventoryViewConfig config;
 
-  private final CountDownLatch inventoryInitializationLatch = new CountDownLatch(1);
-
   @Inject
   public HttpServerInventoryView(
       final @Smile ObjectMapper smileMapper,
@@ -132,7 +129,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
 
 
   @LifecycleStart
-  public void start() throws Exception
+  public void start()
   {
     synchronized (lifecycleLock) {
       if (!lifecycleLock.canStart()) {
@@ -197,12 +194,6 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
         lifecycleLock.exitStart();
       }
 
-      log.info("Waiting for Server Inventory Initialization...");
-
-      while (!inventoryInitializationLatch.await(1, TimeUnit.MINUTES)) {
-        log.info("Still waiting for Server Inventory Initialization...");
-      }
-
       log.info("Started HttpServerInventoryView.");
     }
   }
@@ -371,8 +362,6 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
       }
     }
 
-    inventoryInitializationLatch.countDown();
-
     log.info("Calling SegmentCallback.segmentViewInitialized() for all callbacks.");
 
     runSegmentCallbacks(
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyConfig.java
new file mode 100644
index 0000000..eb0a668
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyConfig.java
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+public class CachingCostBalancerStrategyConfig
+{
+  @JsonProperty
+  private boolean awaitInitialization = false;
+
+  public boolean isAwaitInitialization()
+  {
+    return awaitInitialization;
+  }
+}
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java
index 410a24d..93df9ab 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java
@@ -35,10 +35,10 @@ import org.apache.druid.timeline.DataSegment;
 
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 public class CachingCostBalancerStrategyFactory implements BalancerStrategyFactory
 {
@@ -47,19 +47,19 @@ public class CachingCostBalancerStrategyFactory implements BalancerStrategyFacto
   /** Must be single-threaded, because {@link ClusterCostCache.Builder} and downstream builders are not thread-safe */
   private final ExecutorService executor = Execs.singleThreaded("CachingCostBalancerStrategy-executor");
   private final ClusterCostCache.Builder clusterCostCacheBuilder = ClusterCostCache.builder();
-  /**
-   * Atomic is needed to use compareAndSet(true, true) construction below, that is linearizable with the write made from
-   * callback, that ensures visibility of the write made from callback. Neither plain field nor volatile field read
-   * ensure such visibility
-   */
-  private final AtomicBoolean initialized = new AtomicBoolean(false);
+
+  private final CountDownLatch initialized = new CountDownLatch(1);
+  private final CachingCostBalancerStrategyConfig config;
 
   @JsonCreator
   public CachingCostBalancerStrategyFactory(
       @JacksonInject ServerInventoryView serverInventoryView,
-      @JacksonInject Lifecycle lifecycle
+      @JacksonInject Lifecycle lifecycle,
+      @JacksonInject CachingCostBalancerStrategyConfig config
   ) throws Exception
   {
+    this.config = config;
+
     // Adding to lifecycle dynamically because couldn't use @ManageLifecycle on the class,
     // see https://github.com/apache/incubator-druid/issues/4980
     lifecycle.addMaybeStartManagedInstance(this);
@@ -89,7 +89,7 @@ public class CachingCostBalancerStrategyFactory implements BalancerStrategyFacto
           @Override
           public ServerView.CallbackAction segmentViewInitialized()
           {
-            initialized.set(true);
+            initialized.countDown();
             return ServerView.CallbackAction.CONTINUE;
           }
         }
@@ -118,10 +118,28 @@ public class CachingCostBalancerStrategyFactory implements BalancerStrategyFacto
     executor.shutdownNow();
   }
 
+  private boolean isInitialized()
+  {
+    return initialized.getCount() == 0;
+  }
+
   @Override
   public BalancerStrategy createBalancerStrategy(final ListeningExecutorService exec)
   {
-    if (initialized.compareAndSet(true, true)) {
+    if (!isInitialized() && config.isAwaitInitialization()) {
+      try {
+        final long startMillis = System.currentTimeMillis();
+        LOG.info("Waiting for segment view initialization before creating CachingCostBalancerStrategy.");
+        initialized.await();
+        LOG.info("Segment view initialized in [%,d] ms.", System.currentTimeMillis() - startMillis);
+      }
+      catch (InterruptedException e) {
+        LOG.error(e, "Segment view initialization has been interrupted.");
+        Thread.currentThread().interrupt();
+      }
+    }
+
+    if (isInitialized()) {
       try {
         // Calling clusterCostCacheBuilder.build() in the same thread (executor's sole thread) where
         // clusterCostCacheBuilder is updated, to avoid problems with concurrent updates
diff --git a/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java b/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java
index 6550d97..f6e2698 100644
--- a/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java
+++ b/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java
@@ -328,7 +328,8 @@ public class CoordinatorServerViewTest extends CuratorTestBase
     };
 
     overlordServerView = new CoordinatorServerView(
-        baseView
+        baseView,
+        new CoordinatorSegmentWatcherConfig()
     );
 
     baseView.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 68bce7a..94fecdb 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
@@ -29,6 +29,7 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCache;
 import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
 import org.apache.curator.utils.ZKPaths;
 import org.apache.druid.client.BatchServerInventoryView;
+import org.apache.druid.client.CoordinatorSegmentWatcherConfig;
 import org.apache.druid.client.CoordinatorServerView;
 import org.apache.druid.client.DruidServer;
 import org.apache.druid.client.ImmutableDruidDataSource;
@@ -472,7 +473,7 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
       }
     };
 
-    serverView = new CoordinatorServerView(baseView);
+    serverView = new CoordinatorServerView(baseView, new CoordinatorSegmentWatcherConfig());
 
     baseView.start();
 
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 1e4daae..3ba06cf 100644
--- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java
+++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java
@@ -29,6 +29,7 @@ import com.google.inject.name.Names;
 import io.airlift.airline.Command;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.druid.audit.AuditManager;
+import org.apache.druid.client.CoordinatorSegmentWatcherConfig;
 import org.apache.druid.client.CoordinatorServerView;
 import org.apache.druid.client.HttpServerInventoryViewResource;
 import org.apache.druid.client.coordinator.Coordinator;
@@ -58,6 +59,7 @@ import org.apache.druid.metadata.MetadataStorage;
 import org.apache.druid.metadata.MetadataStorageProvider;
 import org.apache.druid.server.audit.AuditManagerProvider;
 import org.apache.druid.server.coordinator.BalancerStrategyFactory;
+import org.apache.druid.server.coordinator.CachingCostBalancerStrategyConfig;
 import org.apache.druid.server.coordinator.DruidCoordinator;
 import org.apache.druid.server.coordinator.DruidCoordinatorCleanupPendingSegments;
 import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
@@ -148,6 +150,12 @@ public class CliCoordinator extends ServerRunnable
             JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class);
             JsonConfigProvider.bind(binder, "druid.manager.lookups", LookupCoordinatorManagerConfig.class);
             JsonConfigProvider.bind(binder, "druid.coordinator.balancer", BalancerStrategyFactory.class);
+            JsonConfigProvider.bind(binder, "druid.coordinator.segment", CoordinatorSegmentWatcherConfig.class);
+            JsonConfigProvider.bind(
+                binder,
+                "druid.coordinator.balancer.cachingCost",
+                CachingCostBalancerStrategyConfig.class
+            );
 
             binder.bind(RedirectFilter.class).in(LazySingleton.class);
             if (beOverlord) {
@@ -169,11 +177,12 @@ public class CliCoordinator extends ServerRunnable
                   .in(ManageLifecycle.class);
 
             binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class);
-            binder.bind(CoordinatorServerView.class).in(LazySingleton.class);
 
             binder.bind(LookupCoordinatorManager.class).in(LazySingleton.class);
+            binder.bind(CoordinatorServerView.class);
             binder.bind(DruidCoordinator.class);
 
+            LifecycleModule.register(binder, CoordinatorServerView.class);
             LifecycleModule.register(binder, MetadataStorage.class);
             LifecycleModule.register(binder, DruidCoordinator.class);
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org