Skip to content

Coordinator: Allow dropping all segments.#7447

Merged
fjy merged 1 commit intoapache:masterfrom
gianm:coord-clean-down-to-zero
Apr 11, 2019
Merged

Coordinator: Allow dropping all segments.#7447
fjy merged 1 commit intoapache:masterfrom
gianm:coord-clean-down-to-zero

Conversation

@gianm
Copy link
Copy Markdown
Contributor

@gianm gianm commented Apr 11, 2019

Removes the coordinator sanity check that prevents it from dropping all
segments. It's useful to get rid of this, since the behavior is
unintuitive for dev/testing clusters where users might regularly want
to drop all their data to get back to a clean slate.

But the sanity check was there for a reason: to prevent a race condition
where the coordinator might drop all segments if it ran before the
first metadata store poll finished. This patch addresses that concern
differently, by allowing methods in MetadataSegmentManager to return
null if a poll has not happened yet, and canceling coordinator runs
in that case.

This patch also makes the "dataSources" reference in
SQLMetadataSegmentManager volatile. I'm not sure why it wasn't volatile
before, but it seems necessary to me: it's not final, and it's dereferenced
from multiple threads without synchronization.

Removes the coordinator sanity check that prevents it from dropping all
segments. It's useful to get rid of this, since the behavior is
unintuitive for dev/testing clusters where users might regularly want
to drop all their data to get back to a clean slate.

But the sanity check was there for a reason: to prevent a race condition
where the coordinator might drop all segments if it ran before the
first metadata store poll finished. This patch addresses that concern
differently, by allowing methods in MetadataSegmentManager to return
null if a poll has not happened yet, and canceling coordinator runs
in that case.

This patch also makes the "dataSources" reference in
SQLMetadataSegmentManager volatile. I'm not sure why it wasn't volatile
before, but it seems necessary to me: it's not final, and it's dereferenced
from multiple threads without synchronization.
Copy link
Copy Markdown
Member

@clintropolis clintropolis left a comment

Choose a reason for hiding this comment

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

I think this seems reasonable 👍

I poked around and it didn't seem like there would be any way to end up with an illegitimate empty list of segments once polling has started...


final Iterable<DataSegment> dataSegments = coordinator.iterateAvailableDataSegments();
if (dataSegments == null) {
log.info("Metadata store not polled yet, canceling this run.");
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 maybe using 'delay' instead of 'cancel', maybe something like "delaying segment coordination" or something of that sort would read better in logs.

@fjy fjy added this to the 0.15.0 milestone Apr 11, 2019
@fjy fjy merged commit a517f8c into apache:master Apr 11, 2019
@leventov
Copy link
Copy Markdown
Member

I think this PR should had a Design Review tag. Such PRs shouldn't be merged 10 hours after opening.

);

dataSources.remove(dataSource);
Optional.ofNullable(dataSources).ifPresent(m -> m.remove(dataSource));
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.

Why do you use Optional.ofNullable(dataSources).ifPresent() instead of

if (dataSources != null) {
  ...
}

in this PR?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

It was because dataSources can become null after being non-null, if stop() is called. Since stop() could be called at any time, dataSources should only be dereferenced one time per method.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The 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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Added in #7452.

.stream()
.map(DruidDataSource::toImmutableDruidDataSource)
.collect(Collectors.toList());
return Optional.ofNullable(dataSources)
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.

Why not just

if (dataSources != null) {
  return dataSources.values().stream().map(...).collect(...);
} else {
 return null;
}

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Just to avoid reading the dataSources reference twice in the same method. (Same reason as https://github.com/apache/incubator-druid/pull/7447/files/39dcd326be350ca6b66e4de884708cf77413c166#r274563782)

ImmutableSet.copyOf(manager.getDataSource("wikipedia").getSegments())
);
Assert.assertEquals(
ImmutableSet.of(segment1, segment2),
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.

Improper formatting

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Oh yeah, I should fix that. Sorry.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Added in #7452.

// 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"
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.

IMO it's better to identify symbols in comments the following ways instead of putting them into double quotes:

  • Adding () to the end of method names
  • Class names start with a capital and have CamelCase, so they don't need any extra identification. Same about variable names with camelCase.
  • Only single-word variable names may need to be identified, but IMO better to use backticks (`) instead of double quotes.

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.
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.

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 of volatile may 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 dataSources with 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.)

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.

Ok, because of this: #7447 (comment) the previous comment is irrelevant, there is actually no reason why the field should be volatile in the current version of the code.

Copy link
Copy Markdown
Contributor Author

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).

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.

Practically, as you noted, it doesn't matter (on x86 platform which Druid targets). Formally, volatile is still not enough to ensure "ever reading non-null" before Java 9 where it was formalized in this document.

@gianm gianm deleted the coord-clean-down-to-zero branch April 11, 2019 18:09
gianm added a commit to gianm/druid that referenced this pull request Apr 11, 2019
@leventov
Copy link
Copy Markdown
Member

leventov commented Apr 11, 2019

The design with SQLMetadataSegmentManager enforcing leadership changes on its callers doesn't feel right to me. It forces all callers to handle this situation, while it doesn't feel to me that they should. For example, REST endpoints in MetadataResource shouldn't be responsible for this, rather, the user is responsible for querying the right Coordinator (the current leader).

I think that upon losing leadership Coordinator should just stop polling database, but still offering the last view of the segments.

fjy pushed a commit that referenced this pull request Apr 12, 2019
@leventov
Copy link
Copy Markdown
Member

@gianm this issue blocks my progress in #7306, I need to know in which direction to resolve conflicts. Please answer the last question.

@gianm
Copy link
Copy Markdown
Contributor Author

gianm commented Apr 16, 2019

@leventov, sorry, what question do you mean? Is it whether or not we should stop polling the DB after losing leadership? (#7447 (comment))

I can only think of one benefit of nullifying the metadata segment cache when losing leadership: it means that next time we gain leadership, we're guaranteed that the segment cache we use is at least as new as the gain of leadership. If we might use an old one, there's a potential for a new leader to use an older view of segments than the old leader. It could be extreme: maybe the new leader, for some reason, hasn't been able to poll for hours or even days, leading to surprising behavior as the cluster 'rolls back' to an earlier state.

This could be mitigated through some code that explicitly makes sure the currently-cached segment metadata is at least as new as the leadership gain, though. I think if you decide it's best to stop nullifying the cache, it'd be good to also add this safety mechanism.

clintropolis pushed a commit to implydata/druid-public that referenced this pull request May 30, 2019
Removes the coordinator sanity check that prevents it from dropping all
segments. It's useful to get rid of this, since the behavior is
unintuitive for dev/testing clusters where users might regularly want
to drop all their data to get back to a clean slate.

But the sanity check was there for a reason: to prevent a race condition
where the coordinator might drop all segments if it ran before the
first metadata store poll finished. This patch addresses that concern
differently, by allowing methods in MetadataSegmentManager to return
null if a poll has not happened yet, and canceling coordinator runs
in that case.

This patch also makes the "dataSources" reference in
SQLMetadataSegmentManager volatile. I'm not sure why it wasn't volatile
before, but it seems necessary to me: it's not final, and it's dereferenced
from multiple threads without synchronization.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants