You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "adarshsanjeev (via GitHub)" <gi...@apache.org> on 2023/05/22 03:17:38 UTC

[GitHub] [druid] adarshsanjeev opened a new pull request, #14322: [WIP][Do not merge] Add code to wait for segments generated to be loaded on historicals

adarshsanjeev opened a new pull request, #14322:
URL: https://github.com/apache/druid/pull/14322

   Currently, after an MSQ query, the web console is responsible for waiting for the segments to load. It does so by checking if there are any segments loading into the datasource ingested into, which can cause some issues, like in cases where the segments would never be loaded, or would end up waiting for other ingests as well.
   
   This PR shifts this responsibility to the controller, which would have the list of segments created.
   
   <hr>
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items from the checklist below are strictly necessary, but it would be very helpful if you at least self-review the PR. -->
   
   This PR has:
   
   - [ ] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] a release note entry in the PR description.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   


-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1288022459


##########
server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.discovery;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Injector;
+import com.google.inject.Key;
+import com.google.inject.Module;
+import com.google.inject.name.Names;
+import org.apache.druid.guice.GuiceInjectors;
+import org.apache.druid.guice.Jerseys;
+import org.apache.druid.guice.JsonConfigProvider;
+import org.apache.druid.guice.LazySingleton;
+import org.apache.druid.guice.LifecycleModule;
+import org.apache.druid.guice.annotations.Self;
+import org.apache.druid.initialization.Initialization;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.server.DruidNode;
+import org.apache.druid.server.initialization.BaseJettyTest;
+import org.apache.druid.server.initialization.jetty.JettyServerInitializer;
+import org.easymock.EasyMock;
+import org.eclipse.jetty.server.Server;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.charset.StandardCharsets;
+
+public class BrokerClientTest extends BaseJettyTest
+{
+
+  private DiscoveryDruidNode discoveryDruidNode;
+  private HttpClient httpClient;
+
+  @Override
+  protected Injector setupInjector()
+  {
+    final DruidNode node = new DruidNode("test", "localhost", false, null, null, true, false);
+    discoveryDruidNode = new DiscoveryDruidNode(node, NodeRole.PEON, ImmutableMap.of());
+
+    Injector injector = Initialization.makeInjectorWithModules(
+        GuiceInjectors.makeStartupInjector(), ImmutableList.<Module>of(
+            binder -> {
+              JsonConfigProvider.bindInstance(
+                  binder,
+                  Key.get(DruidNode.class, Self.class),
+                  node
+              );
+              binder.bind(Integer.class).annotatedWith(Names.named("port")).toInstance(node.getPlaintextPort());
+              binder.bind(JettyServerInitializer.class).to(DruidLeaderClientTest.TestJettyServerInitializer.class).in(
+                  LazySingleton.class);
+              Jerseys.addResource(binder, DruidLeaderClientTest.SimpleResource.class);
+              LifecycleModule.register(binder, Server.class);
+            }
+        )
+    );

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Initialization.makeInjectorWithModules](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5422)



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe merged PR #14322:
URL: https://github.com/apache/druid/pull/14322


-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1289541709


##########
server/src/main/java/org/apache/druid/discovery/ClientUtils.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.discovery;
+
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.http.client.Request;
+
+import javax.annotation.Nullable;
+import java.net.URL;
+import java.util.Iterator;
+
+/**
+ * Utils class for shared client methods
+ */
+public class ClientUtils
+{
+  @Nullable
+  public static String pickOneHost(DruidNodeDiscovery druidNodeDiscovery)
+  {
+    Iterator<DiscoveryDruidNode> iter = druidNodeDiscovery.getAllNodes().iterator();
+    if (iter.hasNext()) {
+      DiscoveryDruidNode node = iter.next();
+      return StringUtils.format(
+          "%s://%s",
+          node.getDruidNode().getServiceScheme(),
+          node.getDruidNode().getHostAndPortToUse()
+      );
+    }
+    return null;
+  }
+
+  public static Request withUrl(Request old, URL url)
+  {
+    Request req = new Request(old.getMethod(), url);
+    req.addHeaderValues(old.getHeaders());
+    if (old.hasContent()) {
+      req.setContent(old.getContent());

Review Comment:
   ```suggestion
         req.setContent(old.getContent().copy());
   ```
   We should probably copy the content values instead of sharing them  



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1296197177


##########
docs/api-reference/sql-ingestion-api.md:
##########
@@ -257,7 +257,15 @@ The response shows an example report for a query.
         "startTime": "2022-09-14T22:12:09.266Z",
         "durationMs": 28227,
         "pendingTasks": 0,
-        "runningTasks": 2
+        "runningTasks": 2,
+        "segmentLoadWaiterStatus": {

Review Comment:
   nit: segmentLoadStatus?
   What is this start time ?
   How would segments which match a drop rule get communicated to the console?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.msq.exec;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.discovery.BrokerClient;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.sql.http.ResultFormat;
+import org.apache.druid.sql.http.SqlQuery;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import javax.ws.rs.core.MediaType;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table
+ * and blocks till it is complete. This will account for and not wait for segments that would never be loaded due to
+ * load rules. Should only be called if the query generates new segments or tombstones.
+ * <br>
+ * If an exception is thrown during operation, this will simply log the exception and exit without failing the task,
+ * since the segments have already been published successfully, and should be loaded eventually.
+ * <br>
+ * If the segments are not loaded within {@link #TIMEOUT_DURATION_MILLIS} milliseconds, this logs a warning and exits
+ * for the same reason.
+ */
+public class SegmentLoadWaiter
+{
+  private static final Logger log = new Logger(SegmentLoadWaiter.class);
+  private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10);
+  private static final String LOAD_QUERY = "SELECT COUNT(*) AS totalSegments,\n"
+                                           + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS loadingSegments\n"
+                                           + "FROM sys.segments\n"
+                                           + "WHERE datasource = '%s' AND is_overshadowed = 0 AND version = '%s'";
+
+  private final BrokerClient brokerClient;
+  private final ObjectMapper objectMapper;
+  // Map of version vs latest load status.
+  private final Map<String, VersionLoadStatus> versionToLoadStatusMap;
+  private final String datasource;
+  private final Set<String> versionsToAwait;
+  private final boolean doWait;
+  private volatile SegmentLoadWaiterStatus status;
+
+  public SegmentLoadWaiter(BrokerClient brokerClient, ObjectMapper objectMapper, String datasource, Set<String> versionsToAwait, int initialSegmentCount, boolean doWait)
+  {
+    this.brokerClient = brokerClient;
+    this.objectMapper = objectMapper;
+    this.datasource = datasource;
+    this.versionsToAwait = new TreeSet<>(versionsToAwait);
+    this.versionToLoadStatusMap = new HashMap<>();
+    this.status = new SegmentLoadWaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount);
+    this.doWait = doWait;
+  }
+
+  /**
+   * Uses broker client to check if all segments created by the ingestion have been loaded and updates the {@link #status)}
+   * periodically.
+   * <br>
+   * If an exception is thrown during operation, this will log the exception and return without failing the task,
+   * since the segments have already been published successfully, and should be loaded eventually.
+   * <br>
+   * Only expected to be called from the main controller thread.
+   */
+  public void waitForSegmentsToLoad()

Review Comment:
   
   We would want the experience on the console to be realtime ie like how counters currently work so that the console can render the waiting segment status progress to the end user. 
   
   With this approach the main thread looks to be blocked until the segment loading is complete and the info is only included in the task report once the call returns?
   You could add a method which check if the segment loading is complete. If not get the status of loading and write it in the task report. 
   We would want to do this until the segment loading is completed. 
   
   
    
   



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -1299,17 +1310,36 @@ private void publishAllSegments(final Set<DataSegment> segments) throws IOExcept
       if (segmentsWithTombstones.isEmpty()) {
         // Nothing to publish, only drop. We already validated that the intervalsToDrop do not have any
         // partially-overlapping segments, so it's safe to drop them as intervals instead of as specific segments.
+        // This should not need a segment load wait as segments are marked as unused immediately.
         for (final Interval interval : intervalsToDrop) {
           context.taskActionClient()
                  .submit(new MarkSegmentsAsUnusedAction(task.getDataSource(), interval));
         }
       } else {
+        Set<String> versionsToAwait = segmentsWithTombstones.stream().map(DataSegment::getVersion).collect(Collectors.toSet());
+        segmentLoadWaiter = new SegmentLoadWaiter(
+            context.injector().getInstance(BrokerClient.class),
+            context.jsonMapper(),
+            task.getDataSource(),
+            versionsToAwait,
+            segmentsWithTombstones.size(),
+            true
+        );
         performSegmentPublish(
             context.taskActionClient(),
             SegmentTransactionalInsertAction.overwriteAction(null, null, segmentsWithTombstones)
         );
       }
     } else if (!segments.isEmpty()) {
+      Set<String> versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet());

Review Comment:
   There would always be one version rite ?
   Can we add a check here ?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.msq.exec;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.discovery.BrokerClient;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.sql.http.ResultFormat;
+import org.apache.druid.sql.http.SqlQuery;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import javax.ws.rs.core.MediaType;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table
+ * and blocks till it is complete. This will account for and not wait for segments that would never be loaded due to
+ * load rules. Should only be called if the query generates new segments or tombstones.
+ * <br>
+ * If an exception is thrown during operation, this will simply log the exception and exit without failing the task,
+ * since the segments have already been published successfully, and should be loaded eventually.
+ * <br>
+ * If the segments are not loaded within {@link #TIMEOUT_DURATION_MILLIS} milliseconds, this logs a warning and exits
+ * for the same reason.
+ */
+public class SegmentLoadWaiter
+{
+  private static final Logger log = new Logger(SegmentLoadWaiter.class);
+  private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10);
+  private static final String LOAD_QUERY = "SELECT COUNT(*) AS totalSegments,\n"
+                                           + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS loadingSegments\n"

Review Comment:
   Why is the replication factor filter needed here ?



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "adarshsanjeev (via GitHub)" <gi...@apache.org>.
adarshsanjeev commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1298580602


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -1299,17 +1310,36 @@ private void publishAllSegments(final Set<DataSegment> segments) throws IOExcept
       if (segmentsWithTombstones.isEmpty()) {
         // Nothing to publish, only drop. We already validated that the intervalsToDrop do not have any
         // partially-overlapping segments, so it's safe to drop them as intervals instead of as specific segments.
+        // This should not need a segment load wait as segments are marked as unused immediately.
         for (final Interval interval : intervalsToDrop) {
           context.taskActionClient()
                  .submit(new MarkSegmentsAsUnusedAction(task.getDataSource(), interval));
         }
       } else {
+        Set<String> versionsToAwait = segmentsWithTombstones.stream().map(DataSegment::getVersion).collect(Collectors.toSet());
+        segmentLoadWaiter = new SegmentLoadWaiter(
+            context.injector().getInstance(BrokerClient.class),
+            context.jsonMapper(),
+            task.getDataSource(),
+            versionsToAwait,
+            segmentsWithTombstones.size(),
+            true
+        );
         performSegmentPublish(
             context.taskActionClient(),
             SegmentTransactionalInsertAction.overwriteAction(null, null, segmentsWithTombstones)
         );
       }
     } else if (!segments.isEmpty()) {
+      Set<String> versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet());

Review Comment:
   I thought this initially as well, but looking into the code, we actually generate the version based on the lock we acquire for the segment. So if there are multiple intervals we are replacing into and therefore multiple locks, we would have more than one version.



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1306662577


##########
docs/api-reference/sql-ingestion-api.md:
##########
@@ -288,7 +288,19 @@ The response shows an example report for a query.
         "startTime": "2022-09-14T22:12:09.266Z",
         "durationMs": 28227,
         "pendingTasks": 0,
-        "runningTasks": 2
+        "runningTasks": 2,
+        "segmentLoadStatus": {
+          "state": "SUCCESS",
+          "dataSource": "kttm_simple",
+          "startTime": "2022-09-14T23:12:09.266Z",
+          "duration": 15,
+          "totalSegments": 1,
+          "usedSegments": 1,
+          "precachedSegments": 0,
+          "asyncOnlySegments": 0,

Review Comment:
   ```suggestion
             "onDemandSegments": 0,
   ```
   Async only seems weird to me since its ties the execution mode to segment. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java:
##########
@@ -58,11 +59,13 @@
 public class SegmentLoadWaiter
 {
   private static final Logger log = new Logger(SegmentLoadWaiter.class);
-  private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
   private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
   private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10);
-  private static final String LOAD_QUERY = "SELECT COUNT(*) AS totalSegments,\n"
-                                           + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS loadingSegments\n"
+  private static final String LOAD_QUERY = "SELECT COUNT(*) AS usedSegments,\n"

Review Comment:
   Probably a writeup here of what each replication_factor means and link to : https://github.com/apache/druid/pull/14403 would be helpful. 
   
   That PR might needs an updated description though . 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -453,14 +454,25 @@ public TaskStatus runTask(final Closer closer)
       }
     }
 
-    cleanUpDurableStorageIfNeeded();
 
-    if (queryKernel != null && queryKernel.isSuccess()) {
-      if (segmentLoadWaiter != null) {
-        // If successful and there are segments created, segmentLoadWaiter should wait for them to become available.
-        segmentLoadWaiter.waitForSegmentsToLoad();
+    try {
+      final List<TaskLock> locks = context.taskActionClient().submit(new LockListAction());

Review Comment:
   I guess this can go in a separate method with Exception Handling clearly mentioning unable to release locks. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java:
##########
@@ -0,0 +1,392 @@
+/*
+ * 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.msq.exec;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.discovery.BrokerClient;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.sql.http.ResultFormat;
+import org.apache.druid.sql.http.SqlQuery;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import javax.ws.rs.core.MediaType;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table
+ * and blocks till it is complete. This will account for and not wait for segments that would never be loaded due to
+ * load rules. Should only be called if the query generates new segments or tombstones.
+ * <br>
+ * If an exception is thrown during operation, this will simply log the exception and exit without failing the task,
+ * since the segments have already been published successfully, and should be loaded eventually.
+ * <br>
+ * If the segments are not loaded within {@link #TIMEOUT_DURATION_MILLIS} milliseconds, this logs a warning and exits
+ * for the same reason.
+ */
+public class SegmentLoadWaiter
+{
+  private static final Logger log = new Logger(SegmentLoadWaiter.class);
+  private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10);
+  private static final String LOAD_QUERY = "SELECT COUNT(*) AS usedSegments,\n"
+                                           + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor > 0) AS precachedSegments,\n"
+                                           + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor = 0) AS asyncOnlySegments,\n"
+                                           + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS pendingSegments,\n"
+                                           + "COUNT(*) FILTER (WHERE replication_factor = -1) AS unknownSegments\n"
+                                           + "FROM sys.segments\n"
+                                           + "WHERE datasource = '%s' AND is_overshadowed = 0 AND version = '%s'";
+
+  private final BrokerClient brokerClient;
+  private final ObjectMapper objectMapper;
+  // Map of version vs latest load status.
+  private final Map<String, VersionLoadStatus> versionToLoadStatusMap;
+  private final String datasource;
+  private final Set<String> versionsToAwait;
+  private final int totalSegmentsGenerated;
+  private final boolean doWait;
+  private final AtomicReference<SegmentLoadWaiterStatus> status;
+
+  public SegmentLoadWaiter(
+      BrokerClient brokerClient,
+      ObjectMapper objectMapper,
+      String datasource,
+      Set<String> versionsToAwait,
+      int totalSegmentsGenerated,
+      boolean doWait
+  )
+  {
+    this.brokerClient = brokerClient;
+    this.objectMapper = objectMapper;
+    this.datasource = datasource;
+    this.versionsToAwait = new TreeSet<>(versionsToAwait);
+    this.versionToLoadStatusMap = new HashMap<>();
+    this.totalSegmentsGenerated = totalSegmentsGenerated;
+    this.status = new AtomicReference<>(new SegmentLoadWaiterStatus(State.INIT, null, 0, totalSegmentsGenerated, 0, 0, 0, 0, totalSegmentsGenerated));
+    this.doWait = doWait;
+  }
+
+  /**
+   * Uses broker client to check if all segments created by the ingestion have been loaded and updates the {@link #status)}
+   * periodically.
+   * <br>
+   * If an exception is thrown during operation, this will log the exception and return without failing the task,
+   * since the segments have already been published successfully, and should be loaded eventually.
+   * <br>
+   * Only expected to be called from the main controller thread.
+   */
+  public void waitForSegmentsToLoad()
+  {
+    DateTime startTime = DateTimes.nowUtc();
+    boolean hasAnySegmentBeenLoaded = false;
+
+    try {
+      while (!versionsToAwait.isEmpty()) {
+        // Check the timeout and exit if exceeded.
+        long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis();
+        if (runningMillis > TIMEOUT_DURATION_MILLIS) {
+          log.warn("Runtime [%s] exceeded timeout [%s] while waiting for segments to load. Exiting.", runningMillis, TIMEOUT_DURATION_MILLIS);
+          updateStatus(State.TIMED_OUT, startTime);
+          return;
+        }
+
+        Iterator<String> iterator = versionsToAwait.iterator();
+
+        // Query the broker for all pending versions
+        while (iterator.hasNext()) {
+          String version = iterator.next();
+
+          // Fetch the load status for this version from the broker
+          VersionLoadStatus loadStatus = fetchLoadStatusForVersion(version);
+          versionToLoadStatusMap.put(version, loadStatus);
+
+          hasAnySegmentBeenLoaded = hasAnySegmentBeenLoaded || loadStatus.getUsedSegments() > 0;
+
+          // If loading is done for this stage, remove it from future loops.
+          if (hasAnySegmentBeenLoaded && loadStatus.isLoadingComplete()) {
+            iterator.remove();
+          }
+        }
+
+        if (!versionsToAwait.isEmpty()) {
+          // Update the status.
+          updateStatus(State.WAITING, startTime);
+
+          // Sleep for a while before retrying.
+          waitIfNeeded(SLEEP_DURATION_MILLIS);
+        }
+      }
+    }
+    catch (Exception e) {
+      log.warn(e, "Exception occurred while waiting for segments to load. Exiting.");
+
+      // Update the status and return.
+      updateStatus(State.FAILED, startTime);
+      return;
+    }
+    // Update the status.
+    updateStatus(State.SUCCESS, startTime);
+  }
+
+  private void waitIfNeeded(long waitTimeMillis) throws Exception
+  {
+    if (doWait) {
+      Thread.sleep(waitTimeMillis);
+    }
+  }
+
+  /**
+   * Updates the {@link #status} with the latest details based on {@link #versionToLoadStatusMap}
+   */
+  private void updateStatus(State state, DateTime startTime)
+  {
+    int pendingSegmentCount = 0, usedSegmentsCount = 0, precachedSegmentCount = 0, asyncOnlySegmentCount = 0, unknownSegmentCount = 0;
+    for (Map.Entry<String, VersionLoadStatus> entry : versionToLoadStatusMap.entrySet()) {
+      usedSegmentsCount += entry.getValue().getUsedSegments();
+      precachedSegmentCount += entry.getValue().getPrecachedSegments();
+      asyncOnlySegmentCount += entry.getValue().getAsyncOnlySegments();
+      unknownSegmentCount += entry.getValue().getUnknownSegments();
+      pendingSegmentCount += entry.getValue().getPendingSegments();
+    }
+
+    long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis();
+    status.set(
+        new SegmentLoadWaiterStatus(
+            state,
+            startTime,
+            runningMillis,
+            totalSegmentsGenerated,
+            usedSegmentsCount,
+            precachedSegmentCount,
+            asyncOnlySegmentCount,
+            pendingSegmentCount,
+            unknownSegmentCount
+        )
+    );
+  }
+
+  /**
+   * Uses {@link #brokerClient} to fetch latest load status for a given version. Converts the response into a
+   * {@link VersionLoadStatus} and returns it.
+   */
+  private VersionLoadStatus fetchLoadStatusForVersion(String version) throws Exception
+  {
+    Request request = brokerClient.makeRequest(HttpMethod.POST, "/druid/v2/sql/");
+    SqlQuery sqlQuery = new SqlQuery(StringUtils.format(LOAD_QUERY, datasource, version),
+                                     ResultFormat.OBJECTLINES,
+                                     false, false, false, null, null);
+    request.setContent(MediaType.APPLICATION_JSON, objectMapper.writeValueAsBytes(sqlQuery));
+
+    String response = brokerClient.sendQuery(request);
+
+    if (response.trim().isEmpty()) {
+      // If no segments are returned for a version, all segments have been dropped by a drop rule.
+      return new VersionLoadStatus(0, 0, 0, 0, 0);
+    } else {
+      return objectMapper.readValue(response, VersionLoadStatus.class);
+    }
+  }
+
+  /**
+   * Returns the current status of the load.
+   */
+  public SegmentLoadWaiterStatus status()
+  {
+    return status.get();
+  }
+
+  public static class SegmentLoadWaiterStatus
+  {
+    private final State state;
+    private final DateTime startTime;
+    private final long duration;
+    private final int totalSegments;
+    private final int usedSegments;
+    private final int precachedSegments;
+    private final int asyncOnlySegments;
+    private final int pendingSegments;
+    private final int unknownSegments;
+
+    @JsonCreator
+    public SegmentLoadWaiterStatus(
+        @JsonProperty("state") SegmentLoadWaiter.State state,
+        @JsonProperty("startTime") @Nullable DateTime startTime,
+        @JsonProperty("duration") long duration,
+        @JsonProperty("totalSegments") int totalSegments,
+        @JsonProperty("usedSegments") int usedSegments,
+        @JsonProperty("precachedSegments") int precachedSegments,
+        @JsonProperty("asyncOnlySegments") int asyncOnlySegments,
+        @JsonProperty("pendingSegments") int pendingSegments,
+        @JsonProperty("unknownSegments") int unknownSegments
+    )
+    {
+      this.state = state;
+      this.startTime = startTime;
+      this.duration = duration;
+      this.totalSegments = totalSegments;
+      this.usedSegments = usedSegments;
+      this.precachedSegments = precachedSegments;
+      this.asyncOnlySegments = asyncOnlySegments;
+      this.pendingSegments = pendingSegments;
+      this.unknownSegments = unknownSegments;
+    }
+
+    @JsonProperty
+    public SegmentLoadWaiter.State getState()
+    {
+      return state;
+    }
+
+    @Nullable
+    @JsonProperty
+    @JsonInclude(JsonInclude.Include.NON_NULL)
+    public DateTime getStartTime()
+    {
+      return startTime;
+    }
+
+    @JsonProperty
+    public long getDuration()
+    {
+      return duration;
+    }
+
+    @JsonProperty
+    public long getTotalSegments()
+    {
+      return totalSegments;
+    }
+
+    @JsonProperty
+    public int getUsedSegments()
+    {
+      return usedSegments;
+    }
+
+    @JsonProperty
+    public int getPrecachedSegments()
+    {
+      return precachedSegments;
+    }
+
+    @JsonProperty
+    public int getAsyncOnlySegments()
+    {
+      return asyncOnlySegments;
+    }
+
+    @JsonProperty
+    public int getPendingSegments()
+    {
+      return pendingSegments;
+    }
+
+    @JsonProperty
+    public int getUnknownSegments()
+    {
+      return unknownSegments;
+    }
+  }
+
+  public enum State
+  {
+    INIT,

Review Comment:
   Could you add more dev notes to this enum. 



##########
server/src/main/java/org/apache/druid/discovery/BrokerClient.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.discovery;
+
+import com.google.inject.Inject;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.guice.annotations.EscalatedGlobal;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.RetryUtils;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHandler;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * This class facilitates interaction with Broker.
+ */
+public class BrokerClient
+{
+  private static final int MAX_RETRIES = 5;
+
+  private final HttpClient brokerHttpClient;
+  private final DruidNodeDiscovery druidNodeDiscovery;
+
+  @Inject
+  public BrokerClient(
+      @EscalatedGlobal HttpClient brokerHttpClient,
+      DruidNodeDiscoveryProvider druidNodeDiscoveryProvider
+  )
+  {
+    this.brokerHttpClient = brokerHttpClient;
+    this.druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER);
+  }
+
+  /**
+   * Creates and returns a {@link Request} after choosing a broker.
+   */
+  public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOException
+  {
+    String host = ClientUtils.pickOneHost(druidNodeDiscovery);
+
+    if (host == null) {
+      throw DruidException.forPersona(DruidException.Persona.ADMIN)
+                          .ofCategory(DruidException.Category.NOT_FOUND)
+                          .build("A leader node could not be found for [%s] service. Check the logs to validate that service is healthy.", NodeRole.BROKER);
+    }
+    return new Request(httpMethod, new URL(StringUtils.format("%s%s", host, urlPath)));
+  }
+
+  public String sendQuery(final Request request) throws Exception
+  {
+    return RetryUtils.retry(
+        () -> {
+          Request newRequestUrl = getNewRequestUrl(request);
+          final StringFullResponseHolder fullResponseHolder = brokerHttpClient.go(newRequestUrl, new StringFullResponseHandler(StandardCharsets.UTF_8)).get();
+
+          HttpResponseStatus responseStatus = fullResponseHolder.getResponse().getStatus();
+          if (HttpResponseStatus.SERVICE_UNAVAILABLE.equals(responseStatus)
+              || HttpResponseStatus.GATEWAY_TIMEOUT.equals(responseStatus)) {
+            throw new IOE(StringUtils.format("Request to broker failed due to failed response status: [%s]", responseStatus));

Review Comment:
   DruidException defensive exceptions here ?



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1289533484


##########
server/src/main/java/org/apache/druid/discovery/BrokerClient.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.discovery;
+
+import com.google.common.base.Throwables;
+import com.google.inject.Inject;
+import org.apache.druid.guice.annotations.EscalatedGlobal;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.RE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHandler;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * This class facilitates interaction with Broker.
+ */
+public class BrokerClient
+{
+  private final Logger log = new Logger(BrokerClient.class);
+  private static final int MAX_RETRIES = 5;
+
+  private final HttpClient httpClient;
+  private final DruidNodeDiscovery druidNodeDiscovery;
+
+  @Inject
+  public BrokerClient(
+      @EscalatedGlobal HttpClient httpClient,
+      DruidNodeDiscoveryProvider druidNodeDiscoveryProvider
+  )
+  {
+    this.httpClient = httpClient;
+    this.druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER);
+  }
+
+  /**
+   * Creates and returns a {@link Request} after choosing a broker.
+   */
+  public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOException
+  {
+    String host = ClientUtils.pickOneHost(druidNodeDiscovery);
+
+    if (host == null) {
+      throw new IOE("No known server.");

Review Comment:
   nit: Seems like this would get surfaced to the end user if the node is not present. Can this be made to `DruidException` with appropriate messaging? Reference PR: https://github.com/apache/druid/pull/14775



##########
server/src/main/java/org/apache/druid/discovery/BrokerClient.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.discovery;
+
+import com.google.common.base.Throwables;
+import com.google.inject.Inject;
+import org.apache.druid.guice.annotations.EscalatedGlobal;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.RE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHandler;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * This class facilitates interaction with Broker.
+ */
+public class BrokerClient
+{
+  private final Logger log = new Logger(BrokerClient.class);
+  private static final int MAX_RETRIES = 5;
+
+  private final HttpClient httpClient;
+  private final DruidNodeDiscovery druidNodeDiscovery;
+
+  @Inject
+  public BrokerClient(
+      @EscalatedGlobal HttpClient httpClient,
+      DruidNodeDiscoveryProvider druidNodeDiscoveryProvider
+  )
+  {
+    this.httpClient = httpClient;
+    this.druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER);
+  }
+
+  /**
+   * Creates and returns a {@link Request} after choosing a broker.
+   */
+  public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOException
+  {
+    String host = ClientUtils.pickOneHost(druidNodeDiscovery);
+
+    if (host == null) {
+      throw new IOE("No known server.");
+    }
+    return new Request(httpMethod, new URL(StringUtils.format("%s%s", host, urlPath)));
+  }
+
+  public String sendQuery(Request request) throws Exception
+  {
+    StringFullResponseHandler responseHandler = new StringFullResponseHandler(StandardCharsets.UTF_8);
+
+    for (int counter = 0; counter < MAX_RETRIES; counter++) {

Review Comment:
   I think we should wait for a while before retrying. This would prevent bombarding the Broker with requests in a short span of time, and also allow any transient failures to auto-resolve before sending another request. We should also have a back-off strategy here. 
   
   Consider refactoring it to `RetryUtils.retry` which does it for us. 



##########
server/src/main/java/org/apache/druid/discovery/BrokerClient.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.discovery;
+
+import com.google.common.base.Throwables;
+import com.google.inject.Inject;
+import org.apache.druid.guice.annotations.EscalatedGlobal;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.RE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHandler;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * This class facilitates interaction with Broker.
+ */
+public class BrokerClient
+{
+  private final Logger log = new Logger(BrokerClient.class);
+  private static final int MAX_RETRIES = 5;
+
+  private final HttpClient httpClient;
+  private final DruidNodeDiscovery druidNodeDiscovery;
+
+  @Inject
+  public BrokerClient(
+      @EscalatedGlobal HttpClient httpClient,
+      DruidNodeDiscoveryProvider druidNodeDiscoveryProvider
+  )
+  {
+    this.httpClient = httpClient;
+    this.druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER);
+  }
+
+  /**
+   * Creates and returns a {@link Request} after choosing a broker.
+   */
+  public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOException
+  {
+    String host = ClientUtils.pickOneHost(druidNodeDiscovery);
+
+    if (host == null) {
+      throw new IOE("No known server.");
+    }
+    return new Request(httpMethod, new URL(StringUtils.format("%s%s", host, urlPath)));
+  }
+
+  public String sendQuery(Request request) throws Exception
+  {
+    StringFullResponseHandler responseHandler = new StringFullResponseHandler(StandardCharsets.UTF_8);
+
+    for (int counter = 0; counter < MAX_RETRIES; counter++) {
+      final StringFullResponseHolder fullResponseHolder;
+
+      try {
+        try {
+          fullResponseHolder = httpClient.go(request, responseHandler).get();
+        }
+        catch (ExecutionException e) {
+          // Unwrap IOExceptions and ChannelExceptions, re-throw others
+          Throwables.propagateIfInstanceOf(e.getCause(), IOException.class);
+          Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class);
+          throw new RE(e, "HTTP request to [%s] failed", request.getUrl());
+        }
+      }
+      catch (IOException | ChannelException ex) {
+        // can happen if the node is stopped.
+        log.warn(ex, "Request [%s] failed.", request.getUrl());
+        request = getNewRequestUrl(request);
+        continue;
+      }
+
+      HttpResponseStatus responseStatus = fullResponseHolder.getResponse().getStatus();
+      if (HttpResponseStatus.SERVICE_UNAVAILABLE.equals(responseStatus)
+          || HttpResponseStatus.GATEWAY_TIMEOUT.equals(responseStatus)) {
+        log.warn(
+            "Request [%s] received a [%s] response. Attempt [%s]/[%s]",
+            request.getUrl(),
+            responseStatus,
+            counter + 1,
+            MAX_RETRIES
+        );
+        request = getNewRequestUrl(request);
+      } else if (responseStatus.getCode() != HttpServletResponse.SC_OK) {
+        log.warn("Request [%s] failed with error code [%s]", request.getUrl(), responseStatus.getCode());
+      } else {
+        return fullResponseHolder.getContent();
+      }
+    }
+
+    throw new IOE("Retries exhausted, couldn't fulfill request to [%s].", request.getUrl());
+  }
+
+  private Request getNewRequestUrl(Request oldRequest)
+  {
+    try {
+      return ClientUtils.withUrl(
+          oldRequest,
+          new URL(StringUtils.format("%s%s", ClientUtils.pickOneHost(druidNodeDiscovery), oldRequest.getUrl().getPath()))
+      );
+    }
+    catch (MalformedURLException e) {
+      // Not an IOException; this is our own fault.
+      throw new ISE(

Review Comment:
   ```suggestion
         throw new DruidException.defensive(
   ```



##########
server/src/main/java/org/apache/druid/discovery/ClientUtils.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.discovery;
+
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.http.client.Request;
+
+import javax.annotation.Nullable;
+import java.net.URL;
+import java.util.Iterator;
+
+/**
+ * Utils class for shared client methods
+ */
+public class ClientUtils
+{
+  @Nullable
+  public static String pickOneHost(DruidNodeDiscovery druidNodeDiscovery)
+  {
+    Iterator<DiscoveryDruidNode> iter = druidNodeDiscovery.getAllNodes().iterator();
+    if (iter.hasNext()) {

Review Comment:
   Seems to me that this would have the affinity of picking the first broker node all the time. It would be better if we choose this in a round-robin fashion or at random. Is there any pre-existing code that gives a server at random, seems like this would be a common use case?



##########
server/src/main/java/org/apache/druid/discovery/BrokerClient.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.discovery;
+
+import com.google.common.base.Throwables;
+import com.google.inject.Inject;
+import org.apache.druid.guice.annotations.EscalatedGlobal;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.RE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHandler;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * This class facilitates interaction with Broker.
+ */
+public class BrokerClient
+{
+  private final Logger log = new Logger(BrokerClient.class);
+  private static final int MAX_RETRIES = 5;
+
+  private final HttpClient httpClient;

Review Comment:
   nit:
   ```suggestion
     private final HttpClient brokerHttpClient;
   ```



##########
server/src/main/java/org/apache/druid/discovery/BrokerClient.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.discovery;
+
+import com.google.common.base.Throwables;
+import com.google.inject.Inject;
+import org.apache.druid.guice.annotations.EscalatedGlobal;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.RE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHandler;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * This class facilitates interaction with Broker.
+ */
+public class BrokerClient
+{
+  private final Logger log = new Logger(BrokerClient.class);
+  private static final int MAX_RETRIES = 5;
+
+  private final HttpClient httpClient;
+  private final DruidNodeDiscovery druidNodeDiscovery;
+
+  @Inject
+  public BrokerClient(
+      @EscalatedGlobal HttpClient httpClient,
+      DruidNodeDiscoveryProvider druidNodeDiscoveryProvider
+  )
+  {
+    this.httpClient = httpClient;
+    this.druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER);
+  }
+
+  /**
+   * Creates and returns a {@link Request} after choosing a broker.
+   */
+  public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOException
+  {
+    String host = ClientUtils.pickOneHost(druidNodeDiscovery);
+
+    if (host == null) {
+      throw new IOE("No known server.");
+    }
+    return new Request(httpMethod, new URL(StringUtils.format("%s%s", host, urlPath)));
+  }
+
+  public String sendQuery(Request request) throws Exception
+  {
+    StringFullResponseHandler responseHandler = new StringFullResponseHandler(StandardCharsets.UTF_8);
+
+    for (int counter = 0; counter < MAX_RETRIES; counter++) {
+      final StringFullResponseHolder fullResponseHolder;
+
+      try {
+        try {
+          fullResponseHolder = httpClient.go(request, responseHandler).get();
+        }
+        catch (ExecutionException e) {
+          // Unwrap IOExceptions and ChannelExceptions, re-throw others
+          Throwables.propagateIfInstanceOf(e.getCause(), IOException.class);
+          Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class);
+          throw new RE(e, "HTTP request to [%s] failed", request.getUrl());
+        }
+      }
+      catch (IOException | ChannelException ex) {
+        // can happen if the node is stopped.
+        log.warn(ex, "Request [%s] failed.", request.getUrl());

Review Comment:
   This would log it after each retry. Since the retries are happening in a short span, there's a high likelihood that we would be posting the same stack over and over. This should log once after all the retries are exhausted. If you refactor it to RetryUtils, I think it also handles that for you. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.msq.exec;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.druid.discovery.BrokerClient;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.sql.http.ResultFormat;
+import org.apache.druid.sql.http.SqlQuery;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import javax.ws.rs.core.MediaType;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table
+ * and blocks till it is complete. This will account for and not wait for segments that would never be loaded due to
+ * load rules. Should only be called if the query generates new segments or tombstones.
+ * <br>
+ * If an exception is thrown during operation, this will simply log the exception and exit without failing the task,
+ * since the segments have already been published successfully, and should be loaded eventually.
+ * <br>
+ * If the segments are not loaded within {@link #TIMEOUT_DURATION_MILLIS} milliseconds, this logs a warning and exits
+ * for the same reason.
+ */
+public class SegmentLoadWaiter
+{
+  private static final Logger log = new Logger(SegmentLoadWaiter.class);
+  private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10);
+  private static final String LOAD_QUERY = "SELECT COUNT(*) AS totalSegments,\n"
+                                           + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS loadingSegments\n"
+                                           + "FROM sys.segments\n"
+                                           + "WHERE datasource = '%s' AND is_overshadowed = 0 AND version = '%s'";
+
+  private final BrokerClient brokerClient;
+  private final ObjectMapper objectMapper;
+  // Map of version vs latest load status.
+  private final Map<String, VersionLoadStatus> versionToLoadStatusMap;
+  private final String datasource;
+  private final Set<String> versionsToAwait;
+  private final boolean doWait;
+  private volatile SegmentLoadWaiterStatus status;
+
+  public SegmentLoadWaiter(ControllerContext context, String datasource, Set<String> versionsToAwait, int initialSegmentCount)
+  {
+    this.brokerClient = context.injector().getInstance(BrokerClient.class);
+    this.objectMapper = context.jsonMapper();
+    this.datasource = datasource;
+    this.versionsToAwait = new TreeSet<>(versionsToAwait);
+    this.versionToLoadStatusMap = new HashMap<>();
+    this.status = new SegmentLoadWaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount);
+    this.doWait = true;
+  }
+
+  @VisibleForTesting
+  SegmentLoadWaiter(BrokerClient brokerClient, ObjectMapper objectMapper, String datasource, Set<String> versionsToAwait, int initialSegmentCount, boolean doWait)

Review Comment:
   ```suggestion
     public SegmentLoadWaiter(BrokerClient brokerClient, ObjectMapper objectMapper, String datasource, Set<String> versionsToAwait, int initialSegmentCount, boolean doWait)
   ```
   The public constructor should be removed, the constructor that's visible for testing should be made public, and the `ControllerImpl` should directly call the constructor with the extracted values instead, otherwise looks like we are duplicating a lot of work here. 



##########
server/src/main/java/org/apache/druid/discovery/ClientUtils.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.discovery;
+
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.http.client.Request;
+
+import javax.annotation.Nullable;
+import java.net.URL;
+import java.util.Iterator;
+
+/**
+ * Utils class for shared client methods
+ */
+public class ClientUtils
+{
+  @Nullable
+  public static String pickOneHost(DruidNodeDiscovery druidNodeDiscovery)
+  {
+    Iterator<DiscoveryDruidNode> iter = druidNodeDiscovery.getAllNodes().iterator();
+    if (iter.hasNext()) {
+      DiscoveryDruidNode node = iter.next();
+      return StringUtils.format(
+          "%s://%s",
+          node.getDruidNode().getServiceScheme(),
+          node.getDruidNode().getHostAndPortToUse()
+      );
+    }
+    return null;
+  }
+
+  public static Request withUrl(Request old, URL url)
+  {
+    Request req = new Request(old.getMethod(), url);
+    req.addHeaderValues(old.getHeaders());
+    if (old.hasContent()) {
+      req.setContent(old.getContent());

Review Comment:
   ```suggestion
         req.setContent(old.getContent()).copy();
   ```
   We should probably copy the content values instead of sharing them  



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1276007485


##########
server/src/main/java/org/apache/druid/discovery/BrokerClient.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.discovery;
+
+import com.google.common.base.Throwables;
+import com.google.inject.Inject;
+import org.apache.druid.guice.annotations.EscalatedGlobal;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.RE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHandler;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * This class facilitates interaction with Broker.
+ */
+public class BrokerClient
+{
+  private final Logger log = new Logger(BrokerClient.class);
+  private static final int MAX_RETRIES = 5;
+
+  private final HttpClient httpClient;
+  private final DruidNodeDiscovery druidNodeDiscovery;
+
+  @Inject
+  public BrokerClient(
+      @EscalatedGlobal HttpClient httpClient,
+      DruidNodeDiscoveryProvider druidNodeDiscoveryProvider
+  )
+  {
+    this.httpClient = httpClient;
+    this.druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER);
+  }
+
+  /**
+   * Creates and returns a {@link Request} after choosing a broker.
+   */
+  public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOException
+  {
+    String host = ClientUtils.pickOneHost(druidNodeDiscovery);
+
+    if (host == null) {
+      throw new IOE("No known server");
+    }
+    return new Request(httpMethod, new URL(StringUtils.format("%s%s", host, urlPath)));
+  }
+
+  public String sendQuery(Request request) throws Exception
+  {
+    StringFullResponseHandler responseHandler = new StringFullResponseHandler(StandardCharsets.UTF_8);
+
+    for (int counter = 0; counter < MAX_RETRIES; counter++) {
+
+      final StringFullResponseHolder fullResponseHolder;
+
+      try {
+        try {
+          fullResponseHolder = httpClient.go(request, responseHandler).get();
+        }
+        catch (ExecutionException e) {
+          // Unwrap IOExceptions and ChannelExceptions, re-throw others
+          Throwables.propagateIfInstanceOf(e.getCause(), IOException.class);
+          Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class);
+          throw new RE(e, "HTTP request to[%s] failed", request.getUrl());
+        }
+      }
+      catch (IOException | ChannelException ex) {
+        // can happen if the node is stopped.
+        log.warn(ex, "Request[%s] failed.", request.getUrl());
+        request = getNewRequestUrl(request);
+        continue;
+      }
+      HttpResponseStatus responseStatus = fullResponseHolder.getResponse().getStatus();
+      if (HttpResponseStatus.SERVICE_UNAVAILABLE.equals(responseStatus)
+          || HttpResponseStatus.GATEWAY_TIMEOUT.equals(responseStatus)) {
+        log.warn(
+            "Request[%s] received a %s response. Attempt %s/%s",
+            request.getUrl(),
+            responseStatus,
+            counter + 1,
+            MAX_RETRIES
+        );
+        request = getNewRequestUrl(request);
+      } else if (responseStatus.getCode() != HttpServletResponse.SC_OK) {
+        log.warn("Request[%s] failed with error", request.getUrl(), responseStatus.getCode());

Review Comment:
   ## Unused format argument
   
   This format call refers to 1 argument(s) but supplies 2 argument(s).
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5297)



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "adarshsanjeev (via GitHub)" <gi...@apache.org>.
adarshsanjeev commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1298287267


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.msq.exec;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.discovery.BrokerClient;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.sql.http.ResultFormat;
+import org.apache.druid.sql.http.SqlQuery;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import javax.ws.rs.core.MediaType;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table
+ * and blocks till it is complete. This will account for and not wait for segments that would never be loaded due to
+ * load rules. Should only be called if the query generates new segments or tombstones.
+ * <br>
+ * If an exception is thrown during operation, this will simply log the exception and exit without failing the task,
+ * since the segments have already been published successfully, and should be loaded eventually.
+ * <br>
+ * If the segments are not loaded within {@link #TIMEOUT_DURATION_MILLIS} milliseconds, this logs a warning and exits
+ * for the same reason.
+ */
+public class SegmentLoadWaiter
+{
+  private static final Logger log = new Logger(SegmentLoadWaiter.class);
+  private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10);
+  private static final String LOAD_QUERY = "SELECT COUNT(*) AS totalSegments,\n"
+                                           + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS loadingSegments\n"

Review Comment:
   Replication factor is used to filter out cold segments. Even if a cold segment is unavailable, we don't wait for it.



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "adarshsanjeev (via GitHub)" <gi...@apache.org>.
adarshsanjeev commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1316728569


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java:
##########
@@ -0,0 +1,392 @@
+/*
+ * 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.msq.exec;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.discovery.BrokerClient;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.sql.http.ResultFormat;
+import org.apache.druid.sql.http.SqlQuery;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import javax.ws.rs.core.MediaType;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table
+ * and blocks till it is complete. This will account for and not wait for segments that would never be loaded due to
+ * load rules. Should only be called if the query generates new segments or tombstones.
+ * <br>
+ * If an exception is thrown during operation, this will simply log the exception and exit without failing the task,
+ * since the segments have already been published successfully, and should be loaded eventually.
+ * <br>
+ * If the segments are not loaded within {@link #TIMEOUT_DURATION_MILLIS} milliseconds, this logs a warning and exits
+ * for the same reason.
+ */
+public class SegmentLoadWaiter
+{
+  private static final Logger log = new Logger(SegmentLoadWaiter.class);
+  private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10);
+  private static final String LOAD_QUERY = "SELECT COUNT(*) AS usedSegments,\n"
+                                           + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor > 0) AS precachedSegments,\n"
+                                           + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor = 0) AS asyncOnlySegments,\n"
+                                           + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS pendingSegments,\n"
+                                           + "COUNT(*) FILTER (WHERE replication_factor = -1) AS unknownSegments\n"
+                                           + "FROM sys.segments\n"
+                                           + "WHERE datasource = '%s' AND is_overshadowed = 0 AND version = '%s'";
+
+  private final BrokerClient brokerClient;
+  private final ObjectMapper objectMapper;
+  // Map of version vs latest load status.
+  private final Map<String, VersionLoadStatus> versionToLoadStatusMap;
+  private final String datasource;
+  private final Set<String> versionsToAwait;
+  private final int totalSegmentsGenerated;
+  private final boolean doWait;
+  private final AtomicReference<SegmentLoadWaiterStatus> status;
+
+  public SegmentLoadWaiter(
+      BrokerClient brokerClient,
+      ObjectMapper objectMapper,
+      String datasource,
+      Set<String> versionsToAwait,
+      int totalSegmentsGenerated,
+      boolean doWait
+  )
+  {
+    this.brokerClient = brokerClient;
+    this.objectMapper = objectMapper;
+    this.datasource = datasource;
+    this.versionsToAwait = new TreeSet<>(versionsToAwait);
+    this.versionToLoadStatusMap = new HashMap<>();
+    this.totalSegmentsGenerated = totalSegmentsGenerated;
+    this.status = new AtomicReference<>(new SegmentLoadWaiterStatus(State.INIT, null, 0, totalSegmentsGenerated, 0, 0, 0, 0, totalSegmentsGenerated));
+    this.doWait = doWait;
+  }
+
+  /**
+   * Uses broker client to check if all segments created by the ingestion have been loaded and updates the {@link #status)}
+   * periodically.
+   * <br>
+   * If an exception is thrown during operation, this will log the exception and return without failing the task,
+   * since the segments have already been published successfully, and should be loaded eventually.
+   * <br>
+   * Only expected to be called from the main controller thread.
+   */
+  public void waitForSegmentsToLoad()
+  {
+    DateTime startTime = DateTimes.nowUtc();
+    boolean hasAnySegmentBeenLoaded = false;
+
+    try {
+      while (!versionsToAwait.isEmpty()) {
+        // Check the timeout and exit if exceeded.
+        long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis();
+        if (runningMillis > TIMEOUT_DURATION_MILLIS) {
+          log.warn("Runtime [%s] exceeded timeout [%s] while waiting for segments to load. Exiting.", runningMillis, TIMEOUT_DURATION_MILLIS);
+          updateStatus(State.TIMED_OUT, startTime);
+          return;
+        }
+
+        Iterator<String> iterator = versionsToAwait.iterator();
+
+        // Query the broker for all pending versions
+        while (iterator.hasNext()) {
+          String version = iterator.next();
+
+          // Fetch the load status for this version from the broker
+          VersionLoadStatus loadStatus = fetchLoadStatusForVersion(version);
+          versionToLoadStatusMap.put(version, loadStatus);
+
+          hasAnySegmentBeenLoaded = hasAnySegmentBeenLoaded || loadStatus.getUsedSegments() > 0;
+
+          // If loading is done for this stage, remove it from future loops.
+          if (hasAnySegmentBeenLoaded && loadStatus.isLoadingComplete()) {
+            iterator.remove();
+          }
+        }
+
+        if (!versionsToAwait.isEmpty()) {
+          // Update the status.
+          updateStatus(State.WAITING, startTime);
+
+          // Sleep for a while before retrying.
+          waitIfNeeded(SLEEP_DURATION_MILLIS);
+        }
+      }
+    }
+    catch (Exception e) {
+      log.warn(e, "Exception occurred while waiting for segments to load. Exiting.");
+
+      // Update the status and return.
+      updateStatus(State.FAILED, startTime);
+      return;
+    }
+    // Update the status.
+    updateStatus(State.SUCCESS, startTime);
+  }
+
+  private void waitIfNeeded(long waitTimeMillis) throws Exception
+  {
+    if (doWait) {
+      Thread.sleep(waitTimeMillis);
+    }
+  }
+
+  /**
+   * Updates the {@link #status} with the latest details based on {@link #versionToLoadStatusMap}
+   */
+  private void updateStatus(State state, DateTime startTime)
+  {
+    int pendingSegmentCount = 0, usedSegmentsCount = 0, precachedSegmentCount = 0, asyncOnlySegmentCount = 0, unknownSegmentCount = 0;
+    for (Map.Entry<String, VersionLoadStatus> entry : versionToLoadStatusMap.entrySet()) {
+      usedSegmentsCount += entry.getValue().getUsedSegments();
+      precachedSegmentCount += entry.getValue().getPrecachedSegments();
+      asyncOnlySegmentCount += entry.getValue().getAsyncOnlySegments();
+      unknownSegmentCount += entry.getValue().getUnknownSegments();
+      pendingSegmentCount += entry.getValue().getPendingSegments();
+    }
+
+    long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis();
+    status.set(
+        new SegmentLoadWaiterStatus(
+            state,
+            startTime,
+            runningMillis,
+            totalSegmentsGenerated,
+            usedSegmentsCount,
+            precachedSegmentCount,
+            asyncOnlySegmentCount,
+            pendingSegmentCount,
+            unknownSegmentCount
+        )
+    );
+  }
+
+  /**
+   * Uses {@link #brokerClient} to fetch latest load status for a given version. Converts the response into a
+   * {@link VersionLoadStatus} and returns it.
+   */
+  private VersionLoadStatus fetchLoadStatusForVersion(String version) throws Exception
+  {
+    Request request = brokerClient.makeRequest(HttpMethod.POST, "/druid/v2/sql/");
+    SqlQuery sqlQuery = new SqlQuery(StringUtils.format(LOAD_QUERY, datasource, version),
+                                     ResultFormat.OBJECTLINES,
+                                     false, false, false, null, null);
+    request.setContent(MediaType.APPLICATION_JSON, objectMapper.writeValueAsBytes(sqlQuery));
+
+    String response = brokerClient.sendQuery(request);
+
+    if (response.trim().isEmpty()) {
+      // If no segments are returned for a version, all segments have been dropped by a drop rule.
+      return new VersionLoadStatus(0, 0, 0, 0, 0);
+    } else {
+      return objectMapper.readValue(response, VersionLoadStatus.class);
+    }
+  }
+
+  /**
+   * Returns the current status of the load.
+   */
+  public SegmentLoadWaiterStatus status()
+  {
+    return status.get();
+  }
+
+  public static class SegmentLoadWaiterStatus
+  {
+    private final State state;
+    private final DateTime startTime;
+    private final long duration;
+    private final int totalSegments;
+    private final int usedSegments;
+    private final int precachedSegments;
+    private final int asyncOnlySegments;
+    private final int pendingSegments;
+    private final int unknownSegments;
+
+    @JsonCreator
+    public SegmentLoadWaiterStatus(
+        @JsonProperty("state") SegmentLoadWaiter.State state,
+        @JsonProperty("startTime") @Nullable DateTime startTime,
+        @JsonProperty("duration") long duration,
+        @JsonProperty("totalSegments") int totalSegments,
+        @JsonProperty("usedSegments") int usedSegments,
+        @JsonProperty("precachedSegments") int precachedSegments,
+        @JsonProperty("asyncOnlySegments") int asyncOnlySegments,
+        @JsonProperty("pendingSegments") int pendingSegments,
+        @JsonProperty("unknownSegments") int unknownSegments
+    )
+    {
+      this.state = state;
+      this.startTime = startTime;
+      this.duration = duration;
+      this.totalSegments = totalSegments;
+      this.usedSegments = usedSegments;
+      this.precachedSegments = precachedSegments;
+      this.asyncOnlySegments = asyncOnlySegments;
+      this.pendingSegments = pendingSegments;
+      this.unknownSegments = unknownSegments;
+    }
+
+    @JsonProperty
+    public SegmentLoadWaiter.State getState()
+    {
+      return state;
+    }
+
+    @Nullable
+    @JsonProperty
+    @JsonInclude(JsonInclude.Include.NON_NULL)
+    public DateTime getStartTime()
+    {
+      return startTime;
+    }
+
+    @JsonProperty
+    public long getDuration()
+    {
+      return duration;
+    }
+
+    @JsonProperty
+    public long getTotalSegments()
+    {
+      return totalSegments;
+    }
+
+    @JsonProperty
+    public int getUsedSegments()
+    {
+      return usedSegments;
+    }
+
+    @JsonProperty
+    public int getPrecachedSegments()
+    {
+      return precachedSegments;
+    }
+
+    @JsonProperty
+    public int getAsyncOnlySegments()
+    {
+      return asyncOnlySegments;
+    }
+
+    @JsonProperty
+    public int getPendingSegments()
+    {
+      return pendingSegments;
+    }
+
+    @JsonProperty
+    public int getUnknownSegments()
+    {
+      return unknownSegments;
+    }
+  }
+
+  public enum State
+  {
+    INIT,

Review Comment:
   Added



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java:
##########
@@ -58,11 +59,13 @@
 public class SegmentLoadWaiter
 {
   private static final Logger log = new Logger(SegmentLoadWaiter.class);
-  private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
   private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
   private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10);
-  private static final String LOAD_QUERY = "SELECT COUNT(*) AS totalSegments,\n"
-                                           + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS loadingSegments\n"
+  private static final String LOAD_QUERY = "SELECT COUNT(*) AS usedSegments,\n"

Review Comment:
   Added



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1316729087


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java:
##########
@@ -61,9 +61,22 @@ public class SegmentLoadWaiter
   private static final Logger log = new Logger(SegmentLoadWaiter.class);
   private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
   private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10);
+
+  /**
+   * The query sent to the broker. This query uses replication_factor to determine how many copies of a segment has to be
+   * loaded as per the load rules.
+   * - If a segment is not used, the broker will not have any information about it, hence, a COUNT(*) should return this count.

Review Comment:
   ```suggestion
      * - If a segment is not used, the broker will not have any information about it, hence, a COUNT(*) should return the used count only.
   ```



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "adarshsanjeev (via GitHub)" <gi...@apache.org>.
adarshsanjeev commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1298288972


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.msq.exec;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.discovery.BrokerClient;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.sql.http.ResultFormat;
+import org.apache.druid.sql.http.SqlQuery;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import javax.ws.rs.core.MediaType;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table
+ * and blocks till it is complete. This will account for and not wait for segments that would never be loaded due to
+ * load rules. Should only be called if the query generates new segments or tombstones.
+ * <br>
+ * If an exception is thrown during operation, this will simply log the exception and exit without failing the task,
+ * since the segments have already been published successfully, and should be loaded eventually.
+ * <br>
+ * If the segments are not loaded within {@link #TIMEOUT_DURATION_MILLIS} milliseconds, this logs a warning and exits
+ * for the same reason.
+ */
+public class SegmentLoadWaiter
+{
+  private static final Logger log = new Logger(SegmentLoadWaiter.class);
+  private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5);
+  private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10);
+  private static final String LOAD_QUERY = "SELECT COUNT(*) AS totalSegments,\n"
+                                           + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS loadingSegments\n"
+                                           + "FROM sys.segments\n"
+                                           + "WHERE datasource = '%s' AND is_overshadowed = 0 AND version = '%s'";
+
+  private final BrokerClient brokerClient;
+  private final ObjectMapper objectMapper;
+  // Map of version vs latest load status.
+  private final Map<String, VersionLoadStatus> versionToLoadStatusMap;
+  private final String datasource;
+  private final Set<String> versionsToAwait;
+  private final boolean doWait;
+  private volatile SegmentLoadWaiterStatus status;
+
+  public SegmentLoadWaiter(BrokerClient brokerClient, ObjectMapper objectMapper, String datasource, Set<String> versionsToAwait, int initialSegmentCount, boolean doWait)
+  {
+    this.brokerClient = brokerClient;
+    this.objectMapper = objectMapper;
+    this.datasource = datasource;
+    this.versionsToAwait = new TreeSet<>(versionsToAwait);
+    this.versionToLoadStatusMap = new HashMap<>();
+    this.status = new SegmentLoadWaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount);
+    this.doWait = doWait;
+  }
+
+  /**
+   * Uses broker client to check if all segments created by the ingestion have been loaded and updates the {@link #status)}
+   * periodically.
+   * <br>
+   * If an exception is thrown during operation, this will log the exception and return without failing the task,
+   * since the segments have already been published successfully, and should be loaded eventually.
+   * <br>
+   * Only expected to be called from the main controller thread.
+   */
+  public void waitForSegmentsToLoad()

Review Comment:
   Discussed offline, the controller thread updates the status periodically without blocking. This status is included from the liveReports() in the controller. I have tested this out as well and the realtime information is available to the console



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "adarshsanjeev (via GitHub)" <gi...@apache.org>.
adarshsanjeev commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1298582512


##########
docs/api-reference/sql-ingestion-api.md:
##########
@@ -257,7 +257,15 @@ The response shows an example report for a query.
         "startTime": "2022-09-14T22:12:09.266Z",
         "durationMs": 28227,
         "pendingTasks": 0,
-        "runningTasks": 2
+        "runningTasks": 2,
+        "segmentLoadWaiterStatus": {

Review Comment:
   startTime is the time at which we started checking the load status. That with duration would give us a clear idea of when it started and when it ended, and this is the structure for other MSQ stages.



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1299351989


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -1299,17 +1310,36 @@ private void publishAllSegments(final Set<DataSegment> segments) throws IOExcept
       if (segmentsWithTombstones.isEmpty()) {
         // Nothing to publish, only drop. We already validated that the intervalsToDrop do not have any
         // partially-overlapping segments, so it's safe to drop them as intervals instead of as specific segments.
+        // This should not need a segment load wait as segments are marked as unused immediately.
         for (final Interval interval : intervalsToDrop) {
           context.taskActionClient()
                  .submit(new MarkSegmentsAsUnusedAction(task.getDataSource(), interval));
         }
       } else {
+        Set<String> versionsToAwait = segmentsWithTombstones.stream().map(DataSegment::getVersion).collect(Collectors.toSet());
+        segmentLoadWaiter = new SegmentLoadWaiter(
+            context.injector().getInstance(BrokerClient.class),
+            context.jsonMapper(),
+            task.getDataSource(),
+            versionsToAwait,
+            segmentsWithTombstones.size(),
+            true
+        );
         performSegmentPublish(
             context.taskActionClient(),
             SegmentTransactionalInsertAction.overwriteAction(null, null, segmentsWithTombstones)
         );
       }
     } else if (!segments.isEmpty()) {
+      Set<String> versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet());

Review Comment:
   Oh i did not know that. Thanks for explaining the rational. We could also document this as well :)



-- 
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


Re: [PR] Add code to wait for segments generated to be loaded on historicals (druid)

Posted by "adarshsanjeev (via GitHub)" <gi...@apache.org>.
adarshsanjeev commented on code in PR #14322:
URL: https://github.com/apache/druid/pull/14322#discussion_r1315343045


##########
server/src/main/java/org/apache/druid/discovery/BrokerClient.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.discovery;
+
+import com.google.inject.Inject;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.guice.annotations.EscalatedGlobal;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.RetryUtils;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHandler;
+import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * This class facilitates interaction with Broker.
+ */
+public class BrokerClient
+{
+  private static final int MAX_RETRIES = 5;
+
+  private final HttpClient brokerHttpClient;
+  private final DruidNodeDiscovery druidNodeDiscovery;
+
+  @Inject
+  public BrokerClient(
+      @EscalatedGlobal HttpClient brokerHttpClient,
+      DruidNodeDiscoveryProvider druidNodeDiscoveryProvider
+  )
+  {
+    this.brokerHttpClient = brokerHttpClient;
+    this.druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER);
+  }
+
+  /**
+   * Creates and returns a {@link Request} after choosing a broker.
+   */
+  public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOException
+  {
+    String host = ClientUtils.pickOneHost(druidNodeDiscovery);
+
+    if (host == null) {
+      throw DruidException.forPersona(DruidException.Persona.ADMIN)
+                          .ofCategory(DruidException.Category.NOT_FOUND)
+                          .build("A leader node could not be found for [%s] service. Check the logs to validate that service is healthy.", NodeRole.BROKER);
+    }
+    return new Request(httpMethod, new URL(StringUtils.format("%s%s", host, urlPath)));
+  }
+
+  public String sendQuery(final Request request) throws Exception
+  {
+    return RetryUtils.retry(
+        () -> {
+          Request newRequestUrl = getNewRequestUrl(request);
+          final StringFullResponseHolder fullResponseHolder = brokerHttpClient.go(newRequestUrl, new StringFullResponseHandler(StandardCharsets.UTF_8)).get();
+
+          HttpResponseStatus responseStatus = fullResponseHolder.getResponse().getStatus();
+          if (HttpResponseStatus.SERVICE_UNAVAILABLE.equals(responseStatus)
+              || HttpResponseStatus.GATEWAY_TIMEOUT.equals(responseStatus)) {
+            throw new IOE(StringUtils.format("Request to broker failed due to failed response status: [%s]", responseStatus));

Review Comment:
   Should these be defensive? These would be thrown if the broker is not reachable which is a condition that can happen without any bugs in the code, right?



-- 
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