Skip to content

Commit

Permalink
Fix replica task failures with metadata inconsistency while running c…
Browse files Browse the repository at this point in the history
…oncurrent append replace (#16614)

Changes:
- Add new task action `RetrieveSegmentsByIdAction`
- Use new task action to retrieve segments irrespective of their visibility
- During rolling upgrades, this task action would fail as Overlord would be on old version
- If new action fails, fall back to just fetching used segments as before
  • Loading branch information
kfaraz committed Jun 24, 2024
1 parent 1a883ba commit 0fe6a2a
Show file tree
Hide file tree
Showing 12 changed files with 300 additions and 144 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,25 +20,30 @@
package org.apache.druid.indexing.appenderator;

import com.google.common.collect.Iterables;
import org.apache.druid.error.InvalidInput;
import org.apache.druid.indexing.common.actions.RetrieveSegmentsByIdAction;
import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;

import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.stream.Collectors;

public class ActionBasedUsedSegmentChecker implements UsedSegmentChecker
{
private static final Logger log = new Logger(ActionBasedUsedSegmentChecker.class);

private final TaskActionClient taskActionClient;

public ActionBasedUsedSegmentChecker(TaskActionClient taskActionClient)
Expand All @@ -47,33 +52,54 @@ public ActionBasedUsedSegmentChecker(TaskActionClient taskActionClient)
}

@Override
public Set<DataSegment> findUsedSegments(Set<SegmentIdWithShardSpec> segmentIds) throws IOException
public Set<DataSegment> findPublishedSegments(Set<SegmentId> segmentIds) throws IOException
{
// Group by dataSource
final Map<String, Set<SegmentId>> idsByDataSource = new TreeMap<>();
for (SegmentIdWithShardSpec segmentId : segmentIds) {
idsByDataSource.computeIfAbsent(segmentId.getDataSource(), i -> new HashSet<>()).add(segmentId.asSegmentId());
if (segmentIds == null || segmentIds.isEmpty()) {
return Collections.emptySet();
}

final Set<DataSegment> usedSegments = new HashSet<>();
// Validate that all segments belong to the same datasource
final String dataSource = segmentIds.iterator().next().getDataSource();
for (SegmentId segmentId : segmentIds) {
if (!segmentId.getDataSource().equals(dataSource)) {
throw InvalidInput.exception(
"Published segment IDs to find cannot belong to multiple datasources[%s, %s].",
dataSource, segmentId.getDataSource()
);
}
}

for (Map.Entry<String, Set<SegmentId>> entry : idsByDataSource.entrySet()) {
String dataSource = entry.getKey();
Set<SegmentId> segmentIdsInDataSource = entry.getValue();
final List<Interval> intervals = JodaUtils.condenseIntervals(
Iterables.transform(segmentIdsInDataSource, SegmentId::getInterval)
// Try to retrieve segments using new task action
final Set<String> serializedSegmentIds = segmentIds.stream()
.map(SegmentId::toString)
.collect(Collectors.toSet());
try {
return taskActionClient.submit(new RetrieveSegmentsByIdAction(dataSource, serializedSegmentIds));
}
catch (Exception e) {
log.warn(
e,
"Could not retrieve published segment IDs[%s] using task action[segmentListById]."
+ " Overlord maybe on an older version, retrying with action[segmentListUsed]."
+ " This task may fail to publish segments if there is a concurrent replace happening.",
serializedSegmentIds
);
}

final Collection<DataSegment> usedSegmentsForIntervals = taskActionClient
.submit(new RetrieveUsedSegmentsAction(dataSource, intervals));

for (DataSegment segment : usedSegmentsForIntervals) {
if (segmentIdsInDataSource.contains(segment.getId())) {
usedSegments.add(segment);
}
// Fall back to using old task action if Overlord is still on an older version
final Set<DataSegment> publishedSegments = new HashSet<>();
final List<Interval> usedSearchIntervals = JodaUtils.condenseIntervals(
Iterables.transform(segmentIds, SegmentId::getInterval)
);
final Collection<DataSegment> foundUsedSegments = taskActionClient.submit(
new RetrieveUsedSegmentsAction(dataSource, null, usedSearchIntervals, Segments.INCLUDING_OVERSHADOWED)
);
for (DataSegment segment : foundUsedSegments) {
if (segmentIds.contains(segment.getId())) {
publishedSegments.add(segment);
}
}

return usedSegments;
return publishedSegments;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
/*
* 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.indexing.common.actions;

import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.timeline.DataSegment;

import java.util.Objects;
import java.util.Set;

/**
* Task action to retrieve segments from the metadata store. Matching segments
* are returned regardless of their visibility i.e. visible, overshadowed or unused.
*/
public class RetrieveSegmentsByIdAction implements TaskAction<Set<DataSegment>>
{
private final String dataSource;
private final Set<String> segmentIds;

@JsonCreator
public RetrieveSegmentsByIdAction(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("segmentIds") Set<String> segmentIds
)
{
this.dataSource = dataSource;
this.segmentIds = segmentIds;
}

@JsonProperty
public String getDataSource()
{
return dataSource;
}

@JsonProperty
public Set<String> getSegmentIds()
{
return segmentIds;
}

@Override
public TypeReference<Set<DataSegment>> getReturnTypeReference()
{
return new TypeReference<Set<DataSegment>>()
{
};
}

@Override
public Set<DataSegment> perform(Task task, TaskActionToolbox toolbox)
{
return toolbox.getIndexerMetadataStorageCoordinator()
.retrieveSegmentsById(dataSource, segmentIds);
}

@Override
public boolean isAudited()
{
return false;
}

@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
RetrieveSegmentsByIdAction that = (RetrieveSegmentsByIdAction) o;
return Objects.equals(dataSource, that.dataSource) && Objects.equals(segmentIds, that.segmentIds);
}

@Override
public int hashCode()
{
return Objects.hash(dataSource, segmentIds);
}

@Override
public String toString()
{
return getClass().getSimpleName() + "{" +
"dataSource='" + dataSource + '\'' +
", segmentIds=" + segmentIds +
'}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -38,9 +38,8 @@
@JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class),
@JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class),
@JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class),
// Type name doesn't correspond to the name of the class for backward compatibility.
@JsonSubTypes.Type(name = "segmentListById", value = RetrieveSegmentsByIdAction.class),
@JsonSubTypes.Type(name = "segmentListUsed", value = RetrieveUsedSegmentsAction.class),
// Type name doesn't correspond to the name of the class for backward compatibility.
@JsonSubTypes.Type(name = "segmentListUnused", value = RetrieveUnusedSegmentsAction.class),
@JsonSubTypes.Type(name = "markSegmentsAsUnused", value = MarkSegmentsAsUnusedAction.class),
@JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class),
Expand Down
Loading

0 comments on commit 0fe6a2a

Please sign in to comment.