You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/09/16 00:41:23 UTC

[GitHub] [druid] paul-rogers commented on a diff in pull request #13074: Add test framework to simulate segment loading and balancing

paul-rogers commented on code in PR #13074:
URL: https://github.com/apache/druid/pull/13074#discussion_r972514017


##########
server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.simulate;
+
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.emitter.core.Event;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.CreateDataSegments;
+import org.apache.druid.server.coordinator.rules.ForeverLoadRule;
+import org.apache.druid.server.coordinator.rules.Rule;
+import org.apache.druid.timeline.DataSegment;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Base test for coordinator simulations.
+ * <p>
+ * Each test must call {@link #startSimulation(CoordinatorSimulation)} to start
+ * the simulation. {@link CoordinatorSimulation#stop()} should not be called as
+ * the simulation is stopped when cleaning up after the test in {@link #tearDown()}.
+ * <p>
+ * Tests that verify balancing behaviour should set
+ * {@link CoordinatorDynamicConfig#useBatchedSegmentSampler()} to true.
+ * Otherwise, the segment sampling is random and can produce repeated values
+ * leading to flakiness in the tests. The simulation sets this field to true by
+ * default.
+ */
+public abstract class CoordinatorSimulationBaseTest

Review Comment:
   Consider making this a "fixture" rather than a base test. With a fixture, you write a test like so:
   
   ```java
   class MyTest
   {
     CoordinatorSimulationFixture fixture = new CoordinatorSimulationFixture(...);
   
     @Before
     public void setup()
     {
       fixture.setup();
       // My own setup
     }
   
     @Test
     public void myTest()
     {
       fixture.startSimulation(...);
       fixture.doOtherStuff(...);
     }
   ```
   
   The point is that the test class is simple. The test writer can pull in other dependencies without having a multiple-inheritance problem. Etc.



##########
core/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java:
##########
@@ -37,7 +41,18 @@ public StubServiceEmitter(String service, String host)
   @Override
   public void emit(Event event)
   {
-    events.add(event);
+    if (event instanceof AlertEvent) {
+      final AlertEvent alertEvent = (AlertEvent) event;
+      log.warn(
+          "[%s] [%s] [%s]: %s%n",
+          alertEvent.getSeverity(),
+          alertEvent.getService(),
+          alertEvent.getFeed(),
+          alertEvent.getDescription()

Review Comment:
   Pattern expects four string and a number; only four strings provided.



##########
server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentBalancingTest.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.simulate;
+
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.query.DruidMetrics;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.timeline.DataSegment;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Coordinator simulation test to verify behaviour of segment balancing.
+ */
+public class SegmentBalancingTest extends CoordinatorSimulationBaseTest
+{
+  private DruidServer historicalT11;
+  private DruidServer historicalT12;
+
+  private final String datasource = DS.WIKI;
+  private final List<DataSegment> segments = Segments.WIKI_10X1D;
+
+  @Override
+  public void setUp()
+  {
+    // Setup historicals for 2 tiers, size 10 GB each
+    historicalT11 = createHistorical(1, Tier.T1, 10_000);
+    historicalT12 = createHistorical(2, Tier.T1, 10_000);
+  }
+
+  @Test
+  public void testBalancingWithSyncedInventory()
+  {
+    testBalancingWithInventorySynced(true);
+  }
+
+  @Test
+  @Ignore("Fix #12881 to enable this test. "
+          + "Current impl requires updated inventory for correct callback behaviour.")
+  public void testBalancingWithUnsyncedInventory()
+  {
+    testBalancingWithInventorySynced(false);
+  }
+
+  private void testBalancingWithInventorySynced(boolean autoSyncInventory)

Review Comment:
   Pretty cool!



##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -483,7 +483,7 @@ public void moveSegment(
         );
       }
 
-      final String toLoadQueueSegPath =
+      final String toLoadQueueSegPath = curator == null ? null :

Review Comment:
   As far as I can tell, none of the arguments here depend on ZK. So, we can define the path even if we don't actually have a ZK.



##########
server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java:
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.simulate;
+
+import com.fasterxml.jackson.databind.InjectableValues;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.common.config.JacksonConfigManager;
+import org.apache.druid.curator.ZkEnablementConfig;
+import org.apache.druid.curator.discovery.ServiceAnnouncer;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.concurrent.DirectExecutorService;
+import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory;
+import org.apache.druid.java.util.common.lifecycle.Lifecycle;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.core.Event;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.java.util.metrics.StubServiceEmitter;
+import org.apache.druid.server.coordinator.BalancerStrategyFactory;
+import org.apache.druid.server.coordinator.CachingCostBalancerStrategyConfig;
+import org.apache.druid.server.coordinator.CachingCostBalancerStrategyFactory;
+import org.apache.druid.server.coordinator.CoordinatorCompactionConfig;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.CostBalancerStrategyFactory;
+import org.apache.druid.server.coordinator.DruidCoordinator;
+import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
+import org.apache.druid.server.coordinator.LoadQueueTaskMaster;
+import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig;
+import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroups;
+import org.apache.druid.server.coordinator.rules.Rule;
+import org.apache.druid.server.lookup.cache.LookupCoordinatorManager;
+import org.apache.druid.timeline.DataSegment;
+import org.easymock.EasyMock;
+import org.joda.time.Duration;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Builder for {@link CoordinatorSimulation}.
+ */
+public class CoordinatorSimulationBuilder
+{
+  private static final long DEFAULT_COORDINATOR_PERIOD = 100L;
+  private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper()
+      .setInjectableValues(
+          new InjectableValues.Std().addValue(
+              DataSegment.PruneSpecsHolder.class,
+              DataSegment.PruneSpecsHolder.DEFAULT
+          )
+      );
+
+  private BalancerStrategyFactory balancerStrategyFactory;
+  private CoordinatorDynamicConfig dynamicConfig =
+      CoordinatorDynamicConfig.builder()
+                              .withUseBatchedSegmentSampler(true)
+                              .build();
+  private List<DruidServer> servers;
+  private List<DataSegment> segments;
+  private final Map<String, List<Rule>> datasourceRules = new HashMap<>();
+  private boolean loadImmediately = false;
+  private boolean autoSyncInventory = true;
+
+  /**
+   * Specifies the balancer strategy to be used.
+   * <p>
+   * Default: "cost" ({@link CostBalancerStrategyFactory})
+   */
+  public CoordinatorSimulationBuilder balancer(BalancerStrategyFactory strategyFactory)
+  {
+    this.balancerStrategyFactory = strategyFactory;
+    return this;
+  }
+
+  public CoordinatorSimulationBuilder servers(List<DruidServer> servers)
+  {
+    this.servers = servers;
+    return this;
+  }
+
+  public CoordinatorSimulationBuilder servers(DruidServer... servers)
+  {
+    return servers(Arrays.asList(servers));
+  }
+
+  public CoordinatorSimulationBuilder segments(List<DataSegment> segments)
+  {
+    this.segments = segments;
+    return this;
+  }
+
+  public CoordinatorSimulationBuilder rules(String datasource, Rule... rules)
+  {
+    this.datasourceRules.put(datasource, Arrays.asList(rules));
+    return this;
+  }
+
+  /**
+   * Specifies whether segments should be loaded as soon as they are queued.
+   * <p>
+   * Default: false
+   */
+  public CoordinatorSimulationBuilder loadSegmentsImmediately(boolean loadImmediately)
+  {
+    this.loadImmediately = loadImmediately;
+    return this;
+  }
+
+  /**
+   * Specifies whether the inventory view maintained by the coordinator
+   * should be auto-synced as soon as any change is made to the cluster.
+   * <p>
+   * Default: true
+   */
+  public CoordinatorSimulationBuilder autoSyncInventory(boolean autoSync)
+  {
+    this.autoSyncInventory = autoSync;
+    return this;
+  }
+
+  /**
+   * Specifies the CoordinatorDynamicConfig to be used in the simulation.
+   * <p>
+   * Default values: Specified in {@link CoordinatorDynamicConfig.Builder}.
+   */
+  public CoordinatorSimulationBuilder dynamicConfig(CoordinatorDynamicConfig dynamicConfig)
+  {
+    this.dynamicConfig = dynamicConfig;
+    return this;
+  }
+
+  public CoordinatorSimulation build()
+  {
+    Preconditions.checkArgument(
+        servers != null && !servers.isEmpty(),
+        "Cannot run simulation for an empty cluster"
+    );
+
+    // Prepare the environment
+    final TestServerInventoryView serverInventoryView = new TestServerInventoryView();
+    servers.forEach(serverInventoryView::addServer);
+
+    final TestSegmentsMetadataManager segmentManager = new TestSegmentsMetadataManager();
+    if (segments != null) {
+      segments.forEach(segmentManager::addSegment);
+    }
+
+    final TestMetadataRuleManager ruleManager = new TestMetadataRuleManager();
+    datasourceRules.forEach(
+        (datasource, rules) ->
+            ruleManager.overrideRule(datasource, rules, null)
+    );
+
+    final Environment env = new Environment(
+        serverInventoryView,
+        segmentManager,
+        ruleManager,
+        dynamicConfig,
+        loadImmediately,
+        autoSyncInventory
+    );
+
+    // Build the coordinator
+    final DruidCoordinator coordinator = new DruidCoordinator(
+        env.coordinatorConfig,
+        null,
+        env.jacksonConfigManager,
+        env.segmentManager,
+        env.coordinatorInventoryView,
+        env.ruleManager,
+        () -> null,
+        env.serviceEmitter,
+        env.executorFactory,
+        null,
+        env.loadQueueTaskMaster,
+        new ServiceAnnouncer.Noop(),
+        null,
+        Collections.emptySet(),
+        null,
+        new CoordinatorCustomDutyGroups(Collections.emptySet()),
+        balancerStrategyFactory != null ? balancerStrategyFactory
+                                        : new CostBalancerStrategyFactory(),
+        env.lookupCoordinatorManager,
+        env.leaderSelector,
+        OBJECT_MAPPER,
+        ZkEnablementConfig.ENABLED
+    );
+
+    return new SimulationImpl(coordinator, env);
+  }
+
+  private BalancerStrategyFactory buildCachingCostBalancerStrategy(Environment env)
+  {
+    try {
+      return new CachingCostBalancerStrategyFactory(
+          env.coordinatorInventoryView,
+          env.lifecycle,
+          new CachingCostBalancerStrategyConfig()
+      );
+    }
+    catch (Exception e) {
+      throw new ISE(e, "Error building balancer strategy");
+    }
+  }
+
+  /**
+   * Implementation of {@link CoordinatorSimulation}.
+   */
+  private static class SimulationImpl implements CoordinatorSimulation,
+      CoordinatorSimulation.CoordinatorState, CoordinatorSimulation.ClusterState
+  {
+    private final AtomicBoolean running = new AtomicBoolean(false);
+
+    private final Environment env;
+    private final DruidCoordinator coordinator;
+
+    private SimulationImpl(DruidCoordinator coordinator, Environment env)
+    {
+      this.env = env;
+      this.coordinator = coordinator;
+    }
+
+    @Override
+    public void start()
+    {
+      if (!running.compareAndSet(false, true)) {
+        throw new ISE("Simulation is already running");
+      }
+
+      try {
+        env.setUp();
+        coordinator.start();
+      }
+      catch (Exception e) {
+        throw new ISE(e, "Exception while running simulation");
+      }
+    }
+
+    @Override
+    public void stop()
+    {
+      coordinator.stop();
+      env.leaderSelector.stopBeingLeader();
+      env.tearDown();
+    }
+
+    @Override
+    public CoordinatorState coordinator()
+    {
+      return this;
+    }
+
+    @Override
+    public ClusterState cluster()
+    {
+      return this;
+    }
+
+    @Override
+    public void runCoordinatorCycle()
+    {
+      verifySimulationRunning();
+      env.serviceEmitter.flush();
+
+      // Invoke historical duties and metadata duties
+      env.executorFactory.coordinatorRunner.finishNextPendingTasks(2);
+    }
+
+    @Override
+    public void syncInventoryView()
+    {
+      verifySimulationRunning();
+      Preconditions.checkState(
+          !env.autoSyncInventory,
+          "Cannot invoke syncInventoryView as simulation is running in auto-sync mode."
+      );
+      env.coordinatorInventoryView.sync(env.historicalInventoryView);
+    }
+
+    @Override
+    public void setDynamicConfig(CoordinatorDynamicConfig dynamicConfig)
+    {
+      env.setDynamicConfig(dynamicConfig);
+    }
+
+    @Override
+    public DruidServer getInventoryView(String serverName)
+    {
+      return env.coordinatorInventoryView.getInventoryValue(serverName);
+    }
+
+    @Override
+    public void loadQueuedSegments()
+    {
+      verifySimulationRunning();
+      Preconditions.checkState(
+          !env.loadImmediately,
+          "Cannot invoke loadQueuedSegments as simulation is running in immediate loading mode."
+      );
+
+      final BlockingExecutorService loadQueueExecutor = env.executorFactory.loadQueueExecutor;
+      while (loadQueueExecutor.hasPendingTasks()) {
+        // Drain all the items from the load queue executor
+        // This sends at most 1 load/drop request to each server
+        loadQueueExecutor.finishAllPendingTasks();
+
+        // Load all the queued segments, handle their responses and execute callbacks
+        int loadedSegments = env.executorFactory.historicalLoader.finishAllPendingTasks();
+        loadQueueExecutor.finishNextPendingTasks(loadedSegments);
+
+        // TODO: sync should happen here?? or should it not??
+
+        env.executorFactory.loadCallbackExecutor.finishAllPendingTasks();
+      }
+    }
+
+    private void verifySimulationRunning()
+    {
+      if (!running.get()) {
+        throw new ISE("Simulation hasn't been started yet.");
+      }
+    }
+
+    @Override
+    public double getLoadPercentage(String datasource)
+    {
+      return coordinator.getLoadStatus().get(datasource);
+    }
+
+    @Override
+    public List<Event> getMetricEvents()
+    {
+      return new ArrayList<>(env.serviceEmitter.getEvents());
+    }
+  }
+
+  /**
+   * Environment for a coordinator simulation.
+   */
+  private static class Environment
+  {
+    private final Lifecycle lifecycle = new Lifecycle("coord-sim");
+
+    // Executors
+    private final ExecutorFactory executorFactory;
+
+    private final TestDruidLeaderSelector leaderSelector = new TestDruidLeaderSelector();
+    private final TestSegmentsMetadataManager segmentManager;
+    private final TestMetadataRuleManager ruleManager;
+    private final TestServerInventoryView historicalInventoryView;
+
+    private final LoadQueueTaskMaster loadQueueTaskMaster;
+    private final StubServiceEmitter serviceEmitter
+        = new StubServiceEmitter("coordinator", "coordinator");
+    private final TestServerInventoryView coordinatorInventoryView;
+
+    private final AtomicReference<CoordinatorDynamicConfig> dynamicConfig = new AtomicReference<>();
+    private final JacksonConfigManager jacksonConfigManager;
+    private final LookupCoordinatorManager lookupCoordinatorManager;
+    private final DruidCoordinatorConfig coordinatorConfig;
+    private final boolean loadImmediately;
+    private final boolean autoSyncInventory;
+
+    private final List<Object> mocks = new ArrayList<>();
+
+    private Environment(
+        TestServerInventoryView clusterInventory,
+        TestSegmentsMetadataManager segmentManager,
+        TestMetadataRuleManager ruleManager,
+        CoordinatorDynamicConfig dynamicConfig,
+        boolean loadImmediately,
+        boolean autoSyncInventory
+    )
+    {
+      this.historicalInventoryView = clusterInventory;
+      this.segmentManager = segmentManager;
+      this.ruleManager = ruleManager;
+      this.loadImmediately = loadImmediately;
+      this.autoSyncInventory = autoSyncInventory;
+
+      this.coordinatorConfig = new TestDruidCoordinatorConfig.Builder()
+          .withCoordinatorStartDelay(new Duration(1L))
+          .withCoordinatorPeriod(new Duration(DEFAULT_COORDINATOR_PERIOD))
+          .withCoordinatorKillPeriod(new Duration(DEFAULT_COORDINATOR_PERIOD))
+          .withLoadQueuePeonRepeatDelay(new Duration("PT0S"))
+          .withLoadQueuePeonType("http")
+          .withCoordinatorKillIgnoreDurationToRetain(false)
+          .build();
+
+      this.executorFactory = new ExecutorFactory(loadImmediately);
+      this.coordinatorInventoryView = autoSyncInventory
+                                      ? clusterInventory
+                                      : new TestServerInventoryView();
+      HttpClient httpClient = new TestSegmentLoadingHttpClient(
+          OBJECT_MAPPER,
+          clusterInventory::getChangeHandlerForHost,
+          executorFactory.create(1, ExecutorFactory.HISTORICAL_LOADER)
+      );
+
+      this.loadQueueTaskMaster = new LoadQueueTaskMaster(
+          null,
+          OBJECT_MAPPER,
+          executorFactory.create(1, ExecutorFactory.LOAD_QUEUE_EXECUTOR),
+          executorFactory.create(1, ExecutorFactory.LOAD_CALLBACK_EXECUTOR),
+          coordinatorConfig,
+          httpClient,
+          null
+      );
+
+      this.jacksonConfigManager = mockConfigManager();
+      setDynamicConfig(dynamicConfig);
+
+      this.lookupCoordinatorManager = EasyMock.createNiceMock(LookupCoordinatorManager.class);
+      mocks.add(jacksonConfigManager);
+      mocks.add(lookupCoordinatorManager);
+    }
+
+    private void setUp() throws Exception
+    {
+      EmittingLogger.registerEmitter(serviceEmitter);
+      historicalInventoryView.setUp();
+      coordinatorInventoryView.setUp();
+      lifecycle.start();
+      executorFactory.setUp();
+      leaderSelector.becomeLeader();
+      EasyMock.replay(mocks.toArray());
+    }
+
+    private void tearDown()
+    {
+      EasyMock.verify(mocks.toArray());
+      executorFactory.tearDown();
+      lifecycle.stop();
+    }
+
+    private void setDynamicConfig(CoordinatorDynamicConfig dynamicConfig)
+    {
+      this.dynamicConfig.set(dynamicConfig);
+    }
+
+    private JacksonConfigManager mockConfigManager()
+    {
+      final JacksonConfigManager jacksonConfigManager
+          = EasyMock.createMock(JacksonConfigManager.class);
+      EasyMock.expect(
+          jacksonConfigManager.watch(
+              EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY),
+              EasyMock.eq(CoordinatorDynamicConfig.class),
+              EasyMock.anyObject()
+          )
+      ).andReturn(dynamicConfig).anyTimes();
+
+      EasyMock.expect(
+          jacksonConfigManager.watch(
+              EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY),
+              EasyMock.eq(CoordinatorCompactionConfig.class),
+              EasyMock.anyObject()

Review Comment:
   Suggestion: rather than mocking a config, add a constructor or builder to the config. A class that cannot be constructed (as with our config classes) is very tedious to use in tests. Mocking isn't the answer since, if there are methods that compute values, those methods also must be mocked.
   
   Another solution is to use Guice. Use the new startup config builder and friends to pass in the set of properties, then ask the injector to create config instances. Going that route is a bit overkill, but it avoids the need to add constructors: we just use the Json config process we already have.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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