Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add replication factor column to sys table #14403

Merged
merged 18 commits into from
Jun 18, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion docs/querying/sql-metadata-tables.md
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ Segments table provides details on all Druid segments, whether they are publishe
|dimensions|VARCHAR|JSON-serialized form of the segment dimensions|
|metrics|VARCHAR|JSON-serialized form of the segment metrics|
|last_compaction_state|VARCHAR|JSON-serialized form of the compaction task's config (compaction task which created this segment). May be null if segment was not created by compaction task.|
|replication_factor|BIGINT|Total number of replicas of the segment that are required to be loaded across all historical tiers, based on the load rule that currently applies to this segment. If this value is 0, the segment is not assigned to any historical and will not be loaded. This value is -1 if load rules for the segment have not been evaluated yet or if the segment is overshadowed and will soon be marked as unused.|
|replication_factor|BIGINT|Total number of replicas of the segment that are required to be loaded across all historical tiers, based on the load rule that currently applies to this segment. If this value is 0, the segment is not assigned to any historical and will not be loaded or is about to be unloaded if it is currently loaded. This value is -1 if load rules for the segment have not been evaluated yet.|
adarshsanjeev marked this conversation as resolved.
Show resolved Hide resolved

For example, to retrieve all currently active segments for datasource "wikipedia", use the query:

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -189,13 +189,22 @@ private Response getAllUsedSegmentsWithAdditionalDetails(
final Set<DataSegment> overshadowedSegments = dataSourcesSnapshot.getOvershadowedSegments();

final Stream<SegmentStatusInCluster> segmentStatus = usedSegments
.map(segment ->
new SegmentStatusInCluster(
segment,
overshadowedSegments.contains(segment),
coordinator.getReplicationFactorForSegment(segment.getId())
)
);
.map(segment -> {
boolean isOvershadowed = overshadowedSegments.contains(segment);
Integer replicationFactor;
if (isOvershadowed) {
// If the segment is overshadowed, the replication factor won't be present in the coordinator, but we know
// that it should be 0 as we will unload it soon.
replicationFactor = 0;
} else {
replicationFactor = coordinator.getReplicationFactorForSegment(segment.getId());
}
return new SegmentStatusInCluster(
segment,
isOvershadowed,
replicationFactor
);
});

final Function<SegmentStatusInCluster, Iterable<ResourceAction>> raGenerator = segment -> Collections
.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSegment().getDataSource()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@
import com.google.common.collect.ImmutableSortedSet;
import com.google.common.util.concurrent.Uninterruptibles;
import com.google.inject.Inject;
import org.apache.curator.shaded.com.google.common.cache.Cache;
import org.apache.curator.shaded.com.google.common.cache.CacheBuilder;
import org.apache.druid.client.BrokerSegmentWatcherConfig;
import org.apache.druid.client.DataSegmentInterner;
import org.apache.druid.client.JsonParserIterator;
Expand All @@ -40,6 +42,7 @@
import org.apache.druid.metadata.SegmentsMetadataManager;
import org.apache.druid.sql.calcite.planner.SegmentMetadataCacheConfig;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.SegmentStatusInCluster;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;

Expand Down Expand Up @@ -77,6 +80,7 @@ public class MetadataSegmentView
*/
@MonotonicNonNull
private volatile ImmutableSortedSet<SegmentStatusInCluster> publishedSegments = null;
private final Cache<SegmentId, Integer> segmentIdToReplicationFactor;
adarshsanjeev marked this conversation as resolved.
Show resolved Hide resolved
private final ScheduledExecutorService scheduledExec;
private final long pollPeriodInMS;
private final LifecycleLock lifecycleLock = new LifecycleLock();
Expand All @@ -97,6 +101,9 @@ public MetadataSegmentView(
this.isCacheEnabled = config.isMetadataSegmentCacheEnable();
this.pollPeriodInMS = config.getMetadataSegmentPollPeriod();
this.scheduledExec = Execs.scheduledSingleThreaded("MetadataSegmentView-Cache--%d");
this.segmentIdToReplicationFactor = CacheBuilder.newBuilder()
.expireAfterWrite(3, TimeUnit.MINUTES)
adarshsanjeev marked this conversation as resolved.
Show resolved Hide resolved
.build();
}

@LifecycleStart
Expand Down Expand Up @@ -143,10 +150,16 @@ private void poll()
while (metadataSegments.hasNext()) {
final SegmentStatusInCluster segment = metadataSegments.next();
final DataSegment interned = DataSegmentInterner.intern(segment.getDataSegment());
Integer replicationFactor = segment.getReplicationFactor();
if (segment.getReplicationFactor() == null) {
replicationFactor = segmentIdToReplicationFactor.getIfPresent(segment.getDataSegment().getId());
} else {
segmentIdToReplicationFactor.put(segment.getDataSegment().getId(), segment.getReplicationFactor());
}
final SegmentStatusInCluster segmentStatusInCluster = new SegmentStatusInCluster(
interned,
segment.isOvershadowed(),
segment.getReplicationFactor()
replicationFactor
adarshsanjeev marked this conversation as resolved.
Show resolved Hide resolved
);
builder.add(segmentStatusInCluster);
}
Expand Down