diff --git a/build.gradle b/build.gradle index ad96eb5f7..2b773e33c 100644 --- a/build.gradle +++ b/build.gradle @@ -121,7 +121,7 @@ ext { } group = "com.amazon.opendistroforelasticsearch" -version = "${opendistroVersion}.0" +version = "${opendistroVersion}.1" dependencies { compileOnly "org.elasticsearch:elasticsearch:${es_version}" @@ -315,3 +315,46 @@ task ktlintFormat(type: JavaExec, group: "formatting") { compileKotlin { kotlinOptions.freeCompilerArgs = ['-Xjsr305=strict'] } apply from: 'build-tools/pkgbuild.gradle' + +// This IT is to simulate the situation +// when there are old version (without metadata change) +// and new version mixed in one cluster +import org.elasticsearch.gradle.test.RestIntegTestTask + +def mixedClusterTest = project.tasks.create('mixedCluster', RestIntegTestTask.class) +def mixedClusterFlag = findProperty('mixed') as Boolean ?: false +println("mixed cluster flag: $mixedClusterFlag") +mixedClusterTest.dependsOn(bundlePlugin) + +testClusters.mixedCluster { + testDistribution = "OSS" + if (_numNodes > 1) numberOfNodes = _numNodes + getNodes().each { node -> + node.plugin(provider({ + new RegularFile() { + @Override + File getAsFile() { fileTree("src/test/resources/job-scheduler").getSingleFile() } + } + })) + + if (mixedClusterFlag && node.name == "mixedCluster-1") { + node.plugin(provider({ + new RegularFile() { + @Override + File getAsFile() { fileTree("src/test/resources/index-management").getSingleFile() } + } + })) + } else { + node.plugin(project.tasks.bundlePlugin.archiveFile) + } + node.plugins.each { println("plugin in the node: ${it.get()}") } + } + setting 'path.repo', repo.absolutePath +} + +mixedCluster { + systemProperty 'tests.security.manager', 'false' + systemProperty 'tests.path.repo', repo.absolutePath + systemProperty 'cluster.mixed', "$mixedClusterFlag" + systemProperty 'cluster.number_of_nodes', "${_numNodes}" +} diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementIndices.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementIndices.kt index ffba055ad..7d3aa0534 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementIndices.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementIndices.kt @@ -13,6 +13,7 @@ * permissions and limitations under the License. */ +@file:Suppress("ReturnCount") package com.amazon.opendistroforelasticsearch.indexmanagement import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementPlugin.Companion.INDEX_MANAGEMENT_INDEX @@ -84,7 +85,6 @@ class IndexManagementIndices( /** * Attempt to create [INDEX_MANAGEMENT_INDEX] and return whether it exists */ - @Suppress("ReturnCount") suspend fun attemptInitStateManagementIndex(client: Client): Boolean { if (indexManagementIndexExists()) return true @@ -103,6 +103,19 @@ class IndexManagementIndices( } } + suspend fun attemptUpdateConfigIndexMapping(): Boolean { + return try { + val response: AcknowledgedResponse = client.suspendUntil { + IndexUtils.checkAndUpdateConfigIndexMapping(clusterService.state(), client, it) } + if (response.isAcknowledged) return true + logger.error("Trying to update config index mapping not acknowledged.") + return false + } catch (e: Exception) { + logger.error("Failed when trying to update config index mapping.", e) + false + } + } + /** * ============== History ============= */ diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementPlugin.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementPlugin.kt index 02e7ac5ef..60b0992ce 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementPlugin.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementPlugin.kt @@ -18,9 +18,12 @@ package com.amazon.opendistroforelasticsearch.indexmanagement import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.IndexStateManagementHistory import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.ManagedIndexCoordinator import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.ManagedIndexRunner +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.MetadataService +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.SkipExecution import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.TransportUpdateManagedIndexMetaDataAction import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataAction import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexConfig +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.Policy import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.resthandler.RestAddPolicyAction import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.resthandler.RestChangePolicyAction @@ -165,6 +168,9 @@ internal class IndexManagementPlugin : JobSchedulerExtension, NetworkPlugin, Act RollupMetadata.ROLLUP_METADATA_TYPE -> { return@ScheduledJobParser null } + ManagedIndexMetaData.MANAGED_INDEX_METADATA_TYPE -> { + return@ScheduledJobParser null + } else -> { logger.warn("Unsupported document was indexed in $INDEX_MANAGEMENT_INDEX with type: $fieldName") xcp.skipChildren() @@ -218,13 +224,6 @@ internal class IndexManagementPlugin : JobSchedulerExtension, NetworkPlugin, Act ): Collection { val settings = environment.settings() this.clusterService = clusterService - val managedIndexRunner = ManagedIndexRunner - .registerClient(client) - .registerClusterService(clusterService) - .registerNamedXContentRegistry(xContentRegistry) - .registerScriptService(scriptService) - .registerSettings(settings) - .registerConsumers() // registerConsumers must happen after registerSettings/clusterService val rollupRunner = RollupRunner .registerClient(client) .registerClusterService(clusterService) @@ -239,6 +238,8 @@ internal class IndexManagementPlugin : JobSchedulerExtension, NetworkPlugin, Act .registerConsumers() rollupInterceptor = RollupInterceptor(clusterService, settings, indexNameExpressionResolver) this.indexNameExpressionResolver = indexNameExpressionResolver + + val skipFlag = SkipExecution(client, clusterService) indexManagementIndices = IndexManagementIndices(settings, client.admin().indices(), clusterService) val indexStateManagementHistory = IndexStateManagementHistory( @@ -249,8 +250,20 @@ internal class IndexManagementPlugin : JobSchedulerExtension, NetworkPlugin, Act indexManagementIndices ) + val managedIndexRunner = ManagedIndexRunner + .registerClient(client) + .registerClusterService(clusterService) + .registerNamedXContentRegistry(xContentRegistry) + .registerScriptService(scriptService) + .registerSettings(settings) + .registerConsumers() // registerConsumers must happen after registerSettings/clusterService + .registerHistoryIndex(indexStateManagementHistory) + .registerSkipFlag(skipFlag) + + val metadataService = MetadataService(client, clusterService, skipFlag, indexManagementIndices) + val managedIndexCoordinator = ManagedIndexCoordinator(environment.settings(), - client, clusterService, threadPool, indexManagementIndices) + client, clusterService, threadPool, indexManagementIndices, metadataService) return listOf(managedIndexRunner, rollupRunner, indexManagementIndices, managedIndexCoordinator, indexStateManagementHistory) } @@ -267,6 +280,7 @@ internal class IndexManagementPlugin : JobSchedulerExtension, NetworkPlugin, Act ManagedIndexSettings.POLICY_ID, ManagedIndexSettings.ROLLOVER_ALIAS, ManagedIndexSettings.INDEX_STATE_MANAGEMENT_ENABLED, + ManagedIndexSettings.METADATA_SERVICE_ENABLED, ManagedIndexSettings.JOB_INTERVAL, ManagedIndexSettings.SWEEP_PERIOD, ManagedIndexSettings.COORDINATOR_BACKOFF_COUNT, diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/IndexStateManagementHistory.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/IndexStateManagementHistory.kt index 07341e812..6b6e7cd9c 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/IndexStateManagementHistory.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/IndexStateManagementHistory.kt @@ -24,6 +24,7 @@ import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagemen import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.INDEX_HIDDEN import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.INDEX_NUMBER_OF_REPLICAS import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.INDEX_NUMBER_OF_SHARDS +import com.amazon.opendistroforelasticsearch.indexmanagement.util.OpenForTesting import com.amazon.opendistroforelasticsearch.indexmanagement.util._DOC import org.apache.logging.log4j.LogManager import org.elasticsearch.action.DocWriteRequest @@ -46,6 +47,7 @@ import org.elasticsearch.threadpool.Scheduler import org.elasticsearch.threadpool.ThreadPool import java.time.Instant +@OpenForTesting class IndexStateManagementHistory( settings: Settings, private val client: Client, diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/ManagedIndexCoordinator.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/ManagedIndexCoordinator.kt index a94b82a49..1ba9b96d9 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/ManagedIndexCoordinator.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/ManagedIndexCoordinator.kt @@ -13,6 +13,7 @@ * permissions and limitations under the License. */ +@file:Suppress("ReturnCount") package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementIndices @@ -20,11 +21,11 @@ import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementPlug import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementPlugin.Companion.INDEX_MANAGEMENT_INDEX import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.contentParser import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.parseWithType -import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.getManagedIndexMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.retry import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.filterNotNullValues import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.getPolicyToTemplateMap import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.suspendUntil +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.mgetManagedIndexMetadata import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ISMTemplate import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexConfig import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexMetaData @@ -34,12 +35,13 @@ import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagemen import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings.Companion.COORDINATOR_BACKOFF_MILLIS import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings.Companion.INDEX_STATE_MANAGEMENT_ENABLED import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings.Companion.JOB_INTERVAL +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings.Companion.METADATA_SERVICE_ENABLED import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings.Companion.SWEEP_PERIOD -import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataAction -import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataRequest import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.ISM_TEMPLATE_FIELD +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.deleteManagedIndexMetadataRequest import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.deleteManagedIndexRequest import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.getDeleteManagedIndexRequests +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.getManagedIndicesToDelete import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.getSweptManagedIndexSearchRequest import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.isFailed import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.isPolicyCompleted @@ -55,8 +57,6 @@ import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager import org.elasticsearch.ExceptionsHelper import org.elasticsearch.action.DocWriteRequest -import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest -import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse import org.elasticsearch.action.bulk.BackoffPolicy import org.elasticsearch.action.bulk.BulkRequest import org.elasticsearch.action.bulk.BulkResponse @@ -65,14 +65,12 @@ import org.elasticsearch.action.get.MultiGetResponse import org.elasticsearch.action.search.SearchPhaseExecutionException import org.elasticsearch.action.search.SearchRequest import org.elasticsearch.action.search.SearchResponse -import org.elasticsearch.action.support.IndicesOptions -import org.elasticsearch.action.support.master.AcknowledgedResponse +import org.elasticsearch.action.update.UpdateRequest import org.elasticsearch.client.Client import org.elasticsearch.cluster.ClusterChangedEvent import org.elasticsearch.cluster.ClusterState import org.elasticsearch.cluster.ClusterStateListener import org.elasticsearch.cluster.block.ClusterBlockException -import org.elasticsearch.cluster.metadata.IndexMetadata import org.elasticsearch.cluster.service.ClusterService import org.elasticsearch.common.component.LifecycleListener import org.elasticsearch.common.settings.Settings @@ -107,7 +105,8 @@ class ManagedIndexCoordinator( private val client: Client, private val clusterService: ClusterService, private val threadPool: ThreadPool, - indexManagementIndices: IndexManagementIndices + indexManagementIndices: IndexManagementIndices, + private val metadataService: MetadataService ) : ClusterStateListener, CoroutineScope by CoroutineScope(SupervisorJob() + Dispatchers.Default + CoroutineName("ManagedIndexCoordinator")), LifecycleListener() { @@ -116,9 +115,11 @@ class ManagedIndexCoordinator( private val ismIndices = indexManagementIndices private var scheduledFullSweep: Scheduler.Cancellable? = null + private var scheduledMoveMetadata: Scheduler.Cancellable? = null @Volatile private var lastFullSweepTimeNano = System.nanoTime() @Volatile private var indexStateManagementEnabled = INDEX_STATE_MANAGEMENT_ENABLED.get(settings) + @Volatile private var metadataServiceEnabled = METADATA_SERVICE_ENABLED.get(settings) @Volatile private var sweepPeriod = SWEEP_PERIOD.get(settings) @Volatile private var retryPolicy = BackoffPolicy.constantBackoff(COORDINATOR_BACKOFF_MILLIS.get(settings), COORDINATOR_BACKOFF_COUNT.get(settings)) @@ -140,6 +141,11 @@ class ManagedIndexCoordinator( indexStateManagementEnabled = it if (!indexStateManagementEnabled) disable() else enable() } + clusterService.clusterSettings.addSettingsUpdateConsumer(METADATA_SERVICE_ENABLED) { + metadataServiceEnabled = it + if (!metadataServiceEnabled) scheduledMoveMetadata?.cancel() + else initMoveMetadata() + } clusterService.clusterSettings.addSettingsUpdateConsumer(COORDINATOR_BACKOFF_MILLIS, COORDINATOR_BACKOFF_COUNT) { millis, count -> retryPolicy = BackoffPolicy.constantBackoff(millis, count) } @@ -152,14 +158,17 @@ class ManagedIndexCoordinator( fun onMaster() { // Init background sweep when promoted to being master initBackgroundSweep() + + initMoveMetadata() } fun offMaster() { // Cancel background sweep when demoted from being master scheduledFullSweep?.cancel() + + scheduledMoveMetadata?.cancel() } - @Suppress("ReturnCount") override fun clusterChanged(event: ClusterChangedEvent) { // Instead of using a LocalNodeMasterListener to track master changes, this service will // track them here to avoid conditions where master listener events run after other @@ -175,10 +184,9 @@ class ManagedIndexCoordinator( if (!isIndexStateManagementEnabled()) return - if (!event.localNodeMaster()) return + if (event.isNewCluster) return - // TODO: Look into event.isNewCluster, can we return early if true? - // if (event.isNewCluster) { } + if (!event.localNodeMaster()) return if (!event.metadataChanged()) return @@ -187,16 +195,22 @@ class ManagedIndexCoordinator( override fun afterStart() { initBackgroundSweep() + + initMoveMetadata() } override fun beforeStop() { scheduledFullSweep?.cancel() + + scheduledMoveMetadata?.cancel() } private fun enable() { initBackgroundSweep() indexStateManagementEnabled = true + initMoveMetadata() + // Calling initBackgroundSweep() beforehand runs a sweep ensuring that policies removed from indices // and indices being deleted are accounted for prior to re-enabling jobs launch { @@ -212,42 +226,39 @@ class ManagedIndexCoordinator( private fun disable() { scheduledFullSweep?.cancel() indexStateManagementEnabled = false + + scheduledMoveMetadata?.cancel() } private suspend fun reenableJobs() { - val clusterStateRequest = ClusterStateRequest() - .clear() - .metadata(true) - .local(false) - .indices("*") - .indicesOptions(IndicesOptions.strictExpand()) - - val response: ClusterStateResponse = client.admin().cluster().suspendUntil { state(clusterStateRequest, it) } - /* * Iterate through all indices and create update requests to update the ManagedIndexConfig for indices that * meet the following conditions: - * 1. Is being managed (has ManagedIndexMetaData) + * 1. Is being managed (has managed-index) * 2. Does not have a completed Policy * 3. Does not have a failed Policy */ - val updateManagedIndicesRequests: List> = response.state.metadata.indices.mapNotNull { - val managedIndexMetaData = it.value.getManagedIndexMetaData() - if (!(managedIndexMetaData == null || managedIndexMetaData.isPolicyCompleted || managedIndexMetaData.isFailed)) { - updateEnableManagedIndexRequest(it.value.indexUUID) - } else { - null + val currentManagedIndices = sweepManagedIndexJobs(client, ismIndices.indexManagementIndexExists()) + val metadataList = client.mgetManagedIndexMetadata(currentManagedIndices.map { Index(it.key, it.value.uuid) }) + val managedIndicesToEnableReq = mutableListOf() + metadataList.forEach { + val metadata = it?.first + if (metadata != null && !(metadata.isPolicyCompleted || metadata.isFailed)) { + managedIndicesToEnableReq.add(updateEnableManagedIndexRequest(metadata.indexUuid)) } } - updateManagedIndices(updateManagedIndicesRequests, false) + updateManagedIndices(managedIndicesToEnableReq, false) } private fun isIndexStateManagementEnabled(): Boolean = indexStateManagementEnabled == true + /** + * create or clean job document and metadata + */ @OpenForTesting suspend fun sweepClusterChangedEvent(event: ClusterChangedEvent) { - // deal with indices delete event + // indices delete event var removeManagedIndexReq = emptyList>() var indicesToClean = emptyList() if (event.indicesDeleted().isNotEmpty()) { @@ -256,16 +267,14 @@ class ManagedIndexCoordinator( removeManagedIndexReq = indicesToClean.map { deleteManagedIndexRequest(it.uuid) } } - // Check if newly created indices matching any ISM templates + // check if newly created indices match with any ISM templates var updateMatchingIndexReq = emptyList>() - // filter out indices with policyID, they will be picked up in previous block - if (event.indicesCreated().isNotEmpty()) // only check template match if there are new created indices + if (event.indicesCreated().isNotEmpty()) updateMatchingIndexReq = getMatchingIndicesUpdateReq(event.state(), event.indicesCreated()) updateManagedIndices(updateMatchingIndexReq + removeManagedIndexReq, updateMatchingIndexReq.isNotEmpty()) - // TODO currently metadata is auto cleaned after indices deleted - // uncomment this after move metadata to config index - // clearManagedIndexMetaData(indicesToClean) + + clearManagedIndexMetaData(indicesToClean.map { deleteManagedIndexMetadataRequest(it.uuid) }) } /** @@ -368,6 +377,36 @@ class ManagedIndexCoordinator( scheduledFullSweep = threadPool.scheduleWithFixedDelay(scheduledSweep, sweepPeriod, executorName()) } + fun initMoveMetadata() { + if (!metadataServiceEnabled) return + if (!isIndexStateManagementEnabled()) return + if (!clusterService.state().nodes().isLocalNodeElectedMaster) return + scheduledMoveMetadata?.cancel() + + if (metadataService.finishFlag) { + logger.info("Re-enable Metadata Service.") + metadataService.reenableMetadataService() + } + + val scheduledJob = Runnable { + launch { + try { + if (metadataService.finishFlag) { + logger.info("Cancel background move metadata process.") + scheduledMoveMetadata?.cancel() + } + + logger.info("Performing move cluster state metadata.") + metadataService.moveMetadata() + } catch (e: Exception) { + logger.error("Failed to move cluster state metadata", e) + } + } + } + + scheduledMoveMetadata = threadPool.scheduleWithFixedDelay(scheduledJob, TimeValue.timeValueMinutes(1), executorName()) + } + private fun getFullSweepElapsedTime(): TimeValue = TimeValue.timeValueNanos(System.nanoTime() - lastFullSweepTimeNano) @@ -379,18 +418,19 @@ class ManagedIndexCoordinator( */ @OpenForTesting suspend fun sweep() { - val currentManagedIndices = sweepManagedIndexJobs(client, ismIndices.indexManagementIndexExists()) - // get all index in the cluster state + // get all indices in the cluster state val currentIndices = clusterService.state().metadata.indices.values().map { it.value } .distinct().filterNotNull() + val currentManagedIndices = sweepManagedIndexJobs(client, ismIndices.indexManagementIndexExists()) + // check all un-managed indices, if its name matches any template val unManagedIndices = currentIndices .filter { it.index.uuid !in currentManagedIndices.keys } - .map { it.index.name }.distinct() - val updateMatchingIndicesReqs = getMatchingIndicesUpdateReq(clusterService.state(), unManagedIndices) + .map { it.index }.distinct() + val updateMatchingIndicesReqs = getMatchingIndicesUpdateReq(clusterService.state(), unManagedIndices.map { it.name }) - // check all managed indices, if the index has been deleted + // check all managed indices, if the index has already been deleted val deleteManagedIndexRequests = getDeleteManagedIndexRequests(currentIndices, currentManagedIndices) @@ -400,9 +440,9 @@ class ManagedIndexCoordinator( ) // clean metadata of un-managed index - val indicesToDeleteManagedIndexMetaDataFrom = - getIndicesToDeleteManagedIndexMetaDataFrom(currentIndices, currentManagedIndices.keys) - clearManagedIndexMetaData(indicesToDeleteManagedIndexMetaDataFrom) + val indicesToDeleteMetadataFrom = + unManagedIndices + getManagedIndicesToDelete(currentIndices, currentManagedIndices) + clearManagedIndexMetaData(indicesToDeleteMetadataFrom.map { deleteManagedIndexMetadataRequest(it.uuid) }) lastFullSweepTimeNano = System.nanoTime() } @@ -435,7 +475,6 @@ class ManagedIndexCoordinator( * * @return map of IndexUuid to [ManagedIndexConfig] */ - @Suppress("ReturnCount") suspend fun getManagedIndices(indexUuids: List): Map { if (indexUuids.isEmpty()) return emptyMap() @@ -471,11 +510,17 @@ class ManagedIndexCoordinator( logger.error("Failed to create $INDEX_MANAGEMENT_INDEX") return } + + val updated = ismIndices.attemptUpdateConfigIndexMapping() + if (!updated) { + logger.error("Failed to update mapping for $INDEX_MANAGEMENT_INDEX") + return + } } retryPolicy.retry(logger, listOf(RestStatus.TOO_MANY_REQUESTS)) { val bulkRequest = BulkRequest().add(requestsToRetry) - val bulkResponse: BulkResponse = client.suspendUntil { client.bulk(bulkRequest, it) } + val bulkResponse: BulkResponse = client.suspendUntil { bulk(bulkRequest, it) } val failedResponses = (bulkResponse.items ?: arrayOf()).filter { it.isFailed } requestsToRetry = failedResponses.filter { it.status() == RestStatus.TOO_MANY_REQUESTS } .map { bulkRequest.requests()[it.itemId] } @@ -488,37 +533,39 @@ class ManagedIndexCoordinator( } /** - * Returns [Index]es in cluster state not being managed by ISM + * Returns [Index]es not being managed by ISM * but still has ISM metadata - * - * @param currentManagedIndices current managed indices' uuids collection */ - @OpenForTesting - fun getIndicesToDeleteManagedIndexMetaDataFrom( - currentIndices: List, - currentManagedIndices: Set - ): List { - return currentIndices.filter { it.getManagedIndexMetaData() != null } - .map { it.index }.filter { it.uuid !in currentManagedIndices }.toList() + suspend fun getIndicesToRemoveMetadataFrom(unManagedIndices: List): List { + val indicesToRemoveManagedIndexMetaDataFrom = mutableListOf() + val metadataList = client.mgetManagedIndexMetadata(unManagedIndices) + metadataList.forEach { + val metadata = it?.first + if (metadata != null) + indicesToRemoveManagedIndexMetaDataFrom.add(Index(metadata.index, metadata.indexUuid)) + } + return indicesToRemoveManagedIndexMetaDataFrom } - /** Removes the [ManagedIndexMetaData] from the given list of [Index]es. */ + /** + * Removes the [ManagedIndexMetaData] from the given list of [Index]es. + */ @OpenForTesting @Suppress("TooGenericExceptionCaught") - suspend fun clearManagedIndexMetaData(indices: List) { + suspend fun clearManagedIndexMetaData(deleteRequests: List>) { try { - // If list of indices is empty, no request necessary - if (indices.isEmpty()) return - - val request = UpdateManagedIndexMetaDataRequest(indicesToRemoveManagedIndexMetaDataFrom = indices) + if (deleteRequests.isEmpty()) return retryPolicy.retry(logger) { - val response: AcknowledgedResponse = client.suspendUntil { execute(UpdateManagedIndexMetaDataAction.INSTANCE, request, it) } - - if (!response.isAcknowledged) logger.error("Failed to remove ManagedIndexMetaData for [indices=$indices]") + val bulkRequest = BulkRequest().add(deleteRequests) + val bulkResponse: BulkResponse = client.suspendUntil { bulk(bulkRequest, it) } + bulkResponse.forEach { + if (it.isFailed) logger.error("Failed to clear ManagedIndexMetadata for " + + "index uuid: [${it.id}], failureMessage: ${it.failureMessage}") + } } } catch (e: Exception) { - logger.error("Failed to remove ManagedIndexMetaData for [indices=$indices]", e) + logger.error("Failed to clear ManagedIndexMetadata ", e) } } diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/ManagedIndexRunner.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/ManagedIndexRunner.kt index d0402109f..b6ded1143 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/ManagedIndexRunner.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/ManagedIndexRunner.kt @@ -21,7 +21,7 @@ import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.convertT import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.parseWithType import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataAction import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataRequest -import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.getManagedIndexMetaData +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.getManagedIndexMetadata import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.retry import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.string import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.suspendUntil @@ -51,8 +51,10 @@ import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagemen import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.hasVersionConflict import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.isAllowed import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.isFailed +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.isMetadataMoved import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.isSafeToChange import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.isSuccessfulDelete +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.managedIndexMetadataIndexRequest import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.shouldBackoff import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.shouldChangePolicy import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.updateDisableManagedIndexRequest @@ -64,6 +66,7 @@ import com.amazon.opendistroforelasticsearch.jobscheduler.spi.schedule.IntervalS import kotlinx.coroutines.CoroutineName import kotlinx.coroutines.CoroutineScope import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.GlobalScope import kotlinx.coroutines.SupervisorJob import kotlinx.coroutines.launch import kotlinx.coroutines.withContext @@ -96,6 +99,7 @@ import org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken import org.elasticsearch.common.xcontent.XContentType import org.elasticsearch.index.Index import org.elasticsearch.index.engine.VersionConflictEngineException +import org.elasticsearch.index.seqno.SequenceNumbers import org.elasticsearch.rest.RestStatus import org.elasticsearch.script.Script import org.elasticsearch.script.ScriptService @@ -114,6 +118,8 @@ object ManagedIndexRunner : ScheduledJobRunner, private lateinit var xContentRegistry: NamedXContentRegistry private lateinit var scriptService: ScriptService private lateinit var settings: Settings + private lateinit var ismHistory: IndexStateManagementHistory + private lateinit var skipExecFlag: SkipExecution private var indexStateManagementEnabled: Boolean = DEFAULT_ISM_ENABLED @Suppress("MagicNumber") private val savePolicyRetryPolicy = BackoffPolicy.exponentialBackoff(TimeValue.timeValueMillis(250), 3) @@ -171,22 +177,37 @@ object ManagedIndexRunner : ScheduledJobRunner, return this } + fun registerHistoryIndex(ismHistory: IndexStateManagementHistory): ManagedIndexRunner { + this.ismHistory = ismHistory + return this + } + + fun registerSkipFlag(flag: SkipExecution): ManagedIndexRunner { + this.skipExecFlag = flag + return this + } + override fun runJob(job: ScheduledJobParameter, context: JobExecutionContext) { if (job !is ManagedIndexConfig) { throw IllegalArgumentException("Invalid job type, found ${job.javaClass.simpleName} with id: ${context.jobId}") } launch { + if (skipExecFlag.flag) { + logger.info("Cluster still has nodes running old version ISM plugin, skip execution on new nodes until all nodes upgraded") + return@launch + } + // Attempt to acquire lock val lock: LockModel? = context.lockService.suspendUntil { acquireLock(job, context, it) } if (lock == null) { - logger.debug("Could not acquire lock for ${job.index}") + logger.debug("Could not acquire lock [${lock?.lockId}] for ${job.index}") } else { runManagedIndexConfig(job) // Release lock val released: Boolean = context.lockService.suspendUntil { release(lock, it) } if (!released) { - logger.debug("Could not release lock for ${job.index}") + logger.debug("Could not release lock [${lock.lockId}] for ${job.index}") } } } @@ -194,6 +215,7 @@ object ManagedIndexRunner : ScheduledJobRunner, @Suppress("ReturnCount", "ComplexMethod", "LongMethod", "ComplexCondition") private suspend fun runManagedIndexConfig(managedIndexConfig: ManagedIndexConfig) { + logger.debug("Run job for index ${managedIndexConfig.index}") // doing a check of local cluster health as we do not want to overload master node with potentially a lot of calls if (clusterIsRed()) { logger.debug("Skipping current execution of ${managedIndexConfig.index} because of red cluster health") @@ -201,14 +223,22 @@ object ManagedIndexRunner : ScheduledJobRunner, } // Get current IndexMetaData and ManagedIndexMetaData - val indexMetaData = getIndexMetaData(managedIndexConfig.index) + val indexMetaData = getIndexMetadata(managedIndexConfig.index) if (indexMetaData == null) { + logger.warn("Failed to retrieve IndexMetadata.") + return + } + val managedIndexMetaData = indexMetaData.getManagedIndexMetadata(client) + val clusterStateMetadata = indexMetaData.getManagedIndexMetadata() + + if (!isMetadataMoved(clusterStateMetadata, managedIndexMetaData, logger)) { + logger.info("Skipping execution while pending migration of metadata for ${managedIndexConfig.jobName}") return } - val managedIndexMetaData = indexMetaData.getManagedIndexMetaData() // If policy or managedIndexMetaData is null then initialize val policy = managedIndexConfig.policy + if (policy == null || managedIndexMetaData == null) { initManagedIndex(managedIndexConfig, managedIndexMetaData) return @@ -222,8 +252,14 @@ object ManagedIndexRunner : ScheduledJobRunner, if (managedIndexMetaData.hasVersionConflict(managedIndexConfig)) { val info = mapOf("message" to "There is a version conflict between your previous execution and your managed index") - val updated = updateManagedIndexMetaData(managedIndexMetaData.copy(policyRetryInfo = PolicyRetryInfoMetaData(true, 0), info = info)) - if (updated) disableManagedIndexConfig(managedIndexConfig) + val result = updateManagedIndexMetaData( + managedIndexMetaData.copy( + policyRetryInfo = PolicyRetryInfoMetaData(true, 0), + info = info + )) + if (result.metadataSaved) { + disableManagedIndexConfig(managedIndexConfig) + } return } @@ -244,7 +280,7 @@ object ManagedIndexRunner : ScheduledJobRunner, logger.error("Action=${action.type.type} has timed out") val updated = updateManagedIndexMetaData(managedIndexMetaData .copy(actionMetaData = currentActionMetaData?.copy(failed = true), info = info)) - if (updated) disableManagedIndexConfig(managedIndexConfig) + if (updated.metadataSaved) disableManagedIndexConfig(managedIndexConfig) return } @@ -265,8 +301,9 @@ object ManagedIndexRunner : ScheduledJobRunner, logger.info("Previous execution failed to update step status, isIdempotent=$isIdempotent") if (isIdempotent != true) { val info = mapOf("message" to "Previous action was not able to update IndexMetaData.") - val updated = updateManagedIndexMetaData(managedIndexMetaData.copy(policyRetryInfo = PolicyRetryInfoMetaData(true, 0), info = info)) - if (updated) disableManagedIndexConfig(managedIndexConfig) + val updated = updateManagedIndexMetaData(managedIndexMetaData.copy( + policyRetryInfo = PolicyRetryInfoMetaData(true, 0), info = info)) + if (updated.metadataSaved) disableManagedIndexConfig(managedIndexConfig) return } } @@ -275,8 +312,9 @@ object ManagedIndexRunner : ScheduledJobRunner, // as this action has been removed from the AllowList, but if its not the first step we will let it finish as it's already inflight if (action?.isAllowed(allowList) == false && action.isFirstStep(step?.name)) { val info = mapOf("message" to "Attempted to execute action=${action.type.type} which is not allowed.") - val updated = updateManagedIndexMetaData(managedIndexMetaData.copy(policyRetryInfo = PolicyRetryInfoMetaData(true, 0), info = info)) - if (updated) disableManagedIndexConfig(managedIndexConfig) + val updated = updateManagedIndexMetaData(managedIndexMetaData.copy( + policyRetryInfo = PolicyRetryInfoMetaData(true, 0), info = info)) + if (updated.metadataSaved) disableManagedIndexConfig(managedIndexConfig) return } @@ -284,7 +322,8 @@ object ManagedIndexRunner : ScheduledJobRunner, val startingManagedIndexMetaData = managedIndexMetaData.getStartingManagedIndexMetaData(state, action, step) val updateResult = updateManagedIndexMetaData(startingManagedIndexMetaData) - if (updateResult && state != null && action != null && step != null && currentActionMetaData != null) { + @Suppress("ComplexCondition") + if (updateResult.metadataSaved && state != null && action != null && step != null && currentActionMetaData != null) { // Step null check is done in getStartingManagedIndexMetaData step.preExecute(logger).execute().postExecute(logger) var executedManagedIndexMetaData = startingManagedIndexMetaData.getCompletedManagedIndexMetaData(action, step) @@ -306,7 +345,7 @@ object ManagedIndexRunner : ScheduledJobRunner, return } - if (!updateManagedIndexMetaData(executedManagedIndexMetaData)) { + if (!updateManagedIndexMetaData(executedManagedIndexMetaData, updateResult).metadataSaved) { logger.error("Failed to update ManagedIndexMetaData after executing the Step : ${step.name}") } @@ -489,7 +528,7 @@ object ManagedIndexRunner : ScheduledJobRunner, ) else -> // else this means we either tried to load a policy with a different id, seqno, or primaryterm from what is - // in the metadata and we cannot gaurantee it will work with the current state in managedIndexMetaData + // in the metadata and we cannot guarantee it will work with the current state in managedIndexMetaData managedIndexMetaData.copy( policyRetryInfo = PolicyRetryInfoMetaData(failed = true, consumedRetries = 0), info = mapOf("message" to "Fail to load policy: ${policy.id} with " + @@ -500,31 +539,72 @@ object ManagedIndexRunner : ScheduledJobRunner, } } - @Suppress("TooGenericExceptionCaught") - private suspend fun updateManagedIndexMetaData(managedIndexMetaData: ManagedIndexMetaData): Boolean { + // delete metadata in cluster state + private suspend fun deleteManagedIndexMetaData(managedIndexMetaData: ManagedIndexMetaData): Boolean { var result = false try { val request = UpdateManagedIndexMetaDataRequest( - indicesToAddManagedIndexMetaDataTo = listOf( - Pair(Index(managedIndexMetaData.index, managedIndexMetaData.indexUuid), managedIndexMetaData) - ) + indicesToRemoveManagedIndexMetaDataFrom = listOf(Index(managedIndexMetaData.index, managedIndexMetaData.indexUuid)) ) updateMetaDataRetryPolicy.retry(logger) { val response: AcknowledgedResponse = client.suspendUntil { execute(UpdateManagedIndexMetaDataAction.INSTANCE, request, it) } if (response.isAcknowledged) { result = true } else { - logger.error("Failed to save ManagedIndexMetaData for [index=${managedIndexMetaData.index}]") + logger.error("Failed to delete ManagedIndexMetaData for [index=${managedIndexMetaData.index}]") } } } catch (e: ClusterBlockException) { - logger.error("There was ClusterBlockException trying to update the metadata for ${managedIndexMetaData.index}. Message: ${e.message}", e) + logger.error("There was ClusterBlockException trying to delete the metadata for ${managedIndexMetaData.index}. Message: ${e.message}", e) + } catch (e: Exception) { + logger.error("Failed to delete ManagedIndexMetaData for [index=${managedIndexMetaData.index}]", e) + } + return result + } + + /** + * update metadata in config index, and save metadata in history after update + * this can be called 2 times in one job run, so need to save seqNo & primeTerm + */ + private suspend fun updateManagedIndexMetaData( + managedIndexMetaData: ManagedIndexMetaData, + lastUpdateResult: UpdateMetadataResult? = null + ): UpdateMetadataResult { + var result = UpdateMetadataResult() + + var metadata: ManagedIndexMetaData = managedIndexMetaData + if (lastUpdateResult != null) { + metadata = managedIndexMetaData.copy(seqNo = lastUpdateResult.seqNo, primaryTerm = lastUpdateResult.primaryTerm) + } + + val indexRequest = managedIndexMetadataIndexRequest(metadata) + try { + updateMetaDataRetryPolicy.retry(logger) { + val indexResponse: IndexResponse = client.suspendUntil { index(indexRequest, it) } + val metadataSaved = indexResponse.status() == RestStatus.OK || + indexResponse.status() == RestStatus.CREATED + result = UpdateMetadataResult(metadataSaved, indexResponse.seqNo, indexResponse.primaryTerm) + } + + GlobalScope.launch(Dispatchers.IO + CoroutineName("ManagedIndexMetaData-AddHistory")) { + ismHistory.addManagedIndexMetaDataHistory(listOf(metadata)) + } + } catch (e: VersionConflictEngineException) { + logger.error("There was VersionConflictEngineException trying to update the metadata for " + + "${managedIndexMetaData.index}. Message: ${e.message}", e) } catch (e: Exception) { logger.error("Failed to save ManagedIndexMetaData for [index=${managedIndexMetaData.index}]", e) } + return result } + data class UpdateMetadataResult( + val metadataSaved: Boolean = false, + val seqNo: Long = SequenceNumbers.UNASSIGNED_SEQ_NO, + val primaryTerm: Long = SequenceNumbers.UNASSIGNED_PRIMARY_TERM + ) + /** * Initializes the change policy process where we will get the policy using the change policy's policyID, update the [ManagedIndexMetaData] * to reflect the new policy, and save the new policy to the [ManagedIndexConfig] while resetting the change policy to null @@ -596,7 +676,7 @@ object ManagedIndexRunner : ScheduledJobRunner, * */ val updated = updateManagedIndexMetaData(updatedManagedIndexMetaData) - if (!updated || policy == null) return + if (!updated.metadataSaved || policy == null) return // this will save the new policy on the job and reset the change policy back to null val saved = savePolicyToManagedIndexConfig(managedIndexConfig, policy) @@ -674,7 +754,7 @@ object ManagedIndexRunner : ScheduledJobRunner, private fun clusterIsRed(): Boolean = ClusterStateHealth(clusterService.state()).status == ClusterHealthStatus.RED - private suspend fun getIndexMetaData(index: String): IndexMetadata? { + private suspend fun getIndexMetadata(index: String): IndexMetadata? { var indexMetaData: IndexMetadata? = null try { val clusterStateRequest = ClusterStateRequest() diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/MetadataService.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/MetadataService.kt new file mode 100644 index 000000000..c28bb9b76 --- /dev/null +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/MetadataService.kt @@ -0,0 +1,207 @@ +/* + * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement + +import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementIndices +import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.retry +import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.suspendUntil +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.getManagedIndexMetadata +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataAction +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataRequest +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.managedIndexMetadataIndexRequest +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.revertManagedIndexMetadataID +import com.amazon.opendistroforelasticsearch.indexmanagement.util.OpenForTesting +import org.apache.logging.log4j.LogManager +import org.elasticsearch.ExceptionsHelper +import org.elasticsearch.action.DocWriteRequest +import org.elasticsearch.action.bulk.BackoffPolicy +import org.elasticsearch.action.bulk.BulkItemResponse +import org.elasticsearch.action.bulk.BulkRequest +import org.elasticsearch.action.bulk.BulkResponse +import org.elasticsearch.action.support.master.AcknowledgedResponse +import org.elasticsearch.client.Client +import org.elasticsearch.cluster.service.ClusterService +import org.elasticsearch.common.unit.TimeValue +import org.elasticsearch.index.Index +import org.elasticsearch.rest.RestStatus +import java.lang.Exception + +/** + * When all nodes have same version IM plugin (CDI/DDI finished) + * MetadataService starts to move metadata from cluster state to config index + */ +@OpenForTesting +class MetadataService( + private val client: Client, + private val clusterService: ClusterService, + private val skipExecution: SkipExecution, + private val imIndices: IndexManagementIndices +) { + private val logger = LogManager.getLogger(javaClass) + + @Volatile private var runningLock = false // in case 2 moveMetadata() process running + + private val successfullyIndexedIndices = mutableSetOf() + private var failedToIndexIndices = mutableMapOf() + private var failedToCleanIndices = mutableSetOf() + + private var counter = 0 + + // used in coordinator sweep to cancel scheduled process + @Volatile final var finishFlag = false + private set + fun reenableMetadataService() { finishFlag = false } + + @Suppress("MagicNumber") + @Volatile private var retryPolicy = + BackoffPolicy.constantBackoff(TimeValue.timeValueMillis(50), 3) + + @Suppress("ReturnCount", "LongMethod", "ComplexMethod") + suspend fun moveMetadata() { + if (runningLock) { + logger.info("There is a move metadata process running...") + return + } else if (finishFlag) { + logger.info("Move metadata has finished.") + return + } + try { + runningLock = true + + if (skipExecution.flag) { + logger.info("Cluster still has nodes running old version of ISM plugin, skip ping execution on new nodes until all nodes upgraded") + runningLock = false + return + } + + val indicesMetadata = clusterService.state().metadata.indices + var clusterStateManagedIndexMetadata = indicesMetadata.map { + it.key to it.value.getManagedIndexMetadata() + }.filter { it.second != null }.distinct().toMap() + // filter out previous failedToClean indices which already been indexed + clusterStateManagedIndexMetadata = + clusterStateManagedIndexMetadata.filter { it.key !in failedToCleanIndices.map { index -> index.name } } + val indexUuidMap = clusterStateManagedIndexMetadata.map { indicesMetadata[it.key].indexUUID to it.key }.toMap() + + if (clusterStateManagedIndexMetadata.isEmpty()) { + if (failedToCleanIndices.isNotEmpty()) { + logger.info("Failed to clean indices: $failedToCleanIndices. Only clean cluster state metadata in this run.") + cleanMetadatas(failedToCleanIndices.toList()) + finishFlag = false; runningLock = false + return + } + if (counter++ > 2) { + logger.info("Move Metadata succeed, set finish flag to true. Indices failed to get indexed: $failedToIndexIndices") + finishFlag = true; runningLock = false + return + } + } else { + counter = 0; finishFlag = false // index metadata for indices which metadata hasn't been indexed + val bulkIndexReq = + clusterStateManagedIndexMetadata.mapNotNull { it.value }.map { + managedIndexMetadataIndexRequest( + it, + waitRefresh = false, // should be set at bulk request level + create = true // restrict this as create operation + ) + } + // remove the part which gonna be indexed from last time failedToIndex + failedToIndexIndices = failedToIndexIndices.filterKeys { it !in indexUuidMap.keys }.toMutableMap() + successfullyIndexedIndices.clear() + indexMetadatas(bulkIndexReq) + + logger.debug("success indexed: ${successfullyIndexedIndices.map { indexUuidMap[it] }}") + logger.debug( + "failed indexed: ${failedToIndexIndices.map { indexUuidMap[it.key] }};" + + "failed reason: ${failedToIndexIndices.values.distinct()}" + ) + } + + // clean metadata for indices which metadata already been indexed + val indicesToCleanMetadata = + indexUuidMap.filter { it.key in successfullyIndexedIndices }.map { Index(it.value, it.key) } + .toList() + failedToCleanIndices + + cleanMetadatas(indicesToCleanMetadata) + logger.debug("Failed to clean cluster metadata for: ${failedToCleanIndices.map { it.name }}") + } finally { + runningLock = false + } + } + + private suspend fun indexMetadatas(requests: List>) { + if (requests.isEmpty()) return + var requestsToRetry = requests + + // when we try to index sth to config index + // we need to make sure the schema is up to date + if (!imIndices.attemptUpdateConfigIndexMapping()) { + logger.error("Failed to update config index mapping.") + return + } + + retryPolicy.retry(logger, listOf(RestStatus.TOO_MANY_REQUESTS)) { + val bulkRequest = BulkRequest().add(requestsToRetry) + val bulkResponse: BulkResponse = client.suspendUntil { bulk(bulkRequest, it) } + val failedResponses = (bulkResponse.items ?: arrayOf()).filter { it.isFailed } + + val retryIndexUuids = mutableListOf() + bulkResponse.items.forEach { + val indexUuid = revertManagedIndexMetadataID(it.id) + if (it.isFailed) { + if (it.status() == RestStatus.TOO_MANY_REQUESTS) { + retryIndexUuids.add(it.itemId) + } else { + logger.error("failed reason: ${it.failure}, ${it.failureMessage}") + failedToIndexIndices[indexUuid] = it.failure + } + } else { + successfullyIndexedIndices.add(indexUuid) + failedToIndexIndices.remove(indexUuid) + } + } + requestsToRetry = retryIndexUuids.map { bulkRequest.requests()[it] } + + if (requestsToRetry.isNotEmpty()) { + val retryCause = failedResponses.first { it.status() == RestStatus.TOO_MANY_REQUESTS }.failure.cause + throw ExceptionsHelper.convertToElastic(retryCause) + } + } + } + + private suspend fun cleanMetadatas(indices: List) { + if (indices.isEmpty()) return + + val request = UpdateManagedIndexMetaDataRequest(indicesToRemoveManagedIndexMetaDataFrom = indices) + try { + retryPolicy.retry(logger) { + val response: AcknowledgedResponse = + client.suspendUntil { execute(UpdateManagedIndexMetaDataAction.INSTANCE, request, it) } + if (response.isAcknowledged) { + failedToCleanIndices.removeAll(indices) + } else { + logger.error("Failed to clean cluster state metadata for indices: [$indices].") + failedToCleanIndices.addAll(indices) + } + } + } catch (e: Exception) { + logger.error("Failed to clean cluster state metadata for indices: [$indices].", e) + failedToCleanIndices.addAll(indices) + } + } +} + +typealias metadataDocID = String diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/SkipExecution.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/SkipExecution.kt new file mode 100644 index 000000000..a984c4998 --- /dev/null +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/SkipExecution.kt @@ -0,0 +1,79 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement + +import com.amazon.opendistroforelasticsearch.indexmanagement.util.OpenForTesting +import org.apache.logging.log4j.LogManager +import org.elasticsearch.action.ActionListener +import org.elasticsearch.action.admin.cluster.node.info.NodesInfoAction +import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest +import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse +import org.elasticsearch.action.admin.cluster.node.info.PluginsAndModules +import org.elasticsearch.client.Client +import org.elasticsearch.cluster.ClusterChangedEvent +import org.elasticsearch.cluster.ClusterStateListener +import org.elasticsearch.cluster.service.ClusterService + +// TODO this can be moved to job scheduler, so that all extended plugin +// can avoid running jobs in an upgrading cluster +@OpenForTesting +class SkipExecution( + private val client: Client, + private val clusterService: ClusterService +) : ClusterStateListener { + private val logger = LogManager.getLogger(javaClass) + + @Volatile final var flag: Boolean = false + private set + + init { + clusterService.addListener(this) + } + + override fun clusterChanged(event: ClusterChangedEvent) { + if (event.nodesChanged() || event.isNewCluster) { + sweepISMPluginVersion() + } + } + + fun sweepISMPluginVersion() { + // if old version ISM plugin exists (2 versions ISM in one cluster), set skip flag to true + val request = NodesInfoRequest().clear().addMetric("plugins") + client.execute(NodesInfoAction.INSTANCE, request, object : ActionListener { + override fun onResponse(response: NodesInfoResponse) { + val versionSet = mutableSetOf() + + response.nodes.map { it.getInfo(PluginsAndModules::class.java).pluginInfos } + .forEach { it.forEach { nodePlugin -> + if (nodePlugin.name == "opendistro-index-management" || + nodePlugin.name == "opendistro_index_management") { + versionSet.add(nodePlugin.version) + } + } } + + if (versionSet.size > 1) { + flag = true + logger.info("There are multiple versions of Index Management plugins in the cluster: $versionSet") + } else flag = false + } + + override fun onFailure(e: Exception) { + logger.error("Failed sweeping nodes for ISM plugin versions: $e") + flag = false + } + }) + } +} diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/elasticapi/ElasticExtensions.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/elasticapi/ElasticExtensions.kt index 277d7aaea..09f27a7b3 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/elasticapi/ElasticExtensions.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/elasticapi/ElasticExtensions.kt @@ -17,18 +17,40 @@ package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi +import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementPlugin.Companion.INDEX_MANAGEMENT_INDEX +import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.contentParser import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.parseWithType +import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.suspendUntil import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ISMTemplate import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.Policy import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.managedIndexMetadataID +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.withContext +import org.apache.logging.log4j.LogManager +import org.elasticsearch.action.ActionRequestValidationException +import org.elasticsearch.action.NoShardAvailableActionException +import org.elasticsearch.action.get.GetRequest +import org.elasticsearch.action.get.GetResponse +import org.elasticsearch.action.get.MultiGetRequest +import org.elasticsearch.action.get.MultiGetResponse import org.elasticsearch.cluster.ClusterState import org.elasticsearch.action.search.SearchResponse +import org.elasticsearch.client.Client import org.elasticsearch.cluster.metadata.IndexMetadata import org.elasticsearch.common.xcontent.LoggingDeprecationHandler import org.elasticsearch.common.xcontent.NamedXContentRegistry +import org.elasticsearch.common.xcontent.ToXContent +import org.elasticsearch.common.xcontent.ToXContentFragment +import org.elasticsearch.common.xcontent.XContentBuilder import org.elasticsearch.common.xcontent.XContentFactory +import org.elasticsearch.common.xcontent.XContentHelper import org.elasticsearch.common.xcontent.XContentType +import org.elasticsearch.index.Index +import org.elasticsearch.index.IndexNotFoundException + +private val log = LogManager.getLogger("Index Management Helper") /** * Returns the current rollover_alias if it exists otherwise returns null. @@ -39,8 +61,8 @@ fun IndexMetadata.getRolloverAlias(): String? { return this.settings.get(ManagedIndexSettings.ROLLOVER_ALIAS.key) } -fun IndexMetadata.getManagedIndexMetaData(): ManagedIndexMetaData? { - val existingMetaDataMap = this.getCustomData(ManagedIndexMetaData.MANAGED_INDEX_METADATA) +fun IndexMetadata.getManagedIndexMetadata(): ManagedIndexMetaData? { + val existingMetaDataMap = this.getCustomData(ManagedIndexMetaData.MANAGED_INDEX_METADATA_TYPE) if (existingMetaDataMap != null) { return ManagedIndexMetaData.fromMap(existingMetaDataMap) @@ -84,3 +106,110 @@ fun getPolicyToTemplateMap(response: SearchResponse, xContentRegistry: NamedXCon @Suppress("UNCHECKED_CAST") fun Map.filterNotNullValues(): Map = filterValues { it != null } as Map + +// get metadata from config index using doc id +@Suppress("ReturnCount") +suspend fun IndexMetadata.getManagedIndexMetadata(client: Client): ManagedIndexMetaData? { + try { + val getRequest = GetRequest(INDEX_MANAGEMENT_INDEX, managedIndexMetadataID(indexUUID)) + .routing(this.indexUUID) + val getResponse: GetResponse = client.suspendUntil { get(getRequest, it) } + if (!getResponse.isExists || getResponse.isSourceEmpty) { + return null + } + + return withContext(Dispatchers.IO) { + val xcp = XContentHelper.createParser( + NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, + getResponse.sourceAsBytesRef, XContentType.JSON) + ManagedIndexMetaData.parseWithType(xcp, getResponse.id, getResponse.seqNo, getResponse.primaryTerm) + } + } catch (e: Exception) { + when (e) { + is IndexNotFoundException, is NoShardAvailableActionException -> { + log.error("Failed to get metadata because no index or shard not available") + } + else -> log.error("Failed to get metadata", e) + } + + return null + } +} + +/** + * multi-get metadata for indices + * + * @return list of metadata + */ +suspend fun Client.mgetManagedIndexMetadata(indices: List): List?> { + log.debug("trying to get back metadata for indices ${indices.map { it.name }}") + + if (indices.isEmpty()) return emptyList() + + val mgetRequest = MultiGetRequest() + indices.forEach { + mgetRequest.add(MultiGetRequest.Item( + INDEX_MANAGEMENT_INDEX, managedIndexMetadataID(it.uuid)).routing(it.uuid)) + } + var mgetMetadataList = listOf?>() + try { + val response: MultiGetResponse = this.suspendUntil { multiGet(mgetRequest, it) } + mgetMetadataList = mgetResponseToList(response) + } catch (e: ActionRequestValidationException) { + log.info("No managed index metadata for indices [$indices], ${e.message}") + } catch (e: Exception) { + log.error("Failed to multi-get managed index metadata for indices [$indices]", e) + } + return mgetMetadataList +} + +/** + * transform multi-get response to list for ManagedIndexMetaData + * + * when this function used in change and retry API, if exception is + * not null, the API will abort and show get metadata failed + * + * @return list of Pair of metadata or exception + */ +fun mgetResponseToList(mgetResponse: MultiGetResponse): + List?> { + val mgetList = mutableListOf?>() + mgetResponse.responses.forEach { + if (it.isFailed) { + mgetList.add(Pair(null, it.failure.failure)) + } else if (it.response != null && !it.response.isSourceEmpty) { + val xcp = contentParser(it.response.sourceAsBytesRef) + mgetList.add(Pair(ManagedIndexMetaData.parseWithType( + xcp, it.response.id, it.response.seqNo, it.response.primaryTerm), null)) + } else { + mgetList.add(null) + } + } + + return mgetList +} + +fun buildMgetMetadataRequest(clusterState: ClusterState): MultiGetRequest { + val mgetMetadataRequest = MultiGetRequest() + clusterState.metadata.indices.map { it.value.index }.forEach { + mgetMetadataRequest.add(MultiGetRequest.Item( + INDEX_MANAGEMENT_INDEX, managedIndexMetadataID(it.uuid)).routing(it.uuid)) + } + return mgetMetadataRequest +} + +// forIndex means saving to config index, distinguish from Explain and History, +// which only show meaningful partial metadata +@Suppress("ReturnCount") +fun XContentBuilder.addObject(name: String, metadata: ToXContentFragment?, params: ToXContent.Params, forIndex: Boolean = false): XContentBuilder { + if (metadata != null) return this.buildMetadata(name, metadata, params) + return if (forIndex) nullField(name) else this +} + +fun XContentBuilder.buildMetadata(name: String, metadata: ToXContentFragment, params: ToXContent.Params): XContentBuilder { + this.startObject(name) + metadata.toXContent(this, params) + this.endObject() + return this +} diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/ManagedIndexMetaData.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/ManagedIndexMetaData.kt index 98ff92994..7c5813b29 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/ManagedIndexMetaData.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/ManagedIndexMetaData.kt @@ -15,6 +15,7 @@ package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.addObject import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.action.ActionConfig import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.ActionMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.PolicyRetryInfoMetaData @@ -33,6 +34,8 @@ import org.elasticsearch.common.xcontent.XContentParser import org.elasticsearch.common.xcontent.XContentParser.Token import org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken import org.elasticsearch.common.xcontent.json.JsonXContent +import org.elasticsearch.index.seqno.SequenceNumbers +import java.io.IOException data class ManagedIndexMetaData( val index: String, @@ -47,7 +50,10 @@ data class ManagedIndexMetaData( val actionMetaData: ActionMetaData?, val stepMetaData: StepMetaData?, val policyRetryInfo: PolicyRetryInfoMetaData?, - val info: Map? + val info: Map?, + val id: String = NO_ID, + val seqNo: Long = SequenceNumbers.UNASSIGNED_SEQ_NO, + val primaryTerm: Long = SequenceNumbers.UNASSIGNED_PRIMARY_TERM ) : Writeable, ToXContentFragment { @Suppress("ComplexMethod") @@ -70,6 +76,31 @@ data class ManagedIndexMetaData( return resultMap } + fun toXContent(builder: XContentBuilder, params: ToXContent.Params, forIndex: Boolean): XContentBuilder { + // forIndex means saving to config index, distinguish from Explain and History, which only save meaningful partial metadata + if (!forIndex) return toXContent(builder, params) + + builder + .startObject() + .startObject(MANAGED_INDEX_METADATA_TYPE) + .field(INDEX, index) + .field(INDEX_UUID, indexUuid) + .field(POLICY_ID, policyID) + .field(POLICY_SEQ_NO, policySeqNo) + .field(POLICY_PRIMARY_TERM, policyPrimaryTerm) + .field(POLICY_COMPLETED, policyCompleted) + .field(ROLLED_OVER, rolledOver) + .field(TRANSITION_TO, transitionTo) + .addObject(StateMetaData.STATE, stateMetaData, params, true) + .addObject(ActionMetaData.ACTION, actionMetaData, params, true) + .addObject(StepMetaData.STEP, stepMetaData, params, true) + .addObject(PolicyRetryInfoMetaData.RETRY_INFO, policyRetryInfo, params, true) + .field(INFO, info) + .endObject() + .endObject() + return builder + } + @Suppress("ComplexMethod") override fun toXContent(builder: XContentBuilder, params: ToXContent.Params): XContentBuilder { // The order we check values matters here as we are only trying to show what is needed for the customer @@ -78,7 +109,6 @@ data class ManagedIndexMetaData( .field(INDEX, index) .field(INDEX_UUID, indexUuid) .field(POLICY_ID, policyID) - if (policySeqNo != null) builder.field(POLICY_SEQ_NO, policySeqNo) if (policyPrimaryTerm != null) builder.field(POLICY_PRIMARY_TERM, policyPrimaryTerm) @@ -93,28 +123,14 @@ data class ManagedIndexMetaData( } val transitionToExists = transitionTo != null - if (transitionToExists) { builder.field(TRANSITION_TO, transitionTo) + } else { + builder.addObject(StateMetaData.STATE, stateMetaData, params) + .addObject(ActionMetaData.ACTION, actionMetaData, params) + .addObject(StepMetaData.STEP, stepMetaData, params) } - - if (stateMetaData != null && !transitionToExists) { - builder.startObject(StateMetaData.STATE) - stateMetaData.toXContent(builder, params) - builder.endObject() - } - - if (actionMetaData != null && !transitionToExists) { - builder.startObject(ActionMetaData.ACTION) - actionMetaData.toXContent(builder, params) - builder.endObject() - } - - if (policyRetryInfo != null) { - builder.startObject(PolicyRetryInfoMetaData.RETRY_INFO) - policyRetryInfo.toXContent(builder, params) - builder.endObject() - } + builder.addObject(PolicyRetryInfoMetaData.RETRY_INFO, policyRetryInfo, params) if (info != null) builder.field(INFO, info) @@ -145,7 +161,8 @@ data class ManagedIndexMetaData( } companion object { - const val MANAGED_INDEX_METADATA = "managed_index_metadata" + const val NO_ID = "" + const val MANAGED_INDEX_METADATA_TYPE = "managed_index_metadata" const val NAME = "name" const val START_TIME = "start_time" @@ -198,8 +215,16 @@ data class ManagedIndexMetaData( ) } - @Suppress("ComplexMethod") - fun parse(xcp: XContentParser): ManagedIndexMetaData { + @Suppress("ComplexMethod", "LongMethod") + @JvmStatic + @JvmOverloads + @Throws(IOException::class) + fun parse( + xcp: XContentParser, + id: String = NO_ID, + seqNo: Long = SequenceNumbers.UNASSIGNED_SEQ_NO, + primaryTerm: Long = SequenceNumbers.UNASSIGNED_PRIMARY_TERM + ): ManagedIndexMetaData { var index: String? = null var indexUuid: String? = null var policyID: String? = null @@ -229,12 +254,22 @@ data class ManagedIndexMetaData( POLICY_PRIMARY_TERM -> policyPrimaryTerm = if (xcp.currentToken() == Token.VALUE_NULL) null else xcp.longValue() POLICY_COMPLETED -> policyCompleted = if (xcp.currentToken() == Token.VALUE_NULL) null else xcp.booleanValue() ROLLED_OVER -> rolledOver = if (xcp.currentToken() == Token.VALUE_NULL) null else xcp.booleanValue() - TRANSITION_TO -> transitionTo = xcp.text() - StateMetaData.STATE -> state = StateMetaData.parse(xcp) - ActionMetaData.ACTION -> action = ActionMetaData.parse(xcp) - StepMetaData.STEP -> step = StepMetaData.parse(xcp) - PolicyRetryInfoMetaData.RETRY_INFO -> retryInfo = PolicyRetryInfoMetaData.parse(xcp) + TRANSITION_TO -> transitionTo = if (xcp.currentToken() == Token.VALUE_NULL) null else xcp.text() + StateMetaData.STATE -> { + state = if (xcp.currentToken() == Token.VALUE_NULL) null else StateMetaData.parse(xcp) + } + ActionMetaData.ACTION -> { + action = if (xcp.currentToken() == Token.VALUE_NULL) null else ActionMetaData.parse(xcp) + } + StepMetaData.STEP -> { + step = if (xcp.currentToken() == Token.VALUE_NULL) null else StepMetaData.parse(xcp) + } + PolicyRetryInfoMetaData.RETRY_INFO -> { + retryInfo = PolicyRetryInfoMetaData.parse(xcp) + } INFO -> info = xcp.map() + // below line will break when getting metadata for explain or history + // else -> throw IllegalArgumentException("Invalid field: [$fieldName] found in ManagedIndexMetaData.") } } @@ -251,10 +286,30 @@ data class ManagedIndexMetaData( action, step, retryInfo, - info + info, + id, + seqNo, + primaryTerm ) } + @JvmStatic + @JvmOverloads + @Throws(IOException::class) + fun parseWithType( + xcp: XContentParser, + id: String = NO_ID, + seqNo: Long = SequenceNumbers.UNASSIGNED_SEQ_NO, + primaryTerm: Long = SequenceNumbers.UNASSIGNED_PRIMARY_TERM + ): ManagedIndexMetaData { + ensureExpectedToken(Token.START_OBJECT, xcp.nextToken(), xcp) + ensureExpectedToken(Token.FIELD_NAME, xcp.nextToken(), xcp) + ensureExpectedToken(Token.START_OBJECT, xcp.nextToken(), xcp) + val managedIndexMetaData = parse(xcp, id, seqNo, primaryTerm) + ensureExpectedToken(Token.END_OBJECT, xcp.nextToken(), xcp) + return managedIndexMetaData + } + fun fromMap(map: Map): ManagedIndexMetaData { return ManagedIndexMetaData( index = requireNotNull(map[INDEX]) { "$INDEX is null" }, diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/managedindexmetadata/ActionMetaData.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/managedindexmetadata/ActionMetaData.kt index 5a1086928..56e959e12 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/managedindexmetadata/ActionMetaData.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/managedindexmetadata/ActionMetaData.kt @@ -105,9 +105,9 @@ data class ActionMetaData( } fun fromManagedIndexMetaDataMap(map: Map): ActionMetaData? { - val stateJsonString = map[ACTION] - return if (stateJsonString != null) { - val inputStream = ByteArrayInputStream(stateJsonString.toByteArray(StandardCharsets.UTF_8)) + val actionJsonString = map[ACTION] + return if (actionJsonString != null) { + val inputStream = ByteArrayInputStream(actionJsonString.toByteArray(StandardCharsets.UTF_8)) val parser = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, inputStream) parser.nextToken() parse(parser) diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/managedindexmetadata/StepMetaData.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/managedindexmetadata/StepMetaData.kt index cdcd72e79..10f5ddb38 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/managedindexmetadata/StepMetaData.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/model/managedindexmetadata/StepMetaData.kt @@ -48,11 +48,12 @@ data class StepMetaData( } override fun toXContent(builder: XContentBuilder, params: ToXContent.Params): XContentBuilder { - return builder.startObject(STEP) + builder .field(NAME, name) .field(START_TIME, startTime) .field(STEP_STATUS, stepStatus.toString()) - .endObject() + + return builder } fun getMapValueString(): String { @@ -76,9 +77,9 @@ data class StepMetaData( } fun fromManagedIndexMetaDataMap(map: Map): StepMetaData? { - val stateJsonString = map[STEP] - return if (stateJsonString != null) { - val inputStream = ByteArrayInputStream(stateJsonString.toByteArray(StandardCharsets.UTF_8)) + val stepJsonString = map[STEP] + return if (stepJsonString != null) { + val inputStream = ByteArrayInputStream(stepJsonString.toByteArray(StandardCharsets.UTF_8)) val parser = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, inputStream) parser.nextToken() parse(parser) diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestExplainAction.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestExplainAction.kt index fd0fbd46f..78a6b1581 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestExplainAction.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestExplainAction.kt @@ -54,7 +54,6 @@ class RestExplainAction : BaseRestHandler() { return "ism_explain_action" } - @Suppress("SpreadOperator") // There is no way around dealing with java vararg without spread operator. override fun prepareRequest(request: RestRequest, client: NodeClient): RestChannelConsumer { log.debug("${request.method()} ${request.path()}") diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/settings/ManagedIndexSettings.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/settings/ManagedIndexSettings.kt index 1253ae9d2..b49ddddb7 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/settings/ManagedIndexSettings.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/settings/ManagedIndexSettings.kt @@ -24,6 +24,7 @@ import java.util.function.Function class ManagedIndexSettings { companion object { const val DEFAULT_ISM_ENABLED = true + const val DEFAULT_METADATA_SERVICE_ENABLED = true const val DEFAULT_JOB_INTERVAL = 5 private val ALLOW_LIST_ALL = ActionConfig.ActionType.values().toList().map { it.type } val ALLOW_LIST_NONE = emptyList() @@ -37,6 +38,13 @@ class ManagedIndexSettings { Setting.Property.Dynamic ) + val METADATA_SERVICE_ENABLED: Setting = Setting.boolSetting( + "opendistro.index_state_management.metadata_service.enabled", + DEFAULT_METADATA_SERVICE_ENABLED, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ) + val POLICY_ID: Setting = Setting.simpleString( "index.opendistro.index_state_management.policy_id", Setting.Property.IndexScope, diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/addpolicy/TransportAddPolicyAction.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/addpolicy/TransportAddPolicyAction.kt index 6791ad6b7..d4eb420f8 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/addpolicy/TransportAddPolicyAction.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/addpolicy/TransportAddPolicyAction.kt @@ -197,7 +197,8 @@ class TransportAddPolicyAction @Inject constructor( response.forEach { val docId = it.id // docId is managed index uuid if (it.isFailed) { - failedIndices.add(FailedIndex(indicesToAdd[docId] as String, docId, "Failed to add policy due to: ${it.failureMessage}")) + failedIndices.add(FailedIndex(indicesToAdd[docId] as String, docId, + "Failed to add policy due to: ${it.failureMessage}")) indicesToAdd.remove(docId) } } diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/changepolicy/TransportChangePolicyAction.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/changepolicy/TransportChangePolicyAction.kt index 1f845a018..e9ba80d0d 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/changepolicy/TransportChangePolicyAction.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/changepolicy/TransportChangePolicyAction.kt @@ -16,9 +16,13 @@ package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.changepolicy import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementPlugin +import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.contentParser import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.parseWithType -import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.getManagedIndexMetaData +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.buildMgetMetadataRequest +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.getManagedIndexMetadata +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.mgetResponseToList import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexConfig +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.Policy import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.coordinator.SweptManagedIndexConfig import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.resthandler.RestChangePolicyAction @@ -45,14 +49,14 @@ import org.elasticsearch.action.support.HandledTransportAction import org.elasticsearch.action.support.IndicesOptions import org.elasticsearch.action.support.master.AcknowledgedResponse import org.elasticsearch.client.node.NodeClient +import org.elasticsearch.cluster.ClusterState import org.elasticsearch.cluster.service.ClusterService -import org.elasticsearch.common.bytes.BytesReference import org.elasticsearch.common.inject.Inject import org.elasticsearch.common.xcontent.LoggingDeprecationHandler import org.elasticsearch.common.xcontent.NamedXContentRegistry import org.elasticsearch.common.xcontent.XContentHelper -import org.elasticsearch.common.xcontent.XContentParser import org.elasticsearch.common.xcontent.XContentType +import org.elasticsearch.index.Index import org.elasticsearch.rest.RestStatus import org.elasticsearch.search.fetch.subphase.FetchSourceContext import org.elasticsearch.tasks.Task @@ -79,33 +83,34 @@ class TransportChangePolicyAction @Inject constructor( private val actionListener: ActionListener, private val request: ChangePolicyRequest ) { + private val failedIndices = mutableListOf() - private val managedIndexUuids = mutableListOf>() + private val managedIndicesToUpdate = mutableListOf>() private val indexUuidToCurrentState = mutableMapOf() - lateinit var policy: Policy - lateinit var response: GetResponse + private val changePolicy = request.changePolicy + private lateinit var policy: Policy + private lateinit var getPolicyResponse: GetResponse + private lateinit var clusterState: ClusterState private var updated: Int = 0 fun start() { - val getRequest = GetRequest(IndexManagementPlugin.INDEX_MANAGEMENT_INDEX, request.changePolicy.policyID) + val getRequest = GetRequest(IndexManagementPlugin.INDEX_MANAGEMENT_INDEX, changePolicy.policyID) client.get(getRequest, ActionListener.wrap(::onGetPolicyResponse, ::onFailure)) } - private fun onFailure(t: Exception) { - actionListener.onFailure(ExceptionsHelper.unwrapCause(t) as Exception) - } - private fun onGetPolicyResponse(response: GetResponse) { if (!response.isExists || response.isSourceEmpty) { actionListener.onFailure(ElasticsearchStatusException("Could not find policy=${request.changePolicy.policyID}", RestStatus.NOT_FOUND)) return } - this.response = response + this.getPolicyResponse = response + IndexUtils.checkAndUpdateConfigIndexMapping( clusterService.state(), client.admin().indices(), - ActionListener.wrap(::onUpdateMapping, ::onFailure)) + ActionListener.wrap(::onUpdateMapping, ::onFailure) + ) } private fun onUpdateMapping(acknowledgedResponse: AcknowledgedResponse) { @@ -115,12 +120,13 @@ class TransportChangePolicyAction @Inject constructor( RestStatus.FAILED_DEPENDENCY)) return } + policy = XContentHelper.createParser( xContentRegistry, LoggingDeprecationHandler.INSTANCE, - response.sourceAsBytesRef, + getPolicyResponse.sourceAsBytesRef, XContentType.JSON - ).use { it.parseWithType(response.id, response.seqNo, response.primaryTerm, Policy.Companion::parse) } + ).use { it.parseWithType(getPolicyResponse.id, getPolicyResponse.seqNo, getPolicyResponse.primaryTerm, Policy.Companion::parse) } getClusterState() } @@ -134,45 +140,72 @@ class TransportChangePolicyAction @Inject constructor( .local(false) .indicesOptions(IndicesOptions.strictExpand()) - client.admin().cluster().state(clusterStateRequest, ActionListener.wrap(::processResponse, ::onFailure)) + client.admin().cluster().state(clusterStateRequest, ActionListener.wrap(::onClusterStateResponse, ::onFailure)) + } + + @Suppress("ComplexMethod") + private fun onClusterStateResponse(response: ClusterStateResponse) { + clusterState = response.state + + // get back managed index metadata + client.multiGet(buildMgetMetadataRequest(clusterState), ActionListener.wrap(::onMgetMetadataResponse, ::onFailure)) } @Suppress("ComplexMethod") - private fun processResponse(response: ClusterStateResponse) { - val includedStates = request.changePolicy.include.map { it.state }.toSet() + private fun onMgetMetadataResponse(mgetResponse: MultiGetResponse) { + val metadataList = mgetResponseToList(mgetResponse) + val includedStates = changePolicy.include.map { it.state }.toSet() - response.state.metadata.indices.forEach { + clusterState.metadata.indices.forEachIndexed { ind, it -> val indexMetaData = it.value - val currentState = indexMetaData.getManagedIndexMetaData()?.stateMetaData?.name + val clusterStateMetadata = it.value.getManagedIndexMetadata() + val mgetFailure = metadataList[ind]?.second + val managedIndexMetadata: ManagedIndexMetaData? = metadataList[ind]?.first + + val currentState = managedIndexMetadata?.stateMetaData?.name if (currentState != null) { indexUuidToCurrentState[indexMetaData.indexUUID] = currentState } + when { + mgetFailure != null -> + failedIndices.add(FailedIndex(indexMetaData.index.name, indexMetaData.index.uuid, + "Failed to get managed index metadata, $mgetFailure" + )) // if there exists a transitionTo on the ManagedIndexMetaData then we will // fail as they might not of meant to add a ChangePolicy when its on the next state - indexMetaData.getManagedIndexMetaData()?.transitionTo != null -> - failedIndices.add(FailedIndex(indexMetaData.index.name, indexMetaData.index.uuid, RestChangePolicyAction.INDEX_IN_TRANSITION)) + managedIndexMetadata?.transitionTo != null -> + failedIndices.add(FailedIndex(indexMetaData.index.name, indexMetaData.index.uuid, + RestChangePolicyAction.INDEX_IN_TRANSITION + )) // else if there is no ManagedIndexMetaData yet then the managed index has not initialized and we can change the policy safely - indexMetaData.getManagedIndexMetaData() == null -> - managedIndexUuids.add(indexMetaData.index.name to indexMetaData.index.uuid) + managedIndexMetadata == null -> { + if (clusterStateMetadata != null) { + failedIndices.add(FailedIndex(indexMetaData.index.name, indexMetaData.index.uuid, + "Cannot change policy until metadata has finished migrating")) + } else { + managedIndicesToUpdate.add(indexMetaData.index.name to indexMetaData.index.uuid) + } + } // else if the includedStates is empty (i.e. not being used) then we will always try to update the managed index - includedStates.isEmpty() -> managedIndexUuids.add(indexMetaData.index.name to indexMetaData.index.uuid) + includedStates.isEmpty() -> managedIndicesToUpdate.add(indexMetaData.index.name to indexMetaData.index.uuid) // else only update the managed index if its currently in one of the included states - includedStates.contains(indexMetaData.getManagedIndexMetaData()?.stateMetaData?.name) -> - managedIndexUuids.add(indexMetaData.index.name to indexMetaData.index.uuid) + includedStates.contains(managedIndexMetadata.stateMetaData?.name) -> + managedIndicesToUpdate.add(indexMetaData.index.name to indexMetaData.index.uuid) // else the managed index did not match any of the included state filters and we will not update it else -> log.debug("Skipping ${indexMetaData.index.name} as it does not match any of the include state filters") } } - if (managedIndexUuids.isEmpty()) { + if (managedIndicesToUpdate.isEmpty()) { updated = 0 actionListener.onResponse(ISMStatusResponse(updated, failedIndices)) return } else { client.multiGet( - getManagedIndexConfigMultiGetRequest(managedIndexUuids.map { (_, indexUuid) -> indexUuid }.toTypedArray()), - ActionListener.wrap(::onMultiGetResponse, ::onFailure)) + mgetManagedIndexConfigRequest(managedIndicesToUpdate.map { (_, indexUuid) -> indexUuid }.toTypedArray()), + ActionListener.wrap(::onMultiGetResponse, ::onFailure) + ) } } @@ -182,7 +215,7 @@ class TransportChangePolicyAction @Inject constructor( // The id is the index uuid if (!it.response.isExists) { // meaning this index is not managed val indexUuid = it.response.id - val indexName = managedIndexUuids.find { (_, second) -> second == indexUuid }?.first + val indexName = managedIndicesToUpdate.find { (_, second) -> second == indexUuid }?.first if (indexName != null) { failedIndices.add(FailedIndex(indexName, indexUuid, RestChangePolicyAction.INDEX_NOT_MANAGED)) } @@ -206,18 +239,17 @@ class TransportChangePolicyAction @Inject constructor( } private fun updateManagedIndexConfig(sweptConfigs: List) { - val mapOfItemIdToIndex = mutableMapOf>() - val bulkRequest = BulkRequest() - sweptConfigs.forEachIndexed { index, sweptConfig -> + val mapOfItemIdToIndex = mutableMapOf() + val bulkUpdateManagedIndexRequest = BulkRequest() + sweptConfigs.forEachIndexed { id, sweptConfig -> // compare the sweptConfig policy to the get policy here and update changePolicy val currentStateName = indexUuidToCurrentState[sweptConfig.uuid] - val updatedChangePolicy = request.changePolicy - .copy(isSafe = sweptConfig.policy?.isSafeToChange(currentStateName, policy, request.changePolicy) == true) - bulkRequest.add(updateManagedIndexRequest(sweptConfig.copy(changePolicy = updatedChangePolicy))) - mapOfItemIdToIndex[index] = sweptConfig.index to sweptConfig.uuid + val updatedChangePolicy = changePolicy + .copy(isSafe = sweptConfig.policy?.isSafeToChange(currentStateName, policy, changePolicy) == true) + bulkUpdateManagedIndexRequest.add(updateManagedIndexRequest(sweptConfig.copy(changePolicy = updatedChangePolicy))) + mapOfItemIdToIndex[id] = Index(sweptConfig.index, sweptConfig.uuid) } - - client.bulk(bulkRequest, object : ActionListener { + client.bulk(bulkUpdateManagedIndexRequest, object : ActionListener { override fun onResponse(response: BulkResponse) { onBulkResponse(response, mapOfItemIdToIndex) } @@ -228,12 +260,12 @@ class TransportChangePolicyAction @Inject constructor( }) } - private fun onBulkResponse(bulkResponse: BulkResponse, mapOfItemIdToIndex: Map>) { + private fun onBulkResponse(bulkResponse: BulkResponse, mapOfItemIdToIndex: Map) { val failedResponses = (bulkResponse.items ?: arrayOf()).filter { it.isFailed } failedResponses.forEach { val indexPair = mapOfItemIdToIndex[it.itemId] if (indexPair != null) { - failedIndices.add(FailedIndex(indexPair.first, indexPair.second, it.failureMessage)) + failedIndices.add(FailedIndex(indexPair.name, indexPair.uuid, it.failureMessage)) } } @@ -241,26 +273,25 @@ class TransportChangePolicyAction @Inject constructor( actionListener.onResponse(ISMStatusResponse(updated, failedIndices)) } - private fun contentParser(bytesReference: BytesReference): XContentParser { - return XContentHelper.createParser(NamedXContentRegistry.EMPTY, - LoggingDeprecationHandler.INSTANCE, bytesReference, XContentType.JSON) - } - @Suppress("SpreadOperator") - private fun getManagedIndexConfigMultiGetRequest(managedIndexUuids: Array): MultiGetRequest { + private fun mgetManagedIndexConfigRequest(managedIndexUuids: Array): MultiGetRequest { val request = MultiGetRequest() val includes = arrayOf( - "${ManagedIndexConfig.MANAGED_INDEX_TYPE}.${ManagedIndexConfig.INDEX_FIELD}", - "${ManagedIndexConfig.MANAGED_INDEX_TYPE}.${ManagedIndexConfig.INDEX_UUID_FIELD}", - "${ManagedIndexConfig.MANAGED_INDEX_TYPE}.${ManagedIndexConfig.POLICY_ID_FIELD}", - "${ManagedIndexConfig.MANAGED_INDEX_TYPE}.${ManagedIndexConfig.POLICY_FIELD}", - "${ManagedIndexConfig.MANAGED_INDEX_TYPE}.${ManagedIndexConfig.CHANGE_POLICY_FIELD}" + "${ManagedIndexConfig.MANAGED_INDEX_TYPE}.${ManagedIndexConfig.INDEX_FIELD}", + "${ManagedIndexConfig.MANAGED_INDEX_TYPE}.${ManagedIndexConfig.INDEX_UUID_FIELD}", + "${ManagedIndexConfig.MANAGED_INDEX_TYPE}.${ManagedIndexConfig.POLICY_ID_FIELD}", + "${ManagedIndexConfig.MANAGED_INDEX_TYPE}.${ManagedIndexConfig.POLICY_FIELD}", + "${ManagedIndexConfig.MANAGED_INDEX_TYPE}.${ManagedIndexConfig.CHANGE_POLICY_FIELD}" ) val excludes = emptyArray() val fetchSourceContext = FetchSourceContext(true, includes, excludes) - managedIndexUuids.forEach { - request.add(MultiGetRequest.Item(IndexManagementPlugin.INDEX_MANAGEMENT_INDEX, it).fetchSourceContext(fetchSourceContext)) } + managedIndexUuids.forEach { request.add(MultiGetRequest.Item( + IndexManagementPlugin.INDEX_MANAGEMENT_INDEX, it).fetchSourceContext(fetchSourceContext).routing(it)) } return request } + + private fun onFailure(t: Exception) { + actionListener.onFailure(ExceptionsHelper.unwrapCause(t) as Exception) + } } } diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/explain/TransportExplainAction.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/explain/TransportExplainAction.kt index 9b6208b51..5a48dc624 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/explain/TransportExplainAction.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/explain/TransportExplainAction.kt @@ -17,20 +17,31 @@ package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanageme import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementPlugin.Companion.INDEX_MANAGEMENT_INDEX import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.ManagedIndexCoordinator.Companion.MAX_HITS +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.getManagedIndexMetadata import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexMetaData +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.isMetadataMoved +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.managedIndexMetadataID import org.apache.logging.log4j.LogManager import org.elasticsearch.ExceptionsHelper import org.elasticsearch.index.IndexNotFoundException import org.elasticsearch.action.ActionListener import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse +import org.elasticsearch.action.get.GetResponse +import org.elasticsearch.action.get.MultiGetRequest +import org.elasticsearch.action.get.MultiGetResponse import org.elasticsearch.action.search.SearchRequest import org.elasticsearch.action.search.SearchResponse import org.elasticsearch.action.support.ActionFilters import org.elasticsearch.action.support.HandledTransportAction import org.elasticsearch.action.support.IndicesOptions import org.elasticsearch.client.node.NodeClient +import org.elasticsearch.cluster.metadata.IndexMetadata import org.elasticsearch.common.inject.Inject +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler +import org.elasticsearch.common.xcontent.NamedXContentRegistry +import org.elasticsearch.common.xcontent.XContentHelper +import org.elasticsearch.common.xcontent.XContentType import org.elasticsearch.index.query.Operator import org.elasticsearch.index.query.QueryBuilders import org.elasticsearch.search.builder.SearchSourceBuilder @@ -45,7 +56,8 @@ private val log = LogManager.getLogger(TransportExplainAction::class.java) class TransportExplainAction @Inject constructor( val client: NodeClient, transportService: TransportService, - actionFilters: ActionFilters + actionFilters: ActionFilters, + val xContentRegistry: NamedXContentRegistry ) : HandledTransportAction( ExplainAction.NAME, transportService, actionFilters, ::ExplainRequest ) { @@ -77,7 +89,7 @@ class TransportExplainAction @Inject constructor( private val rolesMap: MutableMap?> = mutableMapOf() private var totalManagedIndices = 0 - @Suppress("SpreadOperator") + @Suppress("SpreadOperator", "NestedBlockDepth") fun start() { val params = request.searchParams @@ -204,36 +216,63 @@ class TransportExplainAction @Inject constructor( } fun onClusterStateResponse(clusterStateResponse: ClusterStateResponse) { - val state = clusterStateResponse.state - val indexPolicyIDs = mutableListOf() - val indexMetadatas = mutableListOf() + val clusterStateIndexMetadatas = clusterStateResponse.state.metadata.indices.map { it.key to it.value }.toMap() if (wildcard) { indexNames.clear() // clear wildcard (index*) from indexNames - state.metadata.indices.forEach { indexNames.add(it.key) } + clusterStateIndexMetadatas.forEach { indexNames.add(it.key) } } + val indices = clusterStateIndexMetadatas.map { it.key to it.value.indexUUID }.toMap() + val mgetMetadataReq = MultiGetRequest() + indices.map { it.value }.forEach { uuid -> + mgetMetadataReq.add(MultiGetRequest.Item(INDEX_MANAGEMENT_INDEX, managedIndexMetadataID(uuid)).routing(uuid)) + } + client.multiGet(mgetMetadataReq, object : ActionListener { + override fun onResponse(response: MultiGetResponse) { + val metadataMap = response.responses.map { it.id to getMetadata(it.response)?.toMap() }.toMap() + buildResponse(indices, metadataMap, clusterStateIndexMetadatas) + } + + override fun onFailure(t: Exception) { + actionListener.onFailure(ExceptionsHelper.unwrapCause(t) as Exception) + } + }) + } + + // metadataMap: doc id -> metadataMap, doc id for metadata is [managedIndexMetadataID(indexUuid)] + fun buildResponse( + indices: Map, + metadataMap: Map?>, + clusterStateIndexMetadatas: Map + ) { + val indexPolicyIDs = mutableListOf() + val indexMetadatas = mutableListOf() + // cluster state response will not resisting the sort order // so use the order from previous search result saved in indexNames for (indexName in indexNames) { - val indexMetadata = state.metadata.indices[indexName] - var managedIndexMetadataMap = managedIndicesMetaDataMap[indexName] indexPolicyIDs.add(managedIndexMetadataMap?.get("policy_id")) // use policyID from metadata + val clusterStateMetadata = clusterStateIndexMetadatas[indexName]?.getManagedIndexMetadata() var managedIndexMetadata: ManagedIndexMetaData? = null - val clusterStateMetadata = indexMetadata.getCustomData(ManagedIndexMetaData.MANAGED_INDEX_METADATA) + val configIndexMetadataMap = metadataMap[indices[indexName]?.let { managedIndexMetadataID(it) } ] if (managedIndexMetadataMap != null) { - if (clusterStateMetadata != null) { // if has metadata saved, use that - managedIndexMetadataMap = clusterStateMetadata + if (configIndexMetadataMap != null) { // if has metadata saved, use that + managedIndexMetadataMap = configIndexMetadataMap } if (managedIndexMetadataMap.isNotEmpty()) { managedIndexMetadata = ManagedIndexMetaData.fromMap(managedIndexMetadataMap) } + + if (!isMetadataMoved(clusterStateMetadata, configIndexMetadataMap, log)) { + val info = mapOf("message" to "Metadata is pending migration") + managedIndexMetadata = clusterStateMetadata?.copy(info = info) + } } indexMetadatas.add(managedIndexMetadata) } - managedIndicesMetaDataMap.clear() if (explainAll) { @@ -243,7 +282,20 @@ class TransportExplainAction @Inject constructor( actionListener.onResponse(ExplainResponse(indexNames, indexPolicyIDs, indexMetadatas)) } - fun emptyResponse(size: Int = 0) { + private fun getMetadata(response: GetResponse?): ManagedIndexMetaData? { + if (response == null || response.sourceAsBytesRef == null) + return null + + val xcp = XContentHelper.createParser( + xContentRegistry, + LoggingDeprecationHandler.INSTANCE, + response.sourceAsBytesRef, + XContentType.JSON) + return ManagedIndexMetaData.parseWithType(xcp, + response.id, response.seqNo, response.primaryTerm) + } + + private fun emptyResponse(size: Int = 0) { if (explainAll) { actionListener.onResponse(ExplainAllResponse(emptyList(), emptyList(), emptyList(), size, emptyMap())) return diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/removepolicy/TransportRemovePolicyAction.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/removepolicy/TransportRemovePolicyAction.kt index 4a634c73e..b957a05f3 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/removepolicy/TransportRemovePolicyAction.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/removepolicy/TransportRemovePolicyAction.kt @@ -18,9 +18,8 @@ package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanageme import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementPlugin.Companion.INDEX_MANAGEMENT_INDEX import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.getUuidsForClosedIndices import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.ISMStatusResponse -import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataAction -import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataRequest import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.FailedIndex +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.deleteManagedIndexMetadataRequest import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.deleteManagedIndexRequest import com.amazon.opendistroforelasticsearch.indexmanagement.util.IndexManagementException import org.apache.logging.log4j.LogManager @@ -35,7 +34,6 @@ import org.elasticsearch.action.get.MultiGetResponse import org.elasticsearch.action.support.ActionFilters import org.elasticsearch.action.support.HandledTransportAction import org.elasticsearch.action.support.IndicesOptions -import org.elasticsearch.action.support.master.AcknowledgedResponse import org.elasticsearch.client.node.NodeClient import org.elasticsearch.cluster.ClusterState import org.elasticsearch.cluster.block.ClusterBlockException @@ -175,10 +173,18 @@ class TransportRemovePolicyAction @Inject constructor( } fun removeMetadatas(indices: List) { - val request = UpdateManagedIndexMetaDataRequest(indicesToRemoveManagedIndexMetaDataFrom = indices) - - client.execute(UpdateManagedIndexMetaDataAction.INSTANCE, request, object : ActionListener { - override fun onResponse(response: AcknowledgedResponse) { + val request = indices.map { deleteManagedIndexMetadataRequest(it.uuid) } + val bulkReq = BulkRequest().add(request) + client.bulk(bulkReq, object : ActionListener { + override fun onResponse(response: BulkResponse) { + response.forEach { + val docId = it.id + if (it.isFailed) { + failedIndices.add(FailedIndex(indicesToRemove[docId] as String, docId, + "Failed to clean metadata due to: ${it.failureMessage}")) + indicesToRemove.remove(docId) + } + } actionListener.onResponse(ISMStatusResponse(indicesToRemove.size, failedIndices)) } diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/retryfailedmanagedindex/TransportRetryFailedManagedIndexAction.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/retryfailedmanagedindex/TransportRetryFailedManagedIndexAction.kt index fb87ffa8a..e662d7981 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/retryfailedmanagedindex/TransportRetryFailedManagedIndexAction.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/retryfailedmanagedindex/TransportRetryFailedManagedIndexAction.kt @@ -16,19 +16,19 @@ package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.retryfailedmanagedindex import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementPlugin.Companion.INDEX_MANAGEMENT_INDEX -import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.getManagedIndexMetaData +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.buildMgetMetadataRequest +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.getManagedIndexMetadata +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.elasticapi.mgetResponseToList import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.PolicyRetryInfoMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.ISMStatusResponse -import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataAction -import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataRequest import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.FailedIndex import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.isFailed +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.managedIndexMetadataID import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.updateEnableManagedIndexRequest import org.apache.logging.log4j.LogManager import org.elasticsearch.ExceptionsHelper import org.elasticsearch.action.ActionListener -import org.elasticsearch.action.DocWriteRequest import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse import org.elasticsearch.action.bulk.BulkRequest @@ -38,11 +38,13 @@ import org.elasticsearch.action.get.MultiGetResponse import org.elasticsearch.action.support.ActionFilters import org.elasticsearch.action.support.HandledTransportAction import org.elasticsearch.action.support.IndicesOptions -import org.elasticsearch.action.support.master.AcknowledgedResponse +import org.elasticsearch.action.update.UpdateRequest import org.elasticsearch.client.node.NodeClient import org.elasticsearch.cluster.ClusterState import org.elasticsearch.cluster.block.ClusterBlockException import org.elasticsearch.common.inject.Inject +import org.elasticsearch.common.xcontent.ToXContent +import org.elasticsearch.common.xcontent.XContentFactory import org.elasticsearch.index.Index import org.elasticsearch.index.IndexNotFoundException import org.elasticsearch.tasks.Task @@ -67,9 +69,10 @@ class TransportRetryFailedManagedIndexAction @Inject constructor( private val request: RetryFailedManagedIndexRequest ) { private val failedIndices: MutableList = mutableListOf() - private val listOfIndexMetaDataBulk: MutableList = mutableListOf() - private val listOfIndexMetaData: MutableList> = mutableListOf() - private var updated: Int = 0 + private val listOfMetadata: MutableList = mutableListOf() + private val listOfIndexToMetadata: MutableList> = mutableListOf() + private val mapOfItemIdToIndex: MutableMap = mutableMapOf() + private lateinit var clusterState: ClusterState private val indicesManagedState: MutableMap = mutableMapOf() private var indicesToRetry = mutableMapOf() // uuid: name @@ -89,6 +92,7 @@ class TransportRetryFailedManagedIndexAction @Inject constructor( .cluster() .state(clusterStateRequest, object : ActionListener { override fun onResponse(response: ClusterStateResponse) { + clusterState = response.state val indexMetadatas = response.state.metadata.indices indexMetadatas.forEach { indicesToRetry.putIfAbsent(it.value.indexUUID, it.key) @@ -120,19 +124,11 @@ class TransportRetryFailedManagedIndexAction @Inject constructor( } response.forEach { - log.info("index ${it.id} is exist? ${it.response.isExists}") indicesManagedState[it.id] = it.response.isExists } - populateList(clusterStateResponse.state) - - if (listOfIndexMetaDataBulk.isNotEmpty()) { - updateBulkRequest(listOfIndexMetaDataBulk.map { it.indexUuid }) - } else { - updated = 0 - actionListener.onResponse(ISMStatusResponse(updated, failedIndices)) - return - } + // get back metadata from config index + client.multiGet(buildMgetMetadataRequest(clusterState), ActionListener.wrap(::onMgetMetadataResponse, ::onFailure)) } override fun onFailure(t: Exception) { @@ -141,37 +137,58 @@ class TransportRetryFailedManagedIndexAction @Inject constructor( }) } - private fun populateList(state: ClusterState) { - for (indexMetaDataEntry in state.metadata.indices) { - val indexMetaData = indexMetaDataEntry.value - val managedIndexMetaData = indexMetaData.getManagedIndexMetaData() + @Suppress("ComplexMethod") + private fun onMgetMetadataResponse(mgetResponse: MultiGetResponse) { + val metadataList = mgetResponseToList(mgetResponse) + clusterState.metadata.indices.forEachIndexed { ind, it -> + val indexMetaData = it.value + val clusterStateMetadata = it.value.getManagedIndexMetadata() + val mgetFailure = metadataList[ind]?.second + val managedIndexMetadata: ManagedIndexMetaData? = metadataList[ind]?.first when { indicesManagedState[indexMetaData.indexUUID] == false -> failedIndices.add(FailedIndex(indexMetaData.index.name, indexMetaData.index.uuid, "This index is not being managed.")) - managedIndexMetaData == null -> - failedIndices.add(FailedIndex(indexMetaData.index.name, indexMetaData.index.uuid, "There is no IndexMetaData information")) - !managedIndexMetaData.isFailed -> - failedIndices.add(FailedIndex(indexMetaData.index.name, indexMetaData.index.uuid, "This index is not in failed state.")) + mgetFailure != null -> + failedIndices.add(FailedIndex(indexMetaData.index.name, indexMetaData.index.uuid, + "Failed to get managed index metadata, $mgetFailure")) + managedIndexMetadata == null -> { + if (clusterStateMetadata != null) { + failedIndices.add(FailedIndex(indexMetaData.index.name, indexMetaData.index.uuid, + "Cannot retry until metadata has finished migrating")) + } else { + failedIndices.add(FailedIndex(indexMetaData.index.name, indexMetaData.index.uuid, + "This index has no metadata information")) + } + } + !managedIndexMetadata.isFailed -> + failedIndices.add(FailedIndex(indexMetaData.index.name, indexMetaData.index.uuid, + "This index is not in failed state.")) else -> - listOfIndexMetaDataBulk.add(managedIndexMetaData) + listOfMetadata.add(managedIndexMetadata) } } + + if (listOfMetadata.isNotEmpty()) { + bulkEnableJob(listOfMetadata.map { it.indexUuid }) + } else { + actionListener.onResponse(ISMStatusResponse(0, failedIndices)) + } } - private fun updateBulkRequest(documentIds: List) { - val requestsToRetry = createEnableBulkRequest(documentIds) + private fun bulkEnableJob(jobDocIds: List) { + val requestsToRetry = jobDocIds.map { updateEnableManagedIndexRequest(it) } val bulkRequest = BulkRequest().add(requestsToRetry) - client.bulk(bulkRequest, ActionListener.wrap(::onBulkResponse, ::onFailure)) + client.bulk(bulkRequest, ActionListener.wrap(::onEnableJobBulkResponse, ::onFailure)) } - private fun onBulkResponse(bulkResponse: BulkResponse) { + private fun onEnableJobBulkResponse(bulkResponse: BulkResponse) { for (bulkItemResponse in bulkResponse) { - val managedIndexMetaData = listOfIndexMetaDataBulk.first { it.indexUuid == bulkItemResponse.id } + val managedIndexMetaData = listOfMetadata.first { it.indexUuid == bulkItemResponse.id } if (bulkItemResponse.isFailed) { failedIndices.add(FailedIndex(managedIndexMetaData.index, managedIndexMetaData.indexUuid, bulkItemResponse.failureMessage)) } else { - listOfIndexMetaData.add( + listOfIndexToMetadata.add( Pair(Index(managedIndexMetaData.index, managedIndexMetaData.indexUuid), managedIndexMetaData.copy( stepMetaData = null, policyRetryInfo = PolicyRetryInfoMetaData(false, 0), @@ -188,52 +205,48 @@ class TransportRetryFailedManagedIndexAction @Inject constructor( } } - if (listOfIndexMetaData.isNotEmpty()) { - val updateManagedIndexMetaDataRequest = - UpdateManagedIndexMetaDataRequest(indicesToAddManagedIndexMetaDataTo = listOfIndexMetaData) - client.execute( - UpdateManagedIndexMetaDataAction.INSTANCE, - updateManagedIndexMetaDataRequest, - ActionListener.wrap(::onUpdateManagedIndexMetaDataActionResponse, ::onFailure) - ) + if (listOfIndexToMetadata.isNotEmpty()) { + listOfIndexToMetadata.forEachIndexed { ind, (index, _) -> + mapOfItemIdToIndex[ind] = index + } + + val updateMetadataRequests = listOfIndexToMetadata.map { (index, metadata) -> + val builder = metadata.toXContent(XContentFactory.jsonBuilder(), ToXContent.EMPTY_PARAMS, true) + UpdateRequest(INDEX_MANAGEMENT_INDEX, managedIndexMetadataID(index.uuid)).doc(builder) + } + val bulkUpdateMetadataRequest = BulkRequest().add(updateMetadataRequests) + + client.bulk(bulkUpdateMetadataRequest, ActionListener.wrap(::onBulkUpdateMetadataResponse, ::onFailure)) } else { - updated = 0 - actionListener.onResponse(ISMStatusResponse(updated, failedIndices)) - return + actionListener.onResponse(ISMStatusResponse(0, failedIndices)) } } - private fun createEnableBulkRequest(documentIds: List): List> { - return documentIds.map { updateEnableManagedIndexRequest(it) } - } - - private fun onUpdateManagedIndexMetaDataActionResponse(response: AcknowledgedResponse) { - if (response.isAcknowledged) { - updated = listOfIndexMetaData.size - } else { - updated = 0 - failedIndices.addAll(listOfIndexMetaData.map { - FailedIndex(it.first.name, it.first.uuid, "failed to update IndexMetaData") - }) + private fun onBulkUpdateMetadataResponse(bulkResponse: BulkResponse) { + val failedResponses = (bulkResponse.items ?: arrayOf()).filter { it.isFailed } + failedResponses.forEach { + val index = mapOfItemIdToIndex[it.itemId] + if (index != null) { + failedIndices.add(FailedIndex(index.name, index.uuid, "Failed to update metadata for index ${index.name}")) + } } + + val updated = (bulkResponse.items ?: arrayOf()).size - failedResponses.size actionListener.onResponse(ISMStatusResponse(updated, failedIndices)) - return } fun onFailure(e: Exception) { try { if (e is ClusterBlockException) { - failedIndices.addAll(listOfIndexMetaData.map { - FailedIndex(it.first.name, it.first.uuid, "failed to update with ClusterBlockException. ${e.message}") + failedIndices.addAll(listOfIndexToMetadata.map { + FailedIndex(it.first.name, it.first.uuid, "Failed to update due to ClusterBlockException. ${e.message}") }) } - updated = 0 - actionListener.onResponse(ISMStatusResponse(updated, failedIndices)) - return + actionListener.onResponse(ISMStatusResponse(0, failedIndices)) } catch (inner: Exception) { inner.addSuppressed(e) - log.error("failed to send failure response", inner) + log.error("Failed to send failure response", inner) } } } diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/updateindexmetadata/TransportUpdateManagedIndexMetaDataAction.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/updateindexmetadata/TransportUpdateManagedIndexMetaDataAction.kt index 357f893b9..492491a0a 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/updateindexmetadata/TransportUpdateManagedIndexMetaDataAction.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/transport/action/updateindexmetadata/TransportUpdateManagedIndexMetaDataAction.kt @@ -15,13 +15,8 @@ package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata -import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.IndexStateManagementHistory import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementPlugin import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexMetaData -import kotlinx.coroutines.CoroutineName -import kotlinx.coroutines.Dispatchers -import kotlinx.coroutines.GlobalScope -import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager import org.elasticsearch.action.ActionListener import org.elasticsearch.action.support.ActionFilters @@ -46,7 +41,6 @@ import org.elasticsearch.common.io.stream.Writeable import org.elasticsearch.index.Index import org.elasticsearch.threadpool.ThreadPool import org.elasticsearch.transport.TransportService -import java.lang.Exception class TransportUpdateManagedIndexMetaDataAction : TransportMasterNodeAction { @@ -57,8 +51,7 @@ class TransportUpdateManagedIndexMetaDataAction : TransportMasterNodeAction, + currentManagedIndexConfigs: Map +): List { + val currentIndicesSet = currentIndices.map { it.index }.toSet() + val managedIndicesSet = currentManagedIndexConfigs.values.map { Index(it.index, it.uuid) }.toSet() + return (managedIndicesSet - currentIndicesSet).toList() +} + fun getSweptManagedIndexSearchRequest(): SearchRequest { val boolQueryBuilder = BoolQueryBuilder().filter(QueryBuilders.existsQuery(ManagedIndexConfig.MANAGED_INDEX_TYPE)) return SearchRequest() @@ -285,6 +329,7 @@ fun Action.getUpdatedActionMetaData(managedIndexMetaData: ManagedIndexMetaData, val actionMetaData = managedIndexMetaData.actionMetaData return when { + // start a new action stateMetaData?.name != state.name -> ActionMetaData(this.type.type, Instant.now().toEpochMilli(), this.config.actionIndex, false, 0, 0, null) actionMetaData?.index != this.config.actionIndex -> @@ -500,6 +545,36 @@ fun Policy.getDisallowedActions(allowList: List): List { */ fun Action.isAllowed(allowList: List): Boolean = allowList.contains(this.type.type) +/** + * Check if cluster state metadata has been moved to config index + * + * log warning if remaining cluster state metadata has newer last_updated_time + */ +fun isMetadataMoved( + clusterStateMetadata: ManagedIndexMetaData?, + configIndexMetadata: Any?, + logger: Logger +): Boolean { + if (clusterStateMetadata != null) { + if (configIndexMetadata == null) return false + + // compare last updated time between 2 metadatas + val t1 = clusterStateMetadata.stepMetaData?.startTime + val t2 = when (configIndexMetadata) { + is ManagedIndexMetaData? -> configIndexMetadata.stepMetaData?.startTime + is Map<*, *>? -> { + val stepMetadata = configIndexMetadata["step"] as Map? + stepMetadata?.get("start_time") + } + else -> null + } as Long? + if (t1 != null && t2 != null && t1 > t2) { + logger.warn("Cluster state metadata get updates after moved for [${clusterStateMetadata.index}]") + } + } + return true +} + private val baseMessageLogger = LogManager.getLogger(BaseMessage::class.java) fun BaseMessage.isHostInDenylist(networks: List): Boolean { diff --git a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/rollup/model/Rollup.kt b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/rollup/model/Rollup.kt index 452775282..7ff13d92e 100644 --- a/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/rollup/model/Rollup.kt +++ b/src/main/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/rollup/model/Rollup.kt @@ -228,7 +228,7 @@ data class Rollup( const val ROLLUP_DOC_COUNT_FIELD = "$ROLLUP_TYPE._doc_count" const val ROLLUP_DOC_SCHEMA_VERSION_FIELD = "$ROLLUP_TYPE._$SCHEMA_VERSION_FIELD" - @Suppress("ComplexMethod", "LongMethod") + @Suppress("ComplexMethod", "LongMethod", "NestedBlockDepth") @JvmStatic @JvmOverloads @Throws(IOException::class) diff --git a/src/main/resources/mappings/opendistro-ism-config.json b/src/main/resources/mappings/opendistro-ism-config.json index 1f099d85b..6a96bf9eb 100644 --- a/src/main/resources/mappings/opendistro-ism-config.json +++ b/src/main/resources/mappings/opendistro-ism-config.json @@ -1,6 +1,6 @@ { "_meta" : { - "schema_version": 8 + "schema_version": 9 }, "dynamic": "strict", "properties": { @@ -543,6 +543,143 @@ } } }, + "managed_index_metadata": { + "properties": { + "index": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "index_uuid": { + "type": "keyword" + }, + "policy_id": { + "type": "keyword" + }, + "policy_seq_no": { + "type": "long" + }, + "policy_primary_term": { + "type": "long" + }, + "policy_completed": { + "type": "boolean" + }, + "rolled_over": { + "type": "boolean" + }, + "transition_to": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "state": { + "properties": { + "name": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + } + } + }, + "action": { + "properties": { + "name": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "index": { + "type": "integer" + }, + "failed": { + "type": "boolean" + }, + "consumed_retries": { + "type": "integer" + }, + "last_retry_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "action_properties": { + "properties": { + "max_num_segments": { + "type": "integer" + }, + "snapshot_name": { + "type": "keyword" + }, + "rollup_id": { + "type": "keyword" + }, + "has_rollup_failed": { + "type": "boolean" + } + } + } + } + }, + "step": { + "properties": { + "name": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "step_status": { + "type": "keyword" + } + } + }, + "retry_info": { + "properties": { + "failed": { + "type": "boolean" + }, + "consumed_retries": { + "type": "integer" + } + } + }, + "info": { + "type": "object", + "enabled": false + } + } + }, "rollup": { "properties": { "rollup_id": { diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementIndicesIT.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementIndicesIT.kt index 71fdee964..cd3e80b93 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementIndicesIT.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementIndicesIT.kt @@ -24,8 +24,6 @@ import java.util.Locale class IndexManagementIndicesIT : IndexStateManagementRestTestCase() { private val testIndexName = javaClass.simpleName.toLowerCase(Locale.ROOT) - private val configSchemaVersion = 8 - private val historySchemaVersion = 3 /* * If this test fails it means you changed the config mappings diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementRestTestCase.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementRestTestCase.kt index 3e86fa500..f4dbe017b 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementRestTestCase.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/IndexManagementRestTestCase.kt @@ -35,6 +35,9 @@ import javax.management.remote.JMXServiceURL abstract class IndexManagementRestTestCase : ODFERestTestCase() { + val configSchemaVersion = 9 + val historySchemaVersion = 3 + // Having issues with tests leaking into other tests and mappings being incorrect and they are not caught by any pending task wait check as // they do not go through the pending task queue. Ideally this should probably be written in a way to wait for the // jobs themselves to finish and gracefully shut them down.. but for now seeing if this works. diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/IndexStateManagementIntegTestCase.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/IndexStateManagementIntegTestCase.kt new file mode 100644 index 000000000..636f2cd9d --- /dev/null +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/IndexStateManagementIntegTestCase.kt @@ -0,0 +1,347 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement + +import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementPlugin +import com.amazon.opendistroforelasticsearch.indexmanagement.elasticapi.parseWithType +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexConfig +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.ManagedIndexMetaData +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.Policy +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.PolicyRetryInfoMetaData +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.StateMetaData +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.resthandler.RestExplainAction +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.explain.ExplainAction +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.explain.TransportExplainAction +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.TransportUpdateManagedIndexMetaDataAction +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataAction +import com.amazon.opendistroforelasticsearch.indexmanagement.makeRequest +import com.amazon.opendistroforelasticsearch.indexmanagement.waitFor +import com.amazon.opendistroforelasticsearch.jobscheduler.spi.schedule.IntervalSchedule +import org.apache.http.entity.ContentType +import org.apache.http.entity.StringEntity +import org.elasticsearch.ElasticsearchParseException +import org.elasticsearch.action.ActionRequest +import org.elasticsearch.action.ActionResponse +import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteRequest +import org.elasticsearch.action.search.SearchResponse +import org.elasticsearch.client.Request +import org.elasticsearch.client.Response +import org.elasticsearch.cluster.metadata.IndexMetadata +import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand +import org.elasticsearch.common.Strings +import org.elasticsearch.common.settings.Settings +import org.elasticsearch.common.xcontent.DeprecationHandler +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler +import org.elasticsearch.common.xcontent.NamedXContentRegistry +import org.elasticsearch.common.xcontent.XContentHelper +import org.elasticsearch.common.xcontent.XContentParser +import org.elasticsearch.common.xcontent.XContentParserUtils +import org.elasticsearch.common.xcontent.XContentType +import org.elasticsearch.common.xcontent.json.JsonXContent +import org.elasticsearch.plugins.ActionPlugin +import org.elasticsearch.plugins.Plugin +import org.elasticsearch.rest.RestRequest +import org.elasticsearch.rest.RestStatus +import org.elasticsearch.test.ESIntegTestCase +import org.elasticsearch.test.rest.ESRestTestCase.entityAsMap +import java.io.IOException +import java.time.Duration +import java.time.Instant + +abstract class IndexStateManagementIntegTestCase : ESIntegTestCase() { + + protected val isMixedNodeRegressionTest = System.getProperty("cluster.mixed", "false")!!.toBoolean() + + var metadataToClusterState = ManagedIndexMetaData( + index = "", + indexUuid = "", + policyID = "", + policySeqNo = 0, + policyPrimaryTerm = 1, + policyCompleted = false, + rolledOver = false, + transitionTo = null, + stateMetaData = StateMetaData("ReplicaCountState", 1234), + actionMetaData = null, + stepMetaData = null, + policyRetryInfo = PolicyRetryInfoMetaData(false, 0), + info = mapOf("message" to "Happy moving") + ) + + override fun nodePlugins(): Collection> { + return listOf(IndexManagementPlugin::class.java) + } + + class TestPlugin : ActionPlugin, Plugin() { + override fun getActions(): List> { + return listOf( + ActionPlugin.ActionHandler( + UpdateManagedIndexMetaDataAction.INSTANCE, + TransportUpdateManagedIndexMetaDataAction::class.java + ), + ActionPlugin.ActionHandler(ExplainAction.INSTANCE, TransportExplainAction::class.java) + ) + } + } + + override fun transportClientPlugins(): Collection> { + return listOf(TestPlugin::class.java) + } + + protected fun getIndexMetadata(indexName: String): IndexMetadata { + return client().admin().cluster().prepareState() + .setIndices(indexName) + .setMetadata(true).get() + .state.metadata.indices[indexName] + } + + // reuse utility fun from RestTestCase + fun createPolicy( + policy: Policy, + policyId: String = randomAlphaOfLength(10), + refresh: Boolean = true + ): Policy { + val response = createPolicyJson(policy.toJsonString(), policyId, refresh) + + val policyJson = JsonXContent.jsonXContent + .createParser( + NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, + response.entity.content + ).map() + val createdId = policyJson["_id"] as String + assertEquals("policy ids are not the same", policyId, createdId) + return policy.copy( + id = createdId, + seqNo = (policyJson["_seq_no"] as Int).toLong(), + primaryTerm = (policyJson["_primary_term"] as Int).toLong() + ) + } + + protected fun createPolicyJson( + policyString: String, + policyId: String, + refresh: Boolean = true + ): Response { + val response = getRestClient() + .makeRequest( + "PUT", + "${IndexManagementPlugin.POLICY_BASE_URI}/$policyId?refresh=$refresh", + emptyMap(), + StringEntity(policyString, ContentType.APPLICATION_JSON) + ) + assertEquals("Unable to create a new policy", RestStatus.CREATED, response.restStatus()) + return response + } + + protected fun Response.restStatus(): RestStatus = RestStatus.fromCode(this.statusLine.statusCode) + + protected fun addPolicyToIndex( + index: String, + policyID: String + ) { + val body = """ + { + "policy_id": "$policyID" + } + """.trimIndent() + val response = getRestClient() + .makeRequest("POST", "/_opendistro/_ism/add/$index", StringEntity(body, ContentType.APPLICATION_JSON)) + assertEquals("Unexpected RestStatus", RestStatus.OK, response.restStatus()) + } + + protected fun getExistingManagedIndexConfig(index: String): ManagedIndexConfig { + return waitFor { + val config = getManagedIndexConfig(index) + assertNotNull("ManagedIndexConfig is null", config) + config!! + } + } + + protected fun getManagedIndexConfig(index: String): ManagedIndexConfig? { + val request = """ + { + "seq_no_primary_term": true, + "query": { + "term": { + "${ManagedIndexConfig.MANAGED_INDEX_TYPE}.${ManagedIndexConfig.INDEX_FIELD}": "$index" + } + } + } + """.trimIndent() + val response = getRestClient().makeRequest( + "POST", "${IndexManagementPlugin.INDEX_MANAGEMENT_INDEX}/_search", emptyMap(), + StringEntity(request, ContentType.APPLICATION_JSON) + ) + assertEquals("Request failed", RestStatus.OK, response.restStatus()) + val searchResponse = + SearchResponse.fromXContent(createParser(JsonXContent.jsonXContent, response.entity.content)) + assertTrue("Found more than one managed index config", searchResponse.hits.hits.size < 2) + val hit = searchResponse.hits.hits.firstOrNull() + return hit?.run { + val xcp = createParser(JsonXContent.jsonXContent, this.sourceRef) + xcp.parseWithType(id, seqNo, primaryTerm, ManagedIndexConfig.Companion::parse) + } + } + + protected fun updateManagedIndexConfigStartTime(update: ManagedIndexConfig, desiredStartTimeMillis: Long? = null) { + val intervalSchedule = (update.jobSchedule as IntervalSchedule) + val millis = Duration.of(intervalSchedule.interval.toLong(), intervalSchedule.unit).minusSeconds(2).toMillis() + val startTimeMillis = desiredStartTimeMillis ?: Instant.now().toEpochMilli() - millis + val response = getRestClient().makeRequest( + "POST", "${IndexManagementPlugin.INDEX_MANAGEMENT_INDEX}/_update/${update.id}", + StringEntity( + "{\"doc\":{\"managed_index\":{\"schedule\":{\"interval\":{\"start_time\":" + + "\"$startTimeMillis\"}}}}}", + ContentType.APPLICATION_JSON + ) + ) + + assertEquals("Request failed", RestStatus.OK, response.restStatus()) + } + + protected fun updateManagedIndexConfigPolicy(update: ManagedIndexConfig, policy: Policy) { + val policyJsonString = policy.toJsonString() + logger.info("policy string: $policyJsonString") + var response = getRestClient().makeRequest( + "POST", "${IndexManagementPlugin.INDEX_MANAGEMENT_INDEX}/_update/${update.id}", + StringEntity( + "{\"doc\":{\"managed_index\": $policyJsonString }}", + ContentType.APPLICATION_JSON + ) + ) + + assertEquals("Request failed", RestStatus.OK, response.restStatus()) + + response = getRestClient().makeRequest( + "POST", "${IndexManagementPlugin.INDEX_MANAGEMENT_INDEX}/_update/${update.id}", + StringEntity( + "{\"doc\":{\"managed_index\": {\"policy_seq_no\": \"0\", \"policy_primary_term\": \"1\"} }}", + ContentType.APPLICATION_JSON + ) + ) + + assertEquals("Request failed", RestStatus.OK, response.restStatus()) + } + + @Suppress("UNCHECKED_CAST") + protected fun getNumberOfReplicasSetting(indexName: String): Int { + val indexSettings = getIndexSettings(indexName) as Map>> + return (indexSettings[indexName]!!["settings"]!!["index.number_of_replicas"] as String).toInt() + } + + @Throws(IOException::class) + protected open fun getIndexSettings(index: String): Map? { + val request = Request("GET", "/$index/_settings") + request.addParameter("flat_settings", "true") + val response = getRestClient().performRequest(request) + response.entity.content.use { `is` -> + return XContentHelper.convertToMap( + XContentType.JSON.xContent(), + `is`, + true + ) + } + } + + protected fun getExplainManagedIndexMetaData(indexName: String): ManagedIndexMetaData { + if (indexName.contains("*") || indexName.contains(",")) { + throw IllegalArgumentException("This method is only for a single concrete index") + } + + val response = getRestClient().makeRequest( + RestRequest.Method.GET.toString(), + "${RestExplainAction.EXPLAIN_BASE_URI}/$indexName" + ) + assertEquals("Unexpected RestStatus", RestStatus.OK, response.restStatus()) + + lateinit var metadata: ManagedIndexMetaData + val xcp = createParser(XContentType.JSON.xContent(), response.entity.content) + XContentParserUtils.ensureExpectedToken( + XContentParser.Token.START_OBJECT, + xcp.nextToken(), + xcp + ) + while (xcp.nextToken() != XContentParser.Token.END_OBJECT) { + xcp.currentName() + xcp.nextToken() + + metadata = ManagedIndexMetaData.parse(xcp) + } + return metadata + } + + protected fun assertIndexExists(index: String) { + val response = getRestClient().makeRequest("HEAD", index) + assertEquals("Index $index does not exist.", RestStatus.OK, response.restStatus()) + } + + fun getShardSegmentStats(index: String): Map { + val response = getRestClient().makeRequest("GET", "/$index/_stats/segments?level=shards") + + assertEquals("Stats request failed", RestStatus.OK, response.restStatus()) + + return response.asMap() + } + + fun catIndexShard(index: String): List { + val response = getRestClient().makeRequest("GET", "_cat/shards/$index?format=json") + + assertEquals("Stats request failed", RestStatus.OK, response.restStatus()) + + try { + return JsonXContent.jsonXContent + .createParser( + NamedXContentRegistry.EMPTY, + DeprecationHandler.THROW_UNSUPPORTED_OPERATION, + response.entity.content + ) + .use { parser -> parser.list() } + } catch (e: IOException) { + throw ElasticsearchParseException("Failed to parse content to list", e) + } + } + + fun Response.asMap(): Map = entityAsMap(this) + + fun rerouteShard(configIndexName: String, fromNode: String, toNode: String) { + logger.info("Reallocating Shard. From Node: $fromNode To Node: $toNode ") + val moveCommand = MoveAllocationCommand(configIndexName, 0, fromNode, toNode) + val rerouteResponse = client().admin().cluster() + .reroute(ClusterRerouteRequest().add(moveCommand)).actionGet() + logger.info("reroute success? ${rerouteResponse.isAcknowledged}") + } + + fun updateIndexSettings(index: String, settings: Settings) { + val request = Request("PUT", "/$index/_settings") + request.setJsonEntity(Strings.toString(settings)) + getRestClient().performRequest(request) + } + + fun updateClusterSetting(key: String, value: String?, escapeValue: Boolean = true) { + val formattedValue = if (escapeValue) "\"$value\"" else value + val request = """ + { + "persistent": { + "$key": $formattedValue + } + } + """.trimIndent() + val res = getRestClient().makeRequest("PUT", "_cluster/settings", emptyMap(), + StringEntity(request, ContentType.APPLICATION_JSON)) + assertEquals("Request failed", RestStatus.OK, res.restStatus()) + } +} diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/IndexStateManagementRestTestCase.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/IndexStateManagementRestTestCase.kt index c6e7daf11..e69929b03 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/IndexStateManagementRestTestCase.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/IndexStateManagementRestTestCase.kt @@ -33,6 +33,7 @@ import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagemen import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.ActionMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.PolicyRetryInfoMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.StateMetaData +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.StepMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.resthandler.RestExplainAction import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.util.FAILED_INDICES @@ -167,6 +168,7 @@ abstract class IndexStateManagementRestTestCase : IndexManagementRestTestCase() shards: String? = null, mapping: String = "" ): Pair { + val waitForActiveShards = if (isMultiNode) "all" else "1" val settings = Settings.builder().let { if (alias == null) { it.putNull(ManagedIndexSettings.ROLLOVER_ALIAS.key) @@ -175,6 +177,7 @@ abstract class IndexStateManagementRestTestCase : IndexManagementRestTestCase() } it.put(INDEX_NUMBER_OF_REPLICAS, replicas ?: "1") it.put(INDEX_NUMBER_OF_SHARDS, shards ?: "1") + it.put("index.write.wait_for_active_shards", waitForActiveShards) }.build() val aliases = if (alias == null) "" else "\"$alias\": { \"is_write_index\": true }" createIndex(index, settings, mapping, aliases) @@ -500,7 +503,7 @@ abstract class IndexStateManagementRestTestCase : IndexManagementRestTestCase() if (cn == "total_managed_indices") continue metadata = ManagedIndexMetaData.parse(xcp) - break + break // bypass roles field } // make sure metadata is initialised @@ -729,6 +732,16 @@ abstract class IndexStateManagementRestTestCase : IndexManagementRestTestCase() return true } + @Suppress("UNCHECKED_CAST") + protected fun assertStepEquals(expectedStep: StepMetaData, actualStepMap: Any?): Boolean { + actualStepMap as Map + assertEquals(expectedStep.name, actualStepMap[ManagedIndexMetaData.NAME] as String) + assertEquals(expectedStep.stepStatus.toString(), actualStepMap[StepMetaData.STEP_STATUS]) + val expectedStartTime = expectedStep.startTime + assertTrue((actualStepMap[ManagedIndexMetaData.START_TIME] as Long) < expectedStartTime) + return true + } + protected fun assertPredicatesOnISMTemplatesMap( templatePredicates: List Boolean>>>>, // response map name: predicate response: Map diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/MetadataRegressionIT.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/MetadataRegressionIT.kt new file mode 100644 index 000000000..86d26eedd --- /dev/null +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/MetadataRegressionIT.kt @@ -0,0 +1,285 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement + +import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementPlugin.Companion.INDEX_MANAGEMENT_INDEX +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.Policy +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.State +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.action.ReplicaCountActionConfig +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataAction +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.transport.action.updateindexmetadata.UpdateManagedIndexMetaDataRequest +import com.amazon.opendistroforelasticsearch.indexmanagement.waitFor +import com.carrotsearch.randomizedtesting.RandomizedTest.sleep +import org.elasticsearch.action.support.master.AcknowledgedResponse +import org.elasticsearch.cluster.metadata.IndexMetadata +import org.elasticsearch.common.settings.Settings +import org.elasticsearch.index.Index +import org.junit.After +import org.junit.Assume +import org.junit.Before +import java.time.Instant +import java.time.temporal.ChronoUnit +import java.util.Locale +import kotlin.collections.HashMap + +class MetadataRegressionIT : IndexStateManagementIntegTestCase() { + + private val testIndexName = javaClass.simpleName.toLowerCase(Locale.ROOT) + + @Before + fun startMetadataService() { + // metadata service could be stopped before following tests start run + // this will enable metadata service again + updateClusterSetting(ManagedIndexSettings.METADATA_SERVICE_ENABLED.key, "false") + updateClusterSetting(ManagedIndexSettings.METADATA_SERVICE_ENABLED.key, "true") + } + + @After + fun cleanClusterSetting() { + // need to clean up otherwise will throw error + updateClusterSetting(ManagedIndexSettings.METADATA_SERVICE_ENABLED.key, null, false) + } + + fun `test move metadata service`() { + updateClusterSetting(ManagedIndexSettings.METADATA_SERVICE_ENABLED.key, "false") + updateClusterSetting(ManagedIndexSettings.METADATA_SERVICE_ENABLED.key, "true") + + val indexName = "${testIndexName}_index_1" + val policyID = "${testIndexName}_testPolicyName_1" + val actionConfig = ReplicaCountActionConfig(10, 0) + val states = listOf(State(name = "ReplicaCountState", actions = listOf(actionConfig), transitions = listOf())) + val policy = Policy( + id = policyID, + description = "$testIndexName description", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states[0].name, + states = states + ) + + createPolicy(policy, policyID) + createIndex(indexName) + + // put some metadata into cluster state + var indexMetadata = getIndexMetadata(indexName) + metadataToClusterState = metadataToClusterState.copy( + index = indexName, + indexUuid = indexMetadata.indexUUID, + policyID = policyID + ) + val request = UpdateManagedIndexMetaDataRequest( + indicesToAddManagedIndexMetaDataTo = listOf( + Pair(Index(metadataToClusterState.index, metadataToClusterState.indexUuid), metadataToClusterState) + ) + ) + val response: AcknowledgedResponse = client().execute( + UpdateManagedIndexMetaDataAction.INSTANCE, request + ).get() + logger.info(response.isAcknowledged) + indexMetadata = getIndexMetadata(indexName) + logger.info("check if metadata is saved in cluster state: ${indexMetadata.getCustomData("managed_index_metadata")}") + + // create a job + addPolicyToIndex(indexName, policyID) + + waitFor { + assertEquals( + "Metadata is pending migration", + getExplainManagedIndexMetaData(indexName).info?.get("message") + ) + } + + waitFor(Instant.ofEpochSecond(120)) { + assertEquals( + "Happy moving", + getExplainManagedIndexMetaData(indexName).info?.get("message") + ) + assertEquals(null, getIndexMetadata(indexName).getCustomData("managed_index_metadata")) + } + + logger.info("metadata has moved") + + val managedIndexConfig = getExistingManagedIndexConfig(indexName) + // Change the start time so the job will trigger in 2 seconds, this will trigger the first initialization of the policy + updateManagedIndexConfigStartTime(managedIndexConfig) + + waitFor { assertEquals(policyID, getExplainManagedIndexMetaData(indexName).policyID) } + waitFor { + assertEquals( + "Successfully initialized policy: ${policy.id}", + getExplainManagedIndexMetaData(indexName).info?.get("message") + ) + } + + updateManagedIndexConfigStartTime(managedIndexConfig) + waitFor { + assertEquals( + "Index did not set number_of_replicas to ${actionConfig.numOfReplicas}", + actionConfig.numOfReplicas, + getNumberOfReplicasSetting(indexName) + ) + } + } + + fun `test job can continue run from cluster state metadata`() { + /** + * new version of ISM plugin can handle metadata in cluster state + * when job already started + * + * create index, add policy to it + * manually add policy field to managed-index so runner won't do initialisation itself + * add metadata into cluster state + * then check if we can continue run from this added metadata + */ + + val indexName = "${testIndexName}_index_2" + val policyID = "${testIndexName}_testPolicyName_2" + val actionConfig = ReplicaCountActionConfig(10, 0) + val states = listOf(State(name = "ReplicaCountState", actions = listOf(actionConfig), transitions = listOf())) + val policy = Policy( + id = policyID, + description = "$testIndexName description", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states[0].name, + states = states + ) + + createPolicy(policy, policyID) + createIndex(indexName) + addPolicyToIndex(indexName, policyID) + + val managedIndexConfig = getExistingManagedIndexConfig(indexName) + // manually add policy field into managed-index + updateManagedIndexConfigPolicy(managedIndexConfig, policy) + logger.info("managed-index: ${getExistingManagedIndexConfig(indexName)}") + + // manually save metadata into cluster state + var indexMetadata = getIndexMetadata(indexName) + metadataToClusterState = metadataToClusterState.copy( + index = indexName, + indexUuid = indexMetadata.indexUUID, + policyID = policyID + ) + val request = UpdateManagedIndexMetaDataRequest( + indicesToAddManagedIndexMetaDataTo = listOf( + Pair(Index(metadataToClusterState.index, metadataToClusterState.indexUuid), metadataToClusterState) + ) + ) + val response: AcknowledgedResponse = client().execute( + UpdateManagedIndexMetaDataAction.INSTANCE, request + ).get() + + logger.info(response.isAcknowledged) + indexMetadata = getIndexMetadata(indexName) + logger.info("check if metadata is saved in cluster state: ${indexMetadata.getCustomData("managed_index_metadata")}") + + waitFor { + assertEquals( + "Metadata is pending migration", + getExplainManagedIndexMetaData(indexName).info?.get("message") + ) + } + + waitFor(Instant.ofEpochSecond(120)) { + assertEquals( + "Happy moving", + getExplainManagedIndexMetaData(indexName).info?.get("message") + ) + assertEquals(null, getIndexMetadata(indexName).getCustomData("managed_index_metadata")) + } + + logger.info("metadata has moved") + + // start the job run + updateManagedIndexConfigStartTime(managedIndexConfig) + waitFor { + assertEquals( + "Index did not set number_of_replicas to ${actionConfig.numOfReplicas}", + actionConfig.numOfReplicas, + getNumberOfReplicasSetting(indexName) + ) + } + } + + fun `test new node skip execution when old node exist in cluster`() { + Assume.assumeTrue(isMixedNodeRegressionTest) + + /** + * mixedCluster-0 is new node, mixedCluster-1 is old node + * + * set config index to only have one shard on new node + * so old node cannot run job because it has no shard + * new node also cannot run job because there is an old node + * here we check no job can be run + * + * then reroute shard to old node and this old node can run job + */ + + val indexName = "${testIndexName}_index_1" + val policyID = "${testIndexName}_testPolicyName_1" + val actionConfig = ReplicaCountActionConfig(10, 0) + val states = listOf(State(name = "ReplicaCountState", actions = listOf(actionConfig), transitions = listOf())) + val policy = Policy( + id = policyID, + description = "$testIndexName description", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states[0].name, + states = states + ) + + createPolicy(policy, policyID) + createIndex(indexName) + + val settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, "0") + .build() + updateIndexSettings(INDEX_MANAGEMENT_INDEX, settings) + + // check config index shard position + val shardsResponse = catIndexShard(INDEX_MANAGEMENT_INDEX) + logger.info("check config index shard: $shardsResponse") + val shardNode = (shardsResponse[0] as HashMap<*, *>)["node"] + + sleep(3000) // wait some time for cluster to be stable + + // move shard on node1 to node0 if exist + if (shardNode == "mixedCluster-1") rerouteShard(INDEX_MANAGEMENT_INDEX, "mixedCluster-1", "mixedCluster-0") + + addPolicyToIndex(indexName, policyID) + + val managedIndexConfig = getExistingManagedIndexConfig(indexName) + updateManagedIndexConfigStartTime(managedIndexConfig) + + // check no job has been run + wait { assertEquals(null, getExistingManagedIndexConfig(indexName).policy) } + + // reroute shard to node1 + rerouteShard(INDEX_MANAGEMENT_INDEX, "mixedCluster-0", "mixedCluster-1") + + val shardsResponse2 = catIndexShard(INDEX_MANAGEMENT_INDEX) + logger.info("check config index shard: $shardsResponse2") + + // job can be ran now + updateManagedIndexConfigStartTime(managedIndexConfig) + waitFor { assertEquals(policyID, getExplainManagedIndexMetaData(indexName).policyID) } + } +} diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/TestHelpers.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/TestHelpers.kt index 67097dc71..5a3754180 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/TestHelpers.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/TestHelpers.kt @@ -422,3 +422,22 @@ fun ISMTemplate.toJsonString(): String { val builder = XContentFactory.jsonBuilder() return this.toXContent(builder, ToXContent.EMPTY_PARAMS).string() } + +fun wait( + timeout: Instant = Instant.ofEpochSecond(10), + block: () -> T +) { + val startTime = Instant.now().toEpochMilli() + do { + try { + block() + if ((Instant.now().toEpochMilli() - startTime) > timeout.toEpochMilli()) { + return + } else { + Thread.sleep(100L) + } + } catch (e: Throwable) { + throw e + } + } while (true) +} diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/action/ActionRetryIT.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/action/ActionRetryIT.kt index ad8bbe36a..433122f94 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/action/ActionRetryIT.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/action/ActionRetryIT.kt @@ -20,7 +20,9 @@ import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagemen import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.ActionMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.PolicyRetryInfoMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.StateMetaData +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.StepMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.step.Step import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.step.rollover.AttemptRolloverStep import com.amazon.opendistroforelasticsearch.indexmanagement.waitFor import java.time.Instant @@ -154,6 +156,11 @@ class ActionRetryIT : IndexStateManagementRestTestCase() { ActionMetaData("rollover", Instant.now().toEpochMilli(), 0, false, 1, null, null), actionMetaDataMap ), + StepMetaData.STEP to fun(stepMetaDataMap: Any?): Boolean = + assertStepEquals( + StepMetaData("attempt_rollover", Instant.now().toEpochMilli(), Step.StepStatus.FAILED), + stepMetaDataMap + ), PolicyRetryInfoMetaData.RETRY_INFO to fun(retryInfoMetaDataMap: Any?): Boolean = assertRetryInfoEquals(PolicyRetryInfoMetaData(false, 0), retryInfoMetaDataMap), ManagedIndexMetaData.INFO to fun(info: Any?): Boolean = expectedInfoString == info.toString() diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/action/IndexStateManagementHistoryIT.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/action/IndexStateManagementHistoryIT.kt index af83d4168..07c1f3c25 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/action/IndexStateManagementHistoryIT.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/action/IndexStateManagementHistoryIT.kt @@ -25,8 +25,10 @@ import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagemen import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.ActionMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.PolicyRetryInfoMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.StateMetaData +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.managedindexmetadata.StepMetaData import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.randomErrorNotification import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.step.Step import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.step.readonly.SetReadOnlyStep import com.amazon.opendistroforelasticsearch.indexmanagement.waitFor import org.elasticsearch.action.search.SearchResponse @@ -90,7 +92,7 @@ class IndexStateManagementHistoryIT : IndexStateManagementRestTestCase() { transitionTo = null, stateMetaData = StateMetaData("ReadOnlyState", actualHistory.stateMetaData!!.startTime), actionMetaData = ActionMetaData(ActionConfig.ActionType.READ_ONLY.toString(), actualHistory.actionMetaData!!.startTime, 0, false, 0, 0, null), - stepMetaData = null, + stepMetaData = StepMetaData("set_read_only", actualHistory.stepMetaData!!.startTime, Step.StepStatus.COMPLETED), policyRetryInfo = PolicyRetryInfoMetaData(false, 0), info = mapOf("message" to SetReadOnlyStep.getSuccessMessage(indexName)) ) @@ -156,7 +158,7 @@ class IndexStateManagementHistoryIT : IndexStateManagementRestTestCase() { transitionTo = null, stateMetaData = StateMetaData("ReadOnlyState", actualHistory.stateMetaData!!.startTime), actionMetaData = ActionMetaData(ActionConfig.ActionType.READ_ONLY.toString(), actualHistory.actionMetaData!!.startTime, 0, false, 0, 0, null), - stepMetaData = null, + stepMetaData = StepMetaData("set_read_only", actualHistory.stepMetaData!!.startTime, Step.StepStatus.COMPLETED), policyRetryInfo = PolicyRetryInfoMetaData(false, 0), info = mapOf("message" to SetReadOnlyStep.getSuccessMessage(indexName)) ) @@ -222,7 +224,7 @@ class IndexStateManagementHistoryIT : IndexStateManagementRestTestCase() { transitionTo = null, stateMetaData = StateMetaData("ReadOnlyState", actualHistory.stateMetaData!!.startTime), actionMetaData = ActionMetaData(ActionConfig.ActionType.READ_ONLY.toString(), actualHistory.actionMetaData!!.startTime, 0, false, 0, 0, null), - stepMetaData = null, + stepMetaData = StepMetaData("set_read_only", actualHistory.stepMetaData!!.startTime, Step.StepStatus.COMPLETED), policyRetryInfo = PolicyRetryInfoMetaData(false, 0), info = mapOf("message" to SetReadOnlyStep.getSuccessMessage(indexName)) ) @@ -312,7 +314,7 @@ class IndexStateManagementHistoryIT : IndexStateManagementRestTestCase() { transitionTo = null, stateMetaData = StateMetaData(states[0].name, actualHistory1.stateMetaData!!.startTime), actionMetaData = ActionMetaData(ActionConfig.ActionType.READ_ONLY.toString(), actualHistory1.actionMetaData!!.startTime, 0, false, 0, 0, null), - stepMetaData = null, + stepMetaData = StepMetaData("set_read_only", actualHistory1.stepMetaData!!.startTime, Step.StepStatus.COMPLETED), policyRetryInfo = PolicyRetryInfoMetaData(false, 0), info = mapOf("message" to SetReadOnlyStep.getSuccessMessage(indexName)) ) diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/coordinator/ManagedIndexCoordinatorIT.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/coordinator/ManagedIndexCoordinatorIT.kt index 975aa6b0e..35fe476e1 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/coordinator/ManagedIndexCoordinatorIT.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/coordinator/ManagedIndexCoordinatorIT.kt @@ -27,11 +27,15 @@ import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagemen import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.action.ForceMergeActionConfig import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.model.action.RolloverActionConfig import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.randomErrorNotification +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.resthandler.RestExplainAction import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.step.forcemerge.WaitForForceMergeStep import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.step.rollover.AttemptRolloverStep import com.amazon.opendistroforelasticsearch.indexmanagement.waitFor +import org.elasticsearch.client.ResponseException import org.elasticsearch.common.xcontent.XContentType +import org.elasticsearch.rest.RestRequest +import org.elasticsearch.rest.RestStatus import java.time.Instant import java.time.temporal.ChronoUnit @@ -54,13 +58,16 @@ class ManagedIndexCoordinatorIT : IndexStateManagementRestTestCase() { createIndex() waitFor { val response = client().makeRequest("GET", "/$INDEX_MANAGEMENT_INDEX/_mapping") - val parserMap = createParser(XContentType.JSON.xContent(), - response.entity.content).map() as Map>> + val parserMap = createParser( + XContentType.JSON.xContent(), + response.entity.content + ).map() as Map>> val mappingsMap = parserMap[INDEX_MANAGEMENT_INDEX]?.getValue("mappings")!! val expected = createParser( XContentType.JSON.xContent(), - javaClass.classLoader.getResource("mappings/opendistro-ism-config.json").readText()) + javaClass.classLoader.getResource("mappings/opendistro-ism-config.json").readText() + ) val expectedMap = expected.map() assertEquals("Mappings are different", expectedMap, mappingsMap) @@ -107,20 +114,78 @@ class ManagedIndexCoordinatorIT : IndexStateManagementRestTestCase() { // Only ManagedIndexSettings.POLICY_ID set to null should be left in explain output waitFor { assertPredicatesOnMetaData( - listOf(index to listOf(ManagedIndexSettings.POLICY_ID.key to fun(policyID: Any?): Boolean = policyID == null)), + listOf(index to listOf(ManagedIndexSettings.POLICY_ID.key to fun(policyID: Any?): Boolean = + policyID == null)), getExplainMap(index), true ) } } + fun `test managed-index metadata is cleaned up after index deleted`() { + val policyID = "some_policy" + val (index) = createIndex(policyID = policyID) + + val managedIndexConfig = getExistingManagedIndexConfig(index) + + // Speed up execution to initialize policy on job + // Loading policy will fail but ManagedIndexMetaData will be updated + updateManagedIndexConfigStartTime(managedIndexConfig) + + // Verify ManagedIndexMetaData contains information + waitFor { + assertPredicatesOnMetaData( + listOf(index to listOf(ManagedIndexMetaData.POLICY_ID to policyID::equals)), + getExplainMap(index), + false + ) + } + + deleteIndex(index) + + // Verify ManagedIndexMetadata has been cleared + waitFor { + try { + client().makeRequest(RestRequest.Method.GET.toString(), RestExplainAction.EXPLAIN_BASE_URI) + fail("Expected a failure") + } catch (e: ResponseException) { + assertEquals("Unexpected RestStatus", RestStatus.NOT_FOUND, e.response.restStatus()) + val actualMessage = e.response.asMap() + val expectedErrorMessage = mapOf( + "error" to mapOf( + "root_cause" to listOf( + mapOf( + "type" to "illegal_argument_exception", "reason" to "Missing indices", + "reason" to "no such index [$index]", + "index_uuid" to "_na_", + "index" to index, + "resource.type" to "index_or_alias", + "type" to "index_not_found_exception", + "resource.id" to index + ) + ), + "type" to "index_not_found_exception", + "reason" to "no such index [$index]", + "index_uuid" to "_na_", + "index" to index, + "resource.type" to "index_or_alias", + "resource.id" to index + ), + "status" to 404 + ) + assertEquals(expectedErrorMessage, actualMessage) + } + } + } + fun `test disabling and reenabling ism`() { val indexName = "test_disable_ism_index-000001" val policyID = "test_policy_1" // Create a policy with one State that performs rollover val rolloverActionConfig = RolloverActionConfig(index = 0, minDocs = 5, minAge = null, minSize = null) - val states = listOf(State(name = "RolloverState", actions = listOf(rolloverActionConfig), transitions = listOf())) + val states = + listOf(State(name = "RolloverState", actions = listOf(rolloverActionConfig), transitions = listOf())) val policy = Policy( id = policyID, description = "$policyID description", @@ -188,10 +253,16 @@ class ManagedIndexCoordinatorIT : IndexStateManagementRestTestCase() { config } - // Speed up to next execution where the job should be rescheduled and the index rolled over + // TODO seen version conflict flaky failure here + // could be same reason as the test failure in ChangePolicyActionIT updateManagedIndexConfigStartTime(enabledManagedIndexConfig) - waitFor { assertEquals(AttemptRolloverStep.getSuccessMessage(indexName), getExplainManagedIndexMetaData(indexName).info?.get("message")) } + waitFor { + assertEquals( + AttemptRolloverStep.getSuccessMessage(indexName), + getExplainManagedIndexMetaData(indexName).info?.get("message") + ) + } } fun `test not disabling ism on unsafe step`() { @@ -226,7 +297,12 @@ class ManagedIndexCoordinatorIT : IndexStateManagementRestTestCase() { // Add sample data to increase segment count, passing in a delay to ensure multiple segments get created insertSampleData(indexName, 3, 1000) - waitFor { assertTrue("Segment count for [$indexName] was less than expected", validateSegmentCount(indexName, min = 2)) } + waitFor { + assertTrue( + "Segment count for [$indexName] was less than expected", + validateSegmentCount(indexName, min = 2) + ) + } val managedIndexConfig = getExistingManagedIndexConfig(indexName) @@ -262,21 +338,32 @@ class ManagedIndexCoordinatorIT : IndexStateManagementRestTestCase() { updateManagedIndexConfigStartTime(managedIndexConfig) // Confirm we successfully executed the WaitForForceMergeStep - waitFor { assertEquals(WaitForForceMergeStep.getSuccessMessage(indexName), - getExplainManagedIndexMetaData(indexName).info?.get("message")) } + waitFor { + assertEquals( + WaitForForceMergeStep.getSuccessMessage(indexName), + getExplainManagedIndexMetaData(indexName).info?.get("message") + ) + } // Confirm job was not disabled assertEquals("ManagedIndexConfig was disabled early", true, getExistingManagedIndexConfig(indexName).enabled) // Validate segments were merged - assertTrue("Segment count for [$indexName] after force merge is incorrect", validateSegmentCount(indexName, min = 1, max = 1)) + assertTrue( + "Segment count for [$indexName] after force merge is incorrect", + validateSegmentCount(indexName, min = 1, max = 1) + ) // Fifth execution: Attempt transition, which is safe to disable on, so job should be disabled updateManagedIndexConfigStartTime(managedIndexConfig) // Explain API info should still be that of the last executed Step - waitFor { assertEquals(WaitForForceMergeStep.getSuccessMessage(indexName), - getExplainManagedIndexMetaData(indexName).info?.get("message")) } + waitFor { + assertEquals( + WaitForForceMergeStep.getSuccessMessage(indexName), + getExplainManagedIndexMetaData(indexName).info?.get("message") + ) + } // Confirm job was disabled val disabledManagedIndexConfig: ManagedIndexConfig = waitFor { diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/coordinator/ManagedIndexCoordinatorTests.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/coordinator/ManagedIndexCoordinatorTests.kt index 68dda157a..ac9169545 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/coordinator/ManagedIndexCoordinatorTests.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/coordinator/ManagedIndexCoordinatorTests.kt @@ -17,6 +17,7 @@ package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanageme import com.amazon.opendistroforelasticsearch.indexmanagement.IndexManagementIndices import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.ManagedIndexCoordinator +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.MetadataService import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings import org.elasticsearch.Version import org.elasticsearch.client.Client @@ -45,6 +46,7 @@ class ManagedIndexCoordinatorTests : ESAllocationTestCase() { private lateinit var settings: Settings private lateinit var indexManagementIndices: IndexManagementIndices + private lateinit var metadataService: MetadataService private lateinit var coordinator: ManagedIndexCoordinator private lateinit var discoveryNode: DiscoveryNode @@ -55,6 +57,7 @@ class ManagedIndexCoordinatorTests : ESAllocationTestCase() { client = Mockito.mock(Client::class.java) threadPool = Mockito.mock(ThreadPool::class.java) indexManagementIndices = Mockito.mock(IndexManagementIndices::class.java) + metadataService = Mockito.mock(MetadataService::class.java) val namedXContentRegistryEntries = arrayListOf() xContentRegistry = NamedXContentRegistry(namedXContentRegistryEntries) @@ -68,6 +71,7 @@ class ManagedIndexCoordinatorTests : ESAllocationTestCase() { settingSet.add(ManagedIndexSettings.SWEEP_PERIOD) settingSet.add(ManagedIndexSettings.JOB_INTERVAL) settingSet.add(ManagedIndexSettings.INDEX_STATE_MANAGEMENT_ENABLED) + settingSet.add(ManagedIndexSettings.METADATA_SERVICE_ENABLED) settingSet.add(ManagedIndexSettings.COORDINATOR_BACKOFF_COUNT) settingSet.add(ManagedIndexSettings.COORDINATOR_BACKOFF_MILLIS) @@ -75,12 +79,12 @@ class ManagedIndexCoordinatorTests : ESAllocationTestCase() { val originClusterService: ClusterService = ClusterServiceUtils.createClusterService(threadPool, discoveryNode, clusterSettings) clusterService = Mockito.spy(originClusterService) - coordinator = ManagedIndexCoordinator(settings, client, clusterService, threadPool, indexManagementIndices) + coordinator = ManagedIndexCoordinator(settings, client, clusterService, threadPool, indexManagementIndices, metadataService) } fun `test after start`() { coordinator.afterStart() - Mockito.verify(threadPool).scheduleWithFixedDelay(Mockito.any(), Mockito.any(), Mockito.anyString()) + Mockito.verify(threadPool, Mockito.times(2)).scheduleWithFixedDelay(Mockito.any(), Mockito.any(), Mockito.anyString()) } fun `test before stop`() { @@ -97,7 +101,7 @@ class ManagedIndexCoordinatorTests : ESAllocationTestCase() { fun `test on master`() { coordinator.onMaster() - Mockito.verify(threadPool).scheduleWithFixedDelay(Mockito.any(), Mockito.any(), Mockito.anyString()) + Mockito.verify(threadPool, Mockito.times(2)).scheduleWithFixedDelay(Mockito.any(), Mockito.any(), Mockito.anyString()) } fun `test off master`() { diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/coordinator/SkipExecutionTests.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/coordinator/SkipExecutionTests.kt new file mode 100644 index 000000000..0d90fff57 --- /dev/null +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/coordinator/SkipExecutionTests.kt @@ -0,0 +1,47 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.coordinator + +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.SkipExecution +import org.elasticsearch.action.admin.cluster.node.info.NodesInfoAction +import org.elasticsearch.client.Client +import org.elasticsearch.cluster.ClusterChangedEvent +import org.elasticsearch.cluster.ESAllocationTestCase +import org.elasticsearch.cluster.service.ClusterService +import org.junit.Before +import org.mockito.Mockito + +class SkipExecutionTests : ESAllocationTestCase() { + + private lateinit var client: Client + private lateinit var clusterService: ClusterService + private lateinit var skip: SkipExecution + + @Before + @Throws(Exception::class) + fun setup() { + client = Mockito.mock(Client::class.java) + clusterService = Mockito.mock(ClusterService::class.java) + skip = SkipExecution(client, clusterService) + } + + fun `test cluster change event`() { + val event = Mockito.mock(ClusterChangedEvent::class.java) + Mockito.`when`(event.nodesChanged()).thenReturn(true) + skip.clusterChanged(event) + Mockito.verify(client).execute(Mockito.eq(NodesInfoAction.INSTANCE), Mockito.any(), Mockito.any()) + } +} \ No newline at end of file diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/IndexStateManagementRestApiIT.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/IndexStateManagementRestApiIT.kt index 6fd9d9675..e94bd9693 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/IndexStateManagementRestApiIT.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/IndexStateManagementRestApiIT.kt @@ -47,25 +47,25 @@ class IndexStateManagementRestApiIT : IndexStateManagementRestTestCase() { fun `test plugins are loaded`() { val response = entityAsMap(client().makeRequest("GET", "_nodes/plugins")) val nodesInfo = response["nodes"] as Map> - var hasIndexStateMangementPlugin = false + var hasIndexStateManagementPlugin = false var hasJobSchedulerPlugin = false for (nodeInfo in nodesInfo.values) { val plugins = nodeInfo["plugins"] as List> for (plugin in plugins) { if (plugin["name"] == "opendistro-index-management") { - hasIndexStateMangementPlugin = true + hasIndexStateManagementPlugin = true } if (plugin["name"] == "opendistro-job-scheduler") { hasJobSchedulerPlugin = true } } - if (hasIndexStateMangementPlugin && hasJobSchedulerPlugin) { + if (hasIndexStateManagementPlugin && hasJobSchedulerPlugin) { return } } - fail("Plugins not installed, ISMPlugin loaded: $hasIndexStateMangementPlugin, JobScheduler loaded: $hasJobSchedulerPlugin") + fail("Plugins not installed, ISMPlugin loaded: $hasIndexStateManagementPlugin, JobScheduler loaded: $hasJobSchedulerPlugin") } @Throws(Exception::class) diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestChangePolicyActionIT.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestChangePolicyActionIT.kt index 1036a9925..d0f0828ea 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestChangePolicyActionIT.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestChangePolicyActionIT.kt @@ -194,7 +194,7 @@ class RestChangePolicyActionIT : IndexStateManagementRestTestCase() { assertNull("Policy has already initialized", managedIndexConfig.policy) assertEquals("Policy id does not match", policy.id, managedIndexConfig.policyID) - // if we try to change policy now, it'll have no ManagedIndexMetaData yet and should succeed + // If we try to change the policy now, it hasn't actually run and has no ManagedIndexMetaData yet so it should succeed val changePolicy = ChangePolicy(newPolicy.id, null, emptyList(), false) val response = client().makeRequest(RestRequest.Method.POST.toString(), "${RestChangePolicyAction.CHANGE_POLICY_BASE_URI}/$index", emptyMap(), changePolicy.toHttpEntity()) @@ -206,10 +206,14 @@ class RestChangePolicyActionIT : IndexStateManagementRestTestCase() { // speed up to first execution where we initialize the policy on the job updateManagedIndexConfigStartTime(managedIndexConfig) - waitFor { assertEquals(newPolicy.id, getManagedIndexConfigByDocId(managedIndexConfig.id)?.policyID) } + val updatedManagedIndexConfig = waitFor { + // TODO: get by docID could get older version of the doc which could cause flaky failure + val config = getManagedIndexConfigByDocId(managedIndexConfig.id) + assertEquals(newPolicy.id, config?.policyID) + config + } // The initialized policy should be the change policy one - val updatedManagedIndexConfig = getManagedIndexConfigByDocId(managedIndexConfig.id) assertNotNull("Updated managed index config is null", updatedManagedIndexConfig) assertNull("Updated change policy is not null", updatedManagedIndexConfig!!.changePolicy) assertEquals("Initialized policyId is not the change policy id", newPolicy.id, updatedManagedIndexConfig.policyID) diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestExplainActionIT.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestExplainActionIT.kt index 62e4b0f84..abe90cbb3 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestExplainActionIT.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestExplainActionIT.kt @@ -81,14 +81,14 @@ class RestExplainActionIT : IndexStateManagementRestTestCase() { createIndex(indexName2, null) val expected = mapOf( - indexName1 to mapOf( - ManagedIndexSettings.POLICY_ID.key to policy.id, - "index" to indexName1, - "index_uuid" to getUuid(indexName1), - "policy_id" to policy.id, - "enabled" to true - ), - "total_managed_indices" to 1 + indexName1 to mapOf( + ManagedIndexSettings.POLICY_ID.key to policy.id, + "index" to indexName1, + "index_uuid" to getUuid(indexName1), + "policy_id" to policy.id, + "enabled" to true + ), + "total_managed_indices" to 1 ) waitFor { assertResponseMap(expected, getExplainMap(null)) @@ -147,9 +147,12 @@ class RestExplainActionIT : IndexStateManagementRestTestCase() { ManagedIndexMetaData.POLICY_SEQ_NO to policy.seqNo.toInt()::equals, ManagedIndexMetaData.POLICY_PRIMARY_TERM to policy.primaryTerm.toInt()::equals, StateMetaData.STATE to fun(stateMetaDataMap: Any?): Boolean = - assertStateEquals(StateMetaData(policy.defaultState, Instant.now().toEpochMilli()), stateMetaDataMap), + assertStateEquals( + StateMetaData(policy.defaultState, Instant.now().toEpochMilli()), + stateMetaDataMap + ), PolicyRetryInfoMetaData.RETRY_INFO to fun(retryInfoMetaDataMap: Any?): Boolean = - assertRetryInfoEquals(PolicyRetryInfoMetaData(false, 0), retryInfoMetaDataMap), + assertRetryInfoEquals(PolicyRetryInfoMetaData(false, 0), retryInfoMetaDataMap), ManagedIndexMetaData.INFO to fun(info: Any?): Boolean = expectedInfoString == info.toString() ) ), getExplainMap(indexName)) @@ -175,7 +178,7 @@ class RestExplainActionIT : IndexStateManagementRestTestCase() { ManagedIndexMetaData.INDEX_UUID to managedIndexConfig.indexUuid::equals, ManagedIndexMetaData.POLICY_ID to managedIndexConfig.policyID::equals, PolicyRetryInfoMetaData.RETRY_INFO to fun(retryInfoMetaDataMap: Any?): Boolean = - assertRetryInfoEquals(PolicyRetryInfoMetaData(true, 0), retryInfoMetaDataMap), + assertRetryInfoEquals(PolicyRetryInfoMetaData(true, 0), retryInfoMetaDataMap), ManagedIndexMetaData.INFO to fun(info: Any?): Boolean = expectedInfoString == info.toString() ) ), getExplainMap(indexName)) diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestRetryFailedManagedIndexActionIT.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestRetryFailedManagedIndexActionIT.kt index 490d3817e..3d7f07535 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestRetryFailedManagedIndexActionIT.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/resthandler/RestRetryFailedManagedIndexActionIT.kt @@ -85,7 +85,7 @@ class RestRetryFailedManagedIndexActionIT : IndexStateManagementRestTestCase() { mapOf( "index_name" to indexName1, "index_uuid" to getUuid(indexName1), - "reason" to "There is no IndexMetaData information" + "reason" to "This index has no metadata information" ) ) ) @@ -107,6 +107,7 @@ class RestRetryFailedManagedIndexActionIT : IndexStateManagementRestTestCase() { "${RestRetryFailedManagedIndexAction.RETRY_BASE_URI}/$indexName*" ) assertEquals("Unexpected RestStatus", RestStatus.OK, response.restStatus()) + val actualMessage = response.asMap() val expectedErrorMessage = mapOf( FAILURES to true, @@ -125,7 +126,7 @@ class RestRetryFailedManagedIndexActionIT : IndexStateManagementRestTestCase() { mapOf( "index_name" to indexName2, "index_uuid" to getUuid(indexName2), - "reason" to "There is no IndexMetaData information" + "reason" to "This index has no metadata information" ) ) ) @@ -158,6 +159,7 @@ class RestRetryFailedManagedIndexActionIT : IndexStateManagementRestTestCase() { fun `test index has no metadata`() { val indexName = "${testIndexName}_players" createIndex(indexName, "somePolicy") + val response = client().makeRequest( RestRequest.Method.POST.toString(), "${RestRetryFailedManagedIndexAction.RETRY_BASE_URI}/$indexName" @@ -171,7 +173,7 @@ class RestRetryFailedManagedIndexActionIT : IndexStateManagementRestTestCase() { mapOf( "index_name" to indexName, "index_uuid" to getUuid(indexName), - "reason" to "There is no IndexMetaData information" + "reason" to "This index has no metadata information" ) ) ) diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/runner/ManagedIndexRunnerTests.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/runner/ManagedIndexRunnerTests.kt new file mode 100644 index 000000000..debce70ce --- /dev/null +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/indexstatemanagement/runner/ManagedIndexRunnerTests.kt @@ -0,0 +1,92 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.runner + +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.IndexStateManagementHistory +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.ManagedIndexRunner +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.SkipExecution +import com.amazon.opendistroforelasticsearch.indexmanagement.indexstatemanagement.settings.ManagedIndexSettings +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope +import org.elasticsearch.Version +import org.elasticsearch.action.index.IndexResponse +import org.elasticsearch.client.Client +import org.elasticsearch.cluster.node.DiscoveryNode +import org.elasticsearch.cluster.service.ClusterService +import org.elasticsearch.common.settings.ClusterSettings +import org.elasticsearch.common.settings.Setting +import org.elasticsearch.common.settings.Settings +import org.elasticsearch.common.xcontent.NamedXContentRegistry +import org.elasticsearch.env.Environment +import org.elasticsearch.script.ScriptService +import org.elasticsearch.test.ClusterServiceUtils +import org.elasticsearch.test.ESTestCase +import org.elasticsearch.threadpool.ThreadPool +import org.junit.Before +import org.mockito.Mockito + +@ThreadLeakScope(ThreadLeakScope.Scope.NONE) +class ManagedIndexRunnerTests : ESTestCase() { + + private lateinit var client: Client + private lateinit var clusterService: ClusterService + private lateinit var xContentRegistry: NamedXContentRegistry + private lateinit var scriptService: ScriptService + private lateinit var environment: Environment + private lateinit var indexStateManagementHistory: IndexStateManagementHistory + private lateinit var skipFlag: SkipExecution + private lateinit var runner: ManagedIndexRunner + + private lateinit var settings: Settings + private lateinit var discoveryNode: DiscoveryNode + private lateinit var threadPool: ThreadPool + + private lateinit var indexResponse: IndexResponse + + @Before + @Throws(Exception::class) + fun setup() { + clusterService = Mockito.mock(ClusterService::class.java) + xContentRegistry = Mockito.mock(NamedXContentRegistry::class.java) + scriptService = Mockito.mock(ScriptService::class.java) + environment = Mockito.mock(Environment::class.java) + indexStateManagementHistory = Mockito.mock(IndexStateManagementHistory::class.java) + skipFlag = Mockito.mock(SkipExecution::class.java) + + threadPool = Mockito.mock(ThreadPool::class.java) + settings = Settings.builder().build() + discoveryNode = DiscoveryNode("node", buildNewFakeTransportAddress(), Version.CURRENT) + val settingSet = hashSetOf>() + settingSet.addAll(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + settingSet.add(ManagedIndexSettings.SWEEP_PERIOD) + settingSet.add(ManagedIndexSettings.JOB_INTERVAL) + settingSet.add(ManagedIndexSettings.INDEX_STATE_MANAGEMENT_ENABLED) + settingSet.add(ManagedIndexSettings.ALLOW_LIST) + val clusterSettings = ClusterSettings(settings, settingSet) + val originClusterService: ClusterService = ClusterServiceUtils.createClusterService(threadPool, discoveryNode, clusterSettings) + clusterService = Mockito.spy(originClusterService) + + Mockito.`when`(environment.settings()).thenReturn(settings) + + runner = ManagedIndexRunner + .registerClusterService(clusterService) + .registerNamedXContentRegistry(xContentRegistry) + .registerScriptService(scriptService) + .registerSettings(environment.settings()) + .registerConsumers() + .registerHistoryIndex(indexStateManagementHistory) + .registerSkipFlag(skipFlag) + } +} diff --git a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/rollup/resthandler/RestGetRollupActionIT.kt b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/rollup/resthandler/RestGetRollupActionIT.kt index 9dcf387ec..9b8a2b7a8 100644 --- a/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/rollup/resthandler/RestGetRollupActionIT.kt +++ b/src/test/kotlin/com/amazon/opendistroforelasticsearch/indexmanagement/rollup/resthandler/RestGetRollupActionIT.kt @@ -74,7 +74,7 @@ class RestGetRollupActionIT : RollupRestTestCase() { assertEquals(testRollup.primaryTerm, (foundRollup["_primary_term"] as Int).toLong()) assertEquals(testRollup.id, innerRollup["rollup_id"] as String) // Doesn't matter what rollup sets, current system is at schema version 5 - assertEquals(8, (innerRollup["schema_version"] as Int).toLong()) + assertEquals(configSchemaVersion, innerRollup["schema_version"] as Int) assertEquals(testRollup.enabled, innerRollup["enabled"] as Boolean) assertEquals(testRollup.enabledTime?.toEpochMilli(), (innerRollup["enabled_time"] as Number?)?.toLong()) // Last updated time will never be correct as it gets updated in the API call diff --git a/src/test/resources/index-management/opendistro-index-management-1.13.0.0.zip b/src/test/resources/index-management/opendistro-index-management-1.13.0.0.zip new file mode 100644 index 000000000..1a09c750d Binary files /dev/null and b/src/test/resources/index-management/opendistro-index-management-1.13.0.0.zip differ diff --git a/src/test/resources/mappings/cached-opendistro-ism-config.json b/src/test/resources/mappings/cached-opendistro-ism-config.json index 1f099d85b..6a96bf9eb 100644 --- a/src/test/resources/mappings/cached-opendistro-ism-config.json +++ b/src/test/resources/mappings/cached-opendistro-ism-config.json @@ -1,6 +1,6 @@ { "_meta" : { - "schema_version": 8 + "schema_version": 9 }, "dynamic": "strict", "properties": { @@ -543,6 +543,143 @@ } } }, + "managed_index_metadata": { + "properties": { + "index": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "index_uuid": { + "type": "keyword" + }, + "policy_id": { + "type": "keyword" + }, + "policy_seq_no": { + "type": "long" + }, + "policy_primary_term": { + "type": "long" + }, + "policy_completed": { + "type": "boolean" + }, + "rolled_over": { + "type": "boolean" + }, + "transition_to": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "state": { + "properties": { + "name": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + } + } + }, + "action": { + "properties": { + "name": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "index": { + "type": "integer" + }, + "failed": { + "type": "boolean" + }, + "consumed_retries": { + "type": "integer" + }, + "last_retry_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "action_properties": { + "properties": { + "max_num_segments": { + "type": "integer" + }, + "snapshot_name": { + "type": "keyword" + }, + "rollup_id": { + "type": "keyword" + }, + "has_rollup_failed": { + "type": "boolean" + } + } + } + } + }, + "step": { + "properties": { + "name": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "step_status": { + "type": "keyword" + } + } + }, + "retry_info": { + "properties": { + "failed": { + "type": "boolean" + }, + "consumed_retries": { + "type": "integer" + } + } + }, + "info": { + "type": "object", + "enabled": false + } + } + }, "rollup": { "properties": { "rollup_id": {