Optimize overshadowed segments computation#7595
Conversation
| * of the DataSegment object. | ||
| */ | ||
| public class SegmentWithOvershadowedStatus implements Comparable<SegmentWithOvershadowedStatus> | ||
| public class DataSegmentWithOvershadowedStatus implements Comparable<DataSegmentWithOvershadowedStatus> |
There was a problem hiding this comment.
I don't think this is a good rename. I think "DataSegment" is an unfortunate name, so why propagating it.
There was a problem hiding this comment.
hmm, since it's a wrapper on DataSegment, that's why I felt this might make more sense, i'd be okay reverting it back to SegmentWithOvershadowedStatus if there are plans to rename DataSegment , although that seems unclear at this point.
There was a problem hiding this comment.
I would undo the rename, I don't think it's really necessary here
|
|
||
| // Replace "dataSources" atomically. | ||
| dataSources = newDataSources; | ||
| overshadowedSegments = ImmutableSet.copyOf(determineOvershadowedSegments(segments)); |
There was a problem hiding this comment.
There are some changes in this class to dataSources apart from in doPoll() that may make computed overshadowed segments invalid. (Even if they don't, there should be comments about that.)
There was a problem hiding this comment.
Why would the overshadowed segments be invalid, do you think it can happen when some dataSources are enabled or disabled outside doPoll? Also, if they do become invalid, would a comment be enough, or something should be done in code to prevent invalid overshadowed segments ?
There was a problem hiding this comment.
when some dataSources are enabled or disabled outside doPoll
Yes. Also note that there are additional changes here: #7653 (see section "SQLMetadataSegmentManager: remove data from dataSources in markAsUnusedSegmentsInInterval and markSegmentsAsUnused"), which were missed in #7490.
Also, if they do become invalid, would a comment be enough, or something should be done in code to prevent invalid overshadowed segments ?
I think overshadowedSegments should become invalidated.
There was a problem hiding this comment.
looks like dataSources map is not updated when new datasources are enabled, so i have removed the update from removeDataSource to make it consistent, and it should be updated only in doPoll() ideally. And added a comment about that.
There was a problem hiding this comment.
btw, i reverted the above mentioned change and added DataSourcesSnapshot
| * | ||
| * @return set of overshadowed segments | ||
| */ | ||
| private Set<DataSegment> determineOvershadowedSegments(Iterable<DataSegment> segments) |
There was a problem hiding this comment.
Doesn't need to be set, you copy into another Set anyway
| .determineOvershadowedSegments(params.getAvailableSegments()); | ||
| // If metadata store hasn't been polled yet, use empty overshadowed list | ||
| final Collection<DataSegment> overshadowed = Optional | ||
| .ofNullable(coordinator.getMetadataSegmentManager().findOvershadowedSegments()) |
There was a problem hiding this comment.
There are some concurrency issues: you may observe overshadowedSegments from a previous run. I think it would be easier to add isOvershadowed field to DataSegment directly in this PR because that's the plan anyway and that would allow avoiding concurrency issues.
There was a problem hiding this comment.
If it's acceptable to get updated overshadowedSegments in the next run of DruidCoordinatorHelper#run, then I think it might be ok. If not, then may be we should compute the overshadowed list here itself, like before, until the mutability of DataSegment is settled.
There was a problem hiding this comment.
Why are you reluctant to make changes to DataSegment? You refer to the design as "debated" or "not settled", well, we should resolve the debate and settle somewhere. I expressed my view in #7571. If you disagree with some parts of it, please respond in that issue.
I'm concerned with keeping the status quo because if DataSegmentWithOvershadowedStatus will leak into Druid 0.15 API, it will be much harder (transition PRs, compatibility, etc.) to remove it eventually. So I really want it to go away before Druid 0.15 is released.
There was a problem hiding this comment.
This is settled in #7571(comment). will be sending a flat format over the network for SegmentWithOvershadowedStatus without exposing the wrapping.
| * not yet been polled.) | ||
| */ | ||
| @Nullable | ||
| Collection<DataSegment> findOvershadowedSegments(); |
There was a problem hiding this comment.
A method which is a simple getter shouldn't be called find..., implying high cost.
jon-wei
left a comment
There was a problem hiding this comment.
In my view, there are two related but separate issues here:
- Avoiding repeated overshadowed computation for each sys.segments query
- Deciding on DataSegment mutability/wrapper class/etc.
Of these two, I think the optimization to avoid repeated computation is the more crucial issue and the one that I would consider more of a release blocker (the second issue not so much), given the performance implications we should get that into 0.15.0 in some form (it can be adjusted and perfected later).
From the current discussion in #7571, it looks like consensus hasn't been reached on the second issue. While it would certainly be nice to not have to change APIs and transition, if consensus is still forming, it can't be helped. If it's possible to address the first issue without reaching a decision on the second, then we should take that approach for 0.15.0.
So for this PR, I'm okay with it after it addresses @leventov's comments on invalidation of the overshadowed segments set here: #7595 (comment).
For the concern about DruidCoordinatorRuleRunner seeing a stale view, do you foresee issues with that acting on an older view of the overshadowed segments? Seems like it would be fine as later runs would see updated information. If there are problems with that, I think it'd be fine to have the DruidCoordinatorRuleRunner compute a fresh view as @surekhasaharan suggested (since this would only happen once per coordinator period) and for now have the precomputed overshadowed view only used for external queries.
| } | ||
|
|
||
| /** | ||
| * This method builds a timeline from given segments and finds the overshadowed segments |
There was a problem hiding this comment.
nit: the comment refers to a single timeline but multiple timelines are being built
| * of the DataSegment object. | ||
| */ | ||
| public class SegmentWithOvershadowedStatus implements Comparable<SegmentWithOvershadowedStatus> | ||
| public class DataSegmentWithOvershadowedStatus implements Comparable<DataSegmentWithOvershadowedStatus> |
There was a problem hiding this comment.
I would undo the rename, I don't think it's really necessary here
It's not just a view of overshadowed segments, right? The entire timeline would be an older snapshot. I think as long as it's not 'surprisingly rolled back' then it is OK. Meaning: no run should use an older snapshot than a prior run. Within any given coordinator leadership epoch, the snapshot at time T will be newer than the snapshot at previous times < T. So avoiding surprising rollbacks really boils down to avoiding a situation where the new leader uses an older snapshot than the old leader. I think the best way to avoid this is to make sure that any snapshot used in a given leadership epoch must have been taken after the start of that epoch. (If it was from prior to the start of the epoch, it might be out of date and using it might lead to a surprising rollback.) |
…shadowed-segments
…shadowed-segments
|
Re @jon-wei's comment,
That sounds right to me. This PR is meant to address the former and I agree with @jon-wei that it makes sense to treat it as a release blocker. I'll add it to the 0.15.0 milestone, if there are no objections. The latter is still being sorted out on #7571. |
| @@ -744,6 +755,35 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE | |||
|
|
|||
| // Replace "dataSources" atomically. | |||
There was a problem hiding this comment.
"dataSources" and "overshadowedSegments" should be updated atomically, together. Updating them separately means we could get info about overshadowed segments without actually knowing they're overshadowed (or vice versa). The simplest way to do this is to wrap them into a single object and then swap in that object atomically.
One way to do this is to make getDataSources() return Collection<DataSourceSnapshot> instead of Collection<ImmutableDruidDataSource>, and make DataSourceSnapshot something that contains both an ImmutableDruidDataSource and an ImmutableSet<SegmentId> overshadowedSegments. For each one, build the overshadowedSegments list by running VersionedIntervalTimeline.forSegments(...) on the segments for that datasource, and then extracting the overshadowed ones.
There was a problem hiding this comment.
I don't mind if this is implemented, but isn't it a waste of resources since the merge of DataSegment and SegmentWithOvershadowedStatus objects will make it all obsolete? See #7571 (comment).
There was a problem hiding this comment.
@leventov aren't you suggesting the same thing in #7571 (comment), with your first suggestion and your second suggestion
but this problem naturally goes away itself when SQLSegmentMetadataManager will store SegmentWithOvershadowedStatus objects in dataSources
I'm not sure I understand it, so did you mean to store SegmentWithOvershadowedStatus(set of overshadowed segments) in DruidDataSource object?
There was a problem hiding this comment.
created a DataSourceSnapshot class which contains a snapshot of druidDatasources and computes the overshadowed segment from that snapshot, this is invoked from doPoll(), so both dataSources and overshadowedSegments are updated atomically in every poll cycle. And the getters like getDataSources and getOvershadowedSegments use this snapshot class to return the values.
| final Set<DataSegment> overshadowed = ImmutableDruidDataSource | ||
| .determineOvershadowedSegments(params.getAvailableSegments()); | ||
| // If metadata store hasn't been polled yet, use empty overshadowed list | ||
| final Collection<DataSegment> overshadowed = Optional |
There was a problem hiding this comment.
Keep this as a Set, please. Later on, we're doing overshadowed.contains(segment), and that's scary if overshadowed is any kind of Collection other than a Set (or very short List).
By the way, it would be clearer to make this a Set<SegmentId> rather than Set<DataSegment>.
| overshadowedSegments.addAll(ImmutableDruidDataSource.determineOvershadowedSegments(dataSource.getSegments())); | ||
| } | ||
| // If metadata store hasn't been polled yet, use empty overshadowed list | ||
| final Collection<DataSegment> overshadowedSegments = Optional |
There was a problem hiding this comment.
Keep this as a Set, please. Later on, we're doing overshadowedSegments.contains(segment), and that's scary if overshadowed is any kind of Collection other than a Set (or very short List).
By the way, it would be clearer to make this a Set<SegmentId> rather than Set<DataSegment>.
| * DataSegment object plus the overshadowed status for the segment. An immutable object. | ||
| * | ||
| * SegmentWithOvershadowedStatus's {@link #compareTo} method considers only the {@link SegmentId} | ||
| * DataSegmentWithOvershadowedStatus's {@link #compareTo} method considers only the {@link SegmentId} |
There was a problem hiding this comment.
DataSegmentWithOvershadowedStatus -> SegmentWithOvershadowedStatus's in the javadoc
|
|
||
| public ImmutableSet<SegmentId> getOvershadowedSegments() | ||
| { | ||
| return ImmutableSet.copyOf(determineOvershadowedSegments()); |
There was a problem hiding this comment.
getOvershadowedSegments() is recomputing the overshadowed segments every time, this should just return a precomputed ImmutableSet<SegmentId>
| private final boolean overshadowed; | ||
| /** | ||
| * dataSegment is serialized "unwrapped", i.e. it's properties are included as properties of | ||
| * enclosing class. If in future, if {@Code SegmentWithOvershadowedStatus} were to extend {@link DataSegment}, |
There was a problem hiding this comment.
Suggest adding a test that verifies that the serialized form generated with the unwrapped annotation can be successfully deserialized as a subclass (can make a test impl) of DataSegment
There was a problem hiding this comment.
yeah, should test this, will try to add one.
| * overshadowedSegments). Getters of {@link org.apache.druid.metadata.MetadataSegmentManager} should use this snapshot | ||
| * to return dataSources and overshadowedSegments. | ||
| */ | ||
| public class DataSourceSnapshot |
There was a problem hiding this comment.
Suggest DataSourcesSnapshot instead, since this is a snapshot for multiple datasources
|
@surekhasaharan Can you check the travis failures? |
| * enclosing class. If in future, if {@Code SegmentWithOvershadowedStatus} were to extend {@link DataSegment}, | ||
| * there will be no change in the serialized format. | ||
| */ | ||
| @JsonUnwrapped |
There was a problem hiding this comment.
Can you please also ensure that the data is sent in smile format, not JSON, between Coordinator and Brokers?
|
|
||
| @Override | ||
| @Nullable | ||
| public Set<SegmentId> getOvershadowedSegments() |
There was a problem hiding this comment.
Just wrapping/unwrapping objects into snapshot doesn't magically eliminate the race. You have to pass around the snapshot.
There was a problem hiding this comment.
ok, updating the snapshot on dataSources and segments update.
| @@ -108,6 +110,8 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager | |||
| // null and nonnull multiple times as stop() and start() are called. | |||
| @Nullable | |||
| private volatile ConcurrentHashMap<String, DruidDataSource> dataSources = null; | |||
There was a problem hiding this comment.
it was being used, but not anymore with updates via the snapshot.
…shadowed-segments
| ) | ||
| ); | ||
| if (dataSourcesSnapshot != null) { | ||
| final Map<String, ImmutableDruidDataSource> dataSourcesMap = dataSourcesSnapshot.getDataSourcesMap(); |
There was a problem hiding this comment.
The addition of overshadowed computation makes the snapshot invalidation/update an expensive operation (suppose a user issues a lot of single segment remove calls in rapid succession), I'm thinking the following:
- For 0.15.0 release, remove the behavior where the snapshot is being updated outside of poll(): these updates were primarily for user experience (so if someone removes a segment, they would immediately see that reflected in MetadataResource API calls), but the coordinator loop was operating on datasources/available segments snapshots that are not updated by removeDatasource/removeSegment, so removing that update behavior would not break the coordinator loop.
- Adding the complexity of a system for reducing the performance impact of repeated invalidating operations is too risky this close to 0.15.0 release IMO, and I think the potential performance degradation outweighs the user experience benefit of updating the snapshot outside of poll()
- After 0.15.0, if we want the snapshot to more rapidly reflect changes caused by operations outside of the scheduled poll, I think it makes sense to look into that after the on-demand polling changes from Refactor SQLMetadataSegmentManager; Change contract of REST methods in DataSourcesResource #7653 are merged.
Does that sound reasonable? @surekhasaharan @leventov @gianm
There was a problem hiding this comment.
yes, that sounds reasonable to me, I attempted to do this here earlier
There was a problem hiding this comment.
Sounds good. Please ensure that it's documented properly: the comments should include the reasoning and background from @jon-wei's comment.
| private void replaceDataSourcesSnapshot(Map<String, ImmutableDruidDataSource> dataSources) | ||
| { | ||
| dataSourcesSnapshot = new DataSourcesSnapshot(dataSources); | ||
| /** |
There was a problem hiding this comment.
Javadoc comment can't be inside a method
There was a problem hiding this comment.
changed to regular comment
| * added back in removeDataSource and removeSegment methods after the on-demand polling changes from | ||
| * https://github.com/apache/incubator-druid/pull/7653 are in. | ||
| */ | ||
| dataSourcesSnapshot = new DataSourcesSnapshot(newDataSources.entrySet() |
There was a problem hiding this comment.
Please factor the DataSourcesSnapshot()'s argument out for better formatting
There was a problem hiding this comment.
hmm, i don't see any difference in formatting even if i take the argument out, it'd be like
Map<String, ImmutableDruidDataSource> updatedDataSources = newDataSources.entrySet()
.stream()
.collect(Collectors.toMap(
e -> e.getKey(),
e -> e.getValue()
.toImmutableDruidDataSource()
));
dataSourcesSnapshot = new DataSourcesSnapshot(updatedDataSources);
And I don't see the point of creating this map, since it's only used as argument to DataSourcesSnapshot()
There was a problem hiding this comment.
Although it's not formally prohibited by the code style, I prefer fixed indentation rather than aligned indentation. Aligned indentation forces excessive line breakups, as in your example. Also, when variables or types are renamed aligned indentation breaks.
Your code could be
Map<String, ImmutableDruidDataSource> updatedDataSources = newDataSources
.entrySet()
.stream()
.collect(Collectors.toMap(Entry::getKey, e -> e.getValue().toImmutableDruidDataSource()));
dataSourcesSnapshot = new DataSourcesSnapshot(updatedDataSources);But in this case, it's actually better to extract the first part as a method Map<K, V2> mapValues(Map<K, V> map, Function<V, V2> valueMapper) in CollectionUtils because this boilerplate code appears in several places in the codebase. Could you please do that?
There was a problem hiding this comment.
Absolutely, since you asked nicely :) Added a method in CollectionUtils, let me know if that's what you wanted.
| * snapshot is invalidated on each segment removal, especially if a user issues a lot of single segment remove | ||
| * calls in rapid succession. So the snapshot update is not done outside of poll at this time. | ||
| * Updates outside of poll(), were primarily for the user experience, so users would immediately see the effect of | ||
| * a segment remove call reflected in MetadataResource API calls. These updates outside of schecduled poll may be |
| @@ -506,20 +477,7 @@ public boolean removeSegment(String dataSourceName, final String segmentId) | |||
| public boolean removeSegment(SegmentId segmentId) | |||
There was a problem hiding this comment.
This method now looks the same as public boolean removeSegment(String dataSourceName, final String segmentId), i. e. you can simplify (at least dataSourceName param is completely unused), potentially remove one of these methods completely. Also please update their Javadocs.
There was a problem hiding this comment.
both these methods are used from MetadataSegmentManager, so not sure if I can remove it. I made the removeSegment(String dataSourceName, final String segmentId) call removeSegment(SegmentId segmentId) internally. The javadoc for these is in the interface MetadataSegmentManager, will add a note here that they do not update the snapshot now.
| log.error(e, e.toString()); | ||
| return false; | ||
| } | ||
| final SegmentId segmentId = SegmentId.tryParse(dataSourceName, identifier); |
There was a problem hiding this comment.
You parse a SegmentId object and pass it to a method which essentially just converts it back to string. If these methods should still exist, the public boolean removeSegment(SegmentId segmentId) should delegate to this one, not the other way around. But actually I think it's better to leave only string-accepting method and force clients to call it with SegmentId by calling toString() themselves because removeSegment(SegmentId segmentId) just calling toString() and calling into removeSegment(String) is too shallow and makes the false impression of "better quality" (because accepts a non-string object).
There was a problem hiding this comment.
I removed removeSegment(String dataSourceName, final String segmentId) as it was used at only one place and changed removeSegment(SegmentId segmentId) to removeSegment(String segmentId).
| */ | ||
| public static <K, V, V2> Map<K, V2> transformValues(Map<K, V> map, Function<V, V2> valueMapper) | ||
| { | ||
| final Map<K, V2> result = new HashMap<>(); |
There was a problem hiding this comment.
Better to use Maps.newHashMapWithExpectedSize(map.size()) here
| dataSourcesSnapshot = new DataSourcesSnapshot(updatedDataSources); | ||
| } | ||
|
|
||
| /** |
| * Returns a transformed map from the given input map where the value is modified based on the given valueMapper | ||
| * function. | ||
| */ | ||
| public static <K, V, V2> Map<K, V2> transformValues(Map<K, V> map, Function<V, V2> valueMapper) |
There was a problem hiding this comment.
Please use this method elsewhere. There are at least places in IndexTask, TaskQueue (three places), KafkaSupervisor, etc. where it can be used.
It doesn't make sense to add a utility method and not to use it.
To find all such places, use the following structural search:
$x$.entrySet().stream().collect($y$)
There was a problem hiding this comment.
Also, don't call the method "transformValues" because it collides with Guava's Maps.transformValues() which returns a lazy map. This would be a source of confusion.
There was a problem hiding this comment.
You can actually enforce usage of the new method using by adding these two Structural Search patterns:
$x$.entrySet().stream().collect(Collectors.toMap(Entry::getKey, $y$))
and
$x$.entrySet().stream().collect(Collectors.toMap($k$ -> $k$.getKey(), $y$))
There was a problem hiding this comment.
@leventov please note that this is the only remaining release blocker. I don't think any of your comments here should be addressed in this PR. They are out of scope and should be done separately.
There was a problem hiding this comment.
I'll rename the method, but will address the rest in a follow-up PR after this one is merged ?
| * Returns a transformed map from the given input map where the value is modified based on the given valueMapper | ||
| * function. | ||
| */ | ||
| public static <K, V, V2> Map<K, V2> transformValues(Map<K, V> map, Function<V, V2> valueMapper) |
There was a problem hiding this comment.
@leventov please note that this is the only remaining release blocker. I don't think any of your comments here should be addressed in this PR. They are out of scope and should be done separately.
| private final SQLMetadataConnector connector; | ||
|
|
||
| // Volatile since this reference is reassigned in "poll" and then read from in other threads. | ||
| // Starts null so we can differentiate "never polled" (null) from "polled, but empty" (empty map). |
There was a problem hiding this comment.
Why was this comment removed? It looks still valid.
There was a problem hiding this comment.
ah removed it when I removed the dataSources map, but it's still relevant if DataSourcesSnapshot has empty dataSources map and empty overshadowedSegments. Added the comment.
|
Tagged |
|
@leventov it has been a month since we branch out 0.15.0 and so I'm hoping to create the first RC for 0.15.0 by end of this week. Would you mind trying to finish you review as soon as possible? |
| @@ -106,6 +106,8 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager | |||
| private final SQLMetadataConnector connector; | |||
|
|
|||
| // Volatile since this reference is reassigned in "poll" and then read from in other threads. | |||
There was a problem hiding this comment.
@surekhasaharan to the list of things to address in a follow-up PR: make the comment to a field a Javadoc comment and make all references to methods and fields that appear in it links like: {@link }.
| @@ -75,9 +76,9 @@ public int size() | |||
| * Returns a transformed map from the given input map where the value is modified based on the given valueMapper | |||
| * function. | |||
| */ | |||
There was a problem hiding this comment.
To the list of things to address in a follow-up PR: highlight the difference from Maps.transformValues() in the documentation, e. g. add a sentence like Unlike {@link Maps#transformValues}, this method applies the mapping function eagerly to all key-value pairs in the source map and returns a new {@link HashMap}, while {@link Maps#transformValues} returns a lazy map view.
|
@leventov thank you for merging this PR! |
* Move the overshadowed segment computation to SQLMetadataSegmentManager's poll * rename method in MetadataSegmentManager * Fix tests * PR comments * PR comments * PR comments * fix indentation * fix tests * fix test * add test for SegmentWithOvershadowedStatus serde format * PR comments * PR comments * fix test * remove snapshot updates outside poll * PR comments * PR comments * PR comments * removed unused import
* Move the overshadowed segment computation to SQLMetadataSegmentManager's poll * rename method in MetadataSegmentManager * Fix tests * PR comments * PR comments * PR comments * fix indentation * fix tests * fix test * add test for SegmentWithOvershadowedStatus serde format * PR comments * PR comments * fix test * remove snapshot updates outside poll * PR comments * PR comments * PR comments * removed unused import
* Move the overshadowed segment computation to SQLMetadataSegmentManager's poll * rename method in MetadataSegmentManager * Fix tests * PR comments * PR comments * PR comments * fix indentation * fix tests * fix test * add test for SegmentWithOvershadowedStatus serde format * PR comments * PR comments * fix test * remove snapshot updates outside poll * PR comments * PR comments * PR comments * removed unused import
Addresses part of #7571
This patch moves the computation of overshadowed segments to
SQLMetadataSegmentManager's poll method. The endpoint inMetadataResourceandDruidCoordinatorRuleRunneruse the APIMetadataSegmentManager#findOvershadowedSegments()to get overshadowed segments without building theVersionedIntervalTimelineevery time.Renamed
SegmentWithOvershadowedStatustoDataSegmentWithOvershadowedStatus.This PR does not address the mutability of
DataSegmentand getting rid of interning inMatadataSegmentView