From 19c67d68c242083d112500bf462ab9dd7fe6304f Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Fri, 22 Nov 2019 11:31:59 -0500 Subject: [PATCH 01/15] [ML] Add graceful retry for bulk index results failures --- .../ml/integration/BulkFailureRetryIT.java | 214 ++++++++++++++++++ .../xpack/ml/MachineLearning.java | 4 +- .../job/persistence/JobResultsPersister.java | 35 ++- .../job/persistence/TimingStatsReporter.java | 6 +- .../ml/job/process/DataCountsReporter.java | 11 +- .../autodetect/AutodetectProcessManager.java | 11 +- .../output/AutodetectResultProcessor.java | 80 ++++++- .../writer/JsonDataToProcessWriter.java | 2 +- .../AutodetectResultProcessorIT.java | 3 +- .../ml/integration/EstablishedMemUsageIT.java | 2 +- .../persistence/JobResultsPersisterTests.java | 13 +- .../persistence/TimingStatsReporterTests.java | 8 +- .../AutodetectProcessManagerTests.java | 6 +- .../AutodetectResultProcessorTests.java | 132 +++++++++-- 14 files changed, 472 insertions(+), 55 deletions(-) create mode 100644 x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java new file mode 100644 index 0000000000000..2a771130cac70 --- /dev/null +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java @@ -0,0 +1,214 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.integration; + +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.xpack.core.action.util.PageParams; +import org.elasticsearch.xpack.core.ml.action.GetBucketsAction; +import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; +import org.elasticsearch.xpack.core.ml.job.config.AnalysisConfig; +import org.elasticsearch.xpack.core.ml.job.config.DataDescription; +import org.elasticsearch.xpack.core.ml.job.config.Detector; +import org.elasticsearch.xpack.core.ml.job.config.Job; +import org.elasticsearch.xpack.core.ml.job.results.Bucket; +import org.elasticsearch.xpack.core.ml.job.results.Result; +import org.junit.After; +import org.junit.Before; + +import java.util.Collections; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; + +import static org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase.createDatafeedBuilder; +import static org.hamcrest.Matchers.greaterThan; + +public class BulkFailureRetryIT extends MlNativeAutodetectIntegTestCase { + + private String index = "bulk-failure-retry"; + private long now = System.currentTimeMillis(); + private String jobId = "bulk-failure-retry-job"; + private String resultsIndex = ".ml-anomalies-custom-bulk-failure-retry-job"; + + @Before + public void putPastDataIntoIndex() { + client().admin().indices().prepareCreate(index) + .addMapping("type", "time", "type=date", "value", "type=long") + .get(); + long oneDayAgo = now - 86400000; + long twoDaysAgo = oneDayAgo - 86400000; + writeData(logger, index, 128, twoDaysAgo, oneDayAgo); + } + + @After + public void cleanUpTest() { + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder() + .putNull("logger.org.elasticsearch.xpack.ml") + .putNull("xpack.ml.persist_results_max_retries") + .build()).get(); + cleanUp(); + } + + private void ensureAnomaliesWrite() throws InterruptedException { + Settings settings = Settings.builder().put(IndexMetaData.INDEX_READ_ONLY_SETTING.getKey(), false).build(); + AtomicReference acknowledgedResponseHolder = new AtomicReference<>(); + AtomicReference exceptionHolder = new AtomicReference<>(); + blockingCall( + listener -> client().admin().indices().prepareUpdateSettings(resultsIndex).setSettings(settings).execute(listener), + acknowledgedResponseHolder, + exceptionHolder); + if (exceptionHolder.get() != null) { + logger.error("FAILED TO MARK ["+ resultsIndex + "] as read-write again", exceptionHolder.get()); + } + } + + private void setAnomaliesReadOnlyBlock() throws InterruptedException { + Settings settings = Settings.builder().put(IndexMetaData.INDEX_READ_ONLY_SETTING.getKey(), true).build(); + AtomicReference acknowledgedResponseHolder = new AtomicReference<>(); + AtomicReference exceptionHolder = new AtomicReference<>(); + blockingCall( + listener -> client().admin().indices().prepareUpdateSettings(resultsIndex).setSettings(settings).execute(listener), + acknowledgedResponseHolder, + exceptionHolder); + if (exceptionHolder.get() != null) { + logger.error("FAILED TO MARK ["+ resultsIndex + "] as read-write again", exceptionHolder.get()); + } + } + + public void testBulkFailureRetries() throws Exception { + Job.Builder job = createJob(jobId, TimeValue.timeValueMinutes(5), "count", null); + job.setResultsIndexName(jobId); + + DatafeedConfig.Builder datafeedConfigBuilder = + createDatafeedBuilder(job.getId() + "-datafeed", job.getId(), Collections.singletonList(index)); + DatafeedConfig datafeedConfig = datafeedConfigBuilder.build(); + registerJob(job); + putJob(job); + openJob(job.getId()); + registerDatafeed(datafeedConfig); + putDatafeed(datafeedConfig); + startDatafeed(datafeedConfig.getId(), 0L, now - 86400000); + waitUntilJobIsClosed(jobId); + + // Get the job stats + Bucket initialLatestBucket = getLatestFinalizedBucket(jobId); + assertThat(initialLatestBucket.getEpoch(), greaterThan(0L)); + + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder() + .put("logger.org.elasticsearch.xpack.ml", "TRACE") + .put("xpack.ml.persist_results_max_retries", "10000") + .build()).get(); + + setAnomaliesReadOnlyBlock(); + + int moreDocs = 128; + long oneDayAgo = now - 86400000; + writeData(logger, index, moreDocs, oneDayAgo, now); + + openJob(job.getId()); + startDatafeed(datafeedConfig.getId(), oneDayAgo, now); + + // TODO Any better way????? + Thread.sleep(1000); + ensureAnomaliesWrite(); + waitUntilJobIsClosed(jobId); + + Bucket newLatestBucket = getLatestFinalizedBucket(jobId); + assertThat(newLatestBucket.getEpoch(), greaterThan(initialLatestBucket.getEpoch())); + } + + private Job.Builder createJob(String id, TimeValue bucketSpan, String function, String field) { + return createJob(id, bucketSpan, function, field, null); + } + + private Job.Builder createJob(String id, TimeValue bucketSpan, String function, String field, String summaryCountField) { + DataDescription.Builder dataDescription = new DataDescription.Builder(); + dataDescription.setFormat(DataDescription.DataFormat.XCONTENT); + dataDescription.setTimeField("time"); + dataDescription.setTimeFormat(DataDescription.EPOCH_MS); + + Detector.Builder d = new Detector.Builder(function, field); + AnalysisConfig.Builder analysisConfig = new AnalysisConfig.Builder(Collections.singletonList(d.build())); + analysisConfig.setBucketSpan(bucketSpan); + analysisConfig.setSummaryCountFieldName(summaryCountField); + + Job.Builder builder = new Job.Builder(); + builder.setId(id); + builder.setAnalysisConfig(analysisConfig); + builder.setDataDescription(dataDescription); + return builder; + } + + private void writeData(Logger logger, String index, long numDocs, long start, long end) { + int maxDelta = (int) (end - start - 1); + BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); + for (int i = 0; i < numDocs; i++) { + IndexRequest indexRequest = new IndexRequest(index); + long timestamp = start + randomIntBetween(0, maxDelta); + assert timestamp >= start && timestamp < end; + indexRequest.source("time", timestamp, "value", i); + bulkRequestBuilder.add(indexRequest); + } + BulkResponse bulkResponse = bulkRequestBuilder + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .get(); + if (bulkResponse.hasFailures()) { + int failures = 0; + for (BulkItemResponse itemResponse : bulkResponse) { + if (itemResponse.isFailed()) { + failures++; + logger.error("Item response failure [{}]", itemResponse.getFailureMessage()); + } + } + fail("Bulk response contained " + failures + " failures"); + } + logger.info("Indexed [{}] documents", numDocs); + } + + private Bucket getLatestFinalizedBucket(String jobId) { + GetBucketsAction.Request getBucketsRequest = new GetBucketsAction.Request(jobId); + getBucketsRequest.setExcludeInterim(true); + getBucketsRequest.setSort(Result.TIMESTAMP.getPreferredName()); + getBucketsRequest.setDescending(true); + getBucketsRequest.setPageParams(new PageParams(0, 1)); + return getBuckets(getBucketsRequest).get(0); + } + + private void blockingCall(Consumer> function, + AtomicReference response, + AtomicReference error) throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + ActionListener listener = ActionListener.wrap( + r -> { + response.set(r); + latch.countDown(); + }, + e -> { + error.set(e); + latch.countDown(); + } + ); + + function.accept(listener); + latch.await(); + } +} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java index 76442812ef256..4a4fd458b15a2 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java @@ -228,6 +228,7 @@ import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManager; import org.elasticsearch.xpack.ml.job.process.autodetect.BlackHoleAutodetectProcess; import org.elasticsearch.xpack.ml.job.process.autodetect.NativeAutodetectProcessFactory; +import org.elasticsearch.xpack.ml.job.process.autodetect.output.AutodetectResultProcessor; import org.elasticsearch.xpack.ml.job.process.normalizer.MultiplyingNormalizerProcess; import org.elasticsearch.xpack.ml.job.process.normalizer.NativeNormalizerProcessFactory; import org.elasticsearch.xpack.ml.job.process.normalizer.NormalizerFactory; @@ -446,7 +447,8 @@ public List> getSettings() { MlConfigMigrationEligibilityCheck.ENABLE_CONFIG_MIGRATION, InferenceProcessor.MAX_INFERENCE_PROCESSORS, ModelLoadingService.INFERENCE_MODEL_CACHE_SIZE, - ModelLoadingService.INFERENCE_MODEL_CACHE_TTL + ModelLoadingService.INFERENCE_MODEL_CACHE_TTL, + AutodetectResultProcessor.PERSIST_RESULTS_MAX_RETRIES ); } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java index 783706259a17b..d1f717d488f06 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java @@ -97,7 +97,7 @@ private Builder(String jobId) { * @param bucket The bucket to persist * @return this */ - public Builder persistBucket(Bucket bucket) { + public Builder persistBucket(Bucket bucket) throws BulkIndexException { // If the supplied bucket has records then create a copy with records // removed, because we never persist nested records in buckets Bucket bucketWithoutRecords = bucket; @@ -114,7 +114,7 @@ public Builder persistBucket(Bucket bucket) { return this; } - private void persistBucketInfluencersStandalone(String jobId, List bucketInfluencers) { + private void persistBucketInfluencersStandalone(String jobId, List bucketInfluencers) throws BulkIndexException { if (bucketInfluencers != null && bucketInfluencers.isEmpty() == false) { for (BucketInfluencer bucketInfluencer : bucketInfluencers) { String id = bucketInfluencer.getId(); @@ -130,7 +130,7 @@ private void persistBucketInfluencersStandalone(String jobId, List records) { + public Builder persistRecords(List records) throws BulkIndexException { for (AnomalyRecord record : records) { logger.trace("[{}] ES BULK ACTION: index record to index [{}] with ID [{}]", jobId, indexName, record.getId()); indexResult(record.getId(), record, "record"); @@ -161,7 +161,7 @@ public Builder persistRecords(List records) { * @param influencers the influencers to persist * @return this */ - public Builder persistInfluencers(List influencers) { + public Builder persistInfluencers(List influencers) throws BulkIndexException { for (Influencer influencer : influencers) { logger.trace("[{}] ES BULK ACTION: index influencer to index [{}] with ID [{}]", jobId, indexName, influencer.getId()); indexResult(influencer.getId(), influencer, "influencer"); @@ -170,30 +170,30 @@ public Builder persistInfluencers(List influencers) { return this; } - public Builder persistModelPlot(ModelPlot modelPlot) { + public Builder persistModelPlot(ModelPlot modelPlot) throws BulkIndexException { logger.trace("[{}] ES BULK ACTION: index model plot to index [{}] with ID [{}]", jobId, indexName, modelPlot.getId()); indexResult(modelPlot.getId(), modelPlot, "model plot"); return this; } - public Builder persistForecast(Forecast forecast) { + public Builder persistForecast(Forecast forecast) throws BulkIndexException { logger.trace("[{}] ES BULK ACTION: index forecast to index [{}] with ID [{}]", jobId, indexName, forecast.getId()); indexResult(forecast.getId(), forecast, Forecast.RESULT_TYPE_VALUE); return this; } - public Builder persistForecastRequestStats(ForecastRequestStats forecastRequestStats) { + public Builder persistForecastRequestStats(ForecastRequestStats forecastRequestStats) throws BulkIndexException { logger.trace("[{}] ES BULK ACTION: index forecast request stats to index [{}] with ID [{}]", jobId, indexName, forecastRequestStats.getId()); indexResult(forecastRequestStats.getId(), forecastRequestStats, Forecast.RESULT_TYPE_VALUE); return this; } - private void indexResult(String id, ToXContent resultDoc, String resultType) { + private void indexResult(String id, ToXContent resultDoc, String resultType) throws BulkIndexException { indexResult(id, resultDoc, ToXContent.EMPTY_PARAMS, resultType); } - private void indexResult(String id, ToXContent resultDoc, ToXContent.Params params, String resultType) { + private void indexResult(String id, ToXContent resultDoc, ToXContent.Params params, String resultType) throws BulkIndexException { try (XContentBuilder content = toXContentBuilder(resultDoc, params)) { bulkRequest.add(new IndexRequest(indexName).id(id).source(content)); } catch (IOException e) { @@ -208,7 +208,7 @@ private void indexResult(String id, ToXContent resultDoc, ToXContent.Params para /** * Execute the bulk action */ - public void executeRequest() { + public void executeRequest() throws BulkIndexException { if (bulkRequest.numberOfActions() == 0) { return; } @@ -218,6 +218,7 @@ public void executeRequest() { BulkResponse addRecordsResponse = client.bulk(bulkRequest).actionGet(); if (addRecordsResponse.hasFailures()) { logger.error("[{}] Bulk index of results has errors: {}", jobId, addRecordsResponse.buildFailureMessage()); + throw new BulkIndexException(addRecordsResponse); } } @@ -411,4 +412,16 @@ private void logCall(String indexName) { } } } + + public static class BulkIndexException extends Exception { + + public BulkIndexException(String msg) { + super(msg); + } + + public BulkIndexException(BulkResponse bulkResponse) { + this(bulkResponse.buildFailureMessage()); + } + + } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java index 69d6936e9051f..8ac184db61b4b 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java @@ -36,7 +36,7 @@ public TimingStats getCurrentTimingStats() { return new TimingStats(currentTimingStats); } - public void reportBucket(Bucket bucket) { + public void reportBucket(Bucket bucket) throws JobResultsPersister.BulkIndexException { currentTimingStats.updateStats(bucket.getProcessingTimeMs()); currentTimingStats.setLatestRecordTimestamp(bucket.getTimestamp().toInstant().plusSeconds(bucket.getBucketSpan())); if (differSignificantly(currentTimingStats, persistedTimingStats)) { @@ -44,7 +44,7 @@ public void reportBucket(Bucket bucket) { } } - public void finishReporting() { + public void finishReporting() throws JobResultsPersister.BulkIndexException { // Don't flush if current timing stats are identical to the persisted ones if (currentTimingStats.equals(persistedTimingStats)) { return; @@ -52,7 +52,7 @@ public void finishReporting() { flush(); } - private void flush() { + private void flush() throws JobResultsPersister.BulkIndexException { persistedTimingStats = new TimingStats(currentTimingStats); bulkResultsPersister.persistTimingStats(persistedTimingStats); } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporter.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporter.java index dff118011b147..b00ae2816068f 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporter.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporter.java @@ -7,6 +7,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.xpack.core.ml.job.config.Job; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.DataCounts; @@ -235,7 +236,15 @@ public void finishReporting(ActionListener listener) { totalRecordStats.setLastDataTimeStamp(now); diagnostics.flush(); retrieveDiagnosticsIntermediateResults(); - dataCountsPersister.persistDataCounts(job.getId(), runningTotalStats(), listener); + dataCountsPersister.persistDataCounts(job.getId(), runningTotalStats(), ActionListener.wrap( + listener::onResponse, + e -> { + // Recording data counts should not cause the job processing to fail. + // Log the failure and move on. + logger.warn(() -> new ParameterizedMessage("[{}] failed to record data counts", job.getId()), e); + listener.onResponse(true); + } + )); } /** diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java index 9ef733198114e..27a79f4aa2ab1 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java @@ -112,6 +112,7 @@ public class AutodetectProcessManager implements ClusterStateListener { private final AnomalyDetectionAuditor auditor; private volatile boolean upgradeInProgress; + private volatile int maximumBulkFailureRetries; public AutodetectProcessManager(Environment environment, Settings settings, Client client, ThreadPool threadPool, NamedXContentRegistry xContentRegistry, AnomalyDetectionAuditor auditor, ClusterService clusterService, @@ -131,15 +132,22 @@ public AutodetectProcessManager(Environment environment, Settings settings, Clie this.jobDataCountsPersister = jobDataCountsPersister; this.auditor = auditor; this.nativeStorageProvider = Objects.requireNonNull(nativeStorageProvider); + this.maximumBulkFailureRetries = AutodetectResultProcessor.PERSIST_RESULTS_MAX_RETRIES.get(settings); clusterService.addListener(this); clusterService.getClusterSettings() .addSettingsUpdateConsumer(MachineLearning.MAX_OPEN_JOBS_PER_NODE, this::setMaxAllowedRunningJobs); + clusterService.getClusterSettings() + .addSettingsUpdateConsumer(AutodetectResultProcessor.PERSIST_RESULTS_MAX_RETRIES, this::setMaximumBulkFailureRetries); } void setMaxAllowedRunningJobs(int maxAllowedRunningJobs) { this.maxAllowedRunningJobs = maxAllowedRunningJobs; } + void setMaximumBulkFailureRetries(int maximumBulkFailureRetries) { + this.maximumBulkFailureRetries = maximumBulkFailureRetries; + } + public synchronized void closeAllJobsOnThisNode(String reason) { int numJobs = processByAllocation.size(); if (numJobs != 0) { @@ -519,7 +527,8 @@ AutodetectCommunicator create(JobTask jobTask, Job job, AutodetectParams autodet jobResultsPersister, process, autodetectParams.modelSizeStats(), - autodetectParams.timingStats()); + autodetectParams.timingStats(), + maximumBulkFailureRetries); ExecutorService autodetectWorkerExecutor; try (ThreadContext.StoredContext ignore = threadPool.getThreadContext().stashContext()) { autodetectWorkerExecutor = createAutodetectExecutorService(autodetectExecutorService); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java index c9441e9f60c39..610acebfa9604 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java @@ -8,12 +8,14 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.Client; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.xpack.core.ml.MachineLearningField; @@ -71,6 +73,14 @@ */ public class AutodetectResultProcessor { + public static final Setting PERSIST_RESULTS_MAX_RETRIES = Setting.intSetting( + "xpack.ml.persist_results_max_retries", + 2, + 0, + Integer.MAX_VALUE - 2, + Setting.Property.Dynamic, + Setting.Property.NodeScope); + private static final Logger LOGGER = LogManager.getLogger(AutodetectResultProcessor.class); private final Client client; @@ -86,6 +96,7 @@ public class AutodetectResultProcessor { private final FlushListener flushListener; private volatile boolean processKilled; private volatile boolean failed; + private final int maximumFailureRetries; private int bucketCount; // only used from the process() thread, so doesn't need to be volatile private final JobResultsPersister.Builder bulkResultsPersister; private boolean deleteInterimRequired; @@ -102,15 +113,16 @@ public AutodetectResultProcessor(Client client, JobResultsPersister persister, AutodetectProcess process, ModelSizeStats latestModelSizeStats, - TimingStats timingStats) { - this(client, auditor, jobId, renormalizer, persister, process, latestModelSizeStats, timingStats, new FlushListener()); + TimingStats timingStats, + int maximumFailureRetries) { + this(client, auditor, jobId, renormalizer, persister, process, latestModelSizeStats, timingStats, new FlushListener(), + maximumFailureRetries); } // Visible for testing AutodetectResultProcessor(Client client, AnomalyDetectionAuditor auditor, String jobId, Renormalizer renormalizer, JobResultsPersister persister, AutodetectProcess autodetectProcess, ModelSizeStats latestModelSizeStats, - TimingStats timingStats, - FlushListener flushListener) { + TimingStats timingStats, FlushListener flushListener, int maximumFailureRetries) { this.client = Objects.requireNonNull(client); this.auditor = Objects.requireNonNull(auditor); this.jobId = Objects.requireNonNull(jobId); @@ -122,6 +134,7 @@ public AutodetectResultProcessor(Client client, this.bulkResultsPersister = persister.bulkPersisterBuilder(jobId); this.timingStatsReporter = new TimingStatsReporter(timingStats, bulkResultsPersister); this.deleteInterimRequired = true; + this.maximumFailureRetries = maximumFailureRetries; } public void process() { @@ -137,6 +150,8 @@ public void process() { timingStatsReporter.finishReporting(); bulkResultsPersister.executeRequest(); } + } catch (JobResultsPersister.BulkIndexException e){ + bulkPersistWithRetry(); } catch (Exception e) { LOGGER.warn(new ParameterizedMessage("[{}] Error persisting autodetect results", jobId), e); } @@ -176,11 +191,15 @@ private void readResults() { if (result.getBucket() != null) { LOGGER.trace("[{}] Bucket number {} parsed from output", jobId, bucketCount); } - } catch (Exception e) { - if (processKilled) { - throw e; + } catch (JobResultsPersister.BulkIndexException e) { + // Don't throw on bulk failures, just continue + if (isDeadOrDying()) { + continue; } - if (process.isProcessAliveAfterWaiting() == false) { + // attempt to retry if possible + bulkPersistWithRetry(); + } catch (Exception e) { + if (isDeadOrDying()) { throw e; } LOGGER.warn(new ParameterizedMessage("[{}] Error processing autodetect result", jobId), e); @@ -196,7 +215,7 @@ public void setProcessKilled() { renormalizer.shutdown(); } - void processResult(AutodetectResult result) { + void processResult(AutodetectResult result) throws JobResultsPersister.BulkIndexException { if (processKilled) { return; } @@ -310,6 +329,45 @@ void processResult(AutodetectResult result) { } } + void bulkPersistWithRetry() { + int attempts = 0; + while(attempts < maximumFailureRetries) { + try { + bulkResultsPersister.executeRequest(); + return; + } catch (JobResultsPersister.BulkIndexException ex) { + if (isDeadOrDying()) { + return; + } + final int currentAttempt = attempts; + LOGGER.trace( + () -> new ParameterizedMessage("[{}] bulk persist failure on attempt [{}] ", jobId, currentAttempt), + ex + ); + attempts++; + try { + double backOff = ((1 << attempts) - 1) / 2.0; + Thread.sleep((int)(backOff * 100)); + } catch (InterruptedException interrupt) { + LOGGER.warn( + () -> new ParameterizedMessage("[{}] failed bulk indexing of results after [{}] attempts", jobId, currentAttempt), + ex + ); + return; + } + } catch (Exception e) { + if (isDeadOrDying()) { + throw e; + } + LOGGER.warn(new ParameterizedMessage("[{}] Error processing autodetect result", jobId), e); + } + } + LOGGER.warn( + new ParameterizedMessage("[{}] failed bulk indexing of results after [{}] attempts", + jobId, + attempts)); + } + private void processModelSizeStats(ModelSizeStats modelSizeStats) { LOGGER.trace("[{}] Parsed ModelSizeStats: {} / {} / {} / {} / {} / {}", jobId, modelSizeStats.getModelBytes(), modelSizeStats.getTotalByFieldCount(), @@ -435,6 +493,10 @@ boolean isDeleteInterimRequired() { return deleteInterimRequired; } + private boolean isDeadOrDying() { + return processKilled || (process.isProcessAliveAfterWaiting() == false); + } + void setDeleteInterimRequired(boolean deleteInterimRequired) { this.deleteInterimRequired = deleteInterimRequired; } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriter.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriter.java index 92fe2c3b0b50a..902fbb775f474 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriter.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriter.java @@ -53,7 +53,7 @@ class JsonDataToProcessWriter extends AbstractDataToProcessWriter { * the OutputStream. No transformation is applied to the data the timestamp * is expected in seconds from the epoch. If any of the fields in * analysisFields or the DataDescriptions - * timeField is missing from the JOSN inputIndex an exception is thrown + * timeField is missing from the JSON inputIndex an exception is thrown */ @Override public void write(InputStream inputStream, CategorizationAnalyzer categorizationAnalyzer, XContentType xContentType, diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java index f6ec2fc9b89f1..703c1bb78916c 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java @@ -100,7 +100,8 @@ public void createComponents() throws Exception { new JobResultsPersister(client()), process, new ModelSizeStats.Builder(JOB_ID).build(), - new TimingStats(JOB_ID)) { + new TimingStats(JOB_ID), + 2) { @Override protected void updateModelSnapshotOnJob(ModelSnapshot modelSnapshot) { capturedUpdateModelSnapshotOnJobRequests.add(modelSnapshot); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java index a026d5d6c337b..d16164681adc3 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java @@ -221,7 +221,7 @@ private void initClusterAndJob(String jobId) { client().execute(PutJobAction.INSTANCE, putJobRequest).actionGet(); } - private void createBuckets(String jobId, int count) { + private void createBuckets(String jobId, int count) throws Exception { JobResultsPersister.Builder builder = jobResultsPersister.bulkPersisterBuilder(jobId); for (int i = 1; i <= count; ++i) { Bucket bucket = new Bucket(jobId, new Date(bucketSpan * i), bucketSpan); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java index 09e5cf9ce6331..913f297b9c1fe 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java @@ -29,7 +29,6 @@ import org.elasticsearch.xpack.core.ml.utils.ExponentialAverageCalculationContext; import org.mockito.ArgumentCaptor; -import java.io.IOException; import java.time.Instant; import java.util.ArrayList; import java.util.Collections; @@ -51,7 +50,7 @@ public class JobResultsPersisterTests extends ESTestCase { private static final String JOB_ID = "foo"; - public void testPersistBucket_OneRecord() throws IOException { + public void testPersistBucket_OneRecord() throws Exception { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); Bucket bucket = new Bucket("foo", new Date(), 123456); @@ -94,7 +93,7 @@ public void testPersistBucket_OneRecord() throws IOException { assertTrue(s.matches(".*raw_anomaly_score.:19\\.19.*")); } - public void testPersistRecords() throws IOException { + public void testPersistRecords() throws Exception { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); @@ -149,7 +148,7 @@ public void testPersistRecords() throws IOException { assertTrue(s.matches(".*over_field_value.:.overValue.*")); } - public void testPersistInfluencers() throws IOException { + public void testPersistInfluencers() throws Exception { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); @@ -173,7 +172,7 @@ public void testPersistInfluencers() throws IOException { assertTrue(s.matches(".*influencer_score.:16\\.0.*")); } - public void testExecuteRequest_ClearsBulkRequest() { + public void testExecuteRequest_ClearsBulkRequest() throws Exception { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); JobResultsPersister persister = new JobResultsPersister(client); @@ -190,7 +189,7 @@ public void testExecuteRequest_ClearsBulkRequest() { assertEquals(0, builder.getBulkRequest().numberOfActions()); } - public void testBulkRequestExecutesWhenReachMaxDocs() { + public void testBulkRequestExecutesWhenReachMaxDocs() throws Exception { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); JobResultsPersister persister = new JobResultsPersister(client); @@ -206,7 +205,7 @@ public void testBulkRequestExecutesWhenReachMaxDocs() { verifyNoMoreInteractions(client); } - public void testPersistTimingStats() { + public void testPersistTimingStats() throws Exception { ArgumentCaptor bulkRequestCaptor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(bulkRequestCaptor); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporterTests.java index d8a4d5aec24b2..124b0f7433d6f 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporterTests.java @@ -46,7 +46,7 @@ public void testGetCurrentTimingStats() { verifyZeroInteractions(bulkResultsPersister); } - public void testReporting() { + public void testReporting() throws Exception { TimingStatsReporter reporter = createReporter(new TimingStats(JOB_ID)); assertThat(reporter.getCurrentTimingStats(), equalTo(new TimingStats(JOB_ID))); @@ -65,7 +65,7 @@ public void testReporting() { verifyNoMoreInteractions(bulkResultsPersister); } - public void testFinishReporting() { + public void testFinishReporting() throws Exception { TimingStatsReporter reporter = createReporter(new TimingStats(JOB_ID)); assertThat(reporter.getCurrentTimingStats(), equalTo(new TimingStats(JOB_ID))); @@ -87,14 +87,14 @@ public void testFinishReporting() { verifyNoMoreInteractions(bulkResultsPersister); } - public void testFinishReporting_NoChange() { + public void testFinishReporting_NoChange() throws Exception { TimingStatsReporter reporter = createReporter(new TimingStats(JOB_ID)); reporter.finishReporting(); verifyZeroInteractions(bulkResultsPersister); } - public void testFinishReporting_WithChange() { + public void testFinishReporting_WithChange() throws Exception { TimingStatsReporter reporter = createReporter(new TimingStats(JOB_ID)); reporter.reportBucket(createBucket(10)); reporter.finishReporting(); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java index 72bdf45a96c4a..be858570a56b2 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java @@ -47,6 +47,7 @@ import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister; import org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister; import org.elasticsearch.xpack.ml.job.persistence.JobResultsProvider; +import org.elasticsearch.xpack.ml.job.process.autodetect.output.AutodetectResultProcessor; import org.elasticsearch.xpack.ml.job.process.autodetect.params.AutodetectParams; import org.elasticsearch.xpack.ml.job.process.autodetect.params.DataLoadParams; import org.elasticsearch.xpack.ml.job.process.autodetect.params.FlushJobParams; @@ -61,6 +62,7 @@ import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collections; import java.util.Date; import java.util.HashSet; @@ -151,7 +153,9 @@ public void setup() throws Exception { auditor = mock(AnomalyDetectionAuditor.class); clusterService = mock(ClusterService.class); ClusterSettings clusterSettings = - new ClusterSettings(Settings.EMPTY, Collections.singleton(MachineLearning.MAX_OPEN_JOBS_PER_NODE)); + new ClusterSettings(Settings.EMPTY, + new HashSet<>(Arrays.asList(MachineLearning.MAX_OPEN_JOBS_PER_NODE, + AutodetectResultProcessor.PERSIST_RESULTS_MAX_RETRIES))); when(clusterService.getClusterSettings()).thenReturn(clusterSettings); MetaData metaData = mock(MetaData.class); SortedMap aliasOrIndexSortedMap = new TreeMap<>(); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java index 63ca73444b540..aad2b557fe170 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java @@ -107,7 +107,8 @@ public void setUpMocks() { process, new ModelSizeStats.Builder(JOB_ID).setTimestamp(new Date(BUCKET_SPAN_MS)).build(), new TimingStats(JOB_ID), - flushListener); + flushListener, + 2); } @After @@ -130,7 +131,7 @@ public void testProcess() throws TimeoutException { verify(persister).commitStateWrites(JOB_ID); } - public void testProcessResult_bucket() { + public void testProcessResult_bucket() throws Exception { when(bulkBuilder.persistTimingStats(any(TimingStats.class))).thenReturn(bulkBuilder); when(bulkBuilder.persistBucket(any(Bucket.class))).thenReturn(bulkBuilder); AutodetectResult result = mock(AutodetectResult.class); @@ -147,7 +148,7 @@ public void testProcessResult_bucket() { verify(persister, never()).deleteInterimResults(JOB_ID); } - public void testProcessResult_bucket_deleteInterimRequired() { + public void testProcessResult_bucket_deleteInterimRequired() throws Exception { when(bulkBuilder.persistTimingStats(any(TimingStats.class))).thenReturn(bulkBuilder); when(bulkBuilder.persistBucket(any(Bucket.class))).thenReturn(bulkBuilder); AutodetectResult result = mock(AutodetectResult.class); @@ -164,7 +165,7 @@ public void testProcessResult_bucket_deleteInterimRequired() { verify(persister).deleteInterimResults(JOB_ID); } - public void testProcessResult_records() { + public void testProcessResult_records() throws Exception { AutodetectResult result = mock(AutodetectResult.class); List records = Arrays.asList( @@ -180,7 +181,7 @@ public void testProcessResult_records() { verify(persister).bulkPersisterBuilder(JOB_ID); } - public void testProcessResult_influencers() { + public void testProcessResult_influencers() throws Exception { AutodetectResult result = mock(AutodetectResult.class); List influencers = Arrays.asList( @@ -196,7 +197,7 @@ public void testProcessResult_influencers() { verify(persister).bulkPersisterBuilder(JOB_ID); } - public void testProcessResult_categoryDefinition() { + public void testProcessResult_categoryDefinition() throws Exception { AutodetectResult result = mock(AutodetectResult.class); CategoryDefinition categoryDefinition = mock(CategoryDefinition.class); when(result.getCategoryDefinition()).thenReturn(categoryDefinition); @@ -209,7 +210,7 @@ public void testProcessResult_categoryDefinition() { verify(persister).bulkPersisterBuilder(JOB_ID); } - public void testProcessResult_flushAcknowledgement() { + public void testProcessResult_flushAcknowledgement() throws Exception { AutodetectResult result = mock(AutodetectResult.class); FlushAcknowledgement flushAcknowledgement = mock(FlushAcknowledgement.class); when(flushAcknowledgement.getId()).thenReturn(JOB_ID); @@ -225,7 +226,7 @@ public void testProcessResult_flushAcknowledgement() { verify(bulkBuilder).executeRequest(); } - public void testProcessResult_flushAcknowledgementMustBeProcessedLast() { + public void testProcessResult_flushAcknowledgementMustBeProcessedLast() throws Exception { AutodetectResult result = mock(AutodetectResult.class); FlushAcknowledgement flushAcknowledgement = mock(FlushAcknowledgement.class); when(flushAcknowledgement.getId()).thenReturn(JOB_ID); @@ -245,7 +246,7 @@ public void testProcessResult_flushAcknowledgementMustBeProcessedLast() { inOrder.verify(flushListener).acknowledgeFlush(flushAcknowledgement, null); } - public void testProcessResult_modelPlot() { + public void testProcessResult_modelPlot() throws Exception { AutodetectResult result = mock(AutodetectResult.class); ModelPlot modelPlot = mock(ModelPlot.class); when(result.getModelPlot()).thenReturn(modelPlot); @@ -257,7 +258,7 @@ public void testProcessResult_modelPlot() { verify(bulkBuilder).persistModelPlot(modelPlot); } - public void testProcessResult_modelSizeStats() { + public void testProcessResult_modelSizeStats() throws Exception { AutodetectResult result = mock(AutodetectResult.class); ModelSizeStats modelSizeStats = mock(ModelSizeStats.class); when(result.getModelSizeStats()).thenReturn(modelSizeStats); @@ -270,7 +271,7 @@ public void testProcessResult_modelSizeStats() { verify(persister).persistModelSizeStats(modelSizeStats); } - public void testProcessResult_modelSizeStatsWithMemoryStatusChanges() { + public void testProcessResult_modelSizeStatsWithMemoryStatusChanges() throws Exception { TimeValue delay = TimeValue.timeValueSeconds(5); // Set up schedule delay time when(threadPool.schedule(any(Runnable.class), any(TimeValue.class), anyString())) @@ -310,7 +311,7 @@ public void testProcessResult_modelSizeStatsWithMemoryStatusChanges() { verify(auditor).error(JOB_ID, Messages.getMessage(Messages.JOB_AUDIT_MEMORY_STATUS_HARD_LIMIT, "512mb", "1kb")); } - public void testProcessResult_modelSnapshot() { + public void testProcessResult_modelSnapshot() throws Exception { AutodetectResult result = mock(AutodetectResult.class); ModelSnapshot modelSnapshot = new ModelSnapshot.Builder(JOB_ID) .setSnapshotId("a_snapshot_id") @@ -333,7 +334,7 @@ public void testProcessResult_modelSnapshot() { verify(client).execute(same(UpdateJobAction.INSTANCE), eq(expectedJobUpdateRequest), any()); } - public void testProcessResult_quantiles_givenRenormalizationIsEnabled() { + public void testProcessResult_quantiles_givenRenormalizationIsEnabled() throws Exception { AutodetectResult result = mock(AutodetectResult.class); Quantiles quantiles = mock(Quantiles.class); when(result.getQuantiles()).thenReturn(quantiles); @@ -350,7 +351,7 @@ public void testProcessResult_quantiles_givenRenormalizationIsEnabled() { verify(renormalizer).renormalize(quantiles); } - public void testProcessResult_quantiles_givenRenormalizationIsDisabled() { + public void testProcessResult_quantiles_givenRenormalizationIsDisabled() throws Exception { AutodetectResult result = mock(AutodetectResult.class); Quantiles quantiles = mock(Quantiles.class); when(result.getQuantiles()).thenReturn(quantiles); @@ -433,4 +434,107 @@ public void testKill() throws TimeoutException { verify(renormalizer).waitUntilIdle(); verify(flushListener).clear(); } + + public void testBulkPersistWithRetry() throws Exception { + JobResultsPersister persister = mock(JobResultsPersister.class); + JobResultsPersister.Builder bulkBuilder = mock(JobResultsPersister.Builder.class); + when(process.isProcessAlive()).thenReturn(true); + when(process.isProcessAliveAfterWaiting()).thenReturn(true); + doThrow(new JobResultsPersister.BulkIndexException("boom")) + .when(bulkBuilder).executeRequest(); + when(persister.bulkPersisterBuilder(JOB_ID)).thenReturn(bulkBuilder); + AutodetectResultProcessor processorUnderTest = new AutodetectResultProcessor( + client, + auditor, + JOB_ID, + renormalizer, + persister, + process, + new ModelSizeStats.Builder(JOB_ID).setTimestamp(new Date(BUCKET_SPAN_MS)).build(), + new TimingStats(JOB_ID), + flushListener, + 2); + + processorUnderTest.bulkPersistWithRetry(); + + // Hax as memory persister needs to be verified before @After + verify(this.persister).bulkPersisterBuilder(JOB_ID); + verify(bulkBuilder, times(2)).executeRequest(); + } + + public void testBulkPersistWithRetry_withDeadProcess() throws Exception { + JobResultsPersister persister = mock(JobResultsPersister.class); + JobResultsPersister.Builder bulkBuilder = mock(JobResultsPersister.Builder.class); + when(process.isProcessAliveAfterWaiting()).thenReturn(true); + doThrow(new JobResultsPersister.BulkIndexException("boom")) + .when(bulkBuilder).executeRequest(); + when(persister.bulkPersisterBuilder(JOB_ID)).thenReturn(bulkBuilder); + AutodetectResultProcessor processorUnderTest = new AutodetectResultProcessor( + client, + auditor, + JOB_ID, + renormalizer, + persister, + process, + new ModelSizeStats.Builder(JOB_ID).setTimestamp(new Date(BUCKET_SPAN_MS)).build(), + new TimingStats(JOB_ID), + flushListener, + 2); + + processorUnderTest.setProcessKilled(); + processorUnderTest.bulkPersistWithRetry(); + verify(bulkBuilder, times(1)).executeRequest(); + // Hax as memory persister needs to be verified before @After + verify(this.persister).bulkPersisterBuilder(JOB_ID); + verify(this.renormalizer).shutdown(); + } + + public void testBulkPersistWithRetry_withDyingProcess() throws Exception { + JobResultsPersister persister = mock(JobResultsPersister.class); + JobResultsPersister.Builder bulkBuilder = mock(JobResultsPersister.Builder.class); + when(process.isProcessAliveAfterWaiting()).thenReturn(false); + doThrow(new JobResultsPersister.BulkIndexException("boom")) + .when(bulkBuilder).executeRequest(); + when(persister.bulkPersisterBuilder(JOB_ID)).thenReturn(bulkBuilder); + AutodetectResultProcessor processorUnderTest = new AutodetectResultProcessor( + client, + auditor, + JOB_ID, + renormalizer, + persister, + process, + new ModelSizeStats.Builder(JOB_ID).setTimestamp(new Date(BUCKET_SPAN_MS)).build(), + new TimingStats(JOB_ID), + flushListener, + 2); + + processorUnderTest.bulkPersistWithRetry(); + verify(bulkBuilder, times(1)).executeRequest(); + // Hax as memory persister needs to be verified before @After + verify(this.persister).bulkPersisterBuilder(JOB_ID); + } + + public void testBulkPersistWithRetry_withExceptionOtherThanBulkFailure() throws Exception { + JobResultsPersister persister = mock(JobResultsPersister.class); + JobResultsPersister.Builder bulkBuilder = mock(JobResultsPersister.Builder.class); + when(process.isProcessAliveAfterWaiting()).thenReturn(false); + doThrow(new ElasticsearchException("boom")) + .when(bulkBuilder).executeRequest(); + when(persister.bulkPersisterBuilder(JOB_ID)).thenReturn(bulkBuilder); + AutodetectResultProcessor processorUnderTest = new AutodetectResultProcessor( + client, + auditor, + JOB_ID, + renormalizer, + persister, + process, + new ModelSizeStats.Builder(JOB_ID).setTimestamp(new Date(BUCKET_SPAN_MS)).build(), + new TimingStats(JOB_ID), + flushListener, + 2); + + expectThrows(ElasticsearchException.class, processorUnderTest::bulkPersistWithRetry); + // Hax as memory persister needs to be verified before @After + verify(this.persister).bulkPersisterBuilder(JOB_ID); + } } From 1877d4ea0557873f5dc621c0101b84ba8a62fb9b Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Mon, 25 Nov 2019 09:09:34 -0500 Subject: [PATCH 02/15] fixing test, removing unused import --- .../ml/integration/BulkFailureRetryIT.java | 27 +++++++++++-------- .../output/AutodetectResultProcessor.java | 1 - 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java index 2a771130cac70..cfd478ef24b18 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java @@ -40,6 +40,7 @@ public class BulkFailureRetryIT extends MlNativeAutodetectIntegTestCase { private String index = "bulk-failure-retry"; private long now = System.currentTimeMillis(); + private static long DAY = 86400000; private String jobId = "bulk-failure-retry-job"; private String resultsIndex = ".ml-anomalies-custom-bulk-failure-retry-job"; @@ -48,9 +49,9 @@ public void putPastDataIntoIndex() { client().admin().indices().prepareCreate(index) .addMapping("type", "time", "type=date", "value", "type=long") .get(); - long oneDayAgo = now - 86400000; - long twoDaysAgo = oneDayAgo - 86400000; - writeData(logger, index, 128, twoDaysAgo, oneDayAgo); + long twoDaysAgo = now - DAY * 2; + long threeDaysAgo = now - DAY * 3; + writeData(logger, index, 250, threeDaysAgo, twoDaysAgo); } @After @@ -59,8 +60,10 @@ public void cleanUpTest() { .cluster() .prepareUpdateSettings() .setTransientSettings(Settings.builder() - .putNull("logger.org.elasticsearch.xpack.ml") .putNull("xpack.ml.persist_results_max_retries") + .putNull("logger.org.elasticsearch.xpack.ml.datafeed.DatafeedJob") + .putNull("logger.org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister") + .putNull("logger.org.elasticsearch.xpack.ml.job.process.autodetect.output") .build()).get(); cleanUp(); } @@ -87,7 +90,7 @@ private void setAnomaliesReadOnlyBlock() throws InterruptedException { acknowledgedResponseHolder, exceptionHolder); if (exceptionHolder.get() != null) { - logger.error("FAILED TO MARK ["+ resultsIndex + "] as read-write again", exceptionHolder.get()); + logger.error("FAILED TO MARK ["+ resultsIndex + "] as read-ONLY", exceptionHolder.get()); } } @@ -103,7 +106,8 @@ public void testBulkFailureRetries() throws Exception { openJob(job.getId()); registerDatafeed(datafeedConfig); putDatafeed(datafeedConfig); - startDatafeed(datafeedConfig.getId(), 0L, now - 86400000); + long twoDaysAgo = now - 2 * DAY; + startDatafeed(datafeedConfig.getId(), 0L, twoDaysAgo); waitUntilJobIsClosed(jobId); // Get the job stats @@ -114,18 +118,19 @@ public void testBulkFailureRetries() throws Exception { .cluster() .prepareUpdateSettings() .setTransientSettings(Settings.builder() - .put("logger.org.elasticsearch.xpack.ml", "TRACE") + .put("logger.org.elasticsearch.xpack.ml.datafeed.DatafeedJob", "TRACE") + .put("logger.org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister", "TRACE") + .put("logger.org.elasticsearch.xpack.ml.job.process.autodetect.output", "TRACE") .put("xpack.ml.persist_results_max_retries", "10000") .build()).get(); setAnomaliesReadOnlyBlock(); - int moreDocs = 128; - long oneDayAgo = now - 86400000; - writeData(logger, index, moreDocs, oneDayAgo, now); + int moreDocs = 20_000; + writeData(logger, index, moreDocs, twoDaysAgo, now); openJob(job.getId()); - startDatafeed(datafeedConfig.getId(), oneDayAgo, now); + startDatafeed(datafeedConfig.getId(), twoDaysAgo, now); // TODO Any better way????? Thread.sleep(1000); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java index 610acebfa9604..05efba7aa481f 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java @@ -8,7 +8,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexResponse; From 6d59172d17532c59e66dd1c3795bafa7848e5daa Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Mon, 25 Nov 2019 09:43:01 -0500 Subject: [PATCH 03/15] moving retries to individual actions in `processResult` so result processing does not get skipped --- .../ml/integration/BulkFailureRetryIT.java | 4 +- .../output/AutodetectResultProcessor.java | 45 +++++++++---------- .../AutodetectResultProcessorTests.java | 9 ++-- 3 files changed, 25 insertions(+), 33 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java index cfd478ef24b18..aabef5d2acc8c 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java @@ -126,14 +126,12 @@ public void testBulkFailureRetries() throws Exception { setAnomaliesReadOnlyBlock(); - int moreDocs = 20_000; + int moreDocs = 1_000; writeData(logger, index, moreDocs, twoDaysAgo, now); openJob(job.getId()); startDatafeed(datafeedConfig.getId(), twoDaysAgo, now); - // TODO Any better way????? - Thread.sleep(1000); ensureAnomaliesWrite(); waitUntilJobIsClosed(jobId); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java index 05efba7aa481f..e91b1f5b4b23e 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java @@ -13,6 +13,7 @@ import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.Client; +import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -146,11 +147,11 @@ public void process() { try { if (processKilled == false) { - timingStatsReporter.finishReporting(); - bulkResultsPersister.executeRequest(); + bulkPersistWithRetry(() -> { + timingStatsReporter.finishReporting(); + bulkResultsPersister.executeRequest(); + }); } - } catch (JobResultsPersister.BulkIndexException e){ - bulkPersistWithRetry(); } catch (Exception e) { LOGGER.warn(new ParameterizedMessage("[{}] Error persisting autodetect results", jobId), e); } @@ -190,13 +191,6 @@ private void readResults() { if (result.getBucket() != null) { LOGGER.trace("[{}] Bucket number {} parsed from output", jobId, bucketCount); } - } catch (JobResultsPersister.BulkIndexException e) { - // Don't throw on bulk failures, just continue - if (isDeadOrDying()) { - continue; - } - // attempt to retry if possible - bulkPersistWithRetry(); } catch (Exception e) { if (isDeadOrDying()) { throw e; @@ -214,7 +208,7 @@ public void setProcessKilled() { renormalizer.shutdown(); } - void processResult(AutodetectResult result) throws JobResultsPersister.BulkIndexException { + void processResult(AutodetectResult result) { if (processKilled) { return; } @@ -231,18 +225,19 @@ void processResult(AutodetectResult result) throws JobResultsPersister.BulkIndex // persist after deleting interim results in case the new // results are also interim - timingStatsReporter.reportBucket(bucket); - bulkResultsPersister.persistBucket(bucket).executeRequest(); - + bulkPersistWithRetry(() -> { + timingStatsReporter.reportBucket(bucket); + bulkResultsPersister.persistBucket(bucket).executeRequest(); + }); ++bucketCount; } List records = result.getRecords(); if (records != null && !records.isEmpty()) { - bulkResultsPersister.persistRecords(records); + bulkPersistWithRetry(() -> bulkResultsPersister.persistRecords(records)); } List influencers = result.getInfluencers(); if (influencers != null && !influencers.isEmpty()) { - bulkResultsPersister.persistInfluencers(influencers); + bulkPersistWithRetry(() -> bulkResultsPersister.persistInfluencers(influencers)); } CategoryDefinition categoryDefinition = result.getCategoryDefinition(); if (categoryDefinition != null) { @@ -250,16 +245,16 @@ void processResult(AutodetectResult result) throws JobResultsPersister.BulkIndex } ModelPlot modelPlot = result.getModelPlot(); if (modelPlot != null) { - bulkResultsPersister.persistModelPlot(modelPlot); + bulkPersistWithRetry(() -> bulkResultsPersister.persistModelPlot(modelPlot)); } Forecast forecast = result.getForecast(); if (forecast != null) { - bulkResultsPersister.persistForecast(forecast); + bulkPersistWithRetry(() -> bulkResultsPersister.persistForecast(forecast)); } ForecastRequestStats forecastRequestStats = result.getForecastRequestStats(); if (forecastRequestStats != null) { LOGGER.trace("Received Forecast Stats [{}]", forecastRequestStats.getId()); - bulkResultsPersister.persistForecastRequestStats(forecastRequestStats); + bulkPersistWithRetry(() -> bulkResultsPersister.persistForecastRequestStats(forecastRequestStats)); // execute the bulk request only in some cases or in doubt // otherwise rely on the count-based trigger @@ -271,7 +266,7 @@ void processResult(AutodetectResult result) throws JobResultsPersister.BulkIndex case SCHEDULED: case FINISHED: default: - bulkResultsPersister.executeRequest(); + bulkPersistWithRetry(bulkResultsPersister::executeRequest); } } @@ -291,7 +286,7 @@ void processResult(AutodetectResult result) throws JobResultsPersister.BulkIndex if (quantiles != null) { LOGGER.debug("[{}] Parsed Quantiles with timestamp {}", jobId, quantiles.getTimestamp()); persister.persistQuantiles(quantiles); - bulkResultsPersister.executeRequest(); + bulkPersistWithRetry(bulkResultsPersister::executeRequest); if (processKilled == false && renormalizer.isEnabled()) { // We need to make all results written up to these quantiles available for renormalization @@ -308,7 +303,7 @@ void processResult(AutodetectResult result) throws JobResultsPersister.BulkIndex // through to the data store Exception exception = null; try { - bulkResultsPersister.executeRequest(); + bulkPersistWithRetry(bulkResultsPersister::executeRequest); persister.commitResultWrites(jobId); LOGGER.debug("[{}] Flush acknowledgement sent to listener for ID {}", jobId, flushAcknowledgement.getId()); } catch (Exception e) { @@ -328,11 +323,11 @@ void processResult(AutodetectResult result) throws JobResultsPersister.BulkIndex } } - void bulkPersistWithRetry() { + void bulkPersistWithRetry(CheckedRunnable bulkRunnable) { int attempts = 0; while(attempts < maximumFailureRetries) { try { - bulkResultsPersister.executeRequest(); + bulkRunnable.run(); return; } catch (JobResultsPersister.BulkIndexException ex) { if (isDeadOrDying()) { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java index aad2b557fe170..b5a04adc119f8 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java @@ -455,7 +455,7 @@ public void testBulkPersistWithRetry() throws Exception { flushListener, 2); - processorUnderTest.bulkPersistWithRetry(); + processorUnderTest.bulkPersistWithRetry(bulkBuilder::executeRequest); // Hax as memory persister needs to be verified before @After verify(this.persister).bulkPersisterBuilder(JOB_ID); @@ -482,8 +482,7 @@ public void testBulkPersistWithRetry_withDeadProcess() throws Exception { 2); processorUnderTest.setProcessKilled(); - processorUnderTest.bulkPersistWithRetry(); - verify(bulkBuilder, times(1)).executeRequest(); + processorUnderTest.bulkPersistWithRetry(bulkBuilder::executeRequest); // Hax as memory persister needs to be verified before @After verify(this.persister).bulkPersisterBuilder(JOB_ID); verify(this.renormalizer).shutdown(); @@ -508,7 +507,7 @@ public void testBulkPersistWithRetry_withDyingProcess() throws Exception { flushListener, 2); - processorUnderTest.bulkPersistWithRetry(); + processorUnderTest.bulkPersistWithRetry(bulkBuilder::executeRequest); verify(bulkBuilder, times(1)).executeRequest(); // Hax as memory persister needs to be verified before @After verify(this.persister).bulkPersisterBuilder(JOB_ID); @@ -533,7 +532,7 @@ public void testBulkPersistWithRetry_withExceptionOtherThanBulkFailure() throws flushListener, 2); - expectThrows(ElasticsearchException.class, processorUnderTest::bulkPersistWithRetry); + expectThrows(ElasticsearchException.class, () -> processorUnderTest.bulkPersistWithRetry(bulkBuilder::executeRequest)); // Hax as memory persister needs to be verified before @After verify(this.persister).bulkPersisterBuilder(JOB_ID); } From ed4d9a10c8a3a185305c753cfc0cdaab48a3fc72 Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Mon, 25 Nov 2019 09:49:35 -0500 Subject: [PATCH 04/15] clear out bulk request if not able to persist after retrying --- .../xpack/ml/job/persistence/JobResultsPersister.java | 4 ++++ .../process/autodetect/output/AutodetectResultProcessor.java | 1 + 2 files changed, 5 insertions(+) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java index d1f717d488f06..bfd09c60ab4d2 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java @@ -225,6 +225,10 @@ public void executeRequest() throws BulkIndexException { bulkRequest = new BulkRequest(); } + public void clearBulkRequest() { + bulkRequest = new BulkRequest(); + } + // for testing BulkRequest getBulkRequest() { return bulkRequest; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java index e91b1f5b4b23e..4d9058e74024a 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java @@ -356,6 +356,7 @@ void bulkPersistWithRetry(CheckedRunnable Date: Mon, 25 Nov 2019 15:26:32 -0500 Subject: [PATCH 05/15] removing successful bulk response items from subsequent bulk requests --- .../ml/integration/BulkFailureRetryIT.java | 2 +- .../job/persistence/JobResultsPersister.java | 16 ++++ .../output/AutodetectResultProcessor.java | 22 ++++- .../persistence/JobResultsPersisterTests.java | 96 ++++++++++++++++++- .../AutodetectResultProcessorTests.java | 2 +- 5 files changed, 128 insertions(+), 10 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java index aabef5d2acc8c..e25686065db8c 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java @@ -121,7 +121,7 @@ public void testBulkFailureRetries() throws Exception { .put("logger.org.elasticsearch.xpack.ml.datafeed.DatafeedJob", "TRACE") .put("logger.org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister", "TRACE") .put("logger.org.elasticsearch.xpack.ml.job.process.autodetect.output", "TRACE") - .put("xpack.ml.persist_results_max_retries", "10000") + .put("xpack.ml.persist_results_max_retries", "15") .build()).get(); setAnomaliesReadOnlyBlock(); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java index bfd09c60ab4d2..cebbe3d87e8e3 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java @@ -42,8 +42,10 @@ import java.io.IOException; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Objects; +import java.util.Set; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; @@ -217,7 +219,21 @@ public void executeRequest() throws BulkIndexException { try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { BulkResponse addRecordsResponse = client.bulk(bulkRequest).actionGet(); if (addRecordsResponse.hasFailures()) { + // If we failed, lets set the bulkRequest to be a collection of the failed requests + BulkRequest bulkRequestOfFailures = new BulkRequest(); + Set failedDocIds = new HashSet<>(); + addRecordsResponse.forEach(itemResponse -> { + if (itemResponse.isFailed()) { + failedDocIds.add(itemResponse.getId()); + } + }); + bulkRequest.requests().forEach(docWriteRequest -> { + if (failedDocIds.contains(docWriteRequest.id())) { + bulkRequestOfFailures.add(docWriteRequest); + } + }); logger.error("[{}] Bulk index of results has errors: {}", jobId, addRecordsResponse.buildFailureMessage()); + bulkRequest = bulkRequestOfFailures; throw new BulkIndexException(addRecordsResponse); } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java index 4d9058e74024a..7c79984cd3e15 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java @@ -15,6 +15,7 @@ import org.elasticsearch.client.Client; import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.Randomness; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; @@ -46,6 +47,7 @@ import java.util.Iterator; import java.util.List; import java.util.Objects; +import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; @@ -77,12 +79,17 @@ public class AutodetectResultProcessor { "xpack.ml.persist_results_max_retries", 2, 0, - Integer.MAX_VALUE - 2, + 15, Setting.Property.Dynamic, Setting.Property.NodeScope); + private static final int MAX_RETRY_SLEEP_MILLIS = 5 * 1000; + private static final int MIN_RETRY_SLEEP_MILLIS = 50; + private static final Logger LOGGER = LogManager.getLogger(AutodetectResultProcessor.class); + private final Random random = Randomness.get(); + private final Client client; private final AnomalyDetectionAuditor auditor; private final String jobId; @@ -325,7 +332,7 @@ void processResult(AutodetectResult result) { void bulkPersistWithRetry(CheckedRunnable bulkRunnable) { int attempts = 0; - while(attempts < maximumFailureRetries) { + while(attempts <= maximumFailureRetries) { try { bulkRunnable.run(); return; @@ -341,12 +348,19 @@ void bulkPersistWithRetry(CheckedRunnable new ParameterizedMessage("[{}] failed bulk indexing of results after [{}] attempts", jobId, currentAttempt), + () -> new ParameterizedMessage("[{}] failed bulk indexing of results after [{}] attempts due to interrupt", + jobId, + currentAttempt), ex ); + // propagate the interrupt + Thread.currentThread().interrupt(); return; } } catch (Exception e) { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java index 913f297b9c1fe..4fb1fb0912b13 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java @@ -7,6 +7,7 @@ import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; @@ -20,6 +21,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ml.datafeed.DatafeedTimingStats; +import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.TimingStats; import org.elasticsearch.xpack.core.ml.job.results.AnomalyRecord; import org.elasticsearch.xpack.core.ml.job.results.Bucket; @@ -284,15 +286,101 @@ public void testPersistDatafeedTimingStats() { verifyNoMoreInteractions(client); } - @SuppressWarnings({"unchecked"}) + public void testBulkRequesChangeOnFailures() throws Exception { + Bucket bucket = new Bucket("foo", new Date(), 123456); + bucket.setAnomalyScore(99.9); + bucket.setEventCount(57); + bucket.setInitialAnomalyScore(88.8); + bucket.setProcessingTimeMs(8888); + + BucketInfluencer bi = new BucketInfluencer(JOB_ID, new Date(), 600); + bi.setAnomalyScore(14.15); + bi.setInfluencerFieldName("biOne"); + bi.setInitialAnomalyScore(18.12); + bi.setProbability(0.0054); + bi.setRawAnomalyScore(19.19); + bucket.addBucketInfluencer(bi); + + AnomalyRecord record = new AnomalyRecord(JOB_ID, new Date(), 600); + bucket.setRecords(Collections.singletonList(record)); + + ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); + BulkItemResponse failureItem = new BulkItemResponse(0, + DocWriteRequest.OpType.INDEX, + new BulkItemResponse.Failure(AnomalyDetectorsIndex.jobResultsIndexPrefix() + "shared", + bucket.getId(), + new Exception("something"))); + BulkItemResponse successItem = new BulkItemResponse(1, + DocWriteRequest.OpType.INDEX, + new IndexResponse(new ShardId(AnomalyDetectorsIndex.jobResultsIndexPrefix() + "shared", "uuid", 1), + bi.getId(), + 0, + 0, + 1, + true)); + BulkResponse withFailure = new BulkResponse(new BulkItemResponse[]{ failureItem, successItem }, 0L); + Client client = mockClientWithResponse(captor, withFailure, new BulkResponse(new BulkItemResponse[0], 0L)); + + JobResultsPersister persister = new JobResultsPersister(client); + JobResultsPersister.Builder bulkPersister = persister.bulkPersisterBuilder(JOB_ID); + expectThrows(JobResultsPersister.BulkIndexException.class, () -> bulkPersister.persistBucket(bucket).executeRequest()); + BulkRequest bulkRequest = captor.getValue(); + assertEquals(2, bulkRequest.numberOfActions()); + + String s = ((IndexRequest)bulkRequest.requests().get(0)).source().utf8ToString(); + assertTrue(s.matches(".*anomaly_score.:99\\.9.*")); + assertTrue(s.matches(".*initial_anomaly_score.:88\\.8.*")); + assertTrue(s.matches(".*event_count.:57.*")); + assertTrue(s.matches(".*bucket_span.:123456.*")); + assertTrue(s.matches(".*processing_time_ms.:8888.*")); + // There should NOT be any nested records + assertFalse(s.matches(".*records*")); + + s = ((IndexRequest)bulkRequest.requests().get(1)).source().utf8ToString(); + assertTrue(s.matches(".*probability.:0\\.0054.*")); + assertTrue(s.matches(".*influencer_field_name.:.biOne.*")); + assertTrue(s.matches(".*initial_anomaly_score.:18\\.12.*")); + assertTrue(s.matches(".*anomaly_score.:14\\.15.*")); + assertTrue(s.matches(".*raw_anomaly_score.:19\\.19.*")); + + bulkPersister.executeRequest(); + bulkRequest = captor.getValue(); + assertEquals(1, bulkRequest.numberOfActions()); + s = ((IndexRequest)bulkRequest.requests().get(0)).source().utf8ToString(); + assertTrue(s.matches(".*anomaly_score.:99\\.9.*")); + assertTrue(s.matches(".*initial_anomaly_score.:88\\.8.*")); + assertTrue(s.matches(".*event_count.:57.*")); + assertTrue(s.matches(".*bucket_span.:123456.*")); + assertTrue(s.matches(".*processing_time_ms.:8888.*")); + // There should NOT be any nested records + assertFalse(s.matches(".*records*")); + + assertEquals(0, bulkPersister.getBulkRequest().numberOfActions()); + } + private Client mockClient(ArgumentCaptor captor) { + return mockClientWithResponse(captor, new BulkResponse(new BulkItemResponse[0], 0L)); + } + + @SuppressWarnings("unchecked") + private Client mockClientWithResponse(ArgumentCaptor captor, BulkResponse... responses) { Client client = mock(Client.class); ThreadPool threadPool = mock(ThreadPool.class); when(client.threadPool()).thenReturn(threadPool); when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY)); - ActionFuture future = mock(ActionFuture.class); - when(future.actionGet()).thenReturn(new BulkResponse(new BulkItemResponse[0], 0L)); - when(client.bulk(captor.capture())).thenReturn(future); + List> futures = new ArrayList<>(responses.length - 1); + ActionFuture future1 = makeFuture(responses[0]); + for (int i = 1; i < responses.length; i++) { + futures.add(makeFuture(responses[i])); + } + when(client.bulk(captor.capture())).thenReturn(future1, futures.toArray(ActionFuture[]::new)); return client; } + + @SuppressWarnings("unchecked") + private ActionFuture makeFuture(BulkResponse response) { + ActionFuture future = mock(ActionFuture.class); + when(future.actionGet()).thenReturn(response); + return future; + } } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java index b5a04adc119f8..d86540766f986 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java @@ -459,7 +459,7 @@ public void testBulkPersistWithRetry() throws Exception { // Hax as memory persister needs to be verified before @After verify(this.persister).bulkPersisterBuilder(JOB_ID); - verify(bulkBuilder, times(2)).executeRequest(); + verify(bulkBuilder, times(3)).executeRequest(); } public void testBulkPersistWithRetry_withDeadProcess() throws Exception { From 791c570d627b6bb95709569288958eda2d91664d Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Tue, 3 Dec 2019 15:40:48 -0500 Subject: [PATCH 06/15] adding retries for all synchronous persistent calls --- .../xpack/ml/MachineLearning.java | 7 +- .../job/persistence/JobResultsPersister.java | 110 ++++++------ .../job/persistence/TimingStatsReporter.java | 4 +- .../autodetect/AutodetectProcessManager.java | 10 +- .../output/AutodetectResultProcessor.java | 117 +++--------- .../persistence/ResultsPersisterService.java | 162 +++++++++++++++++ .../AutodetectResultProcessorIT.java | 26 ++- .../ml/integration/EstablishedMemUsageIT.java | 29 ++- .../ml/integration/JobResultsProviderIT.java | 32 +++- .../persistence/JobResultsPersisterTests.java | 155 +++++----------- .../AutodetectProcessManagerTests.java | 6 +- .../AutodetectResultProcessorTests.java | 168 ++++-------------- .../ResultsPersisterServiceTests.java | 161 +++++++++++++++++ 13 files changed, 562 insertions(+), 425 deletions(-) create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java index 4a4fd458b15a2..1aa00fbd7cf77 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java @@ -228,7 +228,6 @@ import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManager; import org.elasticsearch.xpack.ml.job.process.autodetect.BlackHoleAutodetectProcess; import org.elasticsearch.xpack.ml.job.process.autodetect.NativeAutodetectProcessFactory; -import org.elasticsearch.xpack.ml.job.process.autodetect.output.AutodetectResultProcessor; import org.elasticsearch.xpack.ml.job.process.normalizer.MultiplyingNormalizerProcess; import org.elasticsearch.xpack.ml.job.process.normalizer.NativeNormalizerProcessFactory; import org.elasticsearch.xpack.ml.job.process.normalizer.NormalizerFactory; @@ -299,6 +298,7 @@ import org.elasticsearch.xpack.ml.rest.results.RestGetRecordsAction; import org.elasticsearch.xpack.ml.rest.validate.RestValidateDetectorAction; import org.elasticsearch.xpack.ml.rest.validate.RestValidateJobConfigAction; +import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import java.io.IOException; import java.math.BigInteger; @@ -448,7 +448,7 @@ public List> getSettings() { InferenceProcessor.MAX_INFERENCE_PROCESSORS, ModelLoadingService.INFERENCE_MODEL_CACHE_SIZE, ModelLoadingService.INFERENCE_MODEL_CACHE_TTL, - AutodetectResultProcessor.PERSIST_RESULTS_MAX_RETRIES + ResultsPersisterService.PERSIST_RESULTS_MAX_RETRIES ); } @@ -522,8 +522,9 @@ public Collection createComponents(Client client, ClusterService cluster DataFrameAnalyticsAuditor dataFrameAnalyticsAuditor = new DataFrameAnalyticsAuditor(client, clusterService.getNodeName()); InferenceAuditor inferenceAuditor = new InferenceAuditor(client, clusterService.getNodeName()); this.dataFrameAnalyticsAuditor.set(dataFrameAnalyticsAuditor); + ResultsPersisterService resultsPersisterService = new ResultsPersisterService(client, clusterService, settings); JobResultsProvider jobResultsProvider = new JobResultsProvider(client, settings); - JobResultsPersister jobResultsPersister = new JobResultsPersister(client); + JobResultsPersister jobResultsPersister = new JobResultsPersister(client, resultsPersisterService); JobDataCountsPersister jobDataCountsPersister = new JobDataCountsPersister(client); JobConfigProvider jobConfigProvider = new JobConfigProvider(client, xContentRegistry); DatafeedConfigProvider datafeedConfigProvider = new DatafeedConfigProvider(client, xContentRegistry); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java index cebbe3d87e8e3..a2b2ef2361957 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java @@ -8,16 +8,16 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteResponse.Result; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; +import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.Client; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -39,13 +39,13 @@ import org.elasticsearch.xpack.core.ml.job.results.Influencer; import org.elasticsearch.xpack.core.ml.job.results.ModelPlot; import org.elasticsearch.xpack.core.ml.utils.ToXContentParams; +import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import java.io.IOException; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Objects; -import java.util.Set; +import java.util.function.Supplier; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; @@ -72,24 +72,30 @@ public class JobResultsPersister { private static final Logger logger = LogManager.getLogger(JobResultsPersister.class); private final Client client; + private final ResultsPersisterService resultsPersisterService; - public JobResultsPersister(Client client) { + public JobResultsPersister(Client client, ResultsPersisterService resultsPersisterService) { this.client = client; + this.resultsPersisterService = resultsPersisterService; } - public Builder bulkPersisterBuilder(String jobId) { - return new Builder(jobId); + public Builder bulkPersisterBuilder(String jobId, Supplier shouldRetry) { + return new Builder(jobId, resultsPersisterService, shouldRetry); } public class Builder { private BulkRequest bulkRequest; private final String jobId; private final String indexName; + private final Supplier shouldRetry; + private final ResultsPersisterService resultsPersisterService; - private Builder(String jobId) { + private Builder(String jobId, ResultsPersisterService resultsPersisterService, Supplier shouldRetry) { this.jobId = Objects.requireNonNull(jobId); indexName = AnomalyDetectorsIndex.resultsWriteAlias(jobId); bulkRequest = new BulkRequest(); + this.shouldRetry = shouldRetry; + this.resultsPersisterService = resultsPersisterService; } /** @@ -99,7 +105,7 @@ private Builder(String jobId) { * @param bucket The bucket to persist * @return this */ - public Builder persistBucket(Bucket bucket) throws BulkIndexException { + public Builder persistBucket(Bucket bucket) { // If the supplied bucket has records then create a copy with records // removed, because we never persist nested records in buckets Bucket bucketWithoutRecords = bucket; @@ -116,7 +122,7 @@ public Builder persistBucket(Bucket bucket) throws BulkIndexException { return this; } - private void persistBucketInfluencersStandalone(String jobId, List bucketInfluencers) throws BulkIndexException { + private void persistBucketInfluencersStandalone(String jobId, List bucketInfluencers) { if (bucketInfluencers != null && bucketInfluencers.isEmpty() == false) { for (BucketInfluencer bucketInfluencer : bucketInfluencers) { String id = bucketInfluencer.getId(); @@ -132,7 +138,7 @@ private void persistBucketInfluencersStandalone(String jobId, List records) throws BulkIndexException { + public Builder persistRecords(List records) { for (AnomalyRecord record : records) { logger.trace("[{}] ES BULK ACTION: index record to index [{}] with ID [{}]", jobId, indexName, record.getId()); indexResult(record.getId(), record, "record"); @@ -163,7 +169,7 @@ public Builder persistRecords(List records) throws BulkIndexExce * @param influencers the influencers to persist * @return this */ - public Builder persistInfluencers(List influencers) throws BulkIndexException { + public Builder persistInfluencers(List influencers) { for (Influencer influencer : influencers) { logger.trace("[{}] ES BULK ACTION: index influencer to index [{}] with ID [{}]", jobId, indexName, influencer.getId()); indexResult(influencer.getId(), influencer, "influencer"); @@ -172,30 +178,30 @@ public Builder persistInfluencers(List influencers) throws BulkIndex return this; } - public Builder persistModelPlot(ModelPlot modelPlot) throws BulkIndexException { + public Builder persistModelPlot(ModelPlot modelPlot) { logger.trace("[{}] ES BULK ACTION: index model plot to index [{}] with ID [{}]", jobId, indexName, modelPlot.getId()); indexResult(modelPlot.getId(), modelPlot, "model plot"); return this; } - public Builder persistForecast(Forecast forecast) throws BulkIndexException { + public Builder persistForecast(Forecast forecast) { logger.trace("[{}] ES BULK ACTION: index forecast to index [{}] with ID [{}]", jobId, indexName, forecast.getId()); indexResult(forecast.getId(), forecast, Forecast.RESULT_TYPE_VALUE); return this; } - public Builder persistForecastRequestStats(ForecastRequestStats forecastRequestStats) throws BulkIndexException { + public Builder persistForecastRequestStats(ForecastRequestStats forecastRequestStats) { logger.trace("[{}] ES BULK ACTION: index forecast request stats to index [{}] with ID [{}]", jobId, indexName, forecastRequestStats.getId()); indexResult(forecastRequestStats.getId(), forecastRequestStats, Forecast.RESULT_TYPE_VALUE); return this; } - private void indexResult(String id, ToXContent resultDoc, String resultType) throws BulkIndexException { + private void indexResult(String id, ToXContent resultDoc, String resultType) { indexResult(id, resultDoc, ToXContent.EMPTY_PARAMS, resultType); } - private void indexResult(String id, ToXContent resultDoc, ToXContent.Params params, String resultType) throws BulkIndexException { + private void indexResult(String id, ToXContent resultDoc, ToXContent.Params params, String resultType) { try (XContentBuilder content = toXContentBuilder(resultDoc, params)) { bulkRequest.add(new IndexRequest(indexName).id(id).source(content)); } catch (IOException e) { @@ -210,38 +216,12 @@ private void indexResult(String id, ToXContent resultDoc, ToXContent.Params para /** * Execute the bulk action */ - public void executeRequest() throws BulkIndexException { + public void executeRequest() { if (bulkRequest.numberOfActions() == 0) { return; } logger.trace("[{}] ES API CALL: bulk request with {} actions", jobId, bulkRequest.numberOfActions()); - - try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { - BulkResponse addRecordsResponse = client.bulk(bulkRequest).actionGet(); - if (addRecordsResponse.hasFailures()) { - // If we failed, lets set the bulkRequest to be a collection of the failed requests - BulkRequest bulkRequestOfFailures = new BulkRequest(); - Set failedDocIds = new HashSet<>(); - addRecordsResponse.forEach(itemResponse -> { - if (itemResponse.isFailed()) { - failedDocIds.add(itemResponse.getId()); - } - }); - bulkRequest.requests().forEach(docWriteRequest -> { - if (failedDocIds.contains(docWriteRequest.id())) { - bulkRequestOfFailures.add(docWriteRequest); - } - }); - logger.error("[{}] Bulk index of results has errors: {}", jobId, addRecordsResponse.buildFailureMessage()); - bulkRequest = bulkRequestOfFailures; - throw new BulkIndexException(addRecordsResponse); - } - } - - bulkRequest = new BulkRequest(); - } - - public void clearBulkRequest() { + resultsPersisterService.bulkIndexWithRetry(bulkRequest, jobId, shouldRetry); bulkRequest = new BulkRequest(); } @@ -256,10 +236,10 @@ BulkRequest getBulkRequest() { * * @param category The category to be persisted */ - public void persistCategoryDefinition(CategoryDefinition category) { + public void persistCategoryDefinition(CategoryDefinition category, Supplier shouldRetry) { Persistable persistable = new Persistable(category.getJobId(), category, category.getId()); - persistable.persist(AnomalyDetectorsIndex.resultsWriteAlias(category.getJobId())).actionGet(); + persistable.persist(AnomalyDetectorsIndex.resultsWriteAlias(category.getJobId()), shouldRetry); // Don't commit as we expect masses of these updates and they're not // read again by this process } @@ -267,9 +247,9 @@ public void persistCategoryDefinition(CategoryDefinition category) { /** * Persist the quantiles (blocking) */ - public void persistQuantiles(Quantiles quantiles) { + public void persistQuantiles(Quantiles quantiles, Supplier shouldRetry) { Persistable persistable = new Persistable(quantiles.getJobId(), quantiles, Quantiles.documentId(quantiles.getJobId())); - persistable.persist(AnomalyDetectorsIndex.jobStateIndexWriteAlias()).actionGet(); + persistable.persist(AnomalyDetectorsIndex.jobStateIndexWriteAlias(), shouldRetry); } /** @@ -284,20 +264,22 @@ public void persistQuantiles(Quantiles quantiles, WriteRequest.RefreshPolicy ref /** * Persist a model snapshot description */ - public IndexResponse persistModelSnapshot(ModelSnapshot modelSnapshot, WriteRequest.RefreshPolicy refreshPolicy) { + public BulkResponse persistModelSnapshot(ModelSnapshot modelSnapshot, + WriteRequest.RefreshPolicy refreshPolicy, + Supplier shouldRetry) { Persistable persistable = new Persistable(modelSnapshot.getJobId(), modelSnapshot, ModelSnapshot.documentId(modelSnapshot)); persistable.setRefreshPolicy(refreshPolicy); - return persistable.persist(AnomalyDetectorsIndex.resultsWriteAlias(modelSnapshot.getJobId())).actionGet(); + return persistable.persist(AnomalyDetectorsIndex.resultsWriteAlias(modelSnapshot.getJobId()), shouldRetry); } /** * Persist the memory usage data (blocking) */ - public void persistModelSizeStats(ModelSizeStats modelSizeStats) { + public void persistModelSizeStats(ModelSizeStats modelSizeStats, Supplier shouldRetry) { String jobId = modelSizeStats.getJobId(); logger.trace("[{}] Persisting model size stats, for size {}", jobId, modelSizeStats.getModelBytes()); Persistable persistable = new Persistable(jobId, modelSizeStats, modelSizeStats.getId()); - persistable.persist(AnomalyDetectorsIndex.resultsWriteAlias(jobId)).actionGet(); + persistable.persist(AnomalyDetectorsIndex.resultsWriteAlias(jobId), shouldRetry); } /** @@ -362,7 +344,7 @@ public void commitStateWrites(String jobId) { * @param timingStats datafeed timing stats to persist * @param refreshPolicy refresh policy to apply */ - public IndexResponse persistDatafeedTimingStats(DatafeedTimingStats timingStats, WriteRequest.RefreshPolicy refreshPolicy) { + public BulkResponse persistDatafeedTimingStats(DatafeedTimingStats timingStats, WriteRequest.RefreshPolicy refreshPolicy) { String jobId = timingStats.getJobId(); logger.trace("[{}] Persisting datafeed timing stats", jobId); Persistable persistable = new Persistable( @@ -371,7 +353,7 @@ public IndexResponse persistDatafeedTimingStats(DatafeedTimingStats timingStats, new ToXContent.MapParams(Collections.singletonMap(ToXContentParams.FOR_INTERNAL_STORAGE, "true")), DatafeedTimingStats.documentId(timingStats.getJobId())); persistable.setRefreshPolicy(refreshPolicy); - return persistable.persist(AnomalyDetectorsIndex.resultsWriteAlias(jobId)).actionGet(); + return persistable.persist(AnomalyDetectorsIndex.resultsWriteAlias(jobId), () -> true); } private static XContentBuilder toXContentBuilder(ToXContent obj, ToXContent.Params params) throws IOException { @@ -404,10 +386,18 @@ void setRefreshPolicy(WriteRequest.RefreshPolicy refreshPolicy) { this.refreshPolicy = refreshPolicy; } - ActionFuture persist(String indexName) { - PlainActionFuture actionFuture = PlainActionFuture.newFuture(); - persist(indexName, actionFuture); - return actionFuture; + BulkResponse persist(String indexName, Supplier shouldRetry) { + logCall(indexName); + try { + return resultsPersisterService.indexWithRetry(jobId, indexName, object, params, refreshPolicy, id, shouldRetry); + } catch (IOException e) { + logger.error(new ParameterizedMessage("[{}] Error writing [{}]", jobId, (id == null) ? "auto-generated ID" : id), e); + IndexResponse.Builder notCreatedResponse = new IndexResponse.Builder(); + notCreatedResponse.setResult(Result.NOOP); + return new BulkResponse( + new BulkItemResponse[]{new BulkItemResponse(0, DocWriteRequest.OpType.INDEX, notCreatedResponse.build())}, + 0); + } } void persist(String indexName, ActionListener listener) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java index 8ac184db61b4b..8b4d0817c9dba 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java @@ -36,7 +36,7 @@ public TimingStats getCurrentTimingStats() { return new TimingStats(currentTimingStats); } - public void reportBucket(Bucket bucket) throws JobResultsPersister.BulkIndexException { + public void reportBucket(Bucket bucket) { currentTimingStats.updateStats(bucket.getProcessingTimeMs()); currentTimingStats.setLatestRecordTimestamp(bucket.getTimestamp().toInstant().plusSeconds(bucket.getBucketSpan())); if (differSignificantly(currentTimingStats, persistedTimingStats)) { @@ -52,7 +52,7 @@ public void finishReporting() throws JobResultsPersister.BulkIndexException { flush(); } - private void flush() throws JobResultsPersister.BulkIndexException { + private void flush() { persistedTimingStats = new TimingStats(currentTimingStats); bulkResultsPersister.persistTimingStats(persistedTimingStats); } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java index 27a79f4aa2ab1..1c281699c546a 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java @@ -132,22 +132,15 @@ public AutodetectProcessManager(Environment environment, Settings settings, Clie this.jobDataCountsPersister = jobDataCountsPersister; this.auditor = auditor; this.nativeStorageProvider = Objects.requireNonNull(nativeStorageProvider); - this.maximumBulkFailureRetries = AutodetectResultProcessor.PERSIST_RESULTS_MAX_RETRIES.get(settings); clusterService.addListener(this); clusterService.getClusterSettings() .addSettingsUpdateConsumer(MachineLearning.MAX_OPEN_JOBS_PER_NODE, this::setMaxAllowedRunningJobs); - clusterService.getClusterSettings() - .addSettingsUpdateConsumer(AutodetectResultProcessor.PERSIST_RESULTS_MAX_RETRIES, this::setMaximumBulkFailureRetries); } void setMaxAllowedRunningJobs(int maxAllowedRunningJobs) { this.maxAllowedRunningJobs = maxAllowedRunningJobs; } - void setMaximumBulkFailureRetries(int maximumBulkFailureRetries) { - this.maximumBulkFailureRetries = maximumBulkFailureRetries; - } - public synchronized void closeAllJobsOnThisNode(String reason) { int numJobs = processByAllocation.size(); if (numJobs != 0) { @@ -527,8 +520,7 @@ AutodetectCommunicator create(JobTask jobTask, Job job, AutodetectParams autodet jobResultsPersister, process, autodetectParams.modelSizeStats(), - autodetectParams.timingStats(), - maximumBulkFailureRetries); + autodetectParams.timingStats()); ExecutorService autodetectWorkerExecutor; try (ThreadContext.StoredContext ignore = threadPool.getThreadContext().stashContext()) { autodetectWorkerExecutor = createAutodetectExecutorService(autodetectExecutorService); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java index 7c79984cd3e15..8655ea65d0998 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java @@ -10,13 +10,11 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteResponse; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.Client; -import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.xpack.core.ml.MachineLearningField; @@ -47,7 +45,6 @@ import java.util.Iterator; import java.util.List; import java.util.Objects; -import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; @@ -75,21 +72,8 @@ */ public class AutodetectResultProcessor { - public static final Setting PERSIST_RESULTS_MAX_RETRIES = Setting.intSetting( - "xpack.ml.persist_results_max_retries", - 2, - 0, - 15, - Setting.Property.Dynamic, - Setting.Property.NodeScope); - - private static final int MAX_RETRY_SLEEP_MILLIS = 5 * 1000; - private static final int MIN_RETRY_SLEEP_MILLIS = 50; - private static final Logger LOGGER = LogManager.getLogger(AutodetectResultProcessor.class); - private final Random random = Randomness.get(); - private final Client client; private final AnomalyDetectionAuditor auditor; private final String jobId; @@ -103,7 +87,6 @@ public class AutodetectResultProcessor { private final FlushListener flushListener; private volatile boolean processKilled; private volatile boolean failed; - private final int maximumFailureRetries; private int bucketCount; // only used from the process() thread, so doesn't need to be volatile private final JobResultsPersister.Builder bulkResultsPersister; private boolean deleteInterimRequired; @@ -120,16 +103,14 @@ public AutodetectResultProcessor(Client client, JobResultsPersister persister, AutodetectProcess process, ModelSizeStats latestModelSizeStats, - TimingStats timingStats, - int maximumFailureRetries) { - this(client, auditor, jobId, renormalizer, persister, process, latestModelSizeStats, timingStats, new FlushListener(), - maximumFailureRetries); + TimingStats timingStats) { + this(client, auditor, jobId, renormalizer, persister, process, latestModelSizeStats, timingStats, new FlushListener()); } // Visible for testing AutodetectResultProcessor(Client client, AnomalyDetectionAuditor auditor, String jobId, Renormalizer renormalizer, JobResultsPersister persister, AutodetectProcess autodetectProcess, ModelSizeStats latestModelSizeStats, - TimingStats timingStats, FlushListener flushListener, int maximumFailureRetries) { + TimingStats timingStats, FlushListener flushListener) { this.client = Objects.requireNonNull(client); this.auditor = Objects.requireNonNull(auditor); this.jobId = Objects.requireNonNull(jobId); @@ -138,10 +119,9 @@ public AutodetectResultProcessor(Client client, this.process = Objects.requireNonNull(autodetectProcess); this.flushListener = Objects.requireNonNull(flushListener); this.latestModelSizeStats = Objects.requireNonNull(latestModelSizeStats); - this.bulkResultsPersister = persister.bulkPersisterBuilder(jobId); + this.bulkResultsPersister = persister.bulkPersisterBuilder(jobId, () -> isDeadOrDying() == false); this.timingStatsReporter = new TimingStatsReporter(timingStats, bulkResultsPersister); this.deleteInterimRequired = true; - this.maximumFailureRetries = maximumFailureRetries; } public void process() { @@ -154,10 +134,8 @@ public void process() { try { if (processKilled == false) { - bulkPersistWithRetry(() -> { - timingStatsReporter.finishReporting(); - bulkResultsPersister.executeRequest(); - }); + timingStatsReporter.finishReporting(); + bulkResultsPersister.executeRequest(); } } catch (Exception e) { LOGGER.warn(new ParameterizedMessage("[{}] Error persisting autodetect results", jobId), e); @@ -232,36 +210,34 @@ void processResult(AutodetectResult result) { // persist after deleting interim results in case the new // results are also interim - bulkPersistWithRetry(() -> { - timingStatsReporter.reportBucket(bucket); - bulkResultsPersister.persistBucket(bucket).executeRequest(); - }); + timingStatsReporter.reportBucket(bucket); + bulkResultsPersister.persistBucket(bucket).executeRequest(); ++bucketCount; } List records = result.getRecords(); if (records != null && !records.isEmpty()) { - bulkPersistWithRetry(() -> bulkResultsPersister.persistRecords(records)); + bulkResultsPersister.persistRecords(records); } List influencers = result.getInfluencers(); if (influencers != null && !influencers.isEmpty()) { - bulkPersistWithRetry(() -> bulkResultsPersister.persistInfluencers(influencers)); + bulkResultsPersister.persistInfluencers(influencers); } CategoryDefinition categoryDefinition = result.getCategoryDefinition(); if (categoryDefinition != null) { - persister.persistCategoryDefinition(categoryDefinition); + persister.persistCategoryDefinition(categoryDefinition, () -> isDeadOrDying() == false); } ModelPlot modelPlot = result.getModelPlot(); if (modelPlot != null) { - bulkPersistWithRetry(() -> bulkResultsPersister.persistModelPlot(modelPlot)); + bulkResultsPersister.persistModelPlot(modelPlot); } Forecast forecast = result.getForecast(); if (forecast != null) { - bulkPersistWithRetry(() -> bulkResultsPersister.persistForecast(forecast)); + bulkResultsPersister.persistForecast(forecast); } ForecastRequestStats forecastRequestStats = result.getForecastRequestStats(); if (forecastRequestStats != null) { LOGGER.trace("Received Forecast Stats [{}]", forecastRequestStats.getId()); - bulkPersistWithRetry(() -> bulkResultsPersister.persistForecastRequestStats(forecastRequestStats)); + bulkResultsPersister.persistForecastRequestStats(forecastRequestStats); // execute the bulk request only in some cases or in doubt // otherwise rely on the count-based trigger @@ -273,7 +249,7 @@ void processResult(AutodetectResult result) { case SCHEDULED: case FINISHED: default: - bulkPersistWithRetry(bulkResultsPersister::executeRequest); + bulkResultsPersister.executeRequest(); } } @@ -284,7 +260,11 @@ void processResult(AutodetectResult result) { ModelSnapshot modelSnapshot = result.getModelSnapshot(); if (modelSnapshot != null) { // We need to refresh in order for the snapshot to be available when we try to update the job with it - IndexResponse indexResponse = persister.persistModelSnapshot(modelSnapshot, WriteRequest.RefreshPolicy.IMMEDIATE); + BulkResponse bulkResponse = persister.persistModelSnapshot(modelSnapshot, + WriteRequest.RefreshPolicy.IMMEDIATE, + () -> isDeadOrDying() == false); + assert bulkResponse.getItems().length == 1; + IndexResponse indexResponse = bulkResponse.getItems()[0].getResponse(); if (indexResponse.getResult() == DocWriteResponse.Result.CREATED) { updateModelSnapshotOnJob(modelSnapshot); } @@ -292,8 +272,8 @@ void processResult(AutodetectResult result) { Quantiles quantiles = result.getQuantiles(); if (quantiles != null) { LOGGER.debug("[{}] Parsed Quantiles with timestamp {}", jobId, quantiles.getTimestamp()); - persister.persistQuantiles(quantiles); - bulkPersistWithRetry(bulkResultsPersister::executeRequest); + persister.persistQuantiles(quantiles, () -> isDeadOrDying() == false); + bulkResultsPersister.executeRequest(); if (processKilled == false && renormalizer.isEnabled()) { // We need to make all results written up to these quantiles available for renormalization @@ -310,7 +290,7 @@ void processResult(AutodetectResult result) { // through to the data store Exception exception = null; try { - bulkPersistWithRetry(bulkResultsPersister::executeRequest); + bulkResultsPersister.executeRequest(); persister.commitResultWrites(jobId); LOGGER.debug("[{}] Flush acknowledgement sent to listener for ID {}", jobId, flushAcknowledgement.getId()); } catch (Exception e) { @@ -330,60 +310,13 @@ void processResult(AutodetectResult result) { } } - void bulkPersistWithRetry(CheckedRunnable bulkRunnable) { - int attempts = 0; - while(attempts <= maximumFailureRetries) { - try { - bulkRunnable.run(); - return; - } catch (JobResultsPersister.BulkIndexException ex) { - if (isDeadOrDying()) { - return; - } - final int currentAttempt = attempts; - LOGGER.trace( - () -> new ParameterizedMessage("[{}] bulk persist failure on attempt [{}] ", jobId, currentAttempt), - ex - ); - attempts++; - try { - double backOff = ((1 << attempts) - 1) / 2.0; - int max = (int)(backOff * 100); - // Random Int between [0-Math.max(max, MAX_RETRY_SLEEP_MILLIS)) - int randSleep = random.nextInt(Math.max(max, MAX_RETRY_SLEEP_MILLIS)); - Thread.sleep(randSleep + MIN_RETRY_SLEEP_MILLIS); - } catch (InterruptedException interrupt) { - LOGGER.warn( - () -> new ParameterizedMessage("[{}] failed bulk indexing of results after [{}] attempts due to interrupt", - jobId, - currentAttempt), - ex - ); - // propagate the interrupt - Thread.currentThread().interrupt(); - return; - } - } catch (Exception e) { - if (isDeadOrDying()) { - throw e; - } - LOGGER.warn(new ParameterizedMessage("[{}] Error processing autodetect result", jobId), e); - } - } - bulkResultsPersister.clearBulkRequest(); - LOGGER.warn( - new ParameterizedMessage("[{}] failed bulk indexing of results after [{}] attempts", - jobId, - attempts)); - } - private void processModelSizeStats(ModelSizeStats modelSizeStats) { LOGGER.trace("[{}] Parsed ModelSizeStats: {} / {} / {} / {} / {} / {}", jobId, modelSizeStats.getModelBytes(), modelSizeStats.getTotalByFieldCount(), modelSizeStats.getTotalOverFieldCount(), modelSizeStats.getTotalPartitionFieldCount(), modelSizeStats.getBucketAllocationFailuresCount(), modelSizeStats.getMemoryStatus()); - persister.persistModelSizeStats(modelSizeStats); + persister.persistModelSizeStats(modelSizeStats, () -> isDeadOrDying() == false); notifyModelMemoryStatusChange(modelSizeStats); latestModelSizeStats = modelSizeStats; } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java new file mode 100644 index 0000000000000..21b95dd223c9b --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java @@ -0,0 +1,162 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.utils.persistence; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Random; +import java.util.Set; +import java.util.function.Supplier; + + +import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; + +public class ResultsPersisterService { + private static final Logger LOGGER = LogManager.getLogger(ResultsPersisterService.class); + + public static final Setting PERSIST_RESULTS_MAX_RETRIES = Setting.intSetting( + "xpack.ml.persist_results_max_retries", + 15, + 0, + 50, + Setting.Property.Dynamic, + Setting.Property.NodeScope); + private static final int MAX_RETRY_SLEEP_MILLIS = 900_000; // 15 min + private static final int MIN_RETRY_SLEEP_MILLIS = 50; + // Having an exponent higher than this causes integer overflow + private static final int MAX_RETRY_EXPONENT = 29; + + private final Random random = Randomness.get(); + private final Client client; + private volatile int maxFailureRetries; + + public ResultsPersisterService(Client client, ClusterService clusterService, Settings settings) { + this.client = client; + this.maxFailureRetries = PERSIST_RESULTS_MAX_RETRIES.get(settings); + clusterService.getClusterSettings() + .addSettingsUpdateConsumer(PERSIST_RESULTS_MAX_RETRIES, this::setMaxFailureRetries); + } + + void setMaxFailureRetries(int value) { + this.maxFailureRetries = value; + } + + public BulkResponse indexWithRetry(String jobId, + String indexName, + ToXContent object, + ToXContent.Params params, + WriteRequest.RefreshPolicy refreshPolicy, + String id, + Supplier shouldRetry) throws IOException { + BulkRequest bulkRequest = new BulkRequest(); + try (XContentBuilder content = object.toXContent(XContentFactory.jsonBuilder(), params)) { + bulkRequest.add(new IndexRequest(indexName).id(id).source(content).setRefreshPolicy(refreshPolicy)); + } + return bulkIndexWithRetry(bulkRequest, jobId, shouldRetry); + } + + public BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, String jobId, Supplier shouldRetry) { + int currentMin = MIN_RETRY_SLEEP_MILLIS; + int currentMax = MIN_RETRY_SLEEP_MILLIS; + int currentAttempt = 0; + while(currentAttempt <= maxFailureRetries) { + BulkResponse bulkResponse = bulkIndex(bulkRequest); + if (bulkResponse.hasFailures() == false) { + return bulkResponse; + } + if (shouldRetry.get() == false) { + throw new ElasticsearchException("[{}] failed to index all results. {}", jobId, bulkResponse.buildFailureMessage()); + } + if (currentAttempt > maxFailureRetries) { + LOGGER.warn("[{}] failed to index after [{}] attempts.", jobId, currentAttempt); + throw new ElasticsearchException("[{}] failed to index all results after [{}] attempts. {}", + jobId, + currentAttempt, + bulkResponse.buildFailureMessage()); + } + currentAttempt++; + // Since we exponentially increase, we don't want force randomness to have an excessively long sleep + if (currentMax < MAX_RETRY_SLEEP_MILLIS) { + currentMin = currentMax; + } + double backOff = ((1 << Math.min(currentAttempt, MAX_RETRY_EXPONENT)) - 1) / 2.0; + int max = (int)(backOff * 100); + currentMax = Math.min(max, MAX_RETRY_SLEEP_MILLIS); + // Its good to have a random window along the exponentially increasing curve + // so that not all bulk requests rest for the same amount of time + int randSleep = currentMin + random.nextInt(1 + (currentMax - currentMin)); + // We should only retry the docs that failed. + bulkRequest = buildNewRequestFromFailures(bulkRequest, bulkResponse); + try { + Thread.sleep(randSleep); + } catch (InterruptedException interruptedException) { + LOGGER.warn( + new ParameterizedMessage("[{}] failed to index after [{}] attempts due to interruption", + jobId, + currentAttempt), + interruptedException); + Thread.currentThread().interrupt(); + } + } + throw new ElasticsearchException("[{}] failed to index all results after [{}] attempts. {}", + jobId, + currentAttempt); + } + + private BulkResponse bulkIndex(BulkRequest bulkRequest) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { + return client.bulk(bulkRequest).actionGet(); + } + } + + private BulkRequest buildNewRequestFromFailures(BulkRequest bulkRequest, BulkResponse bulkResponse) { + // If we failed, lets set the bulkRequest to be a collection of the failed requests + BulkRequest bulkRequestOfFailures = new BulkRequest(); + Set failedDocIds = new HashSet<>(); + bulkResponse.forEach(itemResponse -> { + if (itemResponse.isFailed()) { + failedDocIds.add(itemResponse.getId()); + } + }); + bulkRequest.requests().forEach(docWriteRequest -> { + if (failedDocIds.contains(docWriteRequest.id())) { + bulkRequestOfFailures.add(docWriteRequest); + } + }); + return bulkRequestOfFailures; + } + + public static class BulkIndexException extends Exception { + + public BulkIndexException(String msg) { + super(msg); + } + + public BulkIndexException(BulkResponse bulkResponse) { + this(bulkResponse.buildFailureMessage()); + } + + } + +} diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java index 703c1bb78916c..e8f433a4871c6 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java @@ -6,13 +6,19 @@ package org.elasticsearch.xpack.ml.integration; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.routing.OperationRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; +import org.elasticsearch.cluster.service.ClusterApplierService; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterService; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.index.reindex.ReindexPlugin; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ml.action.DeleteJobAction; import org.elasticsearch.xpack.core.ml.action.PutJobAction; import org.elasticsearch.xpack.core.action.util.QueryPage; @@ -33,6 +39,7 @@ import org.elasticsearch.xpack.core.ml.job.results.ModelPlot; import org.elasticsearch.xpack.ml.LocalStateMachineLearning; import org.elasticsearch.xpack.ml.MlSingleNodeTestCase; +import org.elasticsearch.xpack.ml.inference.ingest.InferenceProcessor; import org.elasticsearch.xpack.ml.job.persistence.BucketsQueryBuilder; import org.elasticsearch.xpack.ml.job.persistence.InfluencersQueryBuilder; import org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister; @@ -46,10 +53,12 @@ import org.elasticsearch.xpack.ml.job.results.CategoryDefinitionTests; import org.elasticsearch.xpack.ml.job.results.ModelPlotTests; import org.elasticsearch.xpack.ml.notifications.AnomalyDetectionAuditor; +import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import org.junit.After; import org.junit.Before; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Date; @@ -77,6 +86,7 @@ public class AutodetectResultProcessorIT extends MlSingleNodeTestCase { private AutodetectResultProcessor resultProcessor; private Renormalizer renormalizer; private AutodetectProcess process; + private ResultsPersisterService resultsPersisterService; @Override protected Collection> getPlugins() { @@ -92,16 +102,26 @@ public void createComponents() throws Exception { renormalizer = mock(Renormalizer.class); process = mock(AutodetectProcess.class); capturedUpdateModelSnapshotOnJobRequests = new ArrayList<>(); + ThreadPool tp = mock(ThreadPool.class); + Settings settings = Settings.builder().put("node.name", "InferenceProcessorFactoryTests_node").build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, + new HashSet<>(Arrays.asList(InferenceProcessor.MAX_INFERENCE_PROCESSORS, + MasterService.MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, + OperationRouting.USE_ADAPTIVE_REPLICA_SELECTION_SETTING, + ClusterService.USER_DEFINED_META_DATA, + ClusterApplierService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING))); + ClusterService clusterService = new ClusterService(settings, clusterSettings, tp); + + resultsPersisterService = new ResultsPersisterService(client(), clusterService, settings); resultProcessor = new AutodetectResultProcessor( client(), auditor, JOB_ID, renormalizer, - new JobResultsPersister(client()), + new JobResultsPersister(client(), resultsPersisterService), process, new ModelSizeStats.Builder(JOB_ID).build(), - new TimingStats(JOB_ID), - 2) { + new TimingStats(JOB_ID)) { @Override protected void updateModelSnapshotOnJob(ModelSnapshot modelSnapshot) { capturedUpdateModelSnapshotOnJobRequests.add(modelSnapshot); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java index d16164681adc3..d8ecfebe0a64f 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java @@ -5,22 +5,33 @@ */ package org.elasticsearch.xpack.ml.integration; +import org.elasticsearch.cluster.routing.OperationRouting; +import org.elasticsearch.cluster.service.ClusterApplierService; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ml.action.PutJobAction; import org.elasticsearch.xpack.core.ml.job.config.AnalysisConfig; import org.elasticsearch.xpack.core.ml.job.config.Job; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.ModelSizeStats; import org.elasticsearch.xpack.core.ml.job.results.Bucket; +import org.elasticsearch.xpack.ml.inference.ingest.InferenceProcessor; import org.elasticsearch.xpack.ml.job.persistence.JobResultsProvider; import org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister; import org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase; +import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import org.junit.Before; +import java.util.Arrays; import java.util.Date; +import java.util.HashSet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; import static org.hamcrest.CoreMatchers.equalTo; +import static org.mockito.Mockito.mock; public class EstablishedMemUsageIT extends BaseMlIntegTestCase { @@ -32,8 +43,18 @@ public class EstablishedMemUsageIT extends BaseMlIntegTestCase { @Before public void createComponents() { Settings settings = nodeSettings(0); + ThreadPool tp = mock(ThreadPool.class); + ClusterSettings clusterSettings = new ClusterSettings(settings, + new HashSet<>(Arrays.asList(InferenceProcessor.MAX_INFERENCE_PROCESSORS, + MasterService.MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, + OperationRouting.USE_ADAPTIVE_REPLICA_SELECTION_SETTING, + ClusterService.USER_DEFINED_META_DATA, + ClusterApplierService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING))); + ClusterService clusterService = new ClusterService(settings, clusterSettings, tp); + + ResultsPersisterService resultsPersisterService = new ResultsPersisterService(client(), clusterService, settings); jobResultsProvider = new JobResultsProvider(client(), settings); - jobResultsPersister = new JobResultsPersister(client()); + jobResultsPersister = new JobResultsPersister(client(), resultsPersisterService); } public void testEstablishedMem_givenNoResults() throws Exception { @@ -221,8 +242,8 @@ private void initClusterAndJob(String jobId) { client().execute(PutJobAction.INSTANCE, putJobRequest).actionGet(); } - private void createBuckets(String jobId, int count) throws Exception { - JobResultsPersister.Builder builder = jobResultsPersister.bulkPersisterBuilder(jobId); + private void createBuckets(String jobId, int count) { + JobResultsPersister.Builder builder = jobResultsPersister.bulkPersisterBuilder(jobId, () -> true); for (int i = 1; i <= count; ++i) { Bucket bucket = new Bucket(jobId, new Date(bucketSpan * i), bucketSpan); builder.persistBucket(bucket); @@ -235,7 +256,7 @@ private ModelSizeStats createModelSizeStats(String jobId, int bucketNum, long mo .setTimestamp(new Date(bucketSpan * bucketNum)) .setLogTime(new Date(bucketSpan * bucketNum + randomIntBetween(1, 1000))) .setModelBytes(modelBytes).build(); - jobResultsPersister.persistModelSizeStats(modelSizeStats); + jobResultsPersister.persistModelSizeStats(modelSizeStats, () -> true); return modelSizeStats; } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java index bde16f7e9ac07..3e3d68b4270f7 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java @@ -15,14 +15,20 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.cluster.metadata.MappingMetaData; +import org.elasticsearch.cluster.routing.OperationRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; +import org.elasticsearch.cluster.service.ClusterApplierService; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ml.MlMetaIndex; import org.elasticsearch.xpack.core.ml.MlMetadata; import org.elasticsearch.xpack.core.ml.action.PutJobAction; @@ -46,12 +52,14 @@ import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.Quantiles; import org.elasticsearch.xpack.core.ml.utils.ToXContentParams; import org.elasticsearch.xpack.ml.MlSingleNodeTestCase; +import org.elasticsearch.xpack.ml.inference.ingest.InferenceProcessor; import org.elasticsearch.xpack.ml.job.persistence.CalendarQueryBuilder; import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister; import org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister; import org.elasticsearch.xpack.ml.job.persistence.JobResultsProvider; import org.elasticsearch.xpack.ml.job.persistence.ScheduledEventsQueryBuilder; import org.elasticsearch.xpack.ml.job.process.autodetect.params.AutodetectParams; +import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import org.junit.Before; import java.io.IOException; @@ -73,17 +81,29 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.collection.IsEmptyCollection.empty; import static org.hamcrest.core.Is.is; +import static org.mockito.Mockito.mock; public class JobResultsProviderIT extends MlSingleNodeTestCase { private JobResultsProvider jobProvider; + private ResultsPersisterService resultsPersisterService; @Before public void createComponents() throws Exception { Settings.Builder builder = Settings.builder() .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueSeconds(1)); jobProvider = new JobResultsProvider(client(), builder.build()); + ThreadPool tp = mock(ThreadPool.class); + ClusterSettings clusterSettings = new ClusterSettings(builder.build(), + new HashSet<>(Arrays.asList(InferenceProcessor.MAX_INFERENCE_PROCESSORS, + MasterService.MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, + OperationRouting.USE_ADAPTIVE_REPLICA_SELECTION_SETTING, + ClusterService.USER_DEFINED_META_DATA, + ClusterApplierService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING))); + ClusterService clusterService = new ClusterService(builder.build(), clusterSettings, tp); + + resultsPersisterService = new ResultsPersisterService(client(), clusterService, builder.build()); waitForMlTemplates(); } @@ -616,18 +636,18 @@ private void indexFilters(List filters) throws IOException { } private void indexModelSizeStats(ModelSizeStats modelSizeStats) { - JobResultsPersister persister = new JobResultsPersister(client()); - persister.persistModelSizeStats(modelSizeStats); + JobResultsPersister persister = new JobResultsPersister(client(), resultsPersisterService); + persister.persistModelSizeStats(modelSizeStats, () -> true); } private void indexModelSnapshot(ModelSnapshot snapshot) { - JobResultsPersister persister = new JobResultsPersister(client()); - persister.persistModelSnapshot(snapshot, WriteRequest.RefreshPolicy.IMMEDIATE); + JobResultsPersister persister = new JobResultsPersister(client(), resultsPersisterService); + persister.persistModelSnapshot(snapshot, WriteRequest.RefreshPolicy.IMMEDIATE, () -> true); } private void indexQuantiles(Quantiles quantiles) { - JobResultsPersister persister = new JobResultsPersister(client()); - persister.persistQuantiles(quantiles); + JobResultsPersister persister = new JobResultsPersister(client(), resultsPersisterService); + persister.persistQuantiles(quantiles, () -> true); } private void indexCalendars(List calendars) throws IOException { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java index 4fb1fb0912b13..151ab6d90aeb1 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java @@ -6,22 +6,22 @@ package org.elasticsearch.xpack.ml.job.persistence; import org.elasticsearch.action.ActionFuture; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.routing.OperationRouting; +import org.elasticsearch.cluster.service.ClusterApplierService; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ml.datafeed.DatafeedTimingStats; -import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.TimingStats; import org.elasticsearch.xpack.core.ml.job.results.AnomalyRecord; import org.elasticsearch.xpack.core.ml.job.results.Bucket; @@ -29,18 +29,21 @@ import org.elasticsearch.xpack.core.ml.job.results.Influencer; import org.elasticsearch.xpack.core.ml.job.results.ModelPlot; import org.elasticsearch.xpack.core.ml.utils.ExponentialAverageCalculationContext; +import org.elasticsearch.xpack.ml.inference.ingest.InferenceProcessor; +import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import org.mockito.ArgumentCaptor; import java.time.Instant; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Date; +import java.util.HashSet; import java.util.List; import java.util.Map; import static org.hamcrest.Matchers.equalTo; import static org.mockito.Matchers.any; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -52,7 +55,7 @@ public class JobResultsPersisterTests extends ESTestCase { private static final String JOB_ID = "foo"; - public void testPersistBucket_OneRecord() throws Exception { + public void testPersistBucket_OneRecord() { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); Bucket bucket = new Bucket("foo", new Date(), 123456); @@ -73,8 +76,8 @@ public void testPersistBucket_OneRecord() throws Exception { AnomalyRecord record = new AnomalyRecord(JOB_ID, new Date(), 600); bucket.setRecords(Collections.singletonList(record)); - JobResultsPersister persister = new JobResultsPersister(client); - persister.bulkPersisterBuilder(JOB_ID).persistBucket(bucket).executeRequest(); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); + persister.bulkPersisterBuilder(JOB_ID, () -> true).persistBucket(bucket).executeRequest(); BulkRequest bulkRequest = captor.getValue(); assertEquals(2, bulkRequest.numberOfActions()); @@ -95,7 +98,7 @@ public void testPersistBucket_OneRecord() throws Exception { assertTrue(s.matches(".*raw_anomaly_score.:19\\.19.*")); } - public void testPersistRecords() throws Exception { + public void testPersistRecords() { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); @@ -125,8 +128,8 @@ public void testPersistRecords() throws Exception { typicals.add(998765.3); r1.setTypical(typicals); - JobResultsPersister persister = new JobResultsPersister(client); - persister.bulkPersisterBuilder(JOB_ID).persistRecords(records).executeRequest(); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); + persister.bulkPersisterBuilder(JOB_ID, () -> true).persistRecords(records).executeRequest(); BulkRequest bulkRequest = captor.getValue(); assertEquals(1, bulkRequest.numberOfActions()); @@ -150,7 +153,7 @@ public void testPersistRecords() throws Exception { assertTrue(s.matches(".*over_field_value.:.overValue.*")); } - public void testPersistInfluencers() throws Exception { + public void testPersistInfluencers() { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); @@ -161,8 +164,8 @@ public void testPersistInfluencers() throws Exception { inf.setProbability(0.4); influencers.add(inf); - JobResultsPersister persister = new JobResultsPersister(client); - persister.bulkPersisterBuilder(JOB_ID).persistInfluencers(influencers).executeRequest(); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); + persister.bulkPersisterBuilder(JOB_ID, () -> true).persistInfluencers(influencers).executeRequest(); BulkRequest bulkRequest = captor.getValue(); assertEquals(1, bulkRequest.numberOfActions()); @@ -174,10 +177,10 @@ public void testPersistInfluencers() throws Exception { assertTrue(s.matches(".*influencer_score.:16\\.0.*")); } - public void testExecuteRequest_ClearsBulkRequest() throws Exception { + public void testExecuteRequest_ClearsBulkRequest() { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); - JobResultsPersister persister = new JobResultsPersister(client); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); List influencers = new ArrayList<>(); Influencer inf = new Influencer(JOB_ID, "infName1", "infValue1", new Date(), 600); @@ -186,17 +189,17 @@ public void testExecuteRequest_ClearsBulkRequest() throws Exception { inf.setProbability(0.4); influencers.add(inf); - JobResultsPersister.Builder builder = persister.bulkPersisterBuilder(JOB_ID); + JobResultsPersister.Builder builder = persister.bulkPersisterBuilder(JOB_ID, () -> true); builder.persistInfluencers(influencers).executeRequest(); assertEquals(0, builder.getBulkRequest().numberOfActions()); } - public void testBulkRequestExecutesWhenReachMaxDocs() throws Exception { + public void testBulkRequestExecutesWhenReachMaxDocs() { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); - JobResultsPersister persister = new JobResultsPersister(client); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); - JobResultsPersister.Builder bulkBuilder = persister.bulkPersisterBuilder("foo"); + JobResultsPersister.Builder bulkBuilder = persister.bulkPersisterBuilder("foo", () -> true); ModelPlot modelPlot = new ModelPlot("foo", new Date(), 123456, 0); for (int i=0; i<=JobRenormalizedResultsPersister.BULK_LIMIT; i++) { bulkBuilder.persistModelPlot(modelPlot); @@ -207,15 +210,15 @@ public void testBulkRequestExecutesWhenReachMaxDocs() throws Exception { verifyNoMoreInteractions(client); } - public void testPersistTimingStats() throws Exception { + public void testPersistTimingStats() { ArgumentCaptor bulkRequestCaptor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(bulkRequestCaptor); - JobResultsPersister persister = new JobResultsPersister(client); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); TimingStats timingStats = new TimingStats( "foo", 7, 1.0, 2.0, 1.23, 7.89, new ExponentialAverageCalculationContext(600.0, Instant.ofEpochMilli(123456789), 60.0)); - persister.bulkPersisterBuilder(JOB_ID).persistTimingStats(timingStats).executeRequest(); + persister.bulkPersisterBuilder(JOB_ID, () -> true).persistTimingStats(timingStats).executeRequest(); verify(client, times(1)).bulk(bulkRequestCaptor.capture()); BulkRequest bulkRequest = bulkRequestCaptor.getValue(); @@ -246,25 +249,15 @@ public void testPersistTimingStats() throws Exception { @SuppressWarnings({"unchecked", "rawtypes"}) public void testPersistDatafeedTimingStats() { Client client = mockClient(ArgumentCaptor.forClass(BulkRequest.class)); - doAnswer( - invocationOnMock -> { - // Take the listener passed to client::index as 2nd argument - ActionListener listener = (ActionListener) invocationOnMock.getArguments()[1]; - // Handle the response on the listener - listener.onResponse(new IndexResponse(new ShardId("test", "test", 0), "test", 0, 0, 0, false)); - return null; - }) - .when(client).index(any(), any(ActionListener.class)); - - JobResultsPersister persister = new JobResultsPersister(client); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); DatafeedTimingStats timingStats = new DatafeedTimingStats( "foo", 6, 66, 666.0, new ExponentialAverageCalculationContext(600.0, Instant.ofEpochMilli(123456789), 60.0)); persister.persistDatafeedTimingStats(timingStats, WriteRequest.RefreshPolicy.IMMEDIATE); - ArgumentCaptor indexRequestCaptor = ArgumentCaptor.forClass(IndexRequest.class); - verify(client, times(1)).index(indexRequestCaptor.capture(), any(ActionListener.class)); - IndexRequest indexRequest = indexRequestCaptor.getValue(); + ArgumentCaptor indexRequestCaptor = ArgumentCaptor.forClass(BulkRequest.class); + verify(client, times(1)).bulk(indexRequestCaptor.capture()); + IndexRequest indexRequest = (IndexRequest)indexRequestCaptor.getValue().requests().get(0); assertThat(indexRequest.index(), equalTo(".ml-anomalies-.write-foo")); assertThat(indexRequest.id(), equalTo("foo_datafeed_timing_stats")); assertThat(indexRequest.getRefreshPolicy(), equalTo(WriteRequest.RefreshPolicy.IMMEDIATE)); @@ -286,78 +279,6 @@ public void testPersistDatafeedTimingStats() { verifyNoMoreInteractions(client); } - public void testBulkRequesChangeOnFailures() throws Exception { - Bucket bucket = new Bucket("foo", new Date(), 123456); - bucket.setAnomalyScore(99.9); - bucket.setEventCount(57); - bucket.setInitialAnomalyScore(88.8); - bucket.setProcessingTimeMs(8888); - - BucketInfluencer bi = new BucketInfluencer(JOB_ID, new Date(), 600); - bi.setAnomalyScore(14.15); - bi.setInfluencerFieldName("biOne"); - bi.setInitialAnomalyScore(18.12); - bi.setProbability(0.0054); - bi.setRawAnomalyScore(19.19); - bucket.addBucketInfluencer(bi); - - AnomalyRecord record = new AnomalyRecord(JOB_ID, new Date(), 600); - bucket.setRecords(Collections.singletonList(record)); - - ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); - BulkItemResponse failureItem = new BulkItemResponse(0, - DocWriteRequest.OpType.INDEX, - new BulkItemResponse.Failure(AnomalyDetectorsIndex.jobResultsIndexPrefix() + "shared", - bucket.getId(), - new Exception("something"))); - BulkItemResponse successItem = new BulkItemResponse(1, - DocWriteRequest.OpType.INDEX, - new IndexResponse(new ShardId(AnomalyDetectorsIndex.jobResultsIndexPrefix() + "shared", "uuid", 1), - bi.getId(), - 0, - 0, - 1, - true)); - BulkResponse withFailure = new BulkResponse(new BulkItemResponse[]{ failureItem, successItem }, 0L); - Client client = mockClientWithResponse(captor, withFailure, new BulkResponse(new BulkItemResponse[0], 0L)); - - JobResultsPersister persister = new JobResultsPersister(client); - JobResultsPersister.Builder bulkPersister = persister.bulkPersisterBuilder(JOB_ID); - expectThrows(JobResultsPersister.BulkIndexException.class, () -> bulkPersister.persistBucket(bucket).executeRequest()); - BulkRequest bulkRequest = captor.getValue(); - assertEquals(2, bulkRequest.numberOfActions()); - - String s = ((IndexRequest)bulkRequest.requests().get(0)).source().utf8ToString(); - assertTrue(s.matches(".*anomaly_score.:99\\.9.*")); - assertTrue(s.matches(".*initial_anomaly_score.:88\\.8.*")); - assertTrue(s.matches(".*event_count.:57.*")); - assertTrue(s.matches(".*bucket_span.:123456.*")); - assertTrue(s.matches(".*processing_time_ms.:8888.*")); - // There should NOT be any nested records - assertFalse(s.matches(".*records*")); - - s = ((IndexRequest)bulkRequest.requests().get(1)).source().utf8ToString(); - assertTrue(s.matches(".*probability.:0\\.0054.*")); - assertTrue(s.matches(".*influencer_field_name.:.biOne.*")); - assertTrue(s.matches(".*initial_anomaly_score.:18\\.12.*")); - assertTrue(s.matches(".*anomaly_score.:14\\.15.*")); - assertTrue(s.matches(".*raw_anomaly_score.:19\\.19.*")); - - bulkPersister.executeRequest(); - bulkRequest = captor.getValue(); - assertEquals(1, bulkRequest.numberOfActions()); - s = ((IndexRequest)bulkRequest.requests().get(0)).source().utf8ToString(); - assertTrue(s.matches(".*anomaly_score.:99\\.9.*")); - assertTrue(s.matches(".*initial_anomaly_score.:88\\.8.*")); - assertTrue(s.matches(".*event_count.:57.*")); - assertTrue(s.matches(".*bucket_span.:123456.*")); - assertTrue(s.matches(".*processing_time_ms.:8888.*")); - // There should NOT be any nested records - assertFalse(s.matches(".*records*")); - - assertEquals(0, bulkPersister.getBulkRequest().numberOfActions()); - } - private Client mockClient(ArgumentCaptor captor) { return mockClientWithResponse(captor, new BulkResponse(new BulkItemResponse[0], 0L)); } @@ -383,4 +304,18 @@ private ActionFuture makeFuture(BulkResponse response) { when(future.actionGet()).thenReturn(response); return future; } + + private ResultsPersisterService buildResultsPersisterService(Client client) { + ThreadPool tp = mock(ThreadPool.class); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, + new HashSet<>(Arrays.asList(InferenceProcessor.MAX_INFERENCE_PROCESSORS, + MasterService.MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, + OperationRouting.USE_ADAPTIVE_REPLICA_SELECTION_SETTING, + ResultsPersisterService.PERSIST_RESULTS_MAX_RETRIES, + ClusterService.USER_DEFINED_META_DATA, + ClusterApplierService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING))); + ClusterService clusterService = new ClusterService(Settings.EMPTY, clusterSettings, tp); + + return new ResultsPersisterService(client, clusterService, Settings.EMPTY); + } } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java index be858570a56b2..09a92e57d0432 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java @@ -47,7 +47,6 @@ import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister; import org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister; import org.elasticsearch.xpack.ml.job.persistence.JobResultsProvider; -import org.elasticsearch.xpack.ml.job.process.autodetect.output.AutodetectResultProcessor; import org.elasticsearch.xpack.ml.job.process.autodetect.params.AutodetectParams; import org.elasticsearch.xpack.ml.job.process.autodetect.params.DataLoadParams; import org.elasticsearch.xpack.ml.job.process.autodetect.params.FlushJobParams; @@ -55,6 +54,7 @@ import org.elasticsearch.xpack.ml.job.process.normalizer.NormalizerFactory; import org.elasticsearch.xpack.ml.notifications.AnomalyDetectionAuditor; import org.elasticsearch.xpack.ml.process.NativeStorageProvider; +import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import org.junit.Before; import org.mockito.ArgumentCaptor; @@ -145,7 +145,7 @@ public void setup() throws Exception { jobManager = mock(JobManager.class); jobResultsProvider = mock(JobResultsProvider.class); jobResultsPersister = mock(JobResultsPersister.class); - when(jobResultsPersister.bulkPersisterBuilder(any())).thenReturn(mock(JobResultsPersister.Builder.class)); + when(jobResultsPersister.bulkPersisterBuilder(any(), any())).thenReturn(mock(JobResultsPersister.Builder.class)); jobDataCountsPersister = mock(JobDataCountsPersister.class); autodetectCommunicator = mock(AutodetectCommunicator.class); autodetectFactory = mock(AutodetectProcessFactory.class); @@ -155,7 +155,7 @@ public void setup() throws Exception { ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, new HashSet<>(Arrays.asList(MachineLearning.MAX_OPEN_JOBS_PER_NODE, - AutodetectResultProcessor.PERSIST_RESULTS_MAX_RETRIES))); + ResultsPersisterService.PERSIST_RESULTS_MAX_RETRIES))); when(clusterService.getClusterSettings()).thenReturn(clusterSettings); MetaData metaData = mock(MetaData.class); SortedMap aliasOrIndexSortedMap = new TreeMap<>(); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java index d86540766f986..2f31015b575c7 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessorTests.java @@ -8,6 +8,9 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.Version; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.Client; @@ -95,7 +98,7 @@ public void setUpMocks() { renormalizer = mock(Renormalizer.class); persister = mock(JobResultsPersister.class); bulkBuilder = mock(JobResultsPersister.Builder.class); - when(persister.bulkPersisterBuilder(JOB_ID)).thenReturn(bulkBuilder); + when(persister.bulkPersisterBuilder(eq(JOB_ID), any())).thenReturn(bulkBuilder); process = mock(AutodetectProcess.class); flushListener = mock(FlushListener.class); processorUnderTest = new AutodetectResultProcessor( @@ -107,8 +110,7 @@ public void setUpMocks() { process, new ModelSizeStats.Builder(JOB_ID).setTimestamp(new Date(BUCKET_SPAN_MS)).build(), new TimingStats(JOB_ID), - flushListener, - 2); + flushListener); } @After @@ -126,7 +128,7 @@ public void testProcess() throws TimeoutException { assertThat(processorUnderTest.completionLatch.getCount(), is(equalTo(0L))); verify(renormalizer).waitUntilIdle(); - verify(persister).bulkPersisterBuilder(JOB_ID); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); verify(persister).commitResultWrites(JOB_ID); verify(persister).commitStateWrites(JOB_ID); } @@ -144,7 +146,7 @@ public void testProcessResult_bucket() throws Exception { verify(bulkBuilder).persistTimingStats(any(TimingStats.class)); verify(bulkBuilder).persistBucket(bucket); verify(bulkBuilder).executeRequest(); - verify(persister).bulkPersisterBuilder(JOB_ID); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); verify(persister, never()).deleteInterimResults(JOB_ID); } @@ -161,7 +163,7 @@ public void testProcessResult_bucket_deleteInterimRequired() throws Exception { verify(bulkBuilder).persistTimingStats(any(TimingStats.class)); verify(bulkBuilder).persistBucket(bucket); verify(bulkBuilder).executeRequest(); - verify(persister).bulkPersisterBuilder(JOB_ID); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); verify(persister).deleteInterimResults(JOB_ID); } @@ -178,7 +180,7 @@ public void testProcessResult_records() throws Exception { verify(bulkBuilder).persistRecords(records); verify(bulkBuilder, never()).executeRequest(); - verify(persister).bulkPersisterBuilder(JOB_ID); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); } public void testProcessResult_influencers() throws Exception { @@ -194,7 +196,7 @@ public void testProcessResult_influencers() throws Exception { verify(bulkBuilder).persistInfluencers(influencers); verify(bulkBuilder, never()).executeRequest(); - verify(persister).bulkPersisterBuilder(JOB_ID); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); } public void testProcessResult_categoryDefinition() throws Exception { @@ -206,8 +208,8 @@ public void testProcessResult_categoryDefinition() throws Exception { processorUnderTest.processResult(result); verify(bulkBuilder, never()).executeRequest(); - verify(persister).persistCategoryDefinition(categoryDefinition); - verify(persister).bulkPersisterBuilder(JOB_ID); + verify(persister).persistCategoryDefinition(eq(categoryDefinition), any()); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); } public void testProcessResult_flushAcknowledgement() throws Exception { @@ -220,7 +222,7 @@ public void testProcessResult_flushAcknowledgement() throws Exception { processorUnderTest.processResult(result); assertTrue(processorUnderTest.isDeleteInterimRequired()); - verify(persister).bulkPersisterBuilder(JOB_ID); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); verify(flushListener).acknowledgeFlush(flushAcknowledgement, null); verify(persister).commitResultWrites(JOB_ID); verify(bulkBuilder).executeRequest(); @@ -239,8 +241,8 @@ public void testProcessResult_flushAcknowledgementMustBeProcessedLast() throws E assertTrue(processorUnderTest.isDeleteInterimRequired()); InOrder inOrder = inOrder(persister, bulkBuilder, flushListener); - inOrder.verify(persister).bulkPersisterBuilder(JOB_ID); - inOrder.verify(persister).persistCategoryDefinition(categoryDefinition); + inOrder.verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); + inOrder.verify(persister).persistCategoryDefinition(eq(categoryDefinition), any()); inOrder.verify(bulkBuilder).executeRequest(); inOrder.verify(persister).commitResultWrites(JOB_ID); inOrder.verify(flushListener).acknowledgeFlush(flushAcknowledgement, null); @@ -254,7 +256,7 @@ public void testProcessResult_modelPlot() throws Exception { processorUnderTest.setDeleteInterimRequired(false); processorUnderTest.processResult(result); - verify(persister).bulkPersisterBuilder(JOB_ID); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); verify(bulkBuilder).persistModelPlot(modelPlot); } @@ -267,8 +269,8 @@ public void testProcessResult_modelSizeStats() throws Exception { processorUnderTest.processResult(result); assertThat(processorUnderTest.modelSizeStats(), is(equalTo(modelSizeStats))); - verify(persister).bulkPersisterBuilder(JOB_ID); - verify(persister).persistModelSizeStats(modelSizeStats); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); + verify(persister).persistModelSizeStats(eq(modelSizeStats), any()); } public void testProcessResult_modelSizeStatsWithMemoryStatusChanges() throws Exception { @@ -304,8 +306,8 @@ public void testProcessResult_modelSizeStatsWithMemoryStatusChanges() throws Exc when(result.getModelSizeStats()).thenReturn(modelSizeStats); processorUnderTest.processResult(result); - verify(persister).bulkPersisterBuilder(JOB_ID); - verify(persister, times(4)).persistModelSizeStats(any(ModelSizeStats.class)); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); + verify(persister, times(4)).persistModelSizeStats(any(ModelSizeStats.class), any()); // We should have only fired two notifications: one for soft_limit and one for hard_limit verify(auditor).warning(JOB_ID, Messages.getMessage(Messages.JOB_AUDIT_MEMORY_STATUS_SOFT_LIMIT)); verify(auditor).error(JOB_ID, Messages.getMessage(Messages.JOB_AUDIT_MEMORY_STATUS_HARD_LIMIT, "512mb", "1kb")); @@ -318,15 +320,16 @@ public void testProcessResult_modelSnapshot() throws Exception { .setMinVersion(Version.CURRENT) .build(); when(result.getModelSnapshot()).thenReturn(modelSnapshot); + IndexResponse indexResponse = new IndexResponse(new ShardId("ml", "uid", 0), "1", 0L, 0L, 0L, true); - when(persister.persistModelSnapshot(any(), any())) - .thenReturn(new IndexResponse(new ShardId("ml", "uid", 0), "1", 0L, 0L, 0L, true)); + when(persister.persistModelSnapshot(any(), any(), any())) + .thenReturn(new BulkResponse(new BulkItemResponse[]{new BulkItemResponse(0, DocWriteRequest.OpType.INDEX, indexResponse)}, 0)); processorUnderTest.setDeleteInterimRequired(false); processorUnderTest.processResult(result); - verify(persister).bulkPersisterBuilder(JOB_ID); - verify(persister).persistModelSnapshot(modelSnapshot, WriteRequest.RefreshPolicy.IMMEDIATE); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); + verify(persister).persistModelSnapshot(eq(modelSnapshot), eq(WriteRequest.RefreshPolicy.IMMEDIATE), any()); UpdateJobAction.Request expectedJobUpdateRequest = UpdateJobAction.Request.internal(JOB_ID, new JobUpdate.Builder(JOB_ID).setModelSnapshotId("a_snapshot_id").build()); @@ -343,8 +346,8 @@ public void testProcessResult_quantiles_givenRenormalizationIsEnabled() throws E processorUnderTest.setDeleteInterimRequired(false); processorUnderTest.processResult(result); - verify(persister).bulkPersisterBuilder(JOB_ID); - verify(persister).persistQuantiles(quantiles); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); + verify(persister).persistQuantiles(eq(quantiles), any()); verify(bulkBuilder).executeRequest(); verify(persister).commitResultWrites(JOB_ID); verify(renormalizer).isEnabled(); @@ -360,8 +363,8 @@ public void testProcessResult_quantiles_givenRenormalizationIsDisabled() throws processorUnderTest.setDeleteInterimRequired(false); processorUnderTest.processResult(result); - verify(persister).bulkPersisterBuilder(JOB_ID); - verify(persister).persistQuantiles(quantiles); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); + verify(persister).persistQuantiles(eq(quantiles), any()); verify(bulkBuilder).executeRequest(); verify(renormalizer).isEnabled(); } @@ -375,7 +378,7 @@ public void testAwaitCompletion() throws TimeoutException { assertThat(processorUnderTest.completionLatch.getCount(), is(equalTo(0L))); assertThat(processorUnderTest.updateModelSnapshotSemaphore.availablePermits(), is(equalTo(1))); - verify(persister).bulkPersisterBuilder(JOB_ID); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); verify(persister).commitResultWrites(JOB_ID); verify(persister).commitStateWrites(JOB_ID); verify(renormalizer).waitUntilIdle(); @@ -390,12 +393,12 @@ public void testPersisterThrowingDoesntBlockProcessing() { when(process.isProcessAliveAfterWaiting()).thenReturn(true); when(process.readAutodetectResults()).thenReturn(Arrays.asList(autodetectResult, autodetectResult).iterator()); - doThrow(new ElasticsearchException("this test throws")).when(persister).persistModelSnapshot(any(), any()); + doThrow(new ElasticsearchException("this test throws")).when(persister).persistModelSnapshot(any(), any(), any()); processorUnderTest.process(); - verify(persister).bulkPersisterBuilder(JOB_ID); - verify(persister, times(2)).persistModelSnapshot(any(), eq(WriteRequest.RefreshPolicy.IMMEDIATE)); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); + verify(persister, times(2)).persistModelSnapshot(any(), eq(WriteRequest.RefreshPolicy.IMMEDIATE), any()); } public void testParsingErrorSetsFailed() throws Exception { @@ -413,7 +416,7 @@ public void testParsingErrorSetsFailed() throws Exception { processorUnderTest.waitForFlushAcknowledgement(JOB_ID, Duration.of(300, ChronoUnit.SECONDS)); assertThat(flushAcknowledgement, is(nullValue())); - verify(persister).bulkPersisterBuilder(JOB_ID); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); } public void testKill() throws TimeoutException { @@ -426,7 +429,7 @@ public void testKill() throws TimeoutException { assertThat(processorUnderTest.completionLatch.getCount(), is(equalTo(0L))); assertThat(processorUnderTest.updateModelSnapshotSemaphore.availablePermits(), is(equalTo(1))); - verify(persister).bulkPersisterBuilder(JOB_ID); + verify(persister).bulkPersisterBuilder(eq(JOB_ID), any()); verify(persister).commitResultWrites(JOB_ID); verify(persister).commitStateWrites(JOB_ID); verify(renormalizer, never()).renormalize(any()); @@ -435,105 +438,4 @@ public void testKill() throws TimeoutException { verify(flushListener).clear(); } - public void testBulkPersistWithRetry() throws Exception { - JobResultsPersister persister = mock(JobResultsPersister.class); - JobResultsPersister.Builder bulkBuilder = mock(JobResultsPersister.Builder.class); - when(process.isProcessAlive()).thenReturn(true); - when(process.isProcessAliveAfterWaiting()).thenReturn(true); - doThrow(new JobResultsPersister.BulkIndexException("boom")) - .when(bulkBuilder).executeRequest(); - when(persister.bulkPersisterBuilder(JOB_ID)).thenReturn(bulkBuilder); - AutodetectResultProcessor processorUnderTest = new AutodetectResultProcessor( - client, - auditor, - JOB_ID, - renormalizer, - persister, - process, - new ModelSizeStats.Builder(JOB_ID).setTimestamp(new Date(BUCKET_SPAN_MS)).build(), - new TimingStats(JOB_ID), - flushListener, - 2); - - processorUnderTest.bulkPersistWithRetry(bulkBuilder::executeRequest); - - // Hax as memory persister needs to be verified before @After - verify(this.persister).bulkPersisterBuilder(JOB_ID); - verify(bulkBuilder, times(3)).executeRequest(); - } - - public void testBulkPersistWithRetry_withDeadProcess() throws Exception { - JobResultsPersister persister = mock(JobResultsPersister.class); - JobResultsPersister.Builder bulkBuilder = mock(JobResultsPersister.Builder.class); - when(process.isProcessAliveAfterWaiting()).thenReturn(true); - doThrow(new JobResultsPersister.BulkIndexException("boom")) - .when(bulkBuilder).executeRequest(); - when(persister.bulkPersisterBuilder(JOB_ID)).thenReturn(bulkBuilder); - AutodetectResultProcessor processorUnderTest = new AutodetectResultProcessor( - client, - auditor, - JOB_ID, - renormalizer, - persister, - process, - new ModelSizeStats.Builder(JOB_ID).setTimestamp(new Date(BUCKET_SPAN_MS)).build(), - new TimingStats(JOB_ID), - flushListener, - 2); - - processorUnderTest.setProcessKilled(); - processorUnderTest.bulkPersistWithRetry(bulkBuilder::executeRequest); - // Hax as memory persister needs to be verified before @After - verify(this.persister).bulkPersisterBuilder(JOB_ID); - verify(this.renormalizer).shutdown(); - } - - public void testBulkPersistWithRetry_withDyingProcess() throws Exception { - JobResultsPersister persister = mock(JobResultsPersister.class); - JobResultsPersister.Builder bulkBuilder = mock(JobResultsPersister.Builder.class); - when(process.isProcessAliveAfterWaiting()).thenReturn(false); - doThrow(new JobResultsPersister.BulkIndexException("boom")) - .when(bulkBuilder).executeRequest(); - when(persister.bulkPersisterBuilder(JOB_ID)).thenReturn(bulkBuilder); - AutodetectResultProcessor processorUnderTest = new AutodetectResultProcessor( - client, - auditor, - JOB_ID, - renormalizer, - persister, - process, - new ModelSizeStats.Builder(JOB_ID).setTimestamp(new Date(BUCKET_SPAN_MS)).build(), - new TimingStats(JOB_ID), - flushListener, - 2); - - processorUnderTest.bulkPersistWithRetry(bulkBuilder::executeRequest); - verify(bulkBuilder, times(1)).executeRequest(); - // Hax as memory persister needs to be verified before @After - verify(this.persister).bulkPersisterBuilder(JOB_ID); - } - - public void testBulkPersistWithRetry_withExceptionOtherThanBulkFailure() throws Exception { - JobResultsPersister persister = mock(JobResultsPersister.class); - JobResultsPersister.Builder bulkBuilder = mock(JobResultsPersister.Builder.class); - when(process.isProcessAliveAfterWaiting()).thenReturn(false); - doThrow(new ElasticsearchException("boom")) - .when(bulkBuilder).executeRequest(); - when(persister.bulkPersisterBuilder(JOB_ID)).thenReturn(bulkBuilder); - AutodetectResultProcessor processorUnderTest = new AutodetectResultProcessor( - client, - auditor, - JOB_ID, - renormalizer, - persister, - process, - new ModelSizeStats.Builder(JOB_ID).setTimestamp(new Date(BUCKET_SPAN_MS)).build(), - new TimingStats(JOB_ID), - flushListener, - 2); - - expectThrows(ElasticsearchException.class, () -> processorUnderTest.bulkPersistWithRetry(bulkBuilder::executeRequest)); - // Hax as memory persister needs to be verified before @After - verify(this.persister).bulkPersisterBuilder(JOB_ID); - } } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java new file mode 100644 index 0000000000000..b1bfdd91b1fe6 --- /dev/null +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java @@ -0,0 +1,161 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.utils.persistence; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.routing.OperationRouting; +import org.elasticsearch.cluster.service.ClusterApplierService; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; +import org.elasticsearch.xpack.ml.inference.ingest.InferenceProcessor; +import org.mockito.ArgumentCaptor; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class ResultsPersisterServiceTests extends ESTestCase { + + private final String JOB_ID = "results_persister_test_job"; + + public void testBulkRequestChangeOnFailures() { + IndexRequest indexRequestSuccess = new IndexRequest("my-index").id("success").source(Collections.singletonMap("data", "success")); + IndexRequest indexRequestFail = new IndexRequest("my-index").id("fail").source(Collections.singletonMap("data", "fail")); + BulkItemResponse successItem = new BulkItemResponse(1, + DocWriteRequest.OpType.INDEX, + new IndexResponse(new ShardId(AnomalyDetectorsIndex.jobResultsIndexPrefix() + "shared", "uuid", 1), + indexRequestSuccess.id(), + 0, + 0, + 1, + true)); + BulkItemResponse failureItem = new BulkItemResponse(2, + DocWriteRequest.OpType.INDEX, + new BulkItemResponse.Failure("my-index", "fail", new Exception("boom"))); + BulkResponse withFailure = new BulkResponse(new BulkItemResponse[]{ failureItem, successItem }, 0L); + Client client = mockClientWithResponse(withFailure, new BulkResponse(new BulkItemResponse[0], 0L)); + + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.add(indexRequestFail); + bulkRequest.add(indexRequestSuccess); + + ResultsPersisterService resultsPersisterService = buildResultsPersisterService(client); + + resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> true); + + ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); + verify(client, times(2)).bulk(captor.capture()); + + List requests = captor.getAllValues(); + + assertThat(requests.get(0).numberOfActions(), equalTo(2)); + assertThat(requests.get(1).numberOfActions(), equalTo(1)); + } + + public void testBulkRequestDoesNotRetryWhenSupplierIsFalse() { + IndexRequest indexRequestSuccess = new IndexRequest("my-index").id("success").source(Collections.singletonMap("data", "success")); + IndexRequest indexRequestFail = new IndexRequest("my-index").id("fail").source(Collections.singletonMap("data", "fail")); + BulkItemResponse successItem = new BulkItemResponse(1, + DocWriteRequest.OpType.INDEX, + new IndexResponse(new ShardId(AnomalyDetectorsIndex.jobResultsIndexPrefix() + "shared", "uuid", 1), + indexRequestSuccess.id(), + 0, + 0, + 1, + true)); + BulkItemResponse failureItem = new BulkItemResponse(2, + DocWriteRequest.OpType.INDEX, + new BulkItemResponse.Failure("my-index", "fail", new Exception("boom"))); + BulkResponse withFailure = new BulkResponse(new BulkItemResponse[]{ failureItem, successItem }, 0L); + Client client = mockClientWithResponse(withFailure, new BulkResponse(new BulkItemResponse[0], 0L)); + + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.add(indexRequestFail); + bulkRequest.add(indexRequestSuccess); + + ResultsPersisterService resultsPersisterService = buildResultsPersisterService(client); + + expectThrows(ElasticsearchException.class, () -> resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> false)); + } + + public void testBulkRequestRetriesConfiguredAttemptNumber() { + IndexRequest indexRequestFail = new IndexRequest("my-index").id("fail").source(Collections.singletonMap("data", "fail")); + BulkItemResponse failureItem = new BulkItemResponse(2, + DocWriteRequest.OpType.INDEX, + new BulkItemResponse.Failure("my-index", "fail", new Exception("boom"))); + BulkResponse withFailure = new BulkResponse(new BulkItemResponse[]{ failureItem }, 0L); + Client client = mockClientWithResponse(withFailure); + + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.add(indexRequestFail); + + ResultsPersisterService resultsPersisterService = buildResultsPersisterService(client); + + resultsPersisterService.setMaxFailureRetries(1); + expectThrows(ElasticsearchException.class, () -> resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> true)); + verify(client, times(2)).bulk(any(BulkRequest.class)); + } + + @SuppressWarnings("unchecked") + private Client mockClientWithResponse(BulkResponse... responses) { + Client client = mock(Client.class); + ThreadPool threadPool = mock(ThreadPool.class); + when(client.threadPool()).thenReturn(threadPool); + when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY)); + List> futures = new ArrayList<>(responses.length - 1); + ActionFuture future1 = makeFuture(responses[0]); + for (int i = 1; i < responses.length; i++) { + futures.add(makeFuture(responses[i])); + } + when(client.bulk(any(BulkRequest.class))).thenReturn(future1, futures.toArray(ActionFuture[]::new)); + return client; + } + + @SuppressWarnings("unchecked") + private ActionFuture makeFuture(BulkResponse response) { + ActionFuture future = mock(ActionFuture.class); + when(future.actionGet()).thenReturn(response); + return future; + } + + private ResultsPersisterService buildResultsPersisterService(Client client) { + ThreadPool tp = mock(ThreadPool.class); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, + new HashSet<>(Arrays.asList(InferenceProcessor.MAX_INFERENCE_PROCESSORS, + MasterService.MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, + OperationRouting.USE_ADAPTIVE_REPLICA_SELECTION_SETTING, + ClusterService.USER_DEFINED_META_DATA, + ResultsPersisterService.PERSIST_RESULTS_MAX_RETRIES, + ClusterApplierService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING))); + ClusterService clusterService = new ClusterService(Settings.EMPTY, clusterSettings, tp); + + return new ResultsPersisterService(client, clusterService, Settings.EMPTY); + } +} From bf749d7563b052f12a924f8b5cc7c2b1ce179e45 Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Wed, 4 Dec 2019 07:28:48 -0500 Subject: [PATCH 07/15] Fixing refresh policy handling bug and missing settings --- .../xpack/ml/utils/persistence/ResultsPersisterService.java | 4 ++-- .../xpack/ml/integration/AutodetectResultProcessorIT.java | 1 + .../xpack/ml/integration/EstablishedMemUsageIT.java | 1 + .../xpack/ml/integration/JobResultsProviderIT.java | 1 + 4 files changed, 5 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java index 21b95dd223c9b..75ad9f23ccbc8 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java @@ -69,9 +69,9 @@ public BulkResponse indexWithRetry(String jobId, WriteRequest.RefreshPolicy refreshPolicy, String id, Supplier shouldRetry) throws IOException { - BulkRequest bulkRequest = new BulkRequest(); + BulkRequest bulkRequest = new BulkRequest().setRefreshPolicy(refreshPolicy); try (XContentBuilder content = object.toXContent(XContentFactory.jsonBuilder(), params)) { - bulkRequest.add(new IndexRequest(indexName).id(id).source(content).setRefreshPolicy(refreshPolicy)); + bulkRequest.add(new IndexRequest(indexName).id(id).source(content)); } return bulkIndexWithRetry(bulkRequest, jobId, shouldRetry); } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java index e8f433a4871c6..e9bd90003fe83 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java @@ -109,6 +109,7 @@ public void createComponents() throws Exception { MasterService.MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, OperationRouting.USE_ADAPTIVE_REPLICA_SELECTION_SETTING, ClusterService.USER_DEFINED_META_DATA, + ResultsPersisterService.PERSIST_RESULTS_MAX_RETRIES, ClusterApplierService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING))); ClusterService clusterService = new ClusterService(settings, clusterSettings, tp); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java index d8ecfebe0a64f..f9caa5c8f5221 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java @@ -47,6 +47,7 @@ public void createComponents() { ClusterSettings clusterSettings = new ClusterSettings(settings, new HashSet<>(Arrays.asList(InferenceProcessor.MAX_INFERENCE_PROCESSORS, MasterService.MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, + ResultsPersisterService.PERSIST_RESULTS_MAX_RETRIES, OperationRouting.USE_ADAPTIVE_REPLICA_SELECTION_SETTING, ClusterService.USER_DEFINED_META_DATA, ClusterApplierService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING))); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java index 3e3d68b4270f7..e6fe543b060bf 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java @@ -99,6 +99,7 @@ public void createComponents() throws Exception { new HashSet<>(Arrays.asList(InferenceProcessor.MAX_INFERENCE_PROCESSORS, MasterService.MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, OperationRouting.USE_ADAPTIVE_REPLICA_SELECTION_SETTING, + ResultsPersisterService.PERSIST_RESULTS_MAX_RETRIES, ClusterService.USER_DEFINED_META_DATA, ClusterApplierService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING))); ClusterService clusterService = new ClusterService(builder.build(), clusterSettings, tp); From 0e0125933f20694821b33f952bdaa1e7225c02e6 Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Wed, 4 Dec 2019 07:43:58 -0500 Subject: [PATCH 08/15] fixing datafeed timing stats persistence --- .../xpack/ml/job/persistence/JobResultsPersisterTests.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java index 151ab6d90aeb1..0012374230169 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java @@ -257,10 +257,12 @@ public void testPersistDatafeedTimingStats() { ArgumentCaptor indexRequestCaptor = ArgumentCaptor.forClass(BulkRequest.class); verify(client, times(1)).bulk(indexRequestCaptor.capture()); + + // Refresh policy is set on the bulk request, not the individual index requests + assertThat(indexRequestCaptor.getValue().getRefreshPolicy(), equalTo(WriteRequest.RefreshPolicy.IMMEDIATE)); IndexRequest indexRequest = (IndexRequest)indexRequestCaptor.getValue().requests().get(0); assertThat(indexRequest.index(), equalTo(".ml-anomalies-.write-foo")); assertThat(indexRequest.id(), equalTo("foo_datafeed_timing_stats")); - assertThat(indexRequest.getRefreshPolicy(), equalTo(WriteRequest.RefreshPolicy.IMMEDIATE)); assertThat( indexRequest.sourceAsMap(), equalTo( From 471063556e7a54952bd959cdaa46a546c1b33cfe Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Wed, 4 Dec 2019 10:04:20 -0500 Subject: [PATCH 09/15] rewriting datacounts reporter to use retries --- .../xpack/ml/MachineLearning.java | 2 +- .../TransportRevertModelSnapshotAction.java | 2 +- .../persistence/JobDataCountsPersister.java | 36 ++++++++++++++++--- .../ml/job/process/DataCountsReporter.java | 14 ++------ .../writer/CsvDataToProcessWriter.java | 8 ++--- .../writer/JsonDataToProcessWriter.java | 9 ++--- .../ml/integration/JobResultsProviderIT.java | 26 ++------------ .../job/process/DataCountsReporterTests.java | 9 +++-- .../AutodetectCommunicatorTests.java | 7 ++-- .../writer/CsvDataToProcessWriterTests.java | 20 +++++------ .../writer/JsonDataToProcessWriterTests.java | 18 +++++----- 11 files changed, 66 insertions(+), 85 deletions(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java index 1aa00fbd7cf77..cff7d276dc20b 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java @@ -525,7 +525,7 @@ public Collection createComponents(Client client, ClusterService cluster ResultsPersisterService resultsPersisterService = new ResultsPersisterService(client, clusterService, settings); JobResultsProvider jobResultsProvider = new JobResultsProvider(client, settings); JobResultsPersister jobResultsPersister = new JobResultsPersister(client, resultsPersisterService); - JobDataCountsPersister jobDataCountsPersister = new JobDataCountsPersister(client); + JobDataCountsPersister jobDataCountsPersister = new JobDataCountsPersister(client, resultsPersisterService); JobConfigProvider jobConfigProvider = new JobConfigProvider(client, xContentRegistry); DatafeedConfigProvider datafeedConfigProvider = new DatafeedConfigProvider(client, xContentRegistry); UpdateJobProcessNotifier notifier = new UpdateJobProcessNotifier(client, clusterService, threadPool); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportRevertModelSnapshotAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportRevertModelSnapshotAction.java index 7b394b580044f..0406f7b9e3370 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportRevertModelSnapshotAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportRevertModelSnapshotAction.java @@ -171,7 +171,7 @@ private ActionListener wrapRevertDataCountsL return ActionListener.wrap(response -> { jobResultsProvider.dataCounts(jobId, counts -> { counts.setLatestRecordTimeStamp(modelSnapshot.getLatestRecordTimeStamp()); - jobDataCountsPersister.persistDataCounts(jobId, counts, new ActionListener() { + jobDataCountsPersister.persistDataCountsAsync(jobId, counts, new ActionListener() { @Override public void onResponse(Boolean aBoolean) { listener.onResponse(response); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java index 9866e3f56af19..4732f0c948fbc 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java @@ -13,11 +13,13 @@ import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.Client; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.DataCounts; +import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import java.io.IOException; @@ -33,9 +35,11 @@ public class JobDataCountsPersister { private static final Logger logger = LogManager.getLogger(JobDataCountsPersister.class); + private final ResultsPersisterService resultsPersisterService; private final Client client; - public JobDataCountsPersister(Client client) { + public JobDataCountsPersister(Client client, ResultsPersisterService resultsPersisterService) { + this.resultsPersisterService = resultsPersisterService; this.client = client; } @@ -46,16 +50,40 @@ private XContentBuilder serialiseCounts(DataCounts counts) throws IOException { /** * Update the job's data counts stats and figures. + * NOTE: This call is synchronous and pauses the calling thread. + * @param jobId Job to update + * @param counts The counts + */ + public void persistDataCounts(String jobId, DataCounts counts) { + try { + resultsPersisterService.indexWithRetry(jobId, + AnomalyDetectorsIndex.resultsWriteAlias(jobId), + counts, + ToXContent.EMPTY_PARAMS, + WriteRequest.RefreshPolicy.NONE, + DataCounts.documentId(jobId), () -> true); + } catch (IOException ioe) { + logger.warn((Supplier)() -> new ParameterizedMessage("[{}] Error serialising DataCounts stats", jobId), ioe); + } catch (Exception ex) { + logger.warn(() -> new ParameterizedMessage("[{}] Failed to persist DataCounts stats", jobId), ex); + } + } + + /** + * The same as {@link JobDataCountsPersister#persistDataCounts(String, DataCounts)} but done Asynchronously. * + * Two differences are: + * - The listener is notified on persistence failure + * - If the persistence fails, it is not automatically retried * @param jobId Job to update * @param counts The counts * @param listener ActionType response listener */ - public void persistDataCounts(String jobId, DataCounts counts, ActionListener listener) { + public void persistDataCountsAsync(String jobId, DataCounts counts, ActionListener listener) { try (XContentBuilder content = serialiseCounts(counts)) { final IndexRequest request = new IndexRequest(AnomalyDetectorsIndex.resultsWriteAlias(jobId)) - .id(DataCounts.documentId(jobId)) - .source(content); + .id(DataCounts.documentId(jobId)) + .source(content); executeAsyncWithOrigin(client, ML_ORIGIN, IndexAction.INSTANCE, request, new ActionListener() { @Override public void onResponse(IndexResponse indexResponse) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporter.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporter.java index b00ae2816068f..1d3a42b5fa9b6 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporter.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporter.java @@ -7,8 +7,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.xpack.core.ml.job.config.Job; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.DataCounts; import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister; @@ -230,21 +228,13 @@ public long getAnalysedFieldsPerRecord() { /** * Report the counts now regardless of whether or not we are at a reporting boundary. */ - public void finishReporting(ActionListener listener) { + public void finishReporting() { Date now = new Date(); incrementalRecordStats.setLastDataTimeStamp(now); totalRecordStats.setLastDataTimeStamp(now); diagnostics.flush(); retrieveDiagnosticsIntermediateResults(); - dataCountsPersister.persistDataCounts(job.getId(), runningTotalStats(), ActionListener.wrap( - listener::onResponse, - e -> { - // Recording data counts should not cause the job processing to fail. - // Log the failure and move on. - logger.warn(() -> new ParameterizedMessage("[{}] failed to record data counts", job.getId()), e); - listener.onResponse(true); - } - )); + dataCountsPersister.persistDataCounts(job.getId(), runningTotalStats()); } /** diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/CsvDataToProcessWriter.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/CsvDataToProcessWriter.java index 887b43d0b5927..efa5f767720c4 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/CsvDataToProcessWriter.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/CsvDataToProcessWriter.java @@ -7,7 +7,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.xpack.ml.job.categorization.CategorizationAnalyzer; import org.elasticsearch.xpack.core.ml.job.config.AnalysisConfig; @@ -128,11 +127,8 @@ public void write(InputStream inputStream, CategorizationAnalyzer categorization transformTimeAndWrite(record, inputFieldCount); } - // This function can throw - dataCountsReporter.finishReporting(ActionListener.wrap( - response -> handler.accept(dataCountsReporter.incrementalStats(), null), - e -> handler.accept(null, e) - )); + dataCountsReporter.finishReporting(); + handler.accept(dataCountsReporter.incrementalStats(), null); } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriter.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriter.java index 902fbb775f474..59c138cba3758 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriter.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriter.java @@ -7,7 +7,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentFactory; @@ -70,12 +69,8 @@ public void write(InputStream inputStream, CategorizationAnalyzer categorization + "] is not supported by JsonDataToProcessWriter"); } - // this line can throw and will be propagated - dataCountsReporter.finishReporting( - ActionListener.wrap( - response -> handler.accept(dataCountsReporter.incrementalStats(), null), - e -> handler.accept(null, e) - )); + dataCountsReporter.finishReporting(); + handler.accept(dataCountsReporter.incrementalStats(), null); } private void writeJsonXContent(CategorizationAnalyzer categorizationAnalyzer, InputStream inputStream) throws IOException { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java index e6fe543b060bf..2156f37b07340 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java @@ -595,29 +595,9 @@ private void indexScheduledEvents(List events) throws IOExceptio } } - private void indexDataCounts(DataCounts counts, String jobId) throws Exception { - JobDataCountsPersister persister = new JobDataCountsPersister(client()); - - AtomicReference errorHolder = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - persister.persistDataCounts(jobId, counts, new ActionListener() { - @Override - public void onResponse(Boolean aBoolean) { - assertTrue(aBoolean); - latch.countDown(); - } - - @Override - public void onFailure(Exception e) { - errorHolder.set(e); - latch.countDown(); - } - }); - - latch.await(); - if (errorHolder.get() != null) { - throw errorHolder.get(); - } + private void indexDataCounts(DataCounts counts, String jobId) { + JobDataCountsPersister persister = new JobDataCountsPersister(client(), resultsPersisterService); + persister.persistDataCounts(jobId, counts); } private void indexFilters(List filters) throws IOException { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporterTests.java index 5415b46019196..ea4b6869c1b01 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporterTests.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.ml.job.process; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.ml.job.config.AnalysisConfig; @@ -119,7 +118,7 @@ public void testResetIncrementalCounts() throws Exception { assertEquals(602000, dataCountsReporter.runningTotalStats().getLatestRecordTimeStamp().getTime()); // send 'flush' signal - dataCountsReporter.finishReporting(ActionListener.wrap(r -> {}, e -> {})); + dataCountsReporter.finishReporting(); assertEquals(2, dataCountsReporter.runningTotalStats().getBucketCount()); assertEquals(1, dataCountsReporter.runningTotalStats().getEmptyBucketCount()); assertEquals(0, dataCountsReporter.runningTotalStats().getSparseBucketCount()); @@ -157,7 +156,7 @@ public void testReportRecordsWritten() { assertEquals(dataCountsReporter.incrementalStats(), dataCountsReporter.runningTotalStats()); - verify(jobDataCountsPersister, never()).persistDataCounts(anyString(), any(DataCounts.class), any()); + verify(jobDataCountsPersister, never()).persistDataCounts(anyString(), any(DataCounts.class)); } public void testReportRecordsWritten_Given9999Records() { @@ -256,7 +255,7 @@ public void testFinishReporting() { dataCountsReporter.reportRecordWritten(5, 2000); dataCountsReporter.reportRecordWritten(5, 3000); dataCountsReporter.reportMissingField(); - dataCountsReporter.finishReporting(ActionListener.wrap(r -> {}, e -> {})); + dataCountsReporter.finishReporting(); long lastReportedTimeMs = dataCountsReporter.incrementalStats().getLastDataTimeStamp().getTime(); // check last data time is equal to now give or take a second @@ -266,7 +265,7 @@ public void testFinishReporting() { dataCountsReporter.runningTotalStats().getLastDataTimeStamp()); dc.setLastDataTimeStamp(dataCountsReporter.incrementalStats().getLastDataTimeStamp()); - Mockito.verify(jobDataCountsPersister, Mockito.times(1)).persistDataCounts(eq("sr"), eq(dc), any()); + Mockito.verify(jobDataCountsPersister, Mockito.times(1)).persistDataCounts(eq("sr"), eq(dc)); assertEquals(dc, dataCountsReporter.incrementalStats()); } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectCommunicatorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectCommunicatorTests.java index 4562779fc292f..e9fbaf4bbb0d5 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectCommunicatorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectCommunicatorTests.java @@ -6,7 +6,6 @@ package org.elasticsearch.xpack.ml.job.process.autodetect; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentType; @@ -55,6 +54,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -235,10 +235,7 @@ private AutodetectCommunicator createAutodetectCommunicator(ExecutorService exec AutodetectResultProcessor autodetectResultProcessor, BiConsumer finishHandler) throws IOException { DataCountsReporter dataCountsReporter = mock(DataCountsReporter.class); - doAnswer(invocation -> { - ((ActionListener) invocation.getArguments()[0]).onResponse(true); - return null; - }).when(dataCountsReporter).finishReporting(any()); + doNothing().when(dataCountsReporter).finishReporting(); return new AutodetectCommunicator(createJobDetails(), environment, autodetectProcess, stateStreamer, dataCountsReporter, autodetectResultProcessor, finishHandler, new NamedXContentRegistry(Collections.emptyList()), executorService); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/CsvDataToProcessWriterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/CsvDataToProcessWriterTests.java index cf65eec4f04df..05604d0f876c2 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/CsvDataToProcessWriterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/CsvDataToProcessWriterTests.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.ml.job.process.autodetect.writer; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.env.Environment; @@ -42,6 +41,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyLong; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -106,7 +106,7 @@ public void testWrite_GivenTimeFormatIsEpochAndDataIsValid() throws IOException expectedRecords.add(new String[] { "2", "2.0", "" }); assertWrittenRecordsEqualTo(expectedRecords); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_GivenTimeFormatIsEpochAndCategorization() throws IOException { @@ -143,7 +143,7 @@ public void testWrite_GivenTimeFormatIsEpochAndCategorization() throws IOExcepti } assertWrittenRecordsEqualTo(expectedRecords); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_GivenTimeFormatIsEpochAndTimestampsAreOutOfOrder() throws IOException { @@ -166,7 +166,7 @@ public void testWrite_GivenTimeFormatIsEpochAndTimestampsAreOutOfOrder() throws verify(dataCountsReporter, times(2)).reportOutOfOrderRecord(2); verify(dataCountsReporter, never()).reportLatestTimeIncrementalStats(anyLong()); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_GivenTimeFormatIsEpochAndAllRecordsAreOutOfOrder() throws IOException { @@ -190,7 +190,7 @@ public void testWrite_GivenTimeFormatIsEpochAndAllRecordsAreOutOfOrder() throws verify(dataCountsReporter, times(2)).reportOutOfOrderRecord(2); verify(dataCountsReporter, times(2)).reportLatestTimeIncrementalStats(anyLong()); verify(dataCountsReporter, never()).reportRecordWritten(anyLong(), anyLong()); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_GivenTimeFormatIsEpochAndSomeTimestampsWithinLatencySomeOutOfOrder() throws IOException { @@ -224,7 +224,7 @@ public void testWrite_GivenTimeFormatIsEpochAndSomeTimestampsWithinLatencySomeOu verify(dataCountsReporter, times(1)).reportOutOfOrderRecord(2); verify(dataCountsReporter, never()).reportLatestTimeIncrementalStats(anyLong()); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_NullByte() throws IOException { @@ -262,7 +262,7 @@ public void testWrite_NullByte() throws IOException { verify(dataCountsReporter, times(1)).reportRecordWritten(2, 3000); verify(dataCountsReporter, times(1)).reportRecordWritten(2, 4000); verify(dataCountsReporter, times(1)).reportDateParseError(2); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } @SuppressWarnings("unchecked") @@ -274,11 +274,7 @@ public void testWrite_EmptyInput() throws IOException { when(dataCountsReporter.incrementalStats()).thenReturn(new DataCounts("foo")); - doAnswer(invocation -> { - ActionListener listener = (ActionListener) invocation.getArguments()[0]; - listener.onResponse(true); - return null; - }).when(dataCountsReporter).finishReporting(any()); + doNothing().when(dataCountsReporter).finishReporting(); InputStream inputStream = createInputStream(""); CsvDataToProcessWriter writer = createWriter(); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriterTests.java index f16b388edee6f..89ff28928fdae 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/writer/JsonDataToProcessWriterTests.java @@ -106,7 +106,7 @@ public void testWrite_GivenTimeFormatIsEpochAndDataIsValid() throws Exception { expectedRecords.add(new String[]{"2", "2.0", ""}); assertWrittenRecordsEqualTo(expectedRecords); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_GivenTimeFormatIsEpochAndCategorization() throws Exception { @@ -142,7 +142,7 @@ public void testWrite_GivenTimeFormatIsEpochAndCategorization() throws Exception } assertWrittenRecordsEqualTo(expectedRecords); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_GivenTimeFormatIsEpochAndTimestampsAreOutOfOrder() throws Exception { @@ -164,7 +164,7 @@ public void testWrite_GivenTimeFormatIsEpochAndTimestampsAreOutOfOrder() throws verify(dataCountsReporter, times(2)).reportOutOfOrderRecord(2); verify(dataCountsReporter, never()).reportLatestTimeIncrementalStats(anyLong()); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_GivenTimeFormatIsEpochAndSomeTimestampsWithinLatencySomeOutOfOrder() throws Exception { @@ -195,7 +195,7 @@ public void testWrite_GivenTimeFormatIsEpochAndSomeTimestampsWithinLatencySomeOu verify(dataCountsReporter, times(1)).reportOutOfOrderRecord(2); verify(dataCountsReporter, never()).reportLatestTimeIncrementalStats(anyLong()); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_GivenMalformedJsonWithoutNestedLevels() throws Exception { @@ -223,7 +223,7 @@ public void testWrite_GivenMalformedJsonWithoutNestedLevels() throws Exception { assertWrittenRecordsEqualTo(expectedRecords); verify(dataCountsReporter).reportMissingFields(1); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_GivenMalformedJsonWithNestedLevels() @@ -251,7 +251,7 @@ public void testWrite_GivenMalformedJsonWithNestedLevels() expectedRecords.add(new String[]{"3", "3.0", ""}); assertWrittenRecordsEqualTo(expectedRecords); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_GivenMalformedJsonThatNeverRecovers() @@ -293,7 +293,7 @@ public void testWrite_GivenJsonWithArrayField() throws Exception { expectedRecords.add(new String[]{"2", "2.0", ""}); assertWrittenRecordsEqualTo(expectedRecords); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_GivenJsonWithMissingFields() throws Exception { @@ -330,7 +330,7 @@ public void testWrite_GivenJsonWithMissingFields() throws Exception { verify(dataCountsReporter, times(1)).reportRecordWritten(1, 3000); verify(dataCountsReporter, times(1)).reportRecordWritten(1, 4000); verify(dataCountsReporter, times(1)).reportDateParseError(0); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } public void testWrite_Smile() throws Exception { @@ -367,7 +367,7 @@ public void testWrite_Smile() throws Exception { expectedRecords.add(new String[]{"2", "2.0", ""}); assertWrittenRecordsEqualTo(expectedRecords); - verify(dataCountsReporter).finishReporting(any()); + verify(dataCountsReporter).finishReporting(); } private static InputStream createInputStream(String input) { From 7e4642fb50a6f47433cd1a2c0280e13fb05820d0 Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Mon, 9 Dec 2019 08:54:56 -0500 Subject: [PATCH 10/15] cleanup and addressing pr comments --- .../ml/integration/BulkFailureRetryIT.java | 15 +++---- .../persistence/JobDataCountsPersister.java | 9 ++-- .../job/persistence/JobResultsPersister.java | 11 ----- .../job/persistence/TimingStatsReporter.java | 2 +- .../autodetect/AutodetectProcessManager.java | 1 - .../output/AutodetectResultProcessor.java | 21 ++++----- .../persistence/ResultsPersisterService.java | 44 +++++++++---------- .../persistence/JobResultsPersisterTests.java | 2 +- .../job/process/DataCountsReporterTests.java | 14 +++--- .../ResultsPersisterServiceTests.java | 2 +- 10 files changed, 51 insertions(+), 70 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java index e25686065db8c..6caf6a86fd669 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java @@ -28,6 +28,7 @@ import org.junit.After; import org.junit.Before; +import java.time.Duration; import java.util.Collections; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; @@ -38,11 +39,11 @@ public class BulkFailureRetryIT extends MlNativeAutodetectIntegTestCase { - private String index = "bulk-failure-retry"; + private final String index = "bulk-failure-retry"; private long now = System.currentTimeMillis(); - private static long DAY = 86400000; - private String jobId = "bulk-failure-retry-job"; - private String resultsIndex = ".ml-anomalies-custom-bulk-failure-retry-job"; + private static long DAY = Duration.ofDays(1).toMillis(); + private final String jobId = "bulk-failure-retry-job"; + private final String resultsIndex = ".ml-anomalies-custom-bulk-failure-retry-job"; @Before public void putPastDataIntoIndex() { @@ -154,11 +155,7 @@ private Job.Builder createJob(String id, TimeValue bucketSpan, String function, analysisConfig.setBucketSpan(bucketSpan); analysisConfig.setSummaryCountFieldName(summaryCountField); - Job.Builder builder = new Job.Builder(); - builder.setId(id); - builder.setAnalysisConfig(analysisConfig); - builder.setDataDescription(dataDescription); - return builder; + return new Job.Builder().setId(id).setAnalysisConfig(analysisConfig).setDataDescription(dataDescription); } private void writeData(Logger logger, String index, long numDocs, long start, long end) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java index 4732f0c948fbc..8802b57421b04 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java @@ -8,7 +8,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; @@ -43,7 +42,7 @@ public JobDataCountsPersister(Client client, ResultsPersisterService resultsPers this.client = client; } - private XContentBuilder serialiseCounts(DataCounts counts) throws IOException { + private static XContentBuilder serialiseCounts(DataCounts counts) throws IOException { XContentBuilder builder = jsonBuilder(); return counts.toXContent(builder, ToXContent.EMPTY_PARAMS); } @@ -63,7 +62,7 @@ public void persistDataCounts(String jobId, DataCounts counts) { WriteRequest.RefreshPolicy.NONE, DataCounts.documentId(jobId), () -> true); } catch (IOException ioe) { - logger.warn((Supplier)() -> new ParameterizedMessage("[{}] Error serialising DataCounts stats", jobId), ioe); + logger.warn(() -> new ParameterizedMessage("[{}] Error serialising DataCounts stats", jobId), ioe); } catch (Exception ex) { logger.warn(() -> new ParameterizedMessage("[{}] Failed to persist DataCounts stats", jobId), ex); } @@ -84,7 +83,7 @@ public void persistDataCountsAsync(String jobId, DataCounts counts, ActionListen final IndexRequest request = new IndexRequest(AnomalyDetectorsIndex.resultsWriteAlias(jobId)) .id(DataCounts.documentId(jobId)) .source(content); - executeAsyncWithOrigin(client, ML_ORIGIN, IndexAction.INSTANCE, request, new ActionListener() { + executeAsyncWithOrigin(client, ML_ORIGIN, IndexAction.INSTANCE, request, new ActionListener<>() { @Override public void onResponse(IndexResponse indexResponse) { listener.onResponse(true); @@ -96,7 +95,7 @@ public void onFailure(Exception e) { } }); } catch (IOException ioe) { - logger.warn((Supplier)() -> new ParameterizedMessage("[{}] Error serialising DataCounts stats", jobId), ioe); + logger.warn(() -> new ParameterizedMessage("[{}] Error serialising DataCounts stats", jobId), ioe); } } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java index a2b2ef2361957..e0d7ecdbd2f56 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java @@ -423,15 +423,4 @@ private void logCall(String indexName) { } } - public static class BulkIndexException extends Exception { - - public BulkIndexException(String msg) { - super(msg); - } - - public BulkIndexException(BulkResponse bulkResponse) { - this(bulkResponse.buildFailureMessage()); - } - - } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java index 8b4d0817c9dba..69d6936e9051f 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java @@ -44,7 +44,7 @@ public void reportBucket(Bucket bucket) { } } - public void finishReporting() throws JobResultsPersister.BulkIndexException { + public void finishReporting() { // Don't flush if current timing stats are identical to the persisted ones if (currentTimingStats.equals(persistedTimingStats)) { return; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java index 1c281699c546a..9ef733198114e 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java @@ -112,7 +112,6 @@ public class AutodetectProcessManager implements ClusterStateListener { private final AnomalyDetectionAuditor auditor; private volatile boolean upgradeInProgress; - private volatile int maximumBulkFailureRetries; public AutodetectProcessManager(Environment environment, Settings settings, Client client, ThreadPool threadPool, NamedXContentRegistry xContentRegistry, AnomalyDetectionAuditor auditor, ClusterService clusterService, diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java index 8655ea65d0998..422f13926d441 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultProcessor.java @@ -119,7 +119,7 @@ public AutodetectResultProcessor(Client client, this.process = Objects.requireNonNull(autodetectProcess); this.flushListener = Objects.requireNonNull(flushListener); this.latestModelSizeStats = Objects.requireNonNull(latestModelSizeStats); - this.bulkResultsPersister = persister.bulkPersisterBuilder(jobId, () -> isDeadOrDying() == false); + this.bulkResultsPersister = persister.bulkPersisterBuilder(jobId, this::isAlive); this.timingStatsReporter = new TimingStatsReporter(timingStats, bulkResultsPersister); this.deleteInterimRequired = true; } @@ -177,7 +177,7 @@ private void readResults() { LOGGER.trace("[{}] Bucket number {} parsed from output", jobId, bucketCount); } } catch (Exception e) { - if (isDeadOrDying()) { + if (isAlive() == false) { throw e; } LOGGER.warn(new ParameterizedMessage("[{}] Error processing autodetect result", jobId), e); @@ -224,7 +224,7 @@ void processResult(AutodetectResult result) { } CategoryDefinition categoryDefinition = result.getCategoryDefinition(); if (categoryDefinition != null) { - persister.persistCategoryDefinition(categoryDefinition, () -> isDeadOrDying() == false); + persister.persistCategoryDefinition(categoryDefinition, this::isAlive); } ModelPlot modelPlot = result.getModelPlot(); if (modelPlot != null) { @@ -260,9 +260,7 @@ void processResult(AutodetectResult result) { ModelSnapshot modelSnapshot = result.getModelSnapshot(); if (modelSnapshot != null) { // We need to refresh in order for the snapshot to be available when we try to update the job with it - BulkResponse bulkResponse = persister.persistModelSnapshot(modelSnapshot, - WriteRequest.RefreshPolicy.IMMEDIATE, - () -> isDeadOrDying() == false); + BulkResponse bulkResponse = persister.persistModelSnapshot(modelSnapshot, WriteRequest.RefreshPolicy.IMMEDIATE, this::isAlive); assert bulkResponse.getItems().length == 1; IndexResponse indexResponse = bulkResponse.getItems()[0].getResponse(); if (indexResponse.getResult() == DocWriteResponse.Result.CREATED) { @@ -272,7 +270,7 @@ void processResult(AutodetectResult result) { Quantiles quantiles = result.getQuantiles(); if (quantiles != null) { LOGGER.debug("[{}] Parsed Quantiles with timestamp {}", jobId, quantiles.getTimestamp()); - persister.persistQuantiles(quantiles, () -> isDeadOrDying() == false); + persister.persistQuantiles(quantiles, this::isAlive); bulkResultsPersister.executeRequest(); if (processKilled == false && renormalizer.isEnabled()) { @@ -316,7 +314,7 @@ private void processModelSizeStats(ModelSizeStats modelSizeStats) { modelSizeStats.getTotalOverFieldCount(), modelSizeStats.getTotalPartitionFieldCount(), modelSizeStats.getBucketAllocationFailuresCount(), modelSizeStats.getMemoryStatus()); - persister.persistModelSizeStats(modelSizeStats, () -> isDeadOrDying() == false); + persister.persistModelSizeStats(modelSizeStats, this::isAlive); notifyModelMemoryStatusChange(modelSizeStats); latestModelSizeStats = modelSizeStats; } @@ -435,8 +433,11 @@ boolean isDeleteInterimRequired() { return deleteInterimRequired; } - private boolean isDeadOrDying() { - return processKilled || (process.isProcessAliveAfterWaiting() == false); + private boolean isAlive() { + if (processKilled) { + return false; + } + return process.isProcessAliveAfterWaiting(); } void setDeleteInterimRequired(boolean deleteInterimRequired) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java index 75ad9f23ccbc8..92af0d119635a 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java @@ -9,6 +9,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; @@ -24,10 +25,12 @@ import org.elasticsearch.common.xcontent.XContentFactory; import java.io.IOException; -import java.util.HashSet; +import java.time.Duration; +import java.util.Arrays; import java.util.Random; import java.util.Set; import java.util.function.Supplier; +import java.util.stream.Collectors; import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; @@ -42,7 +45,7 @@ public class ResultsPersisterService { 50, Setting.Property.Dynamic, Setting.Property.NodeScope); - private static final int MAX_RETRY_SLEEP_MILLIS = 900_000; // 15 min + private static final int MAX_RETRY_SLEEP_MILLIS = (int)Duration.ofMinutes(15).toMillis(); private static final int MIN_RETRY_SLEEP_MILLIS = 50; // Having an exponent higher than this causes integer overflow private static final int MAX_RETRY_EXPONENT = 29; @@ -80,8 +83,9 @@ public BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, String jobId, Su int currentMin = MIN_RETRY_SLEEP_MILLIS; int currentMax = MIN_RETRY_SLEEP_MILLIS; int currentAttempt = 0; + BulkResponse bulkResponse = null; while(currentAttempt <= maxFailureRetries) { - BulkResponse bulkResponse = bulkIndex(bulkRequest); + bulkResponse = bulkIndex(bulkRequest); if (bulkResponse.hasFailures() == false) { return bulkResponse; } @@ -89,7 +93,9 @@ public BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, String jobId, Su throw new ElasticsearchException("[{}] failed to index all results. {}", jobId, bulkResponse.buildFailureMessage()); } if (currentAttempt > maxFailureRetries) { - LOGGER.warn("[{}] failed to index after [{}] attempts.", jobId, currentAttempt); + LOGGER.warn("[{}] failed to index after [{}] attempts. Setting [xpack.ml.persist_results_max_retries] was reduced", + jobId, + currentAttempt); throw new ElasticsearchException("[{}] failed to index all results after [{}] attempts. {}", jobId, currentAttempt, @@ -105,7 +111,8 @@ public BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, String jobId, Su currentMax = Math.min(max, MAX_RETRY_SLEEP_MILLIS); // Its good to have a random window along the exponentially increasing curve // so that not all bulk requests rest for the same amount of time - int randSleep = currentMin + random.nextInt(1 + (currentMax - currentMin)); + int randBound = 1 + (currentMax - currentMin); + int randSleep = currentMin + random.nextInt(randBound); // We should only retry the docs that failed. bulkRequest = buildNewRequestFromFailures(bulkRequest, bulkResponse); try { @@ -119,9 +126,12 @@ public BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, String jobId, Su Thread.currentThread().interrupt(); } } + String bulkFailureMessage = bulkResponse == null ? "" : bulkResponse.buildFailureMessage(); + LOGGER.warn("[{}] failed to index after [{}] attempts.", jobId, currentAttempt); throw new ElasticsearchException("[{}] failed to index all results after [{}] attempts. {}", jobId, - currentAttempt); + currentAttempt, + bulkFailureMessage); } private BulkResponse bulkIndex(BulkRequest bulkRequest) { @@ -133,12 +143,10 @@ private BulkResponse bulkIndex(BulkRequest bulkRequest) { private BulkRequest buildNewRequestFromFailures(BulkRequest bulkRequest, BulkResponse bulkResponse) { // If we failed, lets set the bulkRequest to be a collection of the failed requests BulkRequest bulkRequestOfFailures = new BulkRequest(); - Set failedDocIds = new HashSet<>(); - bulkResponse.forEach(itemResponse -> { - if (itemResponse.isFailed()) { - failedDocIds.add(itemResponse.getId()); - } - }); + Set failedDocIds = Arrays.stream(bulkResponse.getItems()) + .filter(BulkItemResponse::isFailed) + .map(BulkItemResponse::getId) + .collect(Collectors.toSet()); bulkRequest.requests().forEach(docWriteRequest -> { if (failedDocIds.contains(docWriteRequest.id())) { bulkRequestOfFailures.add(docWriteRequest); @@ -147,16 +155,4 @@ private BulkRequest buildNewRequestFromFailures(BulkRequest bulkRequest, BulkRes return bulkRequestOfFailures; } - public static class BulkIndexException extends Exception { - - public BulkIndexException(String msg) { - super(msg); - } - - public BulkIndexException(BulkResponse bulkResponse) { - this(bulkResponse.buildFailureMessage()); - } - - } - } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java index 0012374230169..d4723f451d10c 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java @@ -301,7 +301,7 @@ private Client mockClientWithResponse(ArgumentCaptor captor, BulkRe } @SuppressWarnings("unchecked") - private ActionFuture makeFuture(BulkResponse response) { + private static ActionFuture makeFuture(BulkResponse response) { ActionFuture future = mock(ActionFuture.class); when(future.actionGet()).thenReturn(response); return future; diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporterTests.java index ea4b6869c1b01..e9faff382edb3 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/DataCountsReporterTests.java @@ -16,7 +16,6 @@ import org.junit.Before; import org.mockito.Mockito; -import java.io.IOException; import java.util.Arrays; import java.util.Date; import java.util.concurrent.TimeUnit; @@ -25,6 +24,7 @@ import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; public class DataCountsReporterTests extends ESTestCase { @@ -49,14 +49,14 @@ public void setUpMocks() { jobDataCountsPersister = Mockito.mock(JobDataCountsPersister.class); } - public void testSimpleConstructor() throws Exception { + public void testSimpleConstructor() { DataCountsReporter dataCountsReporter = new DataCountsReporter(job, new DataCounts(job.getId()), jobDataCountsPersister); DataCounts stats = dataCountsReporter.incrementalStats(); assertNotNull(stats); assertAllCountFieldsEqualZero(stats); } - public void testComplexConstructor() throws Exception { + public void testComplexConstructor() { DataCounts counts = new DataCounts("foo", 1L, 1L, 2L, 0L, 3L, 4L, 5L, 6L, 7L, 8L, new Date(), new Date(), new Date(), new Date(), new Date()); @@ -76,7 +76,7 @@ public void testComplexConstructor() throws Exception { assertNull(stats.getEarliestRecordTimeStamp()); } - public void testResetIncrementalCounts() throws Exception { + public void testResetIncrementalCounts() { DataCountsReporter dataCountsReporter = new DataCountsReporter(job, new DataCounts(job.getId()), jobDataCountsPersister); DataCounts stats = dataCountsReporter.incrementalStats(); assertNotNull(stats); @@ -128,7 +128,7 @@ public void testResetIncrementalCounts() throws Exception { assertEquals(0, dataCountsReporter.incrementalStats().getSparseBucketCount()); } - public void testReportLatestTimeIncrementalStats() throws IOException { + public void testReportLatestTimeIncrementalStats() { DataCountsReporter dataCountsReporter = new DataCountsReporter(job, new DataCounts(job.getId()), jobDataCountsPersister); dataCountsReporter.startNewIncrementalCount(); dataCountsReporter.reportLatestTimeIncrementalStats(5001L); @@ -265,11 +265,11 @@ public void testFinishReporting() { dataCountsReporter.runningTotalStats().getLastDataTimeStamp()); dc.setLastDataTimeStamp(dataCountsReporter.incrementalStats().getLastDataTimeStamp()); - Mockito.verify(jobDataCountsPersister, Mockito.times(1)).persistDataCounts(eq("sr"), eq(dc)); + verify(jobDataCountsPersister, times(1)).persistDataCounts(eq("sr"), eq(dc)); assertEquals(dc, dataCountsReporter.incrementalStats()); } - private void assertAllCountFieldsEqualZero(DataCounts stats) throws Exception { + private void assertAllCountFieldsEqualZero(DataCounts stats) { assertEquals(0L, stats.getProcessedRecordCount()); assertEquals(0L, stats.getProcessedFieldCount()); assertEquals(0L, stats.getInputBytes()); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java index b1bfdd91b1fe6..57fb4a3b91a5e 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java @@ -139,7 +139,7 @@ private Client mockClientWithResponse(BulkResponse... responses) { } @SuppressWarnings("unchecked") - private ActionFuture makeFuture(BulkResponse response) { + private static ActionFuture makeFuture(BulkResponse response) { ActionFuture future = mock(ActionFuture.class); when(future.actionGet()).thenReturn(response); return future; From cb8a5b2d0489df998f4e231e5fda1acb006f5e4f Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Mon, 9 Dec 2019 09:39:46 -0500 Subject: [PATCH 11/15] using builder where possible --- .../xpack/ml/integration/BulkFailureRetryIT.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java index 6caf6a86fd669..26357d5a782d7 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java @@ -151,9 +151,9 @@ private Job.Builder createJob(String id, TimeValue bucketSpan, String function, dataDescription.setTimeFormat(DataDescription.EPOCH_MS); Detector.Builder d = new Detector.Builder(function, field); - AnalysisConfig.Builder analysisConfig = new AnalysisConfig.Builder(Collections.singletonList(d.build())); - analysisConfig.setBucketSpan(bucketSpan); - analysisConfig.setSummaryCountFieldName(summaryCountField); + AnalysisConfig.Builder analysisConfig = new AnalysisConfig.Builder(Collections.singletonList(d.build())) + .setBucketSpan(bucketSpan) + .setSummaryCountFieldName(summaryCountField); return new Job.Builder().setId(id).setAnalysisConfig(analysisConfig).setDataDescription(dataDescription); } From 7814ed23236ce5d1fe9749214a374300594c97b9 Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Mon, 9 Dec 2019 12:24:26 -0500 Subject: [PATCH 12/15] adjusing backoff calculation --- .../xpack/ml/utils/persistence/ResultsPersisterService.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java index 92af0d119635a..79875c032edd9 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java @@ -106,9 +106,9 @@ public BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, String jobId, Su if (currentMax < MAX_RETRY_SLEEP_MILLIS) { currentMin = currentMax; } - double backOff = ((1 << Math.min(currentAttempt, MAX_RETRY_EXPONENT)) - 1) / 2.0; - int max = (int)(backOff * 100); - currentMax = Math.min(max, MAX_RETRY_SLEEP_MILLIS); + // Exponential backoff calculation taken from: https://en.wikipedia.org/wiki/Exponential_backoff + int uncappedBackoff = ((1 << Math.min(currentAttempt, MAX_RETRY_EXPONENT)) - 1) * (50); + currentMax = Math.min(uncappedBackoff, MAX_RETRY_SLEEP_MILLIS); // Its good to have a random window along the exponentially increasing curve // so that not all bulk requests rest for the same amount of time int randBound = 1 + (currentMax - currentMin); From 724dfcf67a5b165bc25df99fc28f0b62e968906c Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Mon, 9 Dec 2019 13:26:51 -0500 Subject: [PATCH 13/15] disallow int wrapping --- .../xpack/ml/utils/persistence/ResultsPersisterService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java index 79875c032edd9..9ebff4bae4d10 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java @@ -48,7 +48,7 @@ public class ResultsPersisterService { private static final int MAX_RETRY_SLEEP_MILLIS = (int)Duration.ofMinutes(15).toMillis(); private static final int MIN_RETRY_SLEEP_MILLIS = 50; // Having an exponent higher than this causes integer overflow - private static final int MAX_RETRY_EXPONENT = 29; + private static final int MAX_RETRY_EXPONENT = 24; private final Random random = Randomness.get(); private final Client client; From 756be6c0525210f9eb46d00a1f3d70b7597eea19 Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Mon, 9 Dec 2019 16:52:14 -0500 Subject: [PATCH 14/15] Adding auditor messaging indicating failures and retries --- .../xpack/ml/MachineLearning.java | 6 ++- .../persistence/JobDataCountsPersister.java | 9 +++- .../job/persistence/JobResultsPersister.java | 20 ++++++-- .../persistence/ResultsPersisterService.java | 35 ++++++++++++-- .../AutodetectResultProcessorIT.java | 2 +- .../ml/integration/EstablishedMemUsageIT.java | 5 +- .../ml/integration/JobResultsProviderIT.java | 11 +++-- .../persistence/JobResultsPersisterTests.java | 24 +++++++--- .../persistence/TimingStatsReporterTests.java | 8 ++-- .../ResultsPersisterServiceTests.java | 47 +++++++++++++++++-- 10 files changed, 136 insertions(+), 31 deletions(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java index cff7d276dc20b..a7b21bbd721ad 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java @@ -524,8 +524,10 @@ public Collection createComponents(Client client, ClusterService cluster this.dataFrameAnalyticsAuditor.set(dataFrameAnalyticsAuditor); ResultsPersisterService resultsPersisterService = new ResultsPersisterService(client, clusterService, settings); JobResultsProvider jobResultsProvider = new JobResultsProvider(client, settings); - JobResultsPersister jobResultsPersister = new JobResultsPersister(client, resultsPersisterService); - JobDataCountsPersister jobDataCountsPersister = new JobDataCountsPersister(client, resultsPersisterService); + JobResultsPersister jobResultsPersister = new JobResultsPersister(client, resultsPersisterService, anomalyDetectionAuditor); + JobDataCountsPersister jobDataCountsPersister = new JobDataCountsPersister(client, + resultsPersisterService, + anomalyDetectionAuditor); JobConfigProvider jobConfigProvider = new JobConfigProvider(client, xContentRegistry); DatafeedConfigProvider datafeedConfigProvider = new DatafeedConfigProvider(client, xContentRegistry); UpdateJobProcessNotifier notifier = new UpdateJobProcessNotifier(client, clusterService, threadPool); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java index 8802b57421b04..0cd81333c9d40 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java @@ -18,6 +18,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.DataCounts; +import org.elasticsearch.xpack.ml.notifications.AnomalyDetectionAuditor; import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import java.io.IOException; @@ -36,10 +37,12 @@ public class JobDataCountsPersister { private final ResultsPersisterService resultsPersisterService; private final Client client; + private final AnomalyDetectionAuditor auditor; - public JobDataCountsPersister(Client client, ResultsPersisterService resultsPersisterService) { + public JobDataCountsPersister(Client client, ResultsPersisterService resultsPersisterService, AnomalyDetectionAuditor auditor) { this.resultsPersisterService = resultsPersisterService; this.client = client; + this.auditor = auditor; } private static XContentBuilder serialiseCounts(DataCounts counts) throws IOException { @@ -60,7 +63,9 @@ public void persistDataCounts(String jobId, DataCounts counts) { counts, ToXContent.EMPTY_PARAMS, WriteRequest.RefreshPolicy.NONE, - DataCounts.documentId(jobId), () -> true); + DataCounts.documentId(jobId), + () -> true, + (msg) -> auditor.warning(jobId, "Job data_counts " + msg)); } catch (IOException ioe) { logger.warn(() -> new ParameterizedMessage("[{}] Error serialising DataCounts stats", jobId), ioe); } catch (Exception ex) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java index e0d7ecdbd2f56..b9a3fbaa570bb 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java @@ -39,6 +39,7 @@ import org.elasticsearch.xpack.core.ml.job.results.Influencer; import org.elasticsearch.xpack.core.ml.job.results.ModelPlot; import org.elasticsearch.xpack.core.ml.utils.ToXContentParams; +import org.elasticsearch.xpack.ml.notifications.AnomalyDetectionAuditor; import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import java.io.IOException; @@ -73,10 +74,14 @@ public class JobResultsPersister { private final Client client; private final ResultsPersisterService resultsPersisterService; + private final AnomalyDetectionAuditor auditor; - public JobResultsPersister(Client client, ResultsPersisterService resultsPersisterService) { + public JobResultsPersister(Client client, + ResultsPersisterService resultsPersisterService, + AnomalyDetectionAuditor auditor) { this.client = client; this.resultsPersisterService = resultsPersisterService; + this.auditor = auditor; } public Builder bulkPersisterBuilder(String jobId, Supplier shouldRetry) { @@ -221,7 +226,9 @@ public void executeRequest() { return; } logger.trace("[{}] ES API CALL: bulk request with {} actions", jobId, bulkRequest.numberOfActions()); - resultsPersisterService.bulkIndexWithRetry(bulkRequest, jobId, shouldRetry); + resultsPersisterService.bulkIndexWithRetry(bulkRequest, jobId, shouldRetry, (msg) -> { + auditor.warning(jobId, "Bulk indexing of results failed " + msg); + }); bulkRequest = new BulkRequest(); } @@ -389,7 +396,14 @@ void setRefreshPolicy(WriteRequest.RefreshPolicy refreshPolicy) { BulkResponse persist(String indexName, Supplier shouldRetry) { logCall(indexName); try { - return resultsPersisterService.indexWithRetry(jobId, indexName, object, params, refreshPolicy, id, shouldRetry); + return resultsPersisterService.indexWithRetry(jobId, + indexName, + object, + params, + refreshPolicy, + id, + shouldRetry, + (msg) -> auditor.warning(jobId, id + " " + msg)); } catch (IOException e) { logger.error(new ParameterizedMessage("[{}] Error writing [{}]", jobId, (id == null) ? "auto-generated ID" : id), e); IndexResponse.Builder notCreatedResponse = new IndexResponse.Builder(); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java index 9ebff4bae4d10..f112ed5053f50 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java @@ -29,6 +29,7 @@ import java.util.Arrays; import java.util.Random; import java.util.Set; +import java.util.function.Consumer; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -40,7 +41,7 @@ public class ResultsPersisterService { public static final Setting PERSIST_RESULTS_MAX_RETRIES = Setting.intSetting( "xpack.ml.persist_results_max_retries", - 15, + 20, 0, 50, Setting.Property.Dynamic, @@ -71,15 +72,27 @@ public BulkResponse indexWithRetry(String jobId, ToXContent.Params params, WriteRequest.RefreshPolicy refreshPolicy, String id, - Supplier shouldRetry) throws IOException { + Supplier shouldRetry, + Consumer msgHandler) throws IOException { BulkRequest bulkRequest = new BulkRequest().setRefreshPolicy(refreshPolicy); try (XContentBuilder content = object.toXContent(XContentFactory.jsonBuilder(), params)) { bulkRequest.add(new IndexRequest(indexName).id(id).source(content)); } - return bulkIndexWithRetry(bulkRequest, jobId, shouldRetry); + return bulkIndexWithRetry(bulkRequest, jobId, shouldRetry, msgHandler); } - public BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, String jobId, Supplier shouldRetry) { + public BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, + String jobId, + Supplier shouldRetry, + Consumer msgHandler) { + return bulkIndexWithRetry(bulkRequest, jobId, shouldRetry, msgHandler, 10); + } + + BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, + String jobId, + Supplier shouldRetry, + Consumer msgHandler, + int msgBarrier) { int currentMin = MIN_RETRY_SLEEP_MILLIS; int currentMax = MIN_RETRY_SLEEP_MILLIS; int currentAttempt = 0; @@ -102,6 +115,20 @@ public BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, String jobId, Su bulkResponse.buildFailureMessage()); } currentAttempt++; + // Before 10, most retries are fairly fast, + // After 10, the time between quickly gets on the magnitude of minutes + if (currentAttempt > msgBarrier) { + int attempt = currentAttempt; + LOGGER.warn(()-> new ParameterizedMessage("[{}] failed to index after [{}] attempts. Will attempt [{}] more times.", + jobId, + attempt, + (maxFailureRetries - attempt))); + msgHandler.accept(new ParameterizedMessage( + "failed to index after [{}] attempts. Will attempt [{}] more times.", + attempt, + (maxFailureRetries - attempt)) + .getFormattedMessage()); + } // Since we exponentially increase, we don't want force randomness to have an excessively long sleep if (currentMax < MAX_RETRY_SLEEP_MILLIS) { currentMin = currentMax; diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java index e9bd90003fe83..a46e83d2488fe 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java @@ -119,7 +119,7 @@ public void createComponents() throws Exception { auditor, JOB_ID, renormalizer, - new JobResultsPersister(client(), resultsPersisterService), + new JobResultsPersister(client(), resultsPersisterService, new AnomalyDetectionAuditor(client(), "test_node")), process, new ModelSizeStats.Builder(JOB_ID).build(), new TimingStats(JOB_ID)) { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java index f9caa5c8f5221..1a7f60cd81d54 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java @@ -20,6 +20,7 @@ import org.elasticsearch.xpack.ml.inference.ingest.InferenceProcessor; import org.elasticsearch.xpack.ml.job.persistence.JobResultsProvider; import org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister; +import org.elasticsearch.xpack.ml.notifications.AnomalyDetectionAuditor; import org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase; import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import org.junit.Before; @@ -55,7 +56,9 @@ public void createComponents() { ResultsPersisterService resultsPersisterService = new ResultsPersisterService(client(), clusterService, settings); jobResultsProvider = new JobResultsProvider(client(), settings); - jobResultsPersister = new JobResultsPersister(client(), resultsPersisterService); + jobResultsPersister = new JobResultsPersister(client(), + resultsPersisterService, + new AnomalyDetectionAuditor(client(), "test_node")); } public void testEstablishedMem_givenNoResults() throws Exception { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java index 2156f37b07340..540ee7b5adfc3 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java @@ -59,6 +59,7 @@ import org.elasticsearch.xpack.ml.job.persistence.JobResultsProvider; import org.elasticsearch.xpack.ml.job.persistence.ScheduledEventsQueryBuilder; import org.elasticsearch.xpack.ml.job.process.autodetect.params.AutodetectParams; +import org.elasticsearch.xpack.ml.notifications.AnomalyDetectionAuditor; import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import org.junit.Before; @@ -88,6 +89,7 @@ public class JobResultsProviderIT extends MlSingleNodeTestCase { private JobResultsProvider jobProvider; private ResultsPersisterService resultsPersisterService; + private AnomalyDetectionAuditor auditor; @Before public void createComponents() throws Exception { @@ -105,6 +107,7 @@ public void createComponents() throws Exception { ClusterService clusterService = new ClusterService(builder.build(), clusterSettings, tp); resultsPersisterService = new ResultsPersisterService(client(), clusterService, builder.build()); + auditor = new AnomalyDetectionAuditor(client(), "test_node"); waitForMlTemplates(); } @@ -596,7 +599,7 @@ private void indexScheduledEvents(List events) throws IOExceptio } private void indexDataCounts(DataCounts counts, String jobId) { - JobDataCountsPersister persister = new JobDataCountsPersister(client(), resultsPersisterService); + JobDataCountsPersister persister = new JobDataCountsPersister(client(), resultsPersisterService, auditor); persister.persistDataCounts(jobId, counts); } @@ -617,17 +620,17 @@ private void indexFilters(List filters) throws IOException { } private void indexModelSizeStats(ModelSizeStats modelSizeStats) { - JobResultsPersister persister = new JobResultsPersister(client(), resultsPersisterService); + JobResultsPersister persister = new JobResultsPersister(client(), resultsPersisterService, auditor); persister.persistModelSizeStats(modelSizeStats, () -> true); } private void indexModelSnapshot(ModelSnapshot snapshot) { - JobResultsPersister persister = new JobResultsPersister(client(), resultsPersisterService); + JobResultsPersister persister = new JobResultsPersister(client(), resultsPersisterService, auditor); persister.persistModelSnapshot(snapshot, WriteRequest.RefreshPolicy.IMMEDIATE, () -> true); } private void indexQuantiles(Quantiles quantiles) { - JobResultsPersister persister = new JobResultsPersister(client(), resultsPersisterService); + JobResultsPersister persister = new JobResultsPersister(client(), resultsPersisterService, auditor); persister.persistQuantiles(quantiles, () -> true); } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java index d4723f451d10c..7bf719878c157 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.xpack.core.ml.job.results.ModelPlot; import org.elasticsearch.xpack.core.ml.utils.ExponentialAverageCalculationContext; import org.elasticsearch.xpack.ml.inference.ingest.InferenceProcessor; +import org.elasticsearch.xpack.ml.notifications.AnomalyDetectionAuditor; import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; import org.mockito.ArgumentCaptor; @@ -44,6 +45,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -76,7 +78,7 @@ public void testPersistBucket_OneRecord() { AnomalyRecord record = new AnomalyRecord(JOB_ID, new Date(), 600); bucket.setRecords(Collections.singletonList(record)); - JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client), makeAuditor()); persister.bulkPersisterBuilder(JOB_ID, () -> true).persistBucket(bucket).executeRequest(); BulkRequest bulkRequest = captor.getValue(); assertEquals(2, bulkRequest.numberOfActions()); @@ -128,7 +130,7 @@ public void testPersistRecords() { typicals.add(998765.3); r1.setTypical(typicals); - JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client), makeAuditor()); persister.bulkPersisterBuilder(JOB_ID, () -> true).persistRecords(records).executeRequest(); BulkRequest bulkRequest = captor.getValue(); assertEquals(1, bulkRequest.numberOfActions()); @@ -164,7 +166,7 @@ public void testPersistInfluencers() { inf.setProbability(0.4); influencers.add(inf); - JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client), makeAuditor()); persister.bulkPersisterBuilder(JOB_ID, () -> true).persistInfluencers(influencers).executeRequest(); BulkRequest bulkRequest = captor.getValue(); assertEquals(1, bulkRequest.numberOfActions()); @@ -180,7 +182,7 @@ public void testPersistInfluencers() { public void testExecuteRequest_ClearsBulkRequest() { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); - JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client), makeAuditor()); List influencers = new ArrayList<>(); Influencer inf = new Influencer(JOB_ID, "infName1", "infValue1", new Date(), 600); @@ -197,7 +199,7 @@ public void testExecuteRequest_ClearsBulkRequest() { public void testBulkRequestExecutesWhenReachMaxDocs() { ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(captor); - JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client), makeAuditor()); JobResultsPersister.Builder bulkBuilder = persister.bulkPersisterBuilder("foo", () -> true); ModelPlot modelPlot = new ModelPlot("foo", new Date(), 123456, 0); @@ -214,7 +216,7 @@ public void testPersistTimingStats() { ArgumentCaptor bulkRequestCaptor = ArgumentCaptor.forClass(BulkRequest.class); Client client = mockClient(bulkRequestCaptor); - JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client), makeAuditor()); TimingStats timingStats = new TimingStats( "foo", 7, 1.0, 2.0, 1.23, 7.89, new ExponentialAverageCalculationContext(600.0, Instant.ofEpochMilli(123456789), 60.0)); @@ -249,7 +251,7 @@ public void testPersistTimingStats() { @SuppressWarnings({"unchecked", "rawtypes"}) public void testPersistDatafeedTimingStats() { Client client = mockClient(ArgumentCaptor.forClass(BulkRequest.class)); - JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client)); + JobResultsPersister persister = new JobResultsPersister(client, buildResultsPersisterService(client), makeAuditor()); DatafeedTimingStats timingStats = new DatafeedTimingStats( "foo", 6, 66, 666.0, new ExponentialAverageCalculationContext(600.0, Instant.ofEpochMilli(123456789), 60.0)); @@ -320,4 +322,12 @@ private ResultsPersisterService buildResultsPersisterService(Client client) { return new ResultsPersisterService(client, clusterService, Settings.EMPTY); } + + private AnomalyDetectionAuditor makeAuditor() { + AnomalyDetectionAuditor anomalyDetectionAuditor = mock(AnomalyDetectionAuditor.class); + doNothing().when(anomalyDetectionAuditor).warning(any(), any()); + doNothing().when(anomalyDetectionAuditor).info(any(), any()); + doNothing().when(anomalyDetectionAuditor).error(any(), any()); + return anomalyDetectionAuditor; + } } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporterTests.java index 124b0f7433d6f..d8a4d5aec24b2 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporterTests.java @@ -46,7 +46,7 @@ public void testGetCurrentTimingStats() { verifyZeroInteractions(bulkResultsPersister); } - public void testReporting() throws Exception { + public void testReporting() { TimingStatsReporter reporter = createReporter(new TimingStats(JOB_ID)); assertThat(reporter.getCurrentTimingStats(), equalTo(new TimingStats(JOB_ID))); @@ -65,7 +65,7 @@ public void testReporting() throws Exception { verifyNoMoreInteractions(bulkResultsPersister); } - public void testFinishReporting() throws Exception { + public void testFinishReporting() { TimingStatsReporter reporter = createReporter(new TimingStats(JOB_ID)); assertThat(reporter.getCurrentTimingStats(), equalTo(new TimingStats(JOB_ID))); @@ -87,14 +87,14 @@ public void testFinishReporting() throws Exception { verifyNoMoreInteractions(bulkResultsPersister); } - public void testFinishReporting_NoChange() throws Exception { + public void testFinishReporting_NoChange() { TimingStatsReporter reporter = createReporter(new TimingStats(JOB_ID)); reporter.finishReporting(); verifyZeroInteractions(bulkResultsPersister); } - public void testFinishReporting_WithChange() throws Exception { + public void testFinishReporting_WithChange() { TimingStatsReporter reporter = createReporter(new TimingStats(JOB_ID)); reporter.reportBucket(createBucket(10)); reporter.finishReporting(); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java index 57fb4a3b91a5e..104d7711d3b29 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java @@ -33,6 +33,8 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import static org.hamcrest.Matchers.equalTo; import static org.mockito.Matchers.any; @@ -44,6 +46,7 @@ public class ResultsPersisterServiceTests extends ESTestCase { private final String JOB_ID = "results_persister_test_job"; + private final Consumer NULL_MSG_HANDLER = (msg) -> {}; public void testBulkRequestChangeOnFailures() { IndexRequest indexRequestSuccess = new IndexRequest("my-index").id("success").source(Collections.singletonMap("data", "success")); @@ -68,7 +71,7 @@ public void testBulkRequestChangeOnFailures() { ResultsPersisterService resultsPersisterService = buildResultsPersisterService(client); - resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> true); + resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> true, NULL_MSG_HANDLER); ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); verify(client, times(2)).bulk(captor.capture()); @@ -102,7 +105,8 @@ public void testBulkRequestDoesNotRetryWhenSupplierIsFalse() { ResultsPersisterService resultsPersisterService = buildResultsPersisterService(client); - expectThrows(ElasticsearchException.class, () -> resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> false)); + expectThrows(ElasticsearchException.class, + () -> resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> false, NULL_MSG_HANDLER)); } public void testBulkRequestRetriesConfiguredAttemptNumber() { @@ -119,10 +123,47 @@ public void testBulkRequestRetriesConfiguredAttemptNumber() { ResultsPersisterService resultsPersisterService = buildResultsPersisterService(client); resultsPersisterService.setMaxFailureRetries(1); - expectThrows(ElasticsearchException.class, () -> resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> true)); + expectThrows(ElasticsearchException.class, + () -> resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> true, NULL_MSG_HANDLER)); verify(client, times(2)).bulk(any(BulkRequest.class)); } + public void testBulkRequestRetriesMsgHandlerIsCalled() { + IndexRequest indexRequestSuccess = new IndexRequest("my-index").id("success").source(Collections.singletonMap("data", "success")); + IndexRequest indexRequestFail = new IndexRequest("my-index").id("fail").source(Collections.singletonMap("data", "fail")); + BulkItemResponse successItem = new BulkItemResponse(1, + DocWriteRequest.OpType.INDEX, + new IndexResponse(new ShardId(AnomalyDetectorsIndex.jobResultsIndexPrefix() + "shared", "uuid", 1), + indexRequestSuccess.id(), + 0, + 0, + 1, + true)); + BulkItemResponse failureItem = new BulkItemResponse(2, + DocWriteRequest.OpType.INDEX, + new BulkItemResponse.Failure("my-index", "fail", new Exception("boom"))); + BulkResponse withFailure = new BulkResponse(new BulkItemResponse[]{ failureItem, successItem }, 0L); + Client client = mockClientWithResponse(withFailure, new BulkResponse(new BulkItemResponse[0], 0L)); + + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.add(indexRequestFail); + bulkRequest.add(indexRequestSuccess); + + ResultsPersisterService resultsPersisterService = buildResultsPersisterService(client); + AtomicReference msgHolder = new AtomicReference<>("not_called"); + + resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> true, msgHolder::set,0); + + ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); + verify(client, times(2)).bulk(captor.capture()); + + List requests = captor.getAllValues(); + + assertThat(requests.get(0).numberOfActions(), equalTo(2)); + assertThat(requests.get(1).numberOfActions(), equalTo(1)); + assertThat(msgHolder.get(), equalTo("failed to index after [1] attempts. Will attempt [19] more times.")); + } + @SuppressWarnings("unchecked") private Client mockClientWithResponse(BulkResponse... responses) { Client client = mock(Client.class); From 6e1a25aec496abdfe22d7b97a24993a4fe2894c2 Mon Sep 17 00:00:00 2001 From: Benjamin Trent <4357155+benwtrent@users.noreply.github.com> Date: Tue, 10 Dec 2019 08:47:41 -0500 Subject: [PATCH 15/15] addressing PR comments --- .../ml/integration/BulkFailureRetryIT.java | 4 +-- .../persistence/JobDataCountsPersister.java | 9 ++++-- .../persistence/ResultsPersisterService.java | 32 ++++++------------- .../ResultsPersisterServiceTests.java | 5 +-- 4 files changed, 21 insertions(+), 29 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java index 26357d5a782d7..6dcf306b04a39 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/BulkFailureRetryIT.java @@ -78,7 +78,7 @@ private void ensureAnomaliesWrite() throws InterruptedException { acknowledgedResponseHolder, exceptionHolder); if (exceptionHolder.get() != null) { - logger.error("FAILED TO MARK ["+ resultsIndex + "] as read-write again", exceptionHolder.get()); + fail("FAILED TO MARK ["+ resultsIndex + "] as read-write again" + exceptionHolder.get()); } } @@ -91,7 +91,7 @@ private void setAnomaliesReadOnlyBlock() throws InterruptedException { acknowledgedResponseHolder, exceptionHolder); if (exceptionHolder.get() != null) { - logger.error("FAILED TO MARK ["+ resultsIndex + "] as read-ONLY", exceptionHolder.get()); + fail("FAILED TO MARK ["+ resultsIndex + "] as read-ONLY: " + exceptionHolder.get()); } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java index 0cd81333c9d40..4925d2c2ac035 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataCountsPersister.java @@ -18,6 +18,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.DataCounts; +import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; import org.elasticsearch.xpack.ml.notifications.AnomalyDetectionAuditor; import org.elasticsearch.xpack.ml.utils.persistence.ResultsPersisterService; @@ -67,9 +68,9 @@ public void persistDataCounts(String jobId, DataCounts counts) { () -> true, (msg) -> auditor.warning(jobId, "Job data_counts " + msg)); } catch (IOException ioe) { - logger.warn(() -> new ParameterizedMessage("[{}] Error serialising DataCounts stats", jobId), ioe); + logger.error(() -> new ParameterizedMessage("[{}] Failed writing data_counts stats", jobId), ioe); } catch (Exception ex) { - logger.warn(() -> new ParameterizedMessage("[{}] Failed to persist DataCounts stats", jobId), ex); + logger.error(() -> new ParameterizedMessage("[{}] Failed persisting data_counts stats", jobId), ex); } } @@ -100,7 +101,9 @@ public void onFailure(Exception e) { } }); } catch (IOException ioe) { - logger.warn(() -> new ParameterizedMessage("[{}] Error serialising DataCounts stats", jobId), ioe); + String msg = new ParameterizedMessage("[{}] Failed writing data_counts stats", jobId).getFormattedMessage(); + logger.error(msg, ioe); + listener.onFailure(ExceptionsHelper.serverError(msg, ioe)); } } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java index f112ed5053f50..6aad7d6a4f9b8 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterService.java @@ -19,6 +19,7 @@ import org.elasticsearch.common.Randomness; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -85,14 +86,6 @@ public BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, String jobId, Supplier shouldRetry, Consumer msgHandler) { - return bulkIndexWithRetry(bulkRequest, jobId, shouldRetry, msgHandler, 10); - } - - BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, - String jobId, - Supplier shouldRetry, - Consumer msgHandler, - int msgBarrier) { int currentMin = MIN_RETRY_SLEEP_MILLIS; int currentMax = MIN_RETRY_SLEEP_MILLIS; int currentAttempt = 0; @@ -115,20 +108,6 @@ BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, bulkResponse.buildFailureMessage()); } currentAttempt++; - // Before 10, most retries are fairly fast, - // After 10, the time between quickly gets on the magnitude of minutes - if (currentAttempt > msgBarrier) { - int attempt = currentAttempt; - LOGGER.warn(()-> new ParameterizedMessage("[{}] failed to index after [{}] attempts. Will attempt [{}] more times.", - jobId, - attempt, - (maxFailureRetries - attempt))); - msgHandler.accept(new ParameterizedMessage( - "failed to index after [{}] attempts. Will attempt [{}] more times.", - attempt, - (maxFailureRetries - attempt)) - .getFormattedMessage()); - } // Since we exponentially increase, we don't want force randomness to have an excessively long sleep if (currentMax < MAX_RETRY_SLEEP_MILLIS) { currentMin = currentMax; @@ -140,6 +119,15 @@ BulkResponse bulkIndexWithRetry(BulkRequest bulkRequest, // so that not all bulk requests rest for the same amount of time int randBound = 1 + (currentMax - currentMin); int randSleep = currentMin + random.nextInt(randBound); + { + String msg = new ParameterizedMessage( + "failed to index after [{}] attempts. Will attempt again in [{}].", + currentAttempt, + TimeValue.timeValueMillis(randSleep).getStringRep()) + .getFormattedMessage(); + LOGGER.warn(()-> new ParameterizedMessage("[{}] {}", jobId, msg)); + msgHandler.accept(msg); + } // We should only retry the docs that failed. bulkRequest = buildNewRequestFromFailures(bulkRequest, bulkResponse); try { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java index 104d7711d3b29..9d5b922ccb0c6 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java @@ -36,6 +36,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; @@ -152,7 +153,7 @@ public void testBulkRequestRetriesMsgHandlerIsCalled() { ResultsPersisterService resultsPersisterService = buildResultsPersisterService(client); AtomicReference msgHolder = new AtomicReference<>("not_called"); - resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> true, msgHolder::set,0); + resultsPersisterService.bulkIndexWithRetry(bulkRequest, JOB_ID, () -> true, msgHolder::set); ArgumentCaptor captor = ArgumentCaptor.forClass(BulkRequest.class); verify(client, times(2)).bulk(captor.capture()); @@ -161,7 +162,7 @@ public void testBulkRequestRetriesMsgHandlerIsCalled() { assertThat(requests.get(0).numberOfActions(), equalTo(2)); assertThat(requests.get(1).numberOfActions(), equalTo(1)); - assertThat(msgHolder.get(), equalTo("failed to index after [1] attempts. Will attempt [19] more times.")); + assertThat(msgHolder.get(), containsString("failed to index after [1] attempts. Will attempt again in")); } @SuppressWarnings("unchecked")