|  | @@ -22,6 +22,7 @@ import org.elasticsearch.action.search.SearchAction;
 | 
	
		
			
				|  |  |  import org.elasticsearch.action.search.SearchRequest;
 | 
	
		
			
				|  |  |  import org.elasticsearch.action.search.SearchResponse;
 | 
	
		
			
				|  |  |  import org.elasticsearch.client.Client;
 | 
	
		
			
				|  |  | +import org.elasticsearch.common.Nullable;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.logging.LoggerMessageFormat;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.settings.Setting;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.unit.TimeValue;
 | 
	
	
		
			
				|  | @@ -54,6 +55,7 @@ import org.elasticsearch.xpack.dataframe.checkpoint.CheckpointProvider;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.dataframe.checkpoint.DataFrameTransformsCheckpointService;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager;
 | 
	
		
			
				|  |  | +import org.elasticsearch.xpack.dataframe.persistence.SeqNoPrimaryTermAndIndex;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.dataframe.transforms.pivot.AggregationResultUtils;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import java.time.Instant;
 | 
	
	
		
			
				|  | @@ -98,6 +100,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      private final AtomicReference<DataFrameTransformTaskState> taskState;
 | 
	
		
			
				|  |  |      private final AtomicReference<String> stateReason;
 | 
	
		
			
				|  |  | +    private final AtomicReference<SeqNoPrimaryTermAndIndex> seqNoPrimaryTermAndIndex = new AtomicReference<>(null);
 | 
	
		
			
				|  |  |      // the checkpoint of this data frame, storing the checkpoint until data indexing from source to dest is _complete_
 | 
	
		
			
				|  |  |      // Note: Each indexer run creates a new future checkpoint which becomes the current checkpoint only after the indexer run finished
 | 
	
		
			
				|  |  |      private final AtomicLong currentCheckpoint;
 | 
	
	
		
			
				|  | @@ -216,31 +219,6 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |                  ));
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    public DataFrameTransformCheckpoint getLastCheckpoint() {
 | 
	
		
			
				|  |  | -        return getIndexer().getLastCheckpoint();
 | 
	
		
			
				|  |  | -    }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    public DataFrameTransformCheckpoint getNextCheckpoint() {
 | 
	
		
			
				|  |  | -        return getIndexer().getNextCheckpoint();
 | 
	
		
			
				|  |  | -    }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    /**
 | 
	
		
			
				|  |  | -     * Get the in-progress checkpoint
 | 
	
		
			
				|  |  | -     *
 | 
	
		
			
				|  |  | -     * @return checkpoint in progress or 0 if task/indexer is not active
 | 
	
		
			
				|  |  | -     */
 | 
	
		
			
				|  |  | -    public long getInProgressCheckpoint() {
 | 
	
		
			
				|  |  | -        if (getIndexer() == null) {
 | 
	
		
			
				|  |  | -            return 0;
 | 
	
		
			
				|  |  | -        } else {
 | 
	
		
			
				|  |  | -            return indexer.get().getState().equals(IndexerState.INDEXING) ? currentCheckpoint.get() + 1L : 0;
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | -    }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    public synchronized void setTaskStateStopped() {
 | 
	
		
			
				|  |  | -        taskState.set(DataFrameTransformTaskState.STOPPED);
 | 
	
		
			
				|  |  | -    }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  |      /**
 | 
	
		
			
				|  |  |       * Start the background indexer and set the task's state to started
 | 
	
		
			
				|  |  |       * @param startingCheckpoint Set the current checkpoint to this value. If null the
 | 
	
	
		
			
				|  | @@ -270,6 +248,15 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |                  msg));
 | 
	
		
			
				|  |  |              return;
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  | +        // If we are already in a `STARTED` state, we should not attempt to call `.start` on the indexer again.
 | 
	
		
			
				|  |  | +        if (taskState.get() == DataFrameTransformTaskState.STARTED) {
 | 
	
		
			
				|  |  | +            listener.onFailure(new ElasticsearchStatusException(
 | 
	
		
			
				|  |  | +                "Cannot start transform [{}] as it is already STARTED.",
 | 
	
		
			
				|  |  | +                RestStatus.CONFLICT,
 | 
	
		
			
				|  |  | +                getTransformId()
 | 
	
		
			
				|  |  | +            ));
 | 
	
		
			
				|  |  | +            return;
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  |          final IndexerState newState = getIndexer().start();
 | 
	
		
			
				|  |  |          if (Arrays.stream(RUNNING_STATES).noneMatch(newState::equals)) {
 | 
	
		
			
				|  |  |              listener.onFailure(new ElasticsearchException("Cannot start task for data frame transform [{}], because state was [{}]",
 | 
	
	
		
			
				|  | @@ -325,7 +312,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |              return;
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        if (getIndexer().getState() == IndexerState.STOPPED) {
 | 
	
		
			
				|  |  | +        if (getIndexer().getState() == IndexerState.STOPPED || getIndexer().getState() == IndexerState.STOPPING) {
 | 
	
		
			
				|  |  |              return;
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
	
		
			
				|  | @@ -339,10 +326,11 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          IndexerState state = getIndexer().stop();
 | 
	
		
			
				|  |  |          stateReason.set(null);
 | 
	
		
			
				|  |  | -        // We just don't want it to be failed if it is failed
 | 
	
		
			
				|  |  | -        // Either we are running, and the STATE is already started or failed
 | 
	
		
			
				|  |  | -        // doSaveState should transfer the state to STOPPED when it needs to.
 | 
	
		
			
				|  |  | -        taskState.set(DataFrameTransformTaskState.STARTED);
 | 
	
		
			
				|  |  | +        // No reason to keep it in the potentially failed state.
 | 
	
		
			
				|  |  | +        // Since we have called `stop` against the indexer, we have no more fear of triggering again.
 | 
	
		
			
				|  |  | +        // But, since `doSaveState` is asynchronous, it is best to set the state as STARTED so that another `start` call cannot be
 | 
	
		
			
				|  |  | +        // executed while we are wrapping up.
 | 
	
		
			
				|  |  | +        taskState.compareAndSet(DataFrameTransformTaskState.FAILED, DataFrameTransformTaskState.STARTED);
 | 
	
		
			
				|  |  |          if (state == IndexerState.STOPPED) {
 | 
	
		
			
				|  |  |              getIndexer().onStop();
 | 
	
		
			
				|  |  |              getIndexer().doSaveState(state, getIndexer().getPosition(), () -> {});
 | 
	
	
		
			
				|  | @@ -361,8 +349,10 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |              return;
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        if (taskState.get() == DataFrameTransformTaskState.FAILED) {
 | 
	
		
			
				|  |  | -            logger.debug("[{}] schedule was triggered for transform but task is failed. Ignoring trigger.", getTransformId());
 | 
	
		
			
				|  |  | +        if (taskState.get() == DataFrameTransformTaskState.FAILED || taskState.get() == DataFrameTransformTaskState.STOPPED) {
 | 
	
		
			
				|  |  | +            logger.debug("[{}] schedule was triggered for transform but task is [{}]. Ignoring trigger.",
 | 
	
		
			
				|  |  | +                getTransformId(),
 | 
	
		
			
				|  |  | +                taskState.get());
 | 
	
		
			
				|  |  |              return;
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
	
		
			
				|  | @@ -379,7 +369,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          // if it runs for the 1st time we just do it, if not we check for changes
 | 
	
		
			
				|  |  |          if (currentCheckpoint.get() == 0) {
 | 
	
		
			
				|  |  | -            logger.debug("Trigger initial run.");
 | 
	
		
			
				|  |  | +            logger.debug("[{}] trigger initial run.", getTransformId());
 | 
	
		
			
				|  |  |              getIndexer().maybeTriggerAsyncJob(System.currentTimeMillis());
 | 
	
		
			
				|  |  |          } else if (getIndexer().isContinuous()) {
 | 
	
		
			
				|  |  |              getIndexer().maybeTriggerAsyncJob(System.currentTimeMillis());
 | 
	
	
		
			
				|  | @@ -395,17 +385,6 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |          markAsCompleted();
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    public DataFrameTransformProgress getProgress() {
 | 
	
		
			
				|  |  | -        if (indexer.get() == null) {
 | 
	
		
			
				|  |  | -            return null;
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | -        DataFrameTransformProgress indexerProgress = indexer.get().getProgress();
 | 
	
		
			
				|  |  | -        if (indexerProgress == null) {
 | 
	
		
			
				|  |  | -            return null;
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | -        return new DataFrameTransformProgress(indexerProgress);
 | 
	
		
			
				|  |  | -    }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  |      void persistStateToClusterState(DataFrameTransformState state,
 | 
	
		
			
				|  |  |                                      ActionListener<PersistentTasksCustomMetaData.PersistentTask<?>> listener) {
 | 
	
		
			
				|  |  |          updatePersistentTaskState(state, ActionListener.wrap(
 | 
	
	
		
			
				|  | @@ -520,6 +499,19 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |          indexer.set(indexerBuilder.build(this));
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    void updateSeqNoPrimaryTermAndIndex(SeqNoPrimaryTermAndIndex expectedValue, SeqNoPrimaryTermAndIndex newValue) {
 | 
	
		
			
				|  |  | +        boolean updated = seqNoPrimaryTermAndIndex.compareAndSet(expectedValue, newValue);
 | 
	
		
			
				|  |  | +        // This should never happen. We ONLY ever update this value if at initialization or we just finished updating the document
 | 
	
		
			
				|  |  | +        // famous last words...
 | 
	
		
			
				|  |  | +        assert updated :
 | 
	
		
			
				|  |  | +            "[" + getTransformId() + "] unexpected change to seqNoPrimaryTermAndIndex.";
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    @Nullable
 | 
	
		
			
				|  |  | +    SeqNoPrimaryTermAndIndex getSeqNoPrimaryTermAndIndex() {
 | 
	
		
			
				|  |  | +        return seqNoPrimaryTermAndIndex.get();
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      static class ClientDataFrameIndexerBuilder {
 | 
	
		
			
				|  |  |          private Client client;
 | 
	
		
			
				|  |  |          private DataFrameTransformsConfigManager transformsConfigManager;
 | 
	
	
		
			
				|  | @@ -879,6 +871,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |                  next.run();
 | 
	
		
			
				|  |  |                  return;
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |              // This means that the indexer was triggered to discover changes, found none, and exited early.
 | 
	
		
			
				|  |  |              // If the state is `STOPPED` this means that DataFrameTransformTask#stop was called while we were checking for changes.
 | 
	
		
			
				|  |  |              // Allow the stop call path to continue
 | 
	
	
		
			
				|  | @@ -886,12 +879,6 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |                  next.run();
 | 
	
		
			
				|  |  |                  return;
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  | -            // If we are `STOPPED` on a `doSaveState` call, that indicates we transitioned to `STOPPED` from `STOPPING`
 | 
	
		
			
				|  |  | -            // OR we called `doSaveState` manually as the indexer was not actively running.
 | 
	
		
			
				|  |  | -            // Since we save the state to an index, we should make sure that our task state is in parity with the indexer state
 | 
	
		
			
				|  |  | -            if (indexerState.equals(IndexerState.STOPPED)) {
 | 
	
		
			
				|  |  | -                transformTask.setTaskStateStopped();
 | 
	
		
			
				|  |  | -            }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              DataFrameTransformTaskState taskState = transformTask.taskState.get();
 | 
	
		
			
				|  |  |  
 | 
	
	
		
			
				|  | @@ -899,13 +886,21 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |                  && transformTask.currentCheckpoint.get() == 1
 | 
	
		
			
				|  |  |                  && this.isContinuous() == false) {
 | 
	
		
			
				|  |  |                  // set both to stopped so they are persisted as such
 | 
	
		
			
				|  |  | -                taskState = DataFrameTransformTaskState.STOPPED;
 | 
	
		
			
				|  |  |                  indexerState = IndexerState.STOPPED;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |                  auditor.info(transformConfig.getId(), "Data frame finished indexing all data, initiating stop");
 | 
	
		
			
				|  |  |                  logger.info("[{}] data frame transform finished indexing all data, initiating stop.", transformConfig.getId());
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +            // If we are `STOPPED` on a `doSaveState` call, that indicates we transitioned to `STOPPED` from `STOPPING`
 | 
	
		
			
				|  |  | +            // OR we called `doSaveState` manually as the indexer was not actively running.
 | 
	
		
			
				|  |  | +            // Since we save the state to an index, we should make sure that our task state is in parity with the indexer state
 | 
	
		
			
				|  |  | +            if (indexerState.equals(IndexerState.STOPPED)) {
 | 
	
		
			
				|  |  | +                // We don't want adjust the stored taskState because as soon as it is `STOPPED` a user could call
 | 
	
		
			
				|  |  | +                // .start again.
 | 
	
		
			
				|  |  | +                taskState = DataFrameTransformTaskState.STOPPED;
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |              final DataFrameTransformState state = new DataFrameTransformState(
 | 
	
		
			
				|  |  |                  taskState,
 | 
	
		
			
				|  |  |                  indexerState,
 | 
	
	
		
			
				|  | @@ -915,13 +910,18 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |                  getProgress());
 | 
	
		
			
				|  |  |              logger.debug("[{}] updating persistent state of transform to [{}].", transformConfig.getId(), state.toString());
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +            // This could be `null` but the putOrUpdateTransformStoredDoc handles that case just fine
 | 
	
		
			
				|  |  | +            SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex = transformTask.getSeqNoPrimaryTermAndIndex();
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |              // Persist the current state and stats in the internal index. The interval of this method being
 | 
	
		
			
				|  |  |              // called is controlled by AsyncTwoPhaseIndexer#onBulkResponse which calls doSaveState every so
 | 
	
		
			
				|  |  |              // often when doing bulk indexing calls or at the end of one indexing run.
 | 
	
		
			
				|  |  |              transformsConfigManager.putOrUpdateTransformStoredDoc(
 | 
	
		
			
				|  |  |                      new DataFrameTransformStoredDoc(transformId, state, getStats()),
 | 
	
		
			
				|  |  | +                    seqNoPrimaryTermAndIndex,
 | 
	
		
			
				|  |  |                      ActionListener.wrap(
 | 
	
		
			
				|  |  |                              r -> {
 | 
	
		
			
				|  |  | +                                transformTask.updateSeqNoPrimaryTermAndIndex(seqNoPrimaryTermAndIndex, r);
 | 
	
		
			
				|  |  |                                  // for auto stop shutdown the task
 | 
	
		
			
				|  |  |                                  if (state.getTaskState().equals(DataFrameTransformTaskState.STOPPED)) {
 | 
	
		
			
				|  |  |                                      transformTask.shutdown();
 | 
	
	
		
			
				|  | @@ -989,6 +989,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
 | 
	
		
			
				|  |  |                  nextCheckpoint = null;
 | 
	
		
			
				|  |  |                  // Reset our failure count as we have finished and may start again with a new checkpoint
 | 
	
		
			
				|  |  |                  failureCount.set(0);
 | 
	
		
			
				|  |  | +                transformTask.stateReason.set(null);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |                  // With bucket_selector we could have read all the buckets and completed the transform
 | 
	
		
			
				|  |  |                  // but not "see" all the buckets since they were filtered out. Consequently, progress would
 |