Browse Source

Run GeoIp YAML tests in multi-project cluster and fix bug discovered by tests (#131521)

Sam Xiao 2 months ago
parent
commit
cc6255569f

+ 1 - 0
modules/ingest-geoip/build.gradle

@@ -12,6 +12,7 @@ import org.elasticsearch.gradle.OS
 apply plugin: 'elasticsearch.internal-yaml-rest-test'
 apply plugin: 'elasticsearch.yaml-rest-compat-test'
 apply plugin: 'elasticsearch.internal-cluster-test'
+apply plugin: 'elasticsearch.internal-test-artifact'
 
 esplugin {
   description = 'Ingest processor that uses lookup geo data based on IP addresses using the MaxMind geo database'

+ 17 - 0
modules/ingest-geoip/qa/multi-project/build.gradle

@@ -8,15 +8,25 @@
  */
 
 apply plugin: 'elasticsearch.internal-java-rest-test'
+apply plugin: 'elasticsearch.internal-yaml-rest-test'
 
 dependencies {
   javaRestTestImplementation project(':modules:ingest-geoip')
   javaRestTestImplementation project(':test:external-modules:test-multi-project')
   javaRestTestImplementation project(':test:fixtures:geoip-fixture')
 
+  yamlRestTestImplementation(testArtifact(project(":modules:ingest-geoip"), "yamlRestTest"))  // includes yaml test code from ingest-geoip
+  yamlRestTestImplementation project(':modules:ingest-geoip')
+  yamlRestTestImplementation project(':test:external-modules:test-multi-project')
+  yamlRestTestImplementation project(':test:fixtures:geoip-fixture')
+  yamlRestTestImplementation project(':x-pack:qa:multi-project:yaml-test-framework')
+
   clusterModules project(':modules:ingest-geoip')
   clusterModules project(':modules:reindex')  // needed for database cleanup
   clusterModules project(':test:external-modules:test-multi-project')
+
+  // includes yaml rest test artifacts from ingest-geoip module
+  restTestConfig project(path: ':modules:ingest-geoip', configuration: "restTests")
 }
 
 tasks.withType(Test).configureEach {
@@ -27,3 +37,10 @@ tasks.withType(Test).configureEach {
 tasks.named { it == "javaRestTest" || it == "yamlRestTest" }.configureEach {
   it.onlyIf("snapshot build") { buildParams.snapshotBuild }
 }
+
+restResources {
+  restTests {
+    // includes yaml rest test from ingest_geoip folder
+    includeCore 'ingest_geoip'
+  }
+}

+ 1 - 3
modules/ingest-geoip/qa/multi-project/src/javaRestTest/java/geoip/GeoIpMultiProjectIT.java → modules/ingest-geoip/qa/multi-project/src/javaRestTest/java/org/elasticsearch/ingest/geoip/GeoIpMultiProjectIT.java

@@ -7,15 +7,13 @@
  * License v3.0 only", or the "Server Side Public License, v 1".
  */
 
-package geoip;
+package org.elasticsearch.ingest.geoip;
 
 import fixture.geoip.GeoIpHttpFixture;
 
 import org.elasticsearch.client.Request;
 import org.elasticsearch.client.RequestOptions;
 import org.elasticsearch.core.Booleans;
-import org.elasticsearch.ingest.geoip.GeoIpDownloader;
-import org.elasticsearch.ingest.geoip.GeoIpDownloaderTaskExecutor;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.test.cluster.ElasticsearchCluster;
 import org.elasticsearch.test.rest.ESRestTestCase;

+ 75 - 0
modules/ingest-geoip/qa/multi-project/src/yamlRestTest/java/org/elasticsearch/ingest/geoip/IngestGeoIpClientMultiProjectYamlTestSuiteIT.java

@@ -0,0 +1,75 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the "Elastic License
+ * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
+ * Public License v 1"; you may not use this file except in compliance with, at
+ * your election, the "Elastic License 2.0", the "GNU Affero General Public
+ * License v3.0 only", or the "Server Side Public License, v 1".
+ */
+
+package org.elasticsearch.ingest.geoip;
+
+import fixture.geoip.GeoIpHttpFixture;
+
+import com.carrotsearch.randomizedtesting.annotations.Name;
+import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
+
+import org.elasticsearch.core.Booleans;
+import org.elasticsearch.core.FixForMultiProject;
+import org.elasticsearch.multiproject.test.MultipleProjectsClientYamlSuiteTestCase;
+import org.elasticsearch.test.cluster.ElasticsearchCluster;
+import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate;
+import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestRule;
+
+import static org.elasticsearch.ingest.geoip.IngestGeoIpClientYamlTestSuiteIT.assertDatabasesLoaded;
+import static org.elasticsearch.ingest.geoip.IngestGeoIpClientYamlTestSuiteIT.putGeoipPipeline;
+
+@FixForMultiProject(description = "Potentially remove this test after https://elasticco.atlassian.net/browse/ES-12094 is implemented")
+public class IngestGeoIpClientMultiProjectYamlTestSuiteIT extends MultipleProjectsClientYamlSuiteTestCase {
+
+    private static final boolean useFixture = Booleans.parseBoolean(System.getProperty("geoip_use_service", "false")) == false;
+
+    private static GeoIpHttpFixture fixture = new GeoIpHttpFixture(useFixture);
+
+    private static ElasticsearchCluster cluster = ElasticsearchCluster.local()
+        .module("reindex")
+        .module("ingest-geoip")
+        .systemProperty("ingest.geoip.downloader.enabled.default", "true")
+        // sets the plain (geoip.elastic.co) downloader endpoint, which is used in these tests
+        .setting("ingest.geoip.downloader.endpoint", () -> fixture.getAddress(), s -> useFixture)
+        // also sets the enterprise downloader maxmind endpoint, to make sure we do not accidentally hit the real endpoint from tests
+        // note: it's not important that the downloading actually work at this point -- the rest tests (so far) don't exercise
+        // the downloading code because of license reasons -- but if they did, then it would be important that we're hitting a fixture
+        .systemProperty("ingest.geoip.downloader.maxmind.endpoint.default", () -> fixture.getAddress(), s -> useFixture)
+        .setting("test.multi_project.enabled", "true")
+        .setting("xpack.license.self_generated.type", "trial")
+        .user(USER, PASS)
+        .build();
+
+    @ClassRule
+    public static TestRule ruleChain = RuleChain.outerRule(fixture).around(cluster);
+
+    @Override
+    protected String getTestRestCluster() {
+        return cluster.getHttpAddresses();
+    }
+
+    public IngestGeoIpClientMultiProjectYamlTestSuiteIT(@Name("yaml") ClientYamlTestCandidate testCandidate) {
+        super(testCandidate);
+    }
+
+    @ParametersFactory
+    public static Iterable<Object[]> parameters() throws Exception {
+        return ESClientYamlSuiteTestCase.createParameters();
+    }
+
+    @Before
+    public void waitForDatabases() throws Exception {
+        putGeoipPipeline("pipeline-with-geoip");
+        assertDatabasesLoaded();
+    }
+}

+ 85 - 86
modules/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/DatabaseNodeService.java

@@ -16,6 +16,7 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.client.internal.Client;
 import org.elasticsearch.client.internal.OriginSettingClient;
 import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.ProjectState;
 import org.elasticsearch.cluster.metadata.IndexAbstraction;
 import org.elasticsearch.cluster.metadata.ProjectId;
 import org.elasticsearch.cluster.metadata.ProjectMetadata;
@@ -288,104 +289,102 @@ public final class DatabaseNodeService implements IpDatabaseProvider {
         }
 
         // Optimization: only load the .geoip_databases for projects that are allocated to this node
-        for (ProjectMetadata projectMetadata : state.metadata().projects().values()) {
-            ProjectId projectId = projectMetadata.id();
+        state.forEachProject(this::checkDatabases);
+    }
 
-            PersistentTasksCustomMetadata persistentTasks = projectMetadata.custom(PersistentTasksCustomMetadata.TYPE);
-            if (persistentTasks == null) {
-                logger.trace("Not checking databases for project [{}] because persistent tasks are null", projectId);
-                continue;
-            }
+    void checkDatabases(ProjectState projectState) {
+        ProjectId projectId = projectState.projectId();
+        ProjectMetadata projectMetadata = projectState.metadata();
+        PersistentTasksCustomMetadata persistentTasks = projectMetadata.custom(PersistentTasksCustomMetadata.TYPE);
+        if (persistentTasks == null) {
+            logger.trace("Not checking databases for project [{}] because persistent tasks are null", projectId);
+            return;
+        }
 
-            IndexAbstraction databasesAbstraction = projectMetadata.getIndicesLookup().get(GeoIpDownloader.DATABASES_INDEX);
-            if (databasesAbstraction == null) {
-                logger.trace("Not checking databases because geoip databases index does not exist for project [{}]", projectId);
+        IndexAbstraction databasesAbstraction = projectMetadata.getIndicesLookup().get(GeoIpDownloader.DATABASES_INDEX);
+        if (databasesAbstraction == null) {
+            logger.trace("Not checking databases because geoip databases index does not exist for project [{}]", projectId);
+            return;
+        } else {
+            // regardless of whether DATABASES_INDEX is an alias, resolve it to a concrete index
+            Index databasesIndex = databasesAbstraction.getWriteIndex();
+            IndexRoutingTable databasesIndexRT = projectState.routingTable().index(databasesIndex);
+            if (databasesIndexRT == null || databasesIndexRT.allPrimaryShardsActive() == false) {
+                logger.trace(
+                    "Not checking databases because geoip databases index does not have all active primary shards for project [{}]",
+                    projectId
+                );
                 return;
-            } else {
-                // regardless of whether DATABASES_INDEX is an alias, resolve it to a concrete index
-                Index databasesIndex = databasesAbstraction.getWriteIndex();
-                IndexRoutingTable databasesIndexRT = state.routingTable(projectId).index(databasesIndex);
-                if (databasesIndexRT == null || databasesIndexRT.allPrimaryShardsActive() == false) {
-                    logger.trace(
-                        "Not checking databases because geoip databases index does not have all active primary shards for"
-                            + " project [{}]",
-                        projectId
-                    );
-                    return;
-                }
             }
+        }
 
-            // we'll consult each of the geoip downloaders to build up a list of database metadatas to work with
-            List<Tuple<String, GeoIpTaskState.Metadata>> validMetadatas = new ArrayList<>();
+        // we'll consult each of the geoip downloaders to build up a list of database metadatas to work with
+        List<Tuple<String, GeoIpTaskState.Metadata>> validMetadatas = new ArrayList<>();
 
-            // process the geoip task state for the (ordinary) geoip downloader
-            {
-                GeoIpTaskState taskState = getGeoIpTaskState(
-                    projectMetadata,
-                    getTaskId(projectId, projectResolver.supportsMultipleProjects())
-                );
-                if (taskState == null) {
-                    // Note: an empty state will purge stale entries in databases map
-                    taskState = GeoIpTaskState.EMPTY;
-                }
-                validMetadatas.addAll(
-                    taskState.getDatabases()
-                        .entrySet()
-                        .stream()
-                        .filter(e -> e.getValue().isNewEnough(state.getMetadata().settings()))
-                        .map(entry -> Tuple.tuple(entry.getKey(), entry.getValue()))
-                        .toList()
-                );
+        // process the geoip task state for the (ordinary) geoip downloader
+        {
+            GeoIpTaskState taskState = getGeoIpTaskState(projectMetadata, getTaskId(projectId, projectResolver.supportsMultipleProjects()));
+            if (taskState == null) {
+                // Note: an empty state will purge stale entries in databases map
+                taskState = GeoIpTaskState.EMPTY;
             }
+            validMetadatas.addAll(
+                taskState.getDatabases()
+                    .entrySet()
+                    .stream()
+                    .filter(e -> e.getValue().isNewEnough(projectState.cluster().metadata().settings()))
+                    .map(entry -> Tuple.tuple(entry.getKey(), entry.getValue()))
+                    .toList()
+            );
+        }
 
-            // process the geoip task state for the enterprise geoip downloader
-            {
-                EnterpriseGeoIpTaskState taskState = getEnterpriseGeoIpTaskState(state);
-                if (taskState == null) {
-                    // Note: an empty state will purge stale entries in databases map
-                    taskState = EnterpriseGeoIpTaskState.EMPTY;
-                }
-                validMetadatas.addAll(
-                    taskState.getDatabases()
-                        .entrySet()
-                        .stream()
-                        .filter(e -> e.getValue().isNewEnough(state.getMetadata().settings()))
-                        .map(entry -> Tuple.tuple(entry.getKey(), entry.getValue()))
-                        .toList()
-                );
+        // process the geoip task state for the enterprise geoip downloader
+        {
+            EnterpriseGeoIpTaskState taskState = getEnterpriseGeoIpTaskState(projectState.metadata());
+            if (taskState == null) {
+                // Note: an empty state will purge stale entries in databases map
+                taskState = EnterpriseGeoIpTaskState.EMPTY;
             }
+            validMetadatas.addAll(
+                taskState.getDatabases()
+                    .entrySet()
+                    .stream()
+                    .filter(e -> e.getValue().isNewEnough(projectState.cluster().metadata().settings()))
+                    .map(entry -> Tuple.tuple(entry.getKey(), entry.getValue()))
+                    .toList()
+            );
+        }
 
-            // run through all the valid metadatas, regardless of source, and retrieve them if the persistent downloader task
-            // has downloaded a new version of the databases
-            validMetadatas.forEach(e -> {
-                String name = e.v1();
-                GeoIpTaskState.Metadata metadata = e.v2();
-                DatabaseReaderLazyLoader reference = getProjectLazyLoader(projectId, name);
-                String remoteMd5 = metadata.md5();
-                String localMd5 = reference != null ? reference.getMd5() : null;
-                if (Objects.equals(localMd5, remoteMd5)) {
-                    logger.debug("[{}] is up to date [{}] with cluster state [{}]", name, localMd5, remoteMd5);
-                    return;
-                }
+        // run through all the valid metadatas, regardless of source, and retrieve them if the persistent downloader task
+        // has downloaded a new version of the databases
+        validMetadatas.forEach(e -> {
+            String name = e.v1();
+            GeoIpTaskState.Metadata metadata = e.v2();
+            DatabaseReaderLazyLoader reference = getProjectLazyLoader(projectId, name);
+            String remoteMd5 = metadata.md5();
+            String localMd5 = reference != null ? reference.getMd5() : null;
+            if (Objects.equals(localMd5, remoteMd5)) {
+                logger.debug("[{}] is up to date [{}] with cluster state [{}]", name, localMd5, remoteMd5);
+                return;
+            }
 
-                try {
-                    retrieveAndUpdateDatabase(projectId, name, metadata);
-                } catch (Exception ex) {
-                    logger.error(() -> "failed to retrieve database [" + name + "]", ex);
-                }
-            });
-
-            // TODO perhaps we need to handle the license flap persistent task state better than we do
-            // i think the ideal end state is that we *do not* drop the files that the enterprise downloader
-            // handled if they fall out -- which means we need to track that in the databases map itself
-
-            // start with the list of all databases we currently know about in this service,
-            // then drop the ones that didn't check out as valid from the task states
-            if (databases.containsKey(projectId)) {
-                Set<String> staleDatabases = new HashSet<>(databases.get(projectId).keySet());
-                staleDatabases.removeAll(validMetadatas.stream().map(Tuple::v1).collect(Collectors.toSet()));
-                removeStaleEntries(projectId, staleDatabases);
+            try {
+                retrieveAndUpdateDatabase(projectId, name, metadata);
+            } catch (Exception ex) {
+                logger.error(() -> "failed to retrieve database [" + name + "]", ex);
             }
+        });
+
+        // TODO perhaps we need to handle the license flap persistent task state better than we do
+        // i think the ideal end state is that we *do not* drop the files that the enterprise downloader
+        // handled if they fall out -- which means we need to track that in the databases map itself
+
+        // start with the list of all databases we currently know about in this service,
+        // then drop the ones that didn't check out as valid from the task states
+        if (databases.containsKey(projectId)) {
+            Set<String> staleDatabases = new HashSet<>(databases.get(projectId).keySet());
+            staleDatabases.removeAll(validMetadatas.stream().map(Tuple::v1).collect(Collectors.toSet()));
+            removeStaleEntries(projectId, staleDatabases);
         }
     }
 

+ 4 - 7
modules/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/EnterpriseGeoIpTaskState.java

@@ -11,12 +11,10 @@ package org.elasticsearch.ingest.geoip;
 
 import org.elasticsearch.TransportVersion;
 import org.elasticsearch.TransportVersions;
-import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.cluster.metadata.ProjectId;
+import org.elasticsearch.cluster.metadata.ProjectMetadata;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.VersionedNamedWriteable;
-import org.elasticsearch.core.FixForMultiProject;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.ingest.EnterpriseGeoIpTask;
@@ -144,14 +142,13 @@ class EnterpriseGeoIpTaskState implements PersistentTaskState, VersionedNamedWri
      * Retrieves the geoip downloader's task state from the cluster state. This may return null in some circumstances,
      * for example if the geoip downloader task hasn't been created yet (which it wouldn't be if it's disabled).
      *
-     * @param state the cluster state to read the task state from
+     * @param projectMetadata the project metatdata to read the task state from
      * @return the geoip downloader's task state or null if there is not a state to read
      */
     @Nullable
-    @FixForMultiProject(description = "Replace ProjectId.DEFAULT")
-    static EnterpriseGeoIpTaskState getEnterpriseGeoIpTaskState(ClusterState state) {
+    static EnterpriseGeoIpTaskState getEnterpriseGeoIpTaskState(ProjectMetadata projectMetadata) {
         PersistentTasksCustomMetadata.PersistentTask<?> task = getTaskWithId(
-            state.metadata().getProject(ProjectId.DEFAULT),
+            projectMetadata,
             EnterpriseGeoIpTask.ENTERPRISE_GEOIP_DOWNLOADER
         );
         return (task == null) ? null : (EnterpriseGeoIpTaskState) task.getState();

+ 2 - 0
modules/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpTaskState.java

@@ -16,6 +16,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.VersionedNamedWriteable;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.core.FixForMultiProject;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.core.Tuple;
@@ -214,6 +215,7 @@ public class GeoIpTaskState implements PersistentTaskState, VersionedNamedWritea
         private static final TimeValue THIRTY_DAYS = TimeValue.timeValueDays(30);
         private static final long THIRTY_DAYS_MILLIS = THIRTY_DAYS.millis();
 
+        @FixForMultiProject(description = "Replace caller from cluster settings to project settings")
         public boolean isNewEnough(Settings settings) {
             // micro optimization: this looks a little silly, but the expected case is that database_validity is only used in tests.
             // we run this code on every document, though, so the argument checking and other bits that getAsTime does is enough

+ 25 - 21
modules/ingest-geoip/src/yamlRestTest/java/org/elasticsearch/ingest/geoip/IngestGeoIpClientYamlTestSuiteIT.java

@@ -74,32 +74,15 @@ public class IngestGeoIpClientYamlTestSuiteIT extends ESClientYamlSuiteTestCase
 
     @Before
     public void waitForDatabases() throws Exception {
-        putGeoipPipeline();
-        assertBusy(() -> {
-            Request request = new Request("GET", "/_ingest/geoip/stats");
-            Map<String, Object> response = entityAsMap(client().performRequest(request));
-
-            Map<?, ?> downloadStats = (Map<?, ?>) response.get("stats");
-            assertThat(downloadStats.get("databases_count"), equalTo(4));
-
-            Map<?, ?> nodes = (Map<?, ?>) response.get("nodes");
-            assertThat(nodes.size(), equalTo(1));
-            Map<?, ?> node = (Map<?, ?>) nodes.values().iterator().next();
-            List<?> databases = ((List<?>) node.get("databases"));
-            assertThat(databases, notNullValue());
-            List<String> databaseNames = databases.stream().map(o -> (String) ((Map<?, ?>) o).get("name")).toList();
-            assertThat(
-                databaseNames,
-                containsInAnyOrder("GeoLite2-City.mmdb", "GeoLite2-Country.mmdb", "GeoLite2-ASN.mmdb", "MyCustomGeoLite2-City.mmdb")
-            );
-        });
+        putGeoipPipeline("pipeline-with-geoip");
+        assertDatabasesLoaded();
     }
 
     /**
      * This creates a pipeline with a geoip processor so that the GeoipDownloader will download its databases.
      * @throws IOException
      */
-    private void putGeoipPipeline() throws IOException {
+    static void putGeoipPipeline(String pipelineName) throws Exception {
         final BytesReference bytes;
         try (XContentBuilder builder = JsonXContent.contentBuilder()) {
             builder.startObject();
@@ -123,9 +106,30 @@ public class IngestGeoIpClientYamlTestSuiteIT extends ESClientYamlSuiteTestCase
             builder.endObject();
             bytes = BytesReference.bytes(builder);
         }
-        Request putPipelineRequest = new Request("PUT", "/_ingest/pipeline/pipeline-with-geoip");
+        Request putPipelineRequest = new Request("PUT", "/_ingest/pipeline/" + pipelineName);
         putPipelineRequest.setEntity(new ByteArrayEntity(bytes.array(), ContentType.APPLICATION_JSON));
         client().performRequest(putPipelineRequest);
     }
 
+    static void assertDatabasesLoaded() throws Exception {
+        // assert that the databases are downloaded and loaded
+        assertBusy(() -> {
+            Request request = new Request("GET", "/_ingest/geoip/stats");
+            Map<String, Object> response = entityAsMap(client().performRequest(request));
+
+            Map<?, ?> downloadStats = (Map<?, ?>) response.get("stats");
+            assertThat(downloadStats.get("databases_count"), equalTo(4));
+
+            Map<?, ?> nodes = (Map<?, ?>) response.get("nodes");
+            assertThat(nodes.size(), equalTo(1));
+            Map<?, ?> node = (Map<?, ?>) nodes.values().iterator().next();
+            List<?> databases = ((List<?>) node.get("databases"));
+            assertThat(databases, notNullValue());
+            List<String> databaseNames = databases.stream().map(o -> (String) ((Map<?, ?>) o).get("name")).toList();
+            assertThat(
+                databaseNames,
+                containsInAnyOrder("GeoLite2-City.mmdb", "GeoLite2-Country.mmdb", "GeoLite2-ASN.mmdb", "MyCustomGeoLite2-City.mmdb")
+            );
+        });
+    }
 }