Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add limit on number of processors in Ingest pipelines #15465

Merged
merged 18 commits into from
Sep 3, 2024
Merged
Show file tree
Hide file tree
Changes from 8 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/),
- Add support for centralize snapshot creation with pinned timestamp ([#15124](https://github.com/opensearch-project/OpenSearch/pull/15124))
- Add concurrent search support for Derived Fields ([#15326](https://github.com/opensearch-project/OpenSearch/pull/15326))
- [Workload Management] Add query group stats constructs ([#15343](https://github.com/opensearch-project/OpenSearch/pull/15343)))
- Add limit on number of processors in Ingest pipelines([#15460](https://github.com/opensearch-project/OpenSearch/pull/15465)).
- Add runAs to Subject interface and introduce IdentityAwarePlugin extension point ([#14630](https://github.com/opensearch-project/OpenSearch/pull/14630))

### Dependencies
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,8 @@
return;
}

ingestService.validateProcessorCountForIngestPipeline(simulateRequest.getPipeline());

Check warning on line 91 in server/src/main/java/org/opensearch/action/ingest/SimulatePipelineTransportAction.java

View check run for this annotation

Codecov / codecov/patch

server/src/main/java/org/opensearch/action/ingest/SimulatePipelineTransportAction.java#L91

Added line #L91 was not covered by tests
anandkrrai marked this conversation as resolved.
Show resolved Hide resolved

executionService.execute(simulateRequest, listener);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,7 @@
import org.opensearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.opensearch.indices.recovery.RecoverySettings;
import org.opensearch.indices.store.IndicesStore;
import org.opensearch.ingest.IngestService;
import org.opensearch.monitor.fs.FsHealthService;
import org.opensearch.monitor.fs.FsService;
import org.opensearch.monitor.jvm.JvmGcMonitorService;
Expand Down Expand Up @@ -405,6 +406,7 @@ public void apply(Settings value, Settings current, Settings previous) {
ClusterService.USER_DEFINED_METADATA,
ClusterManagerService.MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, // deprecated
ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING,
IngestService.MAX_NUMBER_OF_INGEST_PROCESSORS,
SearchService.DEFAULT_SEARCH_TIMEOUT_SETTING,
SearchService.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS,
TransportSearchAction.SHARD_COUNT_LIMIT_SETTING,
Expand Down
37 changes: 37 additions & 0 deletions server/src/main/java/org/opensearch/ingest/IngestService.java
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@
import org.opensearch.common.collect.Tuple;
import org.opensearch.common.metrics.OperationMetrics;
import org.opensearch.common.regex.Regex;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.concurrent.AbstractRunnable;
Expand Down Expand Up @@ -107,6 +108,18 @@ public class IngestService implements ClusterStateApplier, ReportingService<Inge

public static final String INGEST_ORIGIN = "ingest";

/**
* Defines the limit for the number of processors which can run on a given document during ingestion.
*/
public static final Setting<Integer> MAX_NUMBER_OF_INGEST_PROCESSORS = Setting.intSetting(
"cluster.ingest.max_number_processors",
Integer.MAX_VALUE,
anandkrrai marked this conversation as resolved.
Show resolved Hide resolved
1,
Integer.MAX_VALUE,
Setting.Property.NodeScope,
Setting.Property.Dynamic
);

private static final Logger logger = LogManager.getLogger(IngestService.class);

private final ClusterService clusterService;
Expand All @@ -123,6 +136,7 @@ public class IngestService implements ClusterStateApplier, ReportingService<Inge
private final ClusterManagerTaskThrottler.ThrottlingKey putPipelineTaskKey;
private final ClusterManagerTaskThrottler.ThrottlingKey deletePipelineTaskKey;
private volatile ClusterState state;
private volatile int maxIngestProcessorCount;

public IngestService(
ClusterService clusterService,
Expand Down Expand Up @@ -156,6 +170,12 @@ public IngestService(
// Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction.
putPipelineTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.PUT_PIPELINE_KEY, true);
deletePipelineTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.DELETE_PIPELINE_KEY, true);
clusterService.getClusterSettings().addSettingsUpdateConsumer(MAX_NUMBER_OF_INGEST_PROCESSORS, this::setMaxIngestProcessorCount);
setMaxIngestProcessorCount(clusterService.getClusterSettings().get(MAX_NUMBER_OF_INGEST_PROCESSORS));
}

private void setMaxIngestProcessorCount(Integer maxIngestProcessorCount) {
this.maxIngestProcessorCount = maxIngestProcessorCount;
}

private static Map<String, Processor.Factory> processorFactories(List<IngestPlugin> ingestPlugins, Processor.Parameters parameters) {
Expand Down Expand Up @@ -494,6 +514,9 @@ void validatePipeline(Map<DiscoveryNode, IngestInfo> ingestInfos, PutPipelineReq

Map<String, Object> pipelineConfig = XContentHelper.convertToMap(request.getSource(), false, request.getMediaType()).v2();
Pipeline pipeline = Pipeline.create(request.getId(), pipelineConfig, processorFactories, scriptService);

validateProcessorCountForIngestPipeline(pipeline);

List<Exception> exceptions = new ArrayList<>();
for (Processor processor : pipeline.flattenAllProcessors()) {
for (Map.Entry<DiscoveryNode, IngestInfo> entry : ingestInfos.entrySet()) {
Expand All @@ -507,6 +530,20 @@ void validatePipeline(Map<DiscoveryNode, IngestInfo> ingestInfos, PutPipelineReq
ExceptionsHelper.rethrowAndSuppress(exceptions);
}

public void validateProcessorCountForIngestPipeline(Pipeline pipeline) {
List<Processor> processors = pipeline.getCompoundProcessor().getProcessors();
anandkrrai marked this conversation as resolved.
Show resolved Hide resolved

if (processors.size() > maxIngestProcessorCount) {
throw new IllegalStateException(
"Cannot use more than the maximum processors allowed. Number of processors being configured is ["
+ processors.size()
+ "] which exceeds the maximum allowed configuration of ["
+ maxIngestProcessorCount
+ "] processors."
);
}
}

public void executeBulkRequest(
int numberOfActionRequests,
Iterable<DocWriteRequest<?>> actionRequests,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.SetOnce;
import org.opensearch.common.metrics.OperationStats;
import org.opensearch.common.settings.ClusterSettings;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.util.concurrent.OpenSearchExecutors;
import org.opensearch.common.xcontent.XContentType;
Expand Down Expand Up @@ -151,8 +152,12 @@ public void setup() {

public void testIngestPlugin() {
Client client = mock(Client.class);
ClusterService clusterService = mock(ClusterService.class);
when(clusterService.getClusterSettings()).thenReturn(
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
);
IngestService ingestService = new IngestService(
mock(ClusterService.class),
clusterService,
threadPool,
null,
null,
Expand Down Expand Up @@ -186,8 +191,12 @@ public void testIngestPluginDuplicate() {

public void testExecuteIndexPipelineDoesNotExist() {
Client client = mock(Client.class);
ClusterService clusterService = mock(ClusterService.class);
when(clusterService.getClusterSettings()).thenReturn(
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
);
IngestService ingestService = new IngestService(
mock(ClusterService.class),
clusterService,
threadPool,
null,
null,
Expand Down Expand Up @@ -720,6 +729,29 @@ public void testValidate() throws Exception {
ingestService.validatePipeline(ingestInfos, putRequest);
}

public void testValidateProcessorCountForIngestPipelineThrowsException() {
IngestService ingestService = createWithProcessors();
PutPipelineRequest putRequest = new PutPipelineRequest(
"_id",
new BytesArray(
"{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\", \"tag\": \"tag1\"}},"
+ "{\"remove\" : {\"field\": \"_field\", \"tag\": \"tag2\"}}]}"
),
MediaTypeRegistry.JSON
);

DiscoveryNode node1 = new DiscoveryNode("_node_id1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
DiscoveryNode node2 = new DiscoveryNode("_node_id2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
Map<DiscoveryNode, IngestInfo> ingestInfos = new HashMap<>();
ingestInfos.put(node1, new IngestInfo(Arrays.asList(new ProcessorInfo("set"), new ProcessorInfo("remove"))));
ingestInfos.put(node2, new IngestInfo(Arrays.asList(new ProcessorInfo("set"))));

Settings newSettings = Settings.builder().put("cluster.ingest.max_number_processors", 1).build();
ingestService.getClusterService().getClusterSettings().applySettings(newSettings);

IllegalStateException e = expectThrows(IllegalStateException.class, () -> ingestService.validatePipeline(ingestInfos, putRequest));
}

public void testExecuteIndexPipelineExistsButFailedParsing() {
IngestService ingestService = createWithProcessors(
Collections.singletonMap("mock", (factories, tag, description, config) -> new AbstractProcessor("mock", "description") {
Expand Down Expand Up @@ -1506,8 +1538,12 @@ public Map<String, Processor.Factory> getProcessors(Processor.Parameters paramet

// Create ingest service:
Client client = mock(Client.class);
ClusterService clusterService = mock(ClusterService.class);
when(clusterService.getClusterSettings()).thenReturn(
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
);
IngestService ingestService = new IngestService(
mock(ClusterService.class),
clusterService,
threadPool,
null,
null,
Expand Down Expand Up @@ -2058,6 +2094,18 @@ public void testPrepareBatches_different_index_pipeline() {
assertEquals(4, batches.size());
}

public void testUpdateMaxIngestProcessorCountSetting() {
anandkrrai marked this conversation as resolved.
Show resolved Hide resolved
ClusterSettings clusterSettings = new ClusterSettings(Settings.builder().build(), ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);

// verify defaults
assertEquals(Integer.MAX_VALUE, clusterSettings.get(IngestService.MAX_NUMBER_OF_INGEST_PROCESSORS).intValue());

// verify update max processor
Settings newSettings = Settings.builder().put("cluster.ingest.max_number_processors", 3).build();
clusterSettings.applySettings(newSettings);
assertEquals(3, clusterSettings.get(IngestService.MAX_NUMBER_OF_INGEST_PROCESSORS).intValue());
}

private IngestService.IndexRequestWrapper createIndexRequestWrapper(String index, List<String> pipelines) {
IndexRequest indexRequest = new IndexRequest(index);
return new IngestService.IndexRequestWrapper(0, indexRequest, pipelines, true);
Expand Down Expand Up @@ -2093,7 +2141,11 @@ private static IngestService createWithProcessors(Map<String, Processor.Factory>
ExecutorService executorService = OpenSearchExecutors.newDirectExecutorService();
when(threadPool.generic()).thenReturn(executorService);
when(threadPool.executor(anyString())).thenReturn(executorService);
return new IngestService(mock(ClusterService.class), threadPool, null, null, null, Collections.singletonList(new IngestPlugin() {
ClusterService clusterService = mock(ClusterService.class);
when(clusterService.getClusterSettings()).thenReturn(
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
);
return new IngestService(clusterService, threadPool, null, null, null, Collections.singletonList(new IngestPlugin() {
@Override
public Map<String, Processor.Factory> getProcessors(final Processor.Parameters parameters) {
return processors;
Expand Down
Loading