Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
33 commits
Select commit Hold shift + click to select a range
1f342cf
Add is_overshadowed column to sys.segments table
Apr 8, 2019
635921d
update docs
Apr 8, 2019
f59e8cc
Rename class and variables
Apr 10, 2019
9bb9419
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 10, 2019
f899c0d
PR comments
Apr 11, 2019
831752c
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 11, 2019
4afb4d5
PR comments
Apr 12, 2019
b442c77
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 12, 2019
75f69f1
remove unused variables in MetadataResource
Apr 14, 2019
e6d404d
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 14, 2019
1212cef
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 15, 2019
9022399
move constants together
Apr 15, 2019
0e7fe38
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 15, 2019
fd5c8b7
add getFullyOvershadowedSegments method to ImmutableDruidDataSource
Apr 16, 2019
f3f95a4
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 16, 2019
95ad416
Fix compareTo of SegmentWithOvershadowedStatus
Apr 18, 2019
3caa904
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 18, 2019
cd7f468
PR comment
Apr 19, 2019
7e25898
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 19, 2019
48f95c0
PR comments
Apr 21, 2019
5bffa38
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 21, 2019
0c6d27f
PR comments
Apr 23, 2019
5af66b5
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 23, 2019
3ad8e34
PR comments
Apr 24, 2019
d6d0ad7
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 24, 2019
d6bc303
PR comments
Apr 25, 2019
014f4dc
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 25, 2019
2c692bb
fix issue with already consumed stream
Apr 25, 2019
3366b88
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 25, 2019
75183d0
minor refactoring
Apr 26, 2019
e1eecd2
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 26, 2019
52e297d
PR comments
Apr 29, 2019
08f627f
Merge branch 'master' of github.com:druid-io/druid into sys-table-ove…
Apr 29, 2019
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
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
/*
* 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.timeline;

import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;

/**
* DataSegment object plus the overshadowed status for the segment. An immutable object.
*
* SegmentWithOvershadowedStatus's {@link #compareTo} method considers only the {@link SegmentId}
* of the DataSegment object.
*/
public class SegmentWithOvershadowedStatus implements Comparable<SegmentWithOvershadowedStatus>
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did you explore the possibility for this class to extend DataSegment for memory saving purposes?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, in fact I started with extends DataSegment, but in order to call the super(), I had to pass the DataSegment reference to SegmentWithOvershadowedStatus, so that I get the properties for the super constructor call, something like this

 @JsonCreator
  public SegmentWithOvershadowedStatus(
      @JsonProperty("dataSegment") DataSegment segment,
      @JsonProperty("overshadowed") boolean overshadowed
  )
  {
    super(
      segment.getDataSource(),
      segment.getInterval(),
      segment.getVersion(),
      segment.getLoadSpec(),
      segment.getDimensions(),
      segment.getMetrics(),
      segment.getShardSpec(),
      segment.getBinaryVersion(),
      segment.getSize()
  );
    this.dataSegment = segment;
    this.overshadowed = overshadowed;
  }

which didn't seem correct to me, as I am both extending the class and passing the reference to same in sub-class and decided to just keep DataSegment as member of this class. Is there a better way of doing this ?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't see why SegmentWithOvershadowedStatus should have a "dataSegment" field rather than all fields deconstructed. In fact, it would allow saving a little of serialization/deserialization and the number of bytes sent over the network as well.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If I deconstruct the DataSegment object, then we might save few bytes on a reference to DataSegment, but then the memory savings in broker where interned DataSegment is used would be lost. (Getting rid of interning or not is another issue, which should be addressed outside of this PR). If the concern is bytes sent over the network, then moving to smile format instead of json can provide considerable reduction in size of bytes transferred, which I plan to do in a follow-up PR later.

Copy link
Copy Markdown
Member

@leventov leventov Apr 18, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is SegmentWithOvershadowedStatus stored somewhere for a long time? Intering an object upon deserialization and throwing it away soon doesn't make a lot of sense.

And even if the overshadowed status should be kept around on some node for a long time, you would better off apply mapping techniques such as described in #7395 instead of using plain Guava's interners. When you do this, you can insert the "overshadowed" flag wherever you want, or have something like ConcurrentHashMap<DataSegment, SegmentWithOvershadowedStatus> for storage, etc.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The coordinator API is evolvable, and is already evolving in this patch via request parameters: its response structure is different based on whether or not the includeOvershadowedStatus parameter is provided. If it needs to evolve further, then that would be okay and doable. (Although if all we do is switch to Smile, I don't think structural evolution is needed, since I imagine we would do that switch by making various APIs support both JSON and Smile based on a client header.)

