-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Add is_overshadowed column to sys.segments table #7425
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
Changes from all commits
1f342cf
635921d
f59e8cc
9bb9419
f899c0d
831752c
4afb4d5
b442c77
75f69f1
e6d404d
1212cef
9022399
0e7fe38
fd5c8b7
f3f95a4
95ad416
3caa904
cd7f468
7e25898
48f95c0
5bffa38
0c6d27f
5af66b5
3ad8e34
d6d0ad7
d6bc303
014f4dc
2c692bb
3366b88
75183d0
e1eecd2
52e297d
08f627f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| 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> | ||
| { | ||
| 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()); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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; | ||
|
|
@@ -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; | ||
|
|
@@ -147,7 +149,8 @@ public Response getDatabaseSegmentDataSource(@PathParam("dataSourceName") final | |
| @Produces(MediaType.APPLICATION_JSON) | ||
| public Response getDatabaseSegments( | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not addressed
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 |
||
| @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. | ||
|
|
@@ -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())); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not formatted properly
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not addressed
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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())); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 I suggest the following:
Result: we don't do any repetitive computations of overshadowed segments for every
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
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 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.)
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
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:
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 | ||
|
|
||
There was a problem hiding this comment.
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
DataSegmentfor memory saving purposes?There was a problem hiding this comment.
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 theDataSegmentreference toSegmentWithOvershadowedStatus, so that I get the properties for the super constructor call, something like thiswhich 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
DataSegmentas member of this class. Is there a better way of doing this ?There was a problem hiding this comment.
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
SegmentWithOvershadowedStatusshould 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.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If I deconstruct the
DataSegmentobject, 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.Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is
SegmentWithOvershadowedStatusstored 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.There was a problem hiding this comment.
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
includeOvershadowedStatusparameter 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.
There was a problem hiding this comment.
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
@ClusterInternalAPIannotation for this.There was a problem hiding this comment.
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
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
#7542
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks