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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to