-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Make realtimes available for loading segments #4148
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
f4144e7
ce796e3
430021c
6a220e1
b386da6
232a2f0
1fc3f72
cfd61fc
2ef88de
b682064
6d581db
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 |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ | |
|
|
||
| import com.google.common.collect.ImmutableMap; | ||
| import io.druid.server.coordination.DruidServerMetadata; | ||
| import io.druid.server.coordination.ServerType; | ||
| import io.druid.timeline.DataSegment; | ||
|
|
||
| import java.util.Map; | ||
|
|
@@ -72,7 +73,7 @@ public long getMaxSize() | |
| return metadata.getMaxSize(); | ||
| } | ||
|
|
||
| public String getType() | ||
| public ServerType getType() | ||
| { | ||
| return metadata.getType(); | ||
| } | ||
|
|
@@ -106,4 +107,15 @@ public Map<String, DataSegment> getSegments() | |
| { | ||
| return segments; | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() | ||
| { | ||
| // segments is intentionally ignored because it is usually large | ||
| return "ImmutableDruidServer{" | ||
| + "meta='" + metadata | ||
| + "', size='" + currSize | ||
| + "', sources='" + dataSources | ||
| + "'}"; | ||
|
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. Doesn't include
Contributor
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. Yes, a server usually holds a lot of segments.
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. Please leave a comment
Contributor
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. Added a comment. |
||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,196 @@ | ||
| /* | ||
| * Licensed to Metamarkets Group Inc. (Metamarkets) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. Metamarkets 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 io.druid.server; | ||
|
|
||
| import com.google.common.collect.Ordering; | ||
| import com.google.inject.Inject; | ||
| import com.metamx.emitter.EmittingLogger; | ||
| import io.druid.collections.CountingMap; | ||
| import io.druid.segment.ReferenceCountingSegment; | ||
| import io.druid.segment.Segment; | ||
| import io.druid.segment.loading.SegmentLoader; | ||
| import io.druid.segment.loading.SegmentLoadingException; | ||
| import io.druid.timeline.DataSegment; | ||
| import io.druid.timeline.VersionedIntervalTimeline; | ||
| import io.druid.timeline.partition.PartitionChunk; | ||
| import io.druid.timeline.partition.PartitionHolder; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
|
|
||
| public class SegmentManager | ||
| { | ||
| private static final EmittingLogger log = new EmittingLogger(SegmentManager.class); | ||
|
|
||
| private final Object lock = new Object(); | ||
| private final SegmentLoader segmentLoader; | ||
| private final Map<String, VersionedIntervalTimeline<String, ReferenceCountingSegment>> dataSources = new HashMap<>(); | ||
|
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. There could be one
Contributor
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. SegmentManager is simply separated from ServerManager. Your comments around SegmentManager and ServerManager look good, but I think it is not a part of this pr. Maybe better to raise a new issue after this pr.
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. Ok but would be nice if you could do this as part of this PR, it's 15 min of work, another issue and a separate PR will eat more of everybody's attention
Contributor
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. For the issues which are less related to the original issues, I think it would be fine to fix them in a single PR if the changes are truly little and intuitive, but otherwise, it's better to separate to several PRs even though some PRs will be quite small. This is because
Besides, even in the world you mentioned, a lock is needed anyway for synchronization for accessing private final ConcurrentHashMap<String, DataSourceState> dataSources;
...
public boolean loadSegment(final DataSegment segment) throws SegmentLoadingException
{
synchronized (lock) {
final DataSourceState dataSourceState = dataSources.get(dataSource);
final PartitionHolder<ReferenceCountingSegment> entry = dataSourceState.findEntry(segment.getInterval(), segment.getVersion);
...
dataSourceState.add(
segment.getInterval(),
segment.getVersion(),
segment.getShardSpec().createChunk(new ReferenceCountingSegment(adapter))
);
...
}
}For this issue, I prefer to investigate first what the exact requirements are.
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. Ok for not including refactoring into this PR. Synchronization that you mentioned still needed could be managed by ConcurrentHashMap: dataSources.compute(dataSource, dataSourceState -> {
if (dataSourceState == null) ...
else {
dataSourceState.add(...);
return dataSourceState;
}
});ConcurrentHashMap guarantees that executions of lambdas provided to
Contributor
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. Ah, right. That way should work. |
||
| private final CountingMap<String> dataSourceSizes = new CountingMap<>(); | ||
| private final CountingMap<String> dataSourceCounts = new CountingMap<>(); | ||
|
|
||
| @Inject | ||
| public SegmentManager( | ||
| SegmentLoader segmentLoader | ||
| ) | ||
| { | ||
| this.segmentLoader = segmentLoader; | ||
| } | ||
|
|
||
| public Map<String, Long> getDataSourceSizes() | ||
| { | ||
| synchronized (dataSourceSizes) { | ||
| return dataSourceSizes.snapshot(); | ||
| } | ||
| } | ||
|
|
||
| public Map<String, Long> getDataSourceCounts() | ||
| { | ||
| synchronized (dataSourceCounts) { | ||
| return dataSourceCounts.snapshot(); | ||
| } | ||
| } | ||
|
|
||
| public boolean isSegmentCached(final DataSegment segment) throws SegmentLoadingException | ||
| { | ||
| return segmentLoader.isSegmentLoaded(segment); | ||
| } | ||
|
|
||
| public VersionedIntervalTimeline<String, ReferenceCountingSegment> getTimeline(String dataSource) | ||
| { | ||
| synchronized (lock) { | ||
| return dataSources.get(dataSource); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Load a single segment. | ||
| * | ||
| * @param segment segment to load | ||
| * | ||
| * @return true if the segment was newly loaded, false if it was already loaded | ||
| * | ||
| * @throws SegmentLoadingException if the segment cannot be loaded | ||
| */ | ||
| public boolean loadSegment(final DataSegment segment) throws SegmentLoadingException | ||
| { | ||
| final Segment adapter = getAdapter(segment); | ||
|
|
||
| synchronized (lock) { | ||
| final String dataSource = segment.getDataSource(); | ||
| final VersionedIntervalTimeline<String, ReferenceCountingSegment> loadedIntervals = dataSources.computeIfAbsent( | ||
| dataSource, | ||
| k -> new VersionedIntervalTimeline<>(Ordering.natural()) | ||
| ); | ||
|
|
||
| final PartitionHolder<ReferenceCountingSegment> entry = loadedIntervals.findEntry( | ||
| segment.getInterval(), | ||
| segment.getVersion() | ||
| ); | ||
| if ((entry != null) && (entry.getChunk(segment.getShardSpec().getPartitionNum()) != null)) { | ||
| log.warn("Told to load a adapter for a segment[%s] that already exists", segment.getIdentifier()); | ||
| return false; | ||
| } | ||
|
|
||
| loadedIntervals.add( | ||
| segment.getInterval(), | ||
| segment.getVersion(), | ||
| segment.getShardSpec().createChunk(new ReferenceCountingSegment(adapter)) | ||
| ); | ||
| synchronized (dataSourceSizes) { | ||
| dataSourceSizes.add(dataSource, segment.getSize()); | ||
| } | ||
| synchronized (dataSourceCounts) { | ||
| dataSourceCounts.add(dataSource, 1L); | ||
| } | ||
| return true; | ||
| } | ||
| } | ||
|
|
||
| private Segment getAdapter(final DataSegment segment) throws SegmentLoadingException | ||
| { | ||
| final Segment adapter; | ||
| try { | ||
| adapter = segmentLoader.getSegment(segment); | ||
| } | ||
| catch (SegmentLoadingException e) { | ||
| try { | ||
| segmentLoader.cleanup(segment); | ||
| } | ||
| catch (SegmentLoadingException e1) { | ||
| e.addSuppressed(e1); | ||
| } | ||
| throw e; | ||
| } | ||
|
|
||
| if (adapter == null) { | ||
| throw new SegmentLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); | ||
| } | ||
| return adapter; | ||
| } | ||
|
|
||
| public void dropSegment(final DataSegment segment) throws SegmentLoadingException | ||
| { | ||
| String dataSource = segment.getDataSource(); | ||
| synchronized (lock) { | ||
| VersionedIntervalTimeline<String, ReferenceCountingSegment> loadedIntervals = dataSources.get(dataSource); | ||
|
|
||
| if (loadedIntervals == null) { | ||
| log.info("Told to delete a queryable for a dataSource[%s] that doesn't exist.", dataSource); | ||
| return; | ||
| } | ||
|
|
||
| PartitionChunk<ReferenceCountingSegment> removed = loadedIntervals.remove( | ||
| segment.getInterval(), | ||
| segment.getVersion(), | ||
| segment.getShardSpec().createChunk(null) | ||
| ); | ||
| ReferenceCountingSegment oldQueryable = (removed == null) ? null : removed.getObject(); | ||
|
|
||
| if (oldQueryable != null) { | ||
| synchronized (dataSourceSizes) { | ||
| dataSourceSizes.add(dataSource, -segment.getSize()); | ||
| } | ||
| synchronized (dataSourceCounts) { | ||
| dataSourceCounts.add(dataSource, -1L); | ||
| } | ||
|
|
||
| try { | ||
| log.info("Attempting to close segment %s", segment.getIdentifier()); | ||
| oldQueryable.close(); | ||
| } | ||
| catch (IOException e) { | ||
| log.makeAlert(e, "Exception closing segment") | ||
| .addData("dataSource", dataSource) | ||
| .addData("segmentId", segment.getIdentifier()) | ||
| .emit(); | ||
| } | ||
| } else { | ||
| log.info( | ||
| "Told to delete a queryable on dataSource[%s] for interval[%s] and version [%s] that I don't have.", | ||
| dataSource, | ||
| segment.getInterval(), | ||
| segment.getVersion() | ||
| ); | ||
| } | ||
| } | ||
| segmentLoader.cleanup(segment); | ||
| } | ||
| } | ||
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.
Doesn't include
partitionNameson purpose?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,
partitionNamesusually includes a lot of partitions.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.
Please leave a comment
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.
Added a comment.