Skip to content

Commit

Permalink
Introduce global checkpoint listeners (#32696)
Browse files Browse the repository at this point in the history
This commit introduces the ability for global checkpoint listeners to be
registered at the shard level. These listeners are notified when the
global checkpoint is updated, and also when the shard closes. To
encapsulate these listeners, we introduce a shard-level component that
handles synchronization of notification and modifications to the
collection of listeners.
  • Loading branch information
jasontedor committed Aug 15, 2018
1 parent da9e984 commit 3b047ae
Show file tree
Hide file tree
Showing 4 changed files with 658 additions and 4 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,166 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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 org.elasticsearch.index.shard;

import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;

import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.Executor;

import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED;
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;

/**
* Represents a collection of global checkpoint listeners. This collection can be added to, and all listeners present at the time of an
* update will be notified together. All listeners will be notified when the shard is closed.
*/
public class GlobalCheckpointListeners implements Closeable {

/**
* A global checkpoint listener consisting of a callback that is notified when the global checkpoint is updated or the shard is closed.
*/
@FunctionalInterface
public interface GlobalCheckpointListener {
/**
* Callback when the global checkpoint is updated or the shard is closed. If the shard is closed, the value of the global checkpoint
* will be set to {@link org.elasticsearch.index.seqno.SequenceNumbers#UNASSIGNED_SEQ_NO} and the exception will be non-null. If the
* global checkpoint is updated, the exception will be null.
*
* @param globalCheckpoint the updated global checkpoint
* @param e if non-null, the shard is closed
*/
void accept(long globalCheckpoint, IndexShardClosedException e);
}

// guarded by this
private boolean closed;
private volatile List<GlobalCheckpointListener> listeners;
private long lastKnownGlobalCheckpoint = UNASSIGNED_SEQ_NO;

private final ShardId shardId;
private final Executor executor;
private final Logger logger;

/**
* Construct a global checkpoint listeners collection.
*
* @param shardId the shard ID on which global checkpoint updates can be listened to
* @param executor the executor for listener notifications
* @param logger a shard-level logger
*/
GlobalCheckpointListeners(
final ShardId shardId,
final Executor executor,
final Logger logger) {
this.shardId = Objects.requireNonNull(shardId);
this.executor = Objects.requireNonNull(executor);
this.logger = Objects.requireNonNull(logger);
}

/**
* Add a global checkpoint listener. If the global checkpoint is above the current global checkpoint known to the listener then the
* listener will be asynchronously notified on the executor used to construct this collection of global checkpoint listeners. If the
* shard is closed then the listener will be asynchronously notified on the executor used to construct this collection of global
* checkpoint listeners. The listener will only be notified of at most one event, either the global checkpoint is updated or the shard
* is closed. A listener must re-register after one of these events to receive subsequent events.
*
* @param currentGlobalCheckpoint the current global checkpoint known to the listener
* @param listener the listener
*/
synchronized void add(final long currentGlobalCheckpoint, final GlobalCheckpointListener listener) {
if (closed) {
executor.execute(() -> notifyListener(listener, UNASSIGNED_SEQ_NO, new IndexShardClosedException(shardId)));
return;
}
if (lastKnownGlobalCheckpoint > currentGlobalCheckpoint) {
// notify directly
executor.execute(() -> notifyListener(listener, lastKnownGlobalCheckpoint, null));
return;
} else {
if (listeners == null) {
listeners = new ArrayList<>();
}
listeners.add(listener);
}
}

@Override
public synchronized void close() throws IOException {
closed = true;
notifyListeners(UNASSIGNED_SEQ_NO, new IndexShardClosedException(shardId));
}

synchronized int pendingListeners() {
return listeners == null ? 0 : listeners.size();
}

/**
* Invoke to notify all registered listeners of an updated global checkpoint.
*
* @param globalCheckpoint the updated global checkpoint
*/
synchronized void globalCheckpointUpdated(final long globalCheckpoint) {
assert globalCheckpoint >= NO_OPS_PERFORMED;
assert globalCheckpoint > lastKnownGlobalCheckpoint
: "updated global checkpoint [" + globalCheckpoint + "]"
+ " is not more than the last known global checkpoint [" + lastKnownGlobalCheckpoint + "]";
lastKnownGlobalCheckpoint = globalCheckpoint;
notifyListeners(globalCheckpoint, null);
}

private void notifyListeners(final long globalCheckpoint, final IndexShardClosedException e) {
assert Thread.holdsLock(this);
assert (globalCheckpoint == UNASSIGNED_SEQ_NO && e != null) || (globalCheckpoint >= NO_OPS_PERFORMED && e == null);
if (listeners != null) {
// capture the current listeners
final List<GlobalCheckpointListener> currentListeners = listeners;
listeners = null;
if (currentListeners != null) {
executor.execute(() -> {
for (final GlobalCheckpointListener listener : currentListeners) {
notifyListener(listener, globalCheckpoint, e);
}
});
}
}
}

private void notifyListener(final GlobalCheckpointListener listener, final long globalCheckpoint, final IndexShardClosedException e) {
try {
listener.accept(globalCheckpoint, e);
} catch (final Exception caught) {
if (globalCheckpoint != UNASSIGNED_SEQ_NO) {
logger.warn(
new ParameterizedMessage(
"error notifying global checkpoint listener of updated global checkpoint [{}]",
globalCheckpoint),
caught);
} else {
logger.warn("error notifying global checkpoint listener of closed shard", caught);
}
}
}

}
26 changes: 22 additions & 4 deletions server/src/main/java/org/elasticsearch/index/shard/IndexShard.java
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,8 @@
import java.util.stream.StreamSupport;

import static org.elasticsearch.index.mapper.SourceToParse.source;
import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED;
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;

public class IndexShard extends AbstractIndexShardComponent implements IndicesClusterStateService.Shard {

Expand Down Expand Up @@ -190,6 +192,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl

private final SearchOperationListener searchOperationListener;

private final GlobalCheckpointListeners globalCheckpointListeners;
private final ReplicationTracker replicationTracker;

protected volatile ShardRouting shardRouting;
Expand Down Expand Up @@ -296,8 +299,10 @@ public IndexShard(
this.checkIndexOnStartup = indexSettings.getValue(IndexSettings.INDEX_CHECK_ON_STARTUP);
this.translogConfig = new TranslogConfig(shardId, shardPath().resolveTranslog(), indexSettings, bigArrays);
final String aId = shardRouting.allocationId().getId();
this.globalCheckpointListeners = new GlobalCheckpointListeners(shardId, threadPool.executor(ThreadPool.Names.LISTENER), logger);
this.replicationTracker =
new ReplicationTracker(shardId, aId, indexSettings, SequenceNumbers.UNASSIGNED_SEQ_NO, globalCheckpoint -> {});
new ReplicationTracker(shardId, aId, indexSettings, UNASSIGNED_SEQ_NO, globalCheckpointListeners::globalCheckpointUpdated);

// the query cache is a node-level thing, however we want the most popular filters
// to be computed on a per-shard basis
if (IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.get(settings)) {
Expand Down Expand Up @@ -1223,7 +1228,7 @@ public void close(String reason, boolean flushEngine) throws IOException {
} finally {
// playing safe here and close the engine even if the above succeeds - close can be called multiple times
// Also closing refreshListeners to prevent us from accumulating any more listeners
IOUtils.close(engine, refreshListeners);
IOUtils.close(engine, globalCheckpointListeners, refreshListeners);
indexShardOperationPermits.close();
}
}
Expand Down Expand Up @@ -1764,6 +1769,19 @@ public void updateGlobalCheckpointForShard(final String allocationId, final long
replicationTracker.updateGlobalCheckpointForShard(allocationId, globalCheckpoint);
}

/**
* Add a global checkpoint listener. If the global checkpoint is above the current global checkpoint known to the listener then the
* listener will fire immediately on the calling thread.
*
* @param currentGlobalCheckpoint the current global checkpoint known to the listener
* @param listener the listener
*/
public void addGlobalCheckpointListener(
final long currentGlobalCheckpoint,
final GlobalCheckpointListeners.GlobalCheckpointListener listener) {
this.globalCheckpointListeners.add(currentGlobalCheckpoint, listener);
}

/**
* Waits for all operations up to the provided sequence number to complete.
*
Expand Down Expand Up @@ -2308,8 +2326,8 @@ public void acquireReplicaOperationPermit(final long opPrimaryTerm, final long g
updateGlobalCheckpointOnReplica(globalCheckpoint, "primary term transition");
final long currentGlobalCheckpoint = getGlobalCheckpoint();
final long localCheckpoint;
if (currentGlobalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO) {
localCheckpoint = SequenceNumbers.NO_OPS_PERFORMED;
if (currentGlobalCheckpoint == UNASSIGNED_SEQ_NO) {
localCheckpoint = NO_OPS_PERFORMED;
} else {
localCheckpoint = currentGlobalCheckpoint;
}
Expand Down
Loading

0 comments on commit 3b047ae

Please sign in to comment.