|
@@ -22,7 +22,6 @@ import org.elasticsearch.action.support.tasks.TransportTasksAction;
|
|
|
import org.elasticsearch.client.Client;
|
|
|
import org.elasticsearch.cluster.ClusterState;
|
|
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
|
|
-import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
|
|
import org.elasticsearch.cluster.service.ClusterService;
|
|
|
import org.elasticsearch.common.inject.Inject;
|
|
@@ -30,8 +29,6 @@ import org.elasticsearch.common.logging.LoggerMessageFormat;
|
|
|
import org.elasticsearch.common.settings.Settings;
|
|
|
import org.elasticsearch.discovery.MasterNotDiscoveredException;
|
|
|
import org.elasticsearch.ingest.IngestService;
|
|
|
-import org.elasticsearch.license.License;
|
|
|
-import org.elasticsearch.license.RemoteClusterLicenseChecker;
|
|
|
import org.elasticsearch.license.XPackLicenseState;
|
|
|
import org.elasticsearch.persistent.PersistentTasksCustomMetadata;
|
|
|
import org.elasticsearch.rest.RestStatus;
|
|
@@ -41,7 +38,6 @@ import org.elasticsearch.transport.TransportService;
|
|
|
import org.elasticsearch.xpack.core.ClientHelper;
|
|
|
import org.elasticsearch.xpack.core.XPackPlugin;
|
|
|
import org.elasticsearch.xpack.core.XPackSettings;
|
|
|
-import org.elasticsearch.xpack.core.common.validation.SourceDestValidator;
|
|
|
import org.elasticsearch.xpack.core.security.SecurityContext;
|
|
|
import org.elasticsearch.xpack.core.security.action.user.HasPrivilegesAction;
|
|
|
import org.elasticsearch.xpack.core.security.action.user.HasPrivilegesRequest;
|
|
@@ -52,6 +48,7 @@ import org.elasticsearch.xpack.core.transform.TransformMessages;
|
|
|
import org.elasticsearch.xpack.core.transform.action.UpdateTransformAction;
|
|
|
import org.elasticsearch.xpack.core.transform.action.UpdateTransformAction.Request;
|
|
|
import org.elasticsearch.xpack.core.transform.action.UpdateTransformAction.Response;
|
|
|
+import org.elasticsearch.xpack.core.transform.action.ValidateTransformAction;
|
|
|
import org.elasticsearch.xpack.core.transform.transforms.TransformConfig;
|
|
|
import org.elasticsearch.xpack.core.transform.transforms.TransformConfigUpdate;
|
|
|
import org.elasticsearch.xpack.core.transform.transforms.TransformDestIndexSettings;
|
|
@@ -66,7 +63,6 @@ import org.elasticsearch.xpack.transform.persistence.TransformIndex;
|
|
|
import org.elasticsearch.xpack.transform.transforms.Function;
|
|
|
import org.elasticsearch.xpack.transform.transforms.FunctionFactory;
|
|
|
import org.elasticsearch.xpack.transform.transforms.TransformTask;
|
|
|
-import org.elasticsearch.xpack.transform.utils.SourceDestValidations;
|
|
|
|
|
|
import java.time.Clock;
|
|
|
import java.util.List;
|
|
@@ -83,7 +79,6 @@ public class TransportUpdateTransformAction extends TransportTasksAction<Transfo
|
|
|
private final TransformConfigManager transformConfigManager;
|
|
|
private final SecurityContext securityContext;
|
|
|
private final TransformAuditor auditor;
|
|
|
- private final SourceDestValidator sourceDestValidator;
|
|
|
private final ThreadPool threadPool;
|
|
|
private final IndexNameExpressionResolver indexNameExpressionResolver;
|
|
|
|
|
@@ -146,16 +141,6 @@ public class TransportUpdateTransformAction extends TransportTasksAction<Transfo
|
|
|
? new SecurityContext(settings, threadPool.getThreadContext())
|
|
|
: null;
|
|
|
this.auditor = transformServices.getAuditor();
|
|
|
- this.sourceDestValidator = new SourceDestValidator(
|
|
|
- indexNameExpressionResolver,
|
|
|
- transportService.getRemoteClusterService(),
|
|
|
- DiscoveryNode.isRemoteClusterClient(settings)
|
|
|
- /* transforms are BASIC so always allowed, no need to check license */
|
|
|
- ? new RemoteClusterLicenseChecker(client, mode -> true) : null,
|
|
|
- ingestService,
|
|
|
- clusterService.getNodeName(),
|
|
|
- License.OperationMode.BASIC.description()
|
|
|
- );
|
|
|
this.threadPool = threadPool;
|
|
|
this.indexNameExpressionResolver = indexNameExpressionResolver;
|
|
|
}
|
|
@@ -181,7 +166,6 @@ public class TransportUpdateTransformAction extends TransportTasksAction<Transfo
|
|
|
}
|
|
|
return;
|
|
|
}
|
|
|
- TransformNodes.warnIfNoTransformNodes(clusterState);
|
|
|
|
|
|
// set headers to run transform as calling user
|
|
|
Map<String, String> filteredHeaders = ClientHelper.filterSecurityHeaders(threadPool.getThreadContext().getHeaders());
|
|
@@ -235,15 +219,18 @@ public class TransportUpdateTransformAction extends TransportTasksAction<Transfo
|
|
|
updateListener = listener;
|
|
|
}
|
|
|
|
|
|
- sourceDestValidator.validate(
|
|
|
- clusterState,
|
|
|
- updatedConfig.getSource().getIndex(),
|
|
|
- updatedConfig.getDestination().getIndex(),
|
|
|
- updatedConfig.getDestination().getPipeline(),
|
|
|
- SourceDestValidations.getValidations(request.isDeferValidation(), config.getAdditionalValidations()),
|
|
|
+ client.execute(
|
|
|
+ ValidateTransformAction.INSTANCE,
|
|
|
+ new ValidateTransformAction.Request(updatedConfig, request.isDeferValidation()),
|
|
|
ActionListener.wrap(
|
|
|
validationResponse -> {
|
|
|
- checkPriviledgesAndUpdateTransform(request, clusterState, updatedConfig, configAndVersion.v2(), updateListener);
|
|
|
+ checkPriviledgesAndUpdateTransform(
|
|
|
+ request,
|
|
|
+ clusterState,
|
|
|
+ updatedConfig,
|
|
|
+ validationResponse.getDestIndexMappings(),
|
|
|
+ configAndVersion.v2(),
|
|
|
+ updateListener);
|
|
|
},
|
|
|
listener::onFailure
|
|
|
)
|
|
@@ -274,13 +261,14 @@ public class TransportUpdateTransformAction extends TransportTasksAction<Transfo
|
|
|
String username,
|
|
|
Request request,
|
|
|
TransformConfig config,
|
|
|
+ Map<String, String> mappings,
|
|
|
SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex,
|
|
|
ClusterState clusterState,
|
|
|
HasPrivilegesResponse privilegesResponse,
|
|
|
ActionListener<Response> listener
|
|
|
) {
|
|
|
if (privilegesResponse.isCompleteMatch()) {
|
|
|
- updateTransform(request, config, seqNoPrimaryTermAndIndex, clusterState, listener);
|
|
|
+ updateTransform(request, config, mappings, seqNoPrimaryTermAndIndex, clusterState, listener);
|
|
|
} else {
|
|
|
List<String> indices = privilegesResponse.getIndexPrivileges()
|
|
|
.stream()
|
|
@@ -302,6 +290,7 @@ public class TransportUpdateTransformAction extends TransportTasksAction<Transfo
|
|
|
Request request,
|
|
|
ClusterState clusterState,
|
|
|
TransformConfig config,
|
|
|
+ Map<String, String> mappings,
|
|
|
SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex,
|
|
|
ActionListener<Response> listener
|
|
|
) {
|
|
@@ -310,19 +299,20 @@ public class TransportUpdateTransformAction extends TransportTasksAction<Transfo
|
|
|
final String username = securityContext.getUser().principal();
|
|
|
HasPrivilegesRequest privRequest = buildPrivilegeCheck(config, indexNameExpressionResolver, clusterState, username);
|
|
|
ActionListener<HasPrivilegesResponse> privResponseListener = ActionListener.wrap(
|
|
|
- r -> handlePrivsResponse(username, request, config, seqNoPrimaryTermAndIndex, clusterState, r, listener),
|
|
|
+ r -> handlePrivsResponse(username, request, config, mappings, seqNoPrimaryTermAndIndex, clusterState, r, listener),
|
|
|
listener::onFailure
|
|
|
);
|
|
|
|
|
|
client.execute(HasPrivilegesAction.INSTANCE, privRequest, privResponseListener);
|
|
|
} else { // No security enabled, just create the transform
|
|
|
- updateTransform(request, config, seqNoPrimaryTermAndIndex, clusterState, listener);
|
|
|
+ updateTransform(request, config, mappings, seqNoPrimaryTermAndIndex, clusterState, listener);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private void updateTransform(
|
|
|
Request request,
|
|
|
TransformConfig config,
|
|
|
+ Map<String, String> mappings,
|
|
|
SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex,
|
|
|
ClusterState clusterState,
|
|
|
ActionListener<Response> listener
|
|
@@ -351,7 +341,7 @@ public class TransportUpdateTransformAction extends TransportTasksAction<Transfo
|
|
|
);
|
|
|
|
|
|
// <2> Update our transform
|
|
|
- ActionListener<Void> createDestinationListener = ActionListener.wrap(
|
|
|
+ ActionListener<Boolean> createDestinationListener = ActionListener.wrap(
|
|
|
createDestResponse -> transformConfigManager.updateTransformConfiguration(
|
|
|
config,
|
|
|
seqNoPrimaryTermAndIndex,
|
|
@@ -379,7 +369,7 @@ public class TransportUpdateTransformAction extends TransportTasksAction<Transfo
|
|
|
// we allow source indices to disappear. If the source and destination indices do not exist, don't do anything
|
|
|
// the transform will just have to dynamically create the destination index without special mapping.
|
|
|
&& src.length > 0) {
|
|
|
- createDestination(function, config, createDestinationListener);
|
|
|
+ createDestinationIndex(config, mappings, createDestinationListener);
|
|
|
} else {
|
|
|
createDestinationListener.onResponse(null);
|
|
|
}
|
|
@@ -403,36 +393,15 @@ public class TransportUpdateTransformAction extends TransportTasksAction<Transfo
|
|
|
}
|
|
|
});
|
|
|
|
|
|
- function.validateConfig(ActionListener.wrap(r2 -> {
|
|
|
- if (request.isDeferValidation()) {
|
|
|
- functionValidationListener.onResponse(true);
|
|
|
- } else {
|
|
|
- function.validateQuery(client, config.getSource(), functionValidationListener);
|
|
|
- }
|
|
|
- }, listener::onFailure));
|
|
|
+ functionValidationListener.onResponse(true);
|
|
|
}
|
|
|
|
|
|
- private void createDestination(Function function, TransformConfig config, ActionListener<Void> listener) {
|
|
|
- ActionListener<Map<String, String>> deduceMappingsListener = ActionListener.wrap(mappings -> {
|
|
|
- TransformDestIndexSettings generateddestIndexSettings = TransformIndex.createTransformDestIndexSettings(
|
|
|
- mappings,
|
|
|
- config.getId(),
|
|
|
- Clock.systemUTC()
|
|
|
- );
|
|
|
- TransformIndex.createDestinationIndex(
|
|
|
- client,
|
|
|
- config,
|
|
|
- generateddestIndexSettings,
|
|
|
- ActionListener.wrap(r -> listener.onResponse(null), listener::onFailure)
|
|
|
- );
|
|
|
- },
|
|
|
-
|
|
|
- deduceTargetMappingsException -> listener.onFailure(
|
|
|
- new RuntimeException(TransformMessages.REST_PUT_TRANSFORM_FAILED_TO_DEDUCE_DEST_MAPPINGS, deduceTargetMappingsException)
|
|
|
- )
|
|
|
+ private void createDestinationIndex(TransformConfig config, Map<String, String> mappings, ActionListener<Boolean> listener) {
|
|
|
+ TransformDestIndexSettings generatedDestIndexSettings = TransformIndex.createTransformDestIndexSettings(
|
|
|
+ mappings,
|
|
|
+ config.getId(),
|
|
|
+ Clock.systemUTC()
|
|
|
);
|
|
|
-
|
|
|
- function.deduceMappings(client, config.getSource(), deduceMappingsListener);
|
|
|
+ TransformIndex.createDestinationIndex(client, config, generatedDestIndexSettings, listener);
|
|
|
}
|
|
|
-
|
|
|
}
|