From bb39f558896f1bd15dc7628e35a89d355fd1b6bc Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Wed, 8 Aug 2018 17:18:01 +0200 Subject: [PATCH 01/20] refactor rollup job --- .../xpack/core/indexing/IndexerJob.java | 14 + .../job => indexing}/IndexerState.java | 2 +- .../xpack/core/indexing/IndexerStats.java | 154 ++++++++ .../xpack/core/indexing/Iteration.java | 39 ++ .../xpack/core/indexing/IterativeIndexer.java | 349 ++++++++++++++++++ .../xpack/core/rollup/job/RollupJob.java | 9 +- .../xpack/core/rollup/job/RollupJobStats.java | 138 +------ .../core/rollup/job/RollupJobStatus.java | 1 + .../IndexerStateEnumTests.java | 2 +- .../job/JobWrapperSerializingTests.java | 1 + .../core/rollup/job/RollupJobStatusTests.java | 1 + .../xpack/rollup/job/RollupIndexer.java | 332 ++--------------- .../xpack/rollup/job/RollupJobTask.java | 4 +- .../job/RollupIndexerIndexingTests.java | 2 +- .../rollup/job/RollupIndexerStateTests.java | 2 +- .../xpack/rollup/job/RollupJobTaskTests.java | 2 +- 16 files changed, 613 insertions(+), 439 deletions(-) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJob.java rename x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/{rollup/job => indexing}/IndexerState.java (97%) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerStats.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/Iteration.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java rename x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/{rollup/job => indexing}/IndexerStateEnumTests.java (98%) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJob.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJob.java new file mode 100644 index 0000000000000..09d01ef891de8 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJob.java @@ -0,0 +1,14 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.core.indexing; + +import org.elasticsearch.xpack.core.XPackPlugin; + +public interface IndexerJob extends XPackPlugin.XPackPersistentTaskParams { + + String getId(); +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/IndexerState.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerState.java similarity index 97% rename from x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/IndexerState.java rename to x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerState.java index 6e211c1df9e3e..1b6b9a943cba2 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/IndexerState.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerState.java @@ -3,7 +3,7 @@ * or more contributor license agreements. Licensed under the Elastic License; * you may not use this file except in compliance with the Elastic License. */ -package org.elasticsearch.xpack.core.rollup.job; +package org.elasticsearch.xpack.core.indexing; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerStats.java new file mode 100644 index 0000000000000..cada682a88582 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerStats.java @@ -0,0 +1,154 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.core.indexing; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; + +/** + * This class holds the runtime statistics of a job. The stats are not used by any internal process + * and are only for external monitoring/reference. Statistics are not persisted with the job, so if the + * allocated task is shutdown/restarted on a different node all the stats will reset. + */ +public class IndexerStats implements ToXContentObject, Writeable { + + public static final ParseField NAME = new ParseField("job_stats"); + + private static ParseField NUM_PAGES = new ParseField("pages_processed"); + private static ParseField NUM_INPUT_DOCUMENTS = new ParseField("source_documents_processed"); + private static ParseField NUM_OUTPUT_DOCUMENTS = new ParseField("documents_indexed"); + private static ParseField NUM_INVOCATIONS = new ParseField("trigger_count"); + + private long numPages = 0; + private long numInputDocuments = 0; + private long numOuputDocuments = 0; + private long numInvocations = 0; + + public static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>(NAME.getPreferredName(), + args -> new IndexerStats((long) args[0], (long) args[1], (long) args[2], (long) args[3])); + + static { + PARSER.declareLong(constructorArg(), NUM_PAGES); + PARSER.declareLong(constructorArg(), NUM_INPUT_DOCUMENTS); + PARSER.declareLong(constructorArg(), NUM_OUTPUT_DOCUMENTS); + PARSER.declareLong(constructorArg(), NUM_INVOCATIONS); + } + + public IndexerStats() { + } + + public IndexerStats(long numPages, long numDocuments, long numRollups, long numInvocations) { + this.numPages = numPages; + this.numInputDocuments = numDocuments; + this.numOuputDocuments = numRollups; + this.numInvocations = numInvocations; + } + + public IndexerStats(StreamInput in) throws IOException { + this.numPages = in.readVLong(); + this.numInputDocuments = in.readVLong(); + this.numOuputDocuments = in.readVLong(); + this.numInvocations = in.readVLong(); + } + + public long getNumPages() { + return numPages; + } + + public long getNumDocuments() { + return numInputDocuments; + } + + public long getNumInvocations() { + return numInvocations; + } + + public long getNumRollups() { + return numOuputDocuments; + } + + public void incrementNumPages(long n) { + assert(n >= 0); + numPages += n; + } + + public void incrementNumDocuments(long n) { + assert(n >= 0); + numInputDocuments += n; + } + + public void incrementNumInvocations(long n) { + assert(n >= 0); + numInvocations += n; + } + + public void incrementNumRollups(long n) { + assert(n >= 0); + numOuputDocuments += n; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(numPages); + out.writeVLong(numInputDocuments); + out.writeVLong(numOuputDocuments); + out.writeVLong(numInvocations); + } + + public static IndexerStats fromXContent(XContentParser parser) { + try { + return PARSER.parse(parser, null); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(NUM_PAGES.getPreferredName(), numPages); + builder.field(NUM_INPUT_DOCUMENTS.getPreferredName(), numInputDocuments); + builder.field(NUM_OUTPUT_DOCUMENTS.getPreferredName(), numOuputDocuments); + builder.field(NUM_INVOCATIONS.getPreferredName(), numInvocations); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + IndexerStats that = (IndexerStats) other; + + return Objects.equals(this.numPages, that.numPages) + && Objects.equals(this.numInputDocuments, that.numInputDocuments) + && Objects.equals(this.numOuputDocuments, that.numOuputDocuments) + && Objects.equals(this.numInvocations, that.numInvocations); + } + + @Override + public int hashCode() { + return Objects.hash(numPages, numInputDocuments, numOuputDocuments, numInvocations); + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/Iteration.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/Iteration.java new file mode 100644 index 0000000000000..aa2d592e475a3 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/Iteration.java @@ -0,0 +1,39 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.core.indexing; + +import org.elasticsearch.action.index.IndexRequest; + +import java.util.List; + +public class Iteration { + + private final boolean isDone; + + private final JobPosition position; + + private final List toIndex; + + public Iteration(List toIndex, JobPosition position, boolean isDone) { + this.toIndex = toIndex; + this.position = position; + this.isDone = isDone; + } + + public boolean isDone() { + return isDone; + } + + public JobPosition getPosition() { + return position; + } + + public List getToIndex() { + return toIndex; + } + +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java new file mode 100644 index 0000000000000..632ed63022bac --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java @@ -0,0 +1,349 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.core.indexing; + +import org.apache.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicReference; + +public abstract class IterativeIndexer { + private static final Logger logger = Logger.getLogger(IterativeIndexer.class.getName()); + + protected final IndexerStats stats; + + protected final AtomicReference state; + protected final AtomicReference position; + protected final Executor executor; + + protected IterativeIndexer(Executor executor, AtomicReference initialState, JobPosition initialPosition) { + this.executor = executor; + this.state = initialState; + this.position = new AtomicReference<>(initialPosition); + this.stats = newJobStats(); + } + + /** + * Get the current state of the indexer. + */ + public IndexerState getState() { + return state.get(); + } + + /** + * Get the current position of the indexer. + */ + public JobPosition getPosition() { + return position.get(); + } + + /** + * Get the stats of this indexer. + */ + public IndexerStats getStats() { + return stats; + } + + /** + * Sets the internal state to {@link IndexerState#STARTED} if the previous state + * was {@link IndexerState#STOPPED}. Setting the state to STARTED allows a job + * to run in the background when {@link #maybeTriggerAsyncJob(long)} is called. + * + * @return The new state for the indexer (STARTED, INDEXING or ABORTING if the + * job was already aborted). + */ + public synchronized IndexerState start() { + state.compareAndSet(IndexerState.STOPPED, IndexerState.STARTED); + return state.get(); + } + + /** + * Sets the internal state to {@link IndexerState#STOPPING} if an async job is + * running in the background and in such case {@link #onFinish()} will be called + * as soon as the background job detects that the indexer is stopped. If there + * is no job running when this function is called, the state is directly set to + * {@link IndexerState#STOPPED} and {@link #onFinish()} will never be called. + * + * @return The new state for the indexer (STOPPED, STOPPING or ABORTING if the + * job was already aborted). + */ + public synchronized IndexerState stop() { + IndexerState currentState = state.updateAndGet(previousState -> { + if (previousState == IndexerState.INDEXING) { + return IndexerState.STOPPING; + } else if (previousState == IndexerState.STARTED) { + return IndexerState.STOPPED; + } else { + return previousState; + } + }); + return currentState; + } + + /** + * Sets the internal state to {@link IndexerState#ABORTING}. It returns false if + * an async job is running in the background and in such case {@link #onAbort} + * will be called as soon as the background job detects that the indexer is + * aborted. If there is no job running when this function is called, it returns + * true and {@link #onAbort()} will never be called. + * + * @return true if the indexer is aborted, false if a background job is running + * and abort is delayed. + */ + public synchronized boolean abort() { + IndexerState prevState = state.getAndUpdate((prev) -> IndexerState.ABORTING); + return prevState == IndexerState.STOPPED || prevState == IndexerState.STARTED; + } + + /** + * Triggers a background job that builds the rollup index asynchronously iff + * there is no other job that runs and the indexer is started + * ({@link IndexerState#STARTED}. + * + * @param now + * The current time in milliseconds (used to limit the job to + * complete buckets) + * @return true if a job has been triggered, false otherwise + */ + public synchronized boolean maybeTriggerAsyncJob(long now) { + final IndexerState currentState = state.get(); + switch (currentState) { + case INDEXING: + case STOPPING: + case ABORTING: + logger.warn("Schedule was triggered for job [" + getJobId() + "], but prior indexer is still running."); + return false; + + case STOPPED: + logger.debug("Schedule was triggered for job [" + getJobId() + "] but job is stopped. Ignoring trigger."); + return false; + + case STARTED: + logger.debug("Schedule was triggered for job [" + getJobId() + "], state: [" + currentState + "]"); + stats.incrementNumInvocations(1); + onStart(now); + + if (state.compareAndSet(IndexerState.STARTED, IndexerState.INDEXING)) { + // fire off the search. Note this is async, the method will return from here + executor.execute(() -> doNextSearch(buildSearchRequest(), + ActionListener.wrap(this::onSearchResponse, exc -> finishWithFailure(exc)))); + logger.debug("Beginning to index [" + getJobId() + "], state: [" + currentState + "]"); + return true; + } else { + logger.debug("Could not move from STARTED to INDEXING state because current state is [" + state.get() + "]"); + return false; + } + + default: + logger.warn("Encountered unexpected state [" + currentState + "] while indexing"); + throw new IllegalStateException("Job encountered an illegal state [" + currentState + "]"); + } + } + + protected abstract IndexerStats newJobStats(); + + protected abstract String getJobId(); + + protected abstract Iteration doProcess(SearchResponse searchResponse); + + protected abstract SearchRequest buildSearchRequest(); + + protected abstract void onStart(long now); + + /** + * Executes the {@link SearchRequest} and calls nextPhase with the + * response or the exception if an error occurs. + * + * @param request + * The search request to execute + * @param nextPhase + * Listener for the next phase + */ + protected abstract void doNextSearch(SearchRequest request, ActionListener nextPhase); + + /** + * Executes the {@link BulkRequest} and calls nextPhase with the + * response or the exception if an error occurs. + * + * @param request + * The bulk request to execute + * @param nextPhase + * Listener for the next phase + */ + protected abstract void doNextBulk(BulkRequest request, ActionListener nextPhase); + + /** + * Called periodically during the execution of a background job. Implementation + * should persists the state somewhere and continue the execution asynchronously + * using next. + * + * @param state + * The current state of the indexer + * @param position + * The current position of the indexer + * @param next + * Runnable for the next phase + */ + protected abstract void doSaveState(IndexerState state, JobPosition position, Runnable next); + + /** + * Called when a failure occurs in an async job causing the execution to stop. + * + * @param exc + * The exception + */ + protected abstract void onFailure(Exception exc); + + /** + * Called when a background job finishes. + */ + protected abstract void onFinish(); + + /** + * Called when a background job detects that the indexer is aborted causing the + * async execution to stop. + */ + protected abstract void onAbort(); + + private void finishWithFailure(Exception exc) { + doSaveState(finishAndSetState(), position.get(), () -> onFailure(exc)); + } + + private IndexerState finishAndSetState() { + return state.updateAndGet(prev -> { + switch (prev) { + case INDEXING: + // ready for another job + return IndexerState.STARTED; + + case STOPPING: + // must be started again + return IndexerState.STOPPED; + + case ABORTING: + // abort and exit + onAbort(); + return IndexerState.ABORTING; // This shouldn't matter, since onAbort() will kill the task first + + case STOPPED: + // No-op. Shouldn't really be possible to get here (should have to go through + // STOPPING + // first which will be handled) but is harmless to no-op and we don't want to + // throw exception here + return IndexerState.STOPPED; + + default: + // any other state is unanticipated at this point + throw new IllegalStateException("Rollup job encountered an illegal state [" + prev + "]"); + } + }); + } + + private void onSearchResponse(SearchResponse searchResponse) { + try { + if (checkState(getState()) == false) { + return; + } + if (searchResponse.getShardFailures().length != 0) { + throw new RuntimeException("Shard failures encountered while running indexer for job [" + getJobId() + "]: " + + Arrays.toString(searchResponse.getShardFailures())); + } + + stats.incrementNumPages(1); + Iteration iteration = doProcess(searchResponse); + + if (iteration.isDone()) { + logger.debug("Finished indexing for job [" + getJobId() + "], saving state and shutting down."); + + // Change state first, then try to persist. This prevents in-progress + // STOPPING/ABORTING from + // being persisted as STARTED but then stop the job + doSaveState(finishAndSetState(), position.get(), this::onFinish); + return; + } + + final List docs = iteration.getToIndex(); + final BulkRequest bulkRequest = new BulkRequest(); + docs.forEach(bulkRequest::add); + assert bulkRequest.requests().size() > 0; + + doNextBulk(bulkRequest, ActionListener.wrap(bulkResponse -> { + // TODO we should check items in the response and move after accordingly to + // resume the failing buckets ? + if (bulkResponse.hasFailures()) { + logger.warn("Error while attempting to bulk index documents: " + bulkResponse.buildFailureMessage()); + } + stats.incrementNumRollups(bulkResponse.getItems().length); + if (checkState(getState()) == false) { + return; + } + + JobPosition newPosition = iteration.getPosition(); + position.set(newPosition); + + onBulkResponse(bulkResponse, newPosition); + }, exc -> finishWithFailure(exc))); + } catch (Exception e) { + finishWithFailure(e); + } + } + + private void onBulkResponse(BulkResponse response, JobPosition position) { + try { + + ActionListener listener = ActionListener.wrap(this::onSearchResponse, this::finishWithFailure); + // TODO probably something more intelligent than every-50 is needed + if (stats.getNumPages() > 0 && stats.getNumPages() % 50 == 0) { + doSaveState(IndexerState.INDEXING, position, () -> doNextSearch(buildSearchRequest(), listener)); + } else { + doNextSearch(buildSearchRequest(), listener); + } + } catch (Exception e) { + finishWithFailure(e); + } + } + + /** + * Checks the {@link IndexerState} and returns false if the execution should be + * stopped. + */ + private boolean checkState(IndexerState currentState) { + switch (currentState) { + case INDEXING: + // normal state; + return true; + + case STOPPING: + logger.info("Indexer job encountered [" + IndexerState.STOPPING + "] state, halting indexer."); + doSaveState(finishAndSetState(), getPosition(), () -> { + }); + return false; + + case STOPPED: + return false; + + case ABORTING: + logger.info("Requested shutdown of indexer for job [" + getJobId() + "]"); + onAbort(); + return false; + + default: + // Anything other than indexing, aborting or stopping is unanticipated + logger.warn("Encountered unexpected state [" + currentState + "] while indexing"); + throw new IllegalStateException("Rollup job encountered an illegal state [" + currentState + "]"); + } + } + +} \ No newline at end of file diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJob.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJob.java index 94306966a34da..34d4697c5a43e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJob.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJob.java @@ -14,7 +14,7 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.xpack.core.XPackPlugin; +import org.elasticsearch.xpack.core.indexing.IndexerJob; import java.io.IOException; import java.util.Collections; @@ -26,7 +26,7 @@ * It holds the config (RollupJobConfig) and a map of authentication headers. Only RollupJobConfig * is ever serialized to the user, so the headers should never leak */ -public class RollupJob extends AbstractDiffable implements XPackPlugin.XPackPersistentTaskParams { +public class RollupJob extends AbstractDiffable implements IndexerJob { public static final String NAME = "xpack/rollup/job"; @@ -116,4 +116,9 @@ public int hashCode() { public Version getMinimalSupportedVersion() { return Version.V_6_3_0; } + + @Override + public String getId() { + return getConfig().getId(); + } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java index 06cfb520af552..1c9e2dd93953a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java @@ -5,152 +5,28 @@ */ package org.elasticsearch.xpack.core.rollup.job; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ToXContentObject; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.xpack.core.indexing.IndexerStats; import java.io.IOException; -import java.util.Objects; - -import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; /** * This class holds the runtime statistics of a job. The stats are not used by any internal process * and are only for external monitoring/reference. Statistics are not persisted with the job, so if the * allocated task is shutdown/restarted on a different node all the stats will reset. */ -public class RollupJobStats implements ToXContentObject, Writeable { - - public static final ParseField NAME = new ParseField("job_stats"); - - private static ParseField NUM_PAGES = new ParseField("pages_processed"); - private static ParseField NUM_DOCUMENTS = new ParseField("documents_processed"); - private static ParseField NUM_ROLLUPS = new ParseField("rollups_indexed"); - private static ParseField NUM_INVOCATIONS = new ParseField("trigger_count"); - - private long numPages = 0; - private long numDocuments = 0; - private long numRollups = 0; - private long numInvocations = 0; - - public static final ConstructingObjectParser PARSER = - new ConstructingObjectParser<>(NAME.getPreferredName(), - args -> new RollupJobStats((long) args[0], (long) args[1], (long) args[2], (long) args[3])); - - static { - PARSER.declareLong(constructorArg(), NUM_PAGES); - PARSER.declareLong(constructorArg(), NUM_DOCUMENTS); - PARSER.declareLong(constructorArg(), NUM_ROLLUPS); - PARSER.declareLong(constructorArg(), NUM_INVOCATIONS); - } - - public RollupJobStats() { - } - public RollupJobStats(long numPages, long numDocuments, long numRollups, long numInvocations) { - this.numPages = numPages; - this.numDocuments = numDocuments; - this.numRollups = numRollups; - this.numInvocations = numInvocations; - } +// todo: kept for now, consider removing it +public class RollupJobStats extends IndexerStats { public RollupJobStats(StreamInput in) throws IOException { - this.numPages = in.readVLong(); - this.numDocuments = in.readVLong(); - this.numRollups = in.readVLong(); - this.numInvocations = in.readVLong(); - } - - public long getNumPages() { - return numPages; - } - - public long getNumDocuments() { - return numDocuments; + super(in); } - public long getNumInvocations() { - return numInvocations; - } - - public long getNumRollups() { - return numRollups; - } - - public void incrementNumPages(long n) { - assert(n >= 0); - numPages += n; - } - - public void incrementNumDocuments(long n) { - assert(n >= 0); - numDocuments += n; - } - - public void incrementNumInvocations(long n) { - assert(n >= 0); - numInvocations += n; - } - - public void incrementNumRollups(long n) { - assert(n >= 0); - numRollups += n; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeVLong(numPages); - out.writeVLong(numDocuments); - out.writeVLong(numRollups); - out.writeVLong(numInvocations); - } - - public static RollupJobStats fromXContent(XContentParser parser) { - try { - return PARSER.parse(parser, null); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - builder.field(NUM_PAGES.getPreferredName(), numPages); - builder.field(NUM_DOCUMENTS.getPreferredName(), numDocuments); - builder.field(NUM_ROLLUPS.getPreferredName(), numRollups); - builder.field(NUM_INVOCATIONS.getPreferredName(), numInvocations); - builder.endObject(); - return builder; - } - - @Override - public boolean equals(Object other) { - if (this == other) { - return true; - } - - if (other == null || getClass() != other.getClass()) { - return false; - } - - RollupJobStats that = (RollupJobStats) other; - - return Objects.equals(this.numPages, that.numPages) - && Objects.equals(this.numDocuments, that.numDocuments) - && Objects.equals(this.numRollups, that.numRollups) - && Objects.equals(this.numInvocations, that.numInvocations); - } - - @Override - public int hashCode() { - return Objects.hash(numPages, numDocuments, numRollups, numInvocations); + public RollupJobStats() { + super(); } + } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatus.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatus.java index 640385c9c80d5..0a2f046907c80 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatus.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatus.java @@ -17,6 +17,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.persistent.PersistentTaskState; import org.elasticsearch.tasks.Task; +import org.elasticsearch.xpack.core.indexing.IndexerState; import java.io.IOException; import java.util.HashMap; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/IndexerStateEnumTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStateEnumTests.java similarity index 98% rename from x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/IndexerStateEnumTests.java rename to x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStateEnumTests.java index ec17a37e23b2b..329800c2f1a24 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/IndexerStateEnumTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStateEnumTests.java @@ -3,7 +3,7 @@ * or more contributor license agreements. Licensed under the Elastic License; * you may not use this file except in compliance with the Elastic License. */ -package org.elasticsearch.xpack.core.rollup.job; +package org.elasticsearch.xpack.core.indexing; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java index a0df63bc38dde..388831cd7dbf7 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java @@ -8,6 +8,7 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractSerializingTestCase; +import org.elasticsearch.xpack.core.indexing.IndexerState; import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers; import org.elasticsearch.xpack.core.rollup.action.GetRollupJobsAction; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatusTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatusTests.java index 2c802a7e41dc3..f46bda788bf5b 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatusTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatusTests.java @@ -8,6 +8,7 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractSerializingTestCase; +import org.elasticsearch.xpack.core.indexing.IndexerState; import java.util.HashMap; import java.util.Map; diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java index 87294706b3b7d..6696c7f923048 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java @@ -6,10 +6,6 @@ package org.elasticsearch.xpack.rollup.job; import org.apache.log4j.Logger; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.unit.TimeValue; @@ -20,16 +16,17 @@ import org.elasticsearch.search.aggregations.bucket.composite.CompositeValuesSourceBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.xpack.core.indexing.IndexerState; +import org.elasticsearch.xpack.core.indexing.Iteration; +import org.elasticsearch.xpack.core.indexing.IterativeIndexer; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.GroupConfig; -import org.elasticsearch.xpack.core.rollup.job.IndexerState; import org.elasticsearch.xpack.core.rollup.job.RollupJob; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; import org.elasticsearch.xpack.core.rollup.job.RollupJobStats; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -45,18 +42,13 @@ * aborted while a job is running. The indexer must be started ({@link #start()} to allow a background job to run when * {@link #maybeTriggerAsyncJob(long)} is called. {@link #stop()} can be used to stop the background job without aborting the indexer. */ -public abstract class RollupIndexer { +public abstract class RollupIndexer extends IterativeIndexer > { private static final Logger logger = Logger.getLogger(RollupIndexer.class.getName()); static final String AGGREGATION_NAME = RollupField.NAME; private final RollupJob job; - private final RollupJobStats stats; - private final AtomicReference state; - private final AtomicReference> position; - private final Executor executor; protected final AtomicBoolean upgradedDocumentID; - private final CompositeAggregationBuilder compositeBuilder; private long maxBoundary; @@ -68,83 +60,15 @@ public abstract class RollupIndexer { * @param initialPosition The last indexed bucket of the task */ RollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, - Map initialPosition, AtomicBoolean upgradedDocumentID) { - this.executor = executor; + super(executor, initialState, initialPosition); this.job = job; - this.stats = new RollupJobStats(); - this.state = initialState; - this.position = new AtomicReference<>(initialPosition); this.compositeBuilder = createCompositeBuilder(job.getConfig()); this.upgradedDocumentID = upgradedDocumentID; } - /** - * Executes the {@link SearchRequest} and calls nextPhase with the response - * or the exception if an error occurs. - * - * @param request The search request to execute - * @param nextPhase Listener for the next phase - */ - protected abstract void doNextSearch(SearchRequest request, ActionListener nextPhase); - - /** - * Executes the {@link BulkRequest} and calls nextPhase with the response - * or the exception if an error occurs. - * - * @param request The bulk request to execute - * @param nextPhase Listener for the next phase - */ - protected abstract void doNextBulk(BulkRequest request, ActionListener nextPhase); - - /** - * Called periodically during the execution of a background job. Implementation should - * persists the state somewhere and continue the execution asynchronously using next. - * - * @param state The current state of the indexer - * @param position The current position of the indexer - * @param next Runnable for the next phase - */ - protected abstract void doSaveState(IndexerState state, Map position, Runnable next); - - /** - * Called when a failure occurs in an async job causing the execution to stop. - * @param exc The exception - */ - protected abstract void onFailure(Exception exc); - - /** - * Called when a background job finishes. - */ - protected abstract void onFinish(); - - /** - * Called when a background job detects that the indexer is aborted causing the async execution - * to stop. - */ - protected abstract void onAbort(); - - /** - * Get the current state of the indexer. - */ - public IndexerState getState() { - return state.get(); - } - - /** - * Get the current position of the indexer. - */ - public Map getPosition() { - return position.get(); - } - - /** - * Get the stats of this indexer. - */ - public RollupJobStats getStats() { - return stats; - } - - /** + @Override + protected RollupJobStats newJobStats() { + return new RollupJobStats(); * Returns if this job has upgraded it's ID scheme yet or not */ public boolean isUpgradedDocumentID() { @@ -152,228 +76,30 @@ public boolean isUpgradedDocumentID() { } /** - * Sets the internal state to {@link IndexerState#STARTED} if the previous state was {@link IndexerState#STOPPED}. Setting the state to - * STARTED allows a job to run in the background when {@link #maybeTriggerAsyncJob(long)} is called. - * @return The new state for the indexer (STARTED, INDEXING or ABORTING if the job was already aborted). - */ - public synchronized IndexerState start() { - state.compareAndSet(IndexerState.STOPPED, IndexerState.STARTED); - return state.get(); - } - - /** - * Sets the internal state to {@link IndexerState#STOPPING} if an async job is running in the background and in such case - * {@link #onFinish()} will be called as soon as the background job detects that the indexer is stopped. If there is no job running when - * this function is called, the state is directly set to {@link IndexerState#STOPPED} and {@link #onFinish()} will never be called. - * @return The new state for the indexer (STOPPED, STOPPING or ABORTING if the job was already aborted). - */ - public synchronized IndexerState stop() { - IndexerState currentState = state.updateAndGet(previousState -> { - if (previousState == IndexerState.INDEXING) { - return IndexerState.STOPPING; - } else if (previousState == IndexerState.STARTED) { - return IndexerState.STOPPED; - } else { - return previousState; - } - }); - return currentState; - } - - /** - * Sets the internal state to {@link IndexerState#ABORTING}. It returns false if an async job is running in the background and in such - * case {@link #onAbort} will be called as soon as the background job detects that the indexer is aborted. If there is no job running - * when this function is called, it returns true and {@link #onAbort()} will never be called. - * @return true if the indexer is aborted, false if a background job is running and abort is delayed. - */ - public synchronized boolean abort() { - IndexerState prevState = state.getAndUpdate((prev) -> IndexerState.ABORTING); - return prevState == IndexerState.STOPPED || prevState == IndexerState.STARTED; } - - /** - * Triggers a background job that builds the rollup index asynchronously iff there is no other job that runs - * and the indexer is started ({@link IndexerState#STARTED}. - * - * @param now The current time in milliseconds (used to limit the job to complete buckets) - * @return true if a job has been triggered, false otherwise - */ - public synchronized boolean maybeTriggerAsyncJob(long now) { - final IndexerState currentState = state.get(); - switch (currentState) { - case INDEXING: - case STOPPING: - case ABORTING: - logger.warn("Schedule was triggered for rollup job [" + job.getConfig().getId() + "], but prior indexer is still running."); - return false; - - case STOPPED: - logger.debug("Schedule was triggered for rollup job [" + job.getConfig().getId() - + "] but job is stopped. Ignoring trigger."); - return false; - - case STARTED: - logger.debug("Schedule was triggered for rollup job [" + job.getConfig().getId() + "], state: [" + currentState + "]"); - // Only valid time to start indexing is when we are STARTED but not currently INDEXING. - stats.incrementNumInvocations(1); - - // rounds the current time to its current bucket based on the date histogram interval. - // this is needed to exclude buckets that can still receive new documents. - DateHistogramGroupConfig dateHisto = job.getConfig().getGroupConfig().getDateHistogram(); - long rounded = dateHisto.createRounding().round(now); - if (dateHisto.getDelay() != null) { - // if the job has a delay we filter all documents that appear before it. - maxBoundary = rounded - TimeValue.parseTimeValue(dateHisto.getDelay().toString(), "").millis(); - } else { - maxBoundary = rounded; - } - - if (state.compareAndSet(IndexerState.STARTED, IndexerState.INDEXING)) { - // fire off the search. Note this is async, the method will return from here - executor.execute(() -> doNextSearch(buildSearchRequest(), - ActionListener.wrap(this::onSearchResponse, exc -> finishWithFailure(exc)))); - logger.debug("Beginning to rollup [" + job.getConfig().getId() + "], state: [" + currentState + "]"); - return true; - } else { - logger.debug("Could not move from STARTED to INDEXING state because current state is [" + state.get() + "]"); - return false; - } - - default: - logger.warn("Encountered unexpected state [" + currentState + "] while indexing"); - throw new IllegalStateException("Rollup job encountered an illegal state [" + currentState + "]"); - } + + @Override + protected String getJobId() { + return job.getId(); } - /** - * Checks the {@link IndexerState} and returns false if the execution - * should be stopped. - */ - private boolean checkState(IndexerState currentState) { - switch (currentState) { - case INDEXING: - // normal state; - return true; - - case STOPPING: - logger.info("Rollup job encountered [" + IndexerState.STOPPING + "] state, halting indexer."); - doSaveState(finishAndSetState(), getPosition(), () -> {}); - return false; - - case STOPPED: - return false; - - case ABORTING: - logger.info("Requested shutdown of indexer for job [" + job.getConfig().getId() + "]"); - onAbort(); - return false; - - default: - // Anything other than indexing, aborting or stopping is unanticipated - logger.warn("Encountered unexpected state [" + currentState + "] while indexing"); - throw new IllegalStateException("Rollup job encountered an illegal state [" + currentState + "]"); + @Override + protected void onStart(long now) { + // this is needed to exclude buckets that can still receive new documents. + DateHistoGroupConfig dateHisto = job.getConfig().getGroupConfig().getDateHisto(); + long rounded = dateHisto.createRounding().round(now); + if (dateHisto.getDelay() != null) { + // if the job has a delay we filter all documents that appear before it. + maxBoundary = rounded - TimeValue.parseTimeValue(dateHisto.getDelay().toString(), "").millis(); + } else { + maxBoundary = rounded; } } - - private void onBulkResponse(BulkResponse response, Map after) { - // TODO we should check items in the response and move after accordingly to resume the failing buckets ? - stats.incrementNumRollups(response.getItems().length); - if (response.hasFailures()) { - logger.warn("Error while attempting to bulk index rollup documents: " + response.buildFailureMessage()); - } - try { - if (checkState(getState()) == false) { - return ; - } - position.set(after); - ActionListener listener = ActionListener.wrap(this::onSearchResponse, this::finishWithFailure); - // TODO probably something more intelligent than every-50 is needed - if (stats.getNumPages() > 0 && stats.getNumPages() % 50 == 0) { - doSaveState(IndexerState.INDEXING, after, () -> doNextSearch(buildSearchRequest(), listener)); - } else { - doNextSearch(buildSearchRequest(), listener); - } - } catch (Exception e) { - finishWithFailure(e); - } - } - - private void onSearchResponse(SearchResponse searchResponse) { - try { - if (checkState(getState()) == false) { - return ; - } - if (searchResponse.getShardFailures().length != 0) { - throw new RuntimeException("Shard failures encountered while running indexer for rollup job [" - + job.getConfig().getId() + "]: " + Arrays.toString(searchResponse.getShardFailures())); - } - final CompositeAggregation response = searchResponse.getAggregations().get(AGGREGATION_NAME); - if (response == null) { - throw new IllegalStateException("Missing composite response for query: " + compositeBuilder.toString()); - } - stats.incrementNumPages(1); - if (response.getBuckets().isEmpty()) { - // this is the end... - logger.debug("Finished indexing for job [" + job.getConfig().getId() + "], saving state and shutting down."); - - // Change state first, then try to persist. This prevents in-progress STOPPING/ABORTING from - // being persisted as STARTED but then stop the job - doSaveState(finishAndSetState(), position.get(), this::onFinish); - return; - } - - final BulkRequest bulkRequest = new BulkRequest(); + + @Override + protected SearchRequest buildSearchRequest() { // Indexer is single-threaded, and only place that the ID scheme can get upgraded is doSaveState(), so // we can pass down the boolean value rather than the atomic here - final List docs = IndexerUtils.processBuckets(response, job.getConfig().getRollupIndex(), - stats, job.getConfig().getGroupConfig(), job.getConfig().getId(), upgradedDocumentID.get()); - docs.forEach(bulkRequest::add); - assert bulkRequest.requests().size() > 0; - doNextBulk(bulkRequest, - ActionListener.wrap( - bulkResponse -> onBulkResponse(bulkResponse, response.afterKey()), - exc -> finishWithFailure(exc) - ) - ); - } catch(Exception e) { - finishWithFailure(e); - } - } - - private void finishWithFailure(Exception exc) { - doSaveState(finishAndSetState(), position.get(), () -> onFailure(exc)); - } - - private IndexerState finishAndSetState() { - return state.updateAndGet( - prev -> { - switch (prev) { - case INDEXING: - // ready for another job - return IndexerState.STARTED; - - case STOPPING: - // must be started again - return IndexerState.STOPPED; - - case ABORTING: - // abort and exit - onAbort(); - return IndexerState.ABORTING; // This shouldn't matter, since onAbort() will kill the task first - - case STOPPED: - // No-op. Shouldn't really be possible to get here (should have to go through STOPPING - // first which will be handled) but is harmless to no-op and we don't want to throw exception here - return IndexerState.STOPPED; - - default: - // any other state is unanticipated at this point - throw new IllegalStateException("Rollup job encountered an illegal state [" + prev + "]"); - } - }); - } - - private SearchRequest buildSearchRequest() { final Map position = getPosition(); SearchSourceBuilder searchSource = new SearchSourceBuilder() .size(0) @@ -383,6 +109,14 @@ private SearchRequest buildSearchRequest() { .aggregation(compositeBuilder.aggregateAfter(position)); return new SearchRequest(job.getConfig().getIndexPattern()).source(searchSource); } + + @Override + protected Iteration> doProcess(SearchResponse searchResponse) { + final CompositeAggregation response = searchResponse.getAggregations().get(AGGREGATION_NAME); + + return new Iteration<>(IndexerUtils.processBuckets(response, job.getConfig().getRollupIndex(), (RollupJobStats) stats, + job.getConfig().getGroupConfig(), job.getConfig().getId()), response.afterKey(), response.getBuckets().isEmpty()); + } /** * Creates a skeleton {@link CompositeAggregationBuilder} from the provided job config. diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java index 65362f9ad9dd3..a349d5394ad77 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java @@ -25,10 +25,10 @@ import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; +import org.elasticsearch.xpack.core.indexing.IndexerState; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.action.StartRollupJobAction; import org.elasticsearch.xpack.core.rollup.action.StopRollupJobAction; -import org.elasticsearch.xpack.core.rollup.job.IndexerState; import org.elasticsearch.xpack.core.rollup.job.RollupJob; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; import org.elasticsearch.xpack.core.rollup.job.RollupJobStats; @@ -219,7 +219,7 @@ public Status getStatus() { * @return The stats of this task */ public RollupJobStats getStats() { - return indexer.getStats(); + return (RollupJobStats) indexer.getStats(); } /** diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java index 6d29ee9f9ba6d..55f1cfbdbb29c 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java @@ -50,10 +50,10 @@ import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers; import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.GroupConfig; -import org.elasticsearch.xpack.core.rollup.job.IndexerState; import org.elasticsearch.xpack.core.rollup.job.MetricConfig; import org.elasticsearch.xpack.core.rollup.job.RollupJob; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; +import org.elasticsearch.xpack.core.indexing.IndexerState; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.junit.Before; diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java index 955dcbc2beb48..2242a3bcd072c 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java @@ -23,8 +23,8 @@ import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.GroupConfig; -import org.elasticsearch.xpack.core.rollup.job.IndexerState; import org.elasticsearch.xpack.core.rollup.job.RollupJob; +import org.elasticsearch.xpack.core.indexing.IndexerState; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; import org.mockito.stubbing.Answer; diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java index 13290f09e8eb8..c5f20b06a2314 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java @@ -19,11 +19,11 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.indexing.IndexerState; import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.action.StartRollupJobAction; import org.elasticsearch.xpack.core.rollup.action.StopRollupJobAction; -import org.elasticsearch.xpack.core.rollup.job.IndexerState; import org.elasticsearch.xpack.core.rollup.job.RollupJob; import org.elasticsearch.xpack.core.rollup.job.RollupJobStatus; import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; From 57063892a621b7d9facd5cdb097a83a35bc80497 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Thu, 9 Aug 2018 12:22:53 +0200 Subject: [PATCH 02/20] cleanup, simplify and add documentation --- .../xpack/core/indexing/IndexerJob.java | 14 ----- .../xpack/core/indexing/IterativeIndexer.java | 53 +++++++++++++++---- .../xpack/core/rollup/job/RollupJob.java | 9 +--- .../xpack/rollup/job/RollupIndexer.java | 35 +++++------- 4 files changed, 58 insertions(+), 53 deletions(-) delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJob.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJob.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJob.java deleted file mode 100644 index 09d01ef891de8..0000000000000 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJob.java +++ /dev/null @@ -1,14 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License; - * you may not use this file except in compliance with the Elastic License. - */ - -package org.elasticsearch.xpack.core.indexing; - -import org.elasticsearch.xpack.core.XPackPlugin; - -public interface IndexerJob extends XPackPlugin.XPackPersistentTaskParams { - - String getId(); -} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java index 632ed63022bac..09718ce6634da 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java @@ -19,20 +19,33 @@ import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicReference; +/** + * An abstract class that builds an index incrementally. A background job can be launched using {@link #maybeTriggerAsyncJob(long)}, + * it will create the index from the source index up to the last complete bucket that is allowed to be built (based on job position). + * Only one background job can run simultaneously and {@link #onFinish()} is called when the job + * finishes. {@link #onFailure(Exception)} is called if the job fails with an exception and {@link #onAbort()} is called if the indexer is + * aborted while a job is running. The indexer must be started ({@link #start()} to allow a background job to run when + * {@link #maybeTriggerAsyncJob(long)} is called. {@link #stop()} can be used to stop the background job without aborting the indexer. + * + * In a nutshell this is a 2 cycle engine: 1st it sends a query, 2nd it indexes documents based on the response, sends the next query, + * indexes, queries, indexes, ... until a condition lets the engine stop. + * + * @param Type that defines a job position to be defined by the implementation. + */ public abstract class IterativeIndexer { private static final Logger logger = Logger.getLogger(IterativeIndexer.class.getName()); - protected final IndexerStats stats; + private final IndexerStats stats; - protected final AtomicReference state; - protected final AtomicReference position; - protected final Executor executor; + private final AtomicReference state; + private final AtomicReference position; + private final Executor executor; - protected IterativeIndexer(Executor executor, AtomicReference initialState, JobPosition initialPosition) { + protected IterativeIndexer(Executor executor, AtomicReference initialState, JobPosition initialPosition, IndexerStats stats) { this.executor = executor; this.state = initialState; this.position = new AtomicReference<>(initialPosition); - this.stats = newJobStats(); + this.stats = stats; } /** @@ -152,14 +165,34 @@ public synchronized boolean maybeTriggerAsyncJob(long now) { } } - protected abstract IndexerStats newJobStats(); - + /** + * Called to get the Id of the job, used for logging. + * + * @return + */ protected abstract String getJobId(); - + + /** + * Called to process a response from the 1 search request in order to turn it into a {@link Iteration}. + * + * @param searchResponse response from the search phase. + * @return Iteration object to be passed to indexing phase. + */ protected abstract Iteration doProcess(SearchResponse searchResponse); - + + /** + * Called to build the next search request. + * + * @return SearchRequest to be passed to the search phase. + */ protected abstract SearchRequest buildSearchRequest(); + /** + * Called at startup after job has been triggered using {@link #maybeTriggerAsyncJob(long)} and the + * internal state is {@link IndexerState#STARTED}. + * + * @param now The current time in milliseconds passed through from {@link #maybeTriggerAsyncJob(long)} + */ protected abstract void onStart(long now); /** diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJob.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJob.java index 34d4697c5a43e..94306966a34da 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJob.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJob.java @@ -14,7 +14,7 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.xpack.core.indexing.IndexerJob; +import org.elasticsearch.xpack.core.XPackPlugin; import java.io.IOException; import java.util.Collections; @@ -26,7 +26,7 @@ * It holds the config (RollupJobConfig) and a map of authentication headers. Only RollupJobConfig * is ever serialized to the user, so the headers should never leak */ -public class RollupJob extends AbstractDiffable implements IndexerJob { +public class RollupJob extends AbstractDiffable implements XPackPlugin.XPackPersistentTaskParams { public static final String NAME = "xpack/rollup/job"; @@ -116,9 +116,4 @@ public int hashCode() { public Version getMinimalSupportedVersion() { return Version.V_6_3_0; } - - @Override - public String getId() { - return getConfig().getId(); - } } diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java index 6696c7f923048..72c650c13ef4e 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.rollup.job; -import org.apache.log4j.Logger; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.unit.TimeValue; @@ -35,16 +34,9 @@ import java.util.concurrent.atomic.AtomicReference; /** - * An abstract class that builds a rollup index incrementally. A background job can be launched using {@link #maybeTriggerAsyncJob(long)}, - * it will create the rollup index from the source index up to the last complete bucket that is allowed to be built (based on the current - * time and the delay set on the rollup job). Only one background job can run simultaneously and {@link #onFinish()} is called when the job - * finishes. {@link #onFailure(Exception)} is called if the job fails with an exception and {@link #onAbort()} is called if the indexer is - * aborted while a job is running. The indexer must be started ({@link #start()} to allow a background job to run when - * {@link #maybeTriggerAsyncJob(long)} is called. {@link #stop()} can be used to stop the background job without aborting the indexer. + * An abstract implementation of {@link IterativeIndexer} that builds a rollup index incrementally. */ public abstract class RollupIndexer extends IterativeIndexer > { - private static final Logger logger = Logger.getLogger(RollupIndexer.class.getName()); - static final String AGGREGATION_NAME = RollupField.NAME; private final RollupJob job; @@ -58,35 +50,32 @@ public abstract class RollupIndexer extends IterativeIndexer * @param job The rollup job * @param initialState Initial state for the indexer * @param initialPosition The last indexed bucket of the task + * @param upgradedDocumentID whether job has updated IDs (for BWC) */ - RollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, - super(executor, initialState, initialPosition); + RollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, Map initialPosition, + AtomicBoolean upgradedDocumentID) { + super(executor, initialState, initialPosition, new RollupJobStats()); this.job = job; this.compositeBuilder = createCompositeBuilder(job.getConfig()); this.upgradedDocumentID = upgradedDocumentID; } - @Override - protected RollupJobStats newJobStats() { - return new RollupJobStats(); + /** * Returns if this job has upgraded it's ID scheme yet or not */ public boolean isUpgradedDocumentID() { return upgradedDocumentID.get(); } - /** - } - @Override protected String getJobId() { - return job.getId(); + return job.getConfig().getId(); } @Override protected void onStart(long now) { // this is needed to exclude buckets that can still receive new documents. - DateHistoGroupConfig dateHisto = job.getConfig().getGroupConfig().getDateHisto(); + DateHistogramGroupConfig dateHisto = job.getConfig().getGroupConfig().getDateHistogram(); long rounded = dateHisto.createRounding().round(now); if (dateHisto.getDelay() != null) { // if the job has a delay we filter all documents that appear before it. @@ -114,9 +103,11 @@ protected SearchRequest buildSearchRequest() { protected Iteration> doProcess(SearchResponse searchResponse) { final CompositeAggregation response = searchResponse.getAggregations().get(AGGREGATION_NAME); - return new Iteration<>(IndexerUtils.processBuckets(response, job.getConfig().getRollupIndex(), (RollupJobStats) stats, - job.getConfig().getGroupConfig(), job.getConfig().getId()), response.afterKey(), response.getBuckets().isEmpty()); - } + return new Iteration<>( + IndexerUtils.processBuckets(response, job.getConfig().getRollupIndex(), (RollupJobStats) getStats(), + job.getConfig().getGroupConfig(), job.getConfig().getId(), upgradedDocumentID.get()), + response.afterKey(), response.getBuckets().isEmpty()); + } /** * Creates a skeleton {@link CompositeAggregationBuilder} from the provided job config. From b6d681a1919585b09c6410ec967b86e33c2b3ada Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Thu, 9 Aug 2018 13:56:38 +0200 Subject: [PATCH 03/20] refactor Jobstats --- .../xpack/core/indexing/IterativeIndexer.java | 4 +-- .../rollup/action/GetRollupJobsAction.java | 11 +++--- .../xpack/core/rollup/job/RollupJobStats.java | 11 +++--- .../core/indexing/IndexerStatsTests.java | 34 ++++++++++++++++++ .../core/rollup/job/RollupJobStatsTests.java | 35 ------------------- .../xpack/rollup/job/IndexerUtils.java | 4 +-- .../xpack/rollup/job/RollupIndexer.java | 3 +- .../xpack/rollup/job/RollupJobTask.java | 6 ++-- .../xpack/rollup/job/IndexerUtilsTests.java | 11 +++--- 9 files changed, 60 insertions(+), 59 deletions(-) create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStatsTests.java delete mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatsTests.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java index 09718ce6634da..869b4e1cdd19a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java @@ -28,7 +28,7 @@ * {@link #maybeTriggerAsyncJob(long)} is called. {@link #stop()} can be used to stop the background job without aborting the indexer. * * In a nutshell this is a 2 cycle engine: 1st it sends a query, 2nd it indexes documents based on the response, sends the next query, - * indexes, queries, indexes, ... until a condition lets the engine stop. + * indexes, queries, indexes, ... until a condition lets the engine pause until the source provides new input. * * @param Type that defines a job position to be defined by the implementation. */ @@ -168,7 +168,7 @@ public synchronized boolean maybeTriggerAsyncJob(long now) { /** * Called to get the Id of the job, used for logging. * - * @return + * @return a string with the id of the job */ protected abstract String getJobId(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java index 50f7931508585..e83732e78f483 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.tasks.Task; +import org.elasticsearch.xpack.core.indexing.IndexerStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; import org.elasticsearch.xpack.core.rollup.job.RollupJobStats; @@ -204,20 +205,20 @@ public final String toString() { public static class JobWrapper implements Writeable, ToXContentObject { private final RollupJobConfig job; - private final RollupJobStats stats; + private final IndexerStats stats; private final RollupJobStatus status; public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, a -> new JobWrapper((RollupJobConfig) a[0], - (RollupJobStats) a[1], (RollupJobStatus)a[2])); + (IndexerStats) a[1], (RollupJobStatus)a[2])); static { PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> RollupJobConfig.fromXContent(p, null), CONFIG); - PARSER.declareObject(ConstructingObjectParser.constructorArg(), RollupJobStats.PARSER::apply, STATS); + PARSER.declareObject(ConstructingObjectParser.constructorArg(), IndexerStats.PARSER::apply, STATS); PARSER.declareObject(ConstructingObjectParser.constructorArg(), RollupJobStatus.PARSER::apply, STATUS); } - public JobWrapper(RollupJobConfig job, RollupJobStats stats, RollupJobStatus status) { + public JobWrapper(RollupJobConfig job, IndexerStats stats, RollupJobStatus status) { this.job = job; this.stats = stats; this.status = status; @@ -240,7 +241,7 @@ public RollupJobConfig getJob() { return job; } - public RollupJobStats getStats() { + public IndexerStats getStats() { return stats; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java index 1c9e2dd93953a..f0ade92fab270 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java @@ -11,12 +11,9 @@ import java.io.IOException; /** - * This class holds the runtime statistics of a job. The stats are not used by any internal process - * and are only for external monitoring/reference. Statistics are not persisted with the job, so if the - * allocated task is shutdown/restarted on a different node all the stats will reset. + * This class holds the runtime statistics of a rollup job, derived from {@link IndexerStats}}. + * */ - -// todo: kept for now, consider removing it public class RollupJobStats extends IndexerStats { public RollupJobStats(StreamInput in) throws IOException { @@ -27,6 +24,8 @@ public RollupJobStats() { super(); } - + public RollupJobStats(long numPages, long numDocuments, long numRollups, long numInvocations) { + super (numPages, numDocuments, numRollups, numInvocations); + } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStatsTests.java new file mode 100644 index 0000000000000..e6512a8568754 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStatsTests.java @@ -0,0 +1,34 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.core.indexing; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractSerializingTestCase; + +public class IndexerStatsTests extends AbstractSerializingTestCase { + + @Override + protected IndexerStats createTestInstance() { + return randomStats(); + } + + @Override + protected Writeable.Reader instanceReader() { + return IndexerStats::new; + } + + @Override + protected IndexerStats doParseInstance(XContentParser parser) { + return IndexerStats.fromXContent(parser); + } + + public static IndexerStats randomStats() { + return new IndexerStats(randomNonNegativeLong(), randomNonNegativeLong(), + randomNonNegativeLong(), randomNonNegativeLong()); + } +} + diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatsTests.java deleted file mode 100644 index 0091b21dc40d0..0000000000000 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatsTests.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.core.rollup.job; - -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.test.AbstractSerializingTestCase; -import org.elasticsearch.xpack.core.rollup.job.RollupJobStats; - -public class RollupJobStatsTests extends AbstractSerializingTestCase { - - @Override - protected RollupJobStats createTestInstance() { - return randomStats(); - } - - @Override - protected Writeable.Reader instanceReader() { - return RollupJobStats::new; - } - - @Override - protected RollupJobStats doParseInstance(XContentParser parser) { - return RollupJobStats.fromXContent(parser); - } - - public static RollupJobStats randomStats() { - return new RollupJobStats(randomNonNegativeLong(), randomNonNegativeLong(), - randomNonNegativeLong(), randomNonNegativeLong()); - } -} - diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java index 9119a5445d42e..b660b2ae0b557 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java @@ -14,10 +14,10 @@ import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.xpack.core.indexing.IndexerStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.GroupConfig; -import org.elasticsearch.xpack.core.rollup.job.RollupJobStats; import org.elasticsearch.xpack.rollup.Rollup; import java.util.ArrayList; @@ -46,7 +46,7 @@ class IndexerUtils { * @param isUpgradedDocID `true` if this job is using the new ID scheme * @return A list of rolled documents derived from the response */ - static List processBuckets(CompositeAggregation agg, String rollupIndex, RollupJobStats stats, + static List processBuckets(CompositeAggregation agg, String rollupIndex, IndexerStats stats, GroupConfig groupConfig, String jobId, boolean isUpgradedDocID) { logger.debug("Buckets: [" + agg.getBuckets().size() + "][" + jobId + "]"); diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java index 72c650c13ef4e..e588a6833eed1 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java @@ -16,6 +16,7 @@ import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.xpack.core.indexing.IndexerState; +import org.elasticsearch.xpack.core.indexing.IndexerStats; import org.elasticsearch.xpack.core.indexing.Iteration; import org.elasticsearch.xpack.core.indexing.IterativeIndexer; import org.elasticsearch.xpack.core.rollup.RollupField; @@ -104,7 +105,7 @@ protected Iteration> doProcess(SearchResponse searchResponse final CompositeAggregation response = searchResponse.getAggregations().get(AGGREGATION_NAME); return new Iteration<>( - IndexerUtils.processBuckets(response, job.getConfig().getRollupIndex(), (RollupJobStats) getStats(), + IndexerUtils.processBuckets(response, job.getConfig().getRollupIndex(), getStats(), job.getConfig().getGroupConfig(), job.getConfig().getId(), upgradedDocumentID.get()), response.afterKey(), response.getBuckets().isEmpty()); } diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java index a349d5394ad77..1dd0e7ab0096a 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java @@ -26,12 +26,12 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.indexing.IndexerState; +import org.elasticsearch.xpack.core.indexing.IndexerStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.action.StartRollupJobAction; import org.elasticsearch.xpack.core.rollup.action.StopRollupJobAction; import org.elasticsearch.xpack.core.rollup.job.RollupJob; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; -import org.elasticsearch.xpack.core.rollup.job.RollupJobStats; import org.elasticsearch.xpack.core.rollup.job.RollupJobStatus; import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; import org.elasticsearch.xpack.rollup.Rollup; @@ -218,8 +218,8 @@ public Status getStatus() { * Gets the stats for this task. * @return The stats of this task */ - public RollupJobStats getStats() { - return (RollupJobStats) indexer.getStats(); + public IndexerStats getStats() { + return indexer.getStats(); } /** diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java index e8c66f7e8c118..d81ffde97c384 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java @@ -35,6 +35,7 @@ import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.max.MaxAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.sum.SumAggregationBuilder; +import org.elasticsearch.xpack.core.indexing.IndexerStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.GroupConfig; @@ -64,7 +65,7 @@ public class IndexerUtilsTests extends AggregatorTestCase { public void testMissingFields() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - RollupJobStats stats = new RollupJobStats(0, 0, 0, 0); + IndexerStats stats = new RollupJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -126,7 +127,7 @@ public void testMissingFields() throws IOException { public void testCorrectFields() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - RollupJobStats stats= new RollupJobStats(0, 0, 0, 0); + IndexerStats stats= new RollupJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -193,7 +194,7 @@ public void testCorrectFields() throws IOException { public void testNumericTerms() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - RollupJobStats stats= new RollupJobStats(0, 0, 0, 0); + IndexerStats stats= new RollupJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -249,7 +250,7 @@ public void testNumericTerms() throws IOException { public void testEmptyCounts() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - RollupJobStats stats= new RollupJobStats(0, 0, 0, 0); + IndexerStats stats= new RollupJobStats(0, 0, 0, 0); String timestampField = "ts"; String valueField = "the_avg"; @@ -483,7 +484,7 @@ public void testNullKeys() { public void testMissingBuckets() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - RollupJobStats stats= new RollupJobStats(0, 0, 0, 0); + IndexerStats stats= new RollupJobStats(0, 0, 0, 0); String metricField = "metric_field"; String valueField = "value_field"; From 106f6540bff4b63010a7c5b588f18f1335e97a73 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Thu, 9 Aug 2018 16:11:41 +0200 Subject: [PATCH 04/20] fix some namings and style --- .../xpack/core/indexing/IndexerStats.java | 8 ++++---- .../xpack/core/indexing/IterativeIndexer.java | 11 ++++++----- .../elasticsearch/xpack/rollup/job/RollupIndexer.java | 1 - .../xpack/rollup/job/RollupIndexerStateTests.java | 8 ++++---- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerStats.java index cada682a88582..1d7a09bca2697 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerStats.java @@ -52,10 +52,10 @@ public class IndexerStats implements ToXContentObject, Writeable { public IndexerStats() { } - public IndexerStats(long numPages, long numDocuments, long numRollups, long numInvocations) { + public IndexerStats(long numPages, long numDocuments, long numOuputDocuments, long numInvocations) { this.numPages = numPages; this.numInputDocuments = numDocuments; - this.numOuputDocuments = numRollups; + this.numOuputDocuments = numOuputDocuments; this.numInvocations = numInvocations; } @@ -78,7 +78,7 @@ public long getNumInvocations() { return numInvocations; } - public long getNumRollups() { + public long getOutputDocuments() { return numOuputDocuments; } @@ -97,7 +97,7 @@ public void incrementNumInvocations(long n) { numInvocations += n; } - public void incrementNumRollups(long n) { + public void incrementNumOutputDocuments(long n) { assert(n >= 0); numOuputDocuments += n; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java index 869b4e1cdd19a..1c288249deb8b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java @@ -41,7 +41,8 @@ public abstract class IterativeIndexer { private final AtomicReference position; private final Executor executor; - protected IterativeIndexer(Executor executor, AtomicReference initialState, JobPosition initialPosition, IndexerStats stats) { + protected IterativeIndexer(Executor executor, AtomicReference initialState, JobPosition initialPosition, + IndexerStats stats) { this.executor = executor; this.state = initialState; this.position = new AtomicReference<>(initialPosition); @@ -121,7 +122,7 @@ public synchronized boolean abort() { } /** - * Triggers a background job that builds the rollup index asynchronously iff + * Triggers a background job that builds the index asynchronously iff * there is no other job that runs and the indexer is started * ({@link IndexerState#STARTED}. * @@ -279,7 +280,7 @@ private IndexerState finishAndSetState() { default: // any other state is unanticipated at this point - throw new IllegalStateException("Rollup job encountered an illegal state [" + prev + "]"); + throw new IllegalStateException("Indexer job encountered an illegal state [" + prev + "]"); } }); } @@ -318,7 +319,7 @@ private void onSearchResponse(SearchResponse searchResponse) { if (bulkResponse.hasFailures()) { logger.warn("Error while attempting to bulk index documents: " + bulkResponse.buildFailureMessage()); } - stats.incrementNumRollups(bulkResponse.getItems().length); + stats.incrementNumOutputDocuments(bulkResponse.getItems().length); if (checkState(getState()) == false) { return; } @@ -375,7 +376,7 @@ private boolean checkState(IndexerState currentState) { default: // Anything other than indexing, aborting or stopping is unanticipated logger.warn("Encountered unexpected state [" + currentState + "] while indexing"); - throw new IllegalStateException("Rollup job encountered an illegal state [" + currentState + "]"); + throw new IllegalStateException("Indexer job encountered an illegal state [" + currentState + "]"); } } diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java index e588a6833eed1..dcd8d8ee33910 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java @@ -16,7 +16,6 @@ import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.xpack.core.indexing.IndexerState; -import org.elasticsearch.xpack.core.indexing.IndexerStats; import org.elasticsearch.xpack.core.indexing.Iteration; import org.elasticsearch.xpack.core.indexing.IterativeIndexer; import org.elasticsearch.xpack.core.rollup.RollupField; diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java index 2242a3bcd072c..e5cd0ce13f293 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java @@ -639,7 +639,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws assertThat(indexer.getStats().getNumPages(), equalTo(1L)); // Note: no docs were indexed - assertThat(indexer.getStats().getNumRollups(), equalTo(0L)); + assertThat(indexer.getStats().getOutputDocuments(), equalTo(0L)); assertTrue(indexer.abort()); } finally { executor.shutdownNow(); @@ -743,7 +743,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws assertThat(indexer.getStats().getNumPages(), equalTo(1L)); // Note: no docs were indexed - assertThat(indexer.getStats().getNumRollups(), equalTo(0L)); + assertThat(indexer.getStats().getOutputDocuments(), equalTo(0L)); assertTrue(indexer.abort()); } finally { executor.shutdownNow(); @@ -786,7 +786,7 @@ public void testSearchShardFailure() throws Exception { // Note: no pages processed, no docs were indexed assertThat(indexer.getStats().getNumPages(), equalTo(0L)); - assertThat(indexer.getStats().getNumRollups(), equalTo(0L)); + assertThat(indexer.getStats().getOutputDocuments(), equalTo(0L)); assertTrue(indexer.abort()); } finally { executor.shutdownNow(); @@ -896,7 +896,7 @@ protected void doNextBulk(BulkRequest request, ActionListener next assertThat(indexer.getStats().getNumPages(), equalTo(1L)); // Note: no docs were indexed - assertThat(indexer.getStats().getNumRollups(), equalTo(0L)); + assertThat(indexer.getStats().getOutputDocuments(), equalTo(0L)); assertTrue(indexer.abort()); } finally { executor.shutdownNow(); From a3f53eef1a5aabebf3d798e9ffe03063f90d0d84 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Thu, 9 Aug 2018 19:41:40 +0200 Subject: [PATCH 05/20] remove RollupJobStats --- ...IndexerStats.java => IndexerJobStats.java} | 21 +++++++------ .../xpack/core/indexing/IterativeIndexer.java | 9 +++--- .../rollup/action/GetRollupJobsAction.java | 15 +++++---- .../xpack/core/rollup/job/RollupJobStats.java | 31 ------------------- .../core/indexing/IndexerStatsTests.java | 16 +++++----- .../job/JobWrapperSerializingTests.java | 3 +- .../xpack/rollup/job/IndexerUtils.java | 4 +-- .../xpack/rollup/job/RollupIndexer.java | 3 +- .../xpack/rollup/job/RollupJobTask.java | 4 +-- .../xpack/rollup/job/IndexerUtilsTests.java | 21 ++++++------- .../rollup/job/RollupIndexerStateTests.java | 2 +- 11 files changed, 48 insertions(+), 81 deletions(-) rename x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/{IndexerStats.java => IndexerJobStats.java} (86%) delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java similarity index 86% rename from x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerStats.java rename to x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java index 1d7a09bca2697..df2d91f19e1b0 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java @@ -24,13 +24,14 @@ * and are only for external monitoring/reference. Statistics are not persisted with the job, so if the * allocated task is shutdown/restarted on a different node all the stats will reset. */ -public class IndexerStats implements ToXContentObject, Writeable { +public class IndexerJobStats implements ToXContentObject, Writeable { public static final ParseField NAME = new ParseField("job_stats"); private static ParseField NUM_PAGES = new ParseField("pages_processed"); - private static ParseField NUM_INPUT_DOCUMENTS = new ParseField("source_documents_processed"); - private static ParseField NUM_OUTPUT_DOCUMENTS = new ParseField("documents_indexed"); + private static ParseField NUM_INPUT_DOCUMENTS = new ParseField("documents_processed"); + // BWC for RollupJobStats + private static ParseField NUM_OUTPUT_DOCUMENTS = new ParseField("documents_indexed").withDeprecation("rollups_indexed"); private static ParseField NUM_INVOCATIONS = new ParseField("trigger_count"); private long numPages = 0; @@ -38,9 +39,9 @@ public class IndexerStats implements ToXContentObject, Writeable { private long numOuputDocuments = 0; private long numInvocations = 0; - public static final ConstructingObjectParser PARSER = + public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME.getPreferredName(), - args -> new IndexerStats((long) args[0], (long) args[1], (long) args[2], (long) args[3])); + args -> new IndexerJobStats((long) args[0], (long) args[1], (long) args[2], (long) args[3])); static { PARSER.declareLong(constructorArg(), NUM_PAGES); @@ -49,17 +50,17 @@ public class IndexerStats implements ToXContentObject, Writeable { PARSER.declareLong(constructorArg(), NUM_INVOCATIONS); } - public IndexerStats() { + public IndexerJobStats() { } - public IndexerStats(long numPages, long numDocuments, long numOuputDocuments, long numInvocations) { + public IndexerJobStats(long numPages, long numDocuments, long numOuputDocuments, long numInvocations) { this.numPages = numPages; this.numInputDocuments = numDocuments; this.numOuputDocuments = numOuputDocuments; this.numInvocations = numInvocations; } - public IndexerStats(StreamInput in) throws IOException { + public IndexerJobStats(StreamInput in) throws IOException { this.numPages = in.readVLong(); this.numInputDocuments = in.readVLong(); this.numOuputDocuments = in.readVLong(); @@ -110,7 +111,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(numInvocations); } - public static IndexerStats fromXContent(XContentParser parser) { + public static IndexerJobStats fromXContent(XContentParser parser) { try { return PARSER.parse(parser, null); } catch (IOException e) { @@ -139,7 +140,7 @@ public boolean equals(Object other) { return false; } - IndexerStats that = (IndexerStats) other; + IndexerJobStats that = (IndexerJobStats) other; return Objects.equals(this.numPages, that.numPages) && Objects.equals(this.numInputDocuments, that.numInputDocuments) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java index 1c288249deb8b..2ef3fdef607d2 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java @@ -35,18 +35,17 @@ public abstract class IterativeIndexer { private static final Logger logger = Logger.getLogger(IterativeIndexer.class.getName()); - private final IndexerStats stats; + private final IndexerJobStats stats; private final AtomicReference state; private final AtomicReference position; private final Executor executor; - protected IterativeIndexer(Executor executor, AtomicReference initialState, JobPosition initialPosition, - IndexerStats stats) { + protected IterativeIndexer(Executor executor, AtomicReference initialState, JobPosition initialPosition) { this.executor = executor; this.state = initialState; this.position = new AtomicReference<>(initialPosition); - this.stats = stats; + this.stats = new IndexerJobStats(); } /** @@ -66,7 +65,7 @@ public JobPosition getPosition() { /** * Get the stats of this indexer. */ - public IndexerStats getStats() { + public IndexerJobStats getStats() { return stats; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java index e83732e78f483..4bfd5b621e780 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java @@ -25,10 +25,9 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.tasks.Task; -import org.elasticsearch.xpack.core.indexing.IndexerStats; +import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; -import org.elasticsearch.xpack.core.rollup.job.RollupJobStats; import org.elasticsearch.xpack.core.rollup.job.RollupJobStatus; import java.io.IOException; @@ -205,20 +204,20 @@ public final String toString() { public static class JobWrapper implements Writeable, ToXContentObject { private final RollupJobConfig job; - private final IndexerStats stats; + private final IndexerJobStats stats; private final RollupJobStatus status; public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, a -> new JobWrapper((RollupJobConfig) a[0], - (IndexerStats) a[1], (RollupJobStatus)a[2])); + (IndexerJobStats) a[1], (RollupJobStatus)a[2])); static { PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> RollupJobConfig.fromXContent(p, null), CONFIG); - PARSER.declareObject(ConstructingObjectParser.constructorArg(), IndexerStats.PARSER::apply, STATS); + PARSER.declareObject(ConstructingObjectParser.constructorArg(), IndexerJobStats.PARSER::apply, STATS); PARSER.declareObject(ConstructingObjectParser.constructorArg(), RollupJobStatus.PARSER::apply, STATUS); } - public JobWrapper(RollupJobConfig job, IndexerStats stats, RollupJobStatus status) { + public JobWrapper(RollupJobConfig job, IndexerJobStats stats, RollupJobStatus status) { this.job = job; this.stats = stats; this.status = status; @@ -226,7 +225,7 @@ public JobWrapper(RollupJobConfig job, IndexerStats stats, RollupJobStatus statu public JobWrapper(StreamInput in) throws IOException { this.job = new RollupJobConfig(in); - this.stats = new RollupJobStats(in); + this.stats = new IndexerJobStats(in); this.status = new RollupJobStatus(in); } @@ -241,7 +240,7 @@ public RollupJobConfig getJob() { return job; } - public IndexerStats getStats() { + public IndexerJobStats getStats() { return stats; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java deleted file mode 100644 index f0ade92fab270..0000000000000 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStats.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.core.rollup.job; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.xpack.core.indexing.IndexerStats; - -import java.io.IOException; - -/** - * This class holds the runtime statistics of a rollup job, derived from {@link IndexerStats}}. - * - */ -public class RollupJobStats extends IndexerStats { - - public RollupJobStats(StreamInput in) throws IOException { - super(in); - } - - public RollupJobStats() { - super(); - } - - public RollupJobStats(long numPages, long numDocuments, long numRollups, long numInvocations) { - super (numPages, numDocuments, numRollups, numInvocations); - } -} - diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStatsTests.java index e6512a8568754..1b0a5f51d34f2 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStatsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStatsTests.java @@ -9,25 +9,25 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractSerializingTestCase; -public class IndexerStatsTests extends AbstractSerializingTestCase { +public class IndexerStatsTests extends AbstractSerializingTestCase { @Override - protected IndexerStats createTestInstance() { + protected IndexerJobStats createTestInstance() { return randomStats(); } @Override - protected Writeable.Reader instanceReader() { - return IndexerStats::new; + protected Writeable.Reader instanceReader() { + return IndexerJobStats::new; } @Override - protected IndexerStats doParseInstance(XContentParser parser) { - return IndexerStats.fromXContent(parser); + protected IndexerJobStats doParseInstance(XContentParser parser) { + return IndexerJobStats.fromXContent(parser); } - public static IndexerStats randomStats() { - return new IndexerStats(randomNonNegativeLong(), randomNonNegativeLong(), + public static IndexerJobStats randomStats() { + return new IndexerJobStats(randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong()); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java index 388831cd7dbf7..cb827c040801d 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java @@ -9,6 +9,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractSerializingTestCase; import org.elasticsearch.xpack.core.indexing.IndexerState; +import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers; import org.elasticsearch.xpack.core.rollup.action.GetRollupJobsAction; @@ -41,7 +42,7 @@ protected GetRollupJobsAction.JobWrapper createTestInstance() { } return new GetRollupJobsAction.JobWrapper(ConfigTestHelpers.randomRollupJobConfig(random()), - new RollupJobStats(randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong()), + new IndexerJobStats(randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong()), new RollupJobStatus(state, Collections.emptyMap(), randomBoolean())); } } diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java index b660b2ae0b557..594221d921488 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java @@ -14,7 +14,7 @@ import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; -import org.elasticsearch.xpack.core.indexing.IndexerStats; +import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.GroupConfig; @@ -46,7 +46,7 @@ class IndexerUtils { * @param isUpgradedDocID `true` if this job is using the new ID scheme * @return A list of rolled documents derived from the response */ - static List processBuckets(CompositeAggregation agg, String rollupIndex, IndexerStats stats, + static List processBuckets(CompositeAggregation agg, String rollupIndex, IndexerJobStats stats, GroupConfig groupConfig, String jobId, boolean isUpgradedDocID) { logger.debug("Buckets: [" + agg.getBuckets().size() + "][" + jobId + "]"); diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java index dcd8d8ee33910..c2a07b47349aa 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java @@ -23,7 +23,6 @@ import org.elasticsearch.xpack.core.rollup.job.GroupConfig; import org.elasticsearch.xpack.core.rollup.job.RollupJob; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; -import org.elasticsearch.xpack.core.rollup.job.RollupJobStats; import java.util.ArrayList; import java.util.HashMap; @@ -54,7 +53,7 @@ public abstract class RollupIndexer extends IterativeIndexer */ RollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, Map initialPosition, AtomicBoolean upgradedDocumentID) { - super(executor, initialState, initialPosition, new RollupJobStats()); + super(executor, initialState, initialPosition); this.job = job; this.compositeBuilder = createCompositeBuilder(job.getConfig()); this.upgradedDocumentID = upgradedDocumentID; diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java index 1dd0e7ab0096a..4898e099319b5 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java @@ -26,7 +26,7 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.indexing.IndexerState; -import org.elasticsearch.xpack.core.indexing.IndexerStats; +import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.action.StartRollupJobAction; import org.elasticsearch.xpack.core.rollup.action.StopRollupJobAction; @@ -218,7 +218,7 @@ public Status getStatus() { * Gets the stats for this task. * @return The stats of this task */ - public IndexerStats getStats() { + public IndexerJobStats getStats() { return indexer.getStats(); } diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java index d81ffde97c384..5a16740f94611 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java @@ -35,13 +35,12 @@ import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.max.MaxAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.sum.SumAggregationBuilder; -import org.elasticsearch.xpack.core.indexing.IndexerStats; +import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.GroupConfig; import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.MetricConfig; -import org.elasticsearch.xpack.core.rollup.job.RollupJobStats; import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; import org.joda.time.DateTime; import org.mockito.stubbing.Answer; @@ -65,7 +64,7 @@ public class IndexerUtilsTests extends AggregatorTestCase { public void testMissingFields() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerStats stats = new RollupJobStats(0, 0, 0, 0); + IndexerJobStats stats = new IndexerJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -127,7 +126,7 @@ public void testMissingFields() throws IOException { public void testCorrectFields() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerStats stats= new RollupJobStats(0, 0, 0, 0); + IndexerJobStats stats= new IndexerJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -194,7 +193,7 @@ public void testCorrectFields() throws IOException { public void testNumericTerms() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerStats stats= new RollupJobStats(0, 0, 0, 0); + IndexerJobStats stats= new IndexerJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -250,7 +249,7 @@ public void testNumericTerms() throws IOException { public void testEmptyCounts() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerStats stats= new RollupJobStats(0, 0, 0, 0); + IndexerJobStats stats= new IndexerJobStats(0, 0, 0, 0); String timestampField = "ts"; String valueField = "the_avg"; @@ -356,7 +355,7 @@ public void testKeyOrderingOldID() { // The content of the config don't actually matter for this test // because the test is just looking at agg keys GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), new HistogramGroupConfig(123L, "abc"), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new RollupJobStats(), groupConfig, "foo", false); + List docs = IndexerUtils.processBuckets(composite, "foo", new IndexerJobStats(), groupConfig, "foo", false); assertThat(docs.size(), equalTo(1)); assertThat(docs.get(0).id(), equalTo("1237859798")); } @@ -400,7 +399,7 @@ public void testKeyOrderingNewID() { }); GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), new HistogramGroupConfig(1L, "abc"), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new RollupJobStats(), groupConfig, "foo", true); + List docs = IndexerUtils.processBuckets(composite, "foo", new IndexerJobStats(), groupConfig, "foo", true); assertThat(docs.size(), equalTo(1)); assertThat(docs.get(0).id(), equalTo("foo$c9LcrFqeFW92uN_Z7sv1hA")); } @@ -450,7 +449,7 @@ public void testKeyOrderingNewIDLong() { }); GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), new HistogramGroupConfig(1, "abc"), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new RollupJobStats(), groupConfig, "foo", true); + List docs = IndexerUtils.processBuckets(composite, "foo", new IndexerJobStats(), groupConfig, "foo", true); assertThat(docs.size(), equalTo(1)); assertThat(docs.get(0).id(), equalTo("foo$VAFKZpyaEqYRPLyic57_qw")); } @@ -477,14 +476,14 @@ public void testNullKeys() { }); GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), randomHistogramGroupConfig(random()), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new RollupJobStats(), groupConfig, "foo", randomBoolean()); + List docs = IndexerUtils.processBuckets(composite, "foo", new IndexerJobStats(), groupConfig, "foo", randomBoolean()); assertThat(docs.size(), equalTo(1)); assertFalse(Strings.isNullOrEmpty(docs.get(0).id())); } public void testMissingBuckets() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerStats stats= new RollupJobStats(0, 0, 0, 0); + IndexerJobStats stats= new IndexerJobStats(0, 0, 0, 0); String metricField = "metric_field"; String valueField = "value_field"; diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java index e5cd0ce13f293..c74ecbadf4fbe 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java @@ -763,7 +763,7 @@ public void testSearchShardFailure() throws Exception { Function bulkFunction = bulkRequest -> new BulkResponse(new BulkItemResponse[0], 100); Consumer failureConsumer = e -> { - assertThat(e.getMessage(), startsWith("Shard failures encountered while running indexer for rollup job")); + assertThat(e.getMessage(), startsWith("Shard failures encountered while running indexer for job")); isFinished.set(true); }; From b51853ba570b386ccea467f6d5e5ef7d857bb8f0 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Fri, 10 Aug 2018 09:57:14 +0200 Subject: [PATCH 06/20] adapt tests for fieldname change --- x-pack/docs/en/rest-api/rollup/get-job.asciidoc | 6 +++--- .../test/resources/rest-api-spec/test/rollup/delete_job.yml | 6 +++--- .../test/resources/rest-api-spec/test/rollup/get_jobs.yml | 6 +++--- .../test/resources/rest-api-spec/test/rollup/put_job.yml | 2 +- .../test/java/org/elasticsearch/multi_node/RollupIT.java | 2 +- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/x-pack/docs/en/rest-api/rollup/get-job.asciidoc b/x-pack/docs/en/rest-api/rollup/get-job.asciidoc index 96053dbfea64f..1a90557106bfc 100644 --- a/x-pack/docs/en/rest-api/rollup/get-job.asciidoc +++ b/x-pack/docs/en/rest-api/rollup/get-job.asciidoc @@ -99,7 +99,7 @@ Which will yield the following response: "stats" : { "pages_processed" : 0, "documents_processed" : 0, - "rollups_indexed" : 0, + "documents_indexed" : 0, "trigger_count" : 0 } } @@ -219,7 +219,7 @@ Which will yield the following response: "stats" : { "pages_processed" : 0, "documents_processed" : 0, - "rollups_indexed" : 0, + "documents_indexed" : 0, "trigger_count" : 0 } }, @@ -268,7 +268,7 @@ Which will yield the following response: "stats" : { "pages_processed" : 0, "documents_processed" : 0, - "rollups_indexed" : 0, + "documents_indexed" : 0, "trigger_count" : 0 } } diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml index 298cf27fa2f9d..d157fd61138d6 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml @@ -66,7 +66,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - rollups_indexed: 0 + documents_indexed: 0 trigger_count: 0 status: job_state: "stopped" @@ -113,7 +113,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - rollups_indexed: 0 + documents_indexed: 0 trigger_count: 0 status: job_state: "stopped" @@ -160,7 +160,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - rollups_indexed: 0 + documents_indexed: 0 trigger_count: 0 status: job_state: "stopped" diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml index f3fa8114ddbd0..ec2c6f7581038 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml @@ -67,7 +67,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - rollups_indexed: 0 + documents_indexed: 0 trigger_count: 0 status: job_state: "stopped" @@ -178,7 +178,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - rollups_indexed: 0 + documents_indexed: 0 trigger_count: 0 status: job_state: "stopped" @@ -204,7 +204,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - rollups_indexed: 0 + documents_indexed: 0 trigger_count: 0 status: job_state: "stopped" diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml index 516be25be2a2d..e4d635bbe3995 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml @@ -67,7 +67,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - rollups_indexed: 0 + documents_indexed: 0 trigger_count: 0 status: job_state: "stopped" diff --git a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java index 43ad4dc0a45a2..a26f3cfb2b213 100644 --- a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java +++ b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java @@ -156,7 +156,7 @@ public void testBigRollup() throws Exception { Map job = getJob(getRollupJobResponse, "rollup-job-test"); if (job != null) { assertThat(ObjectPath.eval("status.job_state", job), equalTo("started")); - assertThat(ObjectPath.eval("stats.rollups_indexed", job), equalTo(41)); + assertThat(ObjectPath.eval("stats.documents_indexed", job), equalTo(41)); } }, 30L, TimeUnit.SECONDS); From 5b2794c4229f2ca4cb1dc232651b2735d1a4a5a8 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Fri, 10 Aug 2018 10:23:13 +0200 Subject: [PATCH 07/20] add documentation --- .../xpack/core/indexing/IndexerJobStats.java | 1 - .../xpack/core/indexing/Iteration.java | 29 +++++++++++++++++-- .../xpack/core/indexing/IterativeIndexer.java | 2 ++ ...tsTests.java => IndexerJobStatsTests.java} | 2 +- 4 files changed, 29 insertions(+), 5 deletions(-) rename x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/{IndexerStatsTests.java => IndexerJobStatsTests.java} (92%) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java index df2d91f19e1b0..f64ed5804535d 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java @@ -3,7 +3,6 @@ * or more contributor license agreements. Licensed under the Elastic License; * you may not use this file except in compliance with the Elastic License. */ - package org.elasticsearch.xpack.core.indexing; import org.elasticsearch.common.ParseField; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/Iteration.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/Iteration.java index aa2d592e475a3..5568ecd5ff806 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/Iteration.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/Iteration.java @@ -10,30 +10,53 @@ import java.util.List; +/** + * Result object to hold the result of 1 iteration of iterative indexing. + * Acts as an interface between the implementation and the generic indexer. + */ public class Iteration { private final boolean isDone; - private final JobPosition position; - private final List toIndex; + /** + * Constructor for the result of 1 iteration. + * + * @param toIndex the list of requests to be indexed + * @param position the extracted, persistable position of the job required for the search phase + * @param isDone true if source is exhausted and job should go to sleep + * + * Note: toIndex.empty() != isDone due to possible filtering in the specific implementation + */ public Iteration(List toIndex, JobPosition position, boolean isDone) { this.toIndex = toIndex; this.position = position; this.isDone = isDone; } + /** + * Returns true if this indexing iteration is done and job should go into sleep mode. + */ public boolean isDone() { return isDone; } + /** + * Return the position of the job, a generic to be passed to the next query construction. + * + * @return the position + */ public JobPosition getPosition() { return position; } + /** + * List of requests to be passed to bulk indexing. + * + * @return List of index requests. + */ public List getToIndex() { return toIndex; } - } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java index 2ef3fdef607d2..9ec71b88e466c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java @@ -310,6 +310,8 @@ private void onSearchResponse(SearchResponse searchResponse) { final List docs = iteration.getToIndex(); final BulkRequest bulkRequest = new BulkRequest(); docs.forEach(bulkRequest::add); + + // TODO this might be a valid case, e.g. if implementation filters assert bulkRequest.requests().size() > 0; doNextBulk(bulkRequest, ActionListener.wrap(bulkResponse -> { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerJobStatsTests.java similarity index 92% rename from x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStatsTests.java rename to x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerJobStatsTests.java index 1b0a5f51d34f2..e60573d3ed071 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerStatsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerJobStatsTests.java @@ -9,7 +9,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractSerializingTestCase; -public class IndexerStatsTests extends AbstractSerializingTestCase { +public class IndexerJobStatsTests extends AbstractSerializingTestCase { @Override protected IndexerJobStats createTestInstance() { From 2e858c0903036fb6ca442cb288e5c93587eb6d8e Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Fri, 10 Aug 2018 12:49:38 +0200 Subject: [PATCH 08/20] add a simple test --- .../core/indexing/IterativeIndexerTests.java | 133 ++++++++++++++++++ 1 file changed, 133 insertions(+) create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java new file mode 100644 index 0000000000000..85066cb42f519 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java @@ -0,0 +1,133 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.core.indexing; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.SearchResponseSections; +import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHits; +import org.elasticsearch.test.ESTestCase; + +import java.util.Collections; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static org.hamcrest.Matchers.equalTo; + +public class IterativeIndexerTests extends ESTestCase { + + AtomicBoolean isFinished = new AtomicBoolean(false); + + private class MockIndexer extends IterativeIndexer { + + // test the execution order + private int step; + + protected MockIndexer(Executor executor, AtomicReference initialState, Integer initialPosition) { + super(executor, initialState, initialPosition); + } + + @Override + protected String getJobId() { + return "mock"; + } + + @Override + protected Iteration doProcess(SearchResponse searchResponse) { + assertThat(step, equalTo(3)); + ++step; + return new Iteration(Collections.emptyList(), 3, true); + } + + @Override + protected SearchRequest buildSearchRequest() { + assertThat(step, equalTo(1)); + ++step; + return null; + } + + @Override + protected void onStart(long now) { + assertThat(step, equalTo(0)); + ++step; + } + + @Override + protected void doNextSearch(SearchRequest request, ActionListener nextPhase) { + assertThat(step, equalTo(2)); + ++step; + final SearchResponseSections sections = new SearchResponseSections(new SearchHits(new SearchHit[0], 0, 0), null, null, false, + null, null, 1); + + nextPhase.onResponse(new SearchResponse(sections, null, 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, null)); + } + + @Override + protected void doNextBulk(BulkRequest request, ActionListener nextPhase) { + fail("should not be called"); + } + + @Override + protected void doSaveState(IndexerState state, Integer position, Runnable next) { + assertThat(step, equalTo(4)); + ++step; + next.run(); + } + + @Override + protected void onFailure(Exception exc) { + fail(exc.getMessage()); + } + + @Override + protected void onFinish() { + assertThat(step, equalTo(5)); + ++step; + isFinished.set(true); + } + + @Override + protected void onAbort() { + } + + public int getStep() { + return step; + } + + } + + public void testStateMachine() throws InterruptedException { + AtomicReference state = new AtomicReference<>(IndexerState.STOPPED); + final ExecutorService executor = Executors.newFixedThreadPool(1); + + try { + + MockIndexer indexer = new MockIndexer(executor, state, 2); + indexer.start(); + assertThat(indexer.getState(), equalTo(IndexerState.STARTED)); + assertTrue(indexer.maybeTriggerAsyncJob(System.currentTimeMillis())); + assertThat(indexer.getState(), equalTo(IndexerState.INDEXING)); + assertThat(indexer.getPosition(), equalTo(2)); + ESTestCase.awaitBusy(() -> isFinished.get()); + assertThat(indexer.getStep(), equalTo(6)); + assertThat(indexer.getStats().getNumInvocations(), equalTo(1L)); + assertThat(indexer.getStats().getNumPages(), equalTo(1L)); + assertThat(indexer.getStats().getOutputDocuments(), equalTo(0L)); + assertTrue(indexer.abort()); + } finally { + executor.shutdownNow(); + } + } +} From a5f6d9328e91009207e1f9e0bfc8e29c8007a884 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Mon, 13 Aug 2018 15:36:59 +0200 Subject: [PATCH 09/20] rename Iteration to IterationResult --- .../xpack/core/indexing/{Iteration.java => IterationResult.java} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/{Iteration.java => IterationResult.java} (100%) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/Iteration.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterationResult.java similarity index 100% rename from x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/Iteration.java rename to x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterationResult.java From 57c99864c2c391ef3fc919f4aa2bef1dbeba2221 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Mon, 13 Aug 2018 16:10:39 +0200 Subject: [PATCH 10/20] address review comments --- .../xpack/core/indexing/IndexerJobStats.java | 20 ++++++++++++++----- .../xpack/core/indexing/IterationResult.java | 4 ++-- .../xpack/core/indexing/IterativeIndexer.java | 12 +++++------ .../rollup/action/GetRollupJobsAction.java | 7 ++++++- .../core/indexing/IterativeIndexerTests.java | 4 ++-- .../xpack/rollup/job/RollupIndexer.java | 6 +++--- 6 files changed, 34 insertions(+), 19 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java index f64ed5804535d..7ef01739f1ea8 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java @@ -30,7 +30,8 @@ public class IndexerJobStats implements ToXContentObject, Writeable { private static ParseField NUM_PAGES = new ParseField("pages_processed"); private static ParseField NUM_INPUT_DOCUMENTS = new ParseField("documents_processed"); // BWC for RollupJobStats - private static ParseField NUM_OUTPUT_DOCUMENTS = new ParseField("documents_indexed").withDeprecation("rollups_indexed"); + private static String ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS = "rollups_indexed"; + private static ParseField NUM_OUTPUT_DOCUMENTS = new ParseField("documents_indexed").withDeprecation(ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS); private static ParseField NUM_INVOCATIONS = new ParseField("trigger_count"); private long numPages = 0; @@ -52,9 +53,9 @@ public class IndexerJobStats implements ToXContentObject, Writeable { public IndexerJobStats() { } - public IndexerJobStats(long numPages, long numDocuments, long numOuputDocuments, long numInvocations) { + public IndexerJobStats(long numPages, long numInputDocuments, long numOuputDocuments, long numInvocations) { this.numPages = numPages; - this.numInputDocuments = numDocuments; + this.numInputDocuments = numInputDocuments; this.numOuputDocuments = numOuputDocuments; this.numInvocations = numInvocations; } @@ -121,11 +122,20 @@ public static IndexerJobStats fromXContent(XContentParser parser) { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); + toUnwrappedXContent(builder, false); + builder.endObject(); + return builder; + } + + public XContentBuilder toUnwrappedXContent(XContentBuilder builder, boolean rollupBWC) throws IOException { builder.field(NUM_PAGES.getPreferredName(), numPages); builder.field(NUM_INPUT_DOCUMENTS.getPreferredName(), numInputDocuments); - builder.field(NUM_OUTPUT_DOCUMENTS.getPreferredName(), numOuputDocuments); + if (rollupBWC) { + builder.field(ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS, numOuputDocuments); + } else { + builder.field(NUM_OUTPUT_DOCUMENTS.getPreferredName(), numOuputDocuments); + } builder.field(NUM_INVOCATIONS.getPreferredName(), numInvocations); - builder.endObject(); return builder; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterationResult.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterationResult.java index 5568ecd5ff806..1261daf185b48 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterationResult.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterationResult.java @@ -14,7 +14,7 @@ * Result object to hold the result of 1 iteration of iterative indexing. * Acts as an interface between the implementation and the generic indexer. */ -public class Iteration { +public class IterationResult { private final boolean isDone; private final JobPosition position; @@ -29,7 +29,7 @@ public class Iteration { * * Note: toIndex.empty() != isDone due to possible filtering in the specific implementation */ - public Iteration(List toIndex, JobPosition position, boolean isDone) { + public IterationResult(List toIndex, JobPosition position, boolean isDone) { this.toIndex = toIndex; this.position = position; this.isDone = isDone; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java index 9ec71b88e466c..b3de03fcb9cf2 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java @@ -173,12 +173,12 @@ public synchronized boolean maybeTriggerAsyncJob(long now) { protected abstract String getJobId(); /** - * Called to process a response from the 1 search request in order to turn it into a {@link Iteration}. + * Called to process a response from the 1 search request in order to turn it into a {@link IterationResult}. * * @param searchResponse response from the search phase. * @return Iteration object to be passed to indexing phase. */ - protected abstract Iteration doProcess(SearchResponse searchResponse); + protected abstract IterationResult doProcess(SearchResponse searchResponse); /** * Called to build the next search request. @@ -295,9 +295,9 @@ private void onSearchResponse(SearchResponse searchResponse) { } stats.incrementNumPages(1); - Iteration iteration = doProcess(searchResponse); + IterationResult iterationResult = doProcess(searchResponse); - if (iteration.isDone()) { + if (iterationResult.isDone()) { logger.debug("Finished indexing for job [" + getJobId() + "], saving state and shutting down."); // Change state first, then try to persist. This prevents in-progress @@ -307,7 +307,7 @@ private void onSearchResponse(SearchResponse searchResponse) { return; } - final List docs = iteration.getToIndex(); + final List docs = iterationResult.getToIndex(); final BulkRequest bulkRequest = new BulkRequest(); docs.forEach(bulkRequest::add); @@ -325,7 +325,7 @@ private void onSearchResponse(SearchResponse searchResponse) { return; } - JobPosition newPosition = iteration.getPosition(); + JobPosition newPosition = iterationResult.getPosition(); position.set(newPosition); onBulkResponse(bulkResponse, newPosition); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java index 4bfd5b621e780..435b888cb95f7 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java @@ -254,7 +254,12 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(CONFIG.getPreferredName()); job.toXContent(builder, params); builder.field(STATUS.getPreferredName(), status); - builder.field(STATS.getPreferredName(), stats); + + // special BWC handling for rollup + builder.startObject(STATS.getPreferredName()); + stats.toUnwrappedXContent(builder, true); + builder.endObject(); + builder.endObject(); return builder; } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java index 85066cb42f519..dd605502f2e3e 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java @@ -45,10 +45,10 @@ protected String getJobId() { } @Override - protected Iteration doProcess(SearchResponse searchResponse) { + protected IterationResult doProcess(SearchResponse searchResponse) { assertThat(step, equalTo(3)); ++step; - return new Iteration(Collections.emptyList(), 3, true); + return new IterationResult(Collections.emptyList(), 3, true); } @Override diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java index c2a07b47349aa..64aeac2e8ae5e 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java @@ -16,7 +16,7 @@ import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.xpack.core.indexing.IndexerState; -import org.elasticsearch.xpack.core.indexing.Iteration; +import org.elasticsearch.xpack.core.indexing.IterationResult; import org.elasticsearch.xpack.core.indexing.IterativeIndexer; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig; @@ -99,10 +99,10 @@ protected SearchRequest buildSearchRequest() { } @Override - protected Iteration> doProcess(SearchResponse searchResponse) { + protected IterationResult> doProcess(SearchResponse searchResponse) { final CompositeAggregation response = searchResponse.getAggregations().get(AGGREGATION_NAME); - return new Iteration<>( + return new IterationResult<>( IndexerUtils.processBuckets(response, job.getConfig().getRollupIndex(), getStats(), job.getConfig().getGroupConfig(), job.getConfig().getId(), upgradedDocumentID.get()), response.afterKey(), response.getBuckets().isEmpty()); From 8c339cc5a21cd556fa3e69d007ecc71d6c608399 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Mon, 13 Aug 2018 16:10:50 +0200 Subject: [PATCH 11/20] Revert "adapt tests for fieldname change" This reverts commit b51853ba570b386ccea467f6d5e5ef7d857bb8f0. --- x-pack/docs/en/rest-api/rollup/get-job.asciidoc | 6 +++--- .../test/resources/rest-api-spec/test/rollup/delete_job.yml | 6 +++--- .../test/resources/rest-api-spec/test/rollup/get_jobs.yml | 6 +++--- .../test/resources/rest-api-spec/test/rollup/put_job.yml | 2 +- .../test/java/org/elasticsearch/multi_node/RollupIT.java | 2 +- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/x-pack/docs/en/rest-api/rollup/get-job.asciidoc b/x-pack/docs/en/rest-api/rollup/get-job.asciidoc index 1a90557106bfc..96053dbfea64f 100644 --- a/x-pack/docs/en/rest-api/rollup/get-job.asciidoc +++ b/x-pack/docs/en/rest-api/rollup/get-job.asciidoc @@ -99,7 +99,7 @@ Which will yield the following response: "stats" : { "pages_processed" : 0, "documents_processed" : 0, - "documents_indexed" : 0, + "rollups_indexed" : 0, "trigger_count" : 0 } } @@ -219,7 +219,7 @@ Which will yield the following response: "stats" : { "pages_processed" : 0, "documents_processed" : 0, - "documents_indexed" : 0, + "rollups_indexed" : 0, "trigger_count" : 0 } }, @@ -268,7 +268,7 @@ Which will yield the following response: "stats" : { "pages_processed" : 0, "documents_processed" : 0, - "documents_indexed" : 0, + "rollups_indexed" : 0, "trigger_count" : 0 } } diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml index d157fd61138d6..298cf27fa2f9d 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml @@ -66,7 +66,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - documents_indexed: 0 + rollups_indexed: 0 trigger_count: 0 status: job_state: "stopped" @@ -113,7 +113,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - documents_indexed: 0 + rollups_indexed: 0 trigger_count: 0 status: job_state: "stopped" @@ -160,7 +160,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - documents_indexed: 0 + rollups_indexed: 0 trigger_count: 0 status: job_state: "stopped" diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml index ec2c6f7581038..f3fa8114ddbd0 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml @@ -67,7 +67,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - documents_indexed: 0 + rollups_indexed: 0 trigger_count: 0 status: job_state: "stopped" @@ -178,7 +178,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - documents_indexed: 0 + rollups_indexed: 0 trigger_count: 0 status: job_state: "stopped" @@ -204,7 +204,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - documents_indexed: 0 + rollups_indexed: 0 trigger_count: 0 status: job_state: "stopped" diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml index e4d635bbe3995..516be25be2a2d 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml @@ -67,7 +67,7 @@ setup: stats: pages_processed: 0 documents_processed: 0 - documents_indexed: 0 + rollups_indexed: 0 trigger_count: 0 status: job_state: "stopped" diff --git a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java index a26f3cfb2b213..43ad4dc0a45a2 100644 --- a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java +++ b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java @@ -156,7 +156,7 @@ public void testBigRollup() throws Exception { Map job = getJob(getRollupJobResponse, "rollup-job-test"); if (job != null) { assertThat(ObjectPath.eval("status.job_state", job), equalTo("started")); - assertThat(ObjectPath.eval("stats.documents_indexed", job), equalTo(41)); + assertThat(ObjectPath.eval("stats.rollups_indexed", job), equalTo(41)); } }, 30L, TimeUnit.SECONDS); From c58759ec261e4a9f81d01b45129692cb7e8eed65 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Tue, 14 Aug 2018 08:46:18 +0200 Subject: [PATCH 12/20] fix test for deprecation warning --- .../org/elasticsearch/test/AbstractSerializingTestCase.java | 2 +- .../xpack/core/rollup/job/JobWrapperSerializingTests.java | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractSerializingTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractSerializingTestCase.java index 6ec32f6654fff..f8b921688d516 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractSerializingTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractSerializingTestCase.java @@ -33,7 +33,7 @@ public abstract class AbstractSerializingTestCase Date: Tue, 14 Aug 2018 10:38:08 +0200 Subject: [PATCH 13/20] Revert "fix test for deprecation warning" This reverts commit c58759ec261e4a9f81d01b45129692cb7e8eed65. --- .../org/elasticsearch/test/AbstractSerializingTestCase.java | 2 +- .../xpack/core/rollup/job/JobWrapperSerializingTests.java | 6 ------ 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractSerializingTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractSerializingTestCase.java index f8b921688d516..6ec32f6654fff 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractSerializingTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractSerializingTestCase.java @@ -33,7 +33,7 @@ public abstract class AbstractSerializingTestCase Date: Tue, 14 Aug 2018 15:47:03 +0200 Subject: [PATCH 14/20] implement BWC based on request params --- .../xpack/core/indexing/IndexerJobStats.java | 16 +++++----- .../rollup/action/GetRollupJobsAction.java | 16 +++++----- .../core/indexing/IndexerJobStatsTests.java | 30 +++++++++++++++++-- .../rollup/rest/RestGetRollupJobsAction.java | 12 ++++++-- 4 files changed, 53 insertions(+), 21 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java index 7ef01739f1ea8..0c69571c35726 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java @@ -30,8 +30,9 @@ public class IndexerJobStats implements ToXContentObject, Writeable { private static ParseField NUM_PAGES = new ParseField("pages_processed"); private static ParseField NUM_INPUT_DOCUMENTS = new ParseField("documents_processed"); // BWC for RollupJobStats - private static String ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS = "rollups_indexed"; - private static ParseField NUM_OUTPUT_DOCUMENTS = new ParseField("documents_indexed").withDeprecation(ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS); + public static String ROLLUP_BWC_XCONTENT_PARAM = "rollup_6_4_compat"; + public static String ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS = "rollups_indexed"; + private static ParseField NUM_OUTPUT_DOCUMENTS = new ParseField("documents_indexed", ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS); private static ParseField NUM_INVOCATIONS = new ParseField("trigger_count"); private long numPages = 0; @@ -122,20 +123,17 @@ public static IndexerJobStats fromXContent(XContentParser parser) { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); - toUnwrappedXContent(builder, false); - builder.endObject(); - return builder; - } - - public XContentBuilder toUnwrappedXContent(XContentBuilder builder, boolean rollupBWC) throws IOException { builder.field(NUM_PAGES.getPreferredName(), numPages); builder.field(NUM_INPUT_DOCUMENTS.getPreferredName(), numInputDocuments); - if (rollupBWC) { + + // rollup BWC handling, defaulting here to the new format while rollup overrides this + if (params.paramAsBoolean(ROLLUP_BWC_XCONTENT_PARAM, false)) { builder.field(ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS, numOuputDocuments); } else { builder.field(NUM_OUTPUT_DOCUMENTS.getPreferredName(), numOuputDocuments); } builder.field(NUM_INVOCATIONS.getPreferredName(), numInvocations); + builder.endObject(); return builder; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java index 435b888cb95f7..372288b2a57f3 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java @@ -174,7 +174,14 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); - builder.field(JOBS.getPreferredName(), jobs); + + // XContentBuilder does not support passing the params object for Iterables + builder.field(JOBS.getPreferredName()); + builder.startArray(); + for (JobWrapper job : jobs) { + job.toXContent(builder, params); + } + builder.endArray(); builder.endObject(); return builder; } @@ -254,12 +261,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(CONFIG.getPreferredName()); job.toXContent(builder, params); builder.field(STATUS.getPreferredName(), status); - - // special BWC handling for rollup - builder.startObject(STATS.getPreferredName()); - stats.toUnwrappedXContent(builder, true); - builder.endObject(); - + builder.field(STATS.getPreferredName(), stats, params); builder.endObject(); return builder; } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerJobStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerJobStatsTests.java index e60573d3ed071..0ff7b89feacb8 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerJobStatsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerJobStatsTests.java @@ -5,10 +5,19 @@ */ package org.elasticsearch.xpack.core.indexing; +import com.carrotsearch.randomizedtesting.WriterOutputStream; + import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractSerializingTestCase; +import java.io.IOException; +import java.io.StringWriter; +import java.util.Collections; + public class IndexerJobStatsTests extends AbstractSerializingTestCase { @Override @@ -27,8 +36,23 @@ protected IndexerJobStats doParseInstance(XContentParser parser) { } public static IndexerJobStats randomStats() { - return new IndexerJobStats(randomNonNegativeLong(), randomNonNegativeLong(), - randomNonNegativeLong(), randomNonNegativeLong()); + return new IndexerJobStats(randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong()); } -} + public void testDeprecation() throws IOException { + IndexerJobStats stats = randomStats(); + + assertTrue(toJson(stats, true).indexOf(IndexerJobStats.ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS) > 0); + assertFalse(toJson(stats, false).indexOf(IndexerJobStats.ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS) > 0); + } + + private String toJson(IndexerJobStats stats, boolean bwc) throws IOException { + final StringWriter writer = new StringWriter(); + try (XContentBuilder builder = XContentFactory.jsonBuilder(new WriterOutputStream(writer))) { + ToXContent.Params params = new ToXContent.MapParams( + Collections.singletonMap(IndexerJobStats.ROLLUP_BWC_XCONTENT_PARAM, String.valueOf(bwc))); + stats.toXContent(builder, params); + } + return writer.toString(); + } +} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java index 00aeb0d06ab65..d118d95e65dc5 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java @@ -13,6 +13,7 @@ import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.xpack.rollup.Rollup; +import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.action.GetRollupJobsAction; import java.io.IOException; @@ -22,7 +23,7 @@ public class RestGetRollupJobsAction extends BaseRestHandler { public RestGetRollupJobsAction(Settings settings, RestController controller) { super(settings); - controller.registerHandler(RestRequest.Method.GET, Rollup.BASE_PATH + "job/{id}/", this); + controller.registerHandler(RestRequest.Method.GET, Rollup.BASE_PATH + "job/{id}/", this); } @Override @@ -30,7 +31,14 @@ protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient String id = restRequest.param(ID.getPreferredName()); GetRollupJobsAction.Request request = new GetRollupJobsAction.Request(id); - return channel -> client.execute(GetRollupJobsAction.INSTANCE, request, new RestToXContentListener<>(channel)); + return channel -> { + // BWC: inject parameter to force the old style rollup output + // injecting after request handling avoids IAE + if (channel.request().hasParam(IndexerJobStats.ROLLUP_BWC_XCONTENT_PARAM) == false) { + channel.request().params().put(IndexerJobStats.ROLLUP_BWC_XCONTENT_PARAM, String.valueOf(true)); + } + client.execute(GetRollupJobsAction.INSTANCE, request, new RestToXContentListener<>(channel)); + }; } @Override From f7385b0c2424f3fbbfa5128beb28a7858904ed6d Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Mon, 20 Aug 2018 16:37:49 -0400 Subject: [PATCH 15/20] Add a yaml test, clean up REST action a bit, add back getNumRollups() --- .../xpack/core/indexing/IndexerJobStats.java | 5 ++ .../rollup/rest/RestGetRollupJobsAction.java | 11 ++-- .../rest-api-spec/test/rollup/get_jobs.yml | 63 +++++++++++++++++++ 3 files changed, 72 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java index 0c69571c35726..19a66743c1b6b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java @@ -84,6 +84,11 @@ public long getOutputDocuments() { return numOuputDocuments; } + @Deprecated + public long getNumRollups() { + return getOutputDocuments(); + } + public void incrementNumPages(long n) { assert(n >= 0); numPages += n; diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java index d118d95e65dc5..5c0b8741b89e4 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java @@ -12,11 +12,9 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestToXContentListener; -import org.elasticsearch.xpack.rollup.Rollup; import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.action.GetRollupJobsAction; - -import java.io.IOException; +import org.elasticsearch.xpack.rollup.Rollup; public class RestGetRollupJobsAction extends BaseRestHandler { public static final ParseField ID = new ParseField("id"); @@ -27,16 +25,15 @@ public RestGetRollupJobsAction(Settings settings, RestController controller) { } @Override - protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { + protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) { String id = restRequest.param(ID.getPreferredName()); + boolean rollupsIndexedFormat = restRequest.paramAsBoolean(IndexerJobStats.ROLLUP_BWC_XCONTENT_PARAM, true); GetRollupJobsAction.Request request = new GetRollupJobsAction.Request(id); return channel -> { // BWC: inject parameter to force the old style rollup output // injecting after request handling avoids IAE - if (channel.request().hasParam(IndexerJobStats.ROLLUP_BWC_XCONTENT_PARAM) == false) { - channel.request().params().put(IndexerJobStats.ROLLUP_BWC_XCONTENT_PARAM, String.valueOf(true)); - } + channel.request().params().put(IndexerJobStats.ROLLUP_BWC_XCONTENT_PARAM, String.valueOf(rollupsIndexedFormat)); client.execute(GetRollupJobsAction.INSTANCE, request, new RestToXContentListener<>(channel)); }; } diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml index f3fa8114ddbd0..45f5e832b128b 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml @@ -210,3 +210,66 @@ setup: job_state: "stopped" upgraded_doc_id: true +--- +"Test stats naming param": + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + xpack.rollup.put_job: + id: foo + body: > + { + "index_pattern": "foo", + "rollup_index": "foo_rollup", + "cron": "*/30 * * * * ?", + "page_size" :10, + "groups" : { + "date_histogram": { + "field": "the_field", + "interval": "1h" + } + }, + "metrics": [ + { + "field": "value_field", + "metrics": ["min", "max", "sum"] + } + ] + } + - is_true: acknowledged + + - do: + xpack.rollup.get_jobs: + id: foo + rollup_6_4_compat: false + + - match: + jobs: + - config: + id: "foo" + index_pattern: "foo" + rollup_index: "foo_rollup" + cron: "*/30 * * * * ?" + page_size: 10 + groups : + date_histogram: + interval: "1h" + field: "the_field" + time_zone: "UTC" + metrics: + - field: "value_field" + metrics: + - "min" + - "max" + - "sum" + timeout: "20s" + stats: + pages_processed: 0 + documents_processed: 0 + documents_indexed: 0 + trigger_count: 0 + status: + job_state: "stopped" + upgraded_doc_id: true + From 4e8ab17e74dcf5926dbc4d7e22496a5739117e06 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Tue, 21 Aug 2018 16:24:02 -0400 Subject: [PATCH 16/20] Change method name to onStartJob() --- .../xpack/core/indexing/IterativeIndexer.java | 36 +++++++++---------- .../core/indexing/IterativeIndexerTests.java | 2 +- .../xpack/rollup/job/RollupIndexer.java | 6 ++-- 3 files changed, 22 insertions(+), 22 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java index b3de03fcb9cf2..07731a46c0986 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java @@ -21,13 +21,13 @@ /** * An abstract class that builds an index incrementally. A background job can be launched using {@link #maybeTriggerAsyncJob(long)}, - * it will create the index from the source index up to the last complete bucket that is allowed to be built (based on job position). + * it will create the index from the source index up to the last complete bucket that is allowed to be built (based on job position). * Only one background job can run simultaneously and {@link #onFinish()} is called when the job * finishes. {@link #onFailure(Exception)} is called if the job fails with an exception and {@link #onAbort()} is called if the indexer is * aborted while a job is running. The indexer must be started ({@link #start()} to allow a background job to run when * {@link #maybeTriggerAsyncJob(long)} is called. {@link #stop()} can be used to stop the background job without aborting the indexer. * - * In a nutshell this is a 2 cycle engine: 1st it sends a query, 2nd it indexes documents based on the response, sends the next query, + * In a nutshell this is a 2 cycle engine: 1st it sends a query, 2nd it indexes documents based on the response, sends the next query, * indexes, queries, indexes, ... until a condition lets the engine pause until the source provides new input. * * @param Type that defines a job position to be defined by the implementation. @@ -73,7 +73,7 @@ public IndexerJobStats getStats() { * Sets the internal state to {@link IndexerState#STARTED} if the previous state * was {@link IndexerState#STOPPED}. Setting the state to STARTED allows a job * to run in the background when {@link #maybeTriggerAsyncJob(long)} is called. - * + * * @return The new state for the indexer (STARTED, INDEXING or ABORTING if the * job was already aborted). */ @@ -88,7 +88,7 @@ public synchronized IndexerState start() { * as soon as the background job detects that the indexer is stopped. If there * is no job running when this function is called, the state is directly set to * {@link IndexerState#STOPPED} and {@link #onFinish()} will never be called. - * + * * @return The new state for the indexer (STOPPED, STOPPING or ABORTING if the * job was already aborted). */ @@ -111,7 +111,7 @@ public synchronized IndexerState stop() { * will be called as soon as the background job detects that the indexer is * aborted. If there is no job running when this function is called, it returns * true and {@link #onAbort()} will never be called. - * + * * @return true if the indexer is aborted, false if a background job is running * and abort is delayed. */ @@ -146,7 +146,7 @@ public synchronized boolean maybeTriggerAsyncJob(long now) { case STARTED: logger.debug("Schedule was triggered for job [" + getJobId() + "], state: [" + currentState + "]"); stats.incrementNumInvocations(1); - onStart(now); + onStartJob(now); if (state.compareAndSet(IndexerState.STARTED, IndexerState.INDEXING)) { // fire off the search. Note this is async, the method will return from here @@ -167,14 +167,14 @@ public synchronized boolean maybeTriggerAsyncJob(long now) { /** * Called to get the Id of the job, used for logging. - * + * * @return a string with the id of the job */ protected abstract String getJobId(); /** * Called to process a response from the 1 search request in order to turn it into a {@link IterationResult}. - * + * * @param searchResponse response from the search phase. * @return Iteration object to be passed to indexing phase. */ @@ -182,18 +182,18 @@ public synchronized boolean maybeTriggerAsyncJob(long now) { /** * Called to build the next search request. - * + * * @return SearchRequest to be passed to the search phase. */ protected abstract SearchRequest buildSearchRequest(); /** - * Called at startup after job has been triggered using {@link #maybeTriggerAsyncJob(long)} and the + * Called at startup after job has been triggered using {@link #maybeTriggerAsyncJob(long)} and the * internal state is {@link IndexerState#STARTED}. - * + * * @param now The current time in milliseconds passed through from {@link #maybeTriggerAsyncJob(long)} */ - protected abstract void onStart(long now); + protected abstract void onStartJob(long now); /** * Executes the {@link SearchRequest} and calls nextPhase with the @@ -233,7 +233,7 @@ public synchronized boolean maybeTriggerAsyncJob(long now) { /** * Called when a failure occurs in an async job causing the execution to stop. - * + * * @param exc * The exception */ @@ -293,10 +293,10 @@ private void onSearchResponse(SearchResponse searchResponse) { throw new RuntimeException("Shard failures encountered while running indexer for job [" + getJobId() + "]: " + Arrays.toString(searchResponse.getShardFailures())); } - + stats.incrementNumPages(1); IterationResult iterationResult = doProcess(searchResponse); - + if (iterationResult.isDone()) { logger.debug("Finished indexing for job [" + getJobId() + "], saving state and shutting down."); @@ -334,7 +334,7 @@ private void onSearchResponse(SearchResponse searchResponse) { finishWithFailure(e); } } - + private void onBulkResponse(BulkResponse response, JobPosition position) { try { @@ -349,7 +349,7 @@ private void onBulkResponse(BulkResponse response, JobPosition position) { finishWithFailure(e); } } - + /** * Checks the {@link IndexerState} and returns false if the execution should be * stopped. @@ -381,4 +381,4 @@ private boolean checkState(IndexerState currentState) { } } -} \ No newline at end of file +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java index dd605502f2e3e..95797d7d1fabc 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java @@ -59,7 +59,7 @@ protected SearchRequest buildSearchRequest() { } @Override - protected void onStart(long now) { + protected void onStartJob(long now) { assertThat(step, equalTo(0)); ++step; } diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java index 64aeac2e8ae5e..a4b33674ddbf1 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java @@ -72,7 +72,7 @@ protected String getJobId() { } @Override - protected void onStart(long now) { + protected void onStartJob(long now) { // this is needed to exclude buckets that can still receive new documents. DateHistogramGroupConfig dateHisto = job.getConfig().getGroupConfig().getDateHistogram(); long rounded = dateHisto.createRounding().round(now); @@ -83,7 +83,7 @@ protected void onStart(long now) { maxBoundary = rounded; } } - + @Override protected SearchRequest buildSearchRequest() { // Indexer is single-threaded, and only place that the ID scheme can get upgraded is doSaveState(), so @@ -97,7 +97,7 @@ protected SearchRequest buildSearchRequest() { .aggregation(compositeBuilder.aggregateAfter(position)); return new SearchRequest(job.getConfig().getIndexPattern()).source(searchSource); } - + @Override protected IterationResult> doProcess(SearchResponse searchResponse) { final CompositeAggregation response = searchResponse.getAggregations().get(AGGREGATION_NAME); From 11d27aa1273fe25072b19da48aab4000e13c3fe9 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Mon, 27 Aug 2018 12:47:37 -0400 Subject: [PATCH 17/20] Make IndexerJobStats abstract, rename IterativeIndexer With IndexerJobStats now abstract, each implementation of of the AsyncTwoPhaseIndexer can implement their own job stats. The base abstract class provides some stats (docs indexed, etc), but allows each specific implementation the ability to add more stats and control how they are displayed in XContent. This removes the need for the BWC flag. --- ...Indexer.java => AsyncTwoPhaseIndexer.java} | 9 +-- .../xpack/core/indexing/IndexerJobStats.java | 60 ++-------------- .../rollup/action/GetRollupJobsAction.java | 5 +- .../rollup/job/RollupIndexerJobStats.java | 70 +++++++++++++++++++ ...ts.java => AsyncTwoPhaseIndexerTests.java} | 7 +- .../core/indexing/IndexerJobStatsTests.java | 58 --------------- .../job/JobWrapperSerializingTests.java | 4 +- .../job/RollupIndexerJobStatsTests.java | 34 +++++++++ .../xpack/rollup/job/RollupIndexer.java | 12 ++-- .../rollup/rest/RestGetRollupJobsAction.java | 9 +-- .../xpack/rollup/job/IndexerUtilsTests.java | 20 +++--- .../rest-api-spec/test/rollup/get_jobs.yml | 62 ---------------- 12 files changed, 143 insertions(+), 207 deletions(-) rename x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/{IterativeIndexer.java => AsyncTwoPhaseIndexer.java} (97%) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupIndexerJobStats.java rename x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/{IterativeIndexerTests.java => AsyncTwoPhaseIndexerTests.java} (93%) delete mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerJobStatsTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupIndexerJobStatsTests.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java similarity index 97% rename from x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java rename to x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java index 07731a46c0986..827f9325a35d1 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IterativeIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java @@ -32,8 +32,8 @@ * * @param Type that defines a job position to be defined by the implementation. */ -public abstract class IterativeIndexer { - private static final Logger logger = Logger.getLogger(IterativeIndexer.class.getName()); +public abstract class AsyncTwoPhaseIndexer { + private static final Logger logger = Logger.getLogger(AsyncTwoPhaseIndexer.class.getName()); private final IndexerJobStats stats; @@ -41,11 +41,12 @@ public abstract class IterativeIndexer { private final AtomicReference position; private final Executor executor; - protected IterativeIndexer(Executor executor, AtomicReference initialState, JobPosition initialPosition) { + protected AsyncTwoPhaseIndexer(Executor executor, AtomicReference initialState, + JobPosition initialPosition, IndexerJobStats jobStats) { this.executor = executor; this.state = initialState; this.position = new AtomicReference<>(initialPosition); - this.stats = new IndexerJobStats(); + this.stats = jobStats; } /** diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java index 19a66743c1b6b..e31dabf71df11 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java @@ -9,47 +9,24 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentParser; + import java.io.IOException; import java.util.Objects; -import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; - /** * This class holds the runtime statistics of a job. The stats are not used by any internal process * and are only for external monitoring/reference. Statistics are not persisted with the job, so if the * allocated task is shutdown/restarted on a different node all the stats will reset. */ -public class IndexerJobStats implements ToXContentObject, Writeable { +public abstract class IndexerJobStats implements ToXContentObject, Writeable { public static final ParseField NAME = new ParseField("job_stats"); - private static ParseField NUM_PAGES = new ParseField("pages_processed"); - private static ParseField NUM_INPUT_DOCUMENTS = new ParseField("documents_processed"); - // BWC for RollupJobStats - public static String ROLLUP_BWC_XCONTENT_PARAM = "rollup_6_4_compat"; - public static String ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS = "rollups_indexed"; - private static ParseField NUM_OUTPUT_DOCUMENTS = new ParseField("documents_indexed", ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS); - private static ParseField NUM_INVOCATIONS = new ParseField("trigger_count"); - - private long numPages = 0; - private long numInputDocuments = 0; - private long numOuputDocuments = 0; - private long numInvocations = 0; - - public static final ConstructingObjectParser PARSER = - new ConstructingObjectParser<>(NAME.getPreferredName(), - args -> new IndexerJobStats((long) args[0], (long) args[1], (long) args[2], (long) args[3])); - - static { - PARSER.declareLong(constructorArg(), NUM_PAGES); - PARSER.declareLong(constructorArg(), NUM_INPUT_DOCUMENTS); - PARSER.declareLong(constructorArg(), NUM_OUTPUT_DOCUMENTS); - PARSER.declareLong(constructorArg(), NUM_INVOCATIONS); - } + protected long numPages = 0; + protected long numInputDocuments = 0; + protected long numOuputDocuments = 0; + protected long numInvocations = 0; public IndexerJobStats() { } @@ -117,31 +94,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(numInvocations); } - public static IndexerJobStats fromXContent(XContentParser parser) { - try { - return PARSER.parse(parser, null); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - builder.field(NUM_PAGES.getPreferredName(), numPages); - builder.field(NUM_INPUT_DOCUMENTS.getPreferredName(), numInputDocuments); - - // rollup BWC handling, defaulting here to the new format while rollup overrides this - if (params.paramAsBoolean(ROLLUP_BWC_XCONTENT_PARAM, false)) { - builder.field(ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS, numOuputDocuments); - } else { - builder.field(NUM_OUTPUT_DOCUMENTS.getPreferredName(), numOuputDocuments); - } - builder.field(NUM_INVOCATIONS.getPreferredName(), numInvocations); - builder.endObject(); - return builder; - } - @Override public boolean equals(Object other) { if (this == other) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java index 372288b2a57f3..1fdc986828b46 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java @@ -27,6 +27,7 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.RollupField; +import org.elasticsearch.xpack.core.rollup.job.RollupIndexerJobStats; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; import org.elasticsearch.xpack.core.rollup.job.RollupJobStatus; @@ -220,7 +221,7 @@ public static class JobWrapper implements Writeable, ToXContentObject { static { PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> RollupJobConfig.fromXContent(p, null), CONFIG); - PARSER.declareObject(ConstructingObjectParser.constructorArg(), IndexerJobStats.PARSER::apply, STATS); + PARSER.declareObject(ConstructingObjectParser.constructorArg(), RollupIndexerJobStats.PARSER::apply, STATS); PARSER.declareObject(ConstructingObjectParser.constructorArg(), RollupJobStatus.PARSER::apply, STATUS); } @@ -232,7 +233,7 @@ public JobWrapper(RollupJobConfig job, IndexerJobStats stats, RollupJobStatus st public JobWrapper(StreamInput in) throws IOException { this.job = new RollupJobConfig(in); - this.stats = new IndexerJobStats(in); + this.stats = new RollupIndexerJobStats(in); this.status = new RollupJobStatus(in); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupIndexerJobStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupIndexerJobStats.java new file mode 100644 index 0000000000000..87915671b79a2 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupIndexerJobStats.java @@ -0,0 +1,70 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.core.rollup.job; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.xpack.core.indexing.IndexerJobStats; + +import java.io.IOException; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; + +/** + * The Rollup specialization of stats for the AsyncTwoPhaseIndexer. + * Note: instead of `documents_indexed`, this XContent show `rollups_indexed` + */ +public class RollupIndexerJobStats extends IndexerJobStats { + private static ParseField NUM_PAGES = new ParseField("pages_processed"); + private static ParseField NUM_INPUT_DOCUMENTS = new ParseField("documents_processed"); + private static ParseField NUM_OUTPUT_DOCUMENTS = new ParseField("rollups_indexed"); + private static ParseField NUM_INVOCATIONS = new ParseField("trigger_count"); + + public static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>(NAME.getPreferredName(), + args -> new RollupIndexerJobStats((long) args[0], (long) args[1], (long) args[2], (long) args[3])); + + static { + PARSER.declareLong(constructorArg(), NUM_PAGES); + PARSER.declareLong(constructorArg(), NUM_INPUT_DOCUMENTS); + PARSER.declareLong(constructorArg(), NUM_OUTPUT_DOCUMENTS); + PARSER.declareLong(constructorArg(), NUM_INVOCATIONS); + } + + public RollupIndexerJobStats() { + super(); + } + + public RollupIndexerJobStats(long numPages, long numInputDocuments, long numOuputDocuments, long numInvocations) { + super(numPages, numInputDocuments, numOuputDocuments, numInvocations); + } + + public RollupIndexerJobStats(StreamInput in) throws IOException { + super(in); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(NUM_PAGES.getPreferredName(), numPages); + builder.field(NUM_INPUT_DOCUMENTS.getPreferredName(), numInputDocuments); + builder.field(NUM_OUTPUT_DOCUMENTS.getPreferredName(), numOuputDocuments); + builder.field(NUM_INVOCATIONS.getPreferredName(), numInvocations); + builder.endObject(); + return builder; + } + + public static RollupIndexerJobStats fromXContent(XContentParser parser) { + try { + return PARSER.parse(parser, null); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java similarity index 93% rename from x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java rename to x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java index 95797d7d1fabc..d4407b1c31139 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IterativeIndexerTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java @@ -16,6 +16,7 @@ import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.rollup.job.RollupIndexerJobStats; import java.util.Collections; import java.util.concurrent.Executor; @@ -26,17 +27,17 @@ import static org.hamcrest.Matchers.equalTo; -public class IterativeIndexerTests extends ESTestCase { +public class AsyncTwoPhaseIndexerTests extends ESTestCase { AtomicBoolean isFinished = new AtomicBoolean(false); - private class MockIndexer extends IterativeIndexer { + private class MockIndexer extends AsyncTwoPhaseIndexer { // test the execution order private int step; protected MockIndexer(Executor executor, AtomicReference initialState, Integer initialPosition) { - super(executor, initialState, initialPosition); + super(executor, initialState, initialPosition, new RollupIndexerJobStats()); } @Override diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerJobStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerJobStatsTests.java deleted file mode 100644 index 0ff7b89feacb8..0000000000000 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/IndexerJobStatsTests.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.core.indexing; - -import com.carrotsearch.randomizedtesting.WriterOutputStream; - -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.test.AbstractSerializingTestCase; - -import java.io.IOException; -import java.io.StringWriter; -import java.util.Collections; - -public class IndexerJobStatsTests extends AbstractSerializingTestCase { - - @Override - protected IndexerJobStats createTestInstance() { - return randomStats(); - } - - @Override - protected Writeable.Reader instanceReader() { - return IndexerJobStats::new; - } - - @Override - protected IndexerJobStats doParseInstance(XContentParser parser) { - return IndexerJobStats.fromXContent(parser); - } - - public static IndexerJobStats randomStats() { - return new IndexerJobStats(randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong()); - } - - public void testDeprecation() throws IOException { - IndexerJobStats stats = randomStats(); - - assertTrue(toJson(stats, true).indexOf(IndexerJobStats.ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS) > 0); - assertFalse(toJson(stats, false).indexOf(IndexerJobStats.ROLLUP_BWC_NUM_OUTPUT_DOCUMENTS) > 0); - } - - private String toJson(IndexerJobStats stats, boolean bwc) throws IOException { - final StringWriter writer = new StringWriter(); - try (XContentBuilder builder = XContentFactory.jsonBuilder(new WriterOutputStream(writer))) { - ToXContent.Params params = new ToXContent.MapParams( - Collections.singletonMap(IndexerJobStats.ROLLUP_BWC_XCONTENT_PARAM, String.valueOf(bwc))); - stats.toXContent(builder, params); - } - return writer.toString(); - } -} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java index cb827c040801d..1ab6e6a55d495 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java @@ -9,7 +9,6 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractSerializingTestCase; import org.elasticsearch.xpack.core.indexing.IndexerState; -import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers; import org.elasticsearch.xpack.core.rollup.action.GetRollupJobsAction; @@ -42,7 +41,8 @@ protected GetRollupJobsAction.JobWrapper createTestInstance() { } return new GetRollupJobsAction.JobWrapper(ConfigTestHelpers.randomRollupJobConfig(random()), - new IndexerJobStats(randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong()), + new RollupIndexerJobStats(randomNonNegativeLong(), randomNonNegativeLong(), + randomNonNegativeLong(), randomNonNegativeLong()), new RollupJobStatus(state, Collections.emptyMap(), randomBoolean())); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupIndexerJobStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupIndexerJobStatsTests.java new file mode 100644 index 0000000000000..81f31e2e5c4eb --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupIndexerJobStatsTests.java @@ -0,0 +1,34 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.core.rollup.job; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractSerializingTestCase; + +public class RollupIndexerJobStatsTests extends AbstractSerializingTestCase { + + @Override + protected RollupIndexerJobStats createTestInstance() { + return randomStats(); + } + + @Override + protected Writeable.Reader instanceReader() { + return RollupIndexerJobStats::new; + } + + @Override + protected RollupIndexerJobStats doParseInstance(XContentParser parser) { + return RollupIndexerJobStats.fromXContent(parser); + } + + public static RollupIndexerJobStats randomStats() { + return new RollupIndexerJobStats(randomNonNegativeLong(), randomNonNegativeLong(), + randomNonNegativeLong(), randomNonNegativeLong()); + } + +} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java index 35db308f4a881..55c8a5d779ca3 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java @@ -17,10 +17,12 @@ import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.xpack.core.indexing.IndexerState; import org.elasticsearch.xpack.core.indexing.IterationResult; -import org.elasticsearch.xpack.core.indexing.IterativeIndexer; +import org.elasticsearch.xpack.core.indexing.AsyncTwoPhaseIndexer; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.GroupConfig; +import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.job.RollupIndexerJobStats; import org.elasticsearch.xpack.core.rollup.job.RollupJob; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; @@ -33,9 +35,9 @@ import java.util.concurrent.atomic.AtomicReference; /** - * An abstract implementation of {@link IterativeIndexer} that builds a rollup index incrementally. + * An abstract implementation of {@link AsyncTwoPhaseIndexer} that builds a rollup index incrementally. */ -public abstract class RollupIndexer extends IterativeIndexer > { +public abstract class RollupIndexer extends AsyncTwoPhaseIndexer > { static final String AGGREGATION_NAME = RollupField.NAME; private final RollupJob job; @@ -53,7 +55,7 @@ public abstract class RollupIndexer extends IterativeIndexer */ RollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, Map initialPosition, AtomicBoolean upgradedDocumentID) { - super(executor, initialState, initialPosition); + super(executor, initialState, initialPosition, new RollupIndexerJobStats()); this.job = job; this.compositeBuilder = createCompositeBuilder(job.getConfig()); this.upgradedDocumentID = upgradedDocumentID; @@ -116,7 +118,7 @@ protected IterationResult> doProcess(SearchResponse searchRe private CompositeAggregationBuilder createCompositeBuilder(RollupJobConfig config) { final GroupConfig groupConfig = config.getGroupConfig(); List> builders = new ArrayList<>(); - + // Add all the agg builders to our request in order: date_histo -> histo -> terms if (groupConfig != null) { builders.addAll(groupConfig.getDateHistogram().toBuilders()); diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java index 5c0b8741b89e4..fcc1f2c4f5734 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestGetRollupJobsAction.java @@ -12,7 +12,6 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestToXContentListener; -import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.action.GetRollupJobsAction; import org.elasticsearch.xpack.rollup.Rollup; @@ -27,15 +26,9 @@ public RestGetRollupJobsAction(Settings settings, RestController controller) { @Override protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) { String id = restRequest.param(ID.getPreferredName()); - boolean rollupsIndexedFormat = restRequest.paramAsBoolean(IndexerJobStats.ROLLUP_BWC_XCONTENT_PARAM, true); GetRollupJobsAction.Request request = new GetRollupJobsAction.Request(id); - return channel -> { - // BWC: inject parameter to force the old style rollup output - // injecting after request handling avoids IAE - channel.request().params().put(IndexerJobStats.ROLLUP_BWC_XCONTENT_PARAM, String.valueOf(rollupsIndexedFormat)); - client.execute(GetRollupJobsAction.INSTANCE, request, new RestToXContentListener<>(channel)); - }; + return channel -> client.execute(GetRollupJobsAction.INSTANCE, request, new RestToXContentListener<>(channel)); } @Override diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java index 5a16740f94611..aef100fe2c3ab 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java @@ -41,6 +41,7 @@ import org.elasticsearch.xpack.core.rollup.job.GroupConfig; import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.MetricConfig; +import org.elasticsearch.xpack.core.rollup.job.RollupIndexerJobStats; import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; import org.joda.time.DateTime; import org.mockito.stubbing.Answer; @@ -64,7 +65,7 @@ public class IndexerUtilsTests extends AggregatorTestCase { public void testMissingFields() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerJobStats stats = new IndexerJobStats(0, 0, 0, 0); + IndexerJobStats stats = new RollupIndexerJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -126,7 +127,7 @@ public void testMissingFields() throws IOException { public void testCorrectFields() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerJobStats stats= new IndexerJobStats(0, 0, 0, 0); + IndexerJobStats stats= new RollupIndexerJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -193,7 +194,7 @@ public void testCorrectFields() throws IOException { public void testNumericTerms() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerJobStats stats= new IndexerJobStats(0, 0, 0, 0); + IndexerJobStats stats= new RollupIndexerJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -249,7 +250,7 @@ public void testNumericTerms() throws IOException { public void testEmptyCounts() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerJobStats stats= new IndexerJobStats(0, 0, 0, 0); + IndexerJobStats stats= new RollupIndexerJobStats(0, 0, 0, 0); String timestampField = "ts"; String valueField = "the_avg"; @@ -355,7 +356,7 @@ public void testKeyOrderingOldID() { // The content of the config don't actually matter for this test // because the test is just looking at agg keys GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), new HistogramGroupConfig(123L, "abc"), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new IndexerJobStats(), groupConfig, "foo", false); + List docs = IndexerUtils.processBuckets(composite, "foo", new RollupIndexerJobStats(), groupConfig, "foo", false); assertThat(docs.size(), equalTo(1)); assertThat(docs.get(0).id(), equalTo("1237859798")); } @@ -399,7 +400,7 @@ public void testKeyOrderingNewID() { }); GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), new HistogramGroupConfig(1L, "abc"), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new IndexerJobStats(), groupConfig, "foo", true); + List docs = IndexerUtils.processBuckets(composite, "foo", new RollupIndexerJobStats(), groupConfig, "foo", true); assertThat(docs.size(), equalTo(1)); assertThat(docs.get(0).id(), equalTo("foo$c9LcrFqeFW92uN_Z7sv1hA")); } @@ -449,7 +450,7 @@ public void testKeyOrderingNewIDLong() { }); GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), new HistogramGroupConfig(1, "abc"), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new IndexerJobStats(), groupConfig, "foo", true); + List docs = IndexerUtils.processBuckets(composite, "foo", new RollupIndexerJobStats(), groupConfig, "foo", true); assertThat(docs.size(), equalTo(1)); assertThat(docs.get(0).id(), equalTo("foo$VAFKZpyaEqYRPLyic57_qw")); } @@ -476,14 +477,15 @@ public void testNullKeys() { }); GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), randomHistogramGroupConfig(random()), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new IndexerJobStats(), groupConfig, "foo", randomBoolean()); + List docs = IndexerUtils.processBuckets(composite, "foo", new RollupIndexerJobStats(), + groupConfig, "foo", randomBoolean()); assertThat(docs.size(), equalTo(1)); assertFalse(Strings.isNullOrEmpty(docs.get(0).id())); } public void testMissingBuckets() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerJobStats stats= new IndexerJobStats(0, 0, 0, 0); + IndexerJobStats stats= new RollupIndexerJobStats(0, 0, 0, 0); String metricField = "metric_field"; String valueField = "value_field"; diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml index 45f5e832b128b..759ddbad2b463 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml @@ -210,66 +210,4 @@ setup: job_state: "stopped" upgraded_doc_id: true ---- -"Test stats naming param": - - - do: - headers: - Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser - xpack.rollup.put_job: - id: foo - body: > - { - "index_pattern": "foo", - "rollup_index": "foo_rollup", - "cron": "*/30 * * * * ?", - "page_size" :10, - "groups" : { - "date_histogram": { - "field": "the_field", - "interval": "1h" - } - }, - "metrics": [ - { - "field": "value_field", - "metrics": ["min", "max", "sum"] - } - ] - } - - is_true: acknowledged - - - do: - xpack.rollup.get_jobs: - id: foo - rollup_6_4_compat: false - - - match: - jobs: - - config: - id: "foo" - index_pattern: "foo" - rollup_index: "foo_rollup" - cron: "*/30 * * * * ?" - page_size: 10 - groups : - date_histogram: - interval: "1h" - field: "the_field" - time_zone: "UTC" - metrics: - - field: "value_field" - metrics: - - "min" - - "max" - - "sum" - timeout: "20s" - stats: - pages_processed: 0 - documents_processed: 0 - documents_indexed: 0 - trigger_count: 0 - status: - job_state: "stopped" - upgraded_doc_id: true From 0f02c1979b433fc79a4a3ca770a4722dd504f3f8 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Mon, 27 Aug 2018 16:13:01 -0400 Subject: [PATCH 18/20] address review comments --- .../xpack/core/indexing/AsyncTwoPhaseIndexer.java | 8 ++++---- .../xpack/core/indexing/IndexerJobStats.java | 5 ----- .../xpack/core/indexing/AsyncTwoPhaseIndexerTests.java | 2 +- .../org/elasticsearch/xpack/rollup/job/RollupIndexer.java | 2 +- 4 files changed, 6 insertions(+), 11 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java index 827f9325a35d1..ee0c0de97e0ae 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java @@ -32,17 +32,17 @@ * * @param Type that defines a job position to be defined by the implementation. */ -public abstract class AsyncTwoPhaseIndexer { +public abstract class AsyncTwoPhaseIndexer { private static final Logger logger = Logger.getLogger(AsyncTwoPhaseIndexer.class.getName()); - private final IndexerJobStats stats; + private final JobStats stats; private final AtomicReference state; private final AtomicReference position; private final Executor executor; protected AsyncTwoPhaseIndexer(Executor executor, AtomicReference initialState, - JobPosition initialPosition, IndexerJobStats jobStats) { + JobPosition initialPosition, JobStats jobStats) { this.executor = executor; this.state = initialState; this.position = new AtomicReference<>(initialPosition); @@ -66,7 +66,7 @@ public JobPosition getPosition() { /** * Get the stats of this indexer. */ - public IndexerJobStats getStats() { + public JobStats getStats() { return stats; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java index e31dabf71df11..2453504a5ba77 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/IndexerJobStats.java @@ -61,11 +61,6 @@ public long getOutputDocuments() { return numOuputDocuments; } - @Deprecated - public long getNumRollups() { - return getOutputDocuments(); - } - public void incrementNumPages(long n) { assert(n >= 0); numPages += n; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java index d4407b1c31139..39f2e3cf2f77a 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java @@ -31,7 +31,7 @@ public class AsyncTwoPhaseIndexerTests extends ESTestCase { AtomicBoolean isFinished = new AtomicBoolean(false); - private class MockIndexer extends AsyncTwoPhaseIndexer { + private class MockIndexer extends AsyncTwoPhaseIndexer { // test the execution order private int step; diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java index d7d6124e08848..b1b052a3659d6 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java @@ -56,7 +56,7 @@ /** * An abstract implementation of {@link AsyncTwoPhaseIndexer} that builds a rollup index incrementally. */ -public abstract class RollupIndexer extends AsyncTwoPhaseIndexer > { +public abstract class RollupIndexer extends AsyncTwoPhaseIndexer, RollupIndexerJobStats> { static final String AGGREGATION_NAME = RollupField.NAME; private final RollupJob job; From 6011a27dc04427c6ba7290204bf6dd8ba4def2de Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Mon, 27 Aug 2018 16:16:20 -0400 Subject: [PATCH 19/20] Use mock JobStats instead of Rollup's version --- .../core/indexing/AsyncTwoPhaseIndexerTests.java | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java index 39f2e3cf2f77a..2662e05570c6d 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java @@ -13,11 +13,12 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponseSections; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.core.rollup.job.RollupIndexerJobStats; +import java.io.IOException; import java.util.Collections; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; @@ -31,13 +32,13 @@ public class AsyncTwoPhaseIndexerTests extends ESTestCase { AtomicBoolean isFinished = new AtomicBoolean(false); - private class MockIndexer extends AsyncTwoPhaseIndexer { + private class MockIndexer extends AsyncTwoPhaseIndexer { // test the execution order private int step; protected MockIndexer(Executor executor, AtomicReference initialState, Integer initialPosition) { - super(executor, initialState, initialPosition, new RollupIndexerJobStats()); + super(executor, initialState, initialPosition, new MockJobStats()); } @Override @@ -109,6 +110,14 @@ public int getStep() { } + private static class MockJobStats extends IndexerJobStats { + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return null; + } + } + public void testStateMachine() throws InterruptedException { AtomicReference state = new AtomicReference<>(IndexerState.STOPPED); final ExecutorService executor = Executors.newFixedThreadPool(1); From cff5a924ea68e1cc1f1e9e5a2853dd9f02925915 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Tue, 28 Aug 2018 11:46:01 -0400 Subject: [PATCH 20/20] Remove more uses of abstract class --- .../xpack/core/rollup/action/GetRollupJobsAction.java | 9 ++++----- .../elasticsearch/xpack/rollup/job/IndexerUtils.java | 4 ++-- .../elasticsearch/xpack/rollup/job/RollupJobTask.java | 4 ++-- .../xpack/rollup/job/IndexerUtilsTests.java | 11 +++++------ 4 files changed, 13 insertions(+), 15 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java index 1fdc986828b46..7bbbf07e6dcbe 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java @@ -25,7 +25,6 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.tasks.Task; -import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.RollupIndexerJobStats; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; @@ -212,12 +211,12 @@ public final String toString() { public static class JobWrapper implements Writeable, ToXContentObject { private final RollupJobConfig job; - private final IndexerJobStats stats; + private final RollupIndexerJobStats stats; private final RollupJobStatus status; public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, a -> new JobWrapper((RollupJobConfig) a[0], - (IndexerJobStats) a[1], (RollupJobStatus)a[2])); + (RollupIndexerJobStats) a[1], (RollupJobStatus)a[2])); static { PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> RollupJobConfig.fromXContent(p, null), CONFIG); @@ -225,7 +224,7 @@ public static class JobWrapper implements Writeable, ToXContentObject { PARSER.declareObject(ConstructingObjectParser.constructorArg(), RollupJobStatus.PARSER::apply, STATUS); } - public JobWrapper(RollupJobConfig job, IndexerJobStats stats, RollupJobStatus status) { + public JobWrapper(RollupJobConfig job, RollupIndexerJobStats stats, RollupJobStatus status) { this.job = job; this.stats = stats; this.status = status; @@ -248,7 +247,7 @@ public RollupJobConfig getJob() { return job; } - public IndexerJobStats getStats() { + public RollupIndexerJobStats getStats() { return stats; } diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java index 594221d921488..94d64b17de8f3 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java @@ -14,10 +14,10 @@ import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; -import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.GroupConfig; +import org.elasticsearch.xpack.core.rollup.job.RollupIndexerJobStats; import org.elasticsearch.xpack.rollup.Rollup; import java.util.ArrayList; @@ -46,7 +46,7 @@ class IndexerUtils { * @param isUpgradedDocID `true` if this job is using the new ID scheme * @return A list of rolled documents derived from the response */ - static List processBuckets(CompositeAggregation agg, String rollupIndex, IndexerJobStats stats, + static List processBuckets(CompositeAggregation agg, String rollupIndex, RollupIndexerJobStats stats, GroupConfig groupConfig, String jobId, boolean isUpgradedDocID) { logger.debug("Buckets: [" + agg.getBuckets().size() + "][" + jobId + "]"); diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java index 4898e099319b5..4a4b53575b238 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java @@ -26,10 +26,10 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.indexing.IndexerState; -import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.action.StartRollupJobAction; import org.elasticsearch.xpack.core.rollup.action.StopRollupJobAction; +import org.elasticsearch.xpack.core.rollup.job.RollupIndexerJobStats; import org.elasticsearch.xpack.core.rollup.job.RollupJob; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; import org.elasticsearch.xpack.core.rollup.job.RollupJobStatus; @@ -218,7 +218,7 @@ public Status getStatus() { * Gets the stats for this task. * @return The stats of this task */ - public IndexerJobStats getStats() { + public RollupIndexerJobStats getStats() { return indexer.getStats(); } diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java index 6bf82fabbdf77..098bc83bc7034 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java @@ -36,7 +36,6 @@ import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.max.MaxAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.sum.SumAggregationBuilder; -import org.elasticsearch.xpack.core.indexing.IndexerJobStats; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.GroupConfig; @@ -67,7 +66,7 @@ public class IndexerUtilsTests extends AggregatorTestCase { public void testMissingFields() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerJobStats stats = new RollupIndexerJobStats(0, 0, 0, 0); + RollupIndexerJobStats stats = new RollupIndexerJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -131,7 +130,7 @@ public void testMissingFields() throws IOException { public void testCorrectFields() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerJobStats stats= new RollupIndexerJobStats(0, 0, 0, 0); + RollupIndexerJobStats stats = new RollupIndexerJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -199,7 +198,7 @@ public void testCorrectFields() throws IOException { public void testNumericTerms() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerJobStats stats= new RollupIndexerJobStats(0, 0, 0, 0); + RollupIndexerJobStats stats= new RollupIndexerJobStats(0, 0, 0, 0); String timestampField = "the_histo"; String valueField = "the_avg"; @@ -256,7 +255,7 @@ public void testNumericTerms() throws IOException { public void testEmptyCounts() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerJobStats stats= new RollupIndexerJobStats(0, 0, 0, 0); + RollupIndexerJobStats stats = new RollupIndexerJobStats(0, 0, 0, 0); String timestampField = "ts"; String valueField = "the_avg"; @@ -492,7 +491,7 @@ public void testNullKeys() { public void testMissingBuckets() throws IOException { String indexName = randomAlphaOfLengthBetween(1, 10); - IndexerJobStats stats= new RollupIndexerJobStats(0, 0, 0, 0); + RollupIndexerJobStats stats = new RollupIndexerJobStats(0, 0, 0, 0); String metricField = "metric_field"; String valueField = "value_field";