From 7effc739042ea757b10b4e0b13d3624477b60a56 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 5 Dec 2018 16:42:40 -0600 Subject: [PATCH 01/21] ML: add migrate anomalies assistant --- .../xpack/core/XPackClientPlugin.java | 2 + .../ml/action/ResultsIndexUpgradeAction.java | 158 ++++++ .../ResultsIndexUpgradeRequestTests.java | 24 + .../MlNativeAutodetectIntegTestCase.java | 3 +- .../ml/integration/ResultsIndexUpgradeIT.java | 478 ++++++++++++++++ .../xpack/ml/MachineLearning.java | 9 +- .../xpack/ml/ResultsIndexUpgradeService.java | 516 ++++++++++++++++++ .../TransportResultsIndexUpgradeAction.java | 80 +++ .../results/RestUpgradeResultsAction.java | 65 +++ .../ml/utils/TypedChainTaskExecutor.java | 125 +++++ .../api/ml.upgrade_job_results.json | 21 + .../test/ml/results_indices_upgrade.yml | 70 +++ .../mixed_cluster/80_ml_results_upgrade.yml | 11 + .../old_cluster/80_ml_results_upgrade.yml | 120 ++++ .../80_ml_results_upgrade.yml | 155 ++++++ 15 files changed, 1834 insertions(+), 3 deletions(-) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeAction.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeRequestTests.java create mode 100644 x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportResultsIndexUpgradeAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade_job_results.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/ml/results_indices_upgrade.yml create mode 100644 x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/80_ml_results_upgrade.yml create mode 100644 x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/80_ml_results_upgrade.yml create mode 100644 x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java index 10c30a2f331ee..ed81ef3359e3f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java @@ -103,6 +103,7 @@ import org.elasticsearch.xpack.core.ml.action.PutDatafeedAction; import org.elasticsearch.xpack.core.ml.action.PutFilterAction; import org.elasticsearch.xpack.core.ml.action.PutJobAction; +import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; import org.elasticsearch.xpack.core.ml.action.RevertModelSnapshotAction; import org.elasticsearch.xpack.core.ml.action.StartDatafeedAction; import org.elasticsearch.xpack.core.ml.action.StopDatafeedAction; @@ -287,6 +288,7 @@ public List> getClientActions() { PostCalendarEventsAction.INSTANCE, PersistJobAction.INSTANCE, FindFileStructureAction.INSTANCE, + ResultsIndexUpgradeAction.INSTANCE, // security ClearRealmCacheAction.INSTANCE, ClearRolesCacheAction.INSTANCE, diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeAction.java new file mode 100644 index 0000000000000..16417cf3c657f --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeAction.java @@ -0,0 +1,158 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.core.ml.action; + +import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; +import org.elasticsearch.action.support.master.MasterNodeReadRequest; +import org.elasticsearch.client.ElasticsearchClient; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndexFields; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; + + +public class ResultsIndexUpgradeAction extends Action { + public static final ResultsIndexUpgradeAction INSTANCE = new ResultsIndexUpgradeAction(); + public static final String NAME = "cluster:admin/xpack/ml/results/upgrade"; + + private ResultsIndexUpgradeAction() { + super(NAME); + } + + @Override + public AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); + } + + public static class Request + extends MasterNodeReadRequest + implements IndicesRequest, ToXContentObject { + + private static final ParseField REINDEX_BATCH_SIZE = new ParseField("reindex_batch_size"); + + public static ObjectParser PARSER = new ObjectParser<>("ml_results_index_upgrade", true, Request::new); + static { + PARSER.declareInt(Request::setReindexBatchSize, REINDEX_BATCH_SIZE); + } + + static final String INDEX = AnomalyDetectorsIndexFields.RESULTS_INDEX_PREFIX + "*"; + private int reindexBatchSize = 1000; + + /** + * Should this task store its result? + */ + private boolean shouldStoreResult; + + // for serialization + public Request() { + } + + public Request(StreamInput in) throws IOException { + super(in); + reindexBatchSize = in.readInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeInt(reindexBatchSize); + } + + @Override + public String[] indices() { + return new String[]{INDEX}; + } + + @Override + public IndicesOptions indicesOptions() { + //TODO consider lenientExpandOpen() ? + return IndicesOptions.strictExpandOpenAndForbidClosed(); + } + + /** + * Should this task store its result after it has finished? + */ + public ResultsIndexUpgradeAction.Request setShouldStoreResult(boolean shouldStoreResult) { + this.shouldStoreResult = shouldStoreResult; + return this; + } + + @Override + public boolean getShouldStoreResult() { + return shouldStoreResult; + } + + public ResultsIndexUpgradeAction.Request setReindexBatchSize(int reindexBatchSize) { + this.reindexBatchSize = reindexBatchSize; + return this; + } + + public int getReindexBatchSize() { + return reindexBatchSize; + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ResultsIndexUpgradeAction.Request request = (ResultsIndexUpgradeAction.Request) o; + return Objects.equals(reindexBatchSize, request.reindexBatchSize); + } + + @Override + public int hashCode() { + return Objects.hash(reindexBatchSize); + } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new CancellableTask(id, type, action, getDescription(), parentTaskId, headers) { + @Override + public boolean shouldCancelChildrenOnCancellation() { + return true; + } + }; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(REINDEX_BATCH_SIZE.getPreferredName(), reindexBatchSize); + builder.endObject(); + return builder; + } + } + + public static class RequestBuilder extends MasterNodeReadOperationRequestBuilder { + + public RequestBuilder(ElasticsearchClient client) { + super(client, INSTANCE, new ResultsIndexUpgradeAction.Request()); + } + } + +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeRequestTests.java new file mode 100644 index 0000000000000..70593fe303089 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeRequestTests.java @@ -0,0 +1,24 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.core.ml.action; + +import org.elasticsearch.test.AbstractStreamableTestCase; + +public class ResultsIndexUpgradeRequestTests extends AbstractStreamableTestCase { + + @Override + protected ResultsIndexUpgradeAction.Request createTestInstance() { + ResultsIndexUpgradeAction.Request request = new ResultsIndexUpgradeAction.Request(); + request.setReindexBatchSize(randomIntBetween(1, 10_000)); + request.setShouldStoreResult(randomBoolean()); + return request; + } + + @Override + protected ResultsIndexUpgradeAction.Request createBlankInstance() { + return new ResultsIndexUpgradeAction.Request(); + } +} diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java index a22660dca018f..7b89f9cb8a59b 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.reindex.ReindexPlugin; import org.elasticsearch.persistent.PersistentTaskParams; import org.elasticsearch.persistent.PersistentTaskState; import org.elasticsearch.plugins.Plugin; @@ -120,7 +121,7 @@ protected Collection> nodePlugins() { @Override protected Collection> transportClientPlugins() { - return Arrays.asList(XPackClientPlugin.class, Netty4Plugin.class); + return Arrays.asList(XPackClientPlugin.class, Netty4Plugin.class, ReindexPlugin.class); } @Override diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java new file mode 100644 index 0000000000000..6702bf4a53954 --- /dev/null +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java @@ -0,0 +1,478 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.integration; + +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.reindex.ReindexAction; +import org.elasticsearch.index.reindex.ReindexRequest; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.ml.action.GetJobsStatsAction; +import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; +import org.elasticsearch.xpack.core.ml.action.StartDatafeedAction; +import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; +import org.elasticsearch.xpack.core.ml.job.config.Job; +import org.elasticsearch.xpack.core.ml.job.config.JobState; +import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; +import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndexFields; +import org.elasticsearch.xpack.ml.ResultsIndexUpgradeService; +import org.junit.After; +import org.junit.Assert; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; + +import static org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase.createDatafeedBuilder; +import static org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase.createScheduledJob; +import static org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase.indexDocs; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.core.Is.is; + +public class ResultsIndexUpgradeIT extends MlNativeAutodetectIntegTestCase { + + private AtomicBoolean shouldContinueToIndex = new AtomicBoolean(true); + private AtomicLong dataCount = new AtomicLong(0); + + @After + public void cleanup() throws Exception { + cleanUp(); + } + + public void testMigrationWhenItIsNotNecessary() throws Exception { + String jobId1 = "no-migration-test1"; + String jobId2 = "no-migration-test2"; + String jobId3 = "no-migration-test3"; + + String dataIndex = createDataIndex().v2(); + List jobs = createJobsWithData(jobId1, jobId2, jobId3, dataIndex); + Job job1 = jobs.get(0); + Job job2 = jobs.get(1); + Job job3 = jobs.get(2); + + String job1Index = job1.getResultsIndexName(); + String job2Index = job2.getResultsIndexName(); + String job3Index = job3.getResultsIndexName(); + + assertThat(indexExists(job1Index), is(true)); + assertThat(indexExists(job2Index), is(true)); + assertThat(indexExists(job3Index), is(true)); + + long job1Total = getTotalDocCount(job1Index); + long job2Total = getTotalDocCount(job2Index); + long job3Total = getTotalDocCount(job3Index); + + AcknowledgedResponse resp = ESIntegTestCase.client().execute(ResultsIndexUpgradeAction.INSTANCE, + new ResultsIndexUpgradeAction.Request()).actionGet(); + assertThat(resp.isAcknowledged(), is(true)); + + // Migration should have done nothing + assertThat(indexExists(job1Index), is(true)); + assertThat(indexExists(job2Index), is(true)); + assertThat(indexExists(job3Index), is(true)); + + assertThat(getTotalDocCount(job1Index), equalTo(job1Total)); + assertThat(getTotalDocCount(job2Index), equalTo(job2Total)); + assertThat(getTotalDocCount(job3Index), equalTo(job3Total)); + + ClusterState state = admin().cluster().state(new ClusterStateRequest()).actionGet().getState(); + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + String[] indices = indexNameExpressionResolver.concreteIndexNames(state, + IndicesOptions.strictExpandOpenAndForbidClosed(), + AnomalyDetectorsIndex.jobResultsIndexPrefix() + "*"); + + // Our backing index size should be two as we have a shared and custom index + assertThat(indices.length, equalTo(2)); + } + + public void testMigration() throws Exception { + String jobId1 = "migration-test1"; + String jobId2 = "migration-test2"; + String jobId3 = "migration-test3"; + + String dataIndex = createDataIndex().v2(); + List jobs = createJobsWithData(jobId1, jobId2, jobId3, dataIndex); + Job job1 = jobs.get(0); + Job job2 = jobs.get(1); + Job job3 = jobs.get(2); + + String job1Index = job1.getResultsIndexName(); + String job2Index = job2.getResultsIndexName(); + String job3Index = job3.getResultsIndexName(); + + assertThat(indexExists(job1Index), is(true)); + assertThat(indexExists(job2Index), is(true)); + assertThat(indexExists(job3Index), is(true)); + + long job1Total = getJobResultsCount(job1.getId()); + long job2Total = getJobResultsCount(job2.getId()); + long job3Total = getJobResultsCount(job3.getId()); + + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + + ResultsIndexUpgradeService resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, + logger, + ThreadPool.Names.SAME, + indexMetaData -> true); + + PlainActionFuture future = PlainActionFuture.newFuture(); + + resultsIndexUpgradeService.upgrade(ESIntegTestCase.client(), + new ResultsIndexUpgradeAction.Request(), + ESIntegTestCase.client().admin().cluster().prepareState().get().getState(), + future); + + AcknowledgedResponse response = future.get(); + assertThat(response.isAcknowledged(), is(true)); + + assertThat(indexExists(job1Index), is(false)); + assertThat(indexExists(job2Index), is(false)); + assertThat(indexExists(job3Index), is(false)); + + ClusterState state = admin().cluster().state(new ClusterStateRequest()).actionGet().getState(); + String[] indices = indexNameExpressionResolver.concreteIndexNames(state, + IndicesOptions.strictExpandOpenAndForbidClosed(), + AnomalyDetectorsIndex.jobResultsIndexPrefix() + "*"); + + // Our backing index size should be four as we have a shared and custom index and upgrading doubles the number of indices + Assert.assertThat(indices.length, equalTo(4)); + + assertThat(getJobResultsCount(job1.getId()), equalTo(job1Total)); + assertThat(getJobResultsCount(job2.getId()), equalTo(job2Total)); + assertThat(getJobResultsCount(job3.getId()), equalTo(job3Total)); + + + // WE should still be able to write, and the aliases should allow to read from the appropriate indices + postDataToJob(jobId1); + long newJob1Total = getJobResultsCount(job1.getId()); + assertThat(newJob1Total, greaterThan(job1Total)); + + postDataToJob(jobId2); + long newJob2Total = getJobResultsCount(job2.getId()); + assertThat(newJob2Total, greaterThan(job2Total)); + + postDataToJob(jobId3); + long newJob3Total = getJobResultsCount(job3.getId()); + assertThat(newJob3Total, greaterThan(job3Total)); + + // We should also be able to create new jobs and old jobs should be unaffected. + String jobId4 = "migration-test4"; + Job job4 = createAndOpenJobAndStartDataFeedWithData(jobId4, dataIndex, false); + waitUntilJobIsClosed(jobId4); + + assertThat(getJobResultsCount(jobId4), greaterThan(0L)); + assertThat(getJobResultsCount(jobId1), equalTo(newJob1Total)); + assertThat(getJobResultsCount(jobId2), equalTo(newJob2Total)); + assertThat(getJobResultsCount(jobId3), equalTo(newJob3Total)); + } + + //I think this test name could be a little bit longer.... + public void testMigrationWithManuallyCreatedIndexThatNeedsMigrating() throws Exception { + String jobId1 = "migration-failure-test1"; + String jobId2 = "migration-failure-test2"; + String jobId3 = "migration-failure-test3"; + + String dataIndex = createDataIndex().v2(); + List jobs = createJobsWithData(jobId1, jobId2, jobId3, dataIndex); + Job job1 = jobs.get(0); + Job job2 = jobs.get(1); + Job job3 = jobs.get(2); + + String job1Index = job1.getResultsIndexName(); + String job2Index = job2.getResultsIndexName(); + String job3Index = job3.getResultsIndexName(); + + // This index name should match one of the automatically created migration indices + String manuallyCreatedIndex = job1Index + "-" + Version.CURRENT.major; + client().admin().indices().prepareCreate(manuallyCreatedIndex).execute().actionGet(); + + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + + ResultsIndexUpgradeService resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, + logger, + ThreadPool.Names.SAME, + indexMetaData -> true); //indicates that this manually created index needs migrated + + resultsIndexUpgradeService.upgrade(ESIntegTestCase.client(), + new ResultsIndexUpgradeAction.Request(), + ESIntegTestCase.client().admin().cluster().prepareState().get().getState(), + ActionListener.wrap( + resp -> fail(), + exception -> { + assertThat(exception, instanceOf(IllegalStateException.class)); + assertThat(exception.getMessage(), + equalTo("Index [" + manuallyCreatedIndex + "] already exists and is not the current version.")); + } + )); + } + + public void testMigrationWithExistingIndexWithData() throws Exception { + String jobId1 = "partial-migration-test1"; + String jobId2 = "partial-migration-test2"; + String jobId3 = "partial-migration-test3"; + + String dataIndex = createDataIndex().v2(); + List jobs = createJobsWithData(jobId1, jobId2, jobId3, dataIndex); + Job job1 = jobs.get(0); + Job job2 = jobs.get(1); + Job job3 = jobs.get(2); + + String job1Index = job1.getResultsIndexName(); + String job2Index = job2.getResultsIndexName(); + String job3Index = job3.getResultsIndexName(); + + assertThat(indexExists(job1Index), is(true)); + assertThat(indexExists(job2Index), is(true)); + assertThat(indexExists(job3Index), is(true)); + + long job1Total = getJobResultsCount(job1.getId()); + long job2Total = getJobResultsCount(job2.getId()); + long job3Total = getJobResultsCount(job3.getId()); + + //lets manually create a READ index with reindexed data already + // Should still get aliased appropriately without any additional/duplicate data. + String alreadyMigratedIndex = job1Index + "-" + Version.CURRENT.major + "r"; + ReindexRequest reindexRequest = new ReindexRequest(); + reindexRequest.setSourceIndices(job1Index); + reindexRequest.setDestIndex(alreadyMigratedIndex); + client().execute(ReindexAction.INSTANCE, reindexRequest).actionGet(); + + //New write index as well, should still get aliased appropriately + String alreadyMigratedWriteIndex = job1Index + "-" + Version.CURRENT.major; + client().admin().indices().prepareCreate(alreadyMigratedWriteIndex).execute().actionGet(); + + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + + ResultsIndexUpgradeService resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, + logger, + ThreadPool.Names.SAME, + //indicates that this manually created index is already migrated and should not be included in our migration steps + indexMetaData -> !(indexMetaData.getIndex().getName().equals(alreadyMigratedIndex) || + indexMetaData.getIndex().getName().equals(alreadyMigratedWriteIndex))); + + PlainActionFuture future = PlainActionFuture.newFuture(); + + resultsIndexUpgradeService.upgrade(ESIntegTestCase.client(), + new ResultsIndexUpgradeAction.Request(), + ESIntegTestCase.client().admin().cluster().prepareState().get().getState(), + future); + + AcknowledgedResponse response = future.get(); + assertThat(response.isAcknowledged(), is(true)); + + assertThat(indexExists(job1Index), is(false)); + assertThat(indexExists(job2Index), is(false)); + assertThat(indexExists(job3Index), is(false)); + + ClusterState state = admin().cluster().state(new ClusterStateRequest()).actionGet().getState(); + String[] indices = indexNameExpressionResolver.concreteIndexNames(state, + IndicesOptions.strictExpandOpenAndForbidClosed(), + AnomalyDetectorsIndex.jobResultsIndexPrefix() + "*"); + + // Our backing index size should be four as we have a shared and custom index and upgrading doubles the number of indices + Assert.assertThat(indices.length, equalTo(4)); + + assertThat(getJobResultsCount(job1.getId()), equalTo(job1Total)); + assertThat(getJobResultsCount(job2.getId()), equalTo(job2Total)); + assertThat(getJobResultsCount(job3.getId()), equalTo(job3Total)); + + // WE should still be able to write, and the aliases should allow to read from the appropriate indices + postDataToJob(jobId1); + long newJob1Total = getJobResultsCount(job1.getId()); + assertThat(newJob1Total, greaterThan(job1Total)); + + postDataToJob(jobId2); + long newJob2Total = getJobResultsCount(job2.getId()); + assertThat(newJob2Total, greaterThan(job2Total)); + + postDataToJob(jobId3); + long newJob3Total = getJobResultsCount(job3.getId()); + assertThat(newJob3Total, greaterThan(job3Total)); + } + + public void testMigrationWithOpenJob() throws Exception { + dataCount.set(0); + shouldContinueToIndex.set(true); + Tuple amountAndIndex = createDataIndex(); + String dataIndex = amountAndIndex.v2(); + dataCount.set(amountAndIndex.v1()); + Job closedJob = createAndOpenJobAndStartDataFeedWithData("test-migration-open-job-closed", dataIndex, false); + Job openedJob = createAndOpenJobAndDataFeedWithDataAndNoEnd( + "test-migration-open-job-opened", + "data-for-migration-1", + false); + long closedJobTotal = getJobResultsCount(closedJob.getId()); + + Thread puttingData = new Thread(() -> { + while (shouldContinueToIndex.get()) { + try { + Thread.sleep(1000); + long indexed = indexSomeDocs("data-for-migration-1"); + dataCount.addAndGet(indexed); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + } + }, "testMigrationWithOpenJobIndexer"); + puttingData.start(); + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + + ResultsIndexUpgradeService resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, + logger, + ThreadPool.Names.SAME, + indexMetaData -> true); + + PlainActionFuture future = PlainActionFuture.newFuture(); + + resultsIndexUpgradeService.upgrade(ESIntegTestCase.client(), + new ResultsIndexUpgradeAction.Request(), + ESIntegTestCase.client().admin().cluster().prepareState().get().getState(), + future); + + AcknowledgedResponse response = future.get(); + assertThat(response.isAcknowledged(), is(true)); + + ClusterState state = admin().cluster().state(new ClusterStateRequest()).actionGet().getState(); + String[] indices = indexNameExpressionResolver.concreteIndexNames(state, + IndicesOptions.strictExpandOpenAndForbidClosed(), + AnomalyDetectorsIndex.jobResultsIndexPrefix() + "*"); + + // Our backing index size should be four as we have a shared and custom index and upgrading doubles the number of indices + Assert.assertThat(indices.length, equalTo(2)); + + assertThat(getJobResultsCount(closedJob.getId()), equalTo(closedJobTotal)); + + shouldContinueToIndex.set(false); + puttingData.join(); + flushJob(openedJob.getId(), true); + assertBusy(() -> { + GetJobsStatsAction.Response.JobStats stats = getJobStats(openedJob.getId()).get(0); + assertThat(stats.getDataCounts().getInputRecordCount(), equalTo(dataCount.get())); + }); + + assertThat(getJobResultsCount(openedJob.getId()), greaterThan(closedJobTotal)); + + stopDatafeed(openedJob.getId() + "-datafeed"); + closeJob(openedJob.getId()); + waitUntilJobIsClosed(openedJob.getId()); + } + + private long indexSomeDocs(String index) { + long numDocs = ESTestCase.randomIntBetween(15, 30); + long now = System.currentTimeMillis(); + long abitAgo = now - 500; + + indexDocs(logger, index, numDocs, abitAgo, now); + return numDocs; + } + + private long getTotalDocCount(String indexName) { + SearchResponse searchResponse = ESIntegTestCase.client().prepareSearch(indexName) + .setSize(10_000) + .setQuery(QueryBuilders.matchAllQuery()) + .execute().actionGet(); + return searchResponse.getHits().getTotalHits().value; + } + + private long getJobResultsCount(String jobId) { + String index = AnomalyDetectorsIndexFields.RESULTS_INDEX_PREFIX + jobId; + return getTotalDocCount(index); + } + + private void postDataToJob(String jobId) throws Exception { + openJob(jobId); + ESTestCase.assertBusy(() -> Assert.assertEquals(getJobStats(jobId).get(0).getState(), JobState.OPENED)); + startDatafeed(jobId + "-datafeed", 0L, System.currentTimeMillis()); + waitUntilJobIsClosed(jobId); + } + + private Job createAndOpenJobAndStartDataFeedWithData(String jobId, String dataIndex, boolean isCustom) throws Exception { + Job.Builder jobbuilder = createScheduledJob(jobId); + if (isCustom) { + jobbuilder.setResultsIndexName(jobId); + } + registerJob(jobbuilder); + + Job job = putJob(jobbuilder).getResponse(); + + openJob(job.getId()); + ESTestCase.assertBusy(() -> Assert.assertEquals(getJobStats(job.getId()).get(0).getState(), JobState.OPENED)); + + DatafeedConfig.Builder builder = createDatafeedBuilder(job.getId() + "-datafeed", + job.getId(), + Collections.singletonList(dataIndex)); + builder.setQueryDelay(TimeValue.timeValueSeconds(5)); + builder.setFrequency(TimeValue.timeValueSeconds(5)); + DatafeedConfig datafeedConfig = builder.build(); + registerDatafeed(datafeedConfig); + putDatafeed(datafeedConfig); + startDatafeed(datafeedConfig.getId(), 0L, System.currentTimeMillis()); + waitUntilJobIsClosed(jobId); + return job; + } + + private Job createAndOpenJobAndDataFeedWithDataAndNoEnd(String jobId, String dataIndex, boolean isCustom) throws Exception { + Job.Builder jobbuilder = createScheduledJob(jobId); + if (isCustom) { + jobbuilder.setResultsIndexName(jobId); + } + registerJob(jobbuilder); + + Job job = putJob(jobbuilder).getResponse(); + + openJob(job.getId()); + ESTestCase.assertBusy(() -> Assert.assertEquals(getJobStats(job.getId()).get(0).getState(), JobState.OPENED)); + + DatafeedConfig.Builder builder = createDatafeedBuilder(job.getId() + "-datafeed", + job.getId(), + Collections.singletonList(dataIndex)); + builder.setQueryDelay(TimeValue.timeValueSeconds(5)); + builder.setFrequency(TimeValue.timeValueSeconds(5)); + DatafeedConfig datafeedConfig = builder.build(); + registerDatafeed(datafeedConfig); + putDatafeed(datafeedConfig); + StartDatafeedAction.Request request = new StartDatafeedAction.Request(datafeedConfig.getId(), 0L); + client().execute(StartDatafeedAction.INSTANCE, request).actionGet(); + return job; + } + + private Tuple createDataIndex() { + ESIntegTestCase.client().admin().indices().prepareCreate("data-for-migration-1") + .addMapping("type", "time", "type=date") + .get(); + long numDocs = ESTestCase.randomIntBetween(32, 512); + long now = System.currentTimeMillis(); + long oneWeekAgo = now - 604800000; + long twoWeeksAgo = oneWeekAgo - 604800000; + indexDocs(logger, "data-for-migration-1", numDocs, twoWeeksAgo, oneWeekAgo); + return new Tuple<>(numDocs, "data-for-migration-1"); + } + + private List createJobsWithData(String sharedJobId1, String sharedJobId2, String customJobId, String dataIndex) throws Exception { + + Job job1 = createAndOpenJobAndStartDataFeedWithData(sharedJobId1, dataIndex, false); + Job job2 = createAndOpenJobAndStartDataFeedWithData(sharedJobId2, dataIndex, false); + Job job3 = createAndOpenJobAndStartDataFeedWithData(customJobId, dataIndex, true); + + return Arrays.asList(job1, job2, job3); + } +} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java index 19650b348a2eb..d9600b5975a19 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java @@ -95,6 +95,7 @@ import org.elasticsearch.xpack.core.ml.action.PutDatafeedAction; import org.elasticsearch.xpack.core.ml.action.PutFilterAction; import org.elasticsearch.xpack.core.ml.action.PutJobAction; +import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; import org.elasticsearch.xpack.core.ml.action.RevertModelSnapshotAction; import org.elasticsearch.xpack.core.ml.action.StartDatafeedAction; import org.elasticsearch.xpack.core.ml.action.StopDatafeedAction; @@ -149,6 +150,7 @@ import org.elasticsearch.xpack.ml.action.TransportPutDatafeedAction; import org.elasticsearch.xpack.ml.action.TransportPutFilterAction; import org.elasticsearch.xpack.ml.action.TransportPutJobAction; +import org.elasticsearch.xpack.ml.action.TransportResultsIndexUpgradeAction; import org.elasticsearch.xpack.ml.action.TransportRevertModelSnapshotAction; import org.elasticsearch.xpack.ml.action.TransportStartDatafeedAction; import org.elasticsearch.xpack.ml.action.TransportStopDatafeedAction; @@ -224,6 +226,7 @@ import org.elasticsearch.xpack.ml.rest.results.RestGetInfluencersAction; import org.elasticsearch.xpack.ml.rest.results.RestGetOverallBucketsAction; import org.elasticsearch.xpack.ml.rest.results.RestGetRecordsAction; +import org.elasticsearch.xpack.ml.rest.results.RestUpgradeResultsAction; import org.elasticsearch.xpack.ml.rest.validate.RestValidateDetectorAction; import org.elasticsearch.xpack.ml.rest.validate.RestValidateJobConfigAction; @@ -513,7 +516,8 @@ public List getRestHandlers(Settings settings, RestController restC new RestPutCalendarJobAction(settings, restController), new RestGetCalendarEventsAction(settings, restController), new RestPostCalendarEventAction(settings, restController), - new RestFindFileStructureAction(settings, restController) + new RestFindFileStructureAction(settings, restController), + new RestUpgradeResultsAction(settings, restController) ); } @@ -571,7 +575,8 @@ public List getRestHandlers(Settings settings, RestController restC new ActionHandler<>(GetCalendarEventsAction.INSTANCE, TransportGetCalendarEventsAction.class), new ActionHandler<>(PostCalendarEventsAction.INSTANCE, TransportPostCalendarEventsAction.class), new ActionHandler<>(PersistJobAction.INSTANCE, TransportPersistJobAction.class), - new ActionHandler<>(FindFileStructureAction.INSTANCE, TransportFindFileStructureAction.class) + new ActionHandler<>(FindFileStructureAction.INSTANCE, TransportFindFileStructureAction.class), + new ActionHandler<>(ResultsIndexUpgradeAction.INSTANCE, TransportResultsIndexUpgradeAction.class) ); } @Override diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java new file mode 100644 index 0000000000000..85a5d2d178496 --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java @@ -0,0 +1,516 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml; + +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ElasticsearchTimeoutException; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.ResourceAlreadyExistsException; +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.indices.alias.Alias; +import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; +import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; +import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; +import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; +import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.reindex.BulkByScrollResponse; +import org.elasticsearch.index.reindex.ReindexAction; +import org.elasticsearch.index.reindex.ReindexRequest; +import org.elasticsearch.index.reindex.ScrollableHitSource; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; +import org.elasticsearch.xpack.ml.utils.TypedChainTaskExecutor; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; +import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; + +/** + * ML Job results index upgrade service + */ +public class ResultsIndexUpgradeService { + + // Adjust the following constants as necessary for various versions and backports. + private static final int INDEX_VERSION = Version.CURRENT.major; + private static final Version UPGRADE_INTRODUCED = Version.CURRENT.minimumCompatibilityVersion(); + + private final IndexNameExpressionResolver indexNameExpressionResolver; + private final Predicate shouldUpgrade; + private final String executor; + private final Logger logger; + + /** + * Construct a new upgrade service + * + * @param indexNameExpressionResolver Index expression resolver for the request + * @param logger The logger of the creating object + * @param executor Where to execute client calls + * @param shouldUpgrade Given IndexMetadata indicate if it should be upgraded or not + * {@code true} indicates that it SHOULD upgrade + */ + public ResultsIndexUpgradeService(IndexNameExpressionResolver indexNameExpressionResolver, + Logger logger, + String executor, + Predicate shouldUpgrade) { + this.indexNameExpressionResolver = indexNameExpressionResolver; + this.shouldUpgrade = shouldUpgrade; + this.executor = executor; + this.logger = logger; + } + + //Was the index created in the current major version + public static boolean checkInternalIndexVersion(IndexMetaData indexMetaData) { + return indexMetaData.getCreationVersion().major == Version.CURRENT.major; + } + + /** + * There are two reasons for these indices to exist: + * 1. The upgrade process has ran before and either failed for some reason, or the end user is simply running it again. + * Either way, it should be ok to proceed as this action SHOULD be idempotent, + * unless the shouldUpgrade predicate is poorly formed + * 2. This index was created manually by the user. If the index was created manually and actually needs upgraded, then + * we consider the "new index" to be invalid as it is not the upgraded. + * + * @param metaData Cluster metadata + * @param newIndexName The index to check + * @param shouldUpgrade Should be index be upgraded + * @return {@code true} if the "new index" is not considered valid + */ + private static boolean invalidIndexForUpgrade(MetaData metaData, String newIndexName, Predicate shouldUpgrade) { + return metaData.hasIndex(newIndexName) && shouldUpgrade.test(metaData.index(newIndexName)); + } + + // Are all the nodes the appropriate version for us to do the upgrade? + private static void checkMasterAndDataNodeVersion(ClusterState clusterState) { + if (clusterState.nodes().getMinNodeVersion().before(UPGRADE_INTRODUCED)) { + throw new IllegalStateException("All nodes should have at least version [" + UPGRADE_INTRODUCED + "] to upgrade"); + } + } + + //This method copies the behavior of the normal {index}/_upgrade rest response handler + private static Tuple getStatusAndCause(BulkByScrollResponse response) { + /* + * Return the highest numbered rest status under the assumption that higher numbered statuses are "more error" + * and thus more interesting to the user. + */ + RestStatus status = RestStatus.OK; + Throwable cause = null; + if (response.isTimedOut()) { + status = RestStatus.REQUEST_TIMEOUT; + cause = new ElasticsearchTimeoutException("Reindex request timed out"); + } + for (BulkItemResponse.Failure failure : response.getBulkFailures()) { + if (failure.getStatus().getStatus() > status.getStatus()) { + status = failure.getStatus(); + cause = failure.getCause(); + } + } + for (ScrollableHitSource.SearchFailure failure : response.getSearchFailures()) { + RestStatus failureStatus = ExceptionsHelper.status(failure.getReason()); + if (failureStatus.getStatus() > status.getStatus()) { + status = failureStatus; + cause = failure.getReason(); + } + } + return new Tuple<>(status, cause); + } + + /** + * Upgrade the indices given in the request. + * + * @param client The client to use when making calls + * @param request The upgrade request + * @param state The current cluster state + * @param listener The listener to alert when actions have completed + */ + public void upgrade(Client client, ResultsIndexUpgradeAction.Request request, ClusterState state, + ActionListener listener) { + try { + checkMasterAndDataNodeVersion(state); + String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request.indicesOptions(), request.indices()); + MetaData metaData = state.getMetaData(); + + List indicesToUpgrade = Arrays.stream(concreteIndices) + .filter(indexName -> shouldUpgrade.test(metaData.index(indexName))) + .collect(Collectors.toList()); + + // All the internal indices are up to date + if (indicesToUpgrade.isEmpty()) { + listener.onResponse(new AcknowledgedResponse(true)); + return; + } + + IndexNameAndAliasProvider indexNameAndAliasProvider = new IndexNameAndAliasProvider(indicesToUpgrade, metaData); + Exception validationException = indexNameAndAliasProvider.validate(metaData, shouldUpgrade); + if (validationException != null) { + listener.onFailure(validationException); + return; + } + + // <7> Now that we have deleted the old indices, we are complete, alert the user + ActionListener deleteIndicesListener = ActionListener.wrap( + listener::onResponse, + error -> { + logger.error("Failed to delete old indices: " + Strings.collectionToCommaDelimitedString(indicesToUpgrade), + error); + listener.onResponse(new AcknowledgedResponse(true)); + } + ); + + // <6> Now that aliases are moved, need to delete the old indices + ActionListener readAliasListener = ActionListener.wrap( + resp -> deleteOldIndices(client, indicesToUpgrade, deleteIndicesListener), + error -> { + String msg = "Failed adjusting aliases from old indices to new " + error.getMessage(); + logger.error(msg, error); + listener.onFailure(new ElasticsearchException(msg, error)); + } + ); + + // <5> Documents are now reindexed, time to move read aliases + ActionListener reindexListener = ActionListener.wrap( + resp -> + // Need to make indices writable again so that the aliases can be removed from them + removeReadOnlyBlock(client, indicesToUpgrade, + ActionListener.wrap( + rrob -> adjustAliases(client, + indexNameAndAliasProvider.oldIndicesWithReadAliases(), + indexNameAndAliasProvider.newReadIndicesWithReadAliases(), + readAliasListener), + rrobFailure -> { + String msg = "Failed making old indices writable again so that aliases can be moved. " + + rrobFailure.getMessage(); + logger.error(msg, rrobFailure); + listener.onFailure(new ElasticsearchException(msg, rrobFailure)); + }) + ), + error -> { + logger.error("Failed to reindex old read-only indices", error); + removeReadOnlyBlock(client, indicesToUpgrade, ActionListener.wrap( + empty -> listener.onFailure(error), + removeReadOnlyBlockError -> { + String msg = "Failed making old indices read/write again after failing to reindex: " + error.getMessage(); + logger.error(msg, removeReadOnlyBlockError); + listener.onFailure(new ElasticsearchException(msg, removeReadOnlyBlockError)); + } + )); + } + ); + + // <4> Old indexes are now readOnly, Time to reindex + ActionListener readOnlyListener = ActionListener.wrap( + ack -> reindexOldReadIndicesToNewIndices(client, indexNameAndAliasProvider.needsReindex(), request, reindexListener), + listener::onFailure + ); + + // <3> Set old indices to readOnly + ActionListener writeAliasesMovedListener = ActionListener.wrap( + resp -> setReadOnlyBlock(client, indicesToUpgrade, readOnlyListener), + listener::onFailure + ); + + // <2> Move write index alias to new write indices + ActionListener createWriteIndicesAndSetReadAliasListener = ActionListener.wrap( + resp -> adjustAliases(client, + indexNameAndAliasProvider.oldIndicesWithWriteAliases(), + indexNameAndAliasProvider.newWriteIndicesWithWriteAliases(), + writeAliasesMovedListener), + listener::onFailure + ); + + // <1> Create the new write indices and set the read aliases to include them + createNewWriteIndicesIfNecessary(client, metaData, indexNameAndAliasProvider.newWriteIndices(), + ActionListener.wrap( + indicesCreated -> adjustAliases(client, + Collections.emptyMap(), + indexNameAndAliasProvider.newWriteIndicesWithReadAliases(), + createWriteIndicesAndSetReadAliasListener), + listener::onFailure + )); + + } catch (Exception e) { + listener.onFailure(e); + } + + } + + private void createNewWriteIndicesIfNecessary(Client client, + MetaData metaData, + Collection newWriteIndices, + ActionListener createIndexListener) { + TypedChainTaskExecutor chainTaskExecutor = + new TypedChainTaskExecutor<>( + client.threadPool().executor(executor), + (createIndexResponse -> true), //We always want to complete all our tasks + (exception -> + // Short circuit execution IF the exception is NOT a ResourceAlreadyExistsException + // This should be rare, as it requires the index to be created between our previous check and this exception + exception instanceof ResourceAlreadyExistsException == false + )); + newWriteIndices.forEach((index) -> { + // If the index already exists, don't try and created it + // We have already verified that IF this index exists, that it does not require upgrading + // So, if it was created between that check and this one, we can assume it is the correct version as it was JUST created + if (metaData.hasIndex(index) == false) { + CreateIndexRequest request = new CreateIndexRequest(index); + chainTaskExecutor.add(listener -> + executeAsyncWithOrigin(client.threadPool().getThreadContext(), + ML_ORIGIN, + request, + listener, + client.admin().indices()::create)); + } + }); + + chainTaskExecutor.execute(ActionListener.wrap( + createIndexResponses -> createIndexListener.onResponse(true), + createIndexListener::onFailure + )); + } + + /** + * Makes the indices readonly if it's not set as a readonly yet + */ + private void setReadOnlyBlock(Client client, List indices, ActionListener listener) { + Settings settings = Settings.builder().put(IndexMetaData.INDEX_READ_ONLY_SETTING.getKey(), true).build(); + UpdateSettingsRequest request = new UpdateSettingsRequest(indices.toArray(new String[0])); + request.settings(settings); + executeAsyncWithOrigin(client.threadPool().getThreadContext(), + ML_ORIGIN, + request, + listener, + client.admin().indices()::updateSettings); + } + + private void removeReadOnlyBlock(Client client, List indices, + ActionListener listener) { + Settings settings = Settings.builder().put(IndexMetaData.INDEX_READ_ONLY_SETTING.getKey(), false).build(); + UpdateSettingsRequest request = new UpdateSettingsRequest(indices.toArray(new String[0])); + request.settings(settings); + executeAsyncWithOrigin(client.threadPool().getThreadContext(), + ML_ORIGIN, + request, + listener, + client.admin().indices()::updateSettings); + } + + private void reindexOldReadIndicesToNewIndices(Client client, + Map reindexIndices, + ResultsIndexUpgradeAction.Request request, + ActionListener listener) { + TypedChainTaskExecutor chainTaskExecutor = + new TypedChainTaskExecutor<>( + client.threadPool().executor(executor), + (createIndexResponse) -> { // If there are errors in the reindex, we should stop + Tuple status = getStatusAndCause(createIndexResponse); + if (status.v1().equals(RestStatus.OK)) { + return true; + } else { + // Throw an unchecked exception to keep from executing the rest of the reindexing requests + // and return the failure to the end user. + throw new ElasticsearchException(status.v2()); + } + }, + (exception -> true)); // Short circuit and call onFailure for any exception + + List newIndices = new ArrayList<>(reindexIndices.size()); + reindexIndices.forEach((oldIndex, newIndex) -> { + ReindexRequest reindexRequest = new ReindexRequest(); + reindexRequest.setSourceBatchSize(request.getReindexBatchSize()); + reindexRequest.setSourceIndices(oldIndex); + reindexRequest.setDestIndex(newIndex); + // Don't worry if these indices already exist, we validated index.format earlier + reindexRequest.setConflicts("proceed"); + // If the document exists already in the new index, don't want to update or overwrite as we are pulling from "old data" + reindexRequest.setDestOpType("create"); + newIndices.add(newIndex); + chainTaskExecutor.add(chainedListener -> + executeAsyncWithOrigin(client, + ML_ORIGIN, + ReindexAction.INSTANCE, + reindexRequest, + chainedListener)); + }); + + chainTaskExecutor.execute(ActionListener.wrap( + bulkScrollingResponses -> listener.onResponse(true), + failure -> { + logger.error("Failed to re-index documents"); + List createdIndices = newIndices.subList(0, chainTaskExecutor.getCollectedResponses().size()); + logger.error( + "Failed to reindex all old read indices. Successfully reindexed: [" + + Strings.collectionToCommaDelimitedString(createdIndices) + "]", + failure); + listener.onFailure(failure); + } + )); + + } + + private void deleteOldIndices(Client client, + List oldIndices, + ActionListener deleteIndicesListener) { + DeleteIndexRequest request = new DeleteIndexRequest(oldIndices.toArray(new String[0])); + request.indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN); + executeAsyncWithOrigin(client.threadPool().getThreadContext(), + ML_ORIGIN, + request, + deleteIndicesListener, + client.admin().indices()::delete); + } + + private void adjustAliases(Client client, + Map> oldAliases, + Map> newAliases, + ActionListener indicesAliasListener) { + IndicesAliasesRequest indicesAliasesRequest = new IndicesAliasesRequest(); + oldAliases.forEach((oldIndex, aliases) -> + { + if (aliases.isEmpty() == false) { //if the aliases are empty, that means there are none to remove + indicesAliasesRequest.addAliasAction(IndicesAliasesRequest + .AliasActions + .remove() + .index(oldIndex) + .aliases(aliases.stream().map(Alias::name).toArray(String[]::new))); + } + } + ); + newAliases.forEach((newIndex, aliases) -> + aliases.forEach(alias -> { + IndicesAliasesRequest.AliasActions action = IndicesAliasesRequest.AliasActions.add().index(newIndex); + if (alias.filter() != null) { + action.filter(alias.filter()); + } + action.alias(alias.name()); + indicesAliasesRequest.addAliasAction(action); + }) + ); + executeAsyncWithOrigin(client.threadPool().getThreadContext(), + ML_ORIGIN, + indicesAliasesRequest, + indicesAliasListener, + client.admin().indices()::aliases); + } + + + private static class IndexNameAndAliasProvider { + + private final List oldIndices; + private final Map> writeAliases = new HashMap<>(); + private final Map> readAliases = new HashMap<>(); + + private IndexNameAndAliasProvider(List oldIndices, MetaData metaData) { + this.oldIndices = oldIndices; + oldIndices.forEach(index -> { + IndexMetaData indexMetaData = metaData.index(index); + List writes = new ArrayList<>(); + List reads = new ArrayList<>(); + indexMetaData.getAliases().forEach(aliasCursor -> { + Alias alias = new Alias(aliasCursor.value.alias()); + if (aliasCursor.value.filteringRequired()) { + alias.filter(aliasCursor.value.getFilter().string()); //Set the read alias jobId filter + } + if (alias.name().contains(".write-")) { + writes.add(alias); + } else { + reads.add(alias); + } + }); + + writeAliases.put(index, writes); + readAliases.put(index, reads); + }); + } + + private Exception validate(MetaData metaData, Predicate shouldUpgrade) { + for (String index : oldIndices) { + IndexMetaData indexMetaData = metaData.index(index); + + if (indexMetaData.getState() != IndexMetaData.State.OPEN) { + return new IllegalStateException("unable to upgrade a closed index[" + index + "]"); + } + String newWriteName = newWriteName(index); + // If the "new" indices exist, either they were created from a previous run of the upgrade process or the end user + if (invalidIndexForUpgrade(metaData, newWriteName, shouldUpgrade)) { + return new IllegalStateException("Index [" + newWriteName + "] already exists and is not the current version."); + } + + String newReadName = newReadName(index); + if (invalidIndexForUpgrade(metaData, newReadName, shouldUpgrade)) { + return new IllegalStateException("Index [" + newReadName + "] already exists and is not the current version."); + } + } + return null; + } + + private String newReadName(String oldIndexName) { + return oldIndexName + "-" + INDEX_VERSION + "r"; + } + + private String newWriteName(String oldIndexName) { + return oldIndexName + "-" + INDEX_VERSION; + } + + private List newWriteIndices() { + return oldIndices.stream().map(this::newWriteName).collect(Collectors.toList()); + } + + private List readAliases(String oldIndex) { + return readAliases.get(oldIndex); + } + + private List writeAliases(String oldIndex) { + return writeAliases.get(oldIndex); + } + + private Map> newWriteIndicesWithReadAliases() { + return oldIndices.stream().collect(Collectors.toMap(this::newWriteName, this::readAliases)); + } + + private Map> oldIndicesWithWriteAliases() { + return writeAliases; + } + + private Map> newWriteIndicesWithWriteAliases() { + return oldIndices.stream().collect(Collectors.toMap(this::newWriteName, this::writeAliases)); + } + + private Map> oldIndicesWithReadAliases() { + return readAliases; + } + + private Map> newReadIndicesWithReadAliases() { + return oldIndices.stream().collect(Collectors.toMap(this::newReadName, this::readAliases)); + } + + private Map needsReindex() { + return oldIndices.stream().collect(Collectors.toMap(Function.identity(), this::newReadName)); + } + } +} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportResultsIndexUpgradeAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportResultsIndexUpgradeAction.java new file mode 100644 index 0000000000000..c9b33df2074b7 --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportResultsIndexUpgradeAction.java @@ -0,0 +1,80 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.action; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.ParentTaskAssigningClient; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; +import org.elasticsearch.xpack.ml.ResultsIndexUpgradeService; + +import static org.elasticsearch.xpack.ml.ResultsIndexUpgradeService.checkInternalIndexVersion; + +public class TransportResultsIndexUpgradeAction + extends TransportMasterNodeReadAction { + + private final Client client; + private final ResultsIndexUpgradeService resultsIndexUpgradeService; + + @Inject + public TransportResultsIndexUpgradeAction(TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, Client client, + IndexNameExpressionResolver indexNameExpressionResolver) { + super(ResultsIndexUpgradeAction.NAME, transportService, clusterService, threadPool, + actionFilters, ResultsIndexUpgradeAction.Request::new, indexNameExpressionResolver); + this.client = client; + this.resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, + logger, + executor(), + indexMetadata -> checkInternalIndexVersion(indexMetadata) == false); + } + + @Override + protected void masterOperation(Task task, ResultsIndexUpgradeAction.Request request, ClusterState state, + ActionListener listener) { + TaskId taskId = new TaskId(clusterService.localNode().getId(), task.getId()); + ParentTaskAssigningClient parentAwareClient = new ParentTaskAssigningClient(client, taskId); + try { + resultsIndexUpgradeService.upgrade(parentAwareClient, request, state, listener); + } catch (Exception e) { + listener.onFailure(e); + } + } + + @Override + protected final void masterOperation(ResultsIndexUpgradeAction.Request request, ClusterState state, + ActionListener listener) { + throw new UnsupportedOperationException("the task parameter is required"); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); + } + + @Override + protected ClusterBlockException checkBlock(ResultsIndexUpgradeAction.Request request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ); + } +} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java new file mode 100644 index 0000000000000..8fefafb048960 --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java @@ -0,0 +1,65 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.rest.results; + +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.BytesRestResponse; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.action.RestToXContentListener; +import org.elasticsearch.tasks.LoggingTaskListener; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; +import org.elasticsearch.xpack.ml.MachineLearning; + +import java.io.IOException; + +public class RestUpgradeResultsAction extends BaseRestHandler { + + public RestUpgradeResultsAction(Settings settings, RestController controller) { + super(settings); + controller.registerHandler(RestRequest.Method.POST, + MachineLearning.BASE_PATH + "anomaly_detectors/results/_upgrade", + this); + } + + @Override + public String getName() { + return "xpack_ml_upgrade_results_indices_action"; + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { + ResultsIndexUpgradeAction.Request parsedRequest = new ResultsIndexUpgradeAction.Request(); + if (restRequest.hasContent()) { + XContentParser parser = restRequest.contentParser(); + parsedRequest = ResultsIndexUpgradeAction.Request.PARSER.apply(parser, null); + } + final ResultsIndexUpgradeAction.Request upgradeRequest = parsedRequest; + + if (restRequest.paramAsBoolean("wait_for_completion", false)) { + return channel -> client.execute(ResultsIndexUpgradeAction.INSTANCE, upgradeRequest, new RestToXContentListener<>(channel)); + } else { + upgradeRequest.setShouldStoreResult(true); + + Task task = client.executeLocally(ResultsIndexUpgradeAction.INSTANCE, upgradeRequest, LoggingTaskListener.instance()); + // Send task description id instead of waiting for the message + return channel -> { + try (XContentBuilder builder = channel.newBuilder()) { + builder.startObject(); + builder.field("task", client.getLocalNodeId() + ":" + task.getId()); + builder.endObject(); + channel.sendResponse(new BytesRestResponse(RestStatus.OK, builder)); + } + }; + } + } +} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java new file mode 100644 index 0000000000000..1cfe6a31ceeed --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java @@ -0,0 +1,125 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.utils; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ExecutorService; +import java.util.function.Predicate; + +/** + * A utility that allows chained (serial) execution of a number of tasks + * in async manner. + */ +public class TypedChainTaskExecutor { + + public interface ChainTask { + void run(ActionListener listener); + } + + private final ExecutorService executorService; + private final LinkedList> tasks = new LinkedList<>(); + private final Predicate failureShortCircuitPredicate; + private final Predicate continuationPredicate; + private final List collectedResponses; + + /** + * Creates a new TypedChainTaskExecutor. + * Each chainedTask is executed in order serially and after each execution the continuationPredicate is tested. + * + * On failures teh failureShortCircuitPredicate is tested. + * + * @param executorService The service where to execute the tasks + * @param continuationPredicate The predicate to test on whether to execute the next task or not. + * {@code true} means continue on to the next task. + * Must be able to handle null values. + * @param failureShortCircuitPredicate The predicate on whether to short circuit execution on a give exception. + * {@code true} means that no more tasks should execute and the the listener::onFailure should be + * called. + */ + public TypedChainTaskExecutor(ExecutorService executorService, + Predicate continuationPredicate, + Predicate failureShortCircuitPredicate) { + this.executorService = Objects.requireNonNull(executorService); + this.continuationPredicate = continuationPredicate; + this.failureShortCircuitPredicate = failureShortCircuitPredicate; + this.collectedResponses = new ArrayList<>(); + } + + public synchronized void add(ChainTask task) { + tasks.add(task); + } + + private synchronized void execute(T previousValue, ActionListener> listener) { + collectedResponses.add(previousValue); + if (continuationPredicate.test(previousValue)) { + if (tasks.isEmpty()) { + listener.onResponse(Collections.unmodifiableList(new ArrayList<>(collectedResponses))); + return; + } + ChainTask task = tasks.pop(); + executorService.execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + if (failureShortCircuitPredicate.test(e)) { + listener.onFailure(e); + } else { + execute(null, listener); + } + } + + @Override + protected void doRun() { + task.run(ActionListener.wrap(value -> execute(value, listener), this::onFailure)); + } + }); + } else { + listener.onResponse(Collections.unmodifiableList(new ArrayList<>(collectedResponses))); + } + } + + /** + * Execute all the chained tasks serially, notify listener when completed + * + * @param listener The ActionListener to notify when all executions have been completed, + * or when no further tasks should be executed. + * The resulting list COULD contain null values depending on if execution is continued + * on exceptions or not. + */ + public synchronized void execute(ActionListener> listener) { + if (tasks.isEmpty()) { + listener.onResponse(Collections.emptyList()); + return; + } + collectedResponses.clear(); + ChainTask task = tasks.pop(); + executorService.execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + if (failureShortCircuitPredicate.test(e)) { + listener.onFailure(e); + } else { + execute(null, listener); + } + } + + @Override + protected void doRun() { + task.run(ActionListener.wrap(value -> execute(value, listener), this::onFailure)); + } + }); + } + + public synchronized List getCollectedResponses() { + return Collections.unmodifiableList(new ArrayList<>(collectedResponses)); + } +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade_job_results.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade_job_results.json new file mode 100644 index 0000000000000..94fb0d89b1e59 --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade_job_results.json @@ -0,0 +1,21 @@ +{ + "ml.upgrade_job_results": { + "documentation": "TODO", + "methods": [ "POST" ], + "url": { + "path": "/_ml/anomaly_detectors/results/_upgrade", + "paths": [ "/_ml/anomaly_detectors/results/_upgrade" ], + "params": { + "wait_for_completion": { + "type": "boolean", + "description": "Should this request wait until the operation has completed before returning", + "default": false + } + } + }, + "body": { + "description" : "Upgrade options", + "required" : false + } + } +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/results_indices_upgrade.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/results_indices_upgrade.yml new file mode 100644 index 0000000000000..5f53ba58b77de --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/results_indices_upgrade.yml @@ -0,0 +1,70 @@ +setup: + - skip: + features: headers + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + ml.put_job: + job_id: jobs-upgrade-results + body: > + { + "analysis_config" : { + "detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}] + }, + "data_description" : { + "format":"xcontent", + "time_field":"time" + } + } + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + Content-Type: application/json + index: + index: .ml-anomalies-jobs-upgrade-results + type: doc + id: "jobs-upgrade-results_1464739200000_1" + body: + { + "job_id": "jobs-upgrade-results", + "result_type": "bucket", + "timestamp": "2016-06-01T00:00:00Z", + "anomaly_score": 90.0, + "bucket_span":1 + } + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + indices.refresh: + index: .ml-anomalies-jobs-upgrade-results + +--- +"Upgrade results when there is nothing to upgrade": + - do: + ml.upgrade_job_results: + wait_for_completion: true + + - match: { acknowledged: true } + + - do: + indices.exists: + index: .ml-anomalies-shared + + - is_true: '' +--- +"Upgrade results when there is nothing to upgrade not waiting for results": + - do: + ml.upgrade_job_results: + wait_for_completion: false + + - match: {task: '/.+:\d+/'} + - set: {task: task} + + - do: + tasks.get: + wait_for_completion: true + task_id: $task + - match: {completed: true} + - match: {response.acknowledged: true} diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/80_ml_results_upgrade.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/80_ml_results_upgrade.yml new file mode 100644 index 0000000000000..73478be65597e --- /dev/null +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/80_ml_results_upgrade.yml @@ -0,0 +1,11 @@ +--- +"Verify jobs exist": + - do: + ml.get_jobs: + job_id: old-cluster-job-to-upgrade + - match: { count: 1 } + + - do: + ml.get_jobs: + job_id: old-cluster-job-to-upgrade-custom + - match: { count: 1 } diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/80_ml_results_upgrade.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/80_ml_results_upgrade.yml new file mode 100644 index 0000000000000..d21b5e6def61d --- /dev/null +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/80_ml_results_upgrade.yml @@ -0,0 +1,120 @@ +--- +"Put job on the old cluster and post some data": + + - do: + ml.put_job: + job_id: old-cluster-job-to-upgrade + body: > + { + "description":"Cluster upgrade", + "analysis_config" : { + "bucket_span": "60s", + "detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}] + }, + "analysis_limits" : { + "model_memory_limit": "50mb" + }, + "data_description" : { + "format":"xcontent", + "time_field":"time", + "time_format":"epoch" + } + } + - match: { job_id: old-cluster-job-to-upgrade } + + - do: + ml.open_job: + job_id: old-cluster-job-to-upgrade + + - do: + ml.post_data: + job_id: old-cluster-job-to-upgrade + body: + - airline: AAL + responsetime: 132.2046 + sourcetype: post-data-job + time: 1403481600 + - airline: JZA + responsetime: 990.4628 + sourcetype: post-data-job + time: 1403481700 + - match: { processed_record_count: 2 } + + - do: + ml.close_job: + job_id: old-cluster-job-to-upgrade + + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade + - match: { count: 1 } + +# Wait for indices to be fully allocated before +# killing the node + - do: + cluster.health: + index: [".ml-state", ".ml-anomalies-shared"] + wait_for_status: green + +--- +"Put job on the old cluster with a custom index": + - do: + ml.put_job: + job_id: old-cluster-job-to-upgrade-custom + body: > + { + "description":"Cluster upgrade", + "analysis_config" : { + "bucket_span": "60s", + "detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}] + }, + "analysis_limits" : { + "model_memory_limit": "50mb" + }, + "data_description" : { + "format":"xcontent", + "time_field":"time", + "time_format":"epoch" + }, + "results_index_name": "old-cluster-job-to-upgrade-custom" + } + - match: { job_id: old-cluster-job-to-upgrade-custom } + + - do: + ml.open_job: + job_id: old-cluster-job-to-upgrade-custom + + - do: + ml.post_data: + job_id: old-cluster-job-to-upgrade-custom + body: + - airline: AAL + responsetime: 132.2046 + sourcetype: post-data-job + time: 1403481600 + - airline: JZA + responsetime: 990.4628 + sourcetype: post-data-job + time: 1403481700 + - airline: JZA + responsetime: 423.0000 + sourcetype: post-data-job + time: 1403481800 + - match: { processed_record_count: 3 } + + - do: + ml.close_job: + job_id: old-cluster-job-to-upgrade-custom + + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade-custom + - match: { count: 3 } + +# Wait for indices to be fully allocated before +# killing the node + - do: + cluster.health: + index: [".ml-state", ".ml-anomalies-old-cluster-job-to-upgrade-custom"] + wait_for_status: green + diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml new file mode 100644 index 0000000000000..7e3e76aece7bb --- /dev/null +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml @@ -0,0 +1,155 @@ +--- +"Migrate results data to latest index binary version": + # Verify that all the results are there and the typical indices exist + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade + - match: { count: 1 } + + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade-custom + - match: { count: 3 } + + - do: + indices.exists: + index: .ml-anomalies-shared + + - is_true: '' + + - do: + indices.get_settings: + index: .ml-anomalies-shared + name: index.version.created + + - match: { \.ml-anomalies-shared.settings.index.version.created: '/6\d+/' } + + - do: + indices.exists: + index: .ml-anomalies-custom-old-cluster-job-to-upgrade-custom + + - is_true: '' + + # Do the upgrade + - do: + ml.upgrade_job_results: + wait_for_completion: true + + - match: { acknowledged: true } + + # Verify that old indices are gone + - do: + indices.exists: + index: .ml-anomalies-shared + + - is_false: '' + + - do: + indices.exists: + index: .ml-anomalies-custom-old-cluster-job-to-upgrade-custom + + - is_false: '' + + # Verify that results can still be retrieved + + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade + - match: { count: 1 } + + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade-custom + - match: { count: 3 } + + # Verify the created version is correct + + - do: + indices.get_settings: + index: .ml-anomalies-old-cluster-job-to-upgrade + name: index.version.created + - match: { \.ml-anomalies-shared-7.settings.index.version.created: '/7\d+/' } + - match: { \.ml-anomalies-shared-7r.settings.index.version.created: '/7\d+/' } + + - do: + indices.get_settings: + index: .ml-anomalies-old-cluster-job-to-upgrade-custom + name: index.version.created + - match: { \.ml-anomalies-custom-old-cluster-job-to-upgrade-custom-7.settings.index.version.created: '/7\d+/' } + - match: { \.ml-anomalies-custom-old-cluster-job-to-upgrade-custom-7r.settings.index.version.created: '/7\d+/' } + + # Create a new job to verify that the .ml-anomalies-shared index gets created again without issues + + - do: + ml.put_job: + job_id: upgraded-cluster-job-should-not-upgrade + body: > + { + "description":"Cluster upgrade", + "analysis_config" : { + "bucket_span": "60s", + "detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}] + }, + "analysis_limits" : { + "model_memory_limit": "50mb" + }, + "data_description" : { + "format":"xcontent", + "time_field":"time", + "time_format":"epoch" + } + } + - match: { job_id: upgraded-cluster-job-should-not-upgrade } + + - do: + ml.open_job: + job_id: upgraded-cluster-job-should-not-upgrade + + - do: + ml.post_data: + job_id: upgraded-cluster-job-should-not-upgrade + body: + - airline: AAL + responsetime: 132.2046 + sourcetype: post-data-job + time: 1403481600 + - airline: JZA + responsetime: 990.4628 + sourcetype: post-data-job + time: 1403481700 + - match: { processed_record_count: 2 } + + - do: + ml.close_job: + job_id: upgraded-cluster-job-should-not-upgrade + + - do: + ml.get_buckets: + job_id: upgraded-cluster-job-should-not-upgrade + - match: { count: 1 } + + - do: + indices.exists: + index: .ml-anomalies-shared + + - is_true: '' + + - do: + indices.get_settings: + index: .ml-anomalies-shared + name: index.version.created + + - match: { \.ml-anomalies-shared.settings.index.version.created: '/7\d+/' } + + # Do the upgrade Again as nothing needs upgraded now + - do: + ml.upgrade_job_results: + wait_for_completion: true + + - match: { acknowledged: true } + + - do: + indices.exists: + index: .ml-anomalies-shared + + - is_true: '' From d578d1b2d2bfc6273c273f404752c8e1c948fd02 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Fri, 14 Dec 2018 08:20:08 -0600 Subject: [PATCH 02/21] adjusting failure handling for reindex --- .../xpack/ml/ResultsIndexUpgradeService.java | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java index 85a5d2d178496..3b3f9041ebc57 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java @@ -86,7 +86,7 @@ public ResultsIndexUpgradeService(IndexNameExpressionResolver indexNameExpressio //Was the index created in the current major version public static boolean checkInternalIndexVersion(IndexMetaData indexMetaData) { - return indexMetaData.getCreationVersion().major == Version.CURRENT.major; + return indexMetaData.getCreationVersion().major == INDEX_VERSION; } /** @@ -329,13 +329,7 @@ private void reindexOldReadIndicesToNewIndices(Client client, client.threadPool().executor(executor), (createIndexResponse) -> { // If there are errors in the reindex, we should stop Tuple status = getStatusAndCause(createIndexResponse); - if (status.v1().equals(RestStatus.OK)) { - return true; - } else { - // Throw an unchecked exception to keep from executing the rest of the reindexing requests - // and return the failure to the end user. - throw new ElasticsearchException(status.v2()); - } + return status.v1().equals(RestStatus.OK); }, (exception -> true)); // Short circuit and call onFailure for any exception @@ -345,7 +339,7 @@ private void reindexOldReadIndicesToNewIndices(Client client, reindexRequest.setSourceBatchSize(request.getReindexBatchSize()); reindexRequest.setSourceIndices(oldIndex); reindexRequest.setDestIndex(newIndex); - // Don't worry if these indices already exist, we validated index.format earlier + // Don't worry if these indices already exist, we validated settings.index.created.version earlier reindexRequest.setConflicts("proceed"); // If the document exists already in the new index, don't want to update or overwrite as we are pulling from "old data" reindexRequest.setDestOpType("create"); @@ -359,7 +353,16 @@ private void reindexOldReadIndicesToNewIndices(Client client, }); chainTaskExecutor.execute(ActionListener.wrap( - bulkScrollingResponses -> listener.onResponse(true), + bulkScrollingResponses -> { + BulkByScrollResponse response = bulkScrollingResponses.get(bulkScrollingResponses.size() - 1); + Tuple status = getStatusAndCause(response); + if (status.v1().equals(RestStatus.OK)) { + listener.onResponse(true); + } else { + logger.error("Failed to reindex old results indices.", status.v2()); + listener.onFailure(new ElasticsearchException("Failed to reindex old results indices.",status.v2())); + } + }, failure -> { logger.error("Failed to re-index documents"); List createdIndices = newIndices.subList(0, chainTaskExecutor.getCollectedResponses().size()); From a74bcfe6201a136969f8b4942f541a4e916f4d92 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Fri, 14 Dec 2018 09:27:08 -0600 Subject: [PATCH 03/21] Fixing request and tests --- .../ml/action/ResultsIndexUpgradeAction.java | 9 +++++++-- .../action/ResultsIndexUpgradeRequestTests.java | 16 ++++++++++------ 2 files changed, 17 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeAction.java index 16417cf3c657f..e11f8406f4cba 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeAction.java @@ -116,8 +116,13 @@ public ActionRequestValidationException validate() { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ResultsIndexUpgradeAction.Request request = (ResultsIndexUpgradeAction.Request) o; return Objects.equals(reindexBatchSize, request.reindexBatchSize); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeRequestTests.java index 70593fe303089..8fd3a11b951b1 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeRequestTests.java @@ -5,20 +5,24 @@ */ package org.elasticsearch.xpack.core.ml.action; -import org.elasticsearch.test.AbstractStreamableTestCase; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.test.AbstractWireSerializingTestCase; -public class ResultsIndexUpgradeRequestTests extends AbstractStreamableTestCase { + +public class ResultsIndexUpgradeRequestTests extends AbstractWireSerializingTestCase { @Override protected ResultsIndexUpgradeAction.Request createTestInstance() { ResultsIndexUpgradeAction.Request request = new ResultsIndexUpgradeAction.Request(); - request.setReindexBatchSize(randomIntBetween(1, 10_000)); - request.setShouldStoreResult(randomBoolean()); + if (randomBoolean()) { + request.setReindexBatchSize(randomIntBetween(1, 10_000)); + } return request; } @Override - protected ResultsIndexUpgradeAction.Request createBlankInstance() { - return new ResultsIndexUpgradeAction.Request(); + protected Writeable.Reader instanceReader() { + return ResultsIndexUpgradeAction.Request::new; } + } From 9a43a5482505a2efe4f0c97a05648f5dd95f9ad5 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Fri, 14 Dec 2018 14:10:40 -0600 Subject: [PATCH 04/21] Adding tests to blacklist --- x-pack/plugin/ml/qa/ml-with-security/build.gradle | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/ml/qa/ml-with-security/build.gradle b/x-pack/plugin/ml/qa/ml-with-security/build.gradle index abfed3fd878d0..080a076a77d5d 100644 --- a/x-pack/plugin/ml/qa/ml-with-security/build.gradle +++ b/x-pack/plugin/ml/qa/ml-with-security/build.gradle @@ -93,7 +93,9 @@ integTestRunner { 'ml/validate/Test job config that is invalid only because of the job ID', 'ml/validate_detector/Test invalid detector', 'ml/delete_forecast/Test delete on _all forecasts not allow no forecasts', - 'ml/delete_forecast/Test delete forecast on missing forecast' + 'ml/delete_forecast/Test delete forecast on missing forecast', + 'ml/results_indices_upgrade/Upgrade results when there is nothing to upgrade', + 'ml/results_indices_upgrade/Upgrade results when there is nothing to upgrade not waiting for results' ].join(',') } From b3fccc676bb54c07447b63bee2e5b62f769cc430 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Fri, 14 Dec 2018 16:24:03 -0600 Subject: [PATCH 05/21] adjusting test --- .../xpack/ml/integration/ResultsIndexUpgradeIT.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java index 6702bf4a53954..d08b5ee817cd9 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java @@ -37,6 +37,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -367,7 +368,7 @@ public void testMigrationWithOpenJob() throws Exception { assertBusy(() -> { GetJobsStatsAction.Response.JobStats stats = getJobStats(openedJob.getId()).get(0); assertThat(stats.getDataCounts().getInputRecordCount(), equalTo(dataCount.get())); - }); + }, 120, TimeUnit.SECONDS); assertThat(getJobResultsCount(openedJob.getId()), greaterThan(closedJobTotal)); @@ -379,7 +380,7 @@ public void testMigrationWithOpenJob() throws Exception { private long indexSomeDocs(String index) { long numDocs = ESTestCase.randomIntBetween(15, 30); long now = System.currentTimeMillis(); - long abitAgo = now - 500; + long abitAgo = now - 100; indexDocs(logger, index, numDocs, abitAgo, now); return numDocs; @@ -445,8 +446,8 @@ private Job createAndOpenJobAndDataFeedWithDataAndNoEnd(String jobId, String dat DatafeedConfig.Builder builder = createDatafeedBuilder(job.getId() + "-datafeed", job.getId(), Collections.singletonList(dataIndex)); - builder.setQueryDelay(TimeValue.timeValueSeconds(5)); - builder.setFrequency(TimeValue.timeValueSeconds(5)); + builder.setQueryDelay(TimeValue.timeValueSeconds(60)); + builder.setFrequency(TimeValue.timeValueSeconds(60)); DatafeedConfig datafeedConfig = builder.build(); registerDatafeed(datafeedConfig); putDatafeed(datafeedConfig); From 07ed2a1ac63cdc8c552f421011d966e64cda474d Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Mon, 17 Dec 2018 08:36:31 -0600 Subject: [PATCH 06/21] test fix: posting data directly to the job instead of relying on datafeed --- .../ml/integration/ResultsIndexUpgradeIT.java | 29 ++++++++++++++----- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java index d08b5ee817cd9..d5316923f55de 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java @@ -34,12 +34,18 @@ import org.junit.After; import org.junit.Assert; +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Date; +import java.util.HashMap; import java.util.List; -import java.util.concurrent.TimeUnit; +import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; import static org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase.createDatafeedBuilder; import static org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase.createScheduledJob; @@ -327,9 +333,10 @@ public void testMigrationWithOpenJob() throws Exception { while (shouldContinueToIndex.get()) { try { Thread.sleep(1000); - long indexed = indexSomeDocs("data-for-migration-1"); + long indexed = postSomeDataToJob(openedJob.getId()); + logger.info("POST [" + indexed + "] data points to job [" + openedJob.getId() + "]"); dataCount.addAndGet(indexed); - } catch (InterruptedException ex) { + } catch (InterruptedException | IOException ex) { Thread.currentThread().interrupt(); } } @@ -368,7 +375,7 @@ public void testMigrationWithOpenJob() throws Exception { assertBusy(() -> { GetJobsStatsAction.Response.JobStats stats = getJobStats(openedJob.getId()).get(0); assertThat(stats.getDataCounts().getInputRecordCount(), equalTo(dataCount.get())); - }, 120, TimeUnit.SECONDS); + }); assertThat(getJobResultsCount(openedJob.getId()), greaterThan(closedJobTotal)); @@ -377,12 +384,20 @@ public void testMigrationWithOpenJob() throws Exception { waitUntilJobIsClosed(openedJob.getId()); } - private long indexSomeDocs(String index) { + private long postSomeDataToJob(String jobId) throws IOException { long numDocs = ESTestCase.randomIntBetween(15, 30); long now = System.currentTimeMillis(); long abitAgo = now - 100; - - indexDocs(logger, index, numDocs, abitAgo, now); + long delta = 100/numDocs; + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + List data = new ArrayList<>(); + for (int count = 0; count < numDocs; count++) { + Map record = new HashMap<>(); + record.put("time", sdf.format(new Date(abitAgo))); + data.add(createJsonRecord(record)); + abitAgo += delta; + } + postData(jobId, data.stream().collect(Collectors.joining())); return numDocs; } From bf9e2235711f893d6fb7c2035206fd412f2b2209 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Mon, 17 Dec 2018 10:04:21 -0600 Subject: [PATCH 07/21] adjusting API usage --- .../xpack/ml/integration/ResultsIndexUpgradeIT.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java index d5316923f55de..8c20e74321cdb 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java @@ -35,13 +35,15 @@ import org.junit.Assert; import java.io.IOException; -import java.text.SimpleDateFormat; +import java.time.Instant; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Date; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -389,11 +391,12 @@ private long postSomeDataToJob(String jobId) throws IOException { long now = System.currentTimeMillis(); long abitAgo = now - 100; long delta = 100/numDocs; - SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + final DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss", Locale.ROOT); List data = new ArrayList<>(); for (int count = 0; count < numDocs; count++) { Map record = new HashMap<>(); - record.put("time", sdf.format(new Date(abitAgo))); + String timeStr = Instant.ofEpochSecond(abitAgo).atZone(ZoneId.of("UTC")).format(formatter); + record.put("time", timeStr); data.add(createJsonRecord(record)); abitAgo += delta; } From 064ac692986159e1608ad0661793a2c31c810758 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 2 Jan 2019 08:22:45 -0600 Subject: [PATCH 08/21] adding Todos and adjusting endpoint --- .../MlNativeAutodetectIntegTestCase.java | 1 + .../xpack/ml/ResultsIndexUpgradeService.java | 5 +++-- .../ml/job/persistence/JobResultsProvider.java | 2 ++ .../ml/rest/results/RestUpgradeResultsAction.java | 15 +++++++++++++-- 4 files changed, 19 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java index 71710c17338a4..512b4f2ce8066 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java @@ -348,6 +348,7 @@ protected void waitForecastToFinish(String jobId, String forecastId) throws Exce }, 30, TimeUnit.SECONDS); } + //TODO: 6.7 Adjust so that multiple indices are supported protected ForecastRequestStats getForecastStats(String jobId, String forecastId) { GetResponse getResponse = client().prepareGet() .setIndex(AnomalyDetectorsIndex.jobResultsAliasedName(jobId)) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java index 3b3f9041ebc57..1e6cfa56d5777 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java @@ -12,6 +12,7 @@ import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; @@ -340,9 +341,9 @@ private void reindexOldReadIndicesToNewIndices(Client client, reindexRequest.setSourceIndices(oldIndex); reindexRequest.setDestIndex(newIndex); // Don't worry if these indices already exist, we validated settings.index.created.version earlier - reindexRequest.setConflicts("proceed"); + reindexRequest.setAbortOnVersionConflict(false); // If the document exists already in the new index, don't want to update or overwrite as we are pulling from "old data" - reindexRequest.setDestOpType("create"); + reindexRequest.setDestOpType(DocWriteRequest.OpType.CREATE.getLowercase()); newIndices.add(newIndex); chainTaskExecutor.add(chainedListener -> executeAsyncWithOrigin(client, diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java index cc75d48b81c0b..5fd761f95b4ce 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java @@ -957,6 +957,7 @@ private void searchSingleResult(String jobId, String resultDescription, S ), client::search); } + //TODO: 6.7 Adjust so that multiple indices are supported private void getResult(String jobId, String resultDescription, GetRequest get, BiFunction objectParser, Consumer> handler, Consumer errorHandler, Supplier notFoundSupplier) { @@ -1112,6 +1113,7 @@ public void scheduledEvents(ScheduledEventsQueryBuilder query, ActionListener handler, Consumer errorHandler) { String indexName = AnomalyDetectorsIndex.jobResultsAliasedName(jobId); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java index 8fefafb048960..0f8d95a6a4382 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java @@ -5,7 +5,9 @@ */ package org.elasticsearch.xpack.ml.rest.results; +import org.apache.logging.log4j.LogManager; import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -22,13 +24,22 @@ import java.io.IOException; +import static org.elasticsearch.rest.RestRequest.Method.POST; + public class RestUpgradeResultsAction extends BaseRestHandler { + private static final DeprecationLogger deprecationLogger = + new DeprecationLogger(LogManager.getLogger(RestUpgradeResultsAction.class)); + public RestUpgradeResultsAction(Settings settings, RestController controller) { super(settings); - controller.registerHandler(RestRequest.Method.POST, + controller.registerWithDeprecatedHandler( + POST, MachineLearning.BASE_PATH + "anomaly_detectors/results/_upgrade", - this); + this, + POST, + MachineLearning.PRE_V7_BASE_PATH + "anomaly_detectors/results/_upgrade", + deprecationLogger); } @Override From ba0df3a616ce3da95707496554b69da47a22eeed Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 2 Jan 2019 14:46:58 -0600 Subject: [PATCH 09/21] Adding types to reindexRequest --- .../org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java index 1e6cfa56d5777..814698c976222 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java @@ -36,6 +36,7 @@ import org.elasticsearch.index.reindex.ScrollableHitSource; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; +import org.elasticsearch.xpack.core.ml.job.persistence.ElasticsearchMappings; import org.elasticsearch.xpack.ml.utils.TypedChainTaskExecutor; import java.util.ArrayList; @@ -340,6 +341,8 @@ private void reindexOldReadIndicesToNewIndices(Client client, reindexRequest.setSourceBatchSize(request.getReindexBatchSize()); reindexRequest.setSourceIndices(oldIndex); reindexRequest.setDestIndex(newIndex); + reindexRequest.setSourceDocTypes(ElasticsearchMappings.DOC_TYPE); + reindexRequest.setDestDocType(ElasticsearchMappings.DOC_TYPE); // Don't worry if these indices already exist, we validated settings.index.created.version earlier reindexRequest.setAbortOnVersionConflict(false); // If the document exists already in the new index, don't want to update or overwrite as we are pulling from "old data" From a4c5e86a5621b454dd83adc49b509ffda6eb0932 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 2 Jan 2019 16:44:17 -0600 Subject: [PATCH 10/21] removing unreliable "live" data test --- .../ml/integration/ResultsIndexUpgradeIT.java | 127 ------------------ 1 file changed, 127 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java index 8c20e74321cdb..dcb096582d3fc 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java @@ -22,9 +22,7 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.xpack.core.ml.action.GetJobsStatsAction; import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; -import org.elasticsearch.xpack.core.ml.action.StartDatafeedAction; import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; import org.elasticsearch.xpack.core.ml.job.config.Job; import org.elasticsearch.xpack.core.ml.job.config.JobState; @@ -34,20 +32,9 @@ import org.junit.After; import org.junit.Assert; -import java.io.IOException; -import java.time.Instant; -import java.time.ZoneId; -import java.time.format.DateTimeFormatter; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; import static org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase.createDatafeedBuilder; import static org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase.createScheduledJob; @@ -59,9 +46,6 @@ public class ResultsIndexUpgradeIT extends MlNativeAutodetectIntegTestCase { - private AtomicBoolean shouldContinueToIndex = new AtomicBoolean(true); - private AtomicLong dataCount = new AtomicLong(0); - @After public void cleanup() throws Exception { cleanUp(); @@ -318,92 +302,6 @@ public void testMigrationWithExistingIndexWithData() throws Exception { assertThat(newJob3Total, greaterThan(job3Total)); } - public void testMigrationWithOpenJob() throws Exception { - dataCount.set(0); - shouldContinueToIndex.set(true); - Tuple amountAndIndex = createDataIndex(); - String dataIndex = amountAndIndex.v2(); - dataCount.set(amountAndIndex.v1()); - Job closedJob = createAndOpenJobAndStartDataFeedWithData("test-migration-open-job-closed", dataIndex, false); - Job openedJob = createAndOpenJobAndDataFeedWithDataAndNoEnd( - "test-migration-open-job-opened", - "data-for-migration-1", - false); - long closedJobTotal = getJobResultsCount(closedJob.getId()); - - Thread puttingData = new Thread(() -> { - while (shouldContinueToIndex.get()) { - try { - Thread.sleep(1000); - long indexed = postSomeDataToJob(openedJob.getId()); - logger.info("POST [" + indexed + "] data points to job [" + openedJob.getId() + "]"); - dataCount.addAndGet(indexed); - } catch (InterruptedException | IOException ex) { - Thread.currentThread().interrupt(); - } - } - }, "testMigrationWithOpenJobIndexer"); - puttingData.start(); - IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); - - ResultsIndexUpgradeService resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, - logger, - ThreadPool.Names.SAME, - indexMetaData -> true); - - PlainActionFuture future = PlainActionFuture.newFuture(); - - resultsIndexUpgradeService.upgrade(ESIntegTestCase.client(), - new ResultsIndexUpgradeAction.Request(), - ESIntegTestCase.client().admin().cluster().prepareState().get().getState(), - future); - - AcknowledgedResponse response = future.get(); - assertThat(response.isAcknowledged(), is(true)); - - ClusterState state = admin().cluster().state(new ClusterStateRequest()).actionGet().getState(); - String[] indices = indexNameExpressionResolver.concreteIndexNames(state, - IndicesOptions.strictExpandOpenAndForbidClosed(), - AnomalyDetectorsIndex.jobResultsIndexPrefix() + "*"); - - // Our backing index size should be four as we have a shared and custom index and upgrading doubles the number of indices - Assert.assertThat(indices.length, equalTo(2)); - - assertThat(getJobResultsCount(closedJob.getId()), equalTo(closedJobTotal)); - - shouldContinueToIndex.set(false); - puttingData.join(); - flushJob(openedJob.getId(), true); - assertBusy(() -> { - GetJobsStatsAction.Response.JobStats stats = getJobStats(openedJob.getId()).get(0); - assertThat(stats.getDataCounts().getInputRecordCount(), equalTo(dataCount.get())); - }); - - assertThat(getJobResultsCount(openedJob.getId()), greaterThan(closedJobTotal)); - - stopDatafeed(openedJob.getId() + "-datafeed"); - closeJob(openedJob.getId()); - waitUntilJobIsClosed(openedJob.getId()); - } - - private long postSomeDataToJob(String jobId) throws IOException { - long numDocs = ESTestCase.randomIntBetween(15, 30); - long now = System.currentTimeMillis(); - long abitAgo = now - 100; - long delta = 100/numDocs; - final DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss", Locale.ROOT); - List data = new ArrayList<>(); - for (int count = 0; count < numDocs; count++) { - Map record = new HashMap<>(); - String timeStr = Instant.ofEpochSecond(abitAgo).atZone(ZoneId.of("UTC")).format(formatter); - record.put("time", timeStr); - data.add(createJsonRecord(record)); - abitAgo += delta; - } - postData(jobId, data.stream().collect(Collectors.joining())); - return numDocs; - } - private long getTotalDocCount(String indexName) { SearchResponse searchResponse = ESIntegTestCase.client().prepareSearch(indexName) .setSize(10_000) @@ -449,31 +347,6 @@ private Job createAndOpenJobAndStartDataFeedWithData(String jobId, String dataIn return job; } - private Job createAndOpenJobAndDataFeedWithDataAndNoEnd(String jobId, String dataIndex, boolean isCustom) throws Exception { - Job.Builder jobbuilder = createScheduledJob(jobId); - if (isCustom) { - jobbuilder.setResultsIndexName(jobId); - } - registerJob(jobbuilder); - - Job job = putJob(jobbuilder).getResponse(); - - openJob(job.getId()); - ESTestCase.assertBusy(() -> Assert.assertEquals(getJobStats(job.getId()).get(0).getState(), JobState.OPENED)); - - DatafeedConfig.Builder builder = createDatafeedBuilder(job.getId() + "-datafeed", - job.getId(), - Collections.singletonList(dataIndex)); - builder.setQueryDelay(TimeValue.timeValueSeconds(60)); - builder.setFrequency(TimeValue.timeValueSeconds(60)); - DatafeedConfig datafeedConfig = builder.build(); - registerDatafeed(datafeedConfig); - putDatafeed(datafeedConfig); - StartDatafeedAction.Request request = new StartDatafeedAction.Request(datafeedConfig.getId(), 0L); - client().execute(StartDatafeedAction.INSTANCE, request).actionGet(); - return job; - } - private Tuple createDataIndex() { ESIntegTestCase.client().admin().indices().prepareCreate("data-for-migration-1") .addMapping("type", "time", "type=date") From 28576fdbe76a461a39a45d0412cf15c2fc705656 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 3 Jan 2019 08:06:02 -0600 Subject: [PATCH 11/21] adding index refresh to test --- .../xpack/ml/integration/ResultsIndexUpgradeIT.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java index dcb096582d3fc..47e722b14a07a 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java @@ -283,6 +283,7 @@ public void testMigrationWithExistingIndexWithData() throws Exception { // Our backing index size should be four as we have a shared and custom index and upgrading doubles the number of indices Assert.assertThat(indices.length, equalTo(4)); + refresh(indices); assertThat(getJobResultsCount(job1.getId()), equalTo(job1Total)); assertThat(getJobResultsCount(job2.getId()), equalTo(job2Total)); @@ -290,14 +291,17 @@ public void testMigrationWithExistingIndexWithData() throws Exception { // WE should still be able to write, and the aliases should allow to read from the appropriate indices postDataToJob(jobId1); + postDataToJob(jobId2); + postDataToJob(jobId3); + + refresh(indices); + long newJob1Total = getJobResultsCount(job1.getId()); assertThat(newJob1Total, greaterThan(job1Total)); - postDataToJob(jobId2); long newJob2Total = getJobResultsCount(job2.getId()); assertThat(newJob2Total, greaterThan(job2Total)); - postDataToJob(jobId3); long newJob3Total = getJobResultsCount(job3.getId()); assertThat(newJob3Total, greaterThan(job3Total)); } From 8e9451e704c45335749ed81f8441af6c44292942 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 3 Jan 2019 08:08:28 -0600 Subject: [PATCH 12/21] adding index refresh to test --- .../ml/integration/ResultsIndexUpgradeIT.java | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java index 47e722b14a07a..7b4f6e2f918b1 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java @@ -149,6 +149,7 @@ public void testMigration() throws Exception { // Our backing index size should be four as we have a shared and custom index and upgrading doubles the number of indices Assert.assertThat(indices.length, equalTo(4)); + refresh(indices); assertThat(getJobResultsCount(job1.getId()), equalTo(job1Total)); assertThat(getJobResultsCount(job2.getId()), equalTo(job2Total)); assertThat(getJobResultsCount(job3.getId()), equalTo(job3Total)); @@ -156,22 +157,27 @@ public void testMigration() throws Exception { // WE should still be able to write, and the aliases should allow to read from the appropriate indices postDataToJob(jobId1); + postDataToJob(jobId2); + postDataToJob(jobId3); + // We should also be able to create new jobs and old jobs should be unaffected. + String jobId4 = "migration-test4"; + Job job4 = createAndOpenJobAndStartDataFeedWithData(jobId4, dataIndex, false); + waitUntilJobIsClosed(jobId4); + + indices = indexNameExpressionResolver.concreteIndexNames(state, + IndicesOptions.strictExpandOpenAndForbidClosed(), + AnomalyDetectorsIndex.jobResultsIndexPrefix() + "*"); + refresh(indices); + long newJob1Total = getJobResultsCount(job1.getId()); assertThat(newJob1Total, greaterThan(job1Total)); - postDataToJob(jobId2); long newJob2Total = getJobResultsCount(job2.getId()); assertThat(newJob2Total, greaterThan(job2Total)); - postDataToJob(jobId3); long newJob3Total = getJobResultsCount(job3.getId()); assertThat(newJob3Total, greaterThan(job3Total)); - // We should also be able to create new jobs and old jobs should be unaffected. - String jobId4 = "migration-test4"; - Job job4 = createAndOpenJobAndStartDataFeedWithData(jobId4, dataIndex, false); - waitUntilJobIsClosed(jobId4); - assertThat(getJobResultsCount(jobId4), greaterThan(0L)); assertThat(getJobResultsCount(jobId1), equalTo(newJob1Total)); assertThat(getJobResultsCount(jobId2), equalTo(newJob2Total)); From 1d6c66c9737af9aa451b5ad2fed50f26262062f0 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 3 Jan 2019 15:45:51 -0600 Subject: [PATCH 13/21] adding index refresh to yaml test --- .../test/upgraded_cluster/80_ml_results_upgrade.yml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml index 7e3e76aece7bb..9905e64c36800 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml @@ -46,12 +46,15 @@ - do: indices.exists: - index: .ml-anomalies-custom-old-cluster-job-to-upgrade-custom + index: - is_false: '' # Verify that results can still be retrieved + - do: + indices.refresh: {} + - do: ml.get_buckets: job_id: old-cluster-job-to-upgrade From b34142416e159d72fb0a1195e98f1c811080b78b Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Fri, 4 Jan 2019 07:57:07 -0600 Subject: [PATCH 14/21] fixing bad exists call --- .../test/upgraded_cluster/80_ml_results_upgrade.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml index 9905e64c36800..dc9fc0224cc94 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml @@ -46,7 +46,7 @@ - do: indices.exists: - index: + index: .ml-anomalies-custom-old-cluster-job-to-upgrade-custom - is_false: '' From 3f47ce0a43911c8a67d9c5ba3ea80108f9a2900f Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Mon, 7 Jan 2019 11:13:55 -0600 Subject: [PATCH 15/21] removing todo --- .../xpack/ml/integration/MlNativeAutodetectIntegTestCase.java | 1 - .../xpack/ml/job/persistence/JobResultsProvider.java | 2 -- 2 files changed, 3 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java index 512b4f2ce8066..71710c17338a4 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java @@ -348,7 +348,6 @@ protected void waitForecastToFinish(String jobId, String forecastId) throws Exce }, 30, TimeUnit.SECONDS); } - //TODO: 6.7 Adjust so that multiple indices are supported protected ForecastRequestStats getForecastStats(String jobId, String forecastId) { GetResponse getResponse = client().prepareGet() .setIndex(AnomalyDetectorsIndex.jobResultsAliasedName(jobId)) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java index 5fd761f95b4ce..cc75d48b81c0b 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java @@ -957,7 +957,6 @@ private void searchSingleResult(String jobId, String resultDescription, S ), client::search); } - //TODO: 6.7 Adjust so that multiple indices are supported private void getResult(String jobId, String resultDescription, GetRequest get, BiFunction objectParser, Consumer> handler, Consumer errorHandler, Supplier notFoundSupplier) { @@ -1113,7 +1112,6 @@ public void scheduledEvents(ScheduledEventsQueryBuilder query, ActionListener handler, Consumer errorHandler) { String indexName = AnomalyDetectorsIndex.jobResultsAliasedName(jobId); From 02603efa6d859060dc037bc5a240f6fe71c2b089 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Mon, 7 Jan 2019 12:49:35 -0600 Subject: [PATCH 16/21] Addressing remove comments --- .../xpack/core/XPackClientPlugin.java | 4 +- ...pgradeAction.java => MlUpgradeAction.java} | 37 ++++++------ ...tTests.java => MlUpgradeRequestTests.java} | 10 ++-- .../ml/integration/ResultsIndexUpgradeIT.java | 13 ++-- .../xpack/ml/MachineLearning.java | 6 +- .../xpack/ml/MlConfigMigrator.java | 8 +-- .../xpack/ml/ResultsIndexUpgradeService.java | 43 ++++++------- .../TransportFinalizeJobExecutionAction.java | 10 ++-- ...ion.java => TransportMlUpgradeAction.java} | 26 ++++---- .../xpack/ml/job/JobManager.java | 18 +++--- .../results/RestUpgradeResultsAction.java | 12 ++-- .../xpack/ml/utils/ChainTaskExecutor.java | 60 ------------------- .../xpack/ml/utils/VoidChainTaskExecutor.java | 26 ++++++++ ...s.java => VoidChainTaskExecutorTests.java} | 58 +++++++++--------- 14 files changed, 143 insertions(+), 188 deletions(-) rename x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/{ResultsIndexUpgradeAction.java => MlUpgradeAction.java} (79%) rename x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/{ResultsIndexUpgradeRequestTests.java => MlUpgradeRequestTests.java} (58%) rename x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/{TransportResultsIndexUpgradeAction.java => TransportMlUpgradeAction.java} (68%) delete mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutor.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutor.java rename x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/{ChainTaskExecutorTests.java => VoidChainTaskExecutorTests.java} (62%) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java index 04c73ef35f58e..d7abe9a1f0f03 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java @@ -105,7 +105,7 @@ import org.elasticsearch.xpack.core.ml.action.PutDatafeedAction; import org.elasticsearch.xpack.core.ml.action.PutFilterAction; import org.elasticsearch.xpack.core.ml.action.PutJobAction; -import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; import org.elasticsearch.xpack.core.ml.action.RevertModelSnapshotAction; import org.elasticsearch.xpack.core.ml.action.StartDatafeedAction; import org.elasticsearch.xpack.core.ml.action.StopDatafeedAction; @@ -290,7 +290,7 @@ public List> getClientActions() { PostCalendarEventsAction.INSTANCE, PersistJobAction.INSTANCE, FindFileStructureAction.INSTANCE, - ResultsIndexUpgradeAction.INSTANCE, + MlUpgradeAction.INSTANCE, // security ClearRealmCacheAction.INSTANCE, ClearRolesCacheAction.INSTANCE, diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeAction.java similarity index 79% rename from x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeAction.java rename to x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeAction.java index e11f8406f4cba..993637ee086c3 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeAction.java @@ -7,7 +7,6 @@ import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; @@ -29,11 +28,11 @@ import java.util.Objects; -public class ResultsIndexUpgradeAction extends Action { - public static final ResultsIndexUpgradeAction INSTANCE = new ResultsIndexUpgradeAction(); - public static final String NAME = "cluster:admin/xpack/ml/results/upgrade"; +public class MlUpgradeAction extends Action { + public static final MlUpgradeAction INSTANCE = new MlUpgradeAction(); + public static final String NAME = "cluster:admin/xpack/ml/upgrade"; - private ResultsIndexUpgradeAction() { + private MlUpgradeAction() { super(NAME); } @@ -42,13 +41,11 @@ public AcknowledgedResponse newResponse() { return new AcknowledgedResponse(); } - public static class Request - extends MasterNodeReadRequest - implements IndicesRequest, ToXContentObject { + public static class Request extends MasterNodeReadRequest implements ToXContentObject { private static final ParseField REINDEX_BATCH_SIZE = new ParseField("reindex_batch_size"); - public static ObjectParser PARSER = new ObjectParser<>("ml_results_index_upgrade", true, Request::new); + public static ObjectParser PARSER = new ObjectParser<>("ml_results_upgrade", true, Request::new); static { PARSER.declareInt(Request::setReindexBatchSize, REINDEX_BATCH_SIZE); } @@ -76,21 +73,18 @@ public void writeTo(StreamOutput out) throws IOException { out.writeInt(reindexBatchSize); } - @Override public String[] indices() { return new String[]{INDEX}; } - @Override public IndicesOptions indicesOptions() { - //TODO consider lenientExpandOpen() ? return IndicesOptions.strictExpandOpenAndForbidClosed(); } /** * Should this task store its result after it has finished? */ - public ResultsIndexUpgradeAction.Request setShouldStoreResult(boolean shouldStoreResult) { + public Request setShouldStoreResult(boolean shouldStoreResult) { this.shouldStoreResult = shouldStoreResult; return this; } @@ -100,7 +94,7 @@ public boolean getShouldStoreResult() { return shouldStoreResult; } - public ResultsIndexUpgradeAction.Request setReindexBatchSize(int reindexBatchSize) { + public Request setReindexBatchSize(int reindexBatchSize) { this.reindexBatchSize = reindexBatchSize; return this; } @@ -111,7 +105,12 @@ public int getReindexBatchSize() { @Override public ActionRequestValidationException validate() { - return null; + if (reindexBatchSize <= 0) { + ActionRequestValidationException validationException = new ActionRequestValidationException(); + validationException.addValidationError("["+ REINDEX_BATCH_SIZE.getPreferredName()+"] must be greater than 0."); + return validationException; + } + return null; } @Override @@ -123,7 +122,7 @@ public boolean equals(Object o) { return false; } - ResultsIndexUpgradeAction.Request request = (ResultsIndexUpgradeAction.Request) o; + Request request = (Request) o; return Objects.equals(reindexBatchSize, request.reindexBatchSize); } @@ -151,12 +150,10 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } } - public static class RequestBuilder extends MasterNodeReadOperationRequestBuilder { + public static class RequestBuilder extends MasterNodeReadOperationRequestBuilder { public RequestBuilder(ElasticsearchClient client) { - super(client, INSTANCE, new ResultsIndexUpgradeAction.Request()); + super(client, INSTANCE, new Request()); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeRequestTests.java similarity index 58% rename from x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeRequestTests.java rename to x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeRequestTests.java index 8fd3a11b951b1..227fc20ec9688 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/ResultsIndexUpgradeRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeRequestTests.java @@ -9,11 +9,11 @@ import org.elasticsearch.test.AbstractWireSerializingTestCase; -public class ResultsIndexUpgradeRequestTests extends AbstractWireSerializingTestCase { +public class MlUpgradeRequestTests extends AbstractWireSerializingTestCase { @Override - protected ResultsIndexUpgradeAction.Request createTestInstance() { - ResultsIndexUpgradeAction.Request request = new ResultsIndexUpgradeAction.Request(); + protected MlUpgradeAction.Request createTestInstance() { + MlUpgradeAction.Request request = new MlUpgradeAction.Request(); if (randomBoolean()) { request.setReindexBatchSize(randomIntBetween(1, 10_000)); } @@ -21,8 +21,8 @@ protected ResultsIndexUpgradeAction.Request createTestInstance() { } @Override - protected Writeable.Reader instanceReader() { - return ResultsIndexUpgradeAction.Request::new; + protected Writeable.Reader instanceReader() { + return MlUpgradeAction.Request::new; } } diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java index 7b4f6e2f918b1..767ea34a8ab28 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java @@ -22,7 +22,7 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; import org.elasticsearch.xpack.core.ml.job.config.Job; import org.elasticsearch.xpack.core.ml.job.config.JobState; @@ -74,8 +74,8 @@ public void testMigrationWhenItIsNotNecessary() throws Exception { long job2Total = getTotalDocCount(job2Index); long job3Total = getTotalDocCount(job3Index); - AcknowledgedResponse resp = ESIntegTestCase.client().execute(ResultsIndexUpgradeAction.INSTANCE, - new ResultsIndexUpgradeAction.Request()).actionGet(); + AcknowledgedResponse resp = ESIntegTestCase.client().execute(MlUpgradeAction.INSTANCE, + new MlUpgradeAction.Request()).actionGet(); assertThat(resp.isAcknowledged(), is(true)); // Migration should have done nothing @@ -130,7 +130,7 @@ public void testMigration() throws Exception { PlainActionFuture future = PlainActionFuture.newFuture(); resultsIndexUpgradeService.upgrade(ESIntegTestCase.client(), - new ResultsIndexUpgradeAction.Request(), + new MlUpgradeAction.Request(), ESIntegTestCase.client().admin().cluster().prepareState().get().getState(), future); @@ -212,7 +212,7 @@ public void testMigrationWithManuallyCreatedIndexThatNeedsMigrating() throws Exc indexMetaData -> true); //indicates that this manually created index needs migrated resultsIndexUpgradeService.upgrade(ESIntegTestCase.client(), - new ResultsIndexUpgradeAction.Request(), + new MlUpgradeAction.Request(), ESIntegTestCase.client().admin().cluster().prepareState().get().getState(), ActionListener.wrap( resp -> fail(), @@ -271,7 +271,7 @@ public void testMigrationWithExistingIndexWithData() throws Exception { PlainActionFuture future = PlainActionFuture.newFuture(); resultsIndexUpgradeService.upgrade(ESIntegTestCase.client(), - new ResultsIndexUpgradeAction.Request(), + new MlUpgradeAction.Request(), ESIntegTestCase.client().admin().cluster().prepareState().get().getState(), future); @@ -315,6 +315,7 @@ public void testMigrationWithExistingIndexWithData() throws Exception { private long getTotalDocCount(String indexName) { SearchResponse searchResponse = ESIntegTestCase.client().prepareSearch(indexName) .setSize(10_000) + .setTrackTotalHits(true) .setQuery(QueryBuilders.matchAllQuery()) .execute().actionGet(); return searchResponse.getHits().getTotalHits().value; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java index 154313044ed43..c025a36c6e510 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java @@ -96,7 +96,7 @@ import org.elasticsearch.xpack.core.ml.action.PutDatafeedAction; import org.elasticsearch.xpack.core.ml.action.PutFilterAction; import org.elasticsearch.xpack.core.ml.action.PutJobAction; -import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; import org.elasticsearch.xpack.core.ml.action.RevertModelSnapshotAction; import org.elasticsearch.xpack.core.ml.action.StartDatafeedAction; import org.elasticsearch.xpack.core.ml.action.StopDatafeedAction; @@ -151,7 +151,7 @@ import org.elasticsearch.xpack.ml.action.TransportPutDatafeedAction; import org.elasticsearch.xpack.ml.action.TransportPutFilterAction; import org.elasticsearch.xpack.ml.action.TransportPutJobAction; -import org.elasticsearch.xpack.ml.action.TransportResultsIndexUpgradeAction; +import org.elasticsearch.xpack.ml.action.TransportMlUpgradeAction; import org.elasticsearch.xpack.ml.action.TransportRevertModelSnapshotAction; import org.elasticsearch.xpack.ml.action.TransportStartDatafeedAction; import org.elasticsearch.xpack.ml.action.TransportStopDatafeedAction; @@ -604,7 +604,7 @@ public List getRestHandlers(Settings settings, RestController restC new ActionHandler<>(PostCalendarEventsAction.INSTANCE, TransportPostCalendarEventsAction.class), new ActionHandler<>(PersistJobAction.INSTANCE, TransportPersistJobAction.class), new ActionHandler<>(FindFileStructureAction.INSTANCE, TransportFindFileStructureAction.class), - new ActionHandler<>(ResultsIndexUpgradeAction.INSTANCE, TransportResultsIndexUpgradeAction.class) + new ActionHandler<>(MlUpgradeAction.INSTANCE, TransportMlUpgradeAction.class) ); } @Override diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java index 184ee44cf376c..bb3735f8aa3f1 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java @@ -44,7 +44,7 @@ import org.elasticsearch.xpack.core.ml.utils.ToXContentParams; import org.elasticsearch.xpack.ml.datafeed.persistence.DatafeedConfigProvider; import org.elasticsearch.xpack.ml.job.persistence.JobConfigProvider; -import org.elasticsearch.xpack.ml.utils.ChainTaskExecutor; +import org.elasticsearch.xpack.ml.utils.VoidChainTaskExecutor; import java.io.IOException; import java.util.ArrayList; @@ -179,9 +179,9 @@ public void migrateConfigsWithoutTasks(ClusterState clusterState, ActionListener } private void migrateBatches(List batches, ActionListener listener) { - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(EsExecutors.newDirectExecutorService(), true); + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(EsExecutors.newDirectExecutorService(), true); for (JobsAndDatafeeds batch : batches) { - chainTaskExecutor.add(chainedListener -> writeConfigToIndex(batch.datafeedConfigs, batch.jobs, ActionListener.wrap( + voidChainTaskExecutor.add(chainedListener -> writeConfigToIndex(batch.datafeedConfigs, batch.jobs, ActionListener.wrap( failedDocumentIds -> { List successfulJobWrites = filterFailedJobConfigWrites(failedDocumentIds, batch.jobs); List successfulDatafeedWrites = @@ -191,7 +191,7 @@ private void migrateBatches(List batches, ActionListener listener.onResponse(true), listener::onFailure)); + voidChainTaskExecutor.execute(ActionListener.wrap(aVoids -> listener.onResponse(true), listener::onFailure)); } // Exposed for testing diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java index 814698c976222..9d36674da1901 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java @@ -35,7 +35,7 @@ import org.elasticsearch.index.reindex.ReindexRequest; import org.elasticsearch.index.reindex.ScrollableHitSource; import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; import org.elasticsearch.xpack.core.ml.job.persistence.ElasticsearchMappings; import org.elasticsearch.xpack.ml.utils.TypedChainTaskExecutor; @@ -60,7 +60,7 @@ public class ResultsIndexUpgradeService { // Adjust the following constants as necessary for various versions and backports. private static final int INDEX_VERSION = Version.CURRENT.major; - private static final Version UPGRADE_INTRODUCED = Version.CURRENT.minimumCompatibilityVersion(); + private static final Version MIN_REQUIRED_VERSION = Version.CURRENT.minimumCompatibilityVersion(); private final IndexNameExpressionResolver indexNameExpressionResolver; private final Predicate shouldUpgrade; @@ -86,8 +86,7 @@ public ResultsIndexUpgradeService(IndexNameExpressionResolver indexNameExpressio this.logger = logger; } - //Was the index created in the current major version - public static boolean checkInternalIndexVersion(IndexMetaData indexMetaData) { + public static boolean wasIndexCreatedInCurrentMajorVersion(IndexMetaData indexMetaData) { return indexMetaData.getCreationVersion().major == INDEX_VERSION; } @@ -96,8 +95,8 @@ public static boolean checkInternalIndexVersion(IndexMetaData indexMetaData) { * 1. The upgrade process has ran before and either failed for some reason, or the end user is simply running it again. * Either way, it should be ok to proceed as this action SHOULD be idempotent, * unless the shouldUpgrade predicate is poorly formed - * 2. This index was created manually by the user. If the index was created manually and actually needs upgraded, then - * we consider the "new index" to be invalid as it is not the upgraded. + * 2. This index was created manually by the user. If the index was created manually and actually needs upgrading, then + * we consider the "new index" to be invalid as the passed predicate indicates that it still needs upgrading. * * @param metaData Cluster metadata * @param newIndexName The index to check @@ -108,14 +107,13 @@ private static boolean invalidIndexForUpgrade(MetaData metaData, String newIndex return metaData.hasIndex(newIndexName) && shouldUpgrade.test(metaData.index(newIndexName)); } - // Are all the nodes the appropriate version for us to do the upgrade? - private static void checkMasterAndDataNodeVersion(ClusterState clusterState) { - if (clusterState.nodes().getMinNodeVersion().before(UPGRADE_INTRODUCED)) { - throw new IllegalStateException("All nodes should have at least version [" + UPGRADE_INTRODUCED + "] to upgrade"); + private static void validateMinNodeVersion(ClusterState clusterState) { + if (clusterState.nodes().getMinNodeVersion().before(MIN_REQUIRED_VERSION)) { + throw new IllegalStateException("All nodes should have at least version [" + MIN_REQUIRED_VERSION + "] to upgrade"); } } - //This method copies the behavior of the normal {index}/_upgrade rest response handler + // This method copies the behavior of the normal {index}/_upgrade rest response handler private static Tuple getStatusAndCause(BulkByScrollResponse response) { /* * Return the highest numbered rest status under the assumption that higher numbered statuses are "more error" @@ -151,10 +149,10 @@ private static Tuple getStatusAndCause(BulkByScrollRespon * @param state The current cluster state * @param listener The listener to alert when actions have completed */ - public void upgrade(Client client, ResultsIndexUpgradeAction.Request request, ClusterState state, + public void upgrade(Client client, MlUpgradeAction.Request request, ClusterState state, ActionListener listener) { try { - checkMasterAndDataNodeVersion(state); + validateMinNodeVersion(state); String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request.indicesOptions(), request.indices()); MetaData metaData = state.getMetaData(); @@ -179,9 +177,9 @@ public void upgrade(Client client, ResultsIndexUpgradeAction.Request request, Cl ActionListener deleteIndicesListener = ActionListener.wrap( listener::onResponse, error -> { - logger.error("Failed to delete old indices: " + Strings.collectionToCommaDelimitedString(indicesToUpgrade), - error); - listener.onResponse(new AcknowledgedResponse(true)); + String msg = "Failed to delete old indices: " + Strings.collectionToCommaDelimitedString(indicesToUpgrade); + logger.error(msg, error); + listener.onFailure(new ElasticsearchException(msg, error)); } ); @@ -189,7 +187,7 @@ public void upgrade(Client client, ResultsIndexUpgradeAction.Request request, Cl ActionListener readAliasListener = ActionListener.wrap( resp -> deleteOldIndices(client, indicesToUpgrade, deleteIndicesListener), error -> { - String msg = "Failed adjusting aliases from old indices to new " + error.getMessage(); + String msg = "Failed adjusting aliases from old indices to new."; logger.error(msg, error); listener.onFailure(new ElasticsearchException(msg, error)); } @@ -206,8 +204,7 @@ public void upgrade(Client client, ResultsIndexUpgradeAction.Request request, Cl indexNameAndAliasProvider.newReadIndicesWithReadAliases(), readAliasListener), rrobFailure -> { - String msg = "Failed making old indices writable again so that aliases can be moved. " - + rrobFailure.getMessage(); + String msg = "Failed making old indices writable again so that aliases can be moved."; logger.error(msg, rrobFailure); listener.onFailure(new ElasticsearchException(msg, rrobFailure)); }) @@ -324,7 +321,7 @@ private void removeReadOnlyBlock(Client client, List indices, private void reindexOldReadIndicesToNewIndices(Client client, Map reindexIndices, - ResultsIndexUpgradeAction.Request request, + MlUpgradeAction.Request request, ActionListener listener) { TypedChainTaskExecutor chainTaskExecutor = new TypedChainTaskExecutor<>( @@ -368,7 +365,6 @@ private void reindexOldReadIndicesToNewIndices(Client client, } }, failure -> { - logger.error("Failed to re-index documents"); List createdIndices = newIndices.subList(0, chainTaskExecutor.getCollectedResponses().size()); logger.error( "Failed to reindex all old read indices. Successfully reindexed: [" + @@ -457,11 +453,6 @@ private IndexNameAndAliasProvider(List oldIndices, MetaData metaData) { private Exception validate(MetaData metaData, Predicate shouldUpgrade) { for (String index : oldIndices) { - IndexMetaData indexMetaData = metaData.index(index); - - if (indexMetaData.getState() != IndexMetaData.State.OPEN) { - return new IllegalStateException("unable to upgrade a closed index[" + index + "]"); - } String newWriteName = newWriteName(index); // If the "new" indices exist, either they were created from a previous run of the upgrade process or the end user if (invalidIndexForUpgrade(metaData, newWriteName, shouldUpgrade)) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java index d6c03d6c93fbf..9d76844121cbb 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java @@ -26,7 +26,7 @@ import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.job.persistence.ElasticsearchMappings; import org.elasticsearch.xpack.ml.MachineLearning; -import org.elasticsearch.xpack.ml.utils.ChainTaskExecutor; +import org.elasticsearch.xpack.ml.utils.VoidChainTaskExecutor; import java.util.Collections; import java.util.Date; @@ -65,7 +65,7 @@ protected void masterOperation(FinalizeJobExecutionAction.Request request, Clust String jobIdString = String.join(",", request.getJobIds()); logger.debug("finalizing jobs [{}]", jobIdString); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.executor( + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.executor( MachineLearning.UTILITY_THREAD_POOL_NAME), true); Map update = Collections.singletonMap(Job.FINISHED_TIME.getPreferredName(), new Date()); @@ -77,7 +77,7 @@ protected void masterOperation(FinalizeJobExecutionAction.Request request, Clust updateRequest.doc(update); updateRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - chainTaskExecutor.add(chainedListener -> { + voidChainTaskExecutor.add(chainedListener -> { executeAsyncWithOrigin(client, ML_ORIGIN, UpdateAction.INSTANCE, updateRequest, ActionListener.wrap( updateResponse -> chainedListener.onResponse(null), chainedListener::onFailure @@ -85,8 +85,8 @@ protected void masterOperation(FinalizeJobExecutionAction.Request request, Clust }); } - chainTaskExecutor.execute(ActionListener.wrap( - aVoid -> { + voidChainTaskExecutor.execute(ActionListener.wrap( + aVoids -> { logger.debug("finalized job [{}]", jobIdString); listener.onResponse(new AcknowledgedResponse(true)); }, diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportResultsIndexUpgradeAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportMlUpgradeAction.java similarity index 68% rename from x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportResultsIndexUpgradeAction.java rename to x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportMlUpgradeAction.java index c9b33df2074b7..9f651c6892fb8 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportResultsIndexUpgradeAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportMlUpgradeAction.java @@ -21,32 +21,32 @@ import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; import org.elasticsearch.xpack.ml.ResultsIndexUpgradeService; -import static org.elasticsearch.xpack.ml.ResultsIndexUpgradeService.checkInternalIndexVersion; +import static org.elasticsearch.xpack.ml.ResultsIndexUpgradeService.wasIndexCreatedInCurrentMajorVersion; -public class TransportResultsIndexUpgradeAction - extends TransportMasterNodeReadAction { +public class TransportMlUpgradeAction + extends TransportMasterNodeReadAction { private final Client client; private final ResultsIndexUpgradeService resultsIndexUpgradeService; @Inject - public TransportResultsIndexUpgradeAction(TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, ActionFilters actionFilters, Client client, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(ResultsIndexUpgradeAction.NAME, transportService, clusterService, threadPool, - actionFilters, ResultsIndexUpgradeAction.Request::new, indexNameExpressionResolver); + public TransportMlUpgradeAction(TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, Client client, + IndexNameExpressionResolver indexNameExpressionResolver) { + super(MlUpgradeAction.NAME, transportService, clusterService, threadPool, + actionFilters, MlUpgradeAction.Request::new, indexNameExpressionResolver); this.client = client; this.resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, logger, executor(), - indexMetadata -> checkInternalIndexVersion(indexMetadata) == false); + indexMetadata -> wasIndexCreatedInCurrentMajorVersion(indexMetadata) == false); } @Override - protected void masterOperation(Task task, ResultsIndexUpgradeAction.Request request, ClusterState state, + protected void masterOperation(Task task, MlUpgradeAction.Request request, ClusterState state, ActionListener listener) { TaskId taskId = new TaskId(clusterService.localNode().getId(), task.getId()); ParentTaskAssigningClient parentAwareClient = new ParentTaskAssigningClient(client, taskId); @@ -58,7 +58,7 @@ protected void masterOperation(Task task, ResultsIndexUpgradeAction.Request requ } @Override - protected final void masterOperation(ResultsIndexUpgradeAction.Request request, ClusterState state, + protected final void masterOperation(MlUpgradeAction.Request request, ClusterState state, ActionListener listener) { throw new UnsupportedOperationException("the task parameter is required"); } @@ -74,7 +74,7 @@ protected AcknowledgedResponse newResponse() { } @Override - protected ClusterBlockException checkBlock(ResultsIndexUpgradeAction.Request request, ClusterState state) { + protected ClusterBlockException checkBlock(MlUpgradeAction.Request request, ClusterState state) { return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ); } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java index 33047c1fca39a..53559aee4701b 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java @@ -55,7 +55,7 @@ import org.elasticsearch.xpack.ml.job.persistence.JobResultsProvider; import org.elasticsearch.xpack.ml.job.process.autodetect.UpdateParams; import org.elasticsearch.xpack.ml.notifications.Auditor; -import org.elasticsearch.xpack.ml.utils.ChainTaskExecutor; +import org.elasticsearch.xpack.ml.utils.VoidChainTaskExecutor; import java.io.IOException; import java.util.ArrayList; @@ -397,16 +397,16 @@ private void postJobUpdate(UpdateJobAction.Request request, Job updatedJob, Acti } private void validate(Job job, JobUpdate jobUpdate, ActionListener handler) { - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(client.threadPool().executor( + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(client.threadPool().executor( MachineLearning.UTILITY_THREAD_POOL_NAME), true); - validateModelSnapshotIdUpdate(job, jobUpdate.getModelSnapshotId(), chainTaskExecutor); - validateAnalysisLimitsUpdate(job, jobUpdate.getAnalysisLimits(), chainTaskExecutor); - chainTaskExecutor.execute(handler); + validateModelSnapshotIdUpdate(job, jobUpdate.getModelSnapshotId(), voidChainTaskExecutor); + validateAnalysisLimitsUpdate(job, jobUpdate.getAnalysisLimits(), voidChainTaskExecutor); + voidChainTaskExecutor.execute(ActionListener.wrap(aVoids -> handler.onResponse(null), handler::onFailure)); } - private void validateModelSnapshotIdUpdate(Job job, String modelSnapshotId, ChainTaskExecutor chainTaskExecutor) { + private void validateModelSnapshotIdUpdate(Job job, String modelSnapshotId, VoidChainTaskExecutor voidChainTaskExecutor) { if (modelSnapshotId != null) { - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { jobResultsProvider.getModelSnapshot(job.getId(), modelSnapshotId, newModelSnapshot -> { if (newModelSnapshot == null) { String message = Messages.getMessage(Messages.REST_NO_SUCH_MODEL_SNAPSHOT, modelSnapshotId, @@ -428,12 +428,12 @@ private void validateModelSnapshotIdUpdate(Job job, String modelSnapshotId, Chai } } - private void validateAnalysisLimitsUpdate(Job job, AnalysisLimits newLimits, ChainTaskExecutor chainTaskExecutor) { + private void validateAnalysisLimitsUpdate(Job job, AnalysisLimits newLimits, VoidChainTaskExecutor voidChainTaskExecutor) { if (newLimits == null || newLimits.getModelMemoryLimit() == null) { return; } Long newModelMemoryLimit = newLimits.getModelMemoryLimit(); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { if (isJobOpen(clusterService.state(), job.getId())) { listener.onFailure(ExceptionsHelper.badRequestException("Cannot update " + Job.ANALYSIS_LIMITS.getPreferredName() + " while the job is open")); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java index 0f8d95a6a4382..b568a1dbd6b5a 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java @@ -19,7 +19,7 @@ import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.tasks.LoggingTaskListener; import org.elasticsearch.tasks.Task; -import org.elasticsearch.xpack.core.ml.action.ResultsIndexUpgradeAction; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; import org.elasticsearch.xpack.ml.MachineLearning; import java.io.IOException; @@ -49,19 +49,19 @@ public String getName() { @Override protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { - ResultsIndexUpgradeAction.Request parsedRequest = new ResultsIndexUpgradeAction.Request(); + MlUpgradeAction.Request parsedRequest = new MlUpgradeAction.Request(); if (restRequest.hasContent()) { XContentParser parser = restRequest.contentParser(); - parsedRequest = ResultsIndexUpgradeAction.Request.PARSER.apply(parser, null); + parsedRequest = MlUpgradeAction.Request.PARSER.apply(parser, null); } - final ResultsIndexUpgradeAction.Request upgradeRequest = parsedRequest; + final MlUpgradeAction.Request upgradeRequest = parsedRequest; if (restRequest.paramAsBoolean("wait_for_completion", false)) { - return channel -> client.execute(ResultsIndexUpgradeAction.INSTANCE, upgradeRequest, new RestToXContentListener<>(channel)); + return channel -> client.execute(MlUpgradeAction.INSTANCE, upgradeRequest, new RestToXContentListener<>(channel)); } else { upgradeRequest.setShouldStoreResult(true); - Task task = client.executeLocally(ResultsIndexUpgradeAction.INSTANCE, upgradeRequest, LoggingTaskListener.instance()); + Task task = client.executeLocally(MlUpgradeAction.INSTANCE, upgradeRequest, LoggingTaskListener.instance()); // Send task description id instead of waiting for the message return channel -> { try (XContentBuilder builder = channel.newBuilder()) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutor.java deleted file mode 100644 index 9a0ddb5dd4add..0000000000000 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutor.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.ml.utils; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; - -import java.util.LinkedList; -import java.util.Objects; -import java.util.concurrent.ExecutorService; - -/** - * A utility that allows chained (serial) execution of a number of tasks - * in async manner. - */ -public class ChainTaskExecutor { - - public interface ChainTask { - void run(ActionListener listener); - } - - private final ExecutorService executorService; - private final boolean shortCircuit; - private final LinkedList tasks = new LinkedList<>(); - - public ChainTaskExecutor(ExecutorService executorService, boolean shortCircuit) { - this.executorService = Objects.requireNonNull(executorService); - this.shortCircuit = shortCircuit; - } - - public synchronized void add(ChainTask task) { - tasks.add(task); - } - - public synchronized void execute(ActionListener listener) { - if (tasks.isEmpty()) { - listener.onResponse(null); - return; - } - ChainTask task = tasks.pop(); - executorService.execute(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - if (shortCircuit) { - listener.onFailure(e); - } else { - execute(listener); - } - } - - @Override - protected void doRun() { - task.run(ActionListener.wrap(nullValue -> execute(listener), this::onFailure)); - } - }); - } -} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutor.java new file mode 100644 index 0000000000000..8351c0a81aaf6 --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutor.java @@ -0,0 +1,26 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.utils; + +import java.util.concurrent.ExecutorService; +import java.util.function.Predicate; + +/** + * A utility that allows chained (serial) execution of a number of tasks + * in async manner. + */ +public class VoidChainTaskExecutor extends TypedChainTaskExecutor { + + public VoidChainTaskExecutor(ExecutorService executorService, boolean shortCircuit) { + this(executorService, (a) -> true, (e) -> shortCircuit); + } + + VoidChainTaskExecutor(ExecutorService executorService, + Predicate continuationPredicate, + Predicate failureShortCircuitPredicate) { + super(executorService, continuationPredicate, failureShortCircuitPredicate); + } +} diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutorTests.java similarity index 62% rename from x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutorTests.java rename to x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutorTests.java index 87b83852ff56c..44bf4cf75aa13 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutorTests.java @@ -19,7 +19,7 @@ import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; -public class ChainTaskExecutorTests extends ESTestCase { +public class VoidChainTaskExecutorTests extends ESTestCase { private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); private final CountDownLatch latch = new CountDownLatch(1); @@ -36,18 +36,18 @@ public void tearDown() throws Exception { public void testExecute() throws InterruptedException { final List strings = new ArrayList<>(); - ActionListener finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.generic(), false); - chainTaskExecutor.add(listener -> { + ActionListener> finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.generic(), false); + voidChainTaskExecutor.add(listener -> { strings.add("first"); listener.onResponse(null); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { strings.add("second"); listener.onResponse(null); }); - chainTaskExecutor.execute(finalListener); + voidChainTaskExecutor.execute(finalListener); latch.await(); @@ -56,22 +56,22 @@ public void testExecute() throws InterruptedException { public void testExecute_GivenSingleFailureAndShortCircuit() throws InterruptedException { final List strings = new ArrayList<>(); - ActionListener finalListener = createBlockingListener(() -> fail(), + ActionListener> finalListener = createBlockingListener(() -> fail(), e -> assertThat(e.getMessage(), equalTo("some error"))); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.generic(), true); - chainTaskExecutor.add(listener -> { + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.generic(), true); + voidChainTaskExecutor.add(listener -> { strings.add("before"); listener.onResponse(null); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { throw new RuntimeException("some error"); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { strings.add("after"); listener.onResponse(null); }); - chainTaskExecutor.execute(finalListener); + voidChainTaskExecutor.execute(finalListener); latch.await(); @@ -80,21 +80,21 @@ public void testExecute_GivenSingleFailureAndShortCircuit() throws InterruptedEx public void testExecute_GivenMultipleFailuresAndShortCircuit() throws InterruptedException { final List strings = new ArrayList<>(); - ActionListener finalListener = createBlockingListener(() -> fail(), + ActionListener> finalListener = createBlockingListener(() -> fail(), e -> assertThat(e.getMessage(), equalTo("some error 1"))); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.generic(), true); - chainTaskExecutor.add(listener -> { + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.generic(), true); + voidChainTaskExecutor.add(listener -> { strings.add("before"); listener.onResponse(null); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { throw new RuntimeException("some error 1"); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { throw new RuntimeException("some error 2"); }); - chainTaskExecutor.execute(finalListener); + voidChainTaskExecutor.execute(finalListener); latch.await(); @@ -103,21 +103,21 @@ public void testExecute_GivenMultipleFailuresAndShortCircuit() throws Interrupte public void testExecute_GivenFailureAndNoShortCircuit() throws InterruptedException { final List strings = new ArrayList<>(); - ActionListener finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.generic(), false); - chainTaskExecutor.add(listener -> { + ActionListener> finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.generic(), false); + voidChainTaskExecutor.add(listener -> { strings.add("before"); listener.onResponse(null); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { throw new RuntimeException("some error"); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { strings.add("after"); listener.onResponse(null); }); - chainTaskExecutor.execute(finalListener); + voidChainTaskExecutor.execute(finalListener); latch.await(); @@ -126,17 +126,17 @@ public void testExecute_GivenFailureAndNoShortCircuit() throws InterruptedExcept public void testExecute_GivenNoTasksAdded() throws InterruptedException { final List strings = new ArrayList<>(); - ActionListener finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.generic(), false); + ActionListener> finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.generic(), false); - chainTaskExecutor.execute(finalListener); + voidChainTaskExecutor.execute(finalListener); latch.await(); assertThat(strings, contains("last")); } - private ActionListener createBlockingListener(Runnable runnable, Consumer errorHandler) { + private ActionListener> createBlockingListener(Runnable runnable, Consumer errorHandler) { return ActionListener.wrap(nullValue -> { runnable.run(); latch.countDown(); @@ -145,4 +145,4 @@ private ActionListener createBlockingListener(Runnable runnable, Consumer< latch.countDown(); }); } -} \ No newline at end of file +} From 4e926f979c975fb25b55c520cd84e5d0b04a6418 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Mon, 7 Jan 2019 12:53:30 -0600 Subject: [PATCH 17/21] Adjusting rest endpoint name --- .../org/elasticsearch/xpack/ml/MachineLearning.java | 4 ++-- ...deResultsAction.java => RestUpgradeMlAction.java} | 12 ++++++------ .../{ml.upgrade_job_results.json => ml.upgrade.json} | 6 +++--- .../test/ml/results_indices_upgrade.yml | 4 ++-- .../test/upgraded_cluster/80_ml_results_upgrade.yml | 4 ++-- 5 files changed, 15 insertions(+), 15 deletions(-) rename x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/{RestUpgradeResultsAction.java => RestUpgradeMlAction.java} (86%) rename x-pack/plugin/src/test/resources/rest-api-spec/api/{ml.upgrade_job_results.json => ml.upgrade.json} (72%) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java index c025a36c6e510..418add2757fbc 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java @@ -231,7 +231,7 @@ import org.elasticsearch.xpack.ml.rest.results.RestGetInfluencersAction; import org.elasticsearch.xpack.ml.rest.results.RestGetOverallBucketsAction; import org.elasticsearch.xpack.ml.rest.results.RestGetRecordsAction; -import org.elasticsearch.xpack.ml.rest.results.RestUpgradeResultsAction; +import org.elasticsearch.xpack.ml.rest.results.RestUpgradeMlAction; import org.elasticsearch.xpack.ml.rest.validate.RestValidateDetectorAction; import org.elasticsearch.xpack.ml.rest.validate.RestValidateJobConfigAction; @@ -545,7 +545,7 @@ public List getRestHandlers(Settings settings, RestController restC new RestGetCalendarEventsAction(settings, restController), new RestPostCalendarEventAction(settings, restController), new RestFindFileStructureAction(settings, restController), - new RestUpgradeResultsAction(settings, restController) + new RestUpgradeMlAction(settings, restController) ); } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeMlAction.java similarity index 86% rename from x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java rename to x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeMlAction.java index b568a1dbd6b5a..cad82ce325c27 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeResultsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeMlAction.java @@ -26,25 +26,25 @@ import static org.elasticsearch.rest.RestRequest.Method.POST; -public class RestUpgradeResultsAction extends BaseRestHandler { +public class RestUpgradeMlAction extends BaseRestHandler { private static final DeprecationLogger deprecationLogger = - new DeprecationLogger(LogManager.getLogger(RestUpgradeResultsAction.class)); + new DeprecationLogger(LogManager.getLogger(RestUpgradeMlAction.class)); - public RestUpgradeResultsAction(Settings settings, RestController controller) { + public RestUpgradeMlAction(Settings settings, RestController controller) { super(settings); controller.registerWithDeprecatedHandler( POST, - MachineLearning.BASE_PATH + "anomaly_detectors/results/_upgrade", + MachineLearning.BASE_PATH + "_upgrade", this, POST, - MachineLearning.PRE_V7_BASE_PATH + "anomaly_detectors/results/_upgrade", + MachineLearning.PRE_V7_BASE_PATH + "_upgrade", deprecationLogger); } @Override public String getName() { - return "xpack_ml_upgrade_results_indices_action"; + return "xpack_ml_upgrade_action"; } @Override diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade_job_results.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade.json similarity index 72% rename from x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade_job_results.json rename to x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade.json index 94fb0d89b1e59..b67b125bb692a 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade_job_results.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade.json @@ -1,10 +1,10 @@ { - "ml.upgrade_job_results": { + "ml.upgrade": { "documentation": "TODO", "methods": [ "POST" ], "url": { - "path": "/_ml/anomaly_detectors/results/_upgrade", - "paths": [ "/_ml/anomaly_detectors/results/_upgrade" ], + "path": "/_ml/_upgrade", + "paths": [ "/_ml/_upgrade" ], "params": { "wait_for_completion": { "type": "boolean", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/results_indices_upgrade.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/results_indices_upgrade.yml index 5f53ba58b77de..ee1f9f77f9325 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/results_indices_upgrade.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/results_indices_upgrade.yml @@ -43,7 +43,7 @@ setup: --- "Upgrade results when there is nothing to upgrade": - do: - ml.upgrade_job_results: + ml.upgrade: wait_for_completion: true - match: { acknowledged: true } @@ -56,7 +56,7 @@ setup: --- "Upgrade results when there is nothing to upgrade not waiting for results": - do: - ml.upgrade_job_results: + ml.upgrade: wait_for_completion: false - match: {task: '/.+:\d+/'} diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml index dc9fc0224cc94..f049b9c073ad8 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml @@ -32,7 +32,7 @@ # Do the upgrade - do: - ml.upgrade_job_results: + ml.upgrade: wait_for_completion: true - match: { acknowledged: true } @@ -146,7 +146,7 @@ # Do the upgrade Again as nothing needs upgraded now - do: - ml.upgrade_job_results: + ml.upgrade: wait_for_completion: true - match: { acknowledged: true } From 1661c458610a845d40e0e7bccd173651e12137b3 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Mon, 7 Jan 2019 13:17:25 -0600 Subject: [PATCH 18/21] making service have its own logger --- .../xpack/ml/integration/ResultsIndexUpgradeIT.java | 3 --- .../elasticsearch/xpack/ml/ResultsIndexUpgradeService.java | 7 +++---- .../xpack/ml/action/TransportMlUpgradeAction.java | 1 - 3 files changed, 3 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java index 767ea34a8ab28..5132f27688016 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java @@ -123,7 +123,6 @@ public void testMigration() throws Exception { IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); ResultsIndexUpgradeService resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, - logger, ThreadPool.Names.SAME, indexMetaData -> true); @@ -207,7 +206,6 @@ public void testMigrationWithManuallyCreatedIndexThatNeedsMigrating() throws Exc IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); ResultsIndexUpgradeService resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, - logger, ThreadPool.Names.SAME, indexMetaData -> true); //indicates that this manually created index needs migrated @@ -262,7 +260,6 @@ public void testMigrationWithExistingIndexWithData() throws Exception { IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); ResultsIndexUpgradeService resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, - logger, ThreadPool.Names.SAME, //indicates that this manually created index is already migrated and should not be included in our migration steps indexMetaData -> !(indexMetaData.getIndex().getName().equals(alreadyMigratedIndex) || diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java index 9d36674da1901..5bc7c764f70a0 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.ml; +import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchTimeoutException; @@ -58,6 +59,8 @@ */ public class ResultsIndexUpgradeService { + private static final Logger logger = LogManager.getLogger(ResultsIndexUpgradeService.class); + // Adjust the following constants as necessary for various versions and backports. private static final int INDEX_VERSION = Version.CURRENT.major; private static final Version MIN_REQUIRED_VERSION = Version.CURRENT.minimumCompatibilityVersion(); @@ -65,25 +68,21 @@ public class ResultsIndexUpgradeService { private final IndexNameExpressionResolver indexNameExpressionResolver; private final Predicate shouldUpgrade; private final String executor; - private final Logger logger; /** * Construct a new upgrade service * * @param indexNameExpressionResolver Index expression resolver for the request - * @param logger The logger of the creating object * @param executor Where to execute client calls * @param shouldUpgrade Given IndexMetadata indicate if it should be upgraded or not * {@code true} indicates that it SHOULD upgrade */ public ResultsIndexUpgradeService(IndexNameExpressionResolver indexNameExpressionResolver, - Logger logger, String executor, Predicate shouldUpgrade) { this.indexNameExpressionResolver = indexNameExpressionResolver; this.shouldUpgrade = shouldUpgrade; this.executor = executor; - this.logger = logger; } public static boolean wasIndexCreatedInCurrentMajorVersion(IndexMetaData indexMetaData) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportMlUpgradeAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportMlUpgradeAction.java index 9f651c6892fb8..2b676277aa690 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportMlUpgradeAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportMlUpgradeAction.java @@ -40,7 +40,6 @@ public TransportMlUpgradeAction(TransportService transportService, ClusterServic actionFilters, MlUpgradeAction.Request::new, indexNameExpressionResolver); this.client = client; this.resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, - logger, executor(), indexMetadata -> wasIndexCreatedInCurrentMajorVersion(indexMetadata) == false); } From cd9c74747ae0029707398a2ee2732bbdea6d4b06 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Mon, 7 Jan 2019 13:32:45 -0600 Subject: [PATCH 19/21] adjusting validity check for newindex names --- .../xpack/ml/ResultsIndexUpgradeService.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java index 5bc7c764f70a0..ccbaed13feca0 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java @@ -100,10 +100,10 @@ public static boolean wasIndexCreatedInCurrentMajorVersion(IndexMetaData indexMe * @param metaData Cluster metadata * @param newIndexName The index to check * @param shouldUpgrade Should be index be upgraded - * @return {@code true} if the "new index" is not considered valid + * @return {@code true} if the "new index" is valid */ - private static boolean invalidIndexForUpgrade(MetaData metaData, String newIndexName, Predicate shouldUpgrade) { - return metaData.hasIndex(newIndexName) && shouldUpgrade.test(metaData.index(newIndexName)); + private static boolean validNewIndex(MetaData metaData, String newIndexName, Predicate shouldUpgrade) { + return (metaData.hasIndex(newIndexName) && shouldUpgrade.test(metaData.index(newIndexName))) == false; } private static void validateMinNodeVersion(ClusterState clusterState) { @@ -454,12 +454,12 @@ private Exception validate(MetaData metaData, Predicate shouldUpg for (String index : oldIndices) { String newWriteName = newWriteName(index); // If the "new" indices exist, either they were created from a previous run of the upgrade process or the end user - if (invalidIndexForUpgrade(metaData, newWriteName, shouldUpgrade)) { + if (validNewIndex(metaData, newWriteName, shouldUpgrade) == false) { return new IllegalStateException("Index [" + newWriteName + "] already exists and is not the current version."); } String newReadName = newReadName(index); - if (invalidIndexForUpgrade(metaData, newReadName, shouldUpgrade)) { + if (validNewIndex(metaData, newReadName, shouldUpgrade) == false) { return new IllegalStateException("Index [" + newReadName + "] already exists and is not the current version."); } } From 674463c00fa7b7ac84a55ae985c6de4c11cd83cd Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Tue, 8 Jan 2019 14:39:25 -0600 Subject: [PATCH 20/21] fixing typos --- .../elasticsearch/xpack/core/ml/action/MlUpgradeAction.java | 4 ++-- .../elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeAction.java index 993637ee086c3..404f15d4f6270 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeAction.java @@ -45,7 +45,7 @@ public static class Request extends MasterNodeReadRequest implements To private static final ParseField REINDEX_BATCH_SIZE = new ParseField("reindex_batch_size"); - public static ObjectParser PARSER = new ObjectParser<>("ml_results_upgrade", true, Request::new); + public static ObjectParser PARSER = new ObjectParser<>("ml_upgrade", true, Request::new); static { PARSER.declareInt(Request::setReindexBatchSize, REINDEX_BATCH_SIZE); } @@ -133,7 +133,7 @@ public int hashCode() { @Override public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { - return new CancellableTask(id, type, action, getDescription(), parentTaskId, headers) { + return new CancellableTask(id, type, action, "ml-upgrade", parentTaskId, headers) { @Override public boolean shouldCancelChildrenOnCancellation() { return true; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java index 1cfe6a31ceeed..5af9c53649853 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java @@ -36,7 +36,7 @@ public interface ChainTask { * Creates a new TypedChainTaskExecutor. * Each chainedTask is executed in order serially and after each execution the continuationPredicate is tested. * - * On failures teh failureShortCircuitPredicate is tested. + * On failures the failureShortCircuitPredicate is tested. * * @param executorService The service where to execute the tasks * @param continuationPredicate The predicate to test on whether to execute the next task or not. From 2d67895720b91dd25d70988a05b26d609ebc9821 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 9 Jan 2019 07:48:28 -0600 Subject: [PATCH 21/21] fixing renaming --- x-pack/plugin/ml/qa/ml-with-security/build.gradle | 4 ++-- .../{ResultsIndexUpgradeIT.java => MlUpgradeIT.java} | 2 +- .../test/ml/{results_indices_upgrade.yml => ml_upgrade.yml} | 0 3 files changed, 3 insertions(+), 3 deletions(-) rename x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/{ResultsIndexUpgradeIT.java => MlUpgradeIT.java} (99%) rename x-pack/plugin/src/test/resources/rest-api-spec/test/ml/{results_indices_upgrade.yml => ml_upgrade.yml} (100%) diff --git a/x-pack/plugin/ml/qa/ml-with-security/build.gradle b/x-pack/plugin/ml/qa/ml-with-security/build.gradle index 080a076a77d5d..6e0127f614c9a 100644 --- a/x-pack/plugin/ml/qa/ml-with-security/build.gradle +++ b/x-pack/plugin/ml/qa/ml-with-security/build.gradle @@ -94,8 +94,8 @@ integTestRunner { 'ml/validate_detector/Test invalid detector', 'ml/delete_forecast/Test delete on _all forecasts not allow no forecasts', 'ml/delete_forecast/Test delete forecast on missing forecast', - 'ml/results_indices_upgrade/Upgrade results when there is nothing to upgrade', - 'ml/results_indices_upgrade/Upgrade results when there is nothing to upgrade not waiting for results' + 'ml/ml_upgrade/Upgrade results when there is nothing to upgrade', + 'ml/ml_upgrade/Upgrade results when there is nothing to upgrade not waiting for results' ].join(',') } diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlUpgradeIT.java similarity index 99% rename from x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java rename to x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlUpgradeIT.java index 5132f27688016..a2a05ea1686fa 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/ResultsIndexUpgradeIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlUpgradeIT.java @@ -44,7 +44,7 @@ import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.core.Is.is; -public class ResultsIndexUpgradeIT extends MlNativeAutodetectIntegTestCase { +public class MlUpgradeIT extends MlNativeAutodetectIntegTestCase { @After public void cleanup() throws Exception { diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/results_indices_upgrade.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/ml_upgrade.yml similarity index 100% rename from x-pack/plugin/src/test/resources/rest-api-spec/test/ml/results_indices_upgrade.yml rename to x-pack/plugin/src/test/resources/rest-api-spec/test/ml/ml_upgrade.yml