Fix issue with checking segment load status (#15147)

This PR addresses a bug with waiting for segments to be loaded. In the case of append, segments would be created with the same version. This caused the number of segments returned to be incorrect.

This PR changes this to keep track of the range of partition numbers as well for each version, which lets the task wait for the correct set of segments. The partition numbers are expected to be continuous since the task obtains the lock for the segment while running.
This commit is contained in:
Adarsh Sanjeev 2023-10-13 16:06:13 +05:30 committed by GitHub
parent d25caaefa4
commit 4deeb7e936
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 71 additions and 28 deletions

View File

@ -1429,15 +1429,13 @@ public class ControllerImpl implements Controller
.submit(new MarkSegmentsAsUnusedAction(task.getDataSource(), interval));
}
} else {
Set<String> versionsToAwait = segmentsWithTombstones.stream().map(DataSegment::getVersion).collect(Collectors.toSet());
if (MultiStageQueryContext.shouldWaitForSegmentLoad(task.getQuerySpec().getQuery().context())) {
segmentLoadWaiter = new SegmentLoadStatusFetcher(
context.injector().getInstance(BrokerClient.class),
context.jsonMapper(),
task.getId(),
task.getDataSource(),
versionsToAwait,
segmentsWithTombstones.size(),
segmentsWithTombstones,
true
);
}
@ -1447,15 +1445,13 @@ public class ControllerImpl implements Controller
);
}
} else if (!segments.isEmpty()) {
Set<String> versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet());
if (MultiStageQueryContext.shouldWaitForSegmentLoad(task.getQuerySpec().getQuery().context())) {
segmentLoadWaiter = new SegmentLoadStatusFetcher(
context.injector().getInstance(BrokerClient.class),
context.jsonMapper(),
task.getId(),
task.getDataSource(),
versionsToAwait,
segments.size(),
segments,
true
);
}

View File

