[7.x][Transform] decouple task and indexer (#48812)

decouple TransformTask and ClientTransformIndexer. Interaction between the 2 classes are
now moved into a context class which holds shared information.

relates #45369
This commit is contained in:
Hendrik Muhs 2019-11-01 19:39:35 +01:00 committed by GitHub
parent 6ab4645f4e
commit 5ecde37a68
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 1759 additions and 1363 deletions

View File

@ -258,6 +258,10 @@ public class TransformCheckpoint implements Writeable, ToXContentObject {
return NAME + "-" + transformId + "-" + checkpoint;
}
public static boolean isNullOrEmpty (TransformCheckpoint checkpoint) {
return checkpoint == null || checkpoint.isEmpty();
}
/**
* Calculate the diff of 2 checkpoints
*

View File

@ -98,7 +98,6 @@ import org.elasticsearch.xpack.transform.rest.action.compat.RestStartTransformAc
import org.elasticsearch.xpack.transform.rest.action.compat.RestStopTransformActionDeprecated;
import org.elasticsearch.xpack.transform.rest.action.compat.RestUpdateTransformActionDeprecated;
import org.elasticsearch.xpack.transform.transforms.TransformPersistentTasksExecutor;
import org.elasticsearch.xpack.transform.transforms.TransformTask;
import java.io.IOException;
import java.time.Clock;
@ -128,6 +127,19 @@ public class Transform extends Plugin implements ActionPlugin, PersistentTaskPlu
private final SetOnce<TransformCheckpointService> transformCheckpointService = new SetOnce<>();
private final SetOnce<SchedulerEngine> schedulerEngine = new SetOnce<>();
public static final int DEFAULT_FAILURE_RETRIES = 10;
// How many times the transform task can retry on an non-critical failure
public static final Setting<Integer> NUM_FAILURE_RETRIES_SETTING = Setting
.intSetting(
"xpack.transform.num_transform_failure_retries",
DEFAULT_FAILURE_RETRIES,
0,
100,
Setting.Property.NodeScope,
Setting.Property.Dynamic
);
public Transform(Settings settings) {
this.settings = settings;
this.enabled = XPackSettings.TRANSFORM_ENABLED.get(settings);
@ -146,18 +158,27 @@ public class Transform extends Plugin implements ActionPlugin, PersistentTaskPlu
return modules;
}
protected XPackLicenseState getLicenseState() { return XPackPlugin.getSharedLicenseState(); }
protected XPackLicenseState getLicenseState() {
return XPackPlugin.getSharedLicenseState();
}
@Override
public List<RestHandler> getRestHandlers(final Settings settings, final RestController restController,
final ClusterSettings clusterSettings, final IndexScopedSettings indexScopedSettings, final SettingsFilter settingsFilter,
final IndexNameExpressionResolver indexNameExpressionResolver, final Supplier<DiscoveryNodes> nodesInCluster) {
public List<RestHandler> getRestHandlers(
final Settings settings,
final RestController restController,
final ClusterSettings clusterSettings,
final IndexScopedSettings indexScopedSettings,
final SettingsFilter settingsFilter,
final IndexNameExpressionResolver indexNameExpressionResolver,
final Supplier<DiscoveryNodes> nodesInCluster
) {
if (!enabled) {
return emptyList();
}
return Arrays.asList(
return Arrays
.asList(
new RestPutTransformAction(restController),
new RestStartTransformAction(restController),
new RestStopTransformAction(restController),
@ -176,7 +197,7 @@ public class Transform extends Plugin implements ActionPlugin, PersistentTaskPlu
new RestGetTransformStatsActionDeprecated(restController),
new RestPreviewTransformActionDeprecated(restController),
new RestUpdateTransformActionDeprecated(restController)
);
);
}
@Override
@ -185,7 +206,8 @@ public class Transform extends Plugin implements ActionPlugin, PersistentTaskPlu
return emptyList();
}
return Arrays.asList(
return Arrays
.asList(
new ActionHandler<>(PutTransformAction.INSTANCE, TransportPutTransformAction.class),
new ActionHandler<>(StartTransformAction.INSTANCE, TransportStartTransformAction.class),
new ActionHandler<>(StopTransformAction.INSTANCE, TransportStopTransformAction.class),
@ -213,35 +235,45 @@ public class Transform extends Plugin implements ActionPlugin, PersistentTaskPlu
return emptyList();
}
FixedExecutorBuilder indexing = new FixedExecutorBuilder(settings, TASK_THREAD_POOL_NAME, 4, 4,
"transform.task_thread_pool");
FixedExecutorBuilder indexing = new FixedExecutorBuilder(settings, TASK_THREAD_POOL_NAME, 4, 4, "transform.task_thread_pool");
return Collections.singletonList(indexing);
}
@Override
public Collection<Object> createComponents(Client client, ClusterService clusterService, ThreadPool threadPool,
ResourceWatcherService resourceWatcherService, ScriptService scriptService, NamedXContentRegistry xContentRegistry,
Environment environment, NodeEnvironment nodeEnvironment, NamedWriteableRegistry namedWriteableRegistry) {
public Collection<Object> createComponents(
Client client,
ClusterService clusterService,
ThreadPool threadPool,
ResourceWatcherService resourceWatcherService,
ScriptService scriptService,
NamedXContentRegistry xContentRegistry,
Environment environment,
NodeEnvironment nodeEnvironment,
NamedWriteableRegistry namedWriteableRegistry
) {
if (enabled == false || transportClientMode) {
return emptyList();
}
transformAuditor.set(new TransformAuditor(client, clusterService.getNodeName()));
transformConfigManager.set(new TransformConfigManager(client, xContentRegistry));
transformCheckpointService.set(new TransformCheckpointService(client,
transformConfigManager.get(),
transformAuditor.get()));
transformCheckpointService.set(new TransformCheckpointService(client, transformConfigManager.get(), transformAuditor.get()));
return Arrays.asList(transformConfigManager.get(), transformAuditor.get(), transformCheckpointService.get(),
new TransformClusterStateListener(clusterService, client));
return Arrays
.asList(
transformConfigManager.get(),
transformAuditor.get(),
transformCheckpointService.get(),
new TransformClusterStateListener(clusterService, client)
);
}
@Override
public UnaryOperator<Map<String, IndexTemplateMetaData>> getIndexTemplateMetaDataUpgrader() {
return templates -> {
try {
templates.put(TransformInternalIndexConstants.LATEST_INDEX_VERSIONED_NAME,
TransformInternalIndex.getIndexTemplateMetaData());
templates
.put(TransformInternalIndexConstants.LATEST_INDEX_VERSIONED_NAME, TransformInternalIndex.getIndexTemplateMetaData());
} catch (IOException e) {
logger.error("Error creating data frame index template", e);
}
@ -255,8 +287,12 @@ public class Transform extends Plugin implements ActionPlugin, PersistentTaskPlu
}
@Override
public List<PersistentTasksExecutor<?>> getPersistentTasksExecutor(ClusterService clusterService, ThreadPool threadPool,
Client client, SettingsModule settingsModule) {
public List<PersistentTasksExecutor<?>> getPersistentTasksExecutor(
ClusterService clusterService,
ThreadPool threadPool,
Client client,
SettingsModule settingsModule
) {
if (enabled == false || transportClientMode) {
return emptyList();
}
@ -269,20 +305,24 @@ public class Transform extends Plugin implements ActionPlugin, PersistentTaskPlu
assert transformAuditor.get() != null;
assert transformCheckpointService.get() != null;
return Collections.singletonList(
new TransformPersistentTasksExecutor(client,
transformConfigManager.get(),
transformCheckpointService.get(),
schedulerEngine.get(),
transformAuditor.get(),
threadPool,
clusterService,
settingsModule.getSettings()));
return Collections
.singletonList(
new TransformPersistentTasksExecutor(
client,
transformConfigManager.get(),
transformCheckpointService.get(),
schedulerEngine.get(),
transformAuditor.get(),
threadPool,
clusterService,
settingsModule.getSettings()
)
);
}
@Override
public List<Setting<?>> getSettings() {
return Collections.singletonList(TransformTask.NUM_FAILURE_RETRIES_SETTING);
return Collections.singletonList(NUM_FAILURE_RETRIES_SETTING);
}
@Override

View File

@ -51,8 +51,7 @@ public class DefaultCheckpointProvider implements CheckpointProvider {
private TransformCheckpoint nextCheckpoint;
private TransformCheckpoint sourceCheckpoint;
TransformCheckpointingInfoBuilder() {
}
TransformCheckpointingInfoBuilder() {}
TransformCheckpointingInfo build() {
if (lastCheckpoint == null) {
@ -70,11 +69,22 @@ public class DefaultCheckpointProvider implements CheckpointProvider {
long nextCheckpointNumber = nextCheckpoint.getCheckpoint() > 0 ? nextCheckpoint.getCheckpoint() : 0;
return new TransformCheckpointingInfo(
new TransformCheckpointStats(lastCheckpointNumber, null, null,
lastCheckpoint.getTimestamp(), lastCheckpoint.getTimeUpperBound()),
new TransformCheckpointStats(nextCheckpointNumber, nextCheckpointPosition,
nextCheckpointProgress, nextCheckpoint.getTimestamp(), nextCheckpoint.getTimeUpperBound()),
TransformCheckpoint.getBehind(lastCheckpoint, sourceCheckpoint));
new TransformCheckpointStats(
lastCheckpointNumber,
null,
null,
lastCheckpoint.getTimestamp(),
lastCheckpoint.getTimeUpperBound()
),
new TransformCheckpointStats(
nextCheckpointNumber,
nextCheckpointPosition,
nextCheckpointProgress,
nextCheckpoint.getTimestamp(),
nextCheckpoint.getTimeUpperBound()
),
TransformCheckpoint.getBehind(lastCheckpoint, sourceCheckpoint)
);
}
public TransformCheckpointingInfoBuilder setLastCheckpoint(TransformCheckpoint lastCheckpoint) {
@ -110,10 +120,12 @@ public class DefaultCheckpointProvider implements CheckpointProvider {
protected final TransformAuditor transformAuditor;
protected final TransformConfig transformConfig;
public DefaultCheckpointProvider(final Client client,
final TransformConfigManager transformConfigManager,
final TransformAuditor transformAuditor,
final TransformConfig transformConfig) {
public DefaultCheckpointProvider(
final Client client,
final TransformConfigManager transformConfigManager,
final TransformAuditor transformAuditor,
final TransformConfig transformConfig
) {
this.client = client;
this.transformConfigManager = transformConfigManager;
this.transformAuditor = transformAuditor;
@ -126,54 +138,64 @@ public class DefaultCheckpointProvider implements CheckpointProvider {
}
@Override
public void createNextCheckpoint(final TransformCheckpoint lastCheckpoint,
final ActionListener<TransformCheckpoint> listener) {
public void createNextCheckpoint(final TransformCheckpoint lastCheckpoint, final ActionListener<TransformCheckpoint> listener) {
final long timestamp = System.currentTimeMillis();
final long checkpoint = lastCheckpoint != null ? lastCheckpoint.getCheckpoint() + 1 : 1;
final long checkpoint = TransformCheckpoint.isNullOrEmpty(lastCheckpoint) ? 1 : lastCheckpoint.getCheckpoint() + 1;
getIndexCheckpoints(ActionListener.wrap(checkpointsByIndex -> {
reportSourceIndexChanges(lastCheckpoint != null ? lastCheckpoint.getIndicesCheckpoints().keySet() : Collections.emptySet(),
checkpointsByIndex.keySet());
reportSourceIndexChanges(
TransformCheckpoint.isNullOrEmpty(lastCheckpoint)
? Collections.emptySet()
: lastCheckpoint.getIndicesCheckpoints().keySet(),
checkpointsByIndex.keySet()
);
listener.onResponse(new TransformCheckpoint(transformConfig.getId(), timestamp, checkpoint, checkpointsByIndex, 0L));
}, listener::onFailure));
}
protected void getIndexCheckpoints (ActionListener<Map<String, long[]>> listener) {
// 1st get index to see the indexes the user has access to
protected void getIndexCheckpoints(ActionListener<Map<String, long[]>> listener) {
// 1st get index to see the indexes the user has access to
GetIndexRequest getIndexRequest = new GetIndexRequest()
.indices(transformConfig.getSource().getIndex())
.features(new GetIndexRequest.Feature[0])
.indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN);
ClientHelper.executeWithHeadersAsync(transformConfig.getHeaders(), ClientHelper.TRANSFORM_ORIGIN, client, GetIndexAction.INSTANCE,
getIndexRequest, ActionListener.wrap(getIndexResponse -> {
ClientHelper
.executeWithHeadersAsync(
transformConfig.getHeaders(),
ClientHelper.TRANSFORM_ORIGIN,
client,
GetIndexAction.INSTANCE,
getIndexRequest,
ActionListener.wrap(getIndexResponse -> {
Set<String> userIndices = getIndexResponse.getIndices() != null
? new HashSet<>(Arrays.asList(getIndexResponse.getIndices()))
: Collections.emptySet();
? new HashSet<>(Arrays.asList(getIndexResponse.getIndices()))
: Collections.emptySet();
// 2nd get stats request
ClientHelper.executeAsyncWithOrigin(client,
ClientHelper.TRANSFORM_ORIGIN,
IndicesStatsAction.INSTANCE,
new IndicesStatsRequest()
.indices(transformConfig.getSource().getIndex())
.clear()
.indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN),
ActionListener.wrap(
response -> {
ClientHelper
.executeAsyncWithOrigin(
client,
ClientHelper.TRANSFORM_ORIGIN,
IndicesStatsAction.INSTANCE,
new IndicesStatsRequest()
.indices(transformConfig.getSource().getIndex())
.clear()
.indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN),
ActionListener.wrap(response -> {
if (response.getFailedShards() != 0) {
listener.onFailure(
new CheckpointException("Source has [" + response.getFailedShards() + "] failed shards"));
listener
.onFailure(
new CheckpointException("Source has [" + response.getFailedShards() + "] failed shards")
);
return;
}
listener.onResponse(extractIndexCheckPoints(response.getShards(), userIndices));
},
e-> listener.onFailure(new CheckpointException("Failed to create checkpoint", e))
));
},
e -> listener.onFailure(new CheckpointException("Failed to create checkpoint", e))
));
}, e -> listener.onFailure(new CheckpointException("Failed to create checkpoint", e)))
);
}, e -> listener.onFailure(new CheckpointException("Failed to create checkpoint", e)))
);
}
static Map<String, long[]> extractIndexCheckPoints(ShardStats[] shards, Set<String> userIndices) {
@ -220,23 +242,29 @@ public class DefaultCheckpointProvider implements CheckpointProvider {
// create the final structure
Map<String, long[]> checkpointsByIndexReduced = new TreeMap<>();
checkpointsByIndex.forEach((indexName, checkpoints) -> {
checkpointsByIndexReduced.put(indexName, checkpoints.values().stream().mapToLong(l -> l).toArray());
});
checkpointsByIndex
.forEach(
(indexName, checkpoints) -> {
checkpointsByIndexReduced.put(indexName, checkpoints.values().stream().mapToLong(l -> l).toArray());
}
);
return checkpointsByIndexReduced;
}
@Override
public void getCheckpointingInfo(TransformCheckpoint lastCheckpoint,
TransformCheckpoint nextCheckpoint,
TransformIndexerPosition nextCheckpointPosition,
TransformProgress nextCheckpointProgress,
ActionListener<TransformCheckpointingInfo> listener) {
public void getCheckpointingInfo(
TransformCheckpoint lastCheckpoint,
TransformCheckpoint nextCheckpoint,
TransformIndexerPosition nextCheckpointPosition,
TransformProgress nextCheckpointProgress,
ActionListener<TransformCheckpointingInfo> listener
) {
TransformCheckpointingInfoBuilder checkpointingInfoBuilder = new TransformCheckpointingInfoBuilder();
checkpointingInfoBuilder.setLastCheckpoint(lastCheckpoint)
checkpointingInfoBuilder
.setLastCheckpoint(lastCheckpoint)
.setNextCheckpoint(nextCheckpoint)
.setNextCheckpointPosition(nextCheckpointPosition)
.setNextCheckpointProgress(nextCheckpointProgress);
@ -244,16 +272,19 @@ public class DefaultCheckpointProvider implements CheckpointProvider {
long timestamp = System.currentTimeMillis();
getIndexCheckpoints(ActionListener.wrap(checkpointsByIndex -> {
checkpointingInfoBuilder.setSourceCheckpoint(
new TransformCheckpoint(transformConfig.getId(), timestamp, -1L, checkpointsByIndex, 0L));
checkpointingInfoBuilder
.setSourceCheckpoint(new TransformCheckpoint(transformConfig.getId(), timestamp, -1L, checkpointsByIndex, 0L));
listener.onResponse(checkpointingInfoBuilder.build());
}, listener::onFailure));
}
@Override
public void getCheckpointingInfo(long lastCheckpointNumber, TransformIndexerPosition nextCheckpointPosition,
TransformProgress nextCheckpointProgress,
ActionListener<TransformCheckpointingInfo> listener) {
public void getCheckpointingInfo(
long lastCheckpointNumber,
TransformIndexerPosition nextCheckpointPosition,
TransformProgress nextCheckpointProgress,
ActionListener<TransformCheckpointingInfo> listener
) {
TransformCheckpointingInfoBuilder checkpointingInfoBuilder = new TransformCheckpointingInfoBuilder();
@ -262,47 +293,56 @@ public class DefaultCheckpointProvider implements CheckpointProvider {
long timestamp = System.currentTimeMillis();
// <3> got the source checkpoint, notify the user
ActionListener<Map<String, long[]>> checkpointsByIndexListener = ActionListener.wrap(
checkpointsByIndex -> {
checkpointingInfoBuilder.setSourceCheckpoint(
new TransformCheckpoint(transformConfig.getId(), timestamp, -1L, checkpointsByIndex, 0L));
listener.onResponse(checkpointingInfoBuilder.build());
},
e -> {
logger.debug((Supplier<?>) () -> new ParameterizedMessage(
"Failed to retrieve source checkpoint for transform [{}]", transformConfig.getId()), e);
listener.onFailure(new CheckpointException("Failure during source checkpoint info retrieval", e));
}
);
ActionListener<Map<String, long[]>> checkpointsByIndexListener = ActionListener.wrap(checkpointsByIndex -> {
checkpointingInfoBuilder
.setSourceCheckpoint(new TransformCheckpoint(transformConfig.getId(), timestamp, -1L, checkpointsByIndex, 0L));
listener.onResponse(checkpointingInfoBuilder.build());
}, e -> {
logger
.debug(
(Supplier<?>) () -> new ParameterizedMessage(
"[{}] failed to retrieve source checkpoint for transform",
transformConfig.getId()
),
e
);
listener.onFailure(new CheckpointException("Failure during source checkpoint info retrieval", e));
});
// <2> got the next checkpoint, get the source checkpoint
ActionListener<TransformCheckpoint> nextCheckpointListener = ActionListener.wrap(
nextCheckpointObj -> {
checkpointingInfoBuilder.setNextCheckpoint(nextCheckpointObj);
getIndexCheckpoints(checkpointsByIndexListener);
},
e -> {
logger.debug((Supplier<?>) () -> new ParameterizedMessage(
"Failed to retrieve next checkpoint [{}] for transform [{}]", lastCheckpointNumber + 1,
transformConfig.getId()), e);
listener.onFailure(new CheckpointException("Failure during next checkpoint info retrieval", e));
}
);
ActionListener<TransformCheckpoint> nextCheckpointListener = ActionListener.wrap(nextCheckpointObj -> {
checkpointingInfoBuilder.setNextCheckpoint(nextCheckpointObj);
getIndexCheckpoints(checkpointsByIndexListener);
}, e -> {
logger
.debug(
(Supplier<?>) () -> new ParameterizedMessage(
"[{}] failed to retrieve next checkpoint [{}]",
transformConfig.getId(),
lastCheckpointNumber + 1
),
e
);
listener.onFailure(new CheckpointException("Failure during next checkpoint info retrieval", e));
});
// <1> got last checkpoint, get the next checkpoint
ActionListener<TransformCheckpoint> lastCheckpointListener = ActionListener.wrap(
lastCheckpointObj -> {
checkpointingInfoBuilder.lastCheckpoint = lastCheckpointObj;
transformConfigManager.getTransformCheckpoint(transformConfig.getId(), lastCheckpointNumber + 1,
nextCheckpointListener);
},
e -> {
logger.debug((Supplier<?>) () -> new ParameterizedMessage(
"Failed to retrieve last checkpoint [{}] for transform [{}]", lastCheckpointNumber,
transformConfig.getId()), e);
listener.onFailure(new CheckpointException("Failure during last checkpoint info retrieval", e));
}
);
ActionListener<TransformCheckpoint> lastCheckpointListener = ActionListener.wrap(lastCheckpointObj -> {
checkpointingInfoBuilder.lastCheckpoint = lastCheckpointObj;
transformConfigManager.getTransformCheckpoint(transformConfig.getId(), lastCheckpointNumber + 1, nextCheckpointListener);
}, e -> {
logger
.debug(
(Supplier<?>) () -> new ParameterizedMessage(
"[{}] failed to retrieve last checkpoint [{}]",
transformConfig.getId(),
lastCheckpointNumber
),
e
);
listener.onFailure(new CheckpointException("Failure during last checkpoint info retrieval", e));
});
if (lastCheckpointNumber != 0) {
transformConfigManager.getTransformCheckpoint(transformConfig.getId(), lastCheckpointNumber, lastCheckpointListener);
@ -321,20 +361,25 @@ public class DefaultCheckpointProvider implements CheckpointProvider {
// spam protection: only warn the first time
if (newSourceIndexes.isEmpty() && lastSourceIndexes.isEmpty() == false) {
String message = "Source did not resolve to any open indexes";
logger.warn("{} for transform [{}]", message, transformConfig.getId());
logger.warn("[{}] {}", transformConfig.getId(), message);
transformAuditor.warning(transformConfig.getId(), message);
} else {
Set<String> removedIndexes = Sets.difference(lastSourceIndexes, newSourceIndexes);
Set<String> addedIndexes = Sets.difference(newSourceIndexes, lastSourceIndexes);
if (removedIndexes.size() + addedIndexes.size() > AUDIT_CONCRETED_SOURCE_INDEX_CHANGES) {
String message = "Source index resolve found more than " + AUDIT_CONCRETED_SOURCE_INDEX_CHANGES + " changes, ["
+ removedIndexes.size() + "] removed indexes, [" + addedIndexes.size() + "] new indexes";
logger.debug("{} for transform [{}]", message, transformConfig.getId());
String message = "Source index resolve found more than "
+ AUDIT_CONCRETED_SOURCE_INDEX_CHANGES
+ " changes, ["
+ removedIndexes.size()
+ "] removed indexes, ["
+ addedIndexes.size()
+ "] new indexes";
logger.debug("[{}] {}", transformConfig.getId(), message);
transformAuditor.info(transformConfig.getId(), message);
} else if (removedIndexes.size() + addedIndexes.size() > 0) {
String message = "Source index resolve found changes, removedIndexes: " + removedIndexes + ", new indexes: " + addedIndexes;
logger.debug("{} for transform [{}]", message, transformConfig.getId());
logger.debug("[{}] {}", transformConfig.getId(), message);
transformAuditor.info(transformConfig.getId(), message);
}
}

View File

@ -18,9 +18,9 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.RangeQueryBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.xpack.core.ClientHelper;
import org.elasticsearch.xpack.core.transform.transforms.TimeSyncConfig;
import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpoint;
import org.elasticsearch.xpack.core.transform.transforms.TransformConfig;
import org.elasticsearch.xpack.core.transform.transforms.TimeSyncConfig;
import org.elasticsearch.xpack.transform.notifications.TransformAuditor;
import org.elasticsearch.xpack.transform.persistence.TransformConfigManager;
@ -30,59 +30,75 @@ public class TimeBasedCheckpointProvider extends DefaultCheckpointProvider {
private final TimeSyncConfig timeSyncConfig;
TimeBasedCheckpointProvider(final Client client,
final TransformConfigManager transformConfigManager,
final TransformAuditor transformAuditor,
final TransformConfig transformConfig) {
TimeBasedCheckpointProvider(
final Client client,
final TransformConfigManager transformConfigManager,
final TransformAuditor transformAuditor,
final TransformConfig transformConfig
) {
super(client, transformConfigManager, transformAuditor, transformConfig);
timeSyncConfig = (TimeSyncConfig) transformConfig.getSyncConfig();
}
@Override
public void sourceHasChanged(TransformCheckpoint lastCheckpoint,
ActionListener<Boolean> listener) {
public void sourceHasChanged(TransformCheckpoint lastCheckpoint, ActionListener<Boolean> listener) {
final long timestamp = getTime();
SearchRequest searchRequest = new SearchRequest(transformConfig.getSource().getIndex())
.allowPartialSearchResults(false)
.indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN);
.allowPartialSearchResults(false)
.indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN);
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
.size(0)
// we only want to know if there is at least 1 new document
.trackTotalHitsUpTo(1);
.size(0)
// we only want to know if there is at least 1 new document
.trackTotalHitsUpTo(1);
QueryBuilder queryBuilder = transformConfig.getSource().getQueryConfig().getQuery();
BoolQueryBuilder filteredQuery = new BoolQueryBuilder().
filter(queryBuilder).
filter(new RangeQueryBuilder(timeSyncConfig.getField()).
gte(lastCheckpoint.getTimeUpperBound()).
lt(timestamp - timeSyncConfig.getDelay().millis()).format("epoch_millis"));
BoolQueryBuilder filteredQuery = new BoolQueryBuilder()
.filter(queryBuilder)
.filter(
new RangeQueryBuilder(timeSyncConfig.getField())
.gte(lastCheckpoint.getTimeUpperBound())
.lt(timestamp - timeSyncConfig.getDelay().millis())
.format("epoch_millis")
);
sourceBuilder.query(filteredQuery);
searchRequest.source(sourceBuilder);
logger.trace("query for changes based on time: {}", sourceBuilder);
ClientHelper.executeWithHeadersAsync(transformConfig.getHeaders(), ClientHelper.TRANSFORM_ORIGIN, client, SearchAction.INSTANCE,
searchRequest, ActionListener.wrap(r -> {
listener.onResponse(r.getHits().getTotalHits().value > 0L);
}, listener::onFailure));
ClientHelper
.executeWithHeadersAsync(
transformConfig.getHeaders(),
ClientHelper.TRANSFORM_ORIGIN,
client,
SearchAction.INSTANCE,
searchRequest,
ActionListener.wrap(r -> { listener.onResponse(r.getHits().getTotalHits().value > 0L); }, listener::onFailure)
);
}
@Override
public void createNextCheckpoint(final TransformCheckpoint lastCheckpoint,
final ActionListener<TransformCheckpoint> listener) {
public void createNextCheckpoint(final TransformCheckpoint lastCheckpoint, final ActionListener<TransformCheckpoint> listener) {
final long timestamp = getTime();
final long checkpoint = lastCheckpoint != null ? lastCheckpoint.getCheckpoint() + 1 : 1;
final long checkpoint = TransformCheckpoint.isNullOrEmpty(lastCheckpoint) ? 1 : lastCheckpoint.getCheckpoint() + 1;
// for time based synchronization
long timeUpperBound = timestamp - timeSyncConfig.getDelay().millis();
getIndexCheckpoints(ActionListener.wrap(checkpointsByIndex -> {
listener.onResponse(
new TransformCheckpoint(transformConfig.getId(), timestamp, checkpoint, checkpointsByIndex, timeUpperBound));
}, listener::onFailure));
getIndexCheckpoints(
ActionListener
.wrap(
checkpointsByIndex -> {
listener
.onResponse(
new TransformCheckpoint(transformConfig.getId(), timestamp, checkpoint, checkpointsByIndex, timeUpperBound)
);
},
listener::onFailure
)
);
}
// for the purpose of testing

View File

@ -27,7 +27,7 @@ public class SeqNoPrimaryTermAndIndex {
return new SeqNoPrimaryTermAndIndex(response.getSeqNo(), response.getPrimaryTerm(), response.getIndex());
}
SeqNoPrimaryTermAndIndex(long seqNo, long primaryTerm, String index) {
public SeqNoPrimaryTermAndIndex(long seqNo, long primaryTerm, String index) {
this.seqNo = seqNo;
this.primaryTerm = primaryTerm;
this.index = index;
@ -61,9 +61,7 @@ public class SeqNoPrimaryTermAndIndex {
}
SeqNoPrimaryTermAndIndex other = (SeqNoPrimaryTermAndIndex) obj;
return Objects.equals(seqNo, other.seqNo)
&& Objects.equals(primaryTerm, other.primaryTerm)
&& Objects.equals(index, other.index);
return Objects.equals(seqNo, other.seqNo) && Objects.equals(primaryTerm, other.primaryTerm) && Objects.equals(index, other.index);
}
@Override

View File

@ -10,7 +10,6 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.bulk.BulkAction;
import org.elasticsearch.action.bulk.BulkItemResponse;
@ -20,16 +19,14 @@ 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.index.IndexNotFoundException;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.ClientHelper;
import org.elasticsearch.xpack.core.indexing.IndexerState;
import org.elasticsearch.xpack.core.transform.TransformMessages;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerPosition;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats;
import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpoint;
import org.elasticsearch.xpack.core.transform.transforms.TransformConfig;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerPosition;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats;
import org.elasticsearch.xpack.core.transform.transforms.TransformProgress;
import org.elasticsearch.xpack.core.transform.transforms.TransformState;
import org.elasticsearch.xpack.core.transform.transforms.TransformStoredDoc;
@ -37,54 +34,48 @@ import org.elasticsearch.xpack.core.transform.transforms.TransformTaskState;
import org.elasticsearch.xpack.core.transform.utils.ExceptionsHelper;
import org.elasticsearch.xpack.transform.checkpoint.CheckpointProvider;
import org.elasticsearch.xpack.transform.notifications.TransformAuditor;
import org.elasticsearch.xpack.transform.persistence.TransformConfigManager;
import org.elasticsearch.xpack.transform.persistence.SeqNoPrimaryTermAndIndex;
import org.elasticsearch.xpack.transform.transforms.pivot.AggregationResultUtils;
import org.elasticsearch.xpack.transform.persistence.TransformConfigManager;
import java.time.Instant;
import java.util.Map;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
class ClientTransformIndexer extends TransformIndexer {
private static final Logger logger = LogManager.getLogger(ClientTransformIndexer.class);
private long logEvery = 1;
private long logCount = 0;
private final Client client;
private final TransformConfigManager transformsConfigManager;
private final CheckpointProvider checkpointProvider;
private final TransformTask transformTask;
private final AtomicInteger failureCount;
private volatile boolean auditBulkFailures = true;
// Indicates that the source has changed for the current run
private volatile boolean hasSourceChanged = true;
// Keeps track of the last exception that was written to our audit, keeps us from spamming the audit index
private volatile String lastAuditedExceptionMessage = null;
private final AtomicBoolean oldStatsCleanedUp = new AtomicBoolean(false);
private volatile boolean shouldStopAtCheckpoint = false;
private volatile Instant changesLastDetectedAt;
ClientTransformIndexer(TransformConfigManager transformsConfigManager,
CheckpointProvider checkpointProvider,
AtomicReference<IndexerState> initialState,
TransformIndexerPosition initialPosition,
Client client,
TransformAuditor auditor,
TransformIndexerStats initialStats,
TransformConfig transformConfig,
Map<String, String> fieldMappings,
TransformProgress transformProgress,
TransformCheckpoint lastCheckpoint,
TransformCheckpoint nextCheckpoint,
TransformTask parentTask,
boolean shouldStopAtCheckpoint) {
super(ExceptionsHelper.requireNonNull(parentTask, "parentTask")
.getThreadPool()
.executor(ThreadPool.Names.GENERIC),
ExceptionsHelper.requireNonNull(auditor, "auditor"),
private final AtomicReference<SeqNoPrimaryTermAndIndex> seqNoPrimaryTermAndIndex;
ClientTransformIndexer(
Executor executor,
TransformConfigManager transformsConfigManager,
CheckpointProvider checkpointProvider,
TransformProgressGatherer progressGatherer,
AtomicReference<IndexerState> initialState,
TransformIndexerPosition initialPosition,
Client client,
TransformAuditor auditor,
TransformIndexerStats initialStats,
TransformConfig transformConfig,
Map<String, String> fieldMappings,
TransformProgress transformProgress,
TransformCheckpoint lastCheckpoint,
TransformCheckpoint nextCheckpoint,
SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex,
TransformContext context,
boolean shouldStopAtCheckpoint
) {
super(
ExceptionsHelper.requireNonNull(executor, "executor"),
transformsConfigManager,
checkpointProvider,
progressGatherer,
auditor,
transformConfig,
fieldMappings,
ExceptionsHelper.requireNonNull(initialState, "initialState"),
@ -92,246 +83,117 @@ class ClientTransformIndexer extends TransformIndexer {
initialStats == null ? new TransformIndexerStats() : initialStats,
transformProgress,
lastCheckpoint,
nextCheckpoint);
this.transformsConfigManager = ExceptionsHelper.requireNonNull(transformsConfigManager, "transformsConfigManager");
this.checkpointProvider = ExceptionsHelper.requireNonNull(checkpointProvider, "checkpointProvider");
nextCheckpoint,
context
);
this.client = ExceptionsHelper.requireNonNull(client, "client");
this.transformTask = parentTask;
this.failureCount = new AtomicInteger(0);
this.shouldStopAtCheckpoint = shouldStopAtCheckpoint;
}
this.seqNoPrimaryTermAndIndex = new AtomicReference<>(seqNoPrimaryTermAndIndex);
boolean shouldStopAtCheckpoint() {
return shouldStopAtCheckpoint;
}
void setShouldStopAtCheckpoint(boolean shouldStopAtCheckpoint) {
this.shouldStopAtCheckpoint = shouldStopAtCheckpoint;
// TODO: move into context constructor
context.setShouldStopAtCheckpoint(shouldStopAtCheckpoint);
}
void persistShouldStopAtCheckpoint(boolean shouldStopAtCheckpoint, ActionListener<Void> shouldStopAtCheckpointListener) {
if (this.shouldStopAtCheckpoint == shouldStopAtCheckpoint ||
getState() == IndexerState.STOPPED ||
getState() == IndexerState.STOPPING) {
if (context.shouldStopAtCheckpoint() == shouldStopAtCheckpoint
|| getState() == IndexerState.STOPPED
|| getState() == IndexerState.STOPPING) {
shouldStopAtCheckpointListener.onResponse(null);
return;
}
TransformState state = new TransformState(
transformTask.getTaskState(),
context.getTaskState(),
getState(),
getPosition(),
transformTask.getCheckpoint(),
transformTask.getStateReason(),
context.getCheckpoint(),
context.getStateReason(),
getProgress(),
null, //Node attributes
shouldStopAtCheckpoint);
doSaveState(state,
ActionListener.wrap(
r -> {
// We only want to update this internal value if it is persisted as such
this.shouldStopAtCheckpoint = shouldStopAtCheckpoint;
logger.debug("[{}] successfully persisted should_stop_at_checkpoint update [{}]",
getJobId(),
shouldStopAtCheckpoint);
shouldStopAtCheckpointListener.onResponse(null);
},
statsExc -> {
logger.warn("[{}] failed to persist should_stop_at_checkpoint update [{}]",
getJobId(),
shouldStopAtCheckpoint);
shouldStopAtCheckpointListener.onFailure(statsExc);
}
));
}
@Override
protected void onStart(long now, ActionListener<Boolean> listener) {
if (transformTask.getTaskState() == TransformTaskState.FAILED) {
logger.debug("[{}] attempted to start while failed.", getJobId());
listener.onFailure(new ElasticsearchException("Attempted to start a failed transform [{}].", getJobId()));
return;
}
// On each run, we need to get the total number of docs and reset the count of processed docs
// Since multiple checkpoints can be executed in the task while it is running on the same node, we need to gather
// the progress here, and not in the executor.
ActionListener<Void> updateConfigListener = ActionListener.wrap(
updateConfigResponse -> {
if (initialRun()) {
createCheckpoint(ActionListener.wrap(cp -> {
nextCheckpoint = cp;
// If nextCheckpoint > 1, this means that we are now on the checkpoint AFTER the batch checkpoint
// Consequently, the idea of percent complete no longer makes sense.
if (nextCheckpoint.getCheckpoint() > 1) {
progress = new TransformProgress(null, 0L, 0L);
super.onStart(now, listener);
return;
}
TransformProgressGatherer.getInitialProgress(this.client, buildFilterQuery(), getConfig(), ActionListener.wrap(
newProgress -> {
logger.trace("[{}] reset the progress from [{}] to [{}].", getJobId(), progress, newProgress);
progress = newProgress;
super.onStart(now, listener);
},
failure -> {
progress = null;
logger.warn(new ParameterizedMessage("[{}] unable to load progress information for task.",
getJobId()),
failure);
super.onStart(now, listener);
}
));
}, listener::onFailure));
} else {
super.onStart(now, listener);
}
},
listener::onFailure
null, // Node attributes
shouldStopAtCheckpoint
);
// If we are continuous, we will want to verify we have the latest stored configuration
ActionListener<Void> changedSourceListener = ActionListener.wrap(
r -> {
if (isContinuous()) {
transformsConfigManager.getTransformConfiguration(getJobId(), ActionListener.wrap(
config -> {
transformConfig = config;
logger.debug("[{}] successfully refreshed transform config from index.", getJobId());
updateConfigListener.onResponse(null);
},
failure -> {
String msg = TransformMessages.getMessage(
TransformMessages.FAILED_TO_RELOAD_TRANSFORM_CONFIGURATION,
getJobId());
logger.error(msg, failure);
// If the transform config index or the transform config is gone, something serious occurred
// We are in an unknown state and should fail out
if (failure instanceof ResourceNotFoundException) {
updateConfigListener.onFailure(new TransformConfigReloadingException(msg, failure));
} else {
auditor.warning(getJobId(), msg);
updateConfigListener.onResponse(null);
}
}
));
} else {
updateConfigListener.onResponse(null);
}
},
listener::onFailure
);
// If we are not on the initial batch checkpoint and its the first pass of whatever continuous checkpoint we are on,
// we should verify if there are local changes based on the sync config. If not, do not proceed further and exit.
if (transformTask.getCheckpoint() > 0 && initialRun()) {
sourceHasChanged(ActionListener.wrap(
hasChanged -> {
hasSourceChanged = hasChanged;
if (hasChanged) {
changesLastDetectedAt = Instant.now();
logger.debug("[{}] source has changed, triggering new indexer run.", getJobId());
changedSourceListener.onResponse(null);
} else {
logger.trace("[{}] source has not changed, finish indexer early.", getJobId());
// No changes, stop executing
listener.onResponse(false);
}
},
failure -> {
// If we failed determining if the source changed, it's safer to assume there were changes.
// We should allow the failure path to complete as normal
hasSourceChanged = true;
listener.onFailure(failure);
}
));
} else {
hasSourceChanged = true;
changedSourceListener.onResponse(null);
}
}
public CheckpointProvider getCheckpointProvider() {
return checkpointProvider;
}
Instant getChangesLastDetectedAt() {
return changesLastDetectedAt;
}
@Override
public synchronized boolean maybeTriggerAsyncJob(long now) {
if (transformTask.getTaskState() == TransformTaskState.FAILED) {
logger.debug("[{}] schedule was triggered for transform but task is failed. Ignoring trigger.", getJobId());
return false;
}
// ignore trigger if indexer is running, prevents log spam in A2P indexer
IndexerState indexerState = getState();
if (IndexerState.INDEXING.equals(indexerState) || IndexerState.STOPPING.equals(indexerState)) {
logger.debug("[{}] indexer for transform has state [{}]. Ignoring trigger.", getJobId(), indexerState);
return false;
}
return super.maybeTriggerAsyncJob(now);
doSaveState(state, ActionListener.wrap(r -> {
// We only want to update this internal value if it is persisted as such
context.setShouldStopAtCheckpoint(shouldStopAtCheckpoint);
logger.debug("[{}] successfully persisted should_stop_at_checkpoint update [{}]", getJobId(), shouldStopAtCheckpoint);
shouldStopAtCheckpointListener.onResponse(null);
}, statsExc -> {
logger.warn("[{}] failed to persist should_stop_at_checkpoint update [{}]", getJobId(), shouldStopAtCheckpoint);
shouldStopAtCheckpointListener.onFailure(statsExc);
}));
}
@Override
protected void doNextSearch(SearchRequest request, ActionListener<SearchResponse> nextPhase) {
if (transformTask.getTaskState() == TransformTaskState.FAILED) {
if (context.getTaskState() == TransformTaskState.FAILED) {
logger.debug("[{}] attempted to search while failed.", getJobId());
nextPhase.onFailure(new ElasticsearchException("Attempted to do a search request for failed transform [{}].",
getJobId()));
nextPhase.onFailure(new ElasticsearchException("Attempted to do a search request for failed transform [{}].", getJobId()));
return;
}
ClientHelper.executeWithHeadersAsync(transformConfig.getHeaders(), ClientHelper.TRANSFORM_ORIGIN, client,
SearchAction.INSTANCE, request, nextPhase);
ClientHelper
.executeWithHeadersAsync(
transformConfig.getHeaders(),
ClientHelper.TRANSFORM_ORIGIN,
client,
SearchAction.INSTANCE,
request,
nextPhase
);
}
@Override
protected void doNextBulk(BulkRequest request, ActionListener<BulkResponse> nextPhase) {
if (transformTask.getTaskState() == TransformTaskState.FAILED) {
if (context.getTaskState() == TransformTaskState.FAILED) {
logger.debug("[{}] attempted to bulk index while failed.", getJobId());
nextPhase.onFailure(new ElasticsearchException("Attempted to do a bulk index request for failed transform [{}].",
getJobId()));
nextPhase.onFailure(new ElasticsearchException("Attempted to do a bulk index request for failed transform [{}].", getJobId()));
return;
}
ClientHelper.executeWithHeadersAsync(transformConfig.getHeaders(),
ClientHelper.TRANSFORM_ORIGIN,
client,
BulkAction.INSTANCE,
request,
ActionListener.wrap(bulkResponse -> {
if (bulkResponse.hasFailures()) {
int failureCount = 0;
for(BulkItemResponse item : bulkResponse.getItems()) {
if (item.isFailed()) {
failureCount++;
ClientHelper
.executeWithHeadersAsync(
transformConfig.getHeaders(),
ClientHelper.TRANSFORM_ORIGIN,
client,
BulkAction.INSTANCE,
request,
ActionListener.wrap(bulkResponse -> {
if (bulkResponse.hasFailures()) {
int failureCount = 0;
for (BulkItemResponse item : bulkResponse.getItems()) {
if (item.isFailed()) {
failureCount++;
}
// TODO gather information on irrecoverable failures and update isIrrecoverableFailure
}
// TODO gather information on irrecoverable failures and update isIrrecoverableFailure
if (auditBulkFailures) {
String failureMessage = bulkResponse.buildFailureMessage();
logger.debug("[{}] Bulk index failure encountered: {}", getJobId(), failureMessage);
auditor
.warning(
getJobId(),
"Experienced at least ["
+ failureCount
+ "] bulk index failures. See the logs of the node running the transform for details. "
+ failureMessage
);
auditBulkFailures = false;
}
// This calls AsyncTwoPhaseIndexer#finishWithIndexingFailure
// It increments the indexing failure, and then calls the `onFailure` logic
nextPhase
.onFailure(
new BulkIndexingException(
"Bulk index experienced failures. " + "See the logs of the node running the transform for details."
)
);
} else {
auditBulkFailures = true;
nextPhase.onResponse(bulkResponse);
}
if (auditBulkFailures) {
auditor.warning(getJobId(),
"Experienced at least [" +
failureCount +
"] bulk index failures. See the logs of the node running the transform for details. " +
bulkResponse.buildFailureMessage());
auditBulkFailures = false;
}
// This calls AsyncTwoPhaseIndexer#finishWithIndexingFailure
// It increments the indexing failure, and then calls the `onFailure` logic
nextPhase.onFailure(
new BulkIndexingException("Bulk index experienced failures. " +
"See the logs of the node running the transform for details."));
} else {
auditBulkFailures = true;
nextPhase.onResponse(bulkResponse);
}
}, nextPhase::onFailure));
}, nextPhase::onFailure)
);
}
@Override
protected void doSaveState(IndexerState indexerState, TransformIndexerPosition position, Runnable next) {
if (transformTask.getTaskState() == TransformTaskState.FAILED) {
if (context.getTaskState() == TransformTaskState.FAILED) {
logger.debug("[{}] attempted to save state and stats while failed.", getJobId());
// If we are failed, we should call next to allow failure handling to occur if necessary.
next.run();
@ -343,19 +205,18 @@ class ClientTransformIndexer extends TransformIndexer {
return;
}
boolean shouldStopAtCheckpoint = shouldStopAtCheckpoint();
boolean shouldStopAtCheckpoint = context.shouldStopAtCheckpoint();
// If we should stop at the next checkpoint, are STARTED, and with `initialRun()` we are in one of two states
// 1. We have just called `onFinish` completing our request, but `shouldStopAtCheckpoint` was set to `true` before our check
// there and now
// there and now
// 2. We are on the very first run of a NEW checkpoint and got here either through a failure, or the very first save state call.
//
// In either case, we should stop so that we guarantee a consistent state and that there are no partially completed checkpoints
if (shouldStopAtCheckpoint && initialRun() && indexerState.equals(IndexerState.STARTED)) {
indexerState = IndexerState.STOPPED;
auditor.info(transformConfig.getId(), "Transform is no longer in the middle of a checkpoint, initiating stop.");
logger.info("[{}] transform is no longer in the middle of a checkpoint, initiating stop.",
transformConfig.getId());
logger.info("[{}] transform is no longer in the middle of a checkpoint, initiating stop.", transformConfig.getId());
}
// This means that the indexer was triggered to discover changes, found none, and exited early.
@ -366,11 +227,9 @@ class ClientTransformIndexer extends TransformIndexer {
return;
}
TransformTaskState taskState = transformTask.getTaskState();
TransformTaskState taskState = context.getTaskState();
if (indexerState.equals(IndexerState.STARTED)
&& transformTask.getCheckpoint() == 1
&& this.isContinuous() == false) {
if (indexerState.equals(IndexerState.STARTED) && context.getCheckpoint() == 1 && this.isContinuous() == false) {
// set both to stopped so they are persisted as such
indexerState = IndexerState.STOPPED;
@ -397,266 +256,73 @@ class ClientTransformIndexer extends TransformIndexer {
taskState,
indexerState,
position,
transformTask.getCheckpoint(),
transformTask.getStateReason(),
context.getCheckpoint(),
context.getStateReason(),
getProgress(),
null,
shouldStopAtCheckpoint);
shouldStopAtCheckpoint
);
logger.debug("[{}] updating persistent state of transform to [{}].", transformConfig.getId(), state.toString());
doSaveState(state, ActionListener.wrap(
r -> next.run(),
e -> next.run()
));
doSaveState(state, ActionListener.wrap(r -> next.run(), e -> next.run()));
}
private void doSaveState(TransformState state, ActionListener<Void> listener) {
// This could be `null` but the putOrUpdateTransformStoredDoc handles that case just fine
SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex = transformTask.getSeqNoPrimaryTermAndIndex();
SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex = 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(
transformsConfigManager
.putOrUpdateTransformStoredDoc(
new TransformStoredDoc(getJobId(), state, getStats()),
seqNoPrimaryTermAndIndex,
ActionListener.wrap(
r -> {
transformTask.updateSeqNoPrimaryTermAndIndex(seqNoPrimaryTermAndIndex, r);
// for auto stop shutdown the task
if (state.getTaskState().equals(TransformTaskState.STOPPED)) {
transformTask.shutdown();
}
// Only do this clean up once, if it succeeded, no reason to do the query again.
if (oldStatsCleanedUp.compareAndSet(false, true)) {
transformsConfigManager.deleteOldTransformStoredDocuments(getJobId(), ActionListener.wrap(
nil -> {
logger.trace("[{}] deleted old transform stats and state document", getJobId());
listener.onResponse(null);
},
e -> {
String msg = LoggerMessageFormat.format("[{}] failed deleting old transform configurations.",
getJobId());
logger.warn(msg, e);
// If we have failed, we should attempt the clean up again later
oldStatsCleanedUp.set(false);
listener.onResponse(null);
}
));
} else {
listener.onResponse(null);
}
},
statsExc -> {
logger.error(new ParameterizedMessage("[{}] updating stats of transform failed.",
transformConfig.getId()),
statsExc);
auditor.warning(getJobId(),
"Failure updating stats of transform: " + statsExc.getMessage());
// for auto stop shutdown the task
if (state.getTaskState().equals(TransformTaskState.STOPPED)) {
transformTask.shutdown();
}
listener.onFailure(statsExc);
}
));
ActionListener.wrap(r -> {
updateSeqNoPrimaryTermAndIndex(seqNoPrimaryTermAndIndex, r);
// for auto stop shutdown the task
if (state.getTaskState().equals(TransformTaskState.STOPPED)) {
context.shutdown();
}
// Only do this clean up once, if it succeeded, no reason to do the query again.
if (oldStatsCleanedUp.compareAndSet(false, true)) {
transformsConfigManager.deleteOldTransformStoredDocuments(getJobId(), ActionListener.wrap(nil -> {
logger.trace("[{}] deleted old transform stats and state document", getJobId());
listener.onResponse(null);
}, e -> {
String msg = LoggerMessageFormat.format("[{}] failed deleting old transform configurations.", getJobId());
logger.warn(msg, e);
// If we have failed, we should attempt the clean up again later
oldStatsCleanedUp.set(false);
listener.onResponse(null);
}));
} else {
listener.onResponse(null);
}
}, statsExc -> {
logger.error(new ParameterizedMessage("[{}] updating stats of transform failed.", transformConfig.getId()), statsExc);
auditor.warning(getJobId(), "Failure updating stats of transform: " + statsExc.getMessage());
// for auto stop shutdown the task
if (state.getTaskState().equals(TransformTaskState.STOPPED)) {
context.shutdown();
}
listener.onFailure(statsExc);
})
);
}
@Override
protected void onFailure(Exception exc) {
// the failure handler must not throw an exception due to internal problems
try {
handleFailure(exc);
} catch (Exception e) {
logger.error(
new ParameterizedMessage("[{}] transform encountered an unexpected internal exception: ", getJobId()),
e);
}
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 : "[" + getJobId() + "] unexpected change to seqNoPrimaryTermAndIndex.";
}
@Override
protected void onFinish(ActionListener<Void> listener) {
try {
// This indicates an early exit since no changes were found.
// So, don't treat this like a checkpoint being completed, as no work was done.
if (hasSourceChanged == false) {
if (shouldStopAtCheckpoint) {
stop();
}
listener.onResponse(null);
return;
}
// TODO: needs cleanup super is called with a listener, but listener.onResponse is called below
// super.onFinish() fortunately ignores the listener
super.onFinish(listener);
long checkpoint = transformTask.incrementCheckpoint();
lastCheckpoint = getNextCheckpoint();
nextCheckpoint = null;
// Reset our failure count as we have finished and may start again with a new checkpoint
failureCount.set(0);
transformTask.setStateReason(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
// show less than 100% even though we are done.
// NOTE: this method is called in the same thread as the processing thread.
// Theoretically, there should not be a race condition with updating progress here.
// NOTE 2: getPercentComplete should only NOT be null on the first (batch) checkpoint
if (progress != null && progress.getPercentComplete() != null && progress.getPercentComplete() < 100.0) {
progress.incrementDocsProcessed(progress.getTotalDocs() - progress.getDocumentsProcessed());
}
// If the last checkpoint is now greater than 1, that means that we have just processed the first
// continuous checkpoint and should start recording the exponential averages
if (lastCheckpoint != null && lastCheckpoint.getCheckpoint() > 1) {
long docsIndexed = 0;
long docsProcessed = 0;
// This should not happen as we simply create a new one when we reach continuous checkpoints
// but this is a paranoid `null` check
if (progress != null) {
docsIndexed = progress.getDocumentsIndexed();
docsProcessed = progress.getDocumentsProcessed();
}
long durationMs = System.currentTimeMillis() - lastCheckpoint.getTimestamp();
getStats().incrementCheckpointExponentialAverages(durationMs < 0 ? 0 : durationMs, docsIndexed, docsProcessed);
}
if (shouldAuditOnFinish(checkpoint)) {
auditor.info(getJobId(),
"Finished indexing for transform checkpoint [" + checkpoint + "].");
}
logger.debug(
"[{}] finished indexing for transform checkpoint [{}].", getJobId(), checkpoint);
auditBulkFailures = true;
if (shouldStopAtCheckpoint) {
stop();
}
listener.onResponse(null);
} catch (Exception e) {
listener.onFailure(e);
}
}
/**
* Indicates if an audit message should be written when onFinish is called for the given checkpoint
* We audit the first checkpoint, and then every 10 checkpoints until completedCheckpoint == 99
* Then we audit every 100, until completedCheckpoint == 999
*
* Then we always audit every 1_000 checkpoints
*
* @param completedCheckpoint The checkpoint that was just completed
* @return {@code true} if an audit message should be written
*/
protected boolean shouldAuditOnFinish(long completedCheckpoint) {
if (++logCount % logEvery != 0) {
return false;
}
if (completedCheckpoint == 0) {
return true;
}
int log10Checkpoint = (int) Math.floor(Math.log10(completedCheckpoint));
logEvery = log10Checkpoint >= 3 ? 1_000 : (int)Math.pow(10.0, log10Checkpoint);
logCount = 0;
return true;
}
@Override
protected void onStop() {
auditor.info(transformConfig.getId(), "Transform has stopped.");
logger.info("[{}] transform has stopped.", transformConfig.getId());
}
@Override
protected void onAbort() {
auditor.info(transformConfig.getId(), "Received abort request, stopping transform.");
logger.info("[{}] transform received abort request. Stopping indexer.", transformConfig.getId());
transformTask.shutdown();
}
@Override
protected void createCheckpoint(ActionListener<TransformCheckpoint> listener) {
checkpointProvider.createNextCheckpoint(getLastCheckpoint(), ActionListener.wrap(
checkpoint -> transformsConfigManager.putTransformCheckpoint(checkpoint,
ActionListener.wrap(
putCheckPointResponse -> listener.onResponse(checkpoint),
createCheckpointException -> {
logger.warn(new ParameterizedMessage("[{}] failed to create checkpoint.", getJobId()),
createCheckpointException);
listener.onFailure(
new RuntimeException("Failed to create checkpoint due to " + createCheckpointException.getMessage(),
createCheckpointException));
}
)),
getCheckPointException -> {
logger.warn(new ParameterizedMessage("[{}] failed to retrieve checkpoint.", getJobId()),
getCheckPointException);
listener.onFailure(
new RuntimeException("Failed to retrieve checkpoint due to " + getCheckPointException.getMessage(),
getCheckPointException));
}
));
}
@Override
protected void sourceHasChanged(ActionListener<Boolean> hasChangedListener) {
checkpointProvider.sourceHasChanged(getLastCheckpoint(),
ActionListener.wrap(
hasChanged -> {
logger.trace("[{}] change detected [{}].", getJobId(), hasChanged);
hasChangedListener.onResponse(hasChanged);
},
e -> {
logger.warn(
new ParameterizedMessage(
"[{}] failed to detect changes for transform. Skipping update till next check.",
getJobId()),
e);
auditor.warning(getJobId(),
"Failed to detect changes for transform, skipping update till next check. Exception: "
+ e.getMessage());
hasChangedListener.onResponse(false);
}));
}
private boolean isIrrecoverableFailure(Exception e) {
return e instanceof IndexNotFoundException
|| e instanceof AggregationResultUtils.AggregationExtractionException
|| e instanceof TransformConfigReloadingException;
}
synchronized void handleFailure(Exception e) {
logger.warn(new ParameterizedMessage("[{}] transform encountered an exception: ",
getJobId()),
e);
if (handleCircuitBreakingException(e)) {
return;
}
if (isIrrecoverableFailure(e) || failureCount.incrementAndGet() > transformTask.getNumFailureRetries()) {
String failureMessage = isIrrecoverableFailure(e) ?
"task encountered irrecoverable failure: " + e.getMessage() :
"task encountered more than " + transformTask.getNumFailureRetries() + " failures; latest failure: " + e.getMessage();
failIndexer(failureMessage);
} else {
// Since our schedule fires again very quickly after failures it is possible to run into the same failure numerous
// times in a row, very quickly. We do not want to spam the audit log with repeated failures, so only record the first one
if (e.getMessage().equals(lastAuditedExceptionMessage) == false) {
auditor.warning(getJobId(),
"Transform encountered an exception: " + e.getMessage() +
" Will attempt again at next scheduled trigger.");
lastAuditedExceptionMessage = e.getMessage();
}
}
}
@Override
protected void failIndexer(String failureMessage) {
logger.error("[{}] transform has failed; experienced: [{}].", getJobId(), failureMessage);
auditor.error(getJobId(), failureMessage);
transformTask.markAsFailed(failureMessage, ActionListener.wrap(
r -> {
// Successfully marked as failed, reset counter so that task can be restarted
failureCount.set(0);
}, e -> {}));
@Nullable
SeqNoPrimaryTermAndIndex getSeqNoPrimaryTermAndIndex() {
return seqNoPrimaryTermAndIndex.get();
}
// Considered a recoverable indexing failure
@ -666,9 +332,4 @@ class ClientTransformIndexer extends TransformIndexer {
}
}
private static class TransformConfigReloadingException extends ElasticsearchException {
TransformConfigReloadingException(String msg, Throwable cause, Object... args) {
super(msg, cause, args);
}
}
}