By the way, we could get rid of the old formats after a few releases if we want, by deprecating them and then introducing a hard barrier that rolling updates cannot cross. We usually try to avoid doing this too often but it can be done.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BTW, I think we should have something like @ClusterInternalAPI annotation for this.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sounds like a good idea to add an annotation for all internal API's in a separate PR

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks

{
private final boolean overshadowed;
private final DataSegment dataSegment;

@JsonCreator
public SegmentWithOvershadowedStatus(
@JsonProperty("dataSegment") DataSegment dataSegment,
@JsonProperty("overshadowed") boolean overshadowed
)
{
this.dataSegment = dataSegment;
this.overshadowed = overshadowed;
}

@JsonProperty
public boolean isOvershadowed()
{
return overshadowed;
}

@JsonProperty
public DataSegment getDataSegment()
{
return dataSegment;
}

@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (!(o instanceof SegmentWithOvershadowedStatus)) {
return false;
}
final SegmentWithOvershadowedStatus that = (SegmentWithOvershadowedStatus) o;
if (!dataSegment.equals(that.dataSegment)) {
return false;
}
if (overshadowed != (that.overshadowed)) {
return false;
}
return true;
}

@Override
public int hashCode()
{
int result = dataSegment.hashCode();
result = 31 * result + Boolean.hashCode(overshadowed);
return result;
}

@Override
public int compareTo(SegmentWithOvershadowedStatus o)
{
return dataSegment.getId().compareTo(o.dataSegment.getId());
}
}
1 change: 1 addition & 0 deletions docs/content/querying/sql.md
Original file line number Diff line number Diff line change
Expand Up @@ -612,6 +612,7 @@ Note that a segment can be served by more than one stream ingestion tasks or His
|is_published|LONG|Boolean is represented as long type where 1 = true, 0 = false. 1 represents this segment has been published to the metadata store with `used=1`|
|is_available|LONG|Boolean is represented as long type where 1 = true, 0 = false. 1 if this segment is currently being served by any process(Historical or realtime)|
|is_realtime|LONG|Boolean is represented as long type where 1 = true, 0 = false. 1 if this segment is being served on any type of realtime tasks|
|is_overshadowed|LONG|Boolean is represented as long type where 1 = true, 0 = false. 1 if this segment is published and is _fully_ overshadowed by some other published segments. Currently, is_overshadowed is always false for unpublished segments, although this may change in the future. You can filter for segments that "should be published" by filtering for `is_published = 1 AND is_overshadowed = 0`. Segments can briefly be both published and overshadowed if they were recently replaced, but have not been unpublished yet.
|payload|STRING|JSON-serialized data segment payload|

For example to retrieve all segments for datasource "wikipedia", use the query:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,17 @@
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSortedMap;
import com.google.common.collect.Ordering;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.VersionedIntervalTimeline;

import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;

/**
* An immutable collection of metadata of segments ({@link DataSegment} objects), belonging to a particular data source.
Expand Down Expand Up @@ -109,6 +114,41 @@ public long getTotalSizeOfSegments()
return totalSizeOfSegments;
}

/**
* This method finds the overshadowed segments from the given segments
*
* @return set of overshadowed segments
*/
public static Set<DataSegment> determineOvershadowedSegments(Iterable<DataSegment> segments)
{
final Map<String, VersionedIntervalTimeline<String, DataSegment>> timelines = buildTimelines(segments);

final Set<DataSegment> overshadowedSegments = new HashSet<>();
for (DataSegment dataSegment : segments) {
final VersionedIntervalTimeline<String, DataSegment> timeline = timelines.get(dataSegment.getDataSource());
if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) {
overshadowedSegments.add(dataSegment);
}
}
return overshadowedSegments;
}

/**
* Builds a timeline from given segments
*
* @return map of datasource to VersionedIntervalTimeline of segments
*/
private static Map<String, VersionedIntervalTimeline<String, DataSegment>> buildTimelines(
Iterable<DataSegment> segments
)
{
final Map<String, VersionedIntervalTimeline<String, DataSegment>> timelines = new HashMap<>();
segments.forEach(segment -> timelines
.computeIfAbsent(segment.getDataSource(), dataSource -> new VersionedIntervalTimeline<>(Ordering.natural()))
.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)));
return timelines;
}

@Override
public String toString()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
package org.apache.druid.server.coordinator.helper;

