leventov commented on a change in pull request #7653: Refactor 
SQLMetadataSegmentManager; Change contract of REST methods in 
DataSourcesResource
URL: https://github.com/apache/incubator-druid/pull/7653#discussion_r311698205
 
 

 ##########
 File path: 
server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java
 ##########
 @@ -83,53 +87,140 @@
   private static final EmittingLogger log = new 
EmittingLogger(SQLMetadataSegmentManager.class);
 
   /**
-   * Use to synchronize {@link #start()}, {@link #stop()}, {@link #poll()}, 
and {@link #isStarted()}. These methods
-   * should be synchronized to prevent from being called at the same time if 
two different threads are calling them.
-   * This might be possible if a druid coordinator gets and drops leadership 
repeatedly in quick succession.
+   * Marker interface for objects stored in {@link #latestDatabasePoll}. See 
the comment for that field for details.
    */
-  private final ReentrantReadWriteLock startStopLock = new 
ReentrantReadWriteLock();
+  private interface DatabasePoll
+  {}
+
+  /** Represents periodic {@link #poll}s happening from {@link #exec}. */
+  private static class PeriodicDatabasePoll implements DatabasePoll
+  {
+    /**
+     * This future allows to wait until {@link #dataSourcesSnapshot} is 
initialized in the first {@link #poll()}
+     * happening since {@link #startPollingDatabasePeriodically()} is called 
for the first time, or since the last
+     * visible (in happens-before terms) call to {@link 
#startPollingDatabasePeriodically()} in case of Coordinator's
+     * leadership changes.
+     */
+    final CompletableFuture<Void> firstPollCompletionFuture = new 
CompletableFuture<>();
+  }
+
+  /**
+   * Represents on-demand {@link #poll} initiated at periods of time when 
SqlSegmentsMetadata doesn't poll the database
+   * periodically.
+   */
+  private static class OnDemandDatabasePoll implements DatabasePoll
+  {
+    final long initiationTimeNanos = System.nanoTime();
+    final CompletableFuture<Void> pollCompletionFuture = new 
CompletableFuture<>();
+
+    long nanosElapsedFromInitiation()
+    {
+      return System.nanoTime() - initiationTimeNanos;
+    }
+  }
+
+  /**
+   * Use to synchronize {@link #startPollingDatabasePeriodically}, {@link 
#stopPollingDatabasePeriodically}, {@link
+   * #poll}, and {@link #isPollingDatabasePeriodically}. These methods should 
be synchronized to prevent from being
+   * called at the same time if two different threads are calling them. This 
might be possible if Coordinator gets and
+   * drops leadership repeatedly in quick succession.
+   *
+   * This lock is also used to synchronize {@link #awaitOrPerformDatabasePoll} 
for times when SqlSegmentsMetadata
+   * is not polling the database periodically (in other words, when the 
Coordinator is not the leader).
+   */
+  private final ReentrantReadWriteLock startStopPollLock = new 
ReentrantReadWriteLock();
 
   /**
    * Used to ensure that {@link #poll()} is never run concurrently. It should 
already be so (at least in production
    * code), where {@link #poll()} is called only from the task created in 
{@link #createPollTaskForStartOrder} and is
    * scheduled in a single-threaded {@link #exec}, so this lock is an 
additional safety net in case there are bugs in
    * the code, and for tests, where {@link #poll()} is called from the outside 
code.
    *
-   * Not using {@link #startStopLock}.writeLock() in order to still be able to 
run {@link #poll()} concurrently with
-   * {@link #isStarted()}.
+   * Not using {@link #startStopPollLock}.writeLock() in order to still be 
able to run {@link #poll()} concurrently
+   * with {@link #isPollingDatabasePeriodically()}.
    */
   private final Object pollLock = new Object();
 
   private final ObjectMapper jsonMapper;
-  private final Supplier<MetadataSegmentManagerConfig> config;
+  private final Duration periodicPollDelay;
   private final Supplier<MetadataStorageTablesConfig> dbTables;
   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 dataSources map and
-  // empty overshadowedSegments set).
-  // Note that this is not simply a lazy-initialized variable: it starts off 
as null, and may transition between
-  // null and nonnull multiple times as stop() and start() are called.
-  @Nullable
-  private volatile DataSourcesSnapshot dataSourcesSnapshot = null;
+  /**
+   * This field is made volatile to avoid "ghost secondary reads" that may 
result in NPE, see
+   * https://github.com/code-review-checklists/java-concurrency#safe-local-dcl 
(note that dataSourcesSnapshot resembles
+   * a lazily initialized field). Alternative is to always read the field in a 
snapshot local variable, but it's too
+   * easy to forget to do.
+   *
+   * This field may be updated from {@link #exec}, or from whatever thread 
calling {@link #doOnDemandPoll} via {@link
+   * #awaitOrPerformDatabasePoll()} via one of the public methods of 
SqlSegmentsMetadata.
+   */
+  private volatile @MonotonicNonNull DataSourcesSnapshot dataSourcesSnapshot = 
null;
 
   /**
-   * The number of times this SQLMetadataSegmentManager was started.
+   * The latest {@link DatabasePoll} represent {@link #poll()} calls which 
update {@link #dataSourcesSnapshot}, either
+   * periodically (see {@link PeriodicDatabasePoll}, {@link 
#startPollingDatabasePeriodically}, {@link
+   * #stopPollingDatabasePeriodically}) or "on demand" (see {@link 
OnDemandDatabasePoll}), when one of the methods that
+   * accesses {@link #dataSourcesSnapshot}'s state (such as {@link 
#getImmutableDataSourceWithUsedSegments}) is
+   * called when the Coordinator is not the leader and therefore 
SqlSegmentsMetadata isn't polling the database
 
 Review comment:
   Interesting, thanks. I didn't know about this redirect.
   
   It still seems to me that `SQLMetadataSegmentManager` can be called from 
`MetadataResource` outside of leadership between the moment when this 
Coordinator becomes the leader (`DruidLeaderSelector.getCurrentLeader()` 
returns this Coordinator's id) and `SQLMetadataSegmentManager` gets notified 
about that in `DruidCoordinator.becomeLeader()`. (See also the Javadoc for 
`LeaderLatchListener` which describes another intersting race condition.)
   
   If this is true, I think it's worth reflecting in some comments in code. I 
will think about where to put this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to