View File

@ -8,17 +8,19 @@ package org.elasticsearch.xpack.transform.transforms;
import org.elasticsearch.client.Client;
import org.elasticsearch.xpack.core.indexing.IndexerState;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerPosition;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats;
import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpoint;
import org.elasticsearch.xpack.core.transform.transforms.TransformConfig;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerPosition;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats;
import org.elasticsearch.xpack.core.transform.transforms.TransformProgress;
import org.elasticsearch.xpack.transform.checkpoint.CheckpointProvider;
import org.elasticsearch.xpack.transform.checkpoint.TransformCheckpointService;
import org.elasticsearch.xpack.transform.notifications.TransformAuditor;
import org.elasticsearch.xpack.transform.persistence.SeqNoPrimaryTermAndIndex;
import org.elasticsearch.xpack.transform.persistence.TransformConfigManager;
import java.util.Map;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicReference;
class ClientTransformIndexerBuilder {
@ -34,29 +36,35 @@ class ClientTransformIndexerBuilder {
private TransformProgress progress;
private TransformCheckpoint lastCheckpoint;
private TransformCheckpoint nextCheckpoint;
private SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex;
private boolean shouldStopAtCheckpoint;
ClientTransformIndexerBuilder() {
this.initialStats = new TransformIndexerStats();
}
ClientTransformIndexer build(TransformTask parentTask) {
ClientTransformIndexer build(Executor executor, TransformContext context) {
CheckpointProvider checkpointProvider = transformsCheckpointService.getCheckpointProvider(transformConfig);
return new ClientTransformIndexer(this.transformsConfigManager,
return new ClientTransformIndexer(
executor,
transformsConfigManager,
checkpointProvider,
new TransformProgressGatherer(client),
new AtomicReference<>(this.indexerState),
this.initialPosition,
this.client,
this.auditor,
this.initialStats,
this.transformConfig,
this.fieldMappings,
this.progress,
this.lastCheckpoint,
this.nextCheckpoint,
parentTask,
this.shouldStopAtCheckpoint);
initialPosition,
client,
auditor,
initialStats,
transformConfig,
fieldMappings,
progress,
TransformCheckpoint.isNullOrEmpty(lastCheckpoint) ? TransformCheckpoint.EMPTY : lastCheckpoint,
TransformCheckpoint.isNullOrEmpty(nextCheckpoint) ? TransformCheckpoint.EMPTY : nextCheckpoint,
seqNoPrimaryTermAndIndex,
context,
shouldStopAtCheckpoint
);
}
ClientTransformIndexerBuilder setShouldStopAtCheckpoint(boolean shouldStopAtCheckpoint) {
@ -127,4 +135,10 @@ class ClientTransformIndexerBuilder {
this.nextCheckpoint = nextCheckpoint;
return this;
}
ClientTransformIndexerBuilder setSeqNoPrimaryTermAndIndex(SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex) {
this.seqNoPrimaryTermAndIndex = seqNoPrimaryTermAndIndex;
return this;
}
}

View File

@ -0,0 +1,137 @@
/*
* 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.transform.transforms;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.xpack.core.transform.transforms.TransformTaskState;
import org.elasticsearch.xpack.transform.Transform;
import java.time.Instant;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
class TransformContext {
public interface Listener {
void shutdown();
void fail(String failureMessage, ActionListener<Void> listener);
}
private final AtomicReference<TransformTaskState> taskState;
private final AtomicReference<String> stateReason;
private final Listener taskListener;
private volatile int numFailureRetries = Transform.DEFAULT_FAILURE_RETRIES;
private final AtomicInteger failureCount;
private volatile Instant changesLastDetectedAt;
private volatile boolean shouldStopAtCheckpoint;
// the checkpoint of this transform, 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;
TransformContext(final TransformTaskState taskState, String stateReason, long currentCheckpoint, Listener taskListener) {
this.taskState = new AtomicReference<>(taskState);
this.stateReason = new AtomicReference<>(stateReason);
this.currentCheckpoint = new AtomicLong(currentCheckpoint);
this.taskListener = taskListener;
this.failureCount = new AtomicInteger(0);
this.shouldStopAtCheckpoint = shouldStopAtCheckpoint;
}
TransformTaskState getTaskState() {
return taskState.get();
}
void setTaskState(TransformTaskState newState) {
taskState.set(newState);
}
boolean setTaskState(TransformTaskState oldState, TransformTaskState newState) {
return taskState.compareAndSet(oldState, newState);
}
void resetTaskState() {
taskState.set(TransformTaskState.STARTED);
stateReason.set(null);
}
void setTaskStateToFailed(String reason) {
taskState.set(TransformTaskState.FAILED);
stateReason.set(reason);
}
void resetReasonAndFailureCounter() {
stateReason.set(null);
failureCount.set(0);
}
String getStateReason() {
return stateReason.get();
}
void setCheckpoint(long newValue) {
currentCheckpoint.set(newValue);
}
long getCheckpoint() {
return currentCheckpoint.get();
}
long getAndIncrementCheckpoint() {
return currentCheckpoint.getAndIncrement();
}
void setNumFailureRetries(int numFailureRetries) {
this.numFailureRetries = numFailureRetries;
}
int getNumFailureRetries() {
return numFailureRetries;
}
int getAndIncrementFailureCount() {
return failureCount.getAndIncrement();
}
void setChangesLastDetectedAt(Instant time) {
changesLastDetectedAt = time;
}
Instant getChangesLastDetectedAt() {
return changesLastDetectedAt;
}
public boolean shouldStopAtCheckpoint() {
return shouldStopAtCheckpoint;
}
public void setShouldStopAtCheckpoint(boolean shouldStopAtCheckpoint) {
this.shouldStopAtCheckpoint = shouldStopAtCheckpoint;
}
void shutdown() {
taskListener.shutdown();
}
void markAsFailed(String failureMessage) {
taskListener
.fail(
failureMessage,
ActionListener
.wrap(
r -> {
// Successfully marked as failed, reset counter so that task can be restarted
failureCount.set(0);
},
e -> {}
)
);
}
}

View File

@ -8,6 +8,9 @@ package org.elasticsearch.xpack.transform.transforms;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
@ -18,6 +21,7 @@ import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.query.BoolQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.Aggregations;
@ -29,20 +33,24 @@ import org.elasticsearch.xpack.core.indexing.IndexerState;
import org.elasticsearch.xpack.core.indexing.IterationResult;
import org.elasticsearch.xpack.core.transform.TransformField;
import org.elasticsearch.xpack.core.transform.TransformMessages;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerPosition;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats;
import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpoint;
import org.elasticsearch.xpack.core.transform.transforms.TransformConfig;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerPosition;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats;
import org.elasticsearch.xpack.core.transform.transforms.TransformProgress;
import org.elasticsearch.xpack.core.transform.transforms.TransformTaskState;
import org.elasticsearch.xpack.core.transform.utils.ExceptionsHelper;
import org.elasticsearch.xpack.transform.checkpoint.CheckpointProvider;
import org.elasticsearch.xpack.transform.notifications.TransformAuditor;
import org.elasticsearch.xpack.transform.persistence.TransformConfigManager;
import org.elasticsearch.xpack.transform.transforms.pivot.AggregationResultUtils;
import org.elasticsearch.xpack.transform.transforms.pivot.Pivot;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.time.Instant;
import java.util.Collections;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicReference;
@ -73,46 +81,68 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
public static final String COMPOSITE_AGGREGATION_NAME = "_transform";
private static final Logger logger = LogManager.getLogger(TransformIndexer.class);
protected final TransformConfigManager transformsConfigManager;
private final CheckpointProvider checkpointProvider;
private final TransformProgressGatherer progressGatherer;
protected final TransformAuditor auditor;
protected final TransformContext context;
protected volatile TransformConfig transformConfig;
protected volatile TransformProgress progress;
private volatile TransformProgress progress;
protected volatile boolean auditBulkFailures = true;
// Indicates that the source has changed for the current run
protected volatile boolean hasSourceChanged = true;
private final Map<String, String> fieldMappings;
private Pivot pivot;
private int pageSize = 0;
protected volatile TransformCheckpoint lastCheckpoint;
protected volatile TransformCheckpoint nextCheckpoint;
private long logEvery = 1;
private long logCount = 0;
private volatile TransformCheckpoint lastCheckpoint;
private volatile TransformCheckpoint nextCheckpoint;
// Keeps track of the last exception that was written to our audit, keeps us from spamming the audit index
private volatile String lastAuditedExceptionMessage = null;
private volatile RunState runState;
// hold information for continuous mode (partial updates)
private volatile Map<String, Set<String>> changedBuckets;
private volatile Map<String, Object> changedBucketsAfterKey;
public TransformIndexer(Executor executor,
TransformAuditor auditor,
TransformConfig transformConfig,
Map<String, String> fieldMappings,
AtomicReference<IndexerState> initialState,
TransformIndexerPosition initialPosition,
TransformIndexerStats jobStats,
TransformProgress transformProgress,
TransformCheckpoint lastCheckpoint,
TransformCheckpoint nextCheckpoint) {
public TransformIndexer(
Executor executor,
TransformConfigManager transformsConfigManager,
CheckpointProvider checkpointProvider,
TransformProgressGatherer progressGatherer,
TransformAuditor auditor,
TransformConfig transformConfig,
Map<String, String> fieldMappings,
AtomicReference<IndexerState> initialState,
TransformIndexerPosition initialPosition,
TransformIndexerStats jobStats,
TransformProgress transformProgress,
TransformCheckpoint lastCheckpoint,
TransformCheckpoint nextCheckpoint,
TransformContext context
) {
super(executor, initialState, initialPosition, jobStats);
this.auditor = Objects.requireNonNull(auditor);
this.transformsConfigManager = ExceptionsHelper.requireNonNull(transformsConfigManager, "transformsConfigManager");
this.checkpointProvider = ExceptionsHelper.requireNonNull(checkpointProvider, "checkpointProvider");
this.progressGatherer = ExceptionsHelper.requireNonNull(progressGatherer, "progressGatherer");
this.auditor = ExceptionsHelper.requireNonNull(auditor, "auditor");
this.transformConfig = ExceptionsHelper.requireNonNull(transformConfig, "transformConfig");
this.fieldMappings = ExceptionsHelper.requireNonNull(fieldMappings, "fieldMappings");
this.progress = transformProgress;
this.lastCheckpoint = lastCheckpoint;
this.nextCheckpoint = nextCheckpoint;
this.lastCheckpoint = ExceptionsHelper.requireNonNull(lastCheckpoint, "lastCheckpoint");
this.nextCheckpoint = ExceptionsHelper.requireNonNull(nextCheckpoint, "nextCheckpoint");
this.context = ExceptionsHelper.requireNonNull(context, "context");
// give runState a default
this.runState = RunState.FULL_RUN;
}
protected abstract void failIndexer(String message);
public int getPageSize() {
return pageSize;
}
@ -146,25 +176,153 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
return nextCheckpoint;
}
public CheckpointProvider getCheckpointProvider() {
return checkpointProvider;
}
/**
* Request a checkpoint
*/
protected abstract void createCheckpoint(ActionListener<TransformCheckpoint> listener);
protected void createCheckpoint(ActionListener<TransformCheckpoint> listener) {
checkpointProvider
.createNextCheckpoint(
getLastCheckpoint(),
ActionListener
.wrap(
checkpoint -> transformsConfigManager
.putTransformCheckpoint(
checkpoint,
ActionListener.wrap(putCheckPointResponse -> listener.onResponse(checkpoint), createCheckpointException -> {
logger
.warn(
new ParameterizedMessage("[{}] failed to create checkpoint.", getJobId()),
createCheckpointException
);
listener
.onFailure(
new RuntimeException(
"Failed to create checkpoint due to " + createCheckpointException.getMessage(),
createCheckpointException
)
);
})
),
getCheckPointException -> {
logger
.warn(new ParameterizedMessage("[{}] failed to retrieve checkpoint.", getJobId()), getCheckPointException);
listener
.onFailure(
new RuntimeException(
"Failed to retrieve checkpoint due to " + getCheckPointException.getMessage(),
getCheckPointException
)
);
}
)
);
}
@Override
protected void onStart(long now, ActionListener<Boolean> listener) {
try {
pivot = new Pivot(getConfig().getPivotConfig());
if (context.getTaskState() == TransformTaskState.FAILED) {
logger.debug("[{}] attempted to start while failed.", getJobId());
listener.onFailure(new ElasticsearchException("Attempted to start a failed transform [{}].", getJobId()));
return;
}
// if we haven't set the page size yet, if it is set we might have reduced it after running into an out of memory
if (pageSize == 0) {
pageSize = pivot.getInitialPageSize();
ActionListener<Void> finalListener = ActionListener.wrap(r -> {
try {
pivot = new Pivot(getConfig().getPivotConfig());
// if we haven't set the page size yet, if it is set we might have reduced it after running into an out of memory
if (pageSize == 0) {
pageSize = pivot.getInitialPageSize();
}
runState = determineRunStateAtStart();
listener.onResponse(true);
} catch (Exception e) {
listener.onFailure(e);
return;
}
}, listener::onFailure);
runState = determineRunStateAtStart();
listener.onResponse(true);
} catch (Exception e) {
listener.onFailure(e);
// On each run, we need to get the total number of docs and reset the count of processed docs
// Since multiple checkpoints can be executed in the task while it is running on the same node, we need to gather
// the progress here, and not in the executor.
ActionListener<Void> updateConfigListener = ActionListener.wrap(updateConfigResponse -> {
if (initialRun()) {
createCheckpoint(ActionListener.wrap(cp -> {
nextCheckpoint = cp;
// If nextCheckpoint > 1, this means that we are now on the checkpoint AFTER the batch checkpoint
// Consequently, the idea of percent complete no longer makes sense.
if (nextCheckpoint.getCheckpoint() > 1) {
progress = new TransformProgress(null, 0L, 0L);
finalListener.onResponse(null);
return;
}
progressGatherer.getInitialProgress(buildFilterQuery(), getConfig(), ActionListener.wrap(newProgress -> {
logger.trace("[{}] reset the progress from [{}] to [{}].", getJobId(), progress, newProgress);
progress = newProgress;
finalListener.onResponse(null);
}, failure -> {
progress = null;
logger.warn(new ParameterizedMessage("[{}] unable to load progress information for task.", getJobId()), failure);
finalListener.onResponse(null);
}));
}, listener::onFailure));
} else {
finalListener.onResponse(null);
}
}, listener::onFailure);
// If we are continuous, we will want to verify we have the latest stored configuration
ActionListener<Void> changedSourceListener = ActionListener.wrap(r -> {
if (isContinuous()) {
transformsConfigManager.getTransformConfiguration(getJobId(), ActionListener.wrap(config -> {
transformConfig = config;
logger.debug("[{}] successfully refreshed transform config from index.", getJobId());
updateConfigListener.onResponse(null);
}, failure -> {
String msg = TransformMessages.getMessage(TransformMessages.FAILED_TO_RELOAD_TRANSFORM_CONFIGURATION, getJobId());
logger.error(msg, failure);
// If the transform config index or the transform config is gone, something serious occurred
// We are in an unknown state and should fail out
if (failure instanceof ResourceNotFoundException) {
updateConfigListener.onFailure(new TransformConfigReloadingException(msg, failure));
} else {
auditor.warning(getJobId(), msg);
updateConfigListener.onResponse(null);
}
}));
} else {
updateConfigListener.onResponse(null);
}
}, listener::onFailure);
// If we are not on the initial batch checkpoint and its the first pass of whatever continuous checkpoint we are on,
// we should verify if there are local changes based on the sync config. If not, do not proceed further and exit.
if (context.getCheckpoint() > 0 && initialRun()) {
sourceHasChanged(ActionListener.wrap(hasChanged -> {
hasSourceChanged = hasChanged;
if (hasChanged) {
context.setChangesLastDetectedAt(Instant.now());
logger.debug("[{}] source has changed, triggering new indexer run.", getJobId());
changedSourceListener.onResponse(null);
} else {
logger.trace("[{}] source has not changed, finish indexer early.", getJobId());
// No changes, stop executing
listener.onResponse(false);
}
}, failure -> {
// If we failed determining if the source changed, it's safer to assume there were changes.
// We should allow the failure path to complete as normal
hasSourceChanged = true;
listener.onFailure(failure);
}));
} else {
hasSourceChanged = true;
changedSourceListener.onResponse(null);
}
}
@ -174,10 +332,63 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
@Override
protected void onFinish(ActionListener<Void> listener) {
// reset the page size, so we do not memorize a low page size forever
pageSize = pivot.getInitialPageSize();
// reset the changed bucket to free memory
changedBuckets = null;
try {
// This indicates an early exit since no changes were found.
// So, don't treat this like a checkpoint being completed, as no work was done.
if (hasSourceChanged == false) {
if (context.shouldStopAtCheckpoint()) {
stop();
}
listener.onResponse(null);
return;
}
// reset the page size, so we do not memorize a low page size forever
pageSize = pivot.getInitialPageSize();
// reset the changed bucket to free memory
changedBuckets = null;
long checkpoint = context.getAndIncrementCheckpoint();
lastCheckpoint = getNextCheckpoint();
nextCheckpoint = null;
// Reset our failure count as we have finished and may start again with a new checkpoint
context.resetReasonAndFailureCounter();
// 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
// show less than 100% even though we are done.
// NOTE: this method is called in the same thread as the processing thread.
// Theoretically, there should not be a race condition with updating progress here.
// NOTE 2: getPercentComplete should only NOT be null on the first (batch) checkpoint
if (progress != null && progress.getPercentComplete() != null && progress.getPercentComplete() < 100.0) {
progress.incrementDocsProcessed(progress.getTotalDocs() - progress.getDocumentsProcessed());
}
// If the last checkpoint is now greater than 1, that means that we have just processed the first
// continuous checkpoint and should start recording the exponential averages
if (lastCheckpoint != null && lastCheckpoint.getCheckpoint() > 1) {
long docsIndexed = 0;
long docsProcessed = 0;
// This should not happen as we simply create a new one when we reach continuous checkpoints
// but this is a paranoid `null` check
if (progress != null) {
docsIndexed = progress.getDocumentsIndexed();
docsProcessed = progress.getDocumentsProcessed();
}
long durationMs = System.currentTimeMillis() - lastCheckpoint.getTimestamp();
getStats().incrementCheckpointExponentialAverages(durationMs < 0 ? 0 : durationMs, docsIndexed, docsProcessed);
}
if (shouldAuditOnFinish(checkpoint)) {
auditor.info(getJobId(), "Finished indexing for transform checkpoint [" + checkpoint + "].");
}
logger.debug("[{}] finished indexing for transform checkpoint [{}].", getJobId(), checkpoint);
auditBulkFailures = true;
if (context.shouldStopAtCheckpoint()) {
stop();
}
listener.onResponse(null);
} catch (Exception e) {
listener.onFailure(e);
}
}
@Override
@ -186,32 +397,125 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
// Treat this as a "we reached the end".
// This should only happen when all underlying indices have gone away. Consequently, there is no more data to read.
if (aggregations == null) {
logger.info("[" + getJobId() + "] unexpected null aggregations in search response. " +
"Source indices have been deleted or closed.");
auditor.info(getJobId(),
"Source indices have been deleted or closed. " +
"Please verify that these indices exist and are open [" +
Strings.arrayToCommaDelimitedString(getConfig().getSource().getIndex()) +
"].");
logger
.info("[{}] unexpected null aggregations in search response. " + "Source indices have been deleted or closed.", getJobId());
auditor
.info(
getJobId(),
"Source indices have been deleted or closed. "
+ "Please verify that these indices exist and are open ["
+ Strings.arrayToCommaDelimitedString(getConfig().getSource().getIndex())
+ "]."
);
return new IterationResult<>(Collections.emptyList(), null, true);
}
final CompositeAggregation agg = aggregations.get(COMPOSITE_AGGREGATION_NAME);
switch (runState) {
case FULL_RUN:
return processBuckets(agg);
case PARTIAL_RUN_APPLY_CHANGES:
return processPartialBucketUpdates(agg);
case PARTIAL_RUN_IDENTIFY_CHANGES:
return processChangedBuckets(agg);
case FULL_RUN:
return processBuckets(agg);
case PARTIAL_RUN_APPLY_CHANGES:
return processPartialBucketUpdates(agg);
case PARTIAL_RUN_IDENTIFY_CHANGES:
return processChangedBuckets(agg);
default:
// Any other state is a bug, should not happen
logger.warn("Encountered unexpected run state [" + runState + "]");
throw new IllegalStateException("DataFrame indexer job encountered an illegal state [" + runState + "]");
default:
// Any other state is a bug, should not happen
logger.warn("[{}] Encountered unexpected run state [{}]", getJobId(), runState);
throw new IllegalStateException("DataFrame indexer job encountered an illegal state [" + runState + "]");
}
}
@Override
public synchronized boolean maybeTriggerAsyncJob(long now) {
if (context.getTaskState() == TransformTaskState.FAILED) {
logger.debug("[{}] schedule was triggered for transform but task is failed. Ignoring trigger.", getJobId());
return false;
}
// ignore trigger if indexer is running, prevents log spam in A2P indexer
IndexerState indexerState = getState();
if (IndexerState.INDEXING.equals(indexerState) || IndexerState.STOPPING.equals(indexerState)) {
logger.debug("[{}] indexer for transform has state [{}]. Ignoring trigger.", getJobId(), indexerState);
return false;
}
return super.maybeTriggerAsyncJob(now);
}
@Override
protected void onFailure(Exception exc) {
// the failure handler must not throw an exception due to internal problems
try {
handleFailure(exc);
} catch (Exception e) {
logger.error(new ParameterizedMessage("[{}] transform encountered an unexpected internal exception: ", getJobId()), e);
}
}
@Override
protected void onStop() {
auditor.info(transformConfig.getId(), "Transform has stopped.");
logger.info("[{}] transform has stopped.", transformConfig.getId());
}
@Override
protected void onAbort() {
auditor.info(transformConfig.getId(), "Received abort request, stopping transform.");
logger.info("[{}] transform received abort request. Stopping indexer.", transformConfig.getId());
context.shutdown();
}
synchronized void handleFailure(Exception e) {
logger.warn(new ParameterizedMessage("[{}] transform encountered an exception: ", getJobId()), e);
if (handleCircuitBreakingException(e)) {
return;
}
if (isIrrecoverableFailure(e) || context.getAndIncrementFailureCount() > context.getNumFailureRetries()) {
String failureMessage = isIrrecoverableFailure(e)
? "task encountered irrecoverable failure: " + e.getMessage()
: "task encountered more than " + context.getNumFailureRetries() + " failures; latest failure: " + e.getMessage();
failIndexer(failureMessage);
} else {
// Since our schedule fires again very quickly after failures it is possible to run into the same failure numerous
// times in a row, very quickly. We do not want to spam the audit log with repeated failures, so only record the first one
if (e.getMessage().equals(lastAuditedExceptionMessage) == false) {
auditor
.warning(
getJobId(),
"Transform encountered an exception: " + e.getMessage() + " Will attempt again at next scheduled trigger."
);
lastAuditedExceptionMessage = e.getMessage();
}
}
}
private void sourceHasChanged(ActionListener<Boolean> hasChangedListener) {
checkpointProvider.sourceHasChanged(getLastCheckpoint(), ActionListener.wrap(hasChanged -> {
logger.trace("[{}] change detected [{}].", getJobId(), hasChanged);
hasChangedListener.onResponse(hasChanged);
}, e -> {
logger
.warn(
new ParameterizedMessage("[{}] failed to detect changes for transform. Skipping update till next check.", getJobId()),
e
);
auditor
.warning(
getJobId(),
"Failed to detect changes for transform, skipping update till next check. Exception: " + e.getMessage()
);
hasChangedListener.onResponse(false);
}));
}
private boolean isIrrecoverableFailure(Exception e) {
return e instanceof IndexNotFoundException
|| e instanceof AggregationResultUtils.AggregationExtractionException
|| e instanceof TransformConfigReloadingException;
}
private IterationResult<TransformIndexerPosition> processBuckets(final CompositeAggregation agg) {
// we reached the end
if (agg.getBuckets().isEmpty()) {
@ -221,13 +525,16 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
long docsBeforeProcess = getStats().getNumDocuments();
TransformIndexerPosition oldPosition = getPosition();
TransformIndexerPosition newPosition = new TransformIndexerPosition(agg.afterKey(),
oldPosition != null ? getPosition().getBucketsPosition() : null);
TransformIndexerPosition newPosition = new TransformIndexerPosition(
agg.afterKey(),
oldPosition != null ? getPosition().getBucketsPosition() : null
);
IterationResult<TransformIndexerPosition> result = new IterationResult<>(
processBucketsToIndexRequests(agg).collect(Collectors.toList()),
newPosition,
agg.getBuckets().isEmpty());
processBucketsToIndexRequests(agg).collect(Collectors.toList()),
newPosition,
agg.getBuckets().isEmpty()
);
// NOTE: progress is also mutated in ClientDataFrameIndexer#onFinished
if (progress != null) {
@ -247,14 +554,12 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
// reset the runState to fetch changed buckets
runState = RunState.PARTIAL_RUN_IDENTIFY_CHANGES;
// advance the cursor for changed bucket detection
return new IterationResult<>(Collections.emptyList(),
new TransformIndexerPosition(null, changedBucketsAfterKey), false);
return new IterationResult<>(Collections.emptyList(), new TransformIndexerPosition(null, changedBucketsAfterKey), false);
}
return processBuckets(agg);
}
private IterationResult<TransformIndexerPosition> processChangedBuckets(final CompositeAggregation agg) {
// initialize the map of changed buckets, the map might be empty if source do not require/implement
// changed bucket detection
@ -270,11 +575,7 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
// else
// collect all buckets that require the update
agg.getBuckets().stream().forEach(bucket -> {
bucket.getKey().forEach((k, v) -> {
changedBuckets.get(k).add(v.toString());
});
});
agg.getBuckets().stream().forEach(bucket -> { bucket.getKey().forEach((k, v) -> { changedBuckets.get(k).add(v.toString()); }); });
// remember the after key but do not store it in the state yet (in the failure we need to retrieve it again)
changedBucketsAfterKey = agg.afterKey();
@ -335,8 +636,7 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
TransformConfig config = getConfig();
if (this.isContinuous()) {
BoolQueryBuilder filteredQuery = new BoolQueryBuilder()
.filter(pivotQueryBuilder);
BoolQueryBuilder filteredQuery = new BoolQueryBuilder().filter(pivotQueryBuilder);
if (lastCheckpoint != null) {
filteredQuery.filter(config.getSyncConfig().getRangeQuery(lastCheckpoint, nextCheckpoint));
@ -354,25 +654,24 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
assert nextCheckpoint != null;
SearchRequest searchRequest = new SearchRequest(getConfig().getSource().getIndex())
.allowPartialSearchResults(false)
.indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN);
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
.size(0);
.allowPartialSearchResults(false)
.indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN);
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder().size(0);
switch (runState) {
case FULL_RUN:
buildFullRunQuery(sourceBuilder);
break;
case PARTIAL_RUN_IDENTIFY_CHANGES:
buildChangedBucketsQuery(sourceBuilder);
break;
case PARTIAL_RUN_APPLY_CHANGES:
buildPartialUpdateQuery(sourceBuilder);
break;
default:
// Any other state is a bug, should not happen
logger.warn("Encountered unexpected run state [" + runState + "]");
throw new IllegalStateException("DataFrame indexer job encountered an illegal state [" + runState + "]");
case FULL_RUN:
buildFullRunQuery(sourceBuilder);
break;
case PARTIAL_RUN_IDENTIFY_CHANGES:
buildChangedBucketsQuery(sourceBuilder);
break;
case PARTIAL_RUN_APPLY_CHANGES:
buildPartialUpdateQuery(sourceBuilder);
break;
default:
// Any other state is a bug, should not happen
logger.warn("Encountered unexpected run state [" + runState + "]");
throw new IllegalStateException("DataFrame indexer job encountered an illegal state [" + runState + "]");
}
searchRequest.source(sourceBuilder);
@ -388,9 +687,8 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
QueryBuilder pivotQueryBuilder = config.getSource().getQueryConfig().getQuery();
if (isContinuous()) {
BoolQueryBuilder filteredQuery = new BoolQueryBuilder()
.filter(pivotQueryBuilder)
.filter(config.getSyncConfig()
.getRangeQuery(nextCheckpoint));
.filter(pivotQueryBuilder)
.filter(config.getSyncConfig().getRangeQuery(nextCheckpoint));
sourceBuilder.query(filteredQuery);
} else {
sourceBuilder.query(pivotQueryBuilder);
@ -413,9 +711,9 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
QueryBuilder pivotQueryBuilder = getConfig().getSource().getQueryConfig().getQuery();
TransformConfig config = getConfig();
BoolQueryBuilder filteredQuery = new BoolQueryBuilder().
filter(pivotQueryBuilder).
filter(config.getSyncConfig().getRangeQuery(lastCheckpoint, nextCheckpoint));
BoolQueryBuilder filteredQuery = new BoolQueryBuilder()
.filter(pivotQueryBuilder)
.filter(config.getSyncConfig().getRangeQuery(lastCheckpoint, nextCheckpoint));
sourceBuilder.query(filteredQuery);
@ -434,9 +732,8 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
QueryBuilder pivotQueryBuilder = config.getSource().getQueryConfig().getQuery();
BoolQueryBuilder filteredQuery = new BoolQueryBuilder()
.filter(pivotQueryBuilder)
.filter(config.getSyncConfig()
.getRangeQuery(nextCheckpoint));
.filter(pivotQueryBuilder)
.filter(config.getSyncConfig().getRangeQuery(nextCheckpoint));
if (changedBuckets != null && changedBuckets.isEmpty() == false) {
QueryBuilder pivotFilter = pivot.filterBuckets(changedBuckets);
@ -469,8 +766,11 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
return false;
}
double reducingFactor = Math.min((double) circuitBreakingException.getByteLimit() / circuitBreakingException.getBytesWanted(),
1 - (Math.log10(pageSize) * 0.1));
double reducingFactor = Math
.min(
(double) circuitBreakingException.getByteLimit() / circuitBreakingException.getBytesWanted(),
1 - (Math.log10(pageSize) * 0.1)
);
int newPageSize = (int) Math.round(reducingFactor * pageSize);
@ -480,8 +780,7 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
return true;
}
String message = TransformMessages.getMessage(TransformMessages.LOG_TRANSFORM_PIVOT_REDUCE_PAGE_SIZE, pageSize,
newPageSize);
String message = TransformMessages.getMessage(TransformMessages.LOG_TRANSFORM_PIVOT_REDUCE_PAGE_SIZE, pageSize, newPageSize);
auditor.info(getJobId(), message);
logger.info("Data frame transform [" + getJobId() + "]:" + message);
@ -489,6 +788,35 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
return true;
}
protected void failIndexer(String failureMessage) {
logger.error("[{}] transform has failed; experienced: [{}].", getJobId(), failureMessage);
auditor.error(getJobId(), failureMessage);
context.markAsFailed(failureMessage);
}
/**
* Indicates if an audit message should be written when onFinish is called for the given checkpoint
* We audit the first checkpoint, and then every 10 checkpoints until completedCheckpoint == 99
* Then we audit every 100, until completedCheckpoint == 999
*
* Then we always audit every 1_000 checkpoints
*
* @param completedCheckpoint The checkpoint that was just completed
* @return {@code true} if an audit message should be written
*/
protected boolean shouldAuditOnFinish(long completedCheckpoint) {
if (++logCount % logEvery != 0) {
return false;
}
if (completedCheckpoint == 0) {
return true;
}
int log10Checkpoint = (int) Math.floor(Math.log10(completedCheckpoint));
logEvery = log10Checkpoint >= 3 ? 1_000 : (int) Math.pow(10.0, log10Checkpoint);
logCount = 0;
return true;
}
private RunState determineRunStateAtStart() {
// either 1st run or not a continuous data frame
if (nextCheckpoint.getCheckpoint() == 1 || isContinuous() == false) {
@ -530,5 +858,9 @@ public abstract class TransformIndexer extends AsyncTwoPhaseIndexer<TransformInd
return null;
}
protected abstract void sourceHasChanged(ActionListener<Boolean> hasChangedListener);
static class TransformConfigReloadingException extends ElasticsearchException {
TransformConfigReloadingException(String msg, Throwable cause, Object... args) {
super(msg, cause, args);
}
}
}

View File

@ -69,14 +69,16 @@ public class TransformPersistentTasksExecutor extends PersistentTasksExecutor<Tr
private final TransformAuditor auditor;
private volatile int numFailureRetries;
public TransformPersistentTasksExecutor(Client client,
TransformConfigManager transformsConfigManager,
TransformCheckpointService transformsCheckpointService,
SchedulerEngine schedulerEngine,
TransformAuditor auditor,
ThreadPool threadPool,
ClusterService clusterService,
Settings settings) {
public TransformPersistentTasksExecutor(
Client client,
TransformConfigManager transformsConfigManager,
TransformCheckpointService transformsCheckpointService,
SchedulerEngine schedulerEngine,
TransformAuditor auditor,
ThreadPool threadPool,
ClusterService clusterService,
Settings settings
) {
super(TransformField.TASK_NAME, Transform.TASK_THREAD_POOL_NAME);
this.client = client;
this.transformsConfigManager = transformsConfigManager;
@ -85,43 +87,50 @@ public class TransformPersistentTasksExecutor extends PersistentTasksExecutor<Tr
this.auditor = auditor;
this.threadPool = threadPool;
this.clusterService = clusterService;
this.numFailureRetries = TransformTask.NUM_FAILURE_RETRIES_SETTING.get(settings);
clusterService.getClusterSettings()
.addSettingsUpdateConsumer(TransformTask.NUM_FAILURE_RETRIES_SETTING, this::setNumFailureRetries);
this.numFailureRetries = Transform.NUM_FAILURE_RETRIES_SETTING.get(settings);
clusterService.getClusterSettings().addSettingsUpdateConsumer(Transform.NUM_FAILURE_RETRIES_SETTING, this::setNumFailureRetries);
}
@Override
public PersistentTasksCustomMetaData.Assignment getAssignment(TransformTaskParams params, ClusterState clusterState) {
List<String> unavailableIndices = verifyIndicesPrimaryShardsAreActive(clusterState);
if (unavailableIndices.size() != 0) {
String reason = "Not starting transform [" + params.getId() + "], " +
"because not all primary shards are active for the following indices [" +
String.join(",", unavailableIndices) + "]";
String reason = "Not starting transform ["
+ params.getId()
+ "], "
+ "because not all primary shards are active for the following indices ["
+ String.join(",", unavailableIndices)
+ "]";
logger.debug(reason);
return new PersistentTasksCustomMetaData.Assignment(null, reason);
}
// see gh#48019 disable assignment if any node is using 7.2 or 7.3
if (clusterState.getNodes().getMinNodeVersion().before(Version.V_7_4_0)) {
String reason = "Not starting transform [" + params.getId() + "], " +
"because cluster contains nodes with version older than 7.4.0";
String reason = "Not starting transform ["
+ params.getId()
+ "], "
+ "because cluster contains nodes with version older than 7.4.0";
logger.debug(reason);
return new PersistentTasksCustomMetaData.Assignment(null, reason);
}
DiscoveryNode discoveryNode = selectLeastLoadedNode(clusterState, (node) ->
node.isDataNode() &&
node.getVersion().onOrAfter(params.getVersion())
DiscoveryNode discoveryNode = selectLeastLoadedNode(
clusterState,
(node) -> node.isDataNode() && node.getVersion().onOrAfter(params.getVersion())
);
return discoveryNode == null ? NO_NODE_FOUND : new PersistentTasksCustomMetaData.Assignment(discoveryNode.getId(), "");
}
static List<String> verifyIndicesPrimaryShardsAreActive(ClusterState clusterState) {
IndexNameExpressionResolver resolver = new IndexNameExpressionResolver();
String[] indices = resolver.concreteIndexNames(clusterState,
IndicesOptions.lenientExpandOpen(),
TransformInternalIndexConstants.INDEX_NAME_PATTERN,
TransformInternalIndexConstants.INDEX_NAME_PATTERN_DEPRECATED);
String[] indices = resolver
.concreteIndexNames(
clusterState,
IndicesOptions.lenientExpandOpen(),
TransformInternalIndexConstants.INDEX_NAME_PATTERN,
TransformInternalIndexConstants.INDEX_NAME_PATTERN_DEPRECATED
);
List<String> unavailableIndices = new ArrayList<>(indices.length);
for (String index : indices) {
IndexRoutingTable routingTable = clusterState.getRoutingTable().index(index);
@ -145,88 +154,84 @@ public class TransformPersistentTasksExecutor extends PersistentTasksExecutor<Tr
// We want the rest of the state to be populated in the task when it is loaded on the node so that users can force start it again
// later if they want.
final ClientTransformIndexerBuilder indexerBuilder =
new ClientTransformIndexerBuilder()
.setAuditor(auditor)
.setClient(client)
.setTransformsCheckpointService(transformCheckpointService)
.setTransformsConfigManager(transformsConfigManager);
final ClientTransformIndexerBuilder indexerBuilder = new ClientTransformIndexerBuilder()
.setAuditor(auditor)
.setClient(client)
.setTransformsCheckpointService(transformCheckpointService)
.setTransformsConfigManager(transformsConfigManager);
final SetOnce<TransformState> stateHolder = new SetOnce<>();
ActionListener<StartTransformAction.Response> startTaskListener = ActionListener.wrap(
response -> logger.info("[{}] successfully completed and scheduled task in node operation", transformId),
failure -> {
auditor.error(transformId, "Failed to start transform. " +
"Please stop and attempt to start again. Failure: " + failure.getMessage());
logger.error("Failed to start task ["+ transformId +"] in node operation", failure);
}
);
ActionListener<StartTransformAction.Response> startTaskListener = ActionListener
.wrap(response -> logger.info("[{}] successfully completed and scheduled task in node operation", transformId), failure -> {
auditor
.error(
transformId,
"Failed to start transform. " + "Please stop and attempt to start again. Failure: " + failure.getMessage()
);
logger.error("Failed to start task [" + transformId + "] in node operation", failure);
});
// <7> load next checkpoint
ActionListener<TransformCheckpoint> getTransformNextCheckpointListener = ActionListener.wrap(
nextCheckpoint -> {
ActionListener<TransformCheckpoint> getTransformNextCheckpointListener = ActionListener.wrap(nextCheckpoint -> {
if (nextCheckpoint.isEmpty()) {
// extra safety: reset position and progress if next checkpoint is empty
// prevents a failure if for some reason the next checkpoint has been deleted
indexerBuilder.setInitialPosition(null);
indexerBuilder.setProgress(null);
} else {
logger.trace("[{}] Loaded next checkpoint [{}] found, starting the task", transformId,
nextCheckpoint.getCheckpoint());
indexerBuilder.setNextCheckpoint(nextCheckpoint);
}
if (nextCheckpoint.isEmpty()) {
// extra safety: reset position and progress if next checkpoint is empty
// prevents a failure if for some reason the next checkpoint has been deleted
indexerBuilder.setInitialPosition(null);
indexerBuilder.setProgress(null);
} else {
logger.trace("[{}] Loaded next checkpoint [{}] found, starting the task", transformId, nextCheckpoint.getCheckpoint());
indexerBuilder.setNextCheckpoint(nextCheckpoint);
}
final long lastCheckpoint = stateHolder.get().getCheckpoint();
final long lastCheckpoint = stateHolder.get().getCheckpoint();
startTask(buildTask, indexerBuilder, lastCheckpoint, startTaskListener);
},
error -> {
// TODO: do not use the same error message as for loading the last checkpoint
String msg = TransformMessages.getMessage(TransformMessages.FAILED_TO_LOAD_TRANSFORM_CHECKPOINT, transformId);
logger.error(msg, error);
markAsFailed(buildTask, msg);
}
);
startTask(buildTask, indexerBuilder, lastCheckpoint, startTaskListener);
}, error -> {
// TODO: do not use the same error message as for loading the last checkpoint
String msg = TransformMessages.getMessage(TransformMessages.FAILED_TO_LOAD_TRANSFORM_CHECKPOINT, transformId);
logger.error(msg, error);
markAsFailed(buildTask, msg);
});
// <6> load last checkpoint
ActionListener<TransformCheckpoint> getTransformLastCheckpointListener = ActionListener.wrap(
lastCheckpoint -> {
indexerBuilder.setLastCheckpoint(lastCheckpoint);
ActionListener<TransformCheckpoint> getTransformLastCheckpointListener = ActionListener.wrap(lastCheckpoint -> {
indexerBuilder.setLastCheckpoint(lastCheckpoint);
logger.trace("[{}] Loaded last checkpoint [{}], looking for next checkpoint", transformId,
lastCheckpoint.getCheckpoint());
transformsConfigManager.getTransformCheckpoint(transformId, lastCheckpoint.getCheckpoint() + 1,
getTransformNextCheckpointListener);
},
error -> {
String msg = TransformMessages.getMessage(TransformMessages.FAILED_TO_LOAD_TRANSFORM_CHECKPOINT, transformId);
logger.error(msg, error);
markAsFailed(buildTask, msg);
}
);
logger.trace("[{}] Loaded last checkpoint [{}], looking for next checkpoint", transformId, lastCheckpoint.getCheckpoint());
transformsConfigManager
.getTransformCheckpoint(transformId, lastCheckpoint.getCheckpoint() + 1, getTransformNextCheckpointListener);
}, error -> {
String msg = TransformMessages.getMessage(TransformMessages.FAILED_TO_LOAD_TRANSFORM_CHECKPOINT, transformId);
logger.error(msg, error);
markAsFailed(buildTask, msg);
});
// <5> Set the previous stats (if they exist), initialize the indexer, start the task (If it is STOPPED)
// Since we don't create the task until `_start` is called, if we see that the task state is stopped, attempt to start
// Schedule execution regardless
ActionListener<Tuple<TransformStoredDoc, SeqNoPrimaryTermAndIndex>> transformStatsActionListener = ActionListener.wrap(
stateAndStatsAndSeqNoPrimaryTermAndIndex -> {
ActionListener<Tuple<TransformStoredDoc, SeqNoPrimaryTermAndIndex>> transformStatsActionListener = ActionListener
.wrap(stateAndStatsAndSeqNoPrimaryTermAndIndex -> {
TransformStoredDoc stateAndStats = stateAndStatsAndSeqNoPrimaryTermAndIndex.v1();
SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex = stateAndStatsAndSeqNoPrimaryTermAndIndex.v2();
// Since we have not set the value for this yet, it SHOULD be null
buildTask.updateSeqNoPrimaryTermAndIndex(null, seqNoPrimaryTermAndIndex);
logger.trace("[{}] initializing state and stats: [{}]", transformId, stateAndStats.toString());
TransformState transformState = stateAndStats.getTransformState();
indexerBuilder.setInitialStats(stateAndStats.getTransformStats())
indexerBuilder
.setInitialStats(stateAndStats.getTransformStats())
.setInitialPosition(stateAndStats.getTransformState().getPosition())
.setProgress(stateAndStats.getTransformState().getProgress())
.setIndexerState(currentIndexerState(transformState))
.setSeqNoPrimaryTermAndIndex(seqNoPrimaryTermAndIndex)
.setShouldStopAtCheckpoint(transformState.shouldStopAtNextCheckpoint());
logger.debug("[{}] Loading existing state: [{}], position [{}]",
transformId,
stateAndStats.getTransformState(),
stateAndStats.getTransformState().getPosition());
logger
.debug(
"[{}] Loading existing state: [{}], position [{}]",
transformId,
stateAndStats.getTransformState(),
stateAndStats.getTransformState().getPosition()
);
stateHolder.set(transformState);
final long lastCheckpoint = stateHolder.get().getCheckpoint();
@ -235,11 +240,10 @@ public class TransformPersistentTasksExecutor extends PersistentTasksExecutor<Tr
logger.trace("[{}] No last checkpoint found, looking for next checkpoint", transformId);
transformsConfigManager.getTransformCheckpoint(transformId, lastCheckpoint + 1, getTransformNextCheckpointListener);
} else {
logger.trace ("[{}] Restore last checkpoint: [{}]", transformId, lastCheckpoint);
logger.trace("[{}] Restore last checkpoint: [{}]", transformId, lastCheckpoint);
transformsConfigManager.getTransformCheckpoint(transformId, lastCheckpoint, getTransformLastCheckpointListener);
}
},
error -> {
}, error -> {
if (error instanceof ResourceNotFoundException == false) {
String msg = TransformMessages.getMessage(TransformMessages.FAILED_TO_LOAD_TRANSFORM_STATE, transformId);
logger.error(msg, error);
@ -248,50 +252,43 @@ public class TransformPersistentTasksExecutor extends PersistentTasksExecutor<Tr
logger.trace("[{}] No stats found (new transform), starting the task", transformId);
startTask(buildTask, indexerBuilder, null, startTaskListener);
}
}
);
});
// <4> set fieldmappings for the indexer, get the previous stats (if they exist)
ActionListener<Map<String, String>> getFieldMappingsListener = ActionListener.wrap(
fieldMappings -> {
indexerBuilder.setFieldMappings(fieldMappings);
transformsConfigManager.getTransformStoredDoc(transformId, transformStatsActionListener);
},
error -> {
String msg = TransformMessages.getMessage(TransformMessages.UNABLE_TO_GATHER_FIELD_MAPPINGS,
indexerBuilder.getTransformConfig().getDestination().getIndex());
logger.error(msg, error);
markAsFailed(buildTask, msg);
}
);
ActionListener<Map<String, String>> getFieldMappingsListener = ActionListener.wrap(fieldMappings -> {
indexerBuilder.setFieldMappings(fieldMappings);
transformsConfigManager.getTransformStoredDoc(transformId, transformStatsActionListener);
}, error -> {
String msg = TransformMessages
.getMessage(
TransformMessages.UNABLE_TO_GATHER_FIELD_MAPPINGS,
indexerBuilder.getTransformConfig().getDestination().getIndex()
);
logger.error(msg, error);
markAsFailed(buildTask, msg);
});
// <3> Validate the transform, assigning it to the indexer, and get the field mappings
ActionListener<TransformConfig> getTransformConfigListener = ActionListener.wrap(
config -> {
if (config.isValid()) {
indexerBuilder.setTransformConfig(config);
SchemaUtil.getDestinationFieldMappings(client, config.getDestination().getIndex(), getFieldMappingsListener);
} else {
markAsFailed(buildTask,
TransformMessages.getMessage(TransformMessages.TRANSFORM_CONFIGURATION_INVALID, transformId));
}
},
error -> {
String msg = TransformMessages.getMessage(TransformMessages.FAILED_TO_LOAD_TRANSFORM_CONFIGURATION, transformId);
logger.error(msg, error);
markAsFailed(buildTask, msg);
ActionListener<TransformConfig> getTransformConfigListener = ActionListener.wrap(config -> {
if (config.isValid()) {
indexerBuilder.setTransformConfig(config);
SchemaUtil.getDestinationFieldMappings(client, config.getDestination().getIndex(), getFieldMappingsListener);
} else {
markAsFailed(buildTask, TransformMessages.getMessage(TransformMessages.TRANSFORM_CONFIGURATION_INVALID, transformId));
}
);
}, error -> {
String msg = TransformMessages.getMessage(TransformMessages.FAILED_TO_LOAD_TRANSFORM_CONFIGURATION, transformId);
logger.error(msg, error);
markAsFailed(buildTask, msg);
});
// <2> Get the transform config
ActionListener<Void> templateCheckListener = ActionListener.wrap(
aVoid -> transformsConfigManager.getTransformConfiguration(transformId, getTransformConfigListener),
error -> {
ActionListener<Void> templateCheckListener = ActionListener
.wrap(aVoid -> transformsConfigManager.getTransformConfiguration(transformId, getTransformConfigListener), error -> {
String msg = "Failed to create internal index mappings";
logger.error(msg, error);
markAsFailed(buildTask, msg);
}
);
});
// <1> Check the internal index template is installed
TransformInternalIndex.installLatestVersionedIndexTemplateIfRequired(clusterService, client, templateCheckListener);
@ -301,14 +298,14 @@ public class TransformPersistentTasksExecutor extends PersistentTasksExecutor<Tr
if (previousState == null) {
return IndexerState.STOPPED;
}
switch(previousState.getIndexerState()){
switch (previousState.getIndexerState()) {
// If it is STARTED or INDEXING we want to make sure we revert to started
// Otherwise, the internal indexer will never get scheduled and execute
case STARTED:
case INDEXING:
return IndexerState.STARTED;
// If we are STOPPED, STOPPING, or ABORTING and just started executing on this node,
// then it is safe to say we should be STOPPED
// then it is safe to say we should be STOPPED
case STOPPED:
case STOPPING:
case ABORTING:
@ -320,10 +317,15 @@ public class TransformPersistentTasksExecutor extends PersistentTasksExecutor<Tr
private void markAsFailed(TransformTask task, String reason) {
CountDownLatch latch = new CountDownLatch(1);
task.markAsFailed(reason, new LatchedActionListener<>(ActionListener.wrap(
nil -> {},
failure -> logger.error("Failed to set task [" + task.getTransformId() +"] to failed", failure)
), latch));
task
.fail(
reason,
new LatchedActionListener<>(
ActionListener
.wrap(nil -> {}, failure -> logger.error("Failed to set task [" + task.getTransformId() + "] to failed", failure)),
latch
)
);
try {
latch.await(MARK_AS_FAILED_TIMEOUT_SEC, TimeUnit.SECONDS);
} catch (InterruptedException e) {
@ -331,10 +333,12 @@ public class TransformPersistentTasksExecutor extends PersistentTasksExecutor<Tr
}
}
private void startTask(TransformTask buildTask,
ClientTransformIndexerBuilder indexerBuilder,
Long previousCheckpoint,
ActionListener<StartTransformAction.Response> listener) {
private void startTask(
TransformTask buildTask,
ClientTransformIndexerBuilder indexerBuilder,
Long previousCheckpoint,
ActionListener<StartTransformAction.Response> listener
) {
buildTask.initializeIndexer(indexerBuilder);
// TransformTask#start will fail if the task state is FAILED
buildTask.setNumFailureRetries(numFailureRetries).start(previousCheckpoint, listener);
@ -345,9 +349,25 @@ public class TransformPersistentTasksExecutor extends PersistentTasksExecutor<Tr
}
@Override
protected AllocatedPersistentTask createTask(long id, String type, String action, TaskId parentTaskId,
PersistentTasksCustomMetaData.PersistentTask<TransformTaskParams> persistentTask, Map<String, String> headers) {
return new TransformTask(id, type, action, parentTaskId, persistentTask.getParams(),
(TransformState) persistentTask.getState(), schedulerEngine, auditor, threadPool, headers);
protected AllocatedPersistentTask createTask(
long id,
String type,
String action,
TaskId parentTaskId,
PersistentTasksCustomMetaData.PersistentTask<TransformTaskParams> persistentTask,
Map<String, String> headers
) {
return new TransformTask(
id,
type,
action,
parentTaskId,
persistentTask.getParams(),
(TransformState) persistentTask.getState(),
schedulerEngine,
auditor,
threadPool,
headers
);
}
}

View File

@ -26,53 +26,63 @@ import java.util.function.Function;
*/
public final class TransformProgressGatherer {
private Client client;
TransformProgressGatherer(Client client) {
this.client = client;
}
/**
* This gathers the total docs given the config and search
*
* @param client ES Client to make queries
* @param filterQuery The adapted filter that can optionally take into account checkpoint information
* @param config The transform config containing headers, source, pivot, etc. information
* @param progressListener The listener to notify when progress object has been created
*/
public static void getInitialProgress(Client client,
QueryBuilder filterQuery,
TransformConfig config,
ActionListener<TransformProgress> progressListener) {
public void getInitialProgress(QueryBuilder filterQuery, TransformConfig config, ActionListener<TransformProgress> progressListener) {
SearchRequest request = getSearchRequest(config, filterQuery);
ActionListener<SearchResponse> searchResponseActionListener = ActionListener.wrap(
searchResponse -> progressListener.onResponse(searchResponseToTransformProgressFunction().apply(searchResponse)),
progressListener::onFailure
);
ClientHelper.executeWithHeadersAsync(config.getHeaders(),
ClientHelper.TRANSFORM_ORIGIN,
client,
SearchAction.INSTANCE,
request,
searchResponseActionListener);
ActionListener<SearchResponse> searchResponseActionListener = ActionListener
.wrap(
searchResponse -> progressListener.onResponse(searchResponseToTransformProgressFunction().apply(searchResponse)),
progressListener::onFailure
);
ClientHelper
.executeWithHeadersAsync(
config.getHeaders(),
ClientHelper.TRANSFORM_ORIGIN,
client,
SearchAction.INSTANCE,
request,
searchResponseActionListener
);
}
public static SearchRequest getSearchRequest(TransformConfig config, QueryBuilder filteredQuery) {
SearchRequest request = new SearchRequest(config.getSource().getIndex());
request.allowPartialSearchResults(false);
BoolQueryBuilder existsClauses = QueryBuilders.boolQuery();
config.getPivotConfig()
config
.getPivotConfig()
.getGroupConfig()
.getGroups()
.values()
// TODO change once we allow missing_buckets
.forEach(src -> existsClauses.must(QueryBuilders.existsQuery(src.getField())));
request.source(new SearchSourceBuilder()
.size(0)
.trackTotalHits(true)
.query(QueryBuilders.boolQuery()
.filter(filteredQuery)
.filter(existsClauses)));
request
.source(
new SearchSourceBuilder()
.size(0)
.trackTotalHits(true)
.query(QueryBuilders.boolQuery().filter(filteredQuery).filter(existsClauses))
);
return request;
}
public static Function<SearchResponse, TransformProgress> searchResponseToTransformProgressFunction() {
return searchResponse -> new TransformProgress(searchResponse.getHits().getTotalHits().value, 0L, 0L);
return searchResponse -> searchResponse != null
? new TransformProgress(searchResponse.getHits().getTotalHits().value, 0L, 0L)
: null;
}
}

View File

@ -13,8 +13,6 @@ import org.apache.lucene.util.SetOnce;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.persistent.AllocatedPersistentTask;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData;
@ -27,41 +25,27 @@ import org.elasticsearch.xpack.core.scheduler.SchedulerEngine.Event;
import org.elasticsearch.xpack.core.transform.TransformField;
import org.elasticsearch.xpack.core.transform.TransformMessages;
import org.elasticsearch.xpack.core.transform.action.StartTransformAction;
import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpointingInfo;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerPosition;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats;
import org.elasticsearch.xpack.core.transform.transforms.TransformTaskParams;
import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpointingInfo;
import org.elasticsearch.xpack.core.transform.transforms.TransformState;
import org.elasticsearch.xpack.core.transform.transforms.TransformTaskParams;
import org.elasticsearch.xpack.core.transform.transforms.TransformTaskState;
import org.elasticsearch.xpack.transform.checkpoint.TransformCheckpointService;
import org.elasticsearch.xpack.transform.notifications.TransformAuditor;
import org.elasticsearch.xpack.transform.persistence.SeqNoPrimaryTermAndIndex;
import java.util.Arrays;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.xpack.core.transform.TransformMessages.CANNOT_START_FAILED_TRANSFORM;
import static org.elasticsearch.xpack.core.transform.TransformMessages.CANNOT_STOP_FAILED_TRANSFORM;
public class TransformTask extends AllocatedPersistentTask implements SchedulerEngine.Listener {
public class TransformTask extends AllocatedPersistentTask implements SchedulerEngine.Listener, TransformContext.Listener {
// Default interval the scheduler sends an event if the config does not specify a frequency
private static final long SCHEDULER_NEXT_MILLISECONDS = 60000;
private static final Logger logger = LogManager.getLogger(TransformTask.class);
private static final int DEFAULT_FAILURE_RETRIES = 10;
private volatile int numFailureRetries = DEFAULT_FAILURE_RETRIES;
// How many times the transform task can retry on an non-critical failure
public static final Setting<Integer> NUM_FAILURE_RETRIES_SETTING = Setting.intSetting(
"xpack.transform.num_transform_failure_retries",
DEFAULT_FAILURE_RETRIES,
0,
100,
Setting.Property.NodeScope,
Setting.Property.Dynamic);
private static final IndexerState[] RUNNING_STATES = new IndexerState[]{IndexerState.STARTED, IndexerState.INDEXING};
private static final IndexerState[] RUNNING_STATES = new IndexerState[] { IndexerState.STARTED, IndexerState.INDEXING };
public static final String SCHEDULE_NAME = TransformField.TASK_NAME + "/schedule";
private final TransformTaskParams transform;
@ -70,19 +54,21 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
private final TransformAuditor auditor;
private final TransformIndexerPosition initialPosition;
private final IndexerState initialIndexerState;
private final TransformContext context;
private final SetOnce<ClientTransformIndexer> indexer = new SetOnce<>();
private final AtomicReference<TransformTaskState> taskState;
private final AtomicReference<String> stateReason;
private final AtomicReference<SeqNoPrimaryTermAndIndex> seqNoPrimaryTermAndIndex = new AtomicReference<>(null);
// the checkpoint of this transform, 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;
public TransformTask(long id, String type, String action, TaskId parentTask, TransformTaskParams transform,
TransformState state, SchedulerEngine schedulerEngine, TransformAuditor auditor,
ThreadPool threadPool, Map<String, String> headers) {
public TransformTask(
long id,
String type,
String action,
TaskId parentTask,
TransformTaskParams transform,
TransformState state,
SchedulerEngine schedulerEngine,
TransformAuditor auditor,
ThreadPool threadPool,
Map<String, String> headers
) {
super(id, type, action, TransformField.PERSISTENT_TASK_DESCRIPTION_PREFIX + transform.getId(), parentTask, headers);
this.transform = transform;
this.schedulerEngine = schedulerEngine;
@ -91,8 +77,9 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
IndexerState initialState = IndexerState.STOPPED;
TransformTaskState initialTaskState = TransformTaskState.STOPPED;
String initialReason = null;
long initialGeneration = 0;
long initialCheckpoint = 0;
TransformIndexerPosition initialPosition = null;
if (state != null) {
initialTaskState = state.getTaskState();
initialReason = state.getReason();
@ -107,14 +94,13 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
initialState = existingState;
}
initialPosition = state.getPosition();
initialGeneration = state.getCheckpoint();
initialCheckpoint = state.getCheckpoint();
}
this.initialIndexerState = initialState;
this.initialPosition = initialPosition;
this.currentCheckpoint = new AtomicLong(initialGeneration);
this.taskState = new AtomicReference<>(initialTaskState);
this.stateReason = new AtomicReference<>(initialReason);
this.context = new TransformContext(initialTaskState, initialReason, initialCheckpoint, this);
}
public String getTransformId() {
@ -136,24 +122,26 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
public TransformState getState() {
if (getIndexer() == null) {
return new TransformState(
taskState.get(),
context.getTaskState(),
initialIndexerState,
initialPosition,
currentCheckpoint.get(),
stateReason.get(),
context.getCheckpoint(),
context.getStateReason(),
null,
null,
false);
false
);
} else {
return new TransformState(
taskState.get(),
indexer.get().getState(),
indexer.get().getPosition(),
currentCheckpoint.get(),
stateReason.get(),
getIndexer().getProgress(),
null,
getIndexer().shouldStopAtCheckpoint());
return new TransformState(
context.getTaskState(),
indexer.get().getState(),
indexer.get().getPosition(),
context.getCheckpoint(),
context.getStateReason(),
getIndexer().getProgress(),
null,
context.shouldStopAtCheckpoint()
);
}
}
@ -165,138 +153,149 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
}
}
public long getCheckpoint() {
return currentCheckpoint.get();
}
long incrementCheckpoint() {
return currentCheckpoint.getAndIncrement();
}
public void getCheckpointingInfo(TransformCheckpointService transformsCheckpointService,
ActionListener<TransformCheckpointingInfo> listener) {
public void getCheckpointingInfo(
TransformCheckpointService transformsCheckpointService,
ActionListener<TransformCheckpointingInfo> listener
) {
ClientTransformIndexer indexer = getIndexer();
if (indexer == null) {
transformsCheckpointService.getCheckpointingInfo(
transform.getId(),
currentCheckpoint.get(),
initialPosition,
null,
listener);
transformsCheckpointService.getCheckpointingInfo(transform.getId(), context.getCheckpoint(), initialPosition, null, listener);
return;
}
indexer.getCheckpointProvider().getCheckpointingInfo(
indexer
.getCheckpointProvider()
.getCheckpointingInfo(
indexer.getLastCheckpoint(),
indexer.getNextCheckpoint(),
indexer.getPosition(),
indexer.getProgress(),
ActionListener.wrap(
info -> {
if (indexer.getChangesLastDetectedAt() == null) {
listener.onResponse(info);
} else {
listener.onResponse(info.setChangesLastDetectedAt(indexer.getChangesLastDetectedAt()));
}
},
listener::onFailure
));
ActionListener.wrap(info -> {
if (context.getChangesLastDetectedAt() == null) {
listener.onResponse(info);
} else {
listener.onResponse(info.setChangesLastDetectedAt(context.getChangesLastDetectedAt()));
}
}, listener::onFailure)
);
}
/**
* Starts the transform and schedules it to be triggered in the future.
*
* NOTE: This should ONLY be called via {@link TransformPersistentTasksExecutor}
*
* @param startingCheckpoint The starting checkpoint, could null. Null indicates that there is no starting checkpoint
* @param listener The listener to alert once started
*/
synchronized void start(Long startingCheckpoint, ActionListener<StartTransformAction.Response> listener) {
logger.debug("[{}] start called with state [{}].", getTransformId(), getState());
if (taskState.get() == TransformTaskState.FAILED) {
listener.onFailure(new ElasticsearchStatusException(
TransformMessages.getMessage(CANNOT_START_FAILED_TRANSFORM,
getTransformId(),
stateReason.get()),
RestStatus.CONFLICT));
if (context.getTaskState() == TransformTaskState.FAILED) {
listener
.onFailure(
new ElasticsearchStatusException(
TransformMessages.getMessage(CANNOT_START_FAILED_TRANSFORM, getTransformId(), context.getStateReason()),
RestStatus.CONFLICT
)
);
return;
}
if (getIndexer() == null) {
// If our state is failed AND the indexer is null, the user needs to _stop?force=true so that the indexer gets
// fully initialized.
// If we are NOT failed, then we can assume that `start` was just called early in the process.
String msg = taskState.get() == TransformTaskState.FAILED ?
"It failed during the initialization process; force stop to allow reinitialization." :
"Try again later.";
listener.onFailure(new ElasticsearchStatusException("Task for transform [{}] not fully initialized. {}",
RestStatus.CONFLICT,
getTransformId(),
msg));
String msg = context.getTaskState() == TransformTaskState.FAILED
? "It failed during the initialization process; force stop to allow reinitialization."
: "Try again later.";
listener
.onFailure(
new ElasticsearchStatusException(
"Task for transform [{}] not fully initialized. {}",
RestStatus.CONFLICT,
getTransformId(),
msg
)
);
return;
}
final IndexerState newState = getIndexer().start();
if (Arrays.stream(RUNNING_STATES).noneMatch(newState::equals)) {
listener.onFailure(new ElasticsearchException("Cannot start task for transform [{}], because state was [{}]",
transform.getId(), newState));
listener
.onFailure(
new ElasticsearchException("Cannot start task for transform [{}], because state was [{}]", transform.getId(), newState)
);
return;
}
stateReason.set(null);
taskState.set(TransformTaskState.STARTED);
context.resetTaskState();
if (startingCheckpoint != null) {
currentCheckpoint.set(startingCheckpoint);
context.setCheckpoint(startingCheckpoint);
}
final TransformState state = new TransformState(
TransformTaskState.STARTED,
IndexerState.STOPPED,
getIndexer().getPosition(),
currentCheckpoint.get(),
context.getCheckpoint(),
null,
getIndexer().getProgress(),
null,
getIndexer().shouldStopAtCheckpoint());
context.shouldStopAtCheckpoint()
);
logger.info("[{}] updating state for transform to [{}].", transform.getId(), state.toString());
// Even though the indexer information is persisted to an index, we still need TransformTaskState in the clusterstate
// This keeps track of STARTED, FAILED, STOPPED
// This is because a FAILED state can occur because we cannot read the config from the internal index, which would imply that
// we could not read the previous state information from said index.
persistStateToClusterState(state, ActionListener.wrap(
task -> {
auditor.info(transform.getId(),
"Updated transform state to [" + state.getTaskState() + "].");
long now = System.currentTimeMillis();
// kick off the indexer
triggered(new Event(schedulerJobName(), now, now));
registerWithSchedulerJob();
listener.onResponse(new StartTransformAction.Response(true));
},
exc -> {
auditor.warning(transform.getId(),
"Failed to persist to cluster state while marking task as started. Failure: " + exc.getMessage());
logger.error(new ParameterizedMessage("[{}] failed updating state to [{}].", getTransformId(), state), exc);
getIndexer().stop();
listener.onFailure(new ElasticsearchException("Error while updating state for transform ["
+ transform.getId() + "] to [" + state.getIndexerState() + "].", exc));
}
));
// we could not read the previous state information from said index.
persistStateToClusterState(state, ActionListener.wrap(task -> {
auditor.info(transform.getId(), "Updated transform state to [" + state.getTaskState() + "].");
long now = System.currentTimeMillis();
// kick off the indexer
triggered(new Event(schedulerJobName(), now, now));
registerWithSchedulerJob();
listener.onResponse(new StartTransformAction.Response(true));
}, exc -> {
auditor
.warning(
transform.getId(),
"Failed to persist to cluster state while marking task as started. Failure: " + exc.getMessage()
);
logger.error(new ParameterizedMessage("[{}] failed updating state to [{}].", getTransformId(), state), exc);
getIndexer().stop();
listener
.onFailure(
new ElasticsearchException(
"Error while updating state for transform [" + transform.getId() + "] to [" + state.getIndexerState() + "].",
exc
)
);
}));
}
void setShouldStopAtCheckpoint(boolean shouldStopAtCheckpoint) {
this.context.setShouldStopAtCheckpoint(shouldStopAtCheckpoint);
}
/**
* This sets the flag for the task to stop at the next checkpoint.
*
* If first persists the flag to cluster state, and then mutates the local variable.
* If first persists the flag and then mutates the local variable.
*
* It only persists to cluster state if the value is different than what is currently held in memory.
* It only persists if the value is different than what is currently held in memory.
* @param shouldStopAtCheckpoint whether or not we should stop at the next checkpoint or not
* @param shouldStopAtCheckpointListener the listener to return to when we have persisted the updated value to the state index.
*/
public synchronized void setShouldStopAtCheckpoint(boolean shouldStopAtCheckpoint,
ActionListener<Void> shouldStopAtCheckpointListener) {
logger.debug("[{}] attempted to set task to stop at checkpoint [{}] with state [{}]",
getTransformId(),
shouldStopAtCheckpoint,
getState());
if (taskState.get() != TransformTaskState.STARTED || getIndexer() == null) {
public synchronized void setShouldStopAtCheckpoint(
boolean shouldStopAtCheckpoint,
ActionListener<Void> shouldStopAtCheckpointListener
) {
logger
.debug(
"[{}] attempted to set task to stop at checkpoint [{}] with state [{}]",
getTransformId(),
shouldStopAtCheckpoint,
getState()
);
if (context.getTaskState() != TransformTaskState.STARTED || getIndexer() == null) {
shouldStopAtCheckpointListener.onResponse(null);
return;
}
@ -304,11 +303,14 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
}
public synchronized void stop(boolean force, boolean shouldStopAtCheckpoint) {
logger.debug("[{}] stop called with force [{}], shouldStopAtCheckpoint [{}], state [{}]",
getTransformId(),
force,
shouldStopAtCheckpoint,
getState());
logger
.debug(
"[{}] stop called with force [{}], shouldStopAtCheckpoint [{}], state [{}]",
getTransformId(),
force,
shouldStopAtCheckpoint,
getState()
);
if (getIndexer() == null) {
// If there is no indexer the task has not been triggered
// but it still needs to be stopped and removed
@ -320,21 +322,21 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
return;
}
if (taskState.get() == TransformTaskState.FAILED && force == false) {
if (context.getTaskState() == TransformTaskState.FAILED && force == false) {
throw new ElasticsearchStatusException(
TransformMessages.getMessage(CANNOT_STOP_FAILED_TRANSFORM,
getTransformId(),
stateReason.get()),
RestStatus.CONFLICT);
TransformMessages.getMessage(CANNOT_STOP_FAILED_TRANSFORM, getTransformId(), context.getStateReason()),
RestStatus.CONFLICT
);
}
stateReason.set(null);
context.resetReasonAndFailureCounter();
// No reason to keep it in the potentially failed state.
boolean wasFailed = taskState.compareAndSet(TransformTaskState.FAILED, TransformTaskState.STARTED);
boolean wasFailed = context.setTaskState(TransformTaskState.FAILED, TransformTaskState.STARTED);
// shouldStopAtCheckpoint only comes into play when onFinish is called (or doSaveState right after).
// if it is false, stop immediately
if (shouldStopAtCheckpoint == false ||
// If state was in a failed state, we should stop immediately as we will never reach the next checkpoint
// If state was in a failed state, we should stop immediately as we will never reach the next checkpoint
wasFailed ||
// If the indexerState is STARTED and it is on an initialRun, that means that the indexer has previously finished a checkpoint,
// or has yet to even start one.
@ -351,7 +353,7 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
@Override
public synchronized void triggered(Event event) {
// Ignore if event is not for this job
if (event.getJobName().equals(schedulerJobName()) == false) {
if (event.getJobName().equals(schedulerJobName()) == false) {
return;
}
@ -360,18 +362,21 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
return;
}
if (taskState.get() == TransformTaskState.FAILED || taskState.get() == TransformTaskState.STOPPED) {
logger.debug("[{}] schedule was triggered for transform but task is [{}]. Ignoring trigger.",
getTransformId(),
taskState.get());
if (context.getTaskState() == TransformTaskState.FAILED || context.getTaskState() == TransformTaskState.STOPPED) {
logger
.debug(
"[{}] schedule was triggered for transform but task is [{}]. Ignoring trigger.",
getTransformId(),
context.getTaskState()
);
return;
}
// ignore trigger if indexer is running or completely stopped
IndexerState indexerState = getIndexer().getState();
if (IndexerState.INDEXING.equals(indexerState) ||
IndexerState.STOPPING.equals(indexerState) ||
IndexerState.STOPPED.equals(indexerState)) {
if (IndexerState.INDEXING.equals(indexerState)
|| IndexerState.STOPPING.equals(indexerState)
|| IndexerState.STOPPED.equals(indexerState)) {
logger.debug("[{}] indexer for transform has state [{}]. Ignoring trigger.", getTransformId(), indexerState);
return;
}
@ -379,7 +384,7 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
logger.debug("[{}] transform indexer schedule has triggered, state: [{}].", event.getJobName(), indexerState);
// if it runs for the 1st time we just do it, if not we check for changes
if (currentCheckpoint.get() == 0) {
if (context.getCheckpoint() == 0) {
logger.debug("[{}] trigger initial run.", getTransformId());
getIndexer().maybeTriggerAsyncJob(System.currentTimeMillis());
} else if (getIndexer().isContinuous()) {
@ -391,31 +396,27 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
* Attempt to gracefully cleanup the transform so it can be terminated.
* This tries to remove the job from the scheduler and completes the persistent task
*/
synchronized void shutdown() {
@Override
public synchronized void shutdown() {
deregisterSchedulerJob();
markAsCompleted();
}
void persistStateToClusterState(TransformState state,
ActionListener<PersistentTasksCustomMetaData.PersistentTask<?>> listener) {
updatePersistentTaskState(state, ActionListener.wrap(
success -> {
logger.debug("[{}] successfully updated state for transform to [{}].", transform.getId(), state.toString());
listener.onResponse(success);
},
failure -> {
logger.error(new ParameterizedMessage("[{}] failed to update cluster state for transform.",
transform.getId()),
failure);
listener.onFailure(failure);
}
));
void persistStateToClusterState(TransformState state, ActionListener<PersistentTasksCustomMetaData.PersistentTask<?>> listener) {
updatePersistentTaskState(state, ActionListener.wrap(success -> {
logger.debug("[{}] successfully updated state for transform to [{}].", transform.getId(), state.toString());
listener.onResponse(success);
}, failure -> {
logger.error(new ParameterizedMessage("[{}] failed to update cluster state for transform.", transform.getId()), failure);
listener.onFailure(failure);
}));
}
synchronized void markAsFailed(String reason, ActionListener<Void> listener) {
@Override
public synchronized void fail(String reason, ActionListener<Void> listener) {
// If we are already flagged as failed, this probably means that a second trigger started firing while we were attempting to
// flag the previously triggered indexer as failed. Exit early as we are already flagged as failed.
if (taskState.get() == TransformTaskState.FAILED) {
if (context.getTaskState() == TransformTaskState.FAILED) {
logger.warn("[{}] is already failed but encountered new failure; reason [{}].", getTransformId(), reason);
listener.onResponse(null);
return;
@ -441,28 +442,21 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
deregisterSchedulerJob();
// The idea of stopping at the next checkpoint is no longer valid. Since a failed task could potentially START again,
// we should set this flag to false.
if (getIndexer() != null) {
getIndexer().setShouldStopAtCheckpoint(false);
}
context.setShouldStopAtCheckpoint(false);
// The end user should see that the task is in a failed state, and attempt to stop it again but with force=true
taskState.set(TransformTaskState.FAILED);
stateReason.set(reason);
context.setTaskStateToFailed(reason);
TransformState newState = getState();
// Even though the indexer information is persisted to an index, we still need TransformTaskState in the clusterstate
// This keeps track of STARTED, FAILED, STOPPED
// This is because a FAILED state could occur because we failed to read the config from the internal index, which would imply that
// we could not read the previous state information from said index.
persistStateToClusterState(newState, ActionListener.wrap(
r -> listener.onResponse(null),
e -> {
String msg = "Failed to persist to cluster state while marking task as failed with reason [" + reason + "].";
auditor.warning(transform.getId(),
msg + " Failure: " + e.getMessage());
logger.error(new ParameterizedMessage("[{}] {}", getTransformId(), msg),
e);
listener.onFailure(e);
}
));
// we could not read the previous state information from said index.
persistStateToClusterState(newState, ActionListener.wrap(r -> listener.onResponse(null), e -> {
String msg = "Failed to persist to cluster state while marking task as failed with reason [" + reason + "].";
auditor.warning(transform.getId(), msg + " Failure: " + e.getMessage());
logger.error(new ParameterizedMessage("[{}] {}", getTransformId(), msg), e);
listener.onFailure(e);
}));
}
/**
@ -472,9 +466,7 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
*/
@Override
public synchronized void onCancelled() {
logger.info("[{}] received cancellation request for transform, state: [{}].",
getTransformId(),
taskState.get());
logger.info("[{}] received cancellation request for transform, state: [{}].", getTransformId(), context.getTaskState());
if (getIndexer() != null && getIndexer().abort()) {
// there is no background transform running, we can shutdown safely
shutdown();
@ -482,14 +474,10 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
}
TransformTask setNumFailureRetries(int numFailureRetries) {
this.numFailureRetries = numFailureRetries;
context.setNumFailureRetries(numFailureRetries);
return this;
}
int getNumFailureRetries() {
return numFailureRetries;
}
private void registerWithSchedulerJob() {
schedulerEngine.register(this);
final SchedulerEngine.Job schedulerJob = new SchedulerEngine.Job(schedulerJobName(), next());
@ -513,20 +501,7 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
}
synchronized void initializeIndexer(ClientTransformIndexerBuilder indexerBuilder) {
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();
indexer.set(indexerBuilder.build(getThreadPool().executor(ThreadPool.Names.GENERIC), context));
}
ThreadPool getThreadPool() {
@ -534,14 +509,7 @@ public class TransformTask extends AllocatedPersistentTask implements SchedulerE
}
TransformTaskState getTaskState() {
return taskState.get();
return context.getTaskState();
}
void setStateReason(String reason) {
stateReason.set(reason);
}
String getStateReason() {
return stateReason.get();
}
}

View File

@ -50,33 +50,40 @@ public class DefaultCheckpointProviderTests extends ESTestCase {
client,
transformConfigManager,
transformAuditor,
transformConfig);
transformConfig
);
assertExpectation(
new MockLogAppender.SeenEventExpectation("warn when source is empty",
new MockLogAppender.SeenEventExpectation(
"warn when source is empty",
checkpointProviderlogger.getName(),
Level.WARN,
"Source did not resolve to any open indexes for transform [" + transformId + "]"),
new MockTransformAuditor.SeenAuditExpectation("warn when source is empty",
"[" + transformId + "] Source did not resolve to any open indexes"
),
new MockTransformAuditor.SeenAuditExpectation(
"warn when source is empty",
org.elasticsearch.xpack.core.common.notifications.Level.WARNING,
transformId,
"Source did not resolve to any open indexes"),
() -> {
provider.reportSourceIndexChanges(Collections.singleton("index"), Collections.emptySet());
});
"Source did not resolve to any open indexes"
),
() -> { provider.reportSourceIndexChanges(Collections.singleton("index"), Collections.emptySet()); }
);
assertExpectation(
new MockLogAppender.UnseenEventExpectation("do not warn if empty again",
new MockLogAppender.UnseenEventExpectation(
"do not warn if empty again",
checkpointProviderlogger.getName(),
Level.WARN,
"Source did not resolve to any concrete indexes"),
new MockTransformAuditor.UnseenAuditExpectation("do not warn if empty again",
"Source did not resolve to any concrete indexes"
),
new MockTransformAuditor.UnseenAuditExpectation(
"do not warn if empty again",
org.elasticsearch.xpack.core.common.notifications.Level.WARNING,
transformId,
"Source did not resolve to any concrete indexes"),
() -> {
provider.reportSourceIndexChanges(Collections.emptySet(), Collections.emptySet());
});
"Source did not resolve to any concrete indexes"
),
() -> { provider.reportSourceIndexChanges(Collections.emptySet(), Collections.emptySet()); }
);
}
public void testReportSourceIndexChangesAddDelete() throws Exception {
@ -87,48 +94,55 @@ public class DefaultCheckpointProviderTests extends ESTestCase {
client,
transformConfigManager,
transformAuditor,
transformConfig);
transformConfig
);
assertExpectation(
new MockLogAppender.SeenEventExpectation("info about adds/removal",
new MockLogAppender.SeenEventExpectation(
"info about adds/removal",
checkpointProviderlogger.getName(),
Level.DEBUG,
"Source index resolve found changes, removedIndexes: [index], new indexes: [other_index] for transform [" +
transformId + "]"),
new MockTransformAuditor.SeenAuditExpectation("info about adds/removal",
"[" + transformId + "] Source index resolve found changes, removedIndexes: [index], new indexes: [other_index]"
),
new MockTransformAuditor.SeenAuditExpectation(
"info about adds/removal",
org.elasticsearch.xpack.core.common.notifications.Level.INFO,
transformId,
"Source index resolve found changes, removedIndexes: [index], new indexes: [other_index]"),
() -> {
provider.reportSourceIndexChanges(Collections.singleton("index"), Collections.singleton("other_index"));
});
"Source index resolve found changes, removedIndexes: [index], new indexes: [other_index]"
),
() -> { provider.reportSourceIndexChanges(Collections.singleton("index"), Collections.singleton("other_index")); }
);
assertExpectation(
new MockLogAppender.SeenEventExpectation("info about adds/removal",
new MockLogAppender.SeenEventExpectation(
"info about adds/removal",
checkpointProviderlogger.getName(),
Level.DEBUG,
"Source index resolve found changes, removedIndexes: [index], new indexes: [] for transform [" +
transformId + "]"),
new MockTransformAuditor.SeenAuditExpectation("info about adds/removal",
"[" + transformId + "] Source index resolve found changes, removedIndexes: [index], new indexes: []"
),
new MockTransformAuditor.SeenAuditExpectation(
"info about adds/removal",
org.elasticsearch.xpack.core.common.notifications.Level.INFO,
transformId,
"Source index resolve found changes, removedIndexes: [index], new indexes: []"),
() -> {
provider.reportSourceIndexChanges(Sets.newHashSet("index", "other_index"), Collections.singleton("other_index"));
});
"Source index resolve found changes, removedIndexes: [index], new indexes: []"
),
() -> { provider.reportSourceIndexChanges(Sets.newHashSet("index", "other_index"), Collections.singleton("other_index")); }
);
assertExpectation(
new MockLogAppender.SeenEventExpectation("info about adds/removal",
new MockLogAppender.SeenEventExpectation(
"info about adds/removal",
checkpointProviderlogger.getName(),
Level.DEBUG,
"Source index resolve found changes, removedIndexes: [], new indexes: [other_index] for transform [" +
transformId + "]"),
new MockTransformAuditor.SeenAuditExpectation("info about adds/removal",
"[" + transformId + "] Source index resolve found changes, removedIndexes: [], new indexes: [other_index]"
),
new MockTransformAuditor.SeenAuditExpectation(
"info about adds/removal",
org.elasticsearch.xpack.core.common.notifications.Level.INFO,
transformId,
"Source index resolve found changes, removedIndexes: [], new indexes: [other_index]"),
() -> {
provider.reportSourceIndexChanges(Collections.singleton("index"), Sets.newHashSet("index", "other_index"));
});
"Source index resolve found changes, removedIndexes: [], new indexes: [other_index]"
),
() -> { provider.reportSourceIndexChanges(Collections.singleton("index"), Sets.newHashSet("index", "other_index")); }
);
}
public void testReportSourceIndexChangesAddDeleteMany() throws Exception {
@ -139,7 +153,8 @@ public class DefaultCheckpointProviderTests extends ESTestCase {
client,
transformConfigManager,
transformAuditor,
transformConfig);
transformConfig
);
HashSet<String> oldSet = new HashSet<>();
for (int i = 0; i < 100; ++i) {
@ -151,23 +166,24 @@ public class DefaultCheckpointProviderTests extends ESTestCase {
}
assertExpectation(
new MockLogAppender.SeenEventExpectation("info about adds/removal",
new MockLogAppender.SeenEventExpectation(
"info about adds/removal",
checkpointProviderlogger.getName(),
Level.DEBUG,
"Source index resolve found more than 10 changes, [50] removed indexes, [50] new indexes for transform [" +
transformId + "]"),
new MockTransformAuditor.SeenAuditExpectation("info about adds/removal",
"[" + transformId + "] Source index resolve found more than 10 changes, [50] removed indexes, [50] new indexes"
),
new MockTransformAuditor.SeenAuditExpectation(
"info about adds/removal",
org.elasticsearch.xpack.core.common.notifications.Level.INFO,
transformId,
"Source index resolve found more than 10 changes, [50] removed indexes, [50] new indexes"),
() -> {
provider.reportSourceIndexChanges(oldSet, newSet);
});
"Source index resolve found more than 10 changes, [50] removed indexes, [50] new indexes"
),
() -> { provider.reportSourceIndexChanges(oldSet, newSet); }
);
}
private void assertExpectation(LoggingExpectation loggingExpectation,
AuditExpectation auditExpectation,
Runnable codeBlock) throws IllegalAccessException {
private void assertExpectation(LoggingExpectation loggingExpectation, AuditExpectation auditExpectation, Runnable codeBlock)
throws IllegalAccessException {
MockLogAppender mockLogAppender = new MockLogAppender();
mockLogAppender.start();

View File

@ -7,22 +7,22 @@
package org.elasticsearch.xpack.transform.transforms;
import org.elasticsearch.client.Client;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.indexing.IndexerState;
import org.elasticsearch.xpack.core.scheduler.SchedulerEngine;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats;
import org.elasticsearch.xpack.core.transform.transforms.TransformTaskParams;
import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpoint;
import org.elasticsearch.xpack.core.transform.transforms.TransformConfig;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats;
import org.elasticsearch.xpack.core.transform.transforms.persistence.TransformInternalIndexConstants;
import org.elasticsearch.xpack.transform.checkpoint.CheckpointProvider;
import org.elasticsearch.xpack.transform.notifications.TransformAuditor;
import org.elasticsearch.xpack.transform.persistence.SeqNoPrimaryTermAndIndex;
import org.elasticsearch.xpack.transform.persistence.TransformConfigManager;
import java.time.Instant;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
@ -36,19 +36,12 @@ public class ClientTransformIndexerTests extends ESTestCase {
public void testAudiOnFinishFrequency() {
ThreadPool threadPool = mock(ThreadPool.class);
when(threadPool.executor("generic")).thenReturn(mock(ExecutorService.class));
TransformTask parentTask = new TransformTask(1,
"transform",
"ptask",
new TaskId("transform:1"),
mock(TransformTaskParams.class),
null,
mock(SchedulerEngine.class),
mock(TransformAuditor.class),
threadPool,
Collections.emptyMap());
ClientTransformIndexer indexer = new ClientTransformIndexer(
mock(Executor.class),
mock(TransformConfigManager.class),
mock(CheckpointProvider.class),
new TransformProgressGatherer(mock(Client.class)),
new AtomicReference<>(IndexerState.STOPPED),
null,
mock(Client.class),
@ -57,18 +50,12 @@ public class ClientTransformIndexerTests extends ESTestCase {
mock(TransformConfig.class),
Collections.emptyMap(),
null,
new TransformCheckpoint("transform",
Instant.now().toEpochMilli(),
0L,
Collections.emptyMap(),
Instant.now().toEpochMilli()),
new TransformCheckpoint("transform",
Instant.now().toEpochMilli(),
2L,
Collections.emptyMap(),
Instant.now().toEpochMilli()),
parentTask,
false);
new TransformCheckpoint("transform", Instant.now().toEpochMilli(), 0L, Collections.emptyMap(), Instant.now().toEpochMilli()),
new TransformCheckpoint("transform", Instant.now().toEpochMilli(), 2L, Collections.emptyMap(), Instant.now().toEpochMilli()),
new SeqNoPrimaryTermAndIndex(1, 1, TransformInternalIndexConstants.LATEST_INDEX_NAME),
mock(TransformContext.class),
false
);
List<Boolean> shouldAudit = IntStream.range(0, 100_000).boxed().map(indexer::shouldAuditOnFinish).collect(Collectors.toList());

View File

@ -18,26 +18,28 @@ import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.breaker.CircuitBreaker.Durability;
import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.profile.SearchProfileShardResults;
import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.test.client.NoOpClient;
import org.elasticsearch.xpack.core.indexing.IndexerState;
import org.elasticsearch.xpack.core.indexing.IterationResult;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerPosition;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats;
import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpoint;
import org.elasticsearch.xpack.core.transform.transforms.TransformConfig;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerPosition;
import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats;
import org.elasticsearch.xpack.core.transform.transforms.TransformTaskState;
import org.elasticsearch.xpack.core.transform.transforms.pivot.AggregationConfigTests;
import org.elasticsearch.xpack.core.transform.transforms.pivot.GroupConfigTests;
import org.elasticsearch.xpack.core.transform.transforms.pivot.PivotConfig;
import org.elasticsearch.xpack.transform.checkpoint.CheckpointProvider;
import org.elasticsearch.xpack.transform.notifications.TransformAuditor;
import org.elasticsearch.xpack.transform.persistence.TransformConfigManager;
import org.elasticsearch.xpack.transform.transforms.pivot.Pivot;
import org.junit.After;
import org.junit.Before;
import java.io.PrintWriter;
@ -48,6 +50,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import java.util.function.Function;
@ -63,7 +66,6 @@ import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class TransformIndexerTests extends ESTestCase {
@ -79,18 +81,37 @@ public class TransformIndexerTests extends ESTestCase {
private CountDownLatch latch;
MockedTransformIndexer(
Executor executor,
TransformConfig transformConfig,
Map<String, String> fieldMappings,
TransformAuditor auditor,
AtomicReference<IndexerState> initialState,
TransformIndexerPosition initialPosition,
TransformIndexerStats jobStats,
Function<SearchRequest, SearchResponse> searchFunction,
Function<BulkRequest, BulkResponse> bulkFunction,
Consumer<Exception> failureConsumer) {
super(executor, auditor, transformConfig, fieldMappings, initialState, initialPosition, jobStats,
/* TransformProgress */ null, TransformCheckpoint.EMPTY, TransformCheckpoint.EMPTY);
Executor executor,
TransformConfigManager transformsConfigManager,
CheckpointProvider checkpointProvider,
TransformProgressGatherer progressGatherer,
TransformConfig transformConfig,
Map<String, String> fieldMappings,
TransformAuditor auditor,
AtomicReference<IndexerState> initialState,
TransformIndexerPosition initialPosition,
TransformIndexerStats jobStats,
TransformContext context,
Function<SearchRequest, SearchResponse> searchFunction,
Function<BulkRequest, BulkResponse> bulkFunction,
Consumer<Exception> failureConsumer
) {
super(
executor,
transformsConfigManager,
checkpointProvider,
progressGatherer,
auditor,
transformConfig,
fieldMappings,
initialState,
initialPosition,
jobStats,
/* TransformProgress */ null,
TransformCheckpoint.EMPTY,
TransformCheckpoint.EMPTY,
context
);
this.searchFunction = searchFunction;
this.bulkFunction = bulkFunction;
this.failureConsumer = failureConsumer;
@ -180,36 +201,39 @@ public class TransformIndexerTests extends ESTestCase {
fail("failIndexer should not be called, received error: " + message);
}
@Override
protected void sourceHasChanged(ActionListener<Boolean> listener) {
listener.onResponse(false);
}
}
@Before
public void setUpMocks() {
client = mock(Client.class);
ThreadPool threadPool = mock(ThreadPool.class);
when(client.threadPool()).thenReturn(threadPool);
when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY));
client = new NoOpClient(getTestName());
}
@After
public void tearDownClient() {
client.close();
}
public void testPageSizeAdapt() throws Exception {
Integer pageSize = randomBoolean() ? null : randomIntBetween(500, 10_000);
TransformConfig config = new TransformConfig(randomAlphaOfLength(10),
TransformConfig config = new TransformConfig(
randomAlphaOfLength(10),
randomSourceConfig(),
randomDestConfig(),
null,
null,
null,
new PivotConfig(GroupConfigTests.randomGroupConfig(), AggregationConfigTests.randomAggregationConfig(), pageSize),
randomBoolean() ? null : randomAlphaOfLengthBetween(1, 1000));
randomBoolean() ? null : randomAlphaOfLengthBetween(1, 1000)
);
AtomicReference<IndexerState> state = new AtomicReference<>(IndexerState.STOPPED);
final long initialPageSize = pageSize == null ? Pivot.DEFAULT_INITIAL_PAGE_SIZE : pageSize;
Function<SearchRequest, SearchResponse> searchFunction = searchRequest -> {
throw new SearchPhaseExecutionException("query", "Partial shards failure", new ShardSearchFailure[] {
new ShardSearchFailure(new CircuitBreakingException("to much memory", 110, 100, Durability.TRANSIENT)) });
throw new SearchPhaseExecutionException(
"query",
"Partial shards failure",
new ShardSearchFailure[] {
new ShardSearchFailure(new CircuitBreakingException("to much memory", 110, 100, Durability.TRANSIENT)) }
);
};
Function<BulkRequest, BulkResponse> bulkFunction = bulkRequest -> new BulkResponse(new BulkItemResponse[0], 100);
@ -224,9 +248,24 @@ public class TransformIndexerTests extends ESTestCase {
final ExecutorService executor = Executors.newFixedThreadPool(1);
try {
TransformAuditor auditor = new TransformAuditor(client, "node_1");
TransformContext context = new TransformContext(TransformTaskState.STARTED, "", 0, mock(TransformContext.Listener.class));
MockedTransformIndexer indexer = new MockedTransformIndexer(executor, config, Collections.emptyMap(), auditor, state, null,
new TransformIndexerStats(), searchFunction, bulkFunction, failureConsumer);
MockedTransformIndexer indexer = new MockedTransformIndexer(
executor,
mock(TransformConfigManager.class),
mock(CheckpointProvider.class),
new TransformProgressGatherer(client),
config,
Collections.emptyMap(),
auditor,
state,
null,
new TransformIndexerStats(),
context,
searchFunction,
bulkFunction,
failureConsumer
);
final CountDownLatch latch = indexer.newLatch(1);
indexer.start();
assertThat(indexer.getState(), equalTo(IndexerState.STARTED));
@ -234,10 +273,10 @@ public class TransformIndexerTests extends ESTestCase {
assertThat(indexer.getState(), equalTo(IndexerState.INDEXING));
latch.countDown();
assertBusy(() -> assertThat(indexer.getState(), equalTo(IndexerState.STARTED)));
assertBusy(() -> assertThat(indexer.getState(), equalTo(IndexerState.STARTED)), 10, TimeUnit.MINUTES);
long pageSizeAfterFirstReduction = indexer.getPageSize();
assertThat(initialPageSize, greaterThan(pageSizeAfterFirstReduction));
assertThat(pageSizeAfterFirstReduction, greaterThan((long)TransformIndexer.MINIMUM_PAGE_SIZE));
assertThat(pageSizeAfterFirstReduction, greaterThan((long) TransformIndexer.MINIMUM_PAGE_SIZE));
// run indexer a 2nd time
final CountDownLatch secondRunLatch = indexer.newLatch(1);
@ -251,8 +290,8 @@ public class TransformIndexerTests extends ESTestCase {
assertBusy(() -> assertThat(indexer.getState(), equalTo(IndexerState.STARTED)));
// assert that page size has been reduced again
assertThat(pageSizeAfterFirstReduction, greaterThan((long)indexer.getPageSize()));
assertThat(pageSizeAfterFirstReduction, greaterThan((long)TransformIndexer.MINIMUM_PAGE_SIZE));
assertThat(pageSizeAfterFirstReduction, greaterThan((long) indexer.getPageSize()));
assertThat(pageSizeAfterFirstReduction, greaterThan((long) TransformIndexer.MINIMUM_PAGE_SIZE));
} finally {
executor.shutdownNow();
@ -261,22 +300,35 @@ public class TransformIndexerTests extends ESTestCase {
public void testDoProcessAggNullCheck() {
Integer pageSize = randomBoolean() ? null : randomIntBetween(500, 10_000);
TransformConfig config = new TransformConfig(randomAlphaOfLength(10),
TransformConfig config = new TransformConfig(
randomAlphaOfLength(10),
randomSourceConfig(),
randomDestConfig(),
null,
null,
null,
new PivotConfig(GroupConfigTests.randomGroupConfig(), AggregationConfigTests.randomAggregationConfig(), pageSize),
randomBoolean() ? null : randomAlphaOfLengthBetween(1, 1000));
SearchResponse searchResponse = new SearchResponse(new InternalSearchResponse(
new SearchHits(
new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0.0f),
// Simulate completely null aggs
null,
new Suggest(Collections.emptyList()),
new SearchProfileShardResults(Collections.emptyMap()), false, false, 1),
"", 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY);
randomBoolean() ? null : randomAlphaOfLengthBetween(1, 1000)
);
SearchResponse searchResponse = new SearchResponse(
new InternalSearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0.0f),
// Simulate completely null aggs
null,
new Suggest(Collections.emptyList()),
new SearchProfileShardResults(Collections.emptyMap()),
false,
false,
1
),
"",
1,
1,
0,
0,
ShardSearchFailure.EMPTY_ARRAY,
SearchResponse.Clusters.EMPTY
);
AtomicReference<IndexerState> state = new AtomicReference<>(IndexerState.STOPPED);
Function<SearchRequest, SearchResponse> searchFunction = searchRequest -> searchResponse;
Function<BulkRequest, BulkResponse> bulkFunction = bulkRequest -> new BulkResponse(new BulkItemResponse[0], 100);
@ -291,9 +343,24 @@ public class TransformIndexerTests extends ESTestCase {
final ExecutorService executor = Executors.newFixedThreadPool(1);
try {
TransformAuditor auditor = mock(TransformAuditor.class);
TransformContext context = new TransformContext(TransformTaskState.STARTED, "", 0, mock(TransformContext.Listener.class));
MockedTransformIndexer indexer = new MockedTransformIndexer(executor, config, Collections.emptyMap(), auditor, state, null,
new TransformIndexerStats(), searchFunction, bulkFunction, failureConsumer);
MockedTransformIndexer indexer = new MockedTransformIndexer(
executor,
mock(TransformConfigManager.class),
mock(CheckpointProvider.class),
new TransformProgressGatherer(client),
config,
Collections.emptyMap(),
auditor,
state,
null,
new TransformIndexerStats(),
context,
searchFunction,
bulkFunction,
failureConsumer
);
IterationResult<TransformIndexerPosition> newPosition = indexer.doProcess(searchResponse);
assertThat(newPosition.getToIndex(), is(empty()));

View File

@ -32,6 +32,7 @@ import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.scheduler.SchedulerEngine;
import org.elasticsearch.xpack.core.transform.transforms.TransformTaskParams;
import org.elasticsearch.xpack.core.transform.transforms.persistence.TransformInternalIndexConstants;
import org.elasticsearch.xpack.transform.Transform;
import org.elasticsearch.xpack.transform.checkpoint.TransformCheckpointService;
import org.elasticsearch.xpack.transform.notifications.TransformAuditor;
import org.elasticsearch.xpack.transform.persistence.TransformConfigManager;
@ -53,48 +54,71 @@ public class TransformPersistentTasksExecutorTests extends ESTestCase {
MetaData.Builder metaData = MetaData.builder();
RoutingTable.Builder routingTable = RoutingTable.builder();
addIndices(metaData, routingTable);
PersistentTasksCustomMetaData.Builder pTasksBuilder = PersistentTasksCustomMetaData.builder()
.addTask("transform-task-1",
PersistentTasksCustomMetaData.Builder pTasksBuilder = PersistentTasksCustomMetaData
.builder()
.addTask(
"transform-task-1",
TransformTaskParams.NAME,
new TransformTaskParams("transform-task-1", Version.CURRENT, null),
new PersistentTasksCustomMetaData.Assignment("current-data-node-with-1-tasks", ""))
.addTask("transform-task-2",
new PersistentTasksCustomMetaData.Assignment("current-data-node-with-1-tasks", "")
)
.addTask(
"transform-task-2",
TransformTaskParams.NAME,
new TransformTaskParams("transform-task-2", Version.CURRENT, null),
new PersistentTasksCustomMetaData.Assignment("current-data-node-with-2-tasks", ""))
.addTask("transform-task-3",
new PersistentTasksCustomMetaData.Assignment("current-data-node-with-2-tasks", "")
)
.addTask(
"transform-task-3",
TransformTaskParams.NAME,
new TransformTaskParams("transform-task-3", Version.CURRENT, null),
new PersistentTasksCustomMetaData.Assignment("current-data-node-with-2-tasks", ""));
new PersistentTasksCustomMetaData.Assignment("current-data-node-with-2-tasks", "")
);
PersistentTasksCustomMetaData pTasks = pTasksBuilder.build();
metaData.putCustom(PersistentTasksCustomMetaData.TYPE, pTasks);
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder()
.add(new DiscoveryNode("past-data-node-1",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.V_7_4_0))
.add(new DiscoveryNode("current-data-node-with-2-tasks",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.CURRENT))
.add(new DiscoveryNode("non-data-node-1",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
Collections.singleton(DiscoveryNodeRole.MASTER_ROLE),
Version.CURRENT))
.add(new DiscoveryNode("current-data-node-with-1-tasks",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.CURRENT));
DiscoveryNodes.Builder nodes = DiscoveryNodes
.builder()
.add(
new DiscoveryNode(
"past-data-node-1",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.V_7_4_0
)
)
.add(
new DiscoveryNode(
"current-data-node-with-2-tasks",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.CURRENT
)
)
.add(
new DiscoveryNode(
"non-data-node-1",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
Collections.singleton(DiscoveryNodeRole.MASTER_ROLE),
Version.CURRENT
)
)
.add(
new DiscoveryNode(
"current-data-node-with-1-tasks",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.CURRENT
)
);
ClusterState.Builder csBuilder = ClusterState.builder(new ClusterName("_name"))
.nodes(nodes);
ClusterState.Builder csBuilder = ClusterState.builder(new ClusterName("_name")).nodes(nodes);
csBuilder.routingTable(routingTable.build());
csBuilder.metaData(metaData);
@ -102,60 +126,80 @@ public class TransformPersistentTasksExecutorTests extends ESTestCase {
Client client = mock(Client.class);
TransformAuditor mockAuditor = mock(TransformAuditor.class);
TransformConfigManager transformsConfigManager = new TransformConfigManager(client, xContentRegistry());
TransformCheckpointService transformCheckpointService = new TransformCheckpointService(client,
transformsConfigManager, mockAuditor);
ClusterSettings cSettings = new ClusterSettings(Settings.EMPTY,
Collections.singleton(TransformTask.NUM_FAILURE_RETRIES_SETTING));
TransformCheckpointService transformCheckpointService = new TransformCheckpointService(
client,
transformsConfigManager,
mockAuditor
);
ClusterSettings cSettings = new ClusterSettings(Settings.EMPTY, Collections.singleton(Transform.NUM_FAILURE_RETRIES_SETTING));
ClusterService clusterService = mock(ClusterService.class);
when(clusterService.getClusterSettings()).thenReturn(cSettings);
when(clusterService.state()).thenReturn(TransformInternalIndexTests.STATE_WITH_LATEST_VERSIONED_INDEX_TEMPLATE);
TransformPersistentTasksExecutor executor = new TransformPersistentTasksExecutor(client,
TransformPersistentTasksExecutor executor = new TransformPersistentTasksExecutor(
client,
transformsConfigManager,
transformCheckpointService, mock(SchedulerEngine.class),
transformCheckpointService,
mock(SchedulerEngine.class),
new TransformAuditor(client, ""),
mock(ThreadPool.class),
clusterService,
Settings.EMPTY);
Settings.EMPTY
);
assertThat(executor.getAssignment(new TransformTaskParams("new-task-id", Version.CURRENT, null), cs).getExecutorNode(),
equalTo("current-data-node-with-1-tasks"));
assertThat(executor.getAssignment(new TransformTaskParams("new-old-task-id", Version.V_7_2_0, null), cs).getExecutorNode(),
equalTo("past-data-node-1"));
assertThat(
executor.getAssignment(new TransformTaskParams("new-task-id", Version.CURRENT, null), cs).getExecutorNode(),
equalTo("current-data-node-with-1-tasks")
);
}
public void testDoNotSelectOldNodes() {
MetaData.Builder metaData = MetaData.builder();
RoutingTable.Builder routingTable = RoutingTable.builder();
addIndices(metaData, routingTable);
PersistentTasksCustomMetaData.Builder pTasksBuilder = PersistentTasksCustomMetaData.builder()
.addTask("transform-task-1",
PersistentTasksCustomMetaData.Builder pTasksBuilder = PersistentTasksCustomMetaData
.builder()
.addTask(
"transform-task-1",
TransformTaskParams.NAME,
new TransformTaskParams("transform-task-1", Version.CURRENT, null),
new PersistentTasksCustomMetaData.Assignment("current-data-node-with-1-task", ""));
new PersistentTasksCustomMetaData.Assignment("current-data-node-with-1-task", "")
);
PersistentTasksCustomMetaData pTasks = pTasksBuilder.build();
metaData.putCustom(PersistentTasksCustomMetaData.TYPE, pTasks);
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder()
.add(new DiscoveryNode("old-data-node-1",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.V_7_2_0))
.add(new DiscoveryNode("current-data-node-with-1-task",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.CURRENT))
.add(new DiscoveryNode("non-data-node-1",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
Collections.singleton(DiscoveryNodeRole.MASTER_ROLE),
Version.CURRENT));
DiscoveryNodes.Builder nodes = DiscoveryNodes
.builder()
.add(
new DiscoveryNode(
"old-data-node-1",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.V_7_2_0
)
)
.add(
new DiscoveryNode(
"current-data-node-with-1-task",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.CURRENT
)
)
.add(
new DiscoveryNode(
"non-data-node-1",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
Collections.singleton(DiscoveryNodeRole.MASTER_ROLE),
Version.CURRENT
)
);
ClusterState.Builder csBuilder = ClusterState.builder(new ClusterName("_name"))
.nodes(nodes);
ClusterState.Builder csBuilder = ClusterState.builder(new ClusterName("_name")).nodes(nodes);
csBuilder.routingTable(routingTable.build());
csBuilder.metaData(metaData);
@ -163,41 +207,57 @@ public class TransformPersistentTasksExecutorTests extends ESTestCase {
Client client = mock(Client.class);
TransformAuditor mockAuditor = mock(TransformAuditor.class);
TransformConfigManager transformsConfigManager = new TransformConfigManager(client, xContentRegistry());
TransformCheckpointService transformCheckpointService = new TransformCheckpointService(client,
transformsConfigManager, mockAuditor);
ClusterSettings cSettings = new ClusterSettings(Settings.EMPTY,
Collections.singleton(TransformTask.NUM_FAILURE_RETRIES_SETTING));
TransformCheckpointService transformCheckpointService = new TransformCheckpointService(
client,
transformsConfigManager,
mockAuditor
);
ClusterSettings cSettings = new ClusterSettings(Settings.EMPTY, Collections.singleton(Transform.NUM_FAILURE_RETRIES_SETTING));
ClusterService clusterService = mock(ClusterService.class);
when(clusterService.getClusterSettings()).thenReturn(cSettings);
when(clusterService.state()).thenReturn(TransformInternalIndexTests.STATE_WITH_LATEST_VERSIONED_INDEX_TEMPLATE);
TransformPersistentTasksExecutor executor = new TransformPersistentTasksExecutor(client,
TransformPersistentTasksExecutor executor = new TransformPersistentTasksExecutor(
client,
transformsConfigManager,
transformCheckpointService, mock(SchedulerEngine.class),
transformCheckpointService,
mock(SchedulerEngine.class),
new TransformAuditor(client, ""),
mock(ThreadPool.class),
clusterService,
Settings.EMPTY);
Settings.EMPTY
);
// old-data-node-1 prevents assignment
assertNull(executor.getAssignment(new TransformTaskParams("new-task-id", Version.CURRENT, null), cs).getExecutorNode());
// remove the old 7.2 node
nodes = DiscoveryNodes.builder()
.add(new DiscoveryNode("current-data-node-with-1-task",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.CURRENT))
.add(new DiscoveryNode("non-data-node-1",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
Collections.singleton(DiscoveryNodeRole.MASTER_ROLE),
Version.CURRENT));
nodes = DiscoveryNodes
.builder()
.add(
new DiscoveryNode(
"current-data-node-with-1-task",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)),
Version.CURRENT
)
)
.add(
new DiscoveryNode(
"non-data-node-1",
buildNewFakeTransportAddress(),
Collections.emptyMap(),
Collections.singleton(DiscoveryNodeRole.MASTER_ROLE),
Version.CURRENT
)
);
csBuilder.nodes(nodes);
cs = csBuilder.build();
assertThat(executor.getAssignment(new TransformTaskParams("new-old-task-id", Version.V_7_2_0, null), cs).getExecutorNode(),
equalTo("current-data-node-with-1-task"));
assertThat(
executor.getAssignment(new TransformTaskParams("new-old-task-id", Version.V_7_2_0, null), cs).getExecutorNode(),
equalTo("current-data-node-with-1-task")
);
}
public void testVerifyIndicesPrimaryShardsAreActive() {
@ -220,11 +280,20 @@ public class TransformPersistentTasksExecutorTests extends ESTestCase {
} else {
Index index = new Index(indexToRemove, "_uuid");
ShardId shardId = new ShardId(index, 0);
ShardRouting shardRouting = ShardRouting.newUnassigned(shardId, true, RecoverySource.EmptyStoreRecoverySource.INSTANCE,
new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, ""));
ShardRouting shardRouting = ShardRouting
.newUnassigned(
shardId,
true,
RecoverySource.EmptyStoreRecoverySource.INSTANCE,
new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "")
);
shardRouting = shardRouting.initialize("node_id", null, 0L);
routingTable.add(IndexRoutingTable.builder(index)
.addIndexShard(new IndexShardRoutingTable.Builder(shardId).addShard(shardRouting).build()));
routingTable
.add(
IndexRoutingTable
.builder(index)
.addIndexShard(new IndexShardRoutingTable.Builder(shardId).addShard(shardRouting).build())
);
}
csBuilder.routingTable(routingTable.build());
@ -240,20 +309,32 @@ public class TransformPersistentTasksExecutorTests extends ESTestCase {
indices.add(TransformInternalIndexConstants.LATEST_INDEX_NAME);
for (String indexName : indices) {
IndexMetaData.Builder indexMetaData = IndexMetaData.builder(indexName);
indexMetaData.settings(Settings.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
);
indexMetaData
.settings(
Settings
.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
);
metaData.put(indexMetaData);
Index index = new Index(indexName, "_uuid");
ShardId shardId = new ShardId(index, 0);
ShardRouting shardRouting = ShardRouting.newUnassigned(shardId, true, RecoverySource.EmptyStoreRecoverySource.INSTANCE,
new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, ""));
ShardRouting shardRouting = ShardRouting
.newUnassigned(
shardId,
true,
RecoverySource.EmptyStoreRecoverySource.INSTANCE,
new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "")
);
shardRouting = shardRouting.initialize("node_id", null, 0L);
shardRouting = shardRouting.moveToStarted();
routingTable.add(IndexRoutingTable.builder(index)
.addIndexShard(new IndexShardRoutingTable.Builder(shardId).addShard(shardRouting).build()));
routingTable
.add(
IndexRoutingTable
.builder(index)
.addIndexShard(new IndexShardRoutingTable.Builder(shardId).addShard(shardRouting).build())
);
}
}