import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.metadata.MetadataRuleManager;
Expand All @@ -32,14 +32,9 @@
import org.apache.druid.server.coordinator.rules.Rule;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.joda.time.DateTime;

import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;

/**
Expand Down Expand Up @@ -89,7 +84,8 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
// find available segments which are not overshadowed by other segments in DB
// only those would need to be loaded/dropped
// anything overshadowed by served segments is dropped automatically by DruidCoordinatorCleanupOvershadowed
Set<DataSegment> overshadowed = determineOvershadowedSegments(params);
final Set<DataSegment> overshadowed = ImmutableDruidDataSource
.determineOvershadowedSegments(params.getAvailableSegments());

for (String tier : cluster.getTierNames()) {
replicatorThrottler.updateReplicationState(tier);
Expand Down Expand Up @@ -138,24 +134,4 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)

return params.buildFromExisting().withCoordinatorStats(stats).build();
}

private Set<DataSegment> determineOvershadowedSegments(DruidCoordinatorRuntimeParams params)
{
Map<String, VersionedIntervalTimeline<String, DataSegment>> timelines = new HashMap<>();
for (DataSegment segment : params.getAvailableSegments()) {
timelines
.computeIfAbsent(segment.getDataSource(), dataSource -> new VersionedIntervalTimeline<>(Ordering.natural()))
.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment));
}

Set<DataSegment> overshadowed = new HashSet<>();
for (VersionedIntervalTimeline<String, DataSegment> timeline : timelines.values()) {
for (TimelineObjectHolder<String, DataSegment> holder : timeline.findOvershadowed()) {
for (DataSegment dataSegment : holder.getObject().payloads()) {
overshadowed.add(dataSegment);
}
}
}
return overshadowed;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.SegmentWithOvershadowedStatus;
import org.joda.time.Interval;

import javax.servlet.http.HttpServletRequest;
Expand All @@ -51,6 +52,7 @@
import javax.ws.rs.core.Response;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
Expand Down Expand Up @@ -147,7 +149,8 @@ public Response getDatabaseSegmentDataSource(@PathParam("dataSourceName") final
@Produces(MediaType.APPLICATION_JSON)
public Response getDatabaseSegments(
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this method is too big now, it should be split into smaller methods.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not addressed

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I looked at this, not sure best way to split it, took out parts of finding and authorizing SegmentWithOvershadowedStatus into a helper method.

@Context final HttpServletRequest req,
@QueryParam("datasources") final Set<String> datasources
@QueryParam("datasources") final Set<String> datasources,
@QueryParam("includeOvershadowedStatus") final String includeOvershadowedStatus
)
{
// If we haven't polled the metadata store yet, use an empty list of datasources.
Expand All @@ -159,14 +162,61 @@ public Response getDatabaseSegments(
}
final Stream<DataSegment> metadataSegments = dataSourceStream.flatMap(t -> t.getSegments().stream());

final Function<DataSegment, Iterable<ResourceAction>> raGenerator = segment -> Collections.singletonList(
AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource()));
if (includeOvershadowedStatus != null) {
final Iterable<SegmentWithOvershadowedStatus> authorizedSegments = findAuthorizedSegmentWithOvershadowedStatus(
req,
druidDataSources,
metadataSegments
);
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
return builder.entity(authorizedSegments).build();
} else {

final Function<DataSegment, Iterable<ResourceAction>> raGenerator = segment -> Collections.singletonList(
AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource()));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not formatted properly

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not addressed

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

manually changed


final Iterable<DataSegment> authorizedSegments = AuthorizationUtils.filterAuthorizedResources(
req,
metadataSegments::iterator,
raGenerator,
authorizerMapper
);

Response.ResponseBuilder builder = Response.status(Response.Status.OK);
return builder.entity(authorizedSegments).build();
}
}

final Iterable<DataSegment> authorizedSegments =
AuthorizationUtils.filterAuthorizedResources(req, metadataSegments::iterator, raGenerator, authorizerMapper);
private Iterable<SegmentWithOvershadowedStatus> findAuthorizedSegmentWithOvershadowedStatus(
HttpServletRequest req,
Collection<ImmutableDruidDataSource> druidDataSources,
Stream<DataSegment> metadataSegments
)
{
// It's fine to add all overshadowed segments to a single collection because only
// a small fraction of the segments in the cluster are expected to be overshadowed,
// so building this collection shouldn't generate a lot of garbage.
final Set<DataSegment> overshadowedSegments = new HashSet<>();
for (ImmutableDruidDataSource dataSource : druidDataSources) {
overshadowedSegments.addAll(ImmutableDruidDataSource.determineOvershadowedSegments(dataSource.getSegments()));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What if there are 20 brokers querying this endpoint on Coordinator? They all recompute overshadowed status (which is expensive and memory-intensive, because requires to build a VersionedIntervalTimeline) again and again.

I suggest the following:

  • isOvershadowed becomes a non-final field of DataSegment object itself, not participating in equals() and hashCode().
  • Add interface SegmentsAccess { ImmutableDruidDataSource prepare(String dataSource); Iterable<DataSegment> iterateAll(); } (strawman naming)
  • Add DataSourceAccess computeOvershadowed() method to SQLSegmentMetadataManager, which performs this computation for every snapshot of SQLSegmentMetadataManager.dataSources (which is updated in poll()) at most once, lazily.
  • Both endpoints in MetadataResource and Coordination balancing logic (which currently computes isOvershadowed status on its own, too) use this API.
  • On the side of MetadataSegmentView, maintain something like a Map<DataSegment, DataSegment> and update overshadowed status like map.get(segmentFromCoordinator).setOvershadowed(segmentFromCoordinator.isOvershadowed()).

Result: we don't do any repetitive computations of overshadowed segments for every SQLSegmentMetadataManager.dataSegments snapshot whatsoever.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for your suggestion, but this code change is not contained in this method and would affect other places in the code, some of which are not part of original PR( eg coordinator balancing logic). I would prefer to do this change separately as it suggests changing DataSegment object and adding new interfaces, so there may be more follow-up discussions. Created #7571 to address this comment.

Copy link
Copy Markdown
Member

@leventov leventov Apr 30, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this code change is not contained in this method and would affect other places in the code, some of which are not part of original PR( eg coordinator balancing logic

I don't see how the files touched in the original PR are special. If you have implemented the above suggestion from the beginning those files would be part of the "original" PR.

Touching relatively unrelated files is normal when you do refactoring, in fact, that's one of the objectives of refactoring - to gather functionality that accidentally happens to scatter unrelated places in a single place.

I would prefer to do this change separately as it suggests changing DataSegment object and adding new interfaces, so there may be more follow-up discussions.

I won't block this PR from merging if other reviewers of this PR (@gianm @jihoonson @jon-wei) agree with that design on a high level (or propose another solution that solves the same problem) and it's being implemented just after this PR. Because the current design doesn't seem reasonable to me at this point. (So there won't be much difference from as if you just do the implementation right in this PR, but if you wish you can separate in two PRs.)

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@leventov I am working on #7571. Agree the current API is not most efficient and I acknowledge your concern. While I am not sure what's the most appropriate way to avoid recalculating overshadowed segments yet, I looked at the suggested changes and I have some questions, which I have asked in #7571. Could we agree to discuss the design there, I think it'll make it easier for you and me and others to review those changes as this PR is getting crowded, and we may miss some parts about the new changes as they get mixed up with existing changes.

Copy link
Copy Markdown
Contributor

@jon-wei jon-wei May 1, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I haven't really formed an opinion on DataSegment mutability presently, but I think @leventov's suggestion for lazily computing the overshadowed view at most once per SQLSegmentMetadataManager poll() and sharing that view with the metadata retrieval APIs and the coordinator balancing logic makes a lot of sense.

Because the current design doesn't seem reasonable to me at this point. (So there won't be much difference from as if you just do the implementation right in this PR, but if you wish you can separate in two PRs.)

I agree with making the adjustment to the overshadowed view computation as an immediate follow on, I think a separate PR is a bit better:

  • The coordinator balancing logic is a pretty "core" part of the system, and I feel like it would be better to change that in a separate PR that calls attention more explicitly to that/isolates that change more
  • This PR is getting a bit long, a little tedious to navigate

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks

}

final Response.ResponseBuilder builder = Response.status(Response.Status.OK);
return builder.entity(authorizedSegments).build();
final Stream<SegmentWithOvershadowedStatus> segmentsWithOvershadowedStatus = metadataSegments
.map(segment -> new SegmentWithOvershadowedStatus(
segment,
overshadowedSegments.contains(segment)
));

final Function<SegmentWithOvershadowedStatus, Iterable<ResourceAction>> raGenerator = segment -> Collections
.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSegment().getDataSource()));

final Iterable<SegmentWithOvershadowedStatus> authorizedSegments = AuthorizationUtils.filterAuthorizedResources(
req,
segmentsWithOvershadowedStatus::iterator,
raGenerator,
authorizerMapper
);
return authorizedSegments;
}

@GET
Expand Down
Loading