Refactor AutoFollowCoordinator to track leader indices per remote cluster (#36031)

and replaced poll interval setting with a hardcoded poll interval.
The hard coded interval will be removed in a follow up change to make
use of cluster state API's wait_for_metatdata_version.

Before the auto following was bootstrapped from thread pool scheduler,
but now auto followers for new remote clusters are bootstrapped when
a new cluster state is published.

Originates from #35895
Relates to #33007
This commit is contained in:
Martijn van Groningen 2018-12-05 13:39:14 +01:00 committed by GitHub
parent 60e45cd81d
commit a264cb6ddb
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 319 additions and 182 deletions

View File

@ -46,6 +46,7 @@ followClusterTestCluster {
numNodes = 1 numNodes = 1
clusterName = 'follow-cluster' clusterName = 'follow-cluster'
setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.license.self_generated.type', 'trial'
setting 'cluster.remote.leader_cluster.seeds', "\"${-> leaderClusterTest.nodes.get(0).transportUri()}\""
setting 'cluster.remote.middle_cluster.seeds', "\"${-> middleClusterTest.nodes.get(0).transportUri()}\"" setting 'cluster.remote.middle_cluster.seeds', "\"${-> middleClusterTest.nodes.get(0).transportUri()}\""
setting 'node.name', 'follow' setting 'node.name', 'follow'
} }

View File

@ -0,0 +1,69 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ccr;
import org.elasticsearch.client.Request;
import org.elasticsearch.client.RestClient;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import java.util.Map;
import static org.hamcrest.Matchers.equalTo;
public class AutoFollowIT extends ESCCRRestTestCase {
public void testAutoFollowPatterns() throws Exception {
if ("follow".equals(targetCluster) == false) {
return;
}
Request putPatternRequest = new Request("PUT", "/_ccr/auto_follow/leader_cluster_pattern");
putPatternRequest.setJsonEntity("{\"leader_index_patterns\": [\"logs-*\"], \"remote_cluster\": \"leader_cluster\"}");
assertOK(client().performRequest(putPatternRequest));
putPatternRequest = new Request("PUT", "/_ccr/auto_follow/middle_cluster_pattern");
putPatternRequest.setJsonEntity("{\"leader_index_patterns\": [\"logs-*\"], \"remote_cluster\": \"middle_cluster\"}");
assertOK(client().performRequest(putPatternRequest));
try (RestClient leaderClient = buildLeaderClient()) {
Settings settings = Settings.builder()
.put("index.soft_deletes.enabled", true)
.build();
Request request = new Request("PUT", "/logs-20190101");
request.setJsonEntity("{\"settings\": " + Strings.toString(settings) +
", \"mappings\": {\"_doc\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}} }");
assertOK(leaderClient.performRequest(request));
for (int i = 0; i < 5; i++) {
String id = Integer.toString(i);
index(leaderClient, "logs-20190101", id, "field", i, "filtered_field", "true");
}
}
try (RestClient middleClient = buildMiddleClient()) {
Settings settings = Settings.builder()
.put("index.soft_deletes.enabled", true)
.build();
Request request = new Request("PUT", "/logs-20200101");
request.setJsonEntity("{\"settings\": " + Strings.toString(settings) +
", \"mappings\": {\"_doc\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}} }");
assertOK(middleClient.performRequest(request));
for (int i = 0; i < 5; i++) {
String id = Integer.toString(i);
index(middleClient, "logs-20200101", id, "field", i, "filtered_field", "true");
}
}
assertBusy(() -> {
Request statsRequest = new Request("GET", "/_ccr/stats");
Map<?, ?> response = toMap(client().performRequest(statsRequest));
Map<?, ?> autoFollowStats = (Map<?, ?>) response.get("auto_follow_stats");
assertThat(autoFollowStats.get("number_of_successful_follow_indices"), equalTo(2));
ensureYellow("logs-20190101");
ensureYellow("logs-20200101");
verifyDocuments("logs-20190101", 5, "filtered_field:true");
verifyDocuments("logs-20200101", 5, "filtered_field:true");
});
}
}

View File

@ -155,7 +155,7 @@ public class Ccr extends Plugin implements ActionPlugin, PersistentTaskPlugin, E
return Arrays.asList( return Arrays.asList(
ccrLicenseChecker, ccrLicenseChecker,
new AutoFollowCoordinator(settings, client, threadPool, clusterService, ccrLicenseChecker) new AutoFollowCoordinator(client, threadPool, clusterService, ccrLicenseChecker)
); );
} }

View File

@ -7,7 +7,6 @@ package org.elasticsearch.xpack.ccr;
import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.XPackSettings;
import java.util.Arrays; import java.util.Arrays;
@ -29,12 +28,6 @@ public final class CcrSettings {
public static final Setting<Boolean> CCR_FOLLOWING_INDEX_SETTING = public static final Setting<Boolean> CCR_FOLLOWING_INDEX_SETTING =
Setting.boolSetting("index.xpack.ccr.following_index", false, Property.IndexScope, Property.InternalIndex); Setting.boolSetting("index.xpack.ccr.following_index", false, Property.IndexScope, Property.InternalIndex);
/**
* Setting for controlling the interval in between polling leader clusters to check whether there are indices to follow
*/
public static final Setting<TimeValue> CCR_AUTO_FOLLOW_POLL_INTERVAL =
Setting.timeSetting("xpack.ccr.auto_follow.poll_interval", TimeValue.timeValueMillis(2500), Property.NodeScope);
/** /**
* The settings defined by CCR. * The settings defined by CCR.
* *
@ -43,8 +36,7 @@ public final class CcrSettings {
static List<Setting<?>> getSettings() { static List<Setting<?>> getSettings() {
return Arrays.asList( return Arrays.asList(
XPackSettings.CCR_ENABLED_SETTING, XPackSettings.CCR_ENABLED_SETTING,
CCR_FOLLOWING_INDEX_SETTING, CCR_FOLLOWING_INDEX_SETTING);
CCR_AUTO_FOLLOW_POLL_INTERVAL);
} }
} }

View File

@ -15,14 +15,14 @@ import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.CopyOnWriteHashMap;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.util.concurrent.CountDown;
@ -31,7 +31,6 @@ import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.license.LicenseUtils; import org.elasticsearch.license.LicenseUtils;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.ccr.CcrLicenseChecker; import org.elasticsearch.xpack.ccr.CcrLicenseChecker;
import org.elasticsearch.xpack.ccr.CcrSettings;
import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata;
import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern;
import org.elasticsearch.xpack.core.ccr.AutoFollowStats; import org.elasticsearch.xpack.core.ccr.AutoFollowStats;
@ -45,28 +44,29 @@ import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.Set;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.function.BiConsumer; import java.util.function.BiConsumer;
import java.util.function.Consumer; import java.util.function.Consumer;
import java.util.function.Function; import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors; import java.util.stream.Collectors;
/** /**
* A component that runs only on the elected master node and follows leader indices automatically * A component that runs only on the elected master node and follows leader indices automatically
* if they match with a auto follow pattern that is defined in {@link AutoFollowMetadata}. * if they match with a auto follow pattern that is defined in {@link AutoFollowMetadata}.
*/ */
public class AutoFollowCoordinator implements ClusterStateApplier { public class AutoFollowCoordinator implements ClusterStateListener {
private static final Logger LOGGER = LogManager.getLogger(AutoFollowCoordinator.class); private static final Logger LOGGER = LogManager.getLogger(AutoFollowCoordinator.class);
private static final int MAX_AUTO_FOLLOW_ERRORS = 256; private static final int MAX_AUTO_FOLLOW_ERRORS = 256;
private final Client client; private final Client client;
private final TimeValue pollInterval;
private final ThreadPool threadPool; private final ThreadPool threadPool;
private final ClusterService clusterService; private final ClusterService clusterService;
private final CcrLicenseChecker ccrLicenseChecker; private final CcrLicenseChecker ccrLicenseChecker;
private volatile boolean localNodeMaster = false; private volatile Map<String, AutoFollower> autoFollowers = Collections.emptyMap();
// The following fields are read and updated under a lock: // The following fields are read and updated under a lock:
private long numberOfSuccessfulIndicesAutoFollowed = 0; private long numberOfSuccessfulIndicesAutoFollowed = 0;
@ -75,7 +75,6 @@ public class AutoFollowCoordinator implements ClusterStateApplier {
private final LinkedHashMap<String, ElasticsearchException> recentAutoFollowErrors; private final LinkedHashMap<String, ElasticsearchException> recentAutoFollowErrors;
public AutoFollowCoordinator( public AutoFollowCoordinator(
Settings settings,
Client client, Client client,
ThreadPool threadPool, ThreadPool threadPool,
ClusterService clusterService, ClusterService clusterService,
@ -84,10 +83,7 @@ public class AutoFollowCoordinator implements ClusterStateApplier {
this.threadPool = threadPool; this.threadPool = threadPool;
this.clusterService = clusterService; this.clusterService = clusterService;
this.ccrLicenseChecker = Objects.requireNonNull(ccrLicenseChecker, "ccrLicenseChecker"); this.ccrLicenseChecker = Objects.requireNonNull(ccrLicenseChecker, "ccrLicenseChecker");
clusterService.addListener(this);
this.pollInterval = CcrSettings.CCR_AUTO_FOLLOW_POLL_INTERVAL.get(settings);
clusterService.addStateApplier(this);
this.recentAutoFollowErrors = new LinkedHashMap<String, ElasticsearchException>() { this.recentAutoFollowErrors = new LinkedHashMap<String, ElasticsearchException>() {
@Override @Override
protected boolean removeEldestEntry(final Map.Entry<String, ElasticsearchException> eldest) { protected boolean removeEldestEntry(final Map.Entry<String, ElasticsearchException> eldest) {
@ -130,34 +126,27 @@ public class AutoFollowCoordinator implements ClusterStateApplier {
} }
} }
private void doAutoFollow() { void updateAutoFollowers(ClusterState followerClusterState) {
if (localNodeMaster == false) {
return;
}
ClusterState followerClusterState = clusterService.state();
AutoFollowMetadata autoFollowMetadata = followerClusterState.getMetaData().custom(AutoFollowMetadata.TYPE); AutoFollowMetadata autoFollowMetadata = followerClusterState.getMetaData().custom(AutoFollowMetadata.TYPE);
if (autoFollowMetadata == null) { if (autoFollowMetadata == null) {
threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow);
return;
}
if (autoFollowMetadata.getPatterns().isEmpty()) {
threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow);
return; return;
} }
if (ccrLicenseChecker.isCcrAllowed() == false) { if (ccrLicenseChecker.isCcrAllowed() == false) {
// TODO: set non-compliant status on auto-follow coordination that can be viewed via a stats API // TODO: set non-compliant status on auto-follow coordination that can be viewed via a stats API
LOGGER.warn("skipping auto-follower coordination", LicenseUtils.newComplianceException("ccr")); LOGGER.warn("skipping auto-follower coordination", LicenseUtils.newComplianceException("ccr"));
threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow);
return; return;
} }
Consumer<List<AutoFollowResult>> handler = results -> { final CopyOnWriteHashMap<String, AutoFollower> autoFollowers = CopyOnWriteHashMap.copyOf(this.autoFollowers);
updateStats(results); Set<String> newRemoteClusters = autoFollowMetadata.getPatterns().entrySet().stream()
threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow); .map(entry -> entry.getValue().getRemoteCluster())
}; .filter(remoteCluster -> autoFollowers.containsKey(remoteCluster) == false)
AutoFollower operation = new AutoFollower(handler, followerClusterState) { .collect(Collectors.toSet());
Map<String, AutoFollower> newAutoFollowers = new HashMap<>(newRemoteClusters.size());
for (String remoteCluster : newRemoteClusters) {
AutoFollower autoFollower = new AutoFollower(remoteCluster, threadPool, this::updateStats, clusterService::state) {
@Override @Override
void getLeaderClusterState(final String remoteCluster, void getLeaderClusterState(final String remoteCluster,
@ -211,70 +200,115 @@ public class AutoFollowCoordinator implements ClusterStateApplier {
} }
}; };
operation.autoFollowIndices(); newAutoFollowers.put(remoteCluster, autoFollower);
autoFollower.autoFollowIndices();
}
List<String> removedRemoteClusters = new ArrayList<>();
for (String remoteCluster : autoFollowers.keySet()) {
boolean exist = autoFollowMetadata.getPatterns().values().stream()
.anyMatch(pattern -> pattern.getRemoteCluster().equals(remoteCluster));
if (exist == false) {
removedRemoteClusters.add(remoteCluster);
}
}
this.autoFollowers = autoFollowers
.copyAndPutAll(newAutoFollowers)
.copyAndRemoveAll(removedRemoteClusters);
} }
@Override @Override
public void applyClusterState(ClusterChangedEvent event) { public void clusterChanged(ClusterChangedEvent event) {
final boolean beforeLocalMasterNode = localNodeMaster; if (event.localNodeMaster()) {
localNodeMaster = event.localNodeMaster(); updateAutoFollowers(event.state());
if (beforeLocalMasterNode == false && localNodeMaster) {
threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow);
} }
} }
/**
* Each auto follower independently monitors a remote cluster for new leader indices that should be auto followed.
* The reason that this should happen independently, is that when auto followers start to make use of cluster state
* API's 'wait_for_metadata_version' feature, it may take sometime before a remote cluster responds with a new
* cluster state or times out. Other auto follow patterns for different remote clusters are then forced to wait,
* which can cause new follower indices to unnecessarily start with a large backlog of operations that need to be
* replicated.
*/
abstract static class AutoFollower { abstract static class AutoFollower {
private final Consumer<List<AutoFollowResult>> handler; private final String remoteCluster;
private final ClusterState followerClusterState; private final ThreadPool threadPool;
private final AutoFollowMetadata autoFollowMetadata; private final Consumer<List<AutoFollowResult>> statsUpdater;
private final Supplier<ClusterState> followerClusterStateSupplier;
private final CountDown autoFollowPatternsCountDown; private volatile CountDown autoFollowPatternsCountDown;
private final AtomicArray<AutoFollowResult> autoFollowResults; private volatile AtomicArray<AutoFollowResult> autoFollowResults;
AutoFollower(final Consumer<List<AutoFollowResult>> handler, final ClusterState followerClusterState) { AutoFollower(final String remoteCluster,
this.handler = handler; final ThreadPool threadPool,
this.followerClusterState = followerClusterState; final Consumer<List<AutoFollowResult>> statsUpdater,
this.autoFollowMetadata = followerClusterState.getMetaData().custom(AutoFollowMetadata.TYPE); final Supplier<ClusterState> followerClusterStateSupplier) {
this.autoFollowPatternsCountDown = new CountDown(autoFollowMetadata.getPatterns().size()); this.remoteCluster = remoteCluster;
this.autoFollowResults = new AtomicArray<>(autoFollowMetadata.getPatterns().size()); this.threadPool = threadPool;
this.statsUpdater = statsUpdater;
this.followerClusterStateSupplier = followerClusterStateSupplier;
} }
void autoFollowIndices() { void autoFollowIndices() {
int i = 0; final ClusterState followerClusterState = followerClusterStateSupplier.get();
for (Map.Entry<String, AutoFollowPattern> entry : autoFollowMetadata.getPatterns().entrySet()) { final AutoFollowMetadata autoFollowMetadata = followerClusterState.metaData().custom(AutoFollowMetadata.TYPE);
final int slot = i; if (autoFollowMetadata == null) {
final String autoFollowPattenName = entry.getKey(); LOGGER.info("AutoFollower for cluster [{}] has stopped, because there is no autofollow metadata", remoteCluster);
final AutoFollowPattern autoFollowPattern = entry.getValue(); return;
final String remoteCluster = autoFollowPattern.getRemoteCluster(); }
final List<String> patterns = autoFollowMetadata.getPatterns().entrySet().stream()
.filter(entry -> entry.getValue().getRemoteCluster().equals(remoteCluster))
.map(Map.Entry::getKey)
.collect(Collectors.toList());
if (patterns.isEmpty()) {
LOGGER.info("AutoFollower for cluster [{}] has stopped, because there are no more patterns", remoteCluster);
return;
}
this.autoFollowPatternsCountDown = new CountDown(patterns.size());
this.autoFollowResults = new AtomicArray<>(patterns.size());
Map<String, String> headers = autoFollowMetadata.getHeaders().get(autoFollowPattenName);
getLeaderClusterState(remoteCluster, (leaderClusterState, e) -> { getLeaderClusterState(remoteCluster, (leaderClusterState, e) -> {
if (leaderClusterState != null) { if (leaderClusterState != null) {
assert e == null; assert e == null;
final List<String> followedIndices = autoFollowMetadata.getFollowedLeaderIndexUUIDs().get(autoFollowPattenName);
final List<Index> leaderIndicesToFollow = getLeaderIndicesToFollow(remoteCluster, autoFollowPattern, int i = 0;
leaderClusterState, followerClusterState, followedIndices); for (String autoFollowPatternName : patterns) {
final int slot = i;
AutoFollowPattern autoFollowPattern = autoFollowMetadata.getPatterns().get(autoFollowPatternName);
Map<String, String> headers = autoFollowMetadata.getHeaders().get(autoFollowPatternName);
List<String> followedIndices = autoFollowMetadata.getFollowedLeaderIndexUUIDs().get(autoFollowPatternName);
final List<Index> leaderIndicesToFollow = getLeaderIndicesToFollow(autoFollowPattern, leaderClusterState,
followerClusterState, followedIndices);
if (leaderIndicesToFollow.isEmpty()) { if (leaderIndicesToFollow.isEmpty()) {
finalise(slot, new AutoFollowResult(autoFollowPattenName)); finalise(slot, new AutoFollowResult(autoFollowPatternName));
} else { } else {
List<Tuple<String, AutoFollowPattern>> patternsForTheSameLeaderCluster = autoFollowMetadata.getPatterns() List<Tuple<String, AutoFollowPattern>> patternsForTheSameLeaderCluster = autoFollowMetadata.getPatterns()
.entrySet().stream() .entrySet().stream()
.filter(item -> autoFollowPattenName.equals(item.getKey()) == false) .filter(item -> autoFollowPatternName.equals(item.getKey()) == false)
.filter(item -> remoteCluster.equals(item.getValue().getRemoteCluster())) .filter(item -> remoteCluster.equals(item.getValue().getRemoteCluster()))
.map(item -> new Tuple<>(item.getKey(), item.getValue())) .map(item -> new Tuple<>(item.getKey(), item.getValue()))
.collect(Collectors.toList()); .collect(Collectors.toList());
Consumer<AutoFollowResult> resultHandler = result -> finalise(slot, result); Consumer<AutoFollowResult> resultHandler = result -> finalise(slot, result);
checkAutoFollowPattern(autoFollowPattenName, remoteCluster, autoFollowPattern, leaderIndicesToFollow, headers, checkAutoFollowPattern(autoFollowPatternName, remoteCluster, autoFollowPattern, leaderIndicesToFollow, headers,
patternsForTheSameLeaderCluster, resultHandler); patternsForTheSameLeaderCluster, resultHandler);
} }
} else {
finalise(slot, new AutoFollowResult(autoFollowPattenName, e));
}
});
i++; i++;
} }
} else {
List<AutoFollowResult> results = new ArrayList<>(patterns.size());
for (String autoFollowPatternName : patterns) {
results.add(new AutoFollowResult(autoFollowPatternName, e));
}
statsUpdater.accept(results);
}
});
} }
private void checkAutoFollowPattern(String autoFollowPattenName, private void checkAutoFollowPattern(String autoFollowPattenName,
@ -357,12 +391,13 @@ public class AutoFollowCoordinator implements ClusterStateApplier {
assert autoFollowResults.get(slot) == null; assert autoFollowResults.get(slot) == null;
autoFollowResults.set(slot, result); autoFollowResults.set(slot, result);
if (autoFollowPatternsCountDown.countDown()) { if (autoFollowPatternsCountDown.countDown()) {
handler.accept(autoFollowResults.asList()); statsUpdater.accept(autoFollowResults.asList());
// TODO: Remove scheduling here with using cluster state API's waitForMetadataVersion:
threadPool.schedule(TimeValue.timeValueMillis(2500), ThreadPool.Names.GENERIC, this::autoFollowIndices);
} }
} }
static List<Index> getLeaderIndicesToFollow(String remoteCluster, static List<Index> getLeaderIndicesToFollow(AutoFollowPattern autoFollowPattern,
AutoFollowPattern autoFollowPattern,
ClusterState leaderClusterState, ClusterState leaderClusterState,
ClusterState followerClusterState, ClusterState followerClusterState,
List<String> followedIndexUUIDs) { List<String> followedIndexUUIDs) {

View File

@ -140,7 +140,24 @@ public class CcrLicenseIT extends CcrSingleNodeTestCase {
} }
public void testAutoFollowCoordinatorLogsSkippingAutoFollowCoordinationWithNonCompliantLicense() throws Exception { public void testAutoFollowCoordinatorLogsSkippingAutoFollowCoordinationWithNonCompliantLicense() throws Exception {
// Update the cluster state so that we have auto follow patterns and verify that we log a warning in case of incompatible license: final Logger logger = LogManager.getLogger(AutoFollowCoordinator.class);
final MockLogAppender appender = new MockLogAppender();
appender.start();
appender.addExpectation(
new MockLogAppender.ExceptionSeenEventExpectation(
getTestName(),
logger.getName(),
Level.WARN,
"skipping auto-follower coordination",
ElasticsearchSecurityException.class,
"current license is non-compliant for [ccr]"));
try {
// Need to add mock log appender before submitting CS update, otherwise we miss the expected log:
// (Auto followers for new remote clusters are bootstrapped when a new cluster state is published)
Loggers.addAppender(logger, appender);
// Update the cluster state so that we have auto follow patterns and verify that we log a warning
// in case of incompatible license:
CountDownLatch latch = new CountDownLatch(1); CountDownLatch latch = new CountDownLatch(1);
ClusterService clusterService = getInstanceFromNode(ClusterService.class); ClusterService clusterService = getInstanceFromNode(ClusterService.class);
clusterService.submitStateUpdateTask("test-add-auto-follow-pattern", new ClusterStateUpdateTask() { clusterService.submitStateUpdateTask("test-add-auto-follow-pattern", new ClusterStateUpdateTask() {
@ -173,27 +190,14 @@ public class CcrLicenseIT extends CcrSingleNodeTestCase {
} }
}); });
latch.await(); latch.await();
appender.assertAllExpectationsMatched();
final Logger logger = LogManager.getLogger(AutoFollowCoordinator.class);
final MockLogAppender appender = new MockLogAppender();
appender.start();
appender.addExpectation(
new MockLogAppender.ExceptionSeenEventExpectation(
getTestName(),
logger.getName(),
Level.WARN,
"skipping auto-follower coordination",
ElasticsearchSecurityException.class,
"current license is non-compliant for [ccr]"));
Loggers.addAppender(logger, appender);
try {
assertBusy(appender::assertAllExpectationsMatched);
} finally { } finally {
Loggers.removeAppender(logger, appender); Loggers.removeAppender(logger, appender);
appender.stop(); appender.stop();
} }
} }
private void assertNonCompliantLicense(final Exception e) { private void assertNonCompliantLicense(final Exception e) {
assertThat(e, instanceOf(ElasticsearchSecurityException.class)); assertThat(e, instanceOf(ElasticsearchSecurityException.class));
assertThat(e.getMessage(), equalTo("current license is non-compliant for [ccr]")); assertThat(e.getMessage(), equalTo("current license is non-compliant for [ccr]"));

View File

@ -22,6 +22,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.ccr.CcrLicenseChecker; import org.elasticsearch.xpack.ccr.CcrLicenseChecker;
import org.elasticsearch.xpack.ccr.action.AutoFollowCoordinator.AutoFollower; import org.elasticsearch.xpack.ccr.action.AutoFollowCoordinator.AutoFollower;
import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata;
@ -34,11 +35,13 @@ import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.function.BiConsumer; import java.util.function.BiConsumer;
import java.util.function.Consumer; import java.util.function.Consumer;
import java.util.function.Function; import java.util.function.Function;
import java.util.function.Supplier;
import static org.elasticsearch.xpack.ccr.action.AutoFollowCoordinator.AutoFollower.recordLeaderIndexAsFollowFunction; import static org.elasticsearch.xpack.ccr.action.AutoFollowCoordinator.AutoFollower.recordLeaderIndexAsFollowFunction;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
@ -46,7 +49,9 @@ import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.sameInstance; import static org.hamcrest.Matchers.sameInstance;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
@ -54,6 +59,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
public void testAutoFollower() { public void testAutoFollower() {
Client client = mock(Client.class); Client client = mock(Client.class);
ThreadPool threadPool = mockThreadPool();
when(client.getRemoteClusterClient(anyString())).thenReturn(client); when(client.getRemoteClusterClient(anyString())).thenReturn(client);
ClusterState leaderState = createRemoteClusterState("logs-20190101"); ClusterState leaderState = createRemoteClusterState("logs-20190101");
@ -83,7 +89,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
assertThat(entries.get(0).getKey().getName(), equalTo("logs-20190101")); assertThat(entries.get(0).getKey().getName(), equalTo("logs-20190101"));
assertThat(entries.get(0).getValue(), nullValue()); assertThat(entries.get(0).getValue(), nullValue());
}; };
AutoFollower autoFollower = new AutoFollower(handler, currentState) { AutoFollower autoFollower = new AutoFollower("remote", threadPool, handler, followerClusterStateSupplier(currentState)) {
@Override @Override
void getLeaderClusterState(String remoteCluster, void getLeaderClusterState(String remoteCluster,
BiConsumer<ClusterState, Exception> handler) { BiConsumer<ClusterState, Exception> handler) {
@ -119,6 +125,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
public void testAutoFollowerClusterStateApiFailure() { public void testAutoFollowerClusterStateApiFailure() {
Client client = mock(Client.class); Client client = mock(Client.class);
ThreadPool threadPool = mockThreadPool();
when(client.getRemoteClusterClient(anyString())).thenReturn(client); when(client.getRemoteClusterClient(anyString())).thenReturn(client);
AutoFollowPattern autoFollowPattern = new AutoFollowPattern("remote", Collections.singletonList("logs-*"), AutoFollowPattern autoFollowPattern = new AutoFollowPattern("remote", Collections.singletonList("logs-*"),
@ -142,7 +149,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
assertThat(results.get(0).clusterStateFetchException, sameInstance(failure)); assertThat(results.get(0).clusterStateFetchException, sameInstance(failure));
assertThat(results.get(0).autoFollowExecutionResults.entrySet().size(), equalTo(0)); assertThat(results.get(0).autoFollowExecutionResults.entrySet().size(), equalTo(0));
}; };
AutoFollower autoFollower = new AutoFollower(handler, followerState) { AutoFollower autoFollower = new AutoFollower("remote", threadPool, handler, followerClusterStateSupplier(followerState)) {
@Override @Override
void getLeaderClusterState(String remoteCluster, void getLeaderClusterState(String remoteCluster,
BiConsumer<ClusterState, Exception> handler) { BiConsumer<ClusterState, Exception> handler) {
@ -169,6 +176,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
public void testAutoFollowerUpdateClusterStateFailure() { public void testAutoFollowerUpdateClusterStateFailure() {
Client client = mock(Client.class); Client client = mock(Client.class);
ThreadPool threadPool = mockThreadPool();
when(client.getRemoteClusterClient(anyString())).thenReturn(client); when(client.getRemoteClusterClient(anyString())).thenReturn(client);
ClusterState leaderState = createRemoteClusterState("logs-20190101"); ClusterState leaderState = createRemoteClusterState("logs-20190101");
@ -196,7 +204,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
assertThat(entries.get(0).getKey().getName(), equalTo("logs-20190101")); assertThat(entries.get(0).getKey().getName(), equalTo("logs-20190101"));
assertThat(entries.get(0).getValue(), sameInstance(failure)); assertThat(entries.get(0).getValue(), sameInstance(failure));
}; };
AutoFollower autoFollower = new AutoFollower(handler, followerState) { AutoFollower autoFollower = new AutoFollower("remote", threadPool, handler, followerClusterStateSupplier(followerState)) {
@Override @Override
void getLeaderClusterState(String remoteCluster, void getLeaderClusterState(String remoteCluster,
BiConsumer<ClusterState, Exception> handler) { BiConsumer<ClusterState, Exception> handler) {
@ -225,6 +233,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
public void testAutoFollowerCreateAndFollowApiCallFailure() { public void testAutoFollowerCreateAndFollowApiCallFailure() {
Client client = mock(Client.class); Client client = mock(Client.class);
ThreadPool threadPool = mockThreadPool();
when(client.getRemoteClusterClient(anyString())).thenReturn(client); when(client.getRemoteClusterClient(anyString())).thenReturn(client);
ClusterState leaderState = createRemoteClusterState("logs-20190101"); ClusterState leaderState = createRemoteClusterState("logs-20190101");
@ -252,7 +261,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
assertThat(entries.get(0).getKey().getName(), equalTo("logs-20190101")); assertThat(entries.get(0).getKey().getName(), equalTo("logs-20190101"));
assertThat(entries.get(0).getValue(), sameInstance(failure)); assertThat(entries.get(0).getValue(), sameInstance(failure));
}; };
AutoFollower autoFollower = new AutoFollower(handler, followerState) { AutoFollower autoFollower = new AutoFollower("remote", threadPool, handler, followerClusterStateSupplier(followerState)) {
@Override @Override
void getLeaderClusterState(String remoteCluster, void getLeaderClusterState(String remoteCluster,
BiConsumer<ClusterState, Exception> handler) { BiConsumer<ClusterState, Exception> handler) {
@ -324,7 +333,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
.routingTable(routingTableBuilder.build()) .routingTable(routingTableBuilder.build())
.build(); .build();
List<Index> result = AutoFollower.getLeaderIndicesToFollow("remote", autoFollowPattern, leaderState, followerState, List<Index> result = AutoFollower.getLeaderIndicesToFollow(autoFollowPattern, leaderState, followerState,
Collections.emptyList()); Collections.emptyList());
result.sort(Comparator.comparing(Index::getName)); result.sort(Comparator.comparing(Index::getName));
assertThat(result.size(), equalTo(3)); assertThat(result.size(), equalTo(3));
@ -333,7 +342,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
assertThat(result.get(2).getName(), equalTo("metrics-4")); assertThat(result.get(2).getName(), equalTo("metrics-4"));
List<String> followedIndexUUIDs = Collections.singletonList(leaderState.metaData().index("metrics-2").getIndexUUID()); List<String> followedIndexUUIDs = Collections.singletonList(leaderState.metaData().index("metrics-2").getIndexUUID());
result = AutoFollower.getLeaderIndicesToFollow("remote", autoFollowPattern, leaderState, followerState, followedIndexUUIDs); result = AutoFollower.getLeaderIndicesToFollow(autoFollowPattern, leaderState, followerState, followedIndexUUIDs);
result.sort(Comparator.comparing(Index::getName)); result.sort(Comparator.comparing(Index::getName));
assertThat(result.size(), equalTo(2)); assertThat(result.size(), equalTo(2));
assertThat(result.get(0).getName(), equalTo("metrics-0")); assertThat(result.get(0).getName(), equalTo("metrics-0"));
@ -365,7 +374,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
.routingTable(RoutingTable.builder(leaderState.routingTable()).add(indexRoutingTable).build()) .routingTable(RoutingTable.builder(leaderState.routingTable()).add(indexRoutingTable).build())
.build(); .build();
List<Index> result = AutoFollower.getLeaderIndicesToFollow("remote", autoFollowPattern, leaderState, followerState, List<Index> result = AutoFollower.getLeaderIndicesToFollow(autoFollowPattern, leaderState, followerState,
Collections.emptyList()); Collections.emptyList());
assertThat(result.size(), equalTo(1)); assertThat(result.size(), equalTo(1));
assertThat(result.get(0).getName(), equalTo("index1")); assertThat(result.get(0).getName(), equalTo("index1"));
@ -379,7 +388,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
.routingTable(RoutingTable.builder(leaderState.routingTable()).add(indexRoutingTable).build()) .routingTable(RoutingTable.builder(leaderState.routingTable()).add(indexRoutingTable).build())
.build(); .build();
result = AutoFollower.getLeaderIndicesToFollow("remote", autoFollowPattern, leaderState, followerState, Collections.emptyList()); result = AutoFollower.getLeaderIndicesToFollow(autoFollowPattern, leaderState, followerState, Collections.emptyList());
assertThat(result.size(), equalTo(2)); assertThat(result.size(), equalTo(2));
result.sort(Comparator.comparing(Index::getName)); result.sort(Comparator.comparing(Index::getName));
assertThat(result.get(0).getName(), equalTo("index1")); assertThat(result.get(0).getName(), equalTo("index1"));
@ -429,7 +438,6 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
public void testStats() { public void testStats() {
AutoFollowCoordinator autoFollowCoordinator = new AutoFollowCoordinator( AutoFollowCoordinator autoFollowCoordinator = new AutoFollowCoordinator(
Settings.EMPTY,
null, null,
null, null,
mock(ClusterService.class), mock(ClusterService.class),
@ -503,4 +511,32 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
return csBuilder.build(); return csBuilder.build();
} }
private static Supplier<ClusterState> followerClusterStateSupplier(ClusterState... states) {
final AutoFollowMetadata emptyAutoFollowMetadata =
new AutoFollowMetadata(Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
final ClusterState lastState = ClusterState.builder(new ClusterName("remote"))
.metaData(MetaData.builder().putCustom(AutoFollowMetadata.TYPE, emptyAutoFollowMetadata))
.build();
final LinkedList<ClusterState> queue = new LinkedList<>(Arrays.asList(states));
return () -> {
final ClusterState current = queue.poll();
if (current != null) {
return current;
} else {
return lastState;
}
};
}
private static ThreadPool mockThreadPool() {
ThreadPool threadPool = mock(ThreadPool.class);
doAnswer(invocation -> {
Object[] args = invocation.getArguments();
Runnable task = (Runnable) args[2];
task.run();
return null;
}).when(threadPool).schedule(any(), anyString(), any());
return threadPool;
}
} }