-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Coordinator: Allow dropping all segments. #7447
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
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 |
|---|---|---|
|
|
@@ -66,13 +66,15 @@ | |
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Objects; | ||
| import java.util.Optional; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.ScheduledExecutorService; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.locks.ReentrantReadWriteLock; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| /** | ||
| * | ||
| */ | ||
| @ManageLifecycle | ||
| public class SQLMetadataSegmentManager implements MetadataSegmentManager | ||
|
|
@@ -102,9 +104,14 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager | |
| private final Supplier<MetadataStorageTablesConfig> dbTables; | ||
| private final SQLMetadataConnector connector; | ||
|
|
||
| private ConcurrentHashMap<String, DruidDataSource> dataSources = new ConcurrentHashMap<>(); | ||
| // 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) | ||
| @Nullable | ||
| private volatile ConcurrentHashMap<String, DruidDataSource> dataSources = null; | ||
|
|
||
| /** The number of times this SQLMetadataSegmentManager was started. */ | ||
| /** | ||
| * The number of times this SQLMetadataSegmentManager was started. | ||
| */ | ||
| private long startCount = 0; | ||
| /** | ||
| * Equal to the current {@link #startCount} value, if the SQLMetadataSegmentManager is currently started; -1 if | ||
|
|
@@ -200,7 +207,7 @@ public void stop() | |
| return; | ||
| } | ||
|
|
||
| dataSources = new ConcurrentHashMap<>(); | ||
| dataSources = null; | ||
| currentStartOrder = -1; | ||
| exec.shutdownNow(); | ||
| exec = null; | ||
|
|
@@ -325,7 +332,7 @@ public boolean removeDataSource(final String dataSource) | |
| ).bind("dataSource", dataSource).execute() | ||
| ); | ||
|
|
||
| dataSources.remove(dataSource); | ||
| Optional.ofNullable(dataSources).ifPresent(m -> m.remove(dataSource)); | ||
|
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. Why do you use if (dataSources != null) {
...
}in this PR?
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. It was because
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. I'll add a comment about this - the variable looks at first glance like a lazy-initialization, but it's actually something that can transition back and forth between null and nonnull, so it needs to be handled differently.
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 in #7452. |
||
|
|
||
| if (removed == 0) { | ||
| return false; | ||
|
|
@@ -348,18 +355,21 @@ public boolean removeSegment(String dataSourceName, final String segmentId) | |
| // Call iteratePossibleParsingsWithDataSource() outside of dataSources.computeIfPresent() because the former is a | ||
| // potentially expensive operation, while lambda to be passed into computeIfPresent() should preferably run fast. | ||
| List<SegmentId> possibleSegmentIds = SegmentId.iteratePossibleParsingsWithDataSource(dataSourceName, segmentId); | ||
| dataSources.computeIfPresent( | ||
| dataSourceName, | ||
| (dsName, dataSource) -> { | ||
| for (SegmentId possibleSegmentId : possibleSegmentIds) { | ||
| if (dataSource.removeSegment(possibleSegmentId) != null) { | ||
| break; | ||
| } | ||
| } | ||
| // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. | ||
| //noinspection ReturnOfNull | ||
| return dataSource.isEmpty() ? null : dataSource; | ||
| } | ||
| Optional.ofNullable(dataSources).ifPresent( | ||
| m -> | ||
| m.computeIfPresent( | ||
| dataSourceName, | ||
| (dsName, dataSource) -> { | ||
| for (SegmentId possibleSegmentId : possibleSegmentIds) { | ||
| if (dataSource.removeSegment(possibleSegmentId) != null) { | ||
| break; | ||
| } | ||
| } | ||
| // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. | ||
| //noinspection ReturnOfNull | ||
| return dataSource.isEmpty() ? null : dataSource; | ||
| } | ||
| ) | ||
| ); | ||
|
|
||
| return removed; | ||
|
|
@@ -375,14 +385,17 @@ public boolean removeSegment(SegmentId segmentId) | |
| { | ||
| try { | ||
| final boolean removed = removeSegmentFromTable(segmentId.toString()); | ||
| dataSources.computeIfPresent( | ||
| segmentId.getDataSource(), | ||
| (dsName, dataSource) -> { | ||
| dataSource.removeSegment(segmentId); | ||
| // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. | ||
| //noinspection ReturnOfNull | ||
| return dataSource.isEmpty() ? null : dataSource; | ||
| } | ||
| Optional.ofNullable(dataSources).ifPresent( | ||
| m -> | ||
| m.computeIfPresent( | ||
| segmentId.getDataSource(), | ||
| (dsName, dataSource) -> { | ||
| dataSource.removeSegment(segmentId); | ||
| // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. | ||
| //noinspection ReturnOfNull | ||
| return dataSource.isEmpty() ? null : dataSource; | ||
| } | ||
| ) | ||
| ); | ||
| return removed; | ||
| } | ||
|
|
@@ -422,23 +435,37 @@ public boolean isStarted() | |
| @Nullable | ||
| public ImmutableDruidDataSource getDataSource(String dataSourceName) | ||
| { | ||
| final DruidDataSource dataSource = dataSources.get(dataSourceName); | ||
| final DruidDataSource dataSource = Optional.ofNullable(dataSources).map(m -> m.get(dataSourceName)).orElse(null); | ||
| return dataSource == null ? null : dataSource.toImmutableDruidDataSource(); | ||
| } | ||
|
|
||
| @Override | ||
| @Nullable | ||
| public Collection<ImmutableDruidDataSource> getDataSources() | ||
| { | ||
| return dataSources.values() | ||
| .stream() | ||
| .map(DruidDataSource::toImmutableDruidDataSource) | ||
| .collect(Collectors.toList()); | ||
| return Optional.ofNullable(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. Why not just if (dataSources != null) {
return dataSources.values().stream().map(...).collect(...);
} else {
return null;
}
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. Just to avoid reading the |
||
| .map(m -> | ||
| m.values() | ||
| .stream() | ||
| .map(DruidDataSource::toImmutableDruidDataSource) | ||
| .collect(Collectors.toList()) | ||
| ) | ||
| .orElse(null); | ||
| } | ||
|
|
||
| @Override | ||
| @Nullable | ||
| public Iterable<DataSegment> iterateAllSegments() | ||
| { | ||
| return () -> dataSources.values().stream().flatMap(dataSource -> dataSource.getSegments().stream()).iterator(); | ||
| final ConcurrentHashMap<String, DruidDataSource> dataSourcesSnapshot = dataSources; | ||
| if (dataSourcesSnapshot == null) { | ||
| return null; | ||
| } | ||
|
|
||
| return () -> dataSourcesSnapshot.values() | ||
| .stream() | ||
| .flatMap(dataSource -> dataSource.getSegments().stream()) | ||
| .iterator(); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -543,6 +570,7 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE | |
| .addSegmentIfAbsent(segment); | ||
| }); | ||
|
|
||
| // Replace "dataSources" atomically. | ||
| dataSources = newDataSources; | ||
| } | ||
|
|
||
|
|
@@ -557,7 +585,7 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE | |
| */ | ||
| private DataSegment replaceWithExistingSegmentIfPresent(DataSegment segment) | ||
| { | ||
| DruidDataSource dataSource = dataSources.get(segment.getDataSource()); | ||
| DruidDataSource dataSource = Optional.ofNullable(dataSources).map(m -> m.get(segment.getDataSource())).orElse(null); | ||
| if (dataSource == null) { | ||
| return segment; | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -33,6 +33,7 @@ | |
| import java.util.SortedSet; | ||
|
|
||
| /** | ||
| * | ||
| */ | ||
| public class DruidCoordinatorCleanupUnneeded implements DruidCoordinatorHelper | ||
| { | ||
|
|
@@ -45,21 +46,12 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) | |
| Set<DataSegment> availableSegments = params.getAvailableSegments(); | ||
| DruidCluster cluster = params.getDruidCluster(); | ||
|
|
||
| if (availableSegments.isEmpty()) { | ||
| log.info( | ||
| "Found 0 availableSegments, skipping the cleanup of segments from historicals. This is done to prevent " + | ||
| "a race condition in which the coordinator would drop all segments if it started running cleanup before " + | ||
| "it finished polling the metadata storage for available segments for the first time." | ||
| ); | ||
| return params.buildFromExisting().withCoordinatorStats(stats).build(); | ||
| } | ||
|
|
||
| // Drop segments that no longer exist in the available segments configuration, *if* it has been populated. (It might | ||
| // not have been loaded yet since it's filled asynchronously. But it's also filled atomically, so if there are any | ||
| // segments at all, we should have all of them.) | ||
| // Note that if metadata store has no segments, then availableSegments will stay empty and nothing will be dropped. | ||
| // This is done to prevent a race condition in which the coordinator would drop all segments if it started running | ||
| // cleanup before it finished polling the metadata storage for available segments for the first time. | ||
| // Drop segments that no longer exist in the available segments configuration, *if* it has been populated. (It's | ||
| // also filled atomically, so if there are any segments at all, we should have all of them.) | ||
| // | ||
| // Note that if the metadata store has not been polled yet, "getAvailableSegments" would throw an error since | ||
| // "availableSegments" is null. But this won't happen, since the earlier helper "DruidCoordinatorSegmentInfoLoader" | ||
|
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. IMO it's better to identify symbols in comments the following ways instead of putting them into double quotes:
|
||
| // would have canceled the run. | ||
| for (SortedSet<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) { | ||
| for (ServerHolder serverHolder : serverHolders) { | ||
| ImmutableDruidServer server = serverHolder.getServer(); | ||
|
|
||
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.
This comment doesn't explain why does the field need to be
volatile. The underlying reason is that the field is effectively a lazily initialized field, and the absence ofvolatilemay lead to NPE unless the rest of the code always reads the field to local variables before using, that is too much of a burden for developers: https://github.com/code-review-checklists/java-concurrency#safe-local-dcl(Actually, as you translated all code to monadic Optional use of
dataSourceswith a single read, it does not need to be volatile, but I would say that those monadic Optional chains are worse than simple if-else.)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.
Ok, because of this: #7447 (comment) the previous comment is irrelevant, there is actually no reason why the field should be
volatilein the current version of the code.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.
Are you saying it is fine to have a field that is written from one thread, and read from another, with no synchronization or volatile marker, as long as each reader reads it into a local variable first? My understanding of the JMM is that in this case there's no happens-before relationship established, and all bets are off - readers have no guarantees around ever reading anything nonnull (although in practice they probably will, but that's not something you'd want to depend on).
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.
Practically, as you noted, it doesn't matter (on x86 platform which Druid targets). Formally,
volatileis still not enough to ensure "ever reading non-null" before Java 9 where it was formalized in this document.