From 7514526b6b51c027ac1f6436d2a3f5cb0ec118fb Mon Sep 17 00:00:00 2001 From: Ryan Ernst <ryan@elastic.co> Date: Wed, 5 Aug 2020 14:05:37 -0700 Subject: [PATCH] Convert packaging upgrade tests to java (#60560) (#60680) This commit removes the last of the bats tests, converting the rpm/deb upgrade tests to java. It adds a new pattern of tasks, similar in nature but separate from the existing distro tests, named `distroUpgradeTest`. For each index compatible version, a `distroUpgradeTest.VERSION` task exxists. Each distribution then has a task, named `distroUpgradeTest.VERSION.DISTRO`. One thing to note is these new tests do not cover no-jdk versions of the rpm/deb packages, since the distribution/bwc project does not currently build those. closes #59145 closes #46005 --- .../gradle/ElasticsearchDistribution.java | 5 - .../gradle/test/BatsTestTask.java | 131 ---- .../gradle/test/DistroTestPlugin.java | 413 +++++------- .../gradle/vagrant/BatsProgressLogger.java | 105 --- qa/os/bats/upgrade/80_upgrade.bats | 130 ---- qa/os/bats/utils/packages.bash | 179 ------ qa/os/bats/utils/tar.bash | 110 ---- qa/os/bats/utils/utils.bash | 596 ------------------ qa/os/bats/utils/xpack.bash | 100 --- .../packaging/test/PackageUpgradeTests.java | 110 ++++ .../packaging/util/Distribution.java | 6 + .../packaging/util/Packages.java | 89 ++- 12 files changed, 356 insertions(+), 1618 deletions(-) delete mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/test/BatsTestTask.java delete mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/vagrant/BatsProgressLogger.java delete mode 100644 qa/os/bats/upgrade/80_upgrade.bats delete mode 100644 qa/os/bats/utils/packages.bash delete mode 100644 qa/os/bats/utils/tar.bash delete mode 100644 qa/os/bats/utils/utils.bash delete mode 100644 qa/os/bats/utils/xpack.bash create mode 100644 qa/os/src/test/java/org/elasticsearch/packaging/test/PackageUpgradeTests.java diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/ElasticsearchDistribution.java b/buildSrc/src/main/java/org/elasticsearch/gradle/ElasticsearchDistribution.java index 3605e8029ae..8236426b69a 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/ElasticsearchDistribution.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/ElasticsearchDistribution.java @@ -244,11 +244,6 @@ public class ElasticsearchDistribution implements Buildable, Iterable<File> { return configuration.iterator(); } - // TODO: remove this when distro tests are per distribution - public Configuration getConfiguration() { - return configuration; - } - // internal, make this distribution's configuration unmodifiable void finalizeValues() { diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/test/BatsTestTask.java b/buildSrc/src/main/java/org/elasticsearch/gradle/test/BatsTestTask.java deleted file mode 100644 index 8dbac3f7c19..00000000000 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/test/BatsTestTask.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * 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.gradle.test; - -import org.gradle.api.DefaultTask; -import org.gradle.api.file.Directory; -import org.gradle.api.file.DirectoryProperty; -import org.gradle.api.provider.Provider; -import org.gradle.api.tasks.Input; -import org.gradle.api.tasks.InputDirectory; -import org.gradle.api.tasks.Optional; -import org.gradle.api.tasks.TaskAction; - -import java.util.ArrayList; -import java.util.List; -import java.util.stream.Collectors; - -public class BatsTestTask extends DefaultTask { - - private final DirectoryProperty testsDir; - private final DirectoryProperty utilsDir; - private final DirectoryProperty distributionsDir; - private final DirectoryProperty pluginsDir; - private final DirectoryProperty upgradeDir; - private String packageName; - - public BatsTestTask() { - this.testsDir = getProject().getObjects().directoryProperty(); - this.utilsDir = getProject().getObjects().directoryProperty(); - this.distributionsDir = getProject().getObjects().directoryProperty(); - this.pluginsDir = getProject().getObjects().directoryProperty(); - this.upgradeDir = getProject().getObjects().directoryProperty(); - } - - @InputDirectory - public Provider<Directory> getTestsDir() { - return testsDir; - } - - public void setTestsDir(Directory testsDir) { - this.testsDir.set(testsDir); - } - - @InputDirectory - public Provider<Directory> getUtilsDir() { - return utilsDir; - } - - public void setUtilsDir(Directory utilsDir) { - this.utilsDir.set(utilsDir); - } - - @InputDirectory - public Provider<Directory> getDistributionsDir() { - return distributionsDir; - } - - public void setDistributionsDir(Provider<Directory> distributionsDir) { - this.distributionsDir.set(distributionsDir); - } - - @InputDirectory - @Optional - public Provider<Directory> getPluginsDir() { - return pluginsDir; - } - - public void setPluginsDir(Provider<Directory> pluginsDir) { - this.pluginsDir.set(pluginsDir); - } - - @InputDirectory - @Optional - public Provider<Directory> getUpgradeDir() { - return upgradeDir; - } - - public void setUpgradeDir(Provider<Directory> upgradeDir) { - this.upgradeDir.set(upgradeDir); - } - - @Input - public String getPackageName() { - return packageName; - } - - public void setPackageName(String packageName) { - this.packageName = packageName; - } - - @TaskAction - public void runBats() { - List<Object> command = new ArrayList<>(); - command.add("bats"); - command.add("--tap"); - command.addAll( - testsDir.getAsFileTree().getFiles().stream().filter(f -> f.getName().endsWith(".bats")).sorted().collect(Collectors.toList()) - ); - getProject().exec(spec -> { - spec.setWorkingDir(distributionsDir.getAsFile()); - spec.environment(System.getenv()); - spec.environment("BATS_TESTS", testsDir.getAsFile().get().toString()); - spec.environment("BATS_UTILS", utilsDir.getAsFile().get().toString()); - if (pluginsDir.isPresent()) { - spec.environment("BATS_PLUGINS", pluginsDir.getAsFile().get().toString()); - } - if (upgradeDir.isPresent()) { - spec.environment("BATS_UPGRADE", upgradeDir.getAsFile().get().toString()); - } - spec.environment("PACKAGE_NAME", packageName); - spec.setCommandLine(command); - }); - } -} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/test/DistroTestPlugin.java b/buildSrc/src/main/java/org/elasticsearch/gradle/test/DistroTestPlugin.java index 7ff3fb63828..8f19ede0534 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/test/DistroTestPlugin.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/test/DistroTestPlugin.java @@ -20,7 +20,6 @@ package org.elasticsearch.gradle.test; import org.elasticsearch.gradle.Architecture; -import org.elasticsearch.gradle.BwcVersions; import org.elasticsearch.gradle.DistributionDownloadPlugin; import org.elasticsearch.gradle.ElasticsearchDistribution; import org.elasticsearch.gradle.ElasticsearchDistribution.Flavor; @@ -28,6 +27,7 @@ import org.elasticsearch.gradle.ElasticsearchDistribution.Platform; import org.elasticsearch.gradle.ElasticsearchDistribution.Type; import org.elasticsearch.gradle.Jdk; import org.elasticsearch.gradle.JdkDownloadPlugin; +import org.elasticsearch.gradle.SystemPropertyCommandLineArgumentProvider; import org.elasticsearch.gradle.Version; import org.elasticsearch.gradle.VersionProperties; import org.elasticsearch.gradle.docker.DockerSupportPlugin; @@ -35,37 +35,28 @@ import org.elasticsearch.gradle.docker.DockerSupportService; import org.elasticsearch.gradle.info.BuildParams; import org.elasticsearch.gradle.internal.InternalDistributionDownloadPlugin; import org.elasticsearch.gradle.util.GradleUtils; -import org.elasticsearch.gradle.vagrant.BatsProgressLogger; import org.elasticsearch.gradle.vagrant.VagrantBasePlugin; import org.elasticsearch.gradle.vagrant.VagrantExtension; +import org.gradle.api.Action; import org.gradle.api.NamedDomainObjectContainer; import org.gradle.api.Plugin; import org.gradle.api.Project; import org.gradle.api.Task; import org.gradle.api.artifacts.Configuration; import org.gradle.api.artifacts.dsl.DependencyHandler; -import org.gradle.api.file.Directory; -import org.gradle.api.plugins.ExtraPropertiesExtension; import org.gradle.api.plugins.JavaBasePlugin; import org.gradle.api.provider.Provider; import org.gradle.api.specs.Specs; -import org.gradle.api.tasks.Copy; -import org.gradle.api.tasks.TaskInputs; import org.gradle.api.tasks.TaskProvider; import org.gradle.api.tasks.testing.Test; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.file.Files; -import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Random; -import java.util.stream.Collectors; +import java.util.function.Supplier; import java.util.stream.Stream; import static org.elasticsearch.gradle.vagrant.VagrantMachine.convertLinuxPath; @@ -78,13 +69,10 @@ public class DistroTestPlugin implements Plugin<Project> { private static final String GRADLE_JDK_VENDOR = "openjdk"; // all distributions used by distro tests. this is temporary until tests are per distribution - private static final String DISTRIBUTIONS_CONFIGURATION = "distributions"; - private static final String UPGRADE_CONFIGURATION = "upgradeDistributions"; private static final String EXAMPLE_PLUGIN_CONFIGURATION = "examplePlugin"; - private static final String COPY_DISTRIBUTIONS_TASK = "copyDistributions"; - private static final String COPY_UPGRADE_TASK = "copyUpgradePackages"; private static final String IN_VM_SYSPROP = "tests.inVM"; private static final String DISTRIBUTION_SYSPROP = "tests.distribution"; + private static final String BWC_DISTRIBUTION_SYSPROP = "tests.bwc-distribution"; private static final String EXAMPLE_PLUGIN_SYSPROP = "tests.example-plugin"; @Override @@ -100,34 +88,74 @@ public class DistroTestPlugin implements Plugin<Project> { // TODO: it would be useful to also have the SYSTEM_JAVA_HOME setup in the root project, so that running from GCP only needs // a java for gradle to run, and the tests are self sufficient and consistent with the java they use + NamedDomainObjectContainer<ElasticsearchDistribution> allDistributions = DistributionDownloadPlugin.getContainer(project); + List<ElasticsearchDistribution> testDistributions = configureDistributions(project); - Version upgradeVersion = getUpgradeVersion(project); - Provider<Directory> distributionsDir = project.getLayout().getBuildDirectory().dir("packaging/distributions"); - Provider<Directory> upgradeDir = project.getLayout().getBuildDirectory().dir("packaging/upgrade"); - - List<ElasticsearchDistribution> distributions = configureDistributions(project, upgradeVersion); - TaskProvider<Copy> copyDistributionsTask = configureCopyDistributionsTask(project, distributionsDir); - TaskProvider<Copy> copyUpgradeTask = configureCopyUpgradeTask(project, upgradeVersion, upgradeDir); - - Map<ElasticsearchDistribution.Type, TaskProvider<?>> lifecyleTasks = lifecyleTasks(project, "destructiveDistroTest"); + Map<ElasticsearchDistribution.Type, TaskProvider<?>> lifecycleTasks = lifecycleTasks(project, "destructiveDistroTest"); + Map<String, TaskProvider<?>> versionTasks = versionTasks(project, "destructiveDistroUpgradeTest"); TaskProvider<Task> destructiveDistroTest = project.getTasks().register("destructiveDistroTest"); Configuration examplePlugin = configureExamplePlugin(project); - for (ElasticsearchDistribution distribution : distributions) { - TaskProvider<?> destructiveTask = configureDistroTest(project, distribution, dockerSupport, examplePlugin); - destructiveDistroTest.configure(t -> t.dependsOn(destructiveTask)); - lifecyleTasks.get(distribution.getType()).configure(t -> t.dependsOn(destructiveTask)); - } + List<TaskProvider<Test>> windowsTestTasks = new ArrayList<>(); + Map<Type, List<TaskProvider<Test>>> linuxTestTasks = new HashMap<>(); + Map<String, List<TaskProvider<Test>>> upgradeTestTasks = new HashMap<>(); + Map<String, TaskProvider<?>> depsTasks = new HashMap<>(); + for (ElasticsearchDistribution distribution : testDistributions) { + String taskname = destructiveDistroTestTaskName(distribution); + TaskProvider<?> depsTask = project.getTasks().register(taskname + "#deps"); + depsTask.configure(t -> t.dependsOn(distribution, examplePlugin)); + depsTasks.put(taskname, depsTask); + TaskProvider<Test> destructiveTask = configureTestTask(project, taskname, distribution, t -> { + t.onlyIf(t2 -> distribution.getType() != Type.DOCKER || dockerSupport.get().getDockerAvailability().isAvailable); + addDistributionSysprop(t, DISTRIBUTION_SYSPROP, distribution::toString); + addDistributionSysprop(t, EXAMPLE_PLUGIN_SYSPROP, () -> examplePlugin.getSingleFile().toString()); + t.exclude("**/PackageUpgradeTests.class"); + }, depsTask); - TaskProvider<BatsTestTask> batsUpgradeTest = configureBatsTest( - project, - "upgrade", - distributionsDir, - copyDistributionsTask, - copyUpgradeTask - ); - batsUpgradeTest.configure(t -> t.setUpgradeDir(upgradeDir)); + if (distribution.getPlatform() == Platform.WINDOWS) { + windowsTestTasks.add(destructiveTask); + } else { + linuxTestTasks.computeIfAbsent(distribution.getType(), k -> new ArrayList<>()).add(destructiveTask); + } + destructiveDistroTest.configure(t -> t.dependsOn(destructiveTask)); + lifecycleTasks.get(distribution.getType()).configure(t -> t.dependsOn(destructiveTask)); + + if ((distribution.getType() == Type.DEB || distribution.getType() == Type.RPM) && distribution.getBundledJdk()) { + for (Version version : BuildParams.getBwcVersions().getIndexCompatible()) { + if (distribution.getFlavor() == Flavor.OSS && version.before("6.3.0")) { + continue; // before opening xpack + } + final ElasticsearchDistribution bwcDistro; + if (version.equals(Version.fromString(distribution.getVersion()))) { + // this is the same as the distribution we are testing + bwcDistro = distribution; + } else { + bwcDistro = createDistro( + allDistributions, + distribution.getArchitecture(), + distribution.getType(), + distribution.getPlatform(), + distribution.getFlavor(), + distribution.getBundledJdk(), + version.toString() + ); + + } + String upgradeTaskname = destructiveDistroUpgradeTestTaskName(distribution, version.toString()); + TaskProvider<?> upgradeDepsTask = project.getTasks().register(upgradeTaskname + "#deps"); + upgradeDepsTask.configure(t -> t.dependsOn(distribution, bwcDistro)); + depsTasks.put(upgradeTaskname, upgradeDepsTask); + TaskProvider<Test> upgradeTest = configureTestTask(project, upgradeTaskname, distribution, t -> { + addDistributionSysprop(t, DISTRIBUTION_SYSPROP, distribution::toString); + addDistributionSysprop(t, BWC_DISTRIBUTION_SYSPROP, bwcDistro::toString); + t.include("**/PackageUpgradeTests.class"); + }, upgradeDepsTask); + versionTasks.get(version.toString()).configure(t -> t.dependsOn(upgradeTest)); + upgradeTestTasks.computeIfAbsent(version.toString(), k -> new ArrayList<>()).add(upgradeTest); + } + } + } project.subprojects(vmProject -> { vmProject.getPluginManager().apply(VagrantBasePlugin.class); @@ -135,23 +163,26 @@ public class DistroTestPlugin implements Plugin<Project> { List<Object> vmDependencies = new ArrayList<>(configureVM(vmProject)); vmDependencies.add(project.getConfigurations().getByName("testRuntimeClasspath")); - Map<ElasticsearchDistribution.Type, TaskProvider<?>> vmLifecyleTasks = lifecyleTasks(vmProject, "distroTest"); + Map<ElasticsearchDistribution.Type, TaskProvider<?>> vmLifecyleTasks = lifecycleTasks(vmProject, "distroTest"); + Map<String, TaskProvider<?>> vmVersionTasks = versionTasks(vmProject, "distroUpgradeTest"); TaskProvider<Task> distroTest = vmProject.getTasks().register("distroTest"); - for (ElasticsearchDistribution distribution : distributions) { - String destructiveTaskName = destructiveDistroTestTaskName(distribution); - Platform platform = distribution.getPlatform(); - // this condition ensures windows boxes get windows distributions, and linux boxes get linux distributions - if (isWindows(vmProject) == (platform == Platform.WINDOWS)) { - TaskProvider<GradleDistroTestTask> vmTask = configureVMWrapperTask( - vmProject, - distribution.getName() + " distribution", - destructiveTaskName, - vmDependencies - ); - vmTask.configure(t -> t.dependsOn(distribution, examplePlugin)); - vmLifecyleTasks.get(distribution.getType()).configure(t -> t.dependsOn(vmTask)); - distroTest.configure(t -> { + // windows boxes get windows distributions, and linux boxes get linux distributions + if (isWindows(vmProject)) { + configureVMWrapperTasks( + vmProject, + windowsTestTasks, + depsTasks, + wrapperTask -> { vmLifecyleTasks.get(Type.ARCHIVE).configure(t -> t.dependsOn(wrapperTask)); }, + vmDependencies + ); + } else { + for (var entry : linuxTestTasks.entrySet()) { + Type type = entry.getKey(); + TaskProvider<?> vmLifecycleTask = vmLifecyleTasks.get(type); + configureVMWrapperTasks(vmProject, entry.getValue(), depsTasks, wrapperTask -> { + vmLifecycleTask.configure(t -> t.dependsOn(wrapperTask)); + // Only VM sub-projects that are specifically opted-in to testing Docker should // have the Docker task added as a dependency. Although we control whether Docker // is installed in the VM via `Vagrantfile` and we could auto-detect its presence @@ -160,26 +191,30 @@ public class DistroTestPlugin implements Plugin<Project> { // auto-detection doesn't work. // // The shouldTestDocker property could be null, hence we use Boolean.TRUE.equals() - boolean shouldExecute = distribution.getType() != Type.DOCKER - - || Boolean.TRUE.equals(vmProject.findProperty("shouldTestDocker")); + boolean shouldExecute = type != Type.DOCKER || Boolean.TRUE.equals(vmProject.findProperty("shouldTestDocker")); if (shouldExecute) { - t.dependsOn(vmTask); + distroTest.configure(t -> t.dependsOn(wrapperTask)); } - }); + }, vmDependencies); + } + + for (var entry : upgradeTestTasks.entrySet()) { + String version = entry.getKey(); + TaskProvider<?> vmVersionTask = vmVersionTasks.get(version); + configureVMWrapperTasks( + vmProject, + entry.getValue(), + depsTasks, + wrapperTask -> { vmVersionTask.configure(t -> t.dependsOn(wrapperTask)); }, + vmDependencies + ); } } - - configureVMWrapperTask(vmProject, "bats upgrade", batsUpgradeTest.getName(), vmDependencies).configure(t -> { - t.setProgressHandler(new BatsProgressLogger(project.getLogger())); - t.onlyIf(spec -> isWindows(vmProject) == false); // bats doesn't run on windows - t.dependsOn(copyDistributionsTask, copyUpgradeTask); - }); }); } - private static Map<ElasticsearchDistribution.Type, TaskProvider<?>> lifecyleTasks(Project project, String taskPrefix) { + private static Map<ElasticsearchDistribution.Type, TaskProvider<?>> lifecycleTasks(Project project, String taskPrefix) { Map<ElasticsearchDistribution.Type, TaskProvider<?>> lifecyleTasks = new HashMap<>(); lifecyleTasks.put(Type.DOCKER, project.getTasks().register(taskPrefix + ".docker")); @@ -190,6 +225,16 @@ public class DistroTestPlugin implements Plugin<Project> { return lifecyleTasks; } + private static Map<String, TaskProvider<?>> versionTasks(Project project, String taskPrefix) { + Map<String, TaskProvider<?>> versionTasks = new HashMap<>(); + + for (Version version : BuildParams.getBwcVersions().getIndexCompatible()) { + versionTasks.put(version.toString(), project.getTasks().register(taskPrefix + ".v" + version)); + } + + return versionTasks; + } + private static Jdk createJdk( NamedDomainObjectContainer<Jdk> jdksContainer, String name, @@ -206,27 +251,6 @@ public class DistroTestPlugin implements Plugin<Project> { return jdk; } - private static Version getUpgradeVersion(Project project) { - String upgradeFromVersionRaw = System.getProperty("tests.packaging.upgradeVersion"); - if (upgradeFromVersionRaw != null) { - return Version.fromString(upgradeFromVersionRaw); - } - - // was not passed in, so randomly choose one from bwc versions - ExtraPropertiesExtension extraProperties = project.getExtensions().getByType(ExtraPropertiesExtension.class); - - if ((boolean) extraProperties.get("bwc_tests_enabled") == false) { - // Upgrade tests will go from current to current when the BWC tests are disabled to skip real BWC tests - return Version.fromString(project.getVersion().toString()); - } - - String firstPartOfSeed = BuildParams.getTestSeed().split(":")[0]; - final long seed = Long.parseUnsignedLong(firstPartOfSeed, 16); - BwcVersions bwcVersions = BuildParams.getBwcVersions(); - final List<Version> indexCompatVersions = bwcVersions.getIndexCompatible(); - return indexCompatVersions.get(new Random(seed).nextInt(indexCompatVersions.size())); - } - private static List<Object> configureVM(Project project) { String box = project.getName(); @@ -263,63 +287,6 @@ public class DistroTestPlugin implements Plugin<Project> { }; } - private static TaskProvider<Copy> configureCopyDistributionsTask(Project project, Provider<Directory> distributionsDir) { - - // temporary, until we have tasks per distribution - return project.getTasks().register(COPY_DISTRIBUTIONS_TASK, Copy.class, t -> { - t.into(distributionsDir); - t.from(project.getConfigurations().getByName(DISTRIBUTIONS_CONFIGURATION)); - - Path distributionsPath = distributionsDir.get().getAsFile().toPath(); - TaskInputs inputs = t.getInputs(); - inputs.property("version", VersionProperties.getElasticsearch()); - t.doLast(action -> { - try { - Files.write(distributionsPath.resolve("version"), VersionProperties.getElasticsearch().getBytes()); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - }); - }); - } - - private static TaskProvider<Copy> configureCopyUpgradeTask(Project project, Version upgradeVersion, Provider<Directory> upgradeDir) { - // temporary, until we have tasks per distribution - return project.getTasks().register(COPY_UPGRADE_TASK, Copy.class, t -> { - t.into(upgradeDir); - t.from(project.getConfigurations().getByName(UPGRADE_CONFIGURATION)); - - Path upgradePath = upgradeDir.get().getAsFile().toPath(); - - // write bwc version, and append -SNAPSHOT if it is an unreleased version - BwcVersions bwcVersions = BuildParams.getBwcVersions(); - final String upgradeFromVersion; - if (bwcVersions.unreleasedInfo(upgradeVersion) != null) { - upgradeFromVersion = upgradeVersion.toString() + "-SNAPSHOT"; - } else { - upgradeFromVersion = upgradeVersion.toString(); - } - TaskInputs inputs = t.getInputs(); - inputs.property("upgrade_from_version", upgradeFromVersion); - // TODO: this is serializable, need to think how to represent this as an input - // inputs.property("bwc_versions", bwcVersions); - t.doLast(action -> { - try { - Files.write(upgradePath.resolve("version"), VersionProperties.getElasticsearch().getBytes()); - Files.write(upgradePath.resolve("upgrade_from_version"), upgradeFromVersion.getBytes()); - Path upgradeMarkerPath = upgradePath.resolve("upgrade_is_oss"); - project.delete(upgradeMarkerPath); - // this is always true, but bats tests rely on it. It is just temporary until bats is removed. - if (upgradeVersion.onOrAfter("6.3.0")) { - Files.write(upgradeMarkerPath, new byte[0]); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - }); - }); - } - private static Configuration configureExamplePlugin(Project project) { Configuration examplePlugin = project.getConfigurations().create(EXAMPLE_PLUGIN_CONFIGURATION); DependencyHandler deps = project.getDependencies(); @@ -328,69 +295,52 @@ public class DistroTestPlugin implements Plugin<Project> { return examplePlugin; } - private static TaskProvider<GradleDistroTestTask> configureVMWrapperTask( + private static void configureVMWrapperTasks( Project project, - String type, - String destructiveTaskPath, - List<Object> dependsOn + List<TaskProvider<Test>> destructiveTasks, + Map<String, TaskProvider<?>> depsTasks, + Action<TaskProvider<GradleDistroTestTask>> configure, + Object... additionalDeps ) { - int taskNameStart = destructiveTaskPath.lastIndexOf(':') + "destructive".length() + 1; - String taskname = destructiveTaskPath.substring(taskNameStart); - taskname = taskname.substring(0, 1).toLowerCase(Locale.ROOT) + taskname.substring(1); - return project.getTasks().register(taskname, GradleDistroTestTask.class, t -> { - t.setGroup(JavaBasePlugin.VERIFICATION_GROUP); - t.setDescription("Runs " + type + " tests within vagrant"); - t.setTaskName(destructiveTaskPath); - t.extraArg("-D'" + IN_VM_SYSPROP + "'"); - t.dependsOn(dependsOn); - }); + for (TaskProvider<? extends Task> destructiveTask : destructiveTasks) { + String destructiveTaskName = destructiveTask.getName(); + String taskname = destructiveTaskName.substring("destructive".length()); + taskname = taskname.substring(0, 1).toLowerCase(Locale.ROOT) + taskname.substring(1); + TaskProvider<GradleDistroTestTask> vmTask = project.getTasks().register(taskname, GradleDistroTestTask.class, t -> { + t.setGroup(JavaBasePlugin.VERIFICATION_GROUP); + t.setDescription("Runs " + destructiveTaskName.split("\\.", 2)[1] + " tests within vagrant"); + t.setTaskName(destructiveTaskName); + t.extraArg("-D'" + IN_VM_SYSPROP + "'"); + t.dependsOn(depsTasks.get(destructiveTaskName)); + t.dependsOn(additionalDeps); + }); + configure.execute(vmTask); + } } - private static TaskProvider<?> configureDistroTest( + private static TaskProvider<Test> configureTestTask( Project project, + String taskname, ElasticsearchDistribution distribution, - Provider<DockerSupportService> dockerSupport, - Configuration examplePlugin + Action<? super Test> configure, + Object... deps ) { - return project.getTasks().register(destructiveDistroTestTaskName(distribution), Test.class, t -> { - // Disable Docker distribution tests unless a Docker installation is available - t.onlyIf(t2 -> distribution.getType() != Type.DOCKER || dockerSupport.get().getDockerAvailability().isAvailable); + return project.getTasks().register(taskname, Test.class, t -> { // Only run tests for the current architecture t.onlyIf(t3 -> distribution.getArchitecture() == Architecture.current()); t.getOutputs().doNotCacheIf("Build cache is disabled for packaging tests", Specs.satisfyAll()); t.setMaxParallelForks(1); t.setWorkingDir(project.getProjectDir()); - t.systemProperty(DISTRIBUTION_SYSPROP, distribution.toString()); - t.systemProperty(EXAMPLE_PLUGIN_SYSPROP, examplePlugin.getSingleFile().toString()); - if (System.getProperty(IN_VM_SYSPROP) == null) { - t.dependsOn(distribution); - t.dependsOn(examplePlugin); - } - }); - } - - private static TaskProvider<BatsTestTask> configureBatsTest( - Project project, - String type, - Provider<Directory> distributionsDir, - Object... deps - ) { - return project.getTasks().register("destructiveBatsTest." + type, BatsTestTask.class, t -> { - Directory batsDir = project.getLayout().getProjectDirectory().dir("bats"); - t.setTestsDir(batsDir.dir(type)); - t.setUtilsDir(batsDir.dir("utils")); - t.setDistributionsDir(distributionsDir); - t.setPackageName("elasticsearch" + (type.equals("oss") ? "-oss" : "")); if (System.getProperty(IN_VM_SYSPROP) == null) { t.dependsOn(deps); } + configure.execute(t); }); } - private List<ElasticsearchDistribution> configureDistributions(Project project, Version upgradeVersion) { + private List<ElasticsearchDistribution> configureDistributions(Project project) { NamedDomainObjectContainer<ElasticsearchDistribution> distributions = DistributionDownloadPlugin.getContainer(project); List<ElasticsearchDistribution> currentDistros = new ArrayList<>(); - List<ElasticsearchDistribution> upgradeDistros = new ArrayList<>(); for (Architecture architecture : Architecture.values()) { for (Type type : Arrays.asList(Type.DEB, Type.RPM, Type.DOCKER)) { @@ -401,35 +351,20 @@ public class DistroTestPlugin implements Plugin<Project> { boolean skip = bundledJdk == false && (type == Type.DOCKER || architecture == Architecture.AARCH64); if (skip == false) { - addDistro( - distributions, - architecture, - type, - null, - flavor, - bundledJdk, - VersionProperties.getElasticsearch(), - currentDistros + currentDistros.add( + createDistro( + distributions, + architecture, + type, + null, + flavor, + bundledJdk, + VersionProperties.getElasticsearch() + ) ); } } } - - // We don't configure distributions for prior versions for Docker. This is because doing - // so prompts Gradle to try and resolve the Docker dependencies, which doesn't work as - // they can't be downloaded via Ivy (configured in DistributionDownloadPlugin). Since we - // need these for the BATS upgrade tests, and those tests only cover .rpm and .deb, it's - // OK to omit creating such distributions in the first place. We may need to revisit - // this in the future, so allow upgrade testing using Docker containers. - if (type != Type.DOCKER) { - // upgrade version is always bundled jdk - // NOTE: this is mimicking the old VagrantTestPlugin upgrade behavior. It will eventually be replaced - // witha dedicated upgrade test from every bwc version like other bwc tests - addDistro(distributions, architecture, type, null, Flavor.DEFAULT, true, upgradeVersion.toString(), upgradeDistros); - if (upgradeVersion.onOrAfter("6.3.0")) { - addDistro(distributions, architecture, type, null, Flavor.OSS, true, upgradeVersion.toString(), upgradeDistros); - } - } } } @@ -443,52 +378,35 @@ public class DistroTestPlugin implements Plugin<Project> { continue; } - addDistro( - distributions, - architecture, - Type.ARCHIVE, - platform, - flavor, - bundledJdk, - VersionProperties.getElasticsearch(), - currentDistros + currentDistros.add( + createDistro( + distributions, + architecture, + Type.ARCHIVE, + platform, + flavor, + bundledJdk, + VersionProperties.getElasticsearch() + ) ); } } } } - // temporary until distro tests have one test per distro - Configuration packagingConfig = project.getConfigurations().create(DISTRIBUTIONS_CONFIGURATION); - List<Configuration> distroConfigs = currentDistros.stream() - .filter(d -> d.getType() != Type.DOCKER) - .map(ElasticsearchDistribution::getConfiguration) - .collect(Collectors.toList()); - packagingConfig.setExtendsFrom(distroConfigs); - - Configuration packagingUpgradeConfig = project.getConfigurations().create(UPGRADE_CONFIGURATION); - List<Configuration> distroUpgradeConfigs = upgradeDistros.stream() - .map(ElasticsearchDistribution::getConfiguration) - .collect(Collectors.toList()); - packagingUpgradeConfig.setExtendsFrom(distroUpgradeConfigs); - return currentDistros; } - private static void addDistro( + private static ElasticsearchDistribution createDistro( NamedDomainObjectContainer<ElasticsearchDistribution> distributions, Architecture architecture, Type type, Platform platform, Flavor flavor, boolean bundledJdk, - String version, - List<ElasticsearchDistribution> container + String version ) { String name = distroId(type, platform, flavor, bundledJdk, architecture) + "-" + version; - if (distributions.findByName(name) != null) { - return; - } ElasticsearchDistribution distro = distributions.create(name, d -> { d.setArchitecture(architecture); d.setFlavor(flavor); @@ -508,7 +426,7 @@ public class DistroTestPlugin implements Plugin<Project> { distro.setFailIfUnavailable(false); } - container.add(distro); + return distro; } // return true if the project is for a windows VM, false otherwise @@ -530,4 +448,17 @@ public class DistroTestPlugin implements Plugin<Project> { return "destructiveDistroTest." + distroId(type, distro.getPlatform(), distro.getFlavor(), distro.getBundledJdk(), distro.getArchitecture()); } + + private static String destructiveDistroUpgradeTestTaskName(ElasticsearchDistribution distro, String bwcVersion) { + Type type = distro.getType(); + return "destructiveDistroUpgradeTest.v" + + bwcVersion + + "." + + distroId(type, distro.getPlatform(), distro.getFlavor(), distro.getBundledJdk(), distro.getArchitecture()); + } + + private static void addDistributionSysprop(Test task, String sysprop, Supplier<String> valueSupplier) { + SystemPropertyCommandLineArgumentProvider props = task.getExtensions().getByType(SystemPropertyCommandLineArgumentProvider.class); + props.systemProperty(sysprop, valueSupplier); + } } diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/vagrant/BatsProgressLogger.java b/buildSrc/src/main/java/org/elasticsearch/gradle/vagrant/BatsProgressLogger.java deleted file mode 100644 index 3ace00ef4a9..00000000000 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/vagrant/BatsProgressLogger.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * 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.gradle.vagrant; - -import org.gradle.api.logging.Logger; - -import java.util.Formatter; -import java.util.function.UnaryOperator; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -/** - * Adapts an OutputStream containing TAP output from bats into a ProgressLogger and a Logger. - * - * TAP (Test Anything Protocol, https://testanything.org) is used by BATS for its output format. - * - * Every test output goes to the ProgressLogger and all failures - * and non-test output goes to the Logger. That means you can always glance - * at the result of the last test and the cumulative pass/fail/skip stats and - * the failures are all logged. - * - * There is a Tap4j project but we can't use it because it wants to parse the - * entire TAP stream at once and won't parse it stream-wise. - */ -public class BatsProgressLogger implements UnaryOperator<String> { - - private static final Pattern lineRegex = Pattern.compile( - "(?<status>ok|not ok) \\d+(?<skip> # skip (?<skipReason>\\(.+\\))?)? \\[(?<suite>.+)\\] (?<test>.+)" - ); - private static final Pattern startRegex = Pattern.compile("1..(\\d+)"); - - private final Logger logger; - private int testsCompleted = 0; - private int testsFailed = 0; - private int testsSkipped = 0; - private Integer testCount; - private String countsFormat; - - public BatsProgressLogger(Logger logger) { - this.logger = logger; - } - - @Override - public String apply(String line) { - if (testCount == null) { - Matcher m = startRegex.matcher(line); - if (m.matches() == false) { - // haven't reached start of bats test yet, pass through whatever we see - return line; - } - testCount = Integer.parseInt(m.group(1)); - int length = String.valueOf(testCount).length(); - String count = "%0" + length + "d"; - countsFormat = "[" + count + "|" + count + "|" + count + "/" + count + "]"; - return null; - } - Matcher m = lineRegex.matcher(line); - if (m.matches() == false) { - /* These might be failure report lines or comments or whatever. Its hard - to tell and it doesn't matter. */ - logger.warn(line); - return null; - } - boolean skipped = m.group("skip") != null; - boolean success = skipped == false && m.group("status").equals("ok"); - String skipReason = m.group("skipReason"); - String suiteName = m.group("suite"); - String testName = m.group("test"); - - final String status; - if (skipped) { - status = "SKIPPED"; - testsSkipped++; - } else if (success) { - status = " OK"; - testsCompleted++; - } else { - status = " FAILED"; - testsFailed++; - } - - String counts = new Formatter().format(countsFormat, testsCompleted, testsFailed, testsSkipped, testCount).out().toString(); - if (success == false) { - logger.warn(line); - } - return "BATS " + counts + ", " + status + " [" + suiteName + "] " + testName; - } -} diff --git a/qa/os/bats/upgrade/80_upgrade.bats b/qa/os/bats/upgrade/80_upgrade.bats deleted file mode 100644 index 6c9e608ff68..00000000000 --- a/qa/os/bats/upgrade/80_upgrade.bats +++ /dev/null @@ -1,130 +0,0 @@ -#!/usr/bin/env bats - -# Tests upgrading elasticsearch from a previous version with the deb or rpm -# packages. Just uses a single node cluster on the current machine rather than -# fancy rolling restarts. - -# WARNING: This testing file must be executed as root and can -# dramatically change your system. It should only be executed -# in a throw-away VM like those made by the Vagrantfile at -# the root of the Elasticsearch source code. This should -# cause the script to fail if it is executed any other way: -[ -f /etc/is_vagrant_vm ] || { - >&2 echo "must be run on a vagrant VM" - exit 1 -} - -# The test case can be executed with the Bash Automated -# Testing System tool available at https://github.com/sstephenson/bats -# Thanks to Sam Stephenson! - -# 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. - -# Load test utilities -load $BATS_UTILS/utils.bash -load $BATS_UTILS/packages.bash - -# Cleans everything for the 1st execution -setup() { - skip_not_dpkg_or_rpm - - sameVersion="false" - if [ "$(cat $BATS_UPGRADE/upgrade_from_version)" == "$(cat version)" ]; then - sameVersion="true" - else - echo "BWC test version: $(cat $BATS_UPGRADE/upgrade_from_version)" - fi - # TODO: this needs to conditionally change based on version > 6.3.0 - if [ -f $BATS_UPGRADE/upgrade_is_oss ]; then - export PACKAGE_NAME="elasticsearch-oss" - else - skip "upgrade cannot happen from pre 6.3.0 to elasticsearch-oss" - fi -} - -@test "[UPGRADE] install old version" { - clean_before_test - install_package -v $(cat $BATS_UPGRADE/upgrade_from_version) -d $BATS_UPGRADE -} - -@test "[UPGRADE] modify keystore" { - # deliberately modify the keystore to force it to be preserved during package upgrade - export_elasticsearch_paths - sudo -E "$ESHOME/bin/elasticsearch-keystore" remove keystore.seed - sudo -E echo keystore_seed | "$ESHOME/bin/elasticsearch-keystore" add -x keystore.seed -} - -@test "[UPGRADE] start old version" { - export JAVA_HOME=$SYSTEM_JAVA_HOME - start_elasticsearch_service - unset JAVA_HOME -} - -@test "[UPGRADE] check elasticsearch version is old version" { - check_elasticsearch_version "$(cat $BATS_UPGRADE/upgrade_from_version)" -} - -@test "[UPGRADE] index some documents into a few indexes" { - curl -s -H "Content-Type: application/json" -XPOST localhost:9200/library/book/1?pretty -d '{ - "title": "Elasticsearch - The Definitive Guide" - }' - curl -s -H "Content-Type: application/json" -XPOST localhost:9200/library/book/2?pretty -d '{ - "title": "Brave New World" - }' - curl -s -H "Content-Type: application/json" -XPOST localhost:9200/library2/book/1?pretty -d '{ - "title": "The Left Hand of Darkness" - }' -} - -@test "[UPGRADE] verify that the documents are there" { - curl -s localhost:9200/library/book/1?pretty | grep Elasticsearch - curl -s localhost:9200/library/book/2?pretty | grep World - curl -s localhost:9200/library2/book/1?pretty | grep Darkness -} - -@test "[UPGRADE] stop old version" { - stop_elasticsearch_service -} - -@test "[UPGRADE] install version under test" { - if [ "$sameVersion" == "true" ]; then - install_package -f - else - install_package -u - fi -} - -@test "[UPGRADE] start version under test" { - start_elasticsearch_service yellow library - wait_for_elasticsearch_status yellow library2 -} - -@test "[UPGRADE] check elasticsearch version is version under test" { - check_elasticsearch_version "$(cat version)" -} - -@test "[UPGRADE] verify that the documents are there after restart" { - curl -s localhost:9200/library/book/1?pretty | grep Elasticsearch - curl -s localhost:9200/library/book/2?pretty | grep World - curl -s localhost:9200/library2/book/1?pretty | grep Darkness -} - -@test "[UPGRADE] cleanup version under test" { - stop_elasticsearch_service - clean_before_test -} diff --git a/qa/os/bats/utils/packages.bash b/qa/os/bats/utils/packages.bash deleted file mode 100644 index eebb8bd2b93..00000000000 --- a/qa/os/bats/utils/packages.bash +++ /dev/null @@ -1,179 +0,0 @@ -#!/bin/bash - -# This file contains some utilities to test the elasticsearch scripts with -# the .deb/.rpm packages. - -# WARNING: This testing file must be executed as root and can -# dramatically change your system. It should only be executed -# in a throw-away VM like those made by the Vagrantfile at -# the root of the Elasticsearch source code. This should -# cause the script to fail if it is executed any other way: -[ -f /etc/is_vagrant_vm ] || { - >&2 echo "must be run on a vagrant VM" - exit 1 -} - -# 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. - -env_file() { - if is_dpkg; then - echo "/etc/default/elasticsearch" - fi - if is_rpm; then - echo "/etc/sysconfig/elasticsearch" - fi -} - -# Export some useful paths. -export_elasticsearch_paths() { - export ESHOME="/usr/share/elasticsearch" - export ESPLUGINS="$ESHOME/plugins" - export ESMODULES="$ESHOME/modules" - export ESCONFIG="/etc/elasticsearch" - export ESDATA="/var/lib/elasticsearch" - export ESLOG="/var/log/elasticsearch" - export ESENVFILE=$(env_file) - export PACKAGE_NAME -} - - -# Install the rpm or deb package. -# -u upgrade rather than install. This only matters for rpm. -# -v the version to upgrade to. Defaults to the version under test. -install_package() { - local version=$(cat version) - local rpmCommand='-i' - local dir='./' - while getopts ":ufd:v:" opt; do - case $opt in - u) - rpmCommand='-U' - dpkgCommand='--force-confnew' - ;; - f) - rpmCommand='-U --force' - dpkgCommand='--force-conflicts' - ;; - d) - dir=$OPTARG - ;; - v) - version=$OPTARG - ;; - \?) - echo "Invalid option: -$OPTARG" >&2 - ;; - esac - done - local rpm_classifier="-x86_64" - local deb_classifier="-amd64" - if [[ $version == 6* ]]; then - rpm_classifier="" - deb_classifier="" - fi - if is_rpm; then - rpm $rpmCommand $dir/$PACKAGE_NAME-$version$rpm_classifier.rpm - elif is_dpkg; then - run dpkg $dpkgCommand -i $dir/$PACKAGE_NAME-$version$deb_classifier.deb - [[ "$status" -eq 0 ]] || { - echo "dpkg failed:" - echo "$output" - run lsof /var/lib/dpkg/lock - echo "lsof /var/lib/dpkg/lock:" - echo "$output" - false - } - else - skip "Only rpm or deb supported" - fi - - # pass through java home to package - echo "JAVA_HOME=\"$SYSTEM_JAVA_HOME\"" >> $(env_file) -} - -# Checks that all directories & files are correctly installed after a deb or -# rpm install. -verify_package_installation() { - id elasticsearch - - getent group elasticsearch - # homedir is set in /etc/passwd but to a non existent directory - assert_file_not_exist $(getent passwd elasticsearch | cut -d: -f6) - - assert_file "$ESHOME" d root root 755 - assert_file "$ESHOME/bin" d root root 755 - assert_file "$ESHOME/bin/elasticsearch" f root root 755 - assert_file "$ESHOME/bin/elasticsearch-plugin" f root root 755 - assert_file "$ESHOME/bin/elasticsearch-shard" f root root 755 - assert_file "$ESHOME/bin/elasticsearch-node" f root root 755 - assert_file "$ESHOME/lib" d root root 755 - assert_file "$ESCONFIG" d root elasticsearch 2750 - assert_file "$ESCONFIG/elasticsearch.keystore" f root elasticsearch 660 - - sudo -u elasticsearch "$ESHOME/bin/elasticsearch-keystore" list | grep "keystore.seed" - - assert_file "$ESCONFIG/.elasticsearch.keystore.initial_md5sum" f root elasticsearch 644 - assert_file "$ESCONFIG/elasticsearch.yml" f root elasticsearch 660 - assert_file "$ESCONFIG/jvm.options" f root elasticsearch 660 - assert_file "$ESCONFIG/log4j2.properties" f root elasticsearch 660 - assert_file "$ESDATA" d elasticsearch elasticsearch 2750 - assert_file "$ESLOG" d elasticsearch elasticsearch 2750 - assert_file "$ESPLUGINS" d root root 755 - assert_file "$ESMODULES" d root root 755 - assert_file "$ESHOME/NOTICE.txt" f root root 644 - assert_file "$ESHOME/README.asciidoc" f root root 644 - - if is_dpkg; then - # Env file - assert_file "/etc/default/elasticsearch" f root elasticsearch 660 - - # Machine-readable debian/copyright file - local copyrightDir=$(readlink -f /usr/share/doc/$PACKAGE_NAME) - assert_file $copyrightDir d root root 755 - assert_file "$copyrightDir/copyright" f root root 644 - fi - - if is_rpm; then - # Env file - assert_file "/etc/sysconfig/elasticsearch" f root elasticsearch 660 - # License file - assert_file "/usr/share/elasticsearch/LICENSE.txt" f root root 644 - fi - - if is_systemd; then - assert_file "/usr/lib/systemd/system/elasticsearch.service" f root root 644 - assert_file "/usr/lib/tmpfiles.d/elasticsearch.conf" f root root 644 - assert_file "/usr/lib/sysctl.d/elasticsearch.conf" f root root 644 - if is_rpm; then - [[ $(/usr/sbin/sysctl vm.max_map_count) =~ "vm.max_map_count = 262144" ]] - else - [[ $(/sbin/sysctl vm.max_map_count) =~ "vm.max_map_count = 262144" ]] - fi - fi - - if is_sysvinit; then - assert_file "/etc/init.d/elasticsearch" f root root 750 - fi - - run sudo -E -u vagrant LANG="en_US.UTF-8" cat "$ESCONFIG/elasticsearch.yml" - [ $status = 1 ] - [[ "$output" == *"Permission denied"* ]] || { - echo "Expected permission denied but found $output:" - false - } -} diff --git a/qa/os/bats/utils/tar.bash b/qa/os/bats/utils/tar.bash deleted file mode 100644 index 415e79cac38..00000000000 --- a/qa/os/bats/utils/tar.bash +++ /dev/null @@ -1,110 +0,0 @@ -#!/bin/bash - -# This file contains some utilities to test the elasticsearch -# tar distribution. - -# WARNING: This testing file must be executed as root and can -# dramatically change your system. It should only be executed -# in a throw-away VM like those made by the Vagrantfile at -# the root of the Elasticsearch source code. This should -# cause the script to fail if it is executed any other way: -[ -f /etc/is_vagrant_vm ] || { - >&2 echo "must be run on a vagrant VM" - exit 1 -} - -# 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. - - -# Install the tar.gz archive -install_archive() { - export ESHOME=${1:-/tmp/elasticsearch} - - local version=$(cat version) - - echo "Unpacking tarball to $ESHOME" - rm -rf /tmp/untar - mkdir -p /tmp/untar - tar -xzpf "${PACKAGE_NAME}-${version}-linux-x86_64.tar.gz" -C /tmp/untar - - find /tmp/untar -depth -type d -name 'elasticsearch*' -exec mv {} "$ESHOME" \; > /dev/null - - # ES cannot run as root so create elasticsearch user & group if needed - if ! getent group "elasticsearch" > /dev/null 2>&1 ; then - if is_dpkg; then - addgroup --system "elasticsearch" - else - groupadd -r "elasticsearch" - fi - fi - if ! id "elasticsearch" > /dev/null 2>&1 ; then - if is_dpkg; then - adduser --quiet --system --no-create-home --ingroup "elasticsearch" --disabled-password --shell /bin/false "elasticsearch" - else - useradd --system -M --gid "elasticsearch" --shell /sbin/nologin --comment "elasticsearch user" "elasticsearch" - fi - fi - - chown -R elasticsearch:elasticsearch "$ESHOME" - export_elasticsearch_paths -} - -# Move the unzipped tarball to another location. -move_elasticsearch() { - local oldhome="$ESHOME" - export ESHOME="$1" - rm -rf "$ESHOME" - mv "$oldhome" "$ESHOME" - export_elasticsearch_paths -} - -# Export some useful paths. -export_elasticsearch_paths() { - export ESMODULES="$ESHOME/modules" - export ESPLUGINS="$ESHOME/plugins" - export ESCONFIG="$ESHOME/config" - export ESSCRIPTS="$ESCONFIG/scripts" - export ESDATA="$ESHOME/data" - export ESLOG="$ESHOME/logs" - - export PACKAGE_NAME=${PACKAGE_NAME:-"elasticsearch-oss"} -} - -# Checks that all directories & files are correctly installed -# after a archive (tar.gz/zip) install -verify_archive_installation() { - assert_file "$ESHOME" d elasticsearch elasticsearch 755 - assert_file "$ESHOME/bin" d elasticsearch elasticsearch 755 - assert_file "$ESHOME/bin/elasticsearch" f elasticsearch elasticsearch 755 - assert_file "$ESHOME/bin/elasticsearch-env" f elasticsearch elasticsearch 755 - assert_file "$ESHOME/bin/elasticsearch-keystore" f elasticsearch elasticsearch 755 - assert_file "$ESHOME/bin/elasticsearch-plugin" f elasticsearch elasticsearch 755 - assert_file "$ESHOME/bin/elasticsearch-shard" f elasticsearch elasticsearch 755 - assert_file "$ESHOME/bin/elasticsearch-node" f elasticsearch elasticsearch 755 - assert_file "$ESCONFIG" d elasticsearch elasticsearch 755 - assert_file "$ESCONFIG/elasticsearch.yml" f elasticsearch elasticsearch 660 - assert_file "$ESCONFIG/jvm.options" f elasticsearch elasticsearch 660 - assert_file "$ESCONFIG/log4j2.properties" f elasticsearch elasticsearch 660 - assert_file "$ESPLUGINS" d elasticsearch elasticsearch 755 - assert_file "$ESHOME/lib" d elasticsearch elasticsearch 755 - assert_file "$ESHOME/logs" d elasticsearch elasticsearch 755 - assert_file "$ESHOME/NOTICE.txt" f elasticsearch elasticsearch 644 - assert_file "$ESHOME/LICENSE.txt" f elasticsearch elasticsearch 644 - assert_file "$ESHOME/README.asciidoc" f elasticsearch elasticsearch 644 - assert_file_not_exist "$ESCONFIG/elasticsearch.keystore" -} diff --git a/qa/os/bats/utils/utils.bash b/qa/os/bats/utils/utils.bash deleted file mode 100644 index f97abe47411..00000000000 --- a/qa/os/bats/utils/utils.bash +++ /dev/null @@ -1,596 +0,0 @@ -#!/bin/bash - -# This file contains some utilities to test the .deb/.rpm -# packages and the SysV/Systemd scripts. - -# WARNING: This testing file must be executed as root and can -# dramatically change your system. It should only be executed -# in a throw-away VM like those made by the Vagrantfile at -# the root of the Elasticsearch source code. This should -# cause the script to fail if it is executed any other way: -[ -f /etc/is_vagrant_vm ] || { - >&2 echo "must be run on a vagrant VM" - exit 1 -} - -# 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. - -# Checks if necessary commands are available to run the tests - -if [ ! -x /usr/bin/which ]; then - echo "'which' command is mandatory to run the tests" - exit 1 -fi - -if [ ! -x "`which wget 2>/dev/null`" ]; then - echo "'wget' command is mandatory to run the tests" - exit 1 -fi - -if [ ! -x "`which curl 2>/dev/null`" ]; then - echo "'curl' command is mandatory to run the tests" - exit 1 -fi - -if [ ! -x "`which pgrep 2>/dev/null`" ]; then - echo "'pgrep' command is mandatory to run the tests" - exit 1 -fi - -if [ ! -x "`which unzip 2>/dev/null`" ]; then - echo "'unzip' command is mandatory to run the tests" - exit 1 -fi - -if [ ! -x "`which tar 2>/dev/null`" ]; then - echo "'tar' command is mandatory to run the tests" - exit 1 -fi - -if [ ! -x "`which unzip 2>/dev/null`" ]; then - echo "'unzip' command is mandatory to run the tests" - exit 1 -fi - -if [ ! -x "`which java 2>/dev/null`" ]; then - # there are some tests that move java temporarily - if [ ! -x "`command -v java.bak 2>/dev/null`" ]; then - echo "'java' command is mandatory to run the tests" - exit 1 - fi -fi - -# Returns 0 if the 'dpkg' command is available -is_dpkg() { - [ -x "`which dpkg 2>/dev/null`" ] -} - -# Returns 0 if the 'rpm' command is available -is_rpm() { - [ -x "`which rpm 2>/dev/null`" ] -} - -# Skip test if the 'dpkg' command is not supported -skip_not_dpkg() { - is_dpkg || skip "dpkg is not supported" -} - -# Skip test if the 'rpm' command is not supported -skip_not_rpm() { - is_rpm || skip "rpm is not supported" -} - -skip_not_dpkg_or_rpm() { - is_dpkg || is_rpm || skip "only dpkg or rpm systems are supported" -} - -# Returns 0 if the system supports Systemd -is_systemd() { - [ -x /bin/systemctl ] -} - -# Skip test if Systemd is not supported -skip_not_systemd() { - if [ ! -x /bin/systemctl ]; then - skip "systemd is not supported" - fi -} - -# Returns 0 if the system supports SysV -is_sysvinit() { - [ -x "`which service 2>/dev/null`" ] -} - -# Skip test if SysV is not supported -skip_not_sysvinit() { - if [ -x "`which service 2>/dev/null`" ] && is_systemd; then - skip "sysvinit is supported, but systemd too" - fi - if [ ! -x "`which service 2>/dev/null`" ]; then - skip "sysvinit is not supported" - fi -} - -# Skip if tar is not supported -skip_not_tar_gz() { - if [ ! -x "`which tar 2>/dev/null`" ]; then - skip "tar is not supported" - fi -} - -# Skip if unzip is not supported -skip_not_zip() { - if [ ! -x "`which unzip 2>/dev/null`" ]; then - skip "unzip is not supported" - fi -} - -assert_file_exist() { - local file="$1" - local count=$(echo "$file" | wc -l) - [[ "$count" == "1" ]] || { - echo "assert_file_exist must be run on a single file at a time but was called on [$count] files: $file" - false - } - if [ ! -e "$file" ]; then - echo "Should exist: ${file} but does not" - fi - local file=$(readlink -m "${file}") - [ -e "$file" ] -} - -assert_file_not_exist() { - local file="$1" - if [ -e "$file" ]; then - echo "Should not exist: ${file} but does" - fi - local file=$(readlink -m "${file}") - [ ! -e "$file" ] -} - -assert_file() { - local file="$1" - local type=$2 - local user=$3 - local group=$4 - local privileges=$5 - - assert_file_exist "$file" - - if [ "$type" = "d" ]; then - if [ ! -d "$file" ]; then - echo "[$file] should be a directory but is not" - fi - [ -d "$file" ] - else - if [ ! -f "$file" ]; then - echo "[$file] should be a regular file but is not" - fi - [ -f "$file" ] - fi - - if [ "x$user" != "x" ]; then - realuser=$(find "$file" -maxdepth 0 -printf "%u") - if [ "$realuser" != "$user" ]; then - echo "Expected user: $user, found $realuser [$file]" - fi - [ "$realuser" = "$user" ] - fi - - if [ "x$group" != "x" ]; then - realgroup=$(find "$file" -maxdepth 0 -printf "%g") - if [ "$realgroup" != "$group" ]; then - echo "Expected group: $group, found $realgroup [$file]" - fi - [ "$realgroup" = "$group" ] - fi - - if [ "x$privileges" != "x" ]; then - realprivileges=$(find "$file" -maxdepth 0 -printf "%m") - if [ "$realprivileges" != "$privileges" ]; then - echo "Expected privileges: $privileges, found $realprivileges [$file]" - fi - [ "$realprivileges" = "$privileges" ] - fi -} - -assert_module_or_plugin_directory() { - local directory=$1 - shift - - #owner group and permissions vary depending on how es was installed - #just make sure that everything is the same as $CONFIG_DIR, which was properly set up during install - config_user=$(find "$ESHOME" -maxdepth 0 -printf "%u") - config_owner=$(find "$ESHOME" -maxdepth 0 -printf "%g") - - assert_file $directory d $config_user $config_owner 755 -} - -assert_module_or_plugin_file() { - local file=$1 - shift - - assert_file_exist "$(readlink -m $file)" - assert_file $file f $config_user $config_owner 644 -} - -assert_output() { - echo "$output" | grep -E "$1" -} - -# Deletes everything before running a test file -clean_before_test() { - - # List of files to be deleted - ELASTICSEARCH_TEST_FILES=("/usr/share/elasticsearch" \ - "/etc/elasticsearch" \ - "/var/lib/elasticsearch" \ - "/var/log/elasticsearch" \ - "/tmp/elasticsearch" \ - "/etc/default/elasticsearch" \ - "/etc/sysconfig/elasticsearch" \ - "/var/run/elasticsearch" \ - "/usr/share/doc/elasticsearch" \ - "/usr/share/doc/elasticsearch-oss" \ - "/tmp/elasticsearch" \ - "/usr/lib/systemd/system/elasticsearch.conf" \ - "/usr/lib/tmpfiles.d/elasticsearch.conf" \ - "/usr/lib/sysctl.d/elasticsearch.conf") - - # Kills all processes of user elasticsearch - if id elasticsearch > /dev/null 2>&1; then - pkill -u elasticsearch 2>/dev/null || true - fi - - # Kills all running Elasticsearch processes - ps aux | grep -i "org.elasticsearch.bootstrap.Elasticsearch" | awk {'print $2'} | xargs kill -9 > /dev/null 2>&1 || true - - purge_elasticsearch - - # Removes user & group - userdel elasticsearch > /dev/null 2>&1 || true - groupdel elasticsearch > /dev/null 2>&1 || true - - # Removes all files - for d in "${ELASTICSEARCH_TEST_FILES[@]}"; do - if [ -e "$d" ]; then - rm -rf "$d" - fi - done - - if is_systemd; then - systemctl unmask systemd-sysctl.service - fi -} - -purge_elasticsearch() { - # Removes RPM package - if is_rpm; then - rpm --quiet -e $PACKAGE_NAME > /dev/null 2>&1 || true - fi - - if [ -x "`which yum 2>/dev/null`" ]; then - yum remove -y $PACKAGE_NAME > /dev/null 2>&1 || true - fi - - # Removes DEB package - if is_dpkg; then - dpkg --purge $PACKAGE_NAME > /dev/null 2>&1 || true - fi - - if [ -x "`which apt-get 2>/dev/null`" ]; then - apt-get --quiet --yes purge $PACKAGE_NAME > /dev/null 2>&1 || true - fi -} - -# Start elasticsearch and wait for it to come up with a status. -# $1 - expected status - defaults to green -start_elasticsearch_service() { - local desiredStatus=${1:-green} - local index=$2 - local commandLineArgs=$3 - - run_elasticsearch_service 0 $commandLineArgs - - wait_for_elasticsearch_status $desiredStatus $index - - if [ -r "/tmp/elasticsearch/elasticsearch.pid" ]; then - pid=$(cat /tmp/elasticsearch/elasticsearch.pid) - [ "x$pid" != "x" ] && [ "$pid" -gt 0 ] - echo "Looking for elasticsearch pid...." - ps $pid - elif is_systemd; then - run systemctl is-active elasticsearch.service - [ "$status" -eq 0 ] - - run systemctl status elasticsearch.service - [ "$status" -eq 0 ] - - elif is_sysvinit; then - run service elasticsearch status - [ "$status" -eq 0 ] - fi -} - -# Start elasticsearch -# $1 expected status code -# $2 additional command line args -run_elasticsearch_service() { - local expectedStatus=$1 - local commandLineArgs=$2 - # Set the ES_PATH_CONF setting in case we start as a service - if [ ! -z "$ES_PATH_CONF" ] ; then - if is_dpkg; then - echo "ES_PATH_CONF=$ES_PATH_CONF" >> /etc/default/elasticsearch; - elif is_rpm; then - echo "ES_PATH_CONF=$ES_PATH_CONF" >> /etc/sysconfig/elasticsearch; - fi - fi - - if [ -f "/tmp/elasticsearch/bin/elasticsearch" ]; then - # we must capture the exit code to compare so we don't want to start as background process in case we expect something other than 0 - local background="" - local timeoutCommand="" - if [ "$expectedStatus" = 0 ]; then - background="-d" - else - timeoutCommand="timeout 180s " - fi - # su and the Elasticsearch init script work together to break bats. - # sudo isolates bats enough from the init script so everything continues - # to tick along - run sudo -u elasticsearch bash <<BASH -# If jayatana is installed then we try to use it. Elasticsearch should ignore it even when we try. -# If it doesn't ignore it then Elasticsearch will fail to start because of security errors. -# This line is attempting to emulate the on login behavior of /usr/share/upstart/sessions/jayatana.conf -[ -f /usr/share/java/jayatanaag.jar ] && export JAVA_TOOL_OPTIONS="-javaagent:/usr/share/java/jayatanaag.jar" -# And now we can start Elasticsearch normally, in the background (-d) and with a pidfile (-p). -export ES_PATH_CONF=$ES_PATH_CONF -export ES_JAVA_OPTS=$ES_JAVA_OPTS -$timeoutCommand/tmp/elasticsearch/bin/elasticsearch $background -p /tmp/elasticsearch/elasticsearch.pid $commandLineArgs -BASH - [ "$status" -eq "$expectedStatus" ] - elif is_systemd; then - run systemctl daemon-reload - [ "$status" -eq 0 ] - - run systemctl enable elasticsearch.service - [ "$status" -eq 0 ] - - run systemctl is-enabled elasticsearch.service - [ "$status" -eq 0 ] - - run systemctl start elasticsearch.service - [ "$status" -eq "$expectedStatus" ] - - elif is_sysvinit; then - run service elasticsearch start - [ "$status" -eq "$expectedStatus" ] - fi -} - -stop_elasticsearch_service() { - if [ -r "/tmp/elasticsearch/elasticsearch.pid" ]; then - pid=$(cat /tmp/elasticsearch/elasticsearch.pid) - [ "x$pid" != "x" ] && [ "$pid" -gt 0 ] - - kill -SIGTERM $pid - elif is_systemd; then - run systemctl stop elasticsearch.service - [ "$status" -eq 0 ] - - run systemctl is-active elasticsearch.service - [ "$status" -eq 3 ] - - echo "$output" | grep -E 'inactive|failed' - - elif is_sysvinit; then - run service elasticsearch stop - [ "$status" -eq 0 ] - - run service elasticsearch status - [ "$status" -ne 0 ] - fi -} - -# the default netcat packages in the distributions we test are not all compatible -# so we use /dev/tcp - a feature of bash which makes tcp connections -# http://tldp.org/LDP/abs/html/devref1.html#DEVTCP -test_port() { - local host="$1" - local port="$2" - cat < /dev/null > "/dev/tcp/$host/$port" -} - -describe_port() { - local host="$1" - local port="$2" - if test_port "$host" "$port"; then - echo "port $port on host $host is open" - else - echo "port $port on host $host is not open" - fi -} - -debug_collect_logs() { - local es_logfile="/var/log/elasticsearch/elasticsearch.log" - local system_logfile='/var/log/messages' - - if [ -e "$es_logfile" ]; then - echo "Here's the elasticsearch log:" - cat "$es_logfile" - else - echo "The elasticsearch log doesn't exist at $es_logfile" - fi - - if [ -e "$system_logfile" ]; then - echo "Here's the tail of the log at $system_logfile:" - tail -n20 "$system_logfile" - else - echo "The logfile at $system_logfile doesn't exist" - fi - - echo "Current java processes:" - ps aux | grep java || true - - echo "Testing if ES ports are open:" - describe_port 127.0.0.1 9200 - describe_port 127.0.0.1 9201 -} - -set_debug_logging() { - if [ "$ESCONFIG" ] && [ -d "$ESCONFIG" ] && [ -f /etc/os-release ] && (grep -qi suse /etc/os-release); then - echo 'logger.org.elasticsearch.indices: TRACE' >> "$ESCONFIG/elasticsearch.yml" - echo 'logger.org.elasticsearch.gateway: TRACE' >> "$ESCONFIG/elasticsearch.yml" - echo 'logger.org.elasticsearch.cluster: DEBUG' >> "$ESCONFIG/elasticsearch.yml" - fi -} - -# Waits for Elasticsearch to reach some status. -# $1 - expected status - defaults to green -wait_for_elasticsearch_status() { - local desiredStatus=${1:-green} - local index=$2 - - echo "Making sure elasticsearch is up..." - wget -O - --retry-connrefused --waitretry=1 --timeout=120 --tries=120 http://localhost:9200/_cluster/health || { - echo "Looks like elasticsearch never started" - debug_collect_logs - false - } - - if [ -z "index" ]; then - echo "Tring to connect to elasticsearch and wait for expected status $desiredStatus..." - curl -sS "http://localhost:9200/_cluster/health?wait_for_status=$desiredStatus&timeout=180s&pretty" - else - echo "Trying to connect to elasticsearch and wait for expected status $desiredStatus for index $index" - curl -sS "http://localhost:9200/_cluster/health/$index?wait_for_status=$desiredStatus&timeout=180s&pretty" - fi - if [ $? -eq 0 ]; then - echo "Connected" - else - echo "Unable to connect to Elasticsearch" - false - fi - - echo "Checking that the cluster health matches the waited for status..." - run curl -sS -XGET 'http://localhost:9200/_cat/health?h=status&v=false' - if [ "$status" -ne 0 ]; then - echo "error when checking cluster health. code=$status output=" - echo $output - false - fi - echo $output | grep $desiredStatus || { - echo "unexpected status: '$output' wanted '$desiredStatus'" - debug_collect_logs - false - } -} - -# Checks the current elasticsearch version using the Info REST endpoint -# $1 - expected version -check_elasticsearch_version() { - local version=$1 - local versionToCheck - local major=$(echo ${version} | cut -d. -f1 ) - if [ $major -ge 7 ] ; then - versionToCheck=$version - else - versionToCheck=$(echo ${version} | sed -e 's/-SNAPSHOT//') - fi - - run curl -s localhost:9200 - [ "$status" -eq 0 ] - - echo $output | grep \"number\"\ :\ \"$versionToCheck\" || { - echo "Expected $versionToCheck but installed an unexpected version:" - curl -s localhost:9200 - false - } -} - -# Executes some basic Elasticsearch tests -run_elasticsearch_tests() { - # TODO this assertion is the same the one made when waiting for - # elasticsearch to start - run curl -XGET 'http://localhost:9200/_cat/health?h=status&v=false' - [ "$status" -eq 0 ] - echo "$output" | grep -w "green" - - curl -s -H "Content-Type: application/json" -XPOST 'http://localhost:9200/library/book/1?refresh=true&pretty' -d '{ - "title": "Book #1", - "pages": 123 - }' - - curl -s -H "Content-Type: application/json" -XPOST 'http://localhost:9200/library/book/2?refresh=true&pretty' -d '{ - "title": "Book #2", - "pages": 456 - }' - - curl -s -XGET 'http://localhost:9200/_count?pretty' | - grep \"count\"\ :\ 2 - - curl -s -XDELETE 'http://localhost:9200/_all' -} - -# Move the config directory to another directory and properly chown it. -move_config() { - local oldConfig="$ESCONFIG" - # The custom config directory is not under /tmp or /var/tmp because - # systemd's private temp directory functionally means different - # processes can have different views of what's in these directories - export ESCONFIG="${1:-$(mktemp -p /etc -d -t 'config.XXXX')}" - echo "Moving configuration directory from $oldConfig to $ESCONFIG" - - # Move configuration files to the new configuration directory - mv "$oldConfig"/* "$ESCONFIG" - chown -R elasticsearch:elasticsearch "$ESCONFIG" - assert_file_exist "$ESCONFIG/elasticsearch.yml" - assert_file_exist "$ESCONFIG/jvm.options" - assert_file_exist "$ESCONFIG/log4j2.properties" -} - -# permissions from the user umask with the executable bit set -executable_privileges_for_user_from_umask() { - local user=$1 - shift - - echo $((0777 & ~$(sudo -E -u $user sh -c umask) | 0111)) -} - -# permissions from the user umask without the executable bit set -file_privileges_for_user_from_umask() { - local user=$1 - shift - - echo $((0777 & ~$(sudo -E -u $user sh -c umask) & ~0111)) -} - -# move java to simulate it not being in the path -move_java() { - which_java=`command -v java` - assert_file_exist $which_java - mv $which_java ${which_java}.bak -} - -# move java back to its original location -unmove_java() { - which_java=`command -v java.bak` - assert_file_exist $which_java - mv $which_java `dirname $which_java`/java -} diff --git a/qa/os/bats/utils/xpack.bash b/qa/os/bats/utils/xpack.bash deleted file mode 100644 index c267744194a..00000000000 --- a/qa/os/bats/utils/xpack.bash +++ /dev/null @@ -1,100 +0,0 @@ -#!/bin/bash - -# 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. - -# Checks that X-Pack files are correctly installed -verify_xpack_installation() { - local name="x-pack" - local user="$ESPLUGIN_COMMAND_USER" - local group="$ESPLUGIN_COMMAND_USER" - - # Verify binary files - # nocommit: already verified by "main" package verification - #assert_file "$ESHOME/bin" d $user $group 755 - local binaryFiles=( - 'elasticsearch-certgen' - 'elasticsearch-certutil' - 'elasticsearch-croneval' - 'elasticsearch-migrate' - 'elasticsearch-saml-metadata' - 'elasticsearch-setup-passwords' - 'elasticsearch-sql-cli' - "elasticsearch-sql-cli-$(cat version).jar" # This jar is executable so we pitch it in bin so folks will find it - 'elasticsearch-syskeygen' - 'elasticsearch-users' - 'x-pack-env' - 'x-pack-security-env' - 'x-pack-watcher-env' - ) - - local binaryFilesCount=5 # start with oss distro number - for binaryFile in ${binaryFiles[@]}; do - echo "checking for bin file ${binaryFile}" - assert_file "$ESHOME/bin/${binaryFile}" f $user $group 755 - binaryFilesCount=$(( binaryFilesCount + 1 )) - done - ls "$ESHOME/bin/" - # nocommit: decide whether to check the files added by the distribution, not part of xpack... - #assert_number_of_files "$ESHOME/bin/" $binaryFilesCount - - # Verify config files - # nocommit: already verified by "main" package verification - #assert_file "$ESCONFIG" d $user elasticsearch 755 - local configFiles=( - 'users' - 'users_roles' - 'roles.yml' - 'role_mapping.yml' - 'log4j2.properties' - ) - - local configFilesCount=2 # start with ES files, excluding log4j2 - for configFile in ${configFiles[@]}; do - assert_file "$ESCONFIG/${configFile}" f $user elasticsearch 660 - configFilesCount=$(( configFilesCount + 1 )) - done - # nocommit: decide whether to check the files added by the distribution, not part of xpack... - #assert_number_of_files "$ESCONFIG/" $configFilesCount -} - -assert_number_of_files() { - local directory=$1 - local expected=$2 - - local count=$(ls "$directory" | wc -l) - [ "$count" -eq "$expected" ] || { - echo "Expected $expected files in $directory but found: $count" - false - } -} - -generate_trial_license() { - sudo -E -u $ESPLUGIN_COMMAND_USER sh <<"NODE_SETTINGS" -cat >> $ESCONFIG/elasticsearch.yml <<- EOF -xpack.license.self_generated.type: trial -xpack.security.enabled: true -EOF -NODE_SETTINGS -} - -wait_for_xpack() { - local host=${1:-localhost} - local port=${2:-9200} - local listening=1 - for i in {1..60}; do - if test_port "$host" "$port"; then - listening=0 - break - else - sleep 1 - fi - done - - [ "$listening" -eq 0 ] || { - echo "Looks like elasticsearch with x-pack never started." - debug_collect_logs - false - } -} diff --git a/qa/os/src/test/java/org/elasticsearch/packaging/test/PackageUpgradeTests.java b/qa/os/src/test/java/org/elasticsearch/packaging/test/PackageUpgradeTests.java new file mode 100644 index 00000000000..847282258bf --- /dev/null +++ b/qa/os/src/test/java/org/elasticsearch/packaging/test/PackageUpgradeTests.java @@ -0,0 +1,110 @@ +/* + * 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.packaging.test; + +import org.apache.http.client.fluent.Request; +import org.apache.http.entity.ContentType; +import org.elasticsearch.packaging.util.Distribution; +import org.elasticsearch.packaging.util.Packages; + +import java.nio.file.Paths; + +import static org.elasticsearch.packaging.util.Packages.assertInstalled; +import static org.elasticsearch.packaging.util.Packages.installPackage; +import static org.elasticsearch.packaging.util.Packages.verifyPackageInstallation; +import static org.elasticsearch.packaging.util.ServerUtils.makeRequest; +import static org.hamcrest.Matchers.containsString; + +public class PackageUpgradeTests extends PackagingTestCase { + + // the distribution being upgraded + protected static final Distribution bwcDistribution; + static { + bwcDistribution = new Distribution(Paths.get(System.getProperty("tests.bwc-distribution"))); + } + + public void test10InstallBwcVersion() throws Exception { + installation = installPackage(sh, bwcDistribution); + assertInstalled(bwcDistribution); + verifyPackageInstallation(installation, bwcDistribution, sh); + } + + public void test11ModifyKeystore() throws Exception { + // deliberately modify the keystore to force it to be preserved during package upgrade + installation.executables().keystoreTool.run("remove keystore.seed"); + installation.executables().keystoreTool.run("add -x keystore.seed", "keystore_seed"); + } + + public void test12SetupBwcVersion() throws Exception { + startElasticsearch(); + + // create indexes explicitly with 0 replicas so when restarting we can reach green state + makeRequest( + Request.Put("http://localhost:9200/library") + .bodyString("{\"settings\":{\"index\":{\"number_of_replicas\":0}}}", ContentType.APPLICATION_JSON) + ); + makeRequest( + Request.Put("http://localhost:9200/library2") + .bodyString("{\"settings\":{\"index\":{\"number_of_replicas\":0}}}", ContentType.APPLICATION_JSON) + ); + + // add some docs + makeRequest( + Request.Post("http://localhost:9200/library/_doc/1?refresh=true&pretty") + .bodyString("{ \"title\": \"Elasticsearch - The Definitive Guide\"}", ContentType.APPLICATION_JSON) + ); + makeRequest( + Request.Post("http://localhost:9200/library/_doc/2?refresh=true&pretty") + .bodyString("{ \"title\": \"Brave New World\"}", ContentType.APPLICATION_JSON) + ); + makeRequest( + Request.Post("http://localhost:9200/library2/_doc/1?refresh=true&pretty") + .bodyString("{ \"title\": \"The Left Hand of Darkness\"}", ContentType.APPLICATION_JSON) + ); + + assertDocsExist(); + + stopElasticsearch(); + } + + public void test20InstallUpgradedVersion() throws Exception { + if (bwcDistribution.path.equals(distribution.path)) { + // the old and new distributions are the same, so we are testing force upgrading + Packages.forceUpgradePackage(sh, distribution); + } else { + Packages.upgradePackage(sh, distribution); + } + assertInstalled(distribution); + verifyPackageInstallation(installation, distribution, sh); + } + + public void test21CheckUpgradedVersion() throws Exception { + assertWhileRunning(() -> { assertDocsExist(); }); + } + + private void assertDocsExist() throws Exception { + String response1 = makeRequest(Request.Get("http://localhost:9200/library/_doc/1?pretty")); + assertThat(response1, containsString("Elasticsearch")); + String response2 = makeRequest(Request.Get("http://localhost:9200/library/_doc/2?pretty")); + assertThat(response2, containsString("World")); + String response3 = makeRequest(Request.Get("http://localhost:9200/library2/_doc/1?pretty")); + assertThat(response3, containsString("Darkness")); + } +} diff --git a/qa/os/src/test/java/org/elasticsearch/packaging/util/Distribution.java b/qa/os/src/test/java/org/elasticsearch/packaging/util/Distribution.java index 72bd79ff7b4..d4e3dc79fe7 100644 --- a/qa/os/src/test/java/org/elasticsearch/packaging/util/Distribution.java +++ b/qa/os/src/test/java/org/elasticsearch/packaging/util/Distribution.java @@ -29,6 +29,7 @@ public class Distribution { public final Platform platform; public final Flavor flavor; public final boolean hasJdk; + public final String version; public Distribution(Path path) { this.path = path; @@ -46,6 +47,11 @@ public class Distribution { this.platform = filename.contains("windows") ? Platform.WINDOWS : Platform.LINUX; this.flavor = filename.contains("oss") ? Flavor.OSS : Flavor.DEFAULT; this.hasJdk = filename.contains("no-jdk") == false; + String version = filename.split("-", 3)[1]; + if (filename.contains("-SNAPSHOT")) { + version += "-SNAPSHOT"; + } + this.version = version; } public boolean isDefault() { diff --git a/qa/os/src/test/java/org/elasticsearch/packaging/util/Packages.java b/qa/os/src/test/java/org/elasticsearch/packaging/util/Packages.java index 2eed8e586c9..f562c8a67a5 100644 --- a/qa/os/src/test/java/org/elasticsearch/packaging/util/Packages.java +++ b/qa/os/src/test/java/org/elasticsearch/packaging/util/Packages.java @@ -28,6 +28,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.nio.file.StandardOpenOption; +import java.util.Map; import java.util.regex.Pattern; import java.util.stream.Stream; @@ -40,7 +41,6 @@ import static org.elasticsearch.packaging.util.FileMatcher.p644; import static org.elasticsearch.packaging.util.FileMatcher.p660; import static org.elasticsearch.packaging.util.FileMatcher.p750; import static org.elasticsearch.packaging.util.FileMatcher.p755; -import static org.elasticsearch.packaging.util.FileUtils.getCurrentVersion; import static org.elasticsearch.packaging.util.Platforms.isSysVInit; import static org.elasticsearch.packaging.util.Platforms.isSystemd; import static org.elasticsearch.packaging.util.ServerUtils.waitForElasticsearch; @@ -83,17 +83,8 @@ public class Packages { } public static Result packageStatus(Distribution distribution) { - final Shell sh = new Shell(); - final Result result; - logger.info("Package type: " + distribution.packaging); - if (distribution.packaging == Distribution.Packaging.RPM) { - result = sh.runIgnoreExitCode("rpm -qe " + distribution.flavor.name); - } else { - result = sh.runIgnoreExitCode("dpkg -s " + distribution.flavor.name); - } - - return result; + return runPackageManager(distribution, new Shell(), PackageManagerCommand.QUERY); } public static Installation installPackage(Shell sh, Distribution distribution) throws IOException { @@ -101,7 +92,7 @@ public class Packages { if (distribution.hasJdk == false) { sh.getEnv().put("JAVA_HOME", systemJavaHome); } - final Result result = runInstallCommand(distribution, sh); + final Result result = runPackageManager(distribution, sh, PackageManagerCommand.INSTALL); if (result.exitCode != 0) { throw new RuntimeException("Installing distribution " + distribution + " failed: " + result); } @@ -114,13 +105,35 @@ public class Packages { return installation; } - private static Result runInstallCommand(Distribution distribution, Shell sh) { - final Path distributionFile = distribution.path; + public static Installation upgradePackage(Shell sh, Distribution distribution) throws IOException { + final Result result = runPackageManager(distribution, sh, PackageManagerCommand.UPGRADE); + if (result.exitCode != 0) { + throw new RuntimeException("Upgrading distribution " + distribution + " failed: " + result); + } + + return Installation.ofPackage(sh, distribution); + } + + public static Installation forceUpgradePackage(Shell sh, Distribution distribution) throws IOException { + final Result result = runPackageManager(distribution, sh, PackageManagerCommand.FORCE_UPGRADE); + if (result.exitCode != 0) { + throw new RuntimeException("Force upgrading distribution " + distribution + " failed: " + result); + } + + return Installation.ofPackage(sh, distribution); + } + + private static Result runPackageManager(Distribution distribution, Shell sh, PackageManagerCommand command) { + final String distributionArg = command == PackageManagerCommand.QUERY || command == PackageManagerCommand.REMOVE + ? distribution.flavor.name + : distribution.path.toString(); if (Platforms.isRPM()) { - return sh.runIgnoreExitCode("rpm -i " + distributionFile); + String rpmOptions = RPM_OPTIONS.get(command); + return sh.runIgnoreExitCode("rpm " + rpmOptions + " " + distributionArg); } else { - Result r = sh.runIgnoreExitCode("dpkg -i " + distributionFile); + String debOptions = DEB_OPTIONS.get(command); + Result r = sh.runIgnoreExitCode("dpkg " + debOptions + " " + distributionArg); if (r.exitCode != 0) { Result lockOF = sh.runIgnoreExitCode("lsof /var/lib/dpkg/lock"); if (lockOF.exitCode == 0) { @@ -133,15 +146,15 @@ public class Packages { public static void remove(Distribution distribution) throws Exception { final Shell sh = new Shell(); + Result result = runPackageManager(distribution, sh, PackageManagerCommand.REMOVE); + assertThat(result.toString(), result.isSuccess(), is(true)); Platforms.onRPM(() -> { - sh.run("rpm -e " + distribution.flavor.name); final Result status = packageStatus(distribution); assertThat(status.exitCode, is(1)); }); Platforms.onDPKG(() -> { - sh.run("dpkg -r " + distribution.flavor.name); final Result status = packageStatus(distribution); assertThat(status.exitCode, is(0)); assertTrue(Pattern.compile("(?m)^Status:.+deinstall ok").matcher(status.stdout).find()); @@ -151,7 +164,7 @@ public class Packages { public static void verifyPackageInstallation(Installation installation, Distribution distribution, Shell sh) { verifyOssInstallation(installation, distribution, sh); if (distribution.flavor == Distribution.Flavor.DEFAULT) { - verifyDefaultInstallation(installation); + verifyDefaultInstallation(installation, distribution); } } @@ -215,7 +228,7 @@ public class Packages { } } - private static void verifyDefaultInstallation(Installation es) { + private static void verifyDefaultInstallation(Installation es, Distribution distribution) { Stream.of( "elasticsearch-certgen", @@ -234,7 +247,7 @@ public class Packages { // at this time we only install the current version of archive distributions, but if that changes we'll need to pass // the version through here - assertThat(es.bin("elasticsearch-sql-cli-" + getCurrentVersion() + ".jar"), file(File, "root", "root", p755)); + assertThat(es.bin("elasticsearch-sql-cli-" + distribution.version + ".jar"), file(File, "root", "root", p755)); Stream.of("users", "users_roles", "roles.yml", "role_mapping.yml", "log4j2.properties") .forEach(configFile -> assertThat(es.config(configFile), file(File, "root", "elasticsearch", p660))); @@ -317,4 +330,38 @@ public class Packages { return sh.run("journalctl -u elasticsearch.service --after-cursor='" + this.cursor + "'"); } } + + private enum PackageManagerCommand { + QUERY, + INSTALL, + UPGRADE, + FORCE_UPGRADE, + REMOVE + } + + private static Map<PackageManagerCommand, String> RPM_OPTIONS = org.elasticsearch.common.collect.Map.of( + PackageManagerCommand.QUERY, + "-qe", + PackageManagerCommand.INSTALL, + "-i", + PackageManagerCommand.UPGRADE, + "-U", + PackageManagerCommand.FORCE_UPGRADE, + "-U --force", + PackageManagerCommand.REMOVE, + "-e" + ); + + private static Map<PackageManagerCommand, String> DEB_OPTIONS = org.elasticsearch.common.collect.Map.of( + PackageManagerCommand.QUERY, + "-s", + PackageManagerCommand.INSTALL, + "-i", + PackageManagerCommand.UPGRADE, + "-i --force-confnew", + PackageManagerCommand.FORCE_UPGRADE, + "-i --force-conflicts", + PackageManagerCommand.REMOVE, + "-r" + ); }