Skip to content

[7.x] Updating ingest pipeline without changes is no-op (#78196) #78756

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

Merged
Merged
Show file tree
Hide file tree
Changes from all 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
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,10 @@
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.ingest.IngestInfo;
import org.elasticsearch.ingest.IngestMetadata;
import org.elasticsearch.ingest.IngestService;
import org.elasticsearch.ingest.Pipeline;
import org.elasticsearch.ingest.PipelineConfiguration;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;

Expand Down Expand Up @@ -57,22 +59,43 @@ public PutPipelineTransportAction(ThreadPool threadPool, TransportService transp
@Override
protected void masterOperation(PutPipelineRequest request, ClusterState state, ActionListener<AcknowledgedResponse> listener)
throws Exception {

Map<String, Object> pipelineConfig = null;
IngestMetadata currentIngestMetadata = state.metadata().custom(IngestMetadata.TYPE);
if (currentIngestMetadata != null && currentIngestMetadata.getPipelines().containsKey(request.getId())) {
pipelineConfig = XContentHelper.convertToMap(request.getSource(), false, request.getXContentType()).v2();
PipelineConfiguration currentPipeline = currentIngestMetadata.getPipelines().get(request.getId());
if (currentPipeline.getConfigAsMap().equals(pipelineConfig)) {
// existing pipeline matches request pipeline -- no need to update
listener.onResponse(AcknowledgedResponse.TRUE);
return;
}
}

if (state.getNodes().getMinNodeVersion().before(Version.V_7_15_0)) {
Map<String, Object> pipelineConfig = XContentHelper.convertToMap(request.getSource(), false, request.getXContentType()).v2();
pipelineConfig = pipelineConfig == null
? XContentHelper.convertToMap(request.getSource(), false, request.getXContentType()).v2()
: pipelineConfig;
if (pipelineConfig.containsKey(Pipeline.META_KEY)) {
throw new IllegalStateException("pipelines with _meta field require minimum node version of " + Version.V_7_15_0);
}
}
NodesInfoRequest nodesInfoRequest = new NodesInfoRequest();
nodesInfoRequest.clear()
.addMetric(NodesInfoRequest.Metric.INGEST.metricName());
client.admin().cluster().nodesInfo(nodesInfoRequest, ActionListener.wrap(nodeInfos -> {
Map<DiscoveryNode, IngestInfo> ingestInfos = new HashMap<>();
for (NodeInfo nodeInfo : nodeInfos.getNodes()) {
ingestInfos.put(nodeInfo.getNode(), nodeInfo.getInfo(IngestInfo.class));
}
ingestService.putPipeline(ingestInfos, request, listener);
}, listener::onFailure));
nodesInfoRequest.clear();
nodesInfoRequest.addMetric(NodesInfoRequest.Metric.INGEST.metricName());
client.admin().cluster().nodesInfo(
nodesInfoRequest,
ActionListener.wrap(
nodeInfos -> {
Map<DiscoveryNode, IngestInfo> ingestInfos = new HashMap<>();
for (NodeInfo nodeInfo : nodeInfos.getNodes()) {
ingestInfos.put(nodeInfo.getNode(), nodeInfo.getInfo(IngestInfo.class));
}
ingestService.putPipeline(ingestInfos, request, listener);
},
listener::onFailure
)
);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/

package org.elasticsearch.action.ingest;

import org.elasticsearch.core.Tuple;
import org.elasticsearch.test.ESTestCase;

import static org.elasticsearch.core.Tuple.tuple;

public class PutPipelineTransportActionTests extends ESTestCase {


private static Tuple<String, Object> randomMapEntry() {
return tuple(randomAlphaOfLength(5), randomObject());
}

private static Object randomObject() {
return randomFrom(
random(),
ESTestCase::randomLong,
() -> generateRandomStringArray(10, 5, true),
() -> randomMap(3, 5, PutPipelineTransportActionTests::randomMapEntry),
() -> randomAlphaOfLength(5),
ESTestCase::randomTimeValue,
ESTestCase::randomDouble
);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -781,6 +781,14 @@ public static <T> T randomFrom(Random random, T... array) {
return RandomPicks.randomFrom(random, array);
}

/** Pick a random object from the given array of suppliers. The array must not be empty. */
@SafeVarargs
@SuppressWarnings("varargs")
public static <T> T randomFrom(Random random, Supplier<T>... array) {
Supplier<T> supplier = RandomPicks.randomFrom(random, array);
return supplier.get();
}

/** Pick a random object from the given list. */
public static <T> T randomFrom(List<T> list) {
return RandomPicks.randomFrom(random(), list);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@

import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Supplier;

/**
Expand All @@ -37,6 +38,8 @@
*/
public class NoOpNodeClient extends NodeClient {

private final AtomicLong executionCount = new AtomicLong(0);

/**
* Build with {@link ThreadPool}. This {@linkplain ThreadPool} is terminated on {@link #close()}.
*/
Expand All @@ -54,6 +57,7 @@ public NoOpNodeClient(String testName) {
@Override
public <Request extends ActionRequest, Response extends ActionResponse>
void doExecute(ActionType<Response> action, Request request, ActionListener<Response> listener) {
executionCount.incrementAndGet();
listener.onResponse(null);
}

Expand All @@ -70,13 +74,15 @@ public void initialize(
@Override
public <Request extends ActionRequest, Response extends ActionResponse>
Task executeLocally(ActionType<Response> action, Request request, ActionListener<Response> listener) {
executionCount.incrementAndGet();
listener.onResponse(null);
return null;
}

@Override
public <Request extends ActionRequest, Response extends ActionResponse>
Task executeLocally(ActionType<Response> action, Request request, TaskListener<Response> listener) {
executionCount.incrementAndGet();
listener.onResponse(null, null);
return null;
}
Expand All @@ -99,4 +105,8 @@ public void close() {
throw new ElasticsearchException(e.getMessage(), e);
}
}

public long getExecutionCount() {
return executionCount.get();
}
}