|  | @@ -49,6 +49,7 @@ import org.elasticsearch.xpack.core.XPackField;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.core.ml.MlMetadata;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.core.ml.MlTasks;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.core.ml.action.EstimateMemoryUsageAction;
 | 
	
		
			
				|  |  | +import org.elasticsearch.xpack.core.ml.action.GetDataFrameAnalyticsStatsAction;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.core.ml.action.PutDataFrameAnalyticsAction;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.core.ml.action.StartDataFrameAnalyticsAction;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsConfig;
 | 
	
	
		
			
				|  | @@ -57,6 +58,7 @@ import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsTaskState;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.core.ml.job.messages.Messages;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper;
 | 
	
		
			
				|  |  | +import org.elasticsearch.xpack.core.ml.utils.PhaseProgress;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.ml.MachineLearning;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.ml.dataframe.DataFrameAnalyticsManager;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.ml.dataframe.DataFrameAnalyticsTask;
 | 
	
	
		
			
				|  | @@ -73,9 +75,10 @@ import java.util.ArrayList;
 | 
	
		
			
				|  |  |  import java.util.List;
 | 
	
		
			
				|  |  |  import java.util.Map;
 | 
	
		
			
				|  |  |  import java.util.Objects;
 | 
	
		
			
				|  |  | -import java.util.concurrent.atomic.AtomicReference;
 | 
	
		
			
				|  |  |  import java.util.function.Predicate;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN;
 | 
	
		
			
				|  |  | +import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin;
 | 
	
		
			
				|  |  |  import static org.elasticsearch.xpack.core.ml.MlTasks.AWAITING_UPGRADE;
 | 
	
		
			
				|  |  |  import static org.elasticsearch.xpack.ml.MachineLearning.MAX_OPEN_JOBS_PER_NODE;
 | 
	
		
			
				|  |  |  
 | 
	
	
		
			
				|  | @@ -159,70 +162,75 @@ public class TransportStartDataFrameAnalyticsAction
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  |              };
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        AtomicReference<DataFrameAnalyticsConfig> configHolder = new AtomicReference<>();
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  |          // Start persistent task
 | 
	
		
			
				|  |  | -        ActionListener<Void> memoryRequirementRefreshListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | -            aVoid -> {
 | 
	
		
			
				|  |  | +        ActionListener<StartContext> memoryUsageHandledListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | +            startContext -> {
 | 
	
		
			
				|  |  |                  StartDataFrameAnalyticsAction.TaskParams taskParams = new StartDataFrameAnalyticsAction.TaskParams(
 | 
	
		
			
				|  |  | -                    request.getId(), configHolder.get().getVersion());
 | 
	
		
			
				|  |  | +                    request.getId(), startContext.config.getVersion(), startContext.progressOnStart);
 | 
	
		
			
				|  |  |                  persistentTasksService.sendStartRequest(MlTasks.dataFrameAnalyticsTaskId(request.getId()),
 | 
	
		
			
				|  |  |                      MlTasks.DATA_FRAME_ANALYTICS_TASK_NAME, taskParams, waitForAnalyticsToStart);
 | 
	
		
			
				|  |  |              },
 | 
	
		
			
				|  |  |              listener::onFailure
 | 
	
		
			
				|  |  |          );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +        // Perform memory usage estimation for this config
 | 
	
		
			
				|  |  | +        ActionListener<StartContext> startContextListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | +            startContext -> {
 | 
	
		
			
				|  |  | +                estimateMemoryUsageAndUpdateMemoryTracker(startContext, memoryUsageHandledListener);
 | 
	
		
			
				|  |  | +            },
 | 
	
		
			
				|  |  | +            listener::onFailure
 | 
	
		
			
				|  |  | +        );
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        // Get start context
 | 
	
		
			
				|  |  | +        getStartContext(request.getId(), startContextListener);
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    private void estimateMemoryUsageAndUpdateMemoryTracker(StartContext startContext, ActionListener<StartContext> listener) {
 | 
	
		
			
				|  |  | +        final String jobId = startContext.config.getId();
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |          // Tell the job tracker to refresh the memory requirement for this job and all other jobs that have persistent tasks
 | 
	
		
			
				|  |  |          ActionListener<EstimateMemoryUsageAction.Response> estimateMemoryUsageListener = ActionListener.wrap(
 | 
	
		
			
				|  |  |              estimateMemoryUsageResponse -> {
 | 
	
		
			
				|  |  |                  auditor.info(
 | 
	
		
			
				|  |  | -                    request.getId(),
 | 
	
		
			
				|  |  | +                    jobId,
 | 
	
		
			
				|  |  |                      Messages.getMessage(
 | 
	
		
			
				|  |  |                          Messages.DATA_FRAME_ANALYTICS_AUDIT_ESTIMATED_MEMORY_USAGE,
 | 
	
		
			
				|  |  |                          estimateMemoryUsageResponse.getExpectedMemoryWithoutDisk()));
 | 
	
		
			
				|  |  |                  // Validate that model memory limit is sufficient to run the analysis
 | 
	
		
			
				|  |  | -                if (configHolder.get().getModelMemoryLimit()
 | 
	
		
			
				|  |  | +                if (startContext.config.getModelMemoryLimit()
 | 
	
		
			
				|  |  |                      .compareTo(estimateMemoryUsageResponse.getExpectedMemoryWithoutDisk()) < 0) {
 | 
	
		
			
				|  |  |                      ElasticsearchStatusException e =
 | 
	
		
			
				|  |  |                          ExceptionsHelper.badRequestException(
 | 
	
		
			
				|  |  |                              "Cannot start because the configured model memory limit [{}] is lower than the expected memory usage [{}]",
 | 
	
		
			
				|  |  | -                            configHolder.get().getModelMemoryLimit(), estimateMemoryUsageResponse.getExpectedMemoryWithoutDisk());
 | 
	
		
			
				|  |  | +                            startContext.config.getModelMemoryLimit(), estimateMemoryUsageResponse.getExpectedMemoryWithoutDisk());
 | 
	
		
			
				|  |  |                      listener.onFailure(e);
 | 
	
		
			
				|  |  |                      return;
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  |                  // Refresh memory requirement for jobs
 | 
	
		
			
				|  |  |                  memoryTracker.addDataFrameAnalyticsJobMemoryAndRefreshAllOthers(
 | 
	
		
			
				|  |  | -                    request.getId(), configHolder.get().getModelMemoryLimit().getBytes(), memoryRequirementRefreshListener);
 | 
	
		
			
				|  |  | +                    jobId, startContext.config.getModelMemoryLimit().getBytes(), ActionListener.wrap(
 | 
	
		
			
				|  |  | +                        aVoid -> listener.onResponse(startContext), listener::onFailure));
 | 
	
		
			
				|  |  |              },
 | 
	
		
			
				|  |  |              listener::onFailure
 | 
	
		
			
				|  |  |          );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        // Perform memory usage estimation for this config
 | 
	
		
			
				|  |  | -        ActionListener<DataFrameAnalyticsConfig> configListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | -            config -> {
 | 
	
		
			
				|  |  | -                configHolder.set(config);
 | 
	
		
			
				|  |  | -                PutDataFrameAnalyticsAction.Request estimateMemoryUsageRequest = new PutDataFrameAnalyticsAction.Request(config);
 | 
	
		
			
				|  |  | -                ClientHelper.executeAsyncWithOrigin(
 | 
	
		
			
				|  |  | -                    client,
 | 
	
		
			
				|  |  | -                    ClientHelper.ML_ORIGIN,
 | 
	
		
			
				|  |  | -                    EstimateMemoryUsageAction.INSTANCE,
 | 
	
		
			
				|  |  | -                    estimateMemoryUsageRequest,
 | 
	
		
			
				|  |  | -                    estimateMemoryUsageListener);
 | 
	
		
			
				|  |  | -            },
 | 
	
		
			
				|  |  | -            listener::onFailure
 | 
	
		
			
				|  |  | -        );
 | 
	
		
			
				|  |  | +        PutDataFrameAnalyticsAction.Request estimateMemoryUsageRequest = new PutDataFrameAnalyticsAction.Request(startContext.config);
 | 
	
		
			
				|  |  | +        ClientHelper.executeAsyncWithOrigin(
 | 
	
		
			
				|  |  | +            client,
 | 
	
		
			
				|  |  | +            ClientHelper.ML_ORIGIN,
 | 
	
		
			
				|  |  | +            EstimateMemoryUsageAction.INSTANCE,
 | 
	
		
			
				|  |  | +            estimateMemoryUsageRequest,
 | 
	
		
			
				|  |  | +            estimateMemoryUsageListener);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        // Get config
 | 
	
		
			
				|  |  | -        getConfigAndValidate(request.getId(), configListener);
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    private void getConfigAndValidate(String id, ActionListener<DataFrameAnalyticsConfig> finalListener) {
 | 
	
		
			
				|  |  | +    private void getStartContext(String id, ActionListener<StartContext> finalListener) {
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        // Step 5. Validate that there are analyzable data in the source index
 | 
	
		
			
				|  |  | -        ActionListener<DataFrameAnalyticsConfig> validateMappingsMergeListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | -            config -> DataFrameDataExtractorFactory.createForSourceIndices(client,
 | 
	
		
			
				|  |  | +        // Step 6. Validate that there are analyzable data in the source index
 | 
	
		
			
				|  |  | +        ActionListener<StartContext> validateMappingsMergeListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | +            startContext -> DataFrameDataExtractorFactory.createForSourceIndices(client,
 | 
	
		
			
				|  |  |                  "validate_source_index_has_rows-" + id,
 | 
	
		
			
				|  |  | -                config,
 | 
	
		
			
				|  |  | +                startContext.config,
 | 
	
		
			
				|  |  |                  ActionListener.wrap(
 | 
	
		
			
				|  |  |                      dataFrameDataExtractorFactory ->
 | 
	
		
			
				|  |  |                          dataFrameDataExtractorFactory
 | 
	
	
		
			
				|  | @@ -234,10 +242,10 @@ public class TransportStartDataFrameAnalyticsAction
 | 
	
		
			
				|  |  |                                              "Unable to start {} as there are no analyzable data in source indices [{}].",
 | 
	
		
			
				|  |  |                                              RestStatus.BAD_REQUEST,
 | 
	
		
			
				|  |  |                                              id,
 | 
	
		
			
				|  |  | -                                            Strings.arrayToCommaDelimitedString(config.getSource().getIndex())
 | 
	
		
			
				|  |  | +                                            Strings.arrayToCommaDelimitedString(startContext.config.getSource().getIndex())
 | 
	
		
			
				|  |  |                                          ));
 | 
	
		
			
				|  |  |                                      } else {
 | 
	
		
			
				|  |  | -                                        finalListener.onResponse(config);
 | 
	
		
			
				|  |  | +                                        finalListener.onResponse(startContext);
 | 
	
		
			
				|  |  |                                      }
 | 
	
		
			
				|  |  |                                  },
 | 
	
		
			
				|  |  |                                  finalListener::onFailure
 | 
	
	
		
			
				|  | @@ -248,49 +256,94 @@ public class TransportStartDataFrameAnalyticsAction
 | 
	
		
			
				|  |  |              finalListener::onFailure
 | 
	
		
			
				|  |  |          );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        // Step 4. Validate mappings can be merged
 | 
	
		
			
				|  |  | -        ActionListener<DataFrameAnalyticsConfig> toValidateMappingsListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | -            config -> MappingsMerger.mergeMappings(client, config.getHeaders(), config.getSource().getIndex(), ActionListener.wrap(
 | 
	
		
			
				|  |  | -                mappings -> validateMappingsMergeListener.onResponse(config), finalListener::onFailure)),
 | 
	
		
			
				|  |  | +        // Step 5. Validate mappings can be merged
 | 
	
		
			
				|  |  | +        ActionListener<StartContext> toValidateMappingsListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | +            startContext -> MappingsMerger.mergeMappings(client, startContext.config.getHeaders(),
 | 
	
		
			
				|  |  | +                startContext.config.getSource().getIndex(), ActionListener.wrap(
 | 
	
		
			
				|  |  | +                mappings -> validateMappingsMergeListener.onResponse(startContext), finalListener::onFailure)),
 | 
	
		
			
				|  |  |              finalListener::onFailure
 | 
	
		
			
				|  |  |          );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        // Step 3. Validate dest index is empty
 | 
	
		
			
				|  |  | -        ActionListener<DataFrameAnalyticsConfig> toValidateDestEmptyListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | -            config -> checkDestIndexIsEmptyIfExists(config, toValidateMappingsListener),
 | 
	
		
			
				|  |  | +        // Step 4. Validate dest index is empty if task is starting for first time
 | 
	
		
			
				|  |  | +        ActionListener<StartContext> toValidateDestEmptyListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | +            startContext -> {
 | 
	
		
			
				|  |  | +                DataFrameAnalyticsTask.StartingState startingState = DataFrameAnalyticsTask.determineStartingState(
 | 
	
		
			
				|  |  | +                    startContext.config.getId(), startContext.progressOnStart);
 | 
	
		
			
				|  |  | +                switch (startingState) {
 | 
	
		
			
				|  |  | +                    case FIRST_TIME:
 | 
	
		
			
				|  |  | +                        checkDestIndexIsEmptyIfExists(startContext, toValidateMappingsListener);
 | 
	
		
			
				|  |  | +                        break;
 | 
	
		
			
				|  |  | +                    case RESUMING_REINDEXING:
 | 
	
		
			
				|  |  | +                    case RESUMING_ANALYZING:
 | 
	
		
			
				|  |  | +                        toValidateMappingsListener.onResponse(startContext);
 | 
	
		
			
				|  |  | +                        break;
 | 
	
		
			
				|  |  | +                    case FINISHED:
 | 
	
		
			
				|  |  | +                        LOGGER.info("[{}] Job has already finished", startContext.config.getId());
 | 
	
		
			
				|  |  | +                        finalListener.onFailure(ExceptionsHelper.badRequestException(
 | 
	
		
			
				|  |  | +                            "Cannot start because the job has already finished"));
 | 
	
		
			
				|  |  | +                        break;
 | 
	
		
			
				|  |  | +                    default:
 | 
	
		
			
				|  |  | +                        finalListener.onFailure(ExceptionsHelper.serverError("Unexpected starting state " + startingState));
 | 
	
		
			
				|  |  | +                        break;
 | 
	
		
			
				|  |  | +                }
 | 
	
		
			
				|  |  | +            },
 | 
	
		
			
				|  |  |              finalListener::onFailure
 | 
	
		
			
				|  |  |          );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        // Step 2. Validate source and dest; check data extraction is possible
 | 
	
		
			
				|  |  | -        ActionListener<DataFrameAnalyticsConfig> getConfigListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | -            config -> {
 | 
	
		
			
				|  |  | -                new SourceDestValidator(clusterService.state(), indexNameExpressionResolver).check(config);
 | 
	
		
			
				|  |  | -                DataFrameDataExtractorFactory.validateConfigAndSourceIndex(client, config, toValidateDestEmptyListener);
 | 
	
		
			
				|  |  | +        // Step 3. Validate source and dest; check data extraction is possible
 | 
	
		
			
				|  |  | +        ActionListener<StartContext> startContextListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | +            startContext -> {
 | 
	
		
			
				|  |  | +                new SourceDestValidator(clusterService.state(), indexNameExpressionResolver).check(startContext.config);
 | 
	
		
			
				|  |  | +                DataFrameDataExtractorFactory.validateConfigAndSourceIndex(client, startContext.config, ActionListener.wrap(
 | 
	
		
			
				|  |  | +                    config -> toValidateDestEmptyListener.onResponse(startContext), finalListener::onFailure));
 | 
	
		
			
				|  |  |              },
 | 
	
		
			
				|  |  |              finalListener::onFailure
 | 
	
		
			
				|  |  |          );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +        // Step 2. Get stats to recover progress
 | 
	
		
			
				|  |  | +        ActionListener<DataFrameAnalyticsConfig> getConfigListener = ActionListener.wrap(
 | 
	
		
			
				|  |  | +            config -> getProgress(config, ActionListener.wrap(
 | 
	
		
			
				|  |  | +                progress -> startContextListener.onResponse(new StartContext(config, progress)), finalListener::onFailure)),
 | 
	
		
			
				|  |  | +            finalListener::onFailure
 | 
	
		
			
				|  |  | +        );
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |          // Step 1. Get the config
 | 
	
		
			
				|  |  |          configProvider.get(id, getConfigListener);
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    private void checkDestIndexIsEmptyIfExists(DataFrameAnalyticsConfig config, ActionListener<DataFrameAnalyticsConfig> listener) {
 | 
	
		
			
				|  |  | -        String destIndex = config.getDest().getIndex();
 | 
	
		
			
				|  |  | +    private void getProgress(DataFrameAnalyticsConfig config, ActionListener<List<PhaseProgress>> listener) {
 | 
	
		
			
				|  |  | +        GetDataFrameAnalyticsStatsAction.Request getStatsRequest = new GetDataFrameAnalyticsStatsAction.Request(config.getId());
 | 
	
		
			
				|  |  | +        executeAsyncWithOrigin(client, ML_ORIGIN, GetDataFrameAnalyticsStatsAction.INSTANCE, getStatsRequest, ActionListener.wrap(
 | 
	
		
			
				|  |  | +            statsResponse -> {
 | 
	
		
			
				|  |  | +                List<GetDataFrameAnalyticsStatsAction.Response.Stats> stats = statsResponse.getResponse().results();
 | 
	
		
			
				|  |  | +                if (stats.isEmpty()) {
 | 
	
		
			
				|  |  | +                    // The job has been deleted in between
 | 
	
		
			
				|  |  | +                    listener.onFailure(ExceptionsHelper.missingDataFrameAnalytics(config.getId()));
 | 
	
		
			
				|  |  | +                } else {
 | 
	
		
			
				|  |  | +                    listener.onResponse(stats.get(0).getProgress());
 | 
	
		
			
				|  |  | +                }
 | 
	
		
			
				|  |  | +            },
 | 
	
		
			
				|  |  | +            listener::onFailure
 | 
	
		
			
				|  |  | +        ));
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    private void checkDestIndexIsEmptyIfExists(StartContext startContext, ActionListener<StartContext> listener) {
 | 
	
		
			
				|  |  | +        String destIndex = startContext.config.getDest().getIndex();
 | 
	
		
			
				|  |  |          SearchRequest destEmptySearch = new SearchRequest(destIndex);
 | 
	
		
			
				|  |  |          destEmptySearch.source().size(0);
 | 
	
		
			
				|  |  |          destEmptySearch.allowPartialSearchResults(false);
 | 
	
		
			
				|  |  | -        ClientHelper.executeWithHeadersAsync(config.getHeaders(), ClientHelper.ML_ORIGIN, client, SearchAction.INSTANCE,
 | 
	
		
			
				|  |  | +        ClientHelper.executeWithHeadersAsync(startContext.config.getHeaders(), ClientHelper.ML_ORIGIN, client, SearchAction.INSTANCE,
 | 
	
		
			
				|  |  |              destEmptySearch, ActionListener.wrap(
 | 
	
		
			
				|  |  |                  searchResponse -> {
 | 
	
		
			
				|  |  |                      if (searchResponse.getHits().getTotalHits().value > 0) {
 | 
	
		
			
				|  |  |                          listener.onFailure(ExceptionsHelper.badRequestException("dest index [{}] must be empty", destIndex));
 | 
	
		
			
				|  |  |                      } else {
 | 
	
		
			
				|  |  | -                        listener.onResponse(config);
 | 
	
		
			
				|  |  | +                        listener.onResponse(startContext);
 | 
	
		
			
				|  |  |                      }
 | 
	
		
			
				|  |  |                  },
 | 
	
		
			
				|  |  |                  e -> {
 | 
	
		
			
				|  |  |                      if (e instanceof IndexNotFoundException) {
 | 
	
		
			
				|  |  | -                        listener.onResponse(config);
 | 
	
		
			
				|  |  | +                        listener.onResponse(startContext);
 | 
	
		
			
				|  |  |                      } else {
 | 
	
		
			
				|  |  |                          listener.onFailure(e);
 | 
	
		
			
				|  |  |                      }
 | 
	
	
		
			
				|  | @@ -331,6 +384,16 @@ public class TransportStartDataFrameAnalyticsAction
 | 
	
		
			
				|  |  |          });
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    private static class StartContext {
 | 
	
		
			
				|  |  | +        private final DataFrameAnalyticsConfig config;
 | 
	
		
			
				|  |  | +        private final List<PhaseProgress> progressOnStart;
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        private StartContext(DataFrameAnalyticsConfig config, List<PhaseProgress> progressOnStart) {
 | 
	
		
			
				|  |  | +            this.config = config;
 | 
	
		
			
				|  |  | +            this.progressOnStart = progressOnStart;
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      /**
 | 
	
		
			
				|  |  |       * Important: the methods of this class must NOT throw exceptions.  If they did then the callers
 | 
	
		
			
				|  |  |       * of endpoints waiting for a condition tested by this predicate would never get a response.
 | 
	
	
		
			
				|  | @@ -539,4 +602,6 @@ public class TransportStartDataFrameAnalyticsAction
 | 
	
		
			
				|  |  |              this.maxOpenJobs = maxOpenJobs;
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |  }
 |