@ -29,22 +29,27 @@ import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.discovery.BrokerClient;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
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.apache.druid.timeline.DataSegment;
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.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
/**
* Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table
@ -73,7 +78,7 @@ public class SegmentLoadStatusFetcher implements AutoCloseable
* - If replication_factor is -1, the replication factor is not known currently and will become known after a load rule
* evaluation.
* <br>
* See https://github.com/apache/druid/pull/14403 for more details about replication_factor
* See <a href="https://github.com/apache/druid/pull/14403">this</a> for more details about replication_factor
*/
private static final String LOAD_QUERY = "SELECT COUNT(*) AS usedSegments,\n"
+ "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor > 0) AS precachedSegments,\n"
@ -81,14 +86,14 @@ public class SegmentLoadStatusFetcher implements AutoCloseable
+ "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 in (%s)";
+ "WHERE datasource = '%s' AND is_overshadowed = 0 AND (%s)";
private final BrokerClient brokerClient;
private final ObjectMapper objectMapper;
// Map of version vs latest load status.
private final AtomicReference<VersionLoadStatus> versionLoadStatusReference;
private final String datasource;
private final String versionsInClauseString;
private final String versionsConditionString;
private final int totalSegmentsGenerated;
private final boolean doWait;
// since live reports fetch the value in another thread, we need to use AtomicReference
@ -101,20 +106,16 @@ public class SegmentLoadStatusFetcher implements AutoCloseable
ObjectMapper objectMapper,
String taskId,
String datasource,
Set<String> versionsToAwait,
int totalSegmentsGenerated,
Set<DataSegment> dataSegments,
boolean doWait
)
{
this.brokerClient = brokerClient;
this.objectMapper = objectMapper;
this.datasource = datasource;
this.versionsInClauseString = String.join(
",",
versionsToAwait.stream().map(s -> StringUtils.format("'%s'", s)).collect(Collectors.toSet())
);
this.versionsConditionString = createVersionCondition(dataSegments);
this.totalSegmentsGenerated = dataSegments.size();
this.versionLoadStatusReference = new AtomicReference<>(new VersionLoadStatus(0, 0, 0, 0, totalSegmentsGenerated));
this.totalSegmentsGenerated = totalSegmentsGenerated;
this.status = new AtomicReference<>(new SegmentLoadWaiterStatus(
State.INIT,
null,
@ -163,9 +164,8 @@ public class SegmentLoadStatusFetcher implements AutoCloseable
if (runningMillis - lastLogMillis >= TimeUnit.MINUTES.toMillis(1)) {
lastLogMillis = runningMillis;
log.info(
"Fetching segment load status for datasource[%s] from broker for segment versions[%s]",
datasource,
versionsInClauseString
"Fetching segment load status for datasource[%s] from broker",
datasource
);
}
@ -237,7 +237,7 @@ public class SegmentLoadStatusFetcher implements AutoCloseable
private VersionLoadStatus fetchLoadStatusFromBroker() throws Exception
{
Request request = brokerClient.makeRequest(HttpMethod.POST, "/druid/v2/sql/");
SqlQuery sqlQuery = new SqlQuery(StringUtils.format(LOAD_QUERY, datasource, versionsInClauseString),
SqlQuery sqlQuery = new SqlQuery(StringUtils.format(LOAD_QUERY, datasource, versionsConditionString),
ResultFormat.OBJECTLINES,
false, false, false, null, null
);
@ -255,6 +255,37 @@ public class SegmentLoadStatusFetcher implements AutoCloseable
}
}
/**
* Takes a list of segments and creates the condition for the broker query. Directly creates a string to avoid
* computing it repeatedly.
*/
private static String createVersionCondition(Set<DataSegment> dataSegments)
{
// Creates a map of version to earliest and latest partition numbers created. These would be contiguous since the task
// holds the lock.
Map<String, Pair<Integer, Integer>> versionsVsPartitionNumberRangeMap = new HashMap<>();
dataSegments.forEach(segment -> {
final String version = segment.getVersion();
final int partitionNum = segment.getId().getPartitionNum();
versionsVsPartitionNumberRangeMap.computeIfPresent(version, (k, v) -> Pair.of(
partitionNum < v.lhs ? partitionNum : v.lhs,
partitionNum > v.rhs ? partitionNum : v.rhs
));
versionsVsPartitionNumberRangeMap.computeIfAbsent(version, k -> Pair.of(partitionNum, partitionNum));
});
// Create a condition for each version / partition
List<String> versionConditionList = new ArrayList<>();
for (Map.Entry<String, Pair<Integer, Integer>> stringPairEntry : versionsVsPartitionNumberRangeMap.entrySet()) {
Pair<Integer, Integer> pair = stringPairEntry.getValue();
versionConditionList.add(
StringUtils.format("(version = '%s' AND partition_num BETWEEN %s AND %s)", stringPairEntry.getKey(), pair.lhs, pair.rhs)
);
}
return String.join(" OR ", versionConditionList);
}
/**
* Returns the current status of the load.
*/

View File

@ -20,14 +20,19 @@
package org.apache.druid.msq.exec;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.discovery.BrokerClient;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.http.client.Request;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.doAnswer;
@ -76,8 +81,7 @@ public class SegmentLoadStatusFetcherTest
new ObjectMapper(),
"id",
TEST_DATASOURCE,
ImmutableSet.of("version1"),
5,
IntStream.range(0, 5).boxed().map(partitionNum -> createTestDataSegment("version1", partitionNum)).collect(Collectors.toSet()),
false
);
segmentLoadWaiter.waitForSegmentsToLoad();
@ -114,8 +118,7 @@ public class SegmentLoadStatusFetcherTest
new ObjectMapper(),
"id",
TEST_DATASOURCE,
ImmutableSet.of("version1"),
5,
IntStream.range(0, 5).boxed().map(partitionNum -> createTestDataSegment("version1", partitionNum)).collect(Collectors.toSet()),
false
);
segmentLoadWaiter.waitForSegmentsToLoad();
@ -153,8 +156,7 @@ public class SegmentLoadStatusFetcherTest
new ObjectMapper(),
"id",
TEST_DATASOURCE,
ImmutableSet.of("version1"),
5,
IntStream.range(0, 5).boxed().map(partitionNum -> createTestDataSegment("version1", partitionNum)).collect(Collectors.toSet()),
true
);
@ -169,4 +171,18 @@ public class SegmentLoadStatusFetcherTest
Assert.assertTrue(segmentLoadWaiter.status().getState() == SegmentLoadStatusFetcher.State.FAILED);
}
private static DataSegment createTestDataSegment(String version, int partitionNumber)
{
return new DataSegment(
TEST_DATASOURCE,
Intervals.ETERNITY,
version,
null,
null,
null,
new NumberedShardSpec(partitionNumber, 1),
0,
0
);
}
}