mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-06-28 01:22:26 -04:00
Make ESQL join operators project-aware
This commit is contained in:
parent
bc515c4070
commit
24329bbe64
9 changed files with 79 additions and 41 deletions
|
@ -24,6 +24,8 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
||||||
import org.elasticsearch.cluster.NodeConnectionsService;
|
import org.elasticsearch.cluster.NodeConnectionsService;
|
||||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||||
import org.elasticsearch.cluster.coordination.ClusterStatePublisher;
|
import org.elasticsearch.cluster.coordination.ClusterStatePublisher;
|
||||||
|
import org.elasticsearch.cluster.metadata.ProjectId;
|
||||||
|
import org.elasticsearch.cluster.metadata.ProjectMetadata;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNodeUtils;
|
import org.elasticsearch.cluster.node.DiscoveryNodeUtils;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||||
|
@ -138,6 +140,16 @@ public class ClusterServiceUtils {
|
||||||
DiscoveryNode localNode,
|
DiscoveryNode localNode,
|
||||||
Settings providedSettings,
|
Settings providedSettings,
|
||||||
ClusterSettings clusterSettings
|
ClusterSettings clusterSettings
|
||||||
|
) {
|
||||||
|
return createClusterService(threadPool, localNode, providedSettings, clusterSettings, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static ClusterService createClusterService(
|
||||||
|
ThreadPool threadPool,
|
||||||
|
DiscoveryNode localNode,
|
||||||
|
Settings providedSettings,
|
||||||
|
ClusterSettings clusterSettings,
|
||||||
|
ProjectId projectId
|
||||||
) {
|
) {
|
||||||
Settings settings = Settings.builder()
|
Settings settings = Settings.builder()
|
||||||
.put("node.name", "test")
|
.put("node.name", "test")
|
||||||
|
@ -151,12 +163,14 @@ public class ClusterServiceUtils {
|
||||||
new TaskManager(settings, threadPool, Collections.emptySet(), Tracer.NOOP)
|
new TaskManager(settings, threadPool, Collections.emptySet(), Tracer.NOOP)
|
||||||
);
|
);
|
||||||
clusterService.setNodeConnectionsService(createNoOpNodeConnectionsService());
|
clusterService.setNodeConnectionsService(createNoOpNodeConnectionsService());
|
||||||
ClusterState initialClusterState = ClusterState.builder(new ClusterName(ClusterServiceUtils.class.getSimpleName()))
|
ClusterState.Builder builder = ClusterState.builder(new ClusterName(ClusterServiceUtils.class.getSimpleName()))
|
||||||
.nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).masterNodeId(localNode.getId()))
|
.nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).masterNodeId(localNode.getId()))
|
||||||
.putCompatibilityVersions(localNode.getId(), CompatibilityVersionsUtils.staticCurrent())
|
.putCompatibilityVersions(localNode.getId(), CompatibilityVersionsUtils.staticCurrent())
|
||||||
.blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK)
|
.blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK);
|
||||||
.build();
|
if (projectId != null) {
|
||||||
clusterService.getClusterApplierService().setInitialState(initialClusterState);
|
builder.putProjectMetadata(ProjectMetadata.builder(projectId));
|
||||||
|
}
|
||||||
|
clusterService.getClusterApplierService().setInitialState(builder.build());
|
||||||
clusterService.getMasterService().setClusterStatePublisher(createClusterStatePublisher(clusterService.getClusterApplierService()));
|
clusterService.getMasterService().setClusterStatePublisher(createClusterStatePublisher(clusterService.getClusterApplierService()));
|
||||||
clusterService.getMasterService().setClusterStateSupplier(clusterService.getClusterApplierService()::state);
|
clusterService.getMasterService().setClusterStateSupplier(clusterService.getClusterApplierService()::state);
|
||||||
clusterService.start();
|
clusterService.start();
|
||||||
|
|
|
@ -15,8 +15,8 @@ import org.elasticsearch.action.support.ChannelActionListener;
|
||||||
import org.elasticsearch.action.support.IndicesOptions;
|
import org.elasticsearch.action.support.IndicesOptions;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
import org.elasticsearch.cluster.metadata.ProjectMetadata;
|
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
|
import org.elasticsearch.cluster.project.ProjectResolver;
|
||||||
import org.elasticsearch.cluster.routing.ShardIterator;
|
import org.elasticsearch.cluster.routing.ShardIterator;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
@ -134,6 +134,7 @@ public abstract class AbstractLookupService<R extends AbstractLookupService.Requ
|
||||||
private final BigArrays bigArrays;
|
private final BigArrays bigArrays;
|
||||||
private final BlockFactory blockFactory;
|
private final BlockFactory blockFactory;
|
||||||
private final LocalCircuitBreaker.SizeSettings localBreakerSettings;
|
private final LocalCircuitBreaker.SizeSettings localBreakerSettings;
|
||||||
|
private final ProjectResolver projectResolver;
|
||||||
/**
|
/**
|
||||||
* Should output {@link Page pages} be combined into a single resulting page?
|
* Should output {@link Page pages} be combined into a single resulting page?
|
||||||
* If this is {@code true} we'll run a {@link MergePositionsOperator} to merge
|
* If this is {@code true} we'll run a {@link MergePositionsOperator} to merge
|
||||||
|
@ -154,7 +155,8 @@ public abstract class AbstractLookupService<R extends AbstractLookupService.Requ
|
||||||
BigArrays bigArrays,
|
BigArrays bigArrays,
|
||||||
BlockFactory blockFactory,
|
BlockFactory blockFactory,
|
||||||
boolean mergePages,
|
boolean mergePages,
|
||||||
CheckedBiFunction<StreamInput, BlockFactory, T, IOException> readRequest
|
CheckedBiFunction<StreamInput, BlockFactory, T, IOException> readRequest,
|
||||||
|
ProjectResolver projectResolver
|
||||||
) {
|
) {
|
||||||
this.actionName = actionName;
|
this.actionName = actionName;
|
||||||
this.clusterService = clusterService;
|
this.clusterService = clusterService;
|
||||||
|
@ -167,6 +169,7 @@ public abstract class AbstractLookupService<R extends AbstractLookupService.Requ
|
||||||
this.blockFactory = blockFactory;
|
this.blockFactory = blockFactory;
|
||||||
this.localBreakerSettings = new LocalCircuitBreaker.SizeSettings(clusterService.getSettings());
|
this.localBreakerSettings = new LocalCircuitBreaker.SizeSettings(clusterService.getSettings());
|
||||||
this.mergePages = mergePages;
|
this.mergePages = mergePages;
|
||||||
|
this.projectResolver = projectResolver;
|
||||||
transportService.registerRequestHandler(
|
transportService.registerRequestHandler(
|
||||||
actionName,
|
actionName,
|
||||||
transportService.getThreadPool().executor(EsqlPlugin.ESQL_WORKER_THREAD_POOL_NAME),
|
transportService.getThreadPool().executor(EsqlPlugin.ESQL_WORKER_THREAD_POOL_NAME),
|
||||||
|
@ -227,8 +230,9 @@ public abstract class AbstractLookupService<R extends AbstractLookupService.Requ
|
||||||
*/
|
*/
|
||||||
public final void lookupAsync(R request, CancellableTask parentTask, ActionListener<List<Page>> outListener) {
|
public final void lookupAsync(R request, CancellableTask parentTask, ActionListener<List<Page>> outListener) {
|
||||||
ClusterState clusterState = clusterService.state();
|
ClusterState clusterState = clusterService.state();
|
||||||
|
var projectState = projectResolver.getProjectState(clusterState);
|
||||||
List<ShardIterator> shardIterators = clusterService.operationRouting()
|
List<ShardIterator> shardIterators = clusterService.operationRouting()
|
||||||
.searchShards(clusterState.projectState(), new String[] { request.index }, Map.of(), "_local");
|
.searchShards(projectState, new String[] { request.index }, Map.of(), "_local");
|
||||||
if (shardIterators.size() != 1) {
|
if (shardIterators.size() != 1) {
|
||||||
outListener.onFailure(new EsqlIllegalArgumentException("target index {} has more than one shard", request.index));
|
outListener.onFailure(new EsqlIllegalArgumentException("target index {} has more than one shard", request.index));
|
||||||
return;
|
return;
|
||||||
|
@ -278,12 +282,11 @@ public abstract class AbstractLookupService<R extends AbstractLookupService.Requ
|
||||||
final List<Releasable> releasables = new ArrayList<>(6);
|
final List<Releasable> releasables = new ArrayList<>(6);
|
||||||
boolean started = false;
|
boolean started = false;
|
||||||
try {
|
try {
|
||||||
|
var projectState = projectResolver.getProjectState(clusterService.state());
|
||||||
ProjectMetadata projMeta = clusterService.state().metadata().getProject();
|
|
||||||
AliasFilter aliasFilter = indicesService.buildAliasFilter(
|
AliasFilter aliasFilter = indicesService.buildAliasFilter(
|
||||||
clusterService.state().projectState(),
|
projectState,
|
||||||
request.shardId.getIndex().getName(),
|
request.shardId.getIndex().getName(),
|
||||||
indexNameExpressionResolver.resolveExpressions(projMeta, request.indexPattern)
|
indexNameExpressionResolver.resolveExpressions(projectState.metadata(), request.indexPattern)
|
||||||
);
|
);
|
||||||
|
|
||||||
LookupShardContext shardContext = lookupShardContextFactory.create(request.shardId);
|
LookupShardContext shardContext = lookupShardContextFactory.create(request.shardId);
|
||||||
|
|
|
@ -13,6 +13,7 @@ import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||||
import org.elasticsearch.action.support.ContextPreservingActionListener;
|
import org.elasticsearch.action.support.ContextPreservingActionListener;
|
||||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
|
import org.elasticsearch.cluster.project.ProjectResolver;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
@ -75,7 +76,8 @@ public class EnrichLookupService extends AbstractLookupService<EnrichLookupServi
|
||||||
TransportService transportService,
|
TransportService transportService,
|
||||||
IndexNameExpressionResolver indexNameExpressionResolver,
|
IndexNameExpressionResolver indexNameExpressionResolver,
|
||||||
BigArrays bigArrays,
|
BigArrays bigArrays,
|
||||||
BlockFactory blockFactory
|
BlockFactory blockFactory,
|
||||||
|
ProjectResolver projectResolver
|
||||||
) {
|
) {
|
||||||
super(
|
super(
|
||||||
LOOKUP_ACTION_NAME,
|
LOOKUP_ACTION_NAME,
|
||||||
|
@ -87,7 +89,8 @@ public class EnrichLookupService extends AbstractLookupService<EnrichLookupServi
|
||||||
bigArrays,
|
bigArrays,
|
||||||
blockFactory,
|
blockFactory,
|
||||||
true,
|
true,
|
||||||
TransportRequest::readFrom
|
TransportRequest::readFrom,
|
||||||
|
projectResolver
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -13,6 +13,7 @@ import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||||
import org.elasticsearch.action.support.ChannelActionListener;
|
import org.elasticsearch.action.support.ChannelActionListener;
|
||||||
import org.elasticsearch.action.support.ContextPreservingActionListener;
|
import org.elasticsearch.action.support.ContextPreservingActionListener;
|
||||||
import org.elasticsearch.action.support.RefCountingListener;
|
import org.elasticsearch.action.support.RefCountingListener;
|
||||||
|
import org.elasticsearch.cluster.project.ProjectResolver;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
@ -77,13 +78,20 @@ public class EnrichPolicyResolver {
|
||||||
private final TransportService transportService;
|
private final TransportService transportService;
|
||||||
private final ThreadPool threadPool;
|
private final ThreadPool threadPool;
|
||||||
private final RemoteClusterService remoteClusterService;
|
private final RemoteClusterService remoteClusterService;
|
||||||
|
private final ProjectResolver projectResolver;
|
||||||
|
|
||||||
public EnrichPolicyResolver(ClusterService clusterService, TransportService transportService, IndexResolver indexResolver) {
|
public EnrichPolicyResolver(
|
||||||
|
ClusterService clusterService,
|
||||||
|
TransportService transportService,
|
||||||
|
IndexResolver indexResolver,
|
||||||
|
ProjectResolver projectResolver
|
||||||
|
) {
|
||||||
this.clusterService = clusterService;
|
this.clusterService = clusterService;
|
||||||
this.transportService = transportService;
|
this.transportService = transportService;
|
||||||
this.indexResolver = indexResolver;
|
this.indexResolver = indexResolver;
|
||||||
this.threadPool = transportService.getThreadPool();
|
this.threadPool = transportService.getThreadPool();
|
||||||
this.remoteClusterService = transportService.getRemoteClusterService();
|
this.remoteClusterService = transportService.getRemoteClusterService();
|
||||||
|
this.projectResolver = projectResolver;
|
||||||
transportService.registerRequestHandler(
|
transportService.registerRequestHandler(
|
||||||
RESOLVE_ACTION_NAME,
|
RESOLVE_ACTION_NAME,
|
||||||
threadPool.executor(ThreadPool.Names.SEARCH),
|
threadPool.executor(ThreadPool.Names.SEARCH),
|
||||||
|
@ -445,7 +453,8 @@ public class EnrichPolicyResolver {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected Map<String, EnrichPolicy> availablePolicies() {
|
protected Map<String, EnrichPolicy> availablePolicies() {
|
||||||
final EnrichMetadata metadata = clusterService.state().metadata().getProject().custom(EnrichMetadata.TYPE, EnrichMetadata.EMPTY);
|
final EnrichMetadata metadata = projectResolver.getProjectMetadata(clusterService.state())
|
||||||
|
.custom(EnrichMetadata.TYPE, EnrichMetadata.EMPTY);
|
||||||
return metadata.getPolicies();
|
return metadata.getPolicies();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -9,6 +9,7 @@ package org.elasticsearch.xpack.esql.enrich;
|
||||||
|
|
||||||
import org.elasticsearch.TransportVersions;
|
import org.elasticsearch.TransportVersions;
|
||||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
|
import org.elasticsearch.cluster.project.ProjectResolver;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.collect.Iterators;
|
import org.elasticsearch.common.collect.Iterators;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
@ -55,7 +56,8 @@ public class LookupFromIndexService extends AbstractLookupService<LookupFromInde
|
||||||
TransportService transportService,
|
TransportService transportService,
|
||||||
IndexNameExpressionResolver indexNameExpressionResolver,
|
IndexNameExpressionResolver indexNameExpressionResolver,
|
||||||
BigArrays bigArrays,
|
BigArrays bigArrays,
|
||||||
BlockFactory blockFactory
|
BlockFactory blockFactory,
|
||||||
|
ProjectResolver projectResolver
|
||||||
) {
|
) {
|
||||||
super(
|
super(
|
||||||
LOOKUP_ACTION_NAME,
|
LOOKUP_ACTION_NAME,
|
||||||
|
@ -67,7 +69,8 @@ public class LookupFromIndexService extends AbstractLookupService<LookupFromInde
|
||||||
bigArrays,
|
bigArrays,
|
||||||
blockFactory,
|
blockFactory,
|
||||||
false,
|
false,
|
||||||
TransportRequest::readFrom
|
TransportRequest::readFrom,
|
||||||
|
projectResolver
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -113,7 +113,12 @@ public class TransportEsqlQueryAction extends HandledTransportAction<EsqlQueryRe
|
||||||
this.requestExecutor = threadPool.executor(ThreadPool.Names.SEARCH);
|
this.requestExecutor = threadPool.executor(ThreadPool.Names.SEARCH);
|
||||||
exchangeService.registerTransportHandler(transportService);
|
exchangeService.registerTransportHandler(transportService);
|
||||||
this.exchangeService = exchangeService;
|
this.exchangeService = exchangeService;
|
||||||
this.enrichPolicyResolver = new EnrichPolicyResolver(clusterService, transportService, planExecutor.indexResolver());
|
this.enrichPolicyResolver = new EnrichPolicyResolver(
|
||||||
|
clusterService,
|
||||||
|
transportService,
|
||||||
|
planExecutor.indexResolver(),
|
||||||
|
projectResolver
|
||||||
|
);
|
||||||
AbstractLookupService.LookupShardContextFactory lookupLookupShardContextFactory = AbstractLookupService.LookupShardContextFactory
|
AbstractLookupService.LookupShardContextFactory lookupLookupShardContextFactory = AbstractLookupService.LookupShardContextFactory
|
||||||
.fromSearchService(searchService);
|
.fromSearchService(searchService);
|
||||||
this.enrichLookupService = new EnrichLookupService(
|
this.enrichLookupService = new EnrichLookupService(
|
||||||
|
@ -123,7 +128,8 @@ public class TransportEsqlQueryAction extends HandledTransportAction<EsqlQueryRe
|
||||||
transportService,
|
transportService,
|
||||||
indexNameExpressionResolver,
|
indexNameExpressionResolver,
|
||||||
bigArrays,
|
bigArrays,
|
||||||
blockFactoryProvider.blockFactory()
|
blockFactoryProvider.blockFactory(),
|
||||||
|
projectResolver
|
||||||
);
|
);
|
||||||
this.lookupFromIndexService = new LookupFromIndexService(
|
this.lookupFromIndexService = new LookupFromIndexService(
|
||||||
clusterService,
|
clusterService,
|
||||||
|
@ -132,7 +138,8 @@ public class TransportEsqlQueryAction extends HandledTransportAction<EsqlQueryRe
|
||||||
transportService,
|
transportService,
|
||||||
indexNameExpressionResolver,
|
indexNameExpressionResolver,
|
||||||
bigArrays,
|
bigArrays,
|
||||||
blockFactoryProvider.blockFactory()
|
blockFactoryProvider.blockFactory(),
|
||||||
|
projectResolver
|
||||||
);
|
);
|
||||||
|
|
||||||
this.asyncTaskManagementService = new AsyncTaskManagementService<>(
|
this.asyncTaskManagementService = new AsyncTaskManagementService<>(
|
||||||
|
|
|
@ -22,8 +22,10 @@ import org.elasticsearch.action.support.PlainActionFuture;
|
||||||
import org.elasticsearch.client.internal.FilterClient;
|
import org.elasticsearch.client.internal.FilterClient;
|
||||||
import org.elasticsearch.cluster.ClusterName;
|
import org.elasticsearch.cluster.ClusterName;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.metadata.Metadata;
|
import org.elasticsearch.cluster.metadata.ProjectId;
|
||||||
|
import org.elasticsearch.cluster.metadata.ProjectMetadata;
|
||||||
import org.elasticsearch.cluster.node.VersionInformation;
|
import org.elasticsearch.cluster.node.VersionInformation;
|
||||||
|
import org.elasticsearch.cluster.project.TestProjectResolvers;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||||
|
@ -95,9 +97,10 @@ public class EnrichPolicyResolverTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
AbstractSimpleTransportTestCase.connectToNode(transports.get(""), transports.get("cluster_a").getLocalNode());
|
AbstractSimpleTransportTestCase.connectToNode(transports.get(""), transports.get("cluster_a").getLocalNode());
|
||||||
AbstractSimpleTransportTestCase.connectToNode(transports.get(""), transports.get("cluster_b").getLocalNode());
|
AbstractSimpleTransportTestCase.connectToNode(transports.get(""), transports.get("cluster_b").getLocalNode());
|
||||||
localCluster = newEnrichPolicyResolver(LOCAL_CLUSTER_GROUP_KEY);
|
final var projectId = randomProjectIdOrDefault();
|
||||||
clusterA = newEnrichPolicyResolver("cluster_a");
|
localCluster = newEnrichPolicyResolver(projectId, LOCAL_CLUSTER_GROUP_KEY);
|
||||||
clusterB = newEnrichPolicyResolver("cluster_b");
|
clusterA = newEnrichPolicyResolver(projectId, "cluster_a");
|
||||||
|
clusterB = newEnrichPolicyResolver(projectId, "cluster_b");
|
||||||
|
|
||||||
// hosts policies are the same across clusters
|
// hosts policies are the same across clusters
|
||||||
var hostsPolicy = new EnrichPolicy("match", null, List.of(), "ip", List.of("region", "cost"));
|
var hostsPolicy = new EnrichPolicy("match", null, List.of(), "ip", List.of("region", "cost"));
|
||||||
|
@ -401,8 +404,8 @@ public class EnrichPolicyResolverTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
TestEnrichPolicyResolver newEnrichPolicyResolver(String cluster) {
|
TestEnrichPolicyResolver newEnrichPolicyResolver(ProjectId projectId, String cluster) {
|
||||||
return new TestEnrichPolicyResolver(cluster, new HashMap<>(), new HashMap<>(), new HashMap<>());
|
return new TestEnrichPolicyResolver(projectId, cluster, new HashMap<>(), new HashMap<>(), new HashMap<>());
|
||||||
}
|
}
|
||||||
|
|
||||||
class TestEnrichPolicyResolver extends EnrichPolicyResolver {
|
class TestEnrichPolicyResolver extends EnrichPolicyResolver {
|
||||||
|
@ -412,15 +415,17 @@ public class EnrichPolicyResolverTests extends ESTestCase {
|
||||||
final Map<String, Map<String, String>> mappings;
|
final Map<String, Map<String, String>> mappings;
|
||||||
|
|
||||||
TestEnrichPolicyResolver(
|
TestEnrichPolicyResolver(
|
||||||
|
ProjectId projectId,
|
||||||
String cluster,
|
String cluster,
|
||||||
Map<String, EnrichPolicy> policies,
|
Map<String, EnrichPolicy> policies,
|
||||||
Map<String, String> aliases,
|
Map<String, String> aliases,
|
||||||
Map<String, Map<String, String>> mappings
|
Map<String, Map<String, String>> mappings
|
||||||
) {
|
) {
|
||||||
super(
|
super(
|
||||||
mockClusterService(policies),
|
mockClusterService(projectId, policies),
|
||||||
transports.get(cluster),
|
transports.get(cluster),
|
||||||
new IndexResolver(new FieldCapsClient(threadPool, aliases, mappings))
|
new IndexResolver(new FieldCapsClient(threadPool, aliases, mappings)),
|
||||||
|
TestProjectResolvers.singleProject(projectId)
|
||||||
);
|
);
|
||||||
this.policies = policies;
|
this.policies = policies;
|
||||||
this.cluster = cluster;
|
this.cluster = cluster;
|
||||||
|
@ -457,11 +462,11 @@ public class EnrichPolicyResolverTests extends ESTestCase {
|
||||||
listener.onResponse(transports.get("").getConnection(transports.get(remoteCluster).getLocalNode()));
|
listener.onResponse(transports.get("").getConnection(transports.get(remoteCluster).getLocalNode()));
|
||||||
}
|
}
|
||||||
|
|
||||||
static ClusterService mockClusterService(Map<String, EnrichPolicy> policies) {
|
static ClusterService mockClusterService(ProjectId projectId, Map<String, EnrichPolicy> policies) {
|
||||||
ClusterService clusterService = mock(ClusterService.class);
|
ClusterService clusterService = mock(ClusterService.class);
|
||||||
EnrichMetadata enrichMetadata = new EnrichMetadata(policies);
|
EnrichMetadata enrichMetadata = new EnrichMetadata(policies);
|
||||||
ClusterState state = ClusterState.builder(new ClusterName("test"))
|
ClusterState state = ClusterState.builder(new ClusterName("test"))
|
||||||
.metadata(Metadata.builder().projectCustoms(Map.of(EnrichMetadata.TYPE, enrichMetadata)))
|
.putProjectMetadata(ProjectMetadata.builder(projectId).customs(Map.of(EnrichMetadata.TYPE, enrichMetadata)))
|
||||||
.build();
|
.build();
|
||||||
when(clusterService.state()).thenReturn(state);
|
when(clusterService.state()).thenReturn(state);
|
||||||
return clusterService;
|
return clusterService;
|
||||||
|
|
|
@ -18,6 +18,7 @@ import org.elasticsearch.action.support.replication.ClusterStateCreationUtils;
|
||||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNodeUtils;
|
import org.elasticsearch.cluster.node.DiscoveryNodeUtils;
|
||||||
|
import org.elasticsearch.cluster.project.TestProjectResolvers;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.settings.ClusterSettings;
|
import org.elasticsearch.common.settings.ClusterSettings;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
@ -183,7 +184,8 @@ public class LookupFromIndexOperatorTests extends OperatorTestCase {
|
||||||
IndicesService indicesService = mock(IndicesService.class);
|
IndicesService indicesService = mock(IndicesService.class);
|
||||||
IndexNameExpressionResolver indexNameExpressionResolver = TestIndexNameExpressionResolver.newInstance();
|
IndexNameExpressionResolver indexNameExpressionResolver = TestIndexNameExpressionResolver.newInstance();
|
||||||
releasables.add(clusterService::stop);
|
releasables.add(clusterService::stop);
|
||||||
ClusterServiceUtils.setState(clusterService, ClusterStateCreationUtils.state("idx", 1, 1));
|
final var projectId = randomProjectIdOrDefault();
|
||||||
|
ClusterServiceUtils.setState(clusterService, ClusterStateCreationUtils.state(projectId, "idx", 1, 1));
|
||||||
if (beCranky) {
|
if (beCranky) {
|
||||||
logger.info("building a cranky lookup");
|
logger.info("building a cranky lookup");
|
||||||
}
|
}
|
||||||
|
@ -197,7 +199,8 @@ public class LookupFromIndexOperatorTests extends OperatorTestCase {
|
||||||
transportService(clusterService),
|
transportService(clusterService),
|
||||||
indexNameExpressionResolver,
|
indexNameExpressionResolver,
|
||||||
bigArrays,
|
bigArrays,
|
||||||
blockFactory
|
blockFactory,
|
||||||
|
TestProjectResolvers.singleProject(projectId)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -34,17 +34,8 @@ tasks.named("yamlRestTest").configure {
|
||||||
'^data_streams/10_data_stream_resolvability/*',
|
'^data_streams/10_data_stream_resolvability/*',
|
||||||
'^deprecation/10_basic/*',
|
'^deprecation/10_basic/*',
|
||||||
'^dlm/10_usage/*',
|
'^dlm/10_usage/*',
|
||||||
'^esql/60_enrich/*',
|
|
||||||
'^esql/60_usage/*',
|
'^esql/60_usage/*',
|
||||||
'^esql/61_enrich_ip/*',
|
|
||||||
'^esql/62_extra_enrich/*',
|
|
||||||
'^esql/63_enrich_int_range/*',
|
|
||||||
'^esql/64_enrich_int_match/*',
|
|
||||||
'^esql/180_match_operator/*',
|
'^esql/180_match_operator/*',
|
||||||
'^esql/190_lookup_join/*',
|
|
||||||
'^esql/191_lookup_join_on_datastreams/*',
|
|
||||||
'^esql/191_lookup_join_text/*',
|
|
||||||
'^esql/192_lookup_join_on_aliases/*',
|
|
||||||
'^health/10_usage/*',
|
'^health/10_usage/*',
|
||||||
'^ilm/60_operation_mode/*',
|
'^ilm/60_operation_mode/*',
|
||||||
'^ilm/80_health/*',
|
'^ilm/80_health/*',
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue