|  | @@ -8,7 +8,9 @@
 | 
	
		
			
				|  |  |  package org.elasticsearch.xpack.esql.session;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import org.apache.lucene.index.CorruptIndexException;
 | 
	
		
			
				|  |  | +import org.elasticsearch.ElasticsearchException;
 | 
	
		
			
				|  |  |  import org.elasticsearch.ElasticsearchStatusException;
 | 
	
		
			
				|  |  | +import org.elasticsearch.action.ActionListener;
 | 
	
		
			
				|  |  |  import org.elasticsearch.action.OriginalIndices;
 | 
	
		
			
				|  |  |  import org.elasticsearch.action.fieldcaps.FieldCapabilitiesFailure;
 | 
	
		
			
				|  |  |  import org.elasticsearch.action.search.ShardSearchFailure;
 | 
	
	
		
			
				|  | @@ -20,6 +22,7 @@ import org.elasticsearch.license.License;
 | 
	
		
			
				|  |  |  import org.elasticsearch.license.XPackLicenseState;
 | 
	
		
			
				|  |  |  import org.elasticsearch.license.internal.XPackLicenseStatus;
 | 
	
		
			
				|  |  |  import org.elasticsearch.rest.RestStatus;
 | 
	
		
			
				|  |  | +import org.elasticsearch.tasks.TaskCancelledException;
 | 
	
		
			
				|  |  |  import org.elasticsearch.test.ESTestCase;
 | 
	
		
			
				|  |  |  import org.elasticsearch.transport.ConnectTransportException;
 | 
	
		
			
				|  |  |  import org.elasticsearch.transport.NoSeedNodeLeftException;
 | 
	
	
		
			
				|  | @@ -47,26 +50,30 @@ import java.util.function.Predicate;
 | 
	
		
			
				|  |  |  import java.util.stream.Collectors;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import static org.elasticsearch.xpack.esql.core.tree.Source.EMPTY;
 | 
	
		
			
				|  |  | -import static org.elasticsearch.xpack.esql.session.EsqlSessionCCSUtils.checkForCcsLicense;
 | 
	
		
			
				|  |  | +import static org.elasticsearch.xpack.esql.session.EsqlCCSUtils.checkForCcsLicense;
 | 
	
		
			
				|  |  | +import static org.elasticsearch.xpack.esql.session.EsqlCCSUtils.shouldIgnoreRuntimeError;
 | 
	
		
			
				|  |  | +import static org.elasticsearch.xpack.esql.session.EsqlCCSUtils.skipUnavailableListener;
 | 
	
		
			
				|  |  |  import static org.hamcrest.Matchers.containsString;
 | 
	
		
			
				|  |  |  import static org.hamcrest.Matchers.equalTo;
 | 
	
		
			
				|  |  |  import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 | 
	
		
			
				|  |  | +import static org.hamcrest.Matchers.is;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  | +public class EsqlCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    private final String LOCAL_CLUSTER_ALIAS = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY;
 | 
	
		
			
				|  |  | +    private final String REMOTE1_ALIAS = "remote1";
 | 
	
		
			
				|  |  | +    private final String REMOTE2_ALIAS = "remote2";
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      public void testCreateIndexExpressionFromAvailableClusters() {
 | 
	
		
			
				|  |  | -        final String localClusterAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY;
 | 
	
		
			
				|  |  | -        final String remote1Alias = "remote1";
 | 
	
		
			
				|  |  | -        final String remote2Alias = "remote2";
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          // no clusters marked as skipped
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*", true));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote2Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote2Alias, "mylogs1,mylogs2,logs*", true));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", true));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE2_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "mylogs1,mylogs2,logs*", true));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            String indexExpr = EsqlSessionCCSUtils.createIndexExpressionFromAvailableClusters(executionInfo);
 | 
	
		
			
				|  |  | +            String indexExpr = EsqlCCSUtils.createIndexExpressionFromAvailableClusters(executionInfo);
 | 
	
		
			
				|  |  |              List<String> list = Arrays.stream(Strings.splitStringByCommaToArray(indexExpr)).toList();
 | 
	
		
			
				|  |  |              assertThat(list.size(), equalTo(5));
 | 
	
		
			
				|  |  |              assertThat(
 | 
	
	
		
			
				|  | @@ -78,19 +85,19 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          // one cluster marked as skipped, so not present in revised index expression
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*,foo", true));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*,foo", true));
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(
 | 
	
		
			
				|  |  | -                remote2Alias,
 | 
	
		
			
				|  |  | +                REMOTE2_ALIAS,
 | 
	
		
			
				|  |  |                  (k, v) -> new EsqlExecutionInfo.Cluster(
 | 
	
		
			
				|  |  | -                    remote2Alias,
 | 
	
		
			
				|  |  | +                    REMOTE2_ALIAS,
 | 
	
		
			
				|  |  |                      "mylogs1,mylogs2,logs*",
 | 
	
		
			
				|  |  |                      true,
 | 
	
		
			
				|  |  |                      EsqlExecutionInfo.Cluster.Status.SKIPPED
 | 
	
		
			
				|  |  |                  )
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            String indexExpr = EsqlSessionCCSUtils.createIndexExpressionFromAvailableClusters(executionInfo);
 | 
	
		
			
				|  |  | +            String indexExpr = EsqlCCSUtils.createIndexExpressionFromAvailableClusters(executionInfo);
 | 
	
		
			
				|  |  |              List<String> list = Arrays.stream(Strings.splitStringByCommaToArray(indexExpr)).toList();
 | 
	
		
			
				|  |  |              assertThat(list.size(), equalTo(3));
 | 
	
		
			
				|  |  |              assertThat(new HashSet<>(list), equalTo(Strings.commaDelimitedListToSet("logs*,remote1:*,remote1:foo")));
 | 
	
	
		
			
				|  | @@ -99,73 +106,70 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          // two clusters marked as skipped, so only local cluster present in revised index expression
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(
 | 
	
		
			
				|  |  | -                remote1Alias,
 | 
	
		
			
				|  |  | -                (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*,foo", true, EsqlExecutionInfo.Cluster.Status.SKIPPED)
 | 
	
		
			
				|  |  | +                REMOTE1_ALIAS,
 | 
	
		
			
				|  |  | +                (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*,foo", true, EsqlExecutionInfo.Cluster.Status.SKIPPED)
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(
 | 
	
		
			
				|  |  | -                remote2Alias,
 | 
	
		
			
				|  |  | +                REMOTE2_ALIAS,
 | 
	
		
			
				|  |  |                  (k, v) -> new EsqlExecutionInfo.Cluster(
 | 
	
		
			
				|  |  | -                    remote2Alias,
 | 
	
		
			
				|  |  | +                    REMOTE2_ALIAS,
 | 
	
		
			
				|  |  |                      "mylogs1,mylogs2,logs*",
 | 
	
		
			
				|  |  |                      true,
 | 
	
		
			
				|  |  |                      EsqlExecutionInfo.Cluster.Status.SKIPPED
 | 
	
		
			
				|  |  |                  )
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            assertThat(EsqlSessionCCSUtils.createIndexExpressionFromAvailableClusters(executionInfo), equalTo("logs*"));
 | 
	
		
			
				|  |  | +            assertThat(EsqlCCSUtils.createIndexExpressionFromAvailableClusters(executionInfo), equalTo("logs*"));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          // only remotes present and all marked as skipped, so in revised index expression should be empty string
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(
 | 
	
		
			
				|  |  | -                remote1Alias,
 | 
	
		
			
				|  |  | -                (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*,foo", true, EsqlExecutionInfo.Cluster.Status.SKIPPED)
 | 
	
		
			
				|  |  | +                REMOTE1_ALIAS,
 | 
	
		
			
				|  |  | +                (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*,foo", true, EsqlExecutionInfo.Cluster.Status.SKIPPED)
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(
 | 
	
		
			
				|  |  | -                remote2Alias,
 | 
	
		
			
				|  |  | +                REMOTE2_ALIAS,
 | 
	
		
			
				|  |  |                  (k, v) -> new EsqlExecutionInfo.Cluster(
 | 
	
		
			
				|  |  | -                    remote2Alias,
 | 
	
		
			
				|  |  | +                    REMOTE2_ALIAS,
 | 
	
		
			
				|  |  |                      "mylogs1,mylogs2,logs*",
 | 
	
		
			
				|  |  |                      true,
 | 
	
		
			
				|  |  |                      EsqlExecutionInfo.Cluster.Status.SKIPPED
 | 
	
		
			
				|  |  |                  )
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            assertThat(EsqlSessionCCSUtils.createIndexExpressionFromAvailableClusters(executionInfo), equalTo(""));
 | 
	
		
			
				|  |  | +            assertThat(EsqlCCSUtils.createIndexExpressionFromAvailableClusters(executionInfo), equalTo(""));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      public void testUpdateExecutionInfoWithUnavailableClusters() {
 | 
	
		
			
				|  |  | -        final String localClusterAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY;
 | 
	
		
			
				|  |  | -        final String remote1Alias = "remote1";
 | 
	
		
			
				|  |  | -        final String remote2Alias = "remote2";
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          // skip_unavailable=true clusters are unavailable, both marked as SKIPPED
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*", true));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote2Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote2Alias, "mylogs1,mylogs2,logs*", true));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", true));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE2_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "mylogs1,mylogs2,logs*", true));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              var failure = new FieldCapabilitiesFailure(new String[] { "logs-a" }, new NoSeedNodeLeftException("unable to connect"));
 | 
	
		
			
				|  |  | -            var unvailableClusters = Map.of(remote1Alias, failure, remote2Alias, failure);
 | 
	
		
			
				|  |  | -            EsqlSessionCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, unvailableClusters);
 | 
	
		
			
				|  |  | +            var unvailableClusters = Map.of(REMOTE1_ALIAS, failure, REMOTE2_ALIAS, failure);
 | 
	
		
			
				|  |  | +            EsqlCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, unvailableClusters);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            assertThat(executionInfo.clusterAliases(), equalTo(Set.of(localClusterAlias, remote1Alias, remote2Alias)));
 | 
	
		
			
				|  |  | +            assertThat(executionInfo.clusterAliases(), equalTo(Set.of(LOCAL_CLUSTER_ALIAS, REMOTE1_ALIAS, REMOTE2_ALIAS)));
 | 
	
		
			
				|  |  |              assertNull(executionInfo.overallTook());
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(localClusterAlias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(localCluster.getIndexExpression(), equalTo("logs*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(localCluster, EsqlExecutionInfo.Cluster.Status.RUNNING);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(remote1Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(REMOTE1_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getIndexExpression(), equalTo("*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(remote1Cluster, EsqlExecutionInfo.Cluster.Status.SKIPPED);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(remote2Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(REMOTE2_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote2Cluster.getIndexExpression(), equalTo("mylogs1,mylogs2,logs*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(remote2Cluster, EsqlExecutionInfo.Cluster.Status.SKIPPED);
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -173,14 +177,17 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          // skip_unavailable=false cluster is unavailable, throws Exception
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*", true));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote2Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote2Alias, "mylogs1,mylogs2,logs*", false));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", true));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(
 | 
	
		
			
				|  |  | +                REMOTE2_ALIAS,
 | 
	
		
			
				|  |  | +                (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "mylogs1,mylogs2,logs*", false)
 | 
	
		
			
				|  |  | +            );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              var failure = new FieldCapabilitiesFailure(new String[] { "logs-a" }, new NoSeedNodeLeftException("unable to connect"));
 | 
	
		
			
				|  |  |              RemoteTransportException e = expectThrows(
 | 
	
		
			
				|  |  |                  RemoteTransportException.class,
 | 
	
		
			
				|  |  | -                () -> EsqlSessionCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, Map.of(remote2Alias, failure))
 | 
	
		
			
				|  |  | +                () -> EsqlCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, Map.of(REMOTE2_ALIAS, failure))
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |              assertThat(e.status().getStatus(), equalTo(500));
 | 
	
		
			
				|  |  |              assertThat(
 | 
	
	
		
			
				|  | @@ -193,42 +200,42 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          // all clusters available, no Clusters in ExecutionInfo should be modified
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*", true));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote2Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote2Alias, "mylogs1,mylogs2,logs*", false));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", true));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(
 | 
	
		
			
				|  |  | +                REMOTE2_ALIAS,
 | 
	
		
			
				|  |  | +                (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "mylogs1,mylogs2,logs*", false)
 | 
	
		
			
				|  |  | +            );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlSessionCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, Map.of());
 | 
	
		
			
				|  |  | +            EsqlCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, Map.of());
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            assertThat(executionInfo.clusterAliases(), equalTo(Set.of(localClusterAlias, remote1Alias, remote2Alias)));
 | 
	
		
			
				|  |  | +            assertThat(executionInfo.clusterAliases(), equalTo(Set.of(LOCAL_CLUSTER_ALIAS, REMOTE1_ALIAS, REMOTE2_ALIAS)));
 | 
	
		
			
				|  |  |              assertNull(executionInfo.overallTook());
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(localClusterAlias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(localCluster.getIndexExpression(), equalTo("logs*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(localCluster, EsqlExecutionInfo.Cluster.Status.RUNNING);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(remote1Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(REMOTE1_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getIndexExpression(), equalTo("*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(remote1Cluster, EsqlExecutionInfo.Cluster.Status.RUNNING);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(remote2Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(REMOTE2_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote2Cluster.getIndexExpression(), equalTo("mylogs1,mylogs2,logs*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(remote2Cluster, EsqlExecutionInfo.Cluster.Status.RUNNING);
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      public void testUpdateExecutionInfoWithClustersWithNoMatchingIndices() {
 | 
	
		
			
				|  |  | -        final String localClusterAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY;
 | 
	
		
			
				|  |  | -        final String remote1Alias = "remote1";
 | 
	
		
			
				|  |  | -        final String remote2Alias = "remote2";
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          // all clusters had matching indices from field-caps call, so no updates to EsqlExecutionInfo should happen
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*", randomBoolean()));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", randomBoolean()));
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(
 | 
	
		
			
				|  |  | -                remote2Alias,
 | 
	
		
			
				|  |  | -                (k, v) -> new EsqlExecutionInfo.Cluster(remote2Alias, "mylogs1,mylogs2,logs*", randomBoolean())
 | 
	
		
			
				|  |  | +                REMOTE2_ALIAS,
 | 
	
		
			
				|  |  | +                (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "mylogs1,mylogs2,logs*", randomBoolean())
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              EsIndex esIndex = new EsIndex(
 | 
	
	
		
			
				|  | @@ -251,17 +258,17 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              IndexResolution indexResolution = IndexResolution.valid(esIndex, esIndex.concreteIndices(), Map.of());
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlSessionCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution);
 | 
	
		
			
				|  |  | +            EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(localClusterAlias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(localCluster.getIndexExpression(), equalTo("logs*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(localCluster, EsqlExecutionInfo.Cluster.Status.RUNNING);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(remote1Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(REMOTE1_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getIndexExpression(), equalTo("*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(remote1Cluster, EsqlExecutionInfo.Cluster.Status.RUNNING);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(remote2Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(REMOTE2_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote2Cluster.getIndexExpression(), equalTo("mylogs1,mylogs2,logs*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(remote2Cluster, EsqlExecutionInfo.Cluster.Status.RUNNING);
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -270,11 +277,11 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          // marked as SKIPPED with 0 total shards, 0 took time, etc.
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*", randomBoolean()));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", randomBoolean()));
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(
 | 
	
		
			
				|  |  | -                remote2Alias,
 | 
	
		
			
				|  |  | -                (k, v) -> new EsqlExecutionInfo.Cluster(remote2Alias, "mylogs1,mylogs2,logs*", randomBoolean())
 | 
	
		
			
				|  |  | +                REMOTE2_ALIAS,
 | 
	
		
			
				|  |  | +                (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "mylogs1,mylogs2,logs*", randomBoolean())
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              EsIndex esIndex = new EsIndex(
 | 
	
	
		
			
				|  | @@ -295,13 +302,13 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |              Map<String, FieldCapabilitiesFailure> unavailableClusters = Map.of();
 | 
	
		
			
				|  |  |              IndexResolution indexResolution = IndexResolution.valid(esIndex, esIndex.concreteIndices(), unavailableClusters);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlSessionCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution);
 | 
	
		
			
				|  |  | +            EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(localClusterAlias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(localCluster.getIndexExpression(), equalTo("logs*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(localCluster, EsqlExecutionInfo.Cluster.Status.RUNNING);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(remote1Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(REMOTE1_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getIndexExpression(), equalTo("*"));
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SKIPPED));
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getTook().millis(), equalTo(0L));
 | 
	
	
		
			
				|  | @@ -310,7 +317,7 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getSkippedShards(), equalTo(0));
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getFailedShards(), equalTo(0));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(remote2Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(REMOTE2_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote2Cluster.getIndexExpression(), equalTo("mylogs1,mylogs2,logs*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(remote2Cluster, EsqlExecutionInfo.Cluster.Status.RUNNING);
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -320,11 +327,11 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          // marked as SKIPPED
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*", randomBoolean()));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", randomBoolean()));
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(
 | 
	
		
			
				|  |  | -                remote2Alias,
 | 
	
		
			
				|  |  | -                (k, v) -> new EsqlExecutionInfo.Cluster(remote2Alias, "mylogs1*,mylogs2*,logs*", randomBoolean())
 | 
	
		
			
				|  |  | +                REMOTE2_ALIAS,
 | 
	
		
			
				|  |  | +                (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "mylogs1*,mylogs2*,logs*", randomBoolean())
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              EsIndex esIndex = new EsIndex(
 | 
	
	
		
			
				|  | @@ -334,22 +341,22 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |              // remote1 is unavailable
 | 
	
		
			
				|  |  |              var failure = new FieldCapabilitiesFailure(new String[] { "logs-a" }, new NoSeedNodeLeftException("unable to connect"));
 | 
	
		
			
				|  |  | -            Map<String, FieldCapabilitiesFailure> unavailableClusters = Map.of(remote1Alias, failure);
 | 
	
		
			
				|  |  | +            Map<String, FieldCapabilitiesFailure> unavailableClusters = Map.of(REMOTE1_ALIAS, failure);
 | 
	
		
			
				|  |  |              IndexResolution indexResolution = IndexResolution.valid(esIndex, esIndex.concreteIndices(), unavailableClusters);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlSessionCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution);
 | 
	
		
			
				|  |  | +            EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(localClusterAlias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(localCluster.getIndexExpression(), equalTo("logs*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(localCluster, EsqlExecutionInfo.Cluster.Status.RUNNING);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(remote1Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(REMOTE1_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getIndexExpression(), equalTo("*"));
 | 
	
		
			
				|  |  |              // since remote1 is in the unavailable Map (passed to IndexResolution.valid), it's status will not be changed
 | 
	
		
			
				|  |  |              // by updateExecutionInfoWithClustersWithNoMatchingIndices (it is handled in updateExecutionInfoWithUnavailableClusters)
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.RUNNING));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(remote2Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(REMOTE2_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote2Cluster.getIndexExpression(), equalTo("mylogs1*,mylogs2*,logs*"));
 | 
	
		
			
				|  |  |              assertThat(remote2Cluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SKIPPED));
 | 
	
		
			
				|  |  |              assertThat(remote2Cluster.getTook().millis(), equalTo(0L));
 | 
	
	
		
			
				|  | @@ -363,11 +370,11 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          // but had no matching indices and since a concrete index was requested, a VerificationException is thrown
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*"));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*", randomBoolean()));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*"));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", randomBoolean()));
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(
 | 
	
		
			
				|  |  | -                remote2Alias,
 | 
	
		
			
				|  |  | -                (k, v) -> new EsqlExecutionInfo.Cluster(remote2Alias, "mylogs1,mylogs2,logs*", randomBoolean())
 | 
	
		
			
				|  |  | +                REMOTE2_ALIAS,
 | 
	
		
			
				|  |  | +                (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "mylogs1,mylogs2,logs*", randomBoolean())
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              EsIndex esIndex = new EsIndex(
 | 
	
	
		
			
				|  | @@ -377,11 +384,11 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              var failure = new FieldCapabilitiesFailure(new String[] { "logs-a" }, new NoSeedNodeLeftException("unable to connect"));
 | 
	
		
			
				|  |  | -            Map<String, FieldCapabilitiesFailure> unavailableClusters = Map.of(remote1Alias, failure);
 | 
	
		
			
				|  |  | +            Map<String, FieldCapabilitiesFailure> unavailableClusters = Map.of(REMOTE1_ALIAS, failure);
 | 
	
		
			
				|  |  |              IndexResolution indexResolution = IndexResolution.valid(esIndex, esIndex.concreteIndices(), unavailableClusters);
 | 
	
		
			
				|  |  |              VerificationException ve = expectThrows(
 | 
	
		
			
				|  |  |                  VerificationException.class,
 | 
	
		
			
				|  |  | -                () -> EsqlSessionCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution)
 | 
	
		
			
				|  |  | +                () -> EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution)
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |              assertThat(ve.getDetailedMessage(), containsString("Unknown index [remote2:mylogs1,mylogs2,logs*]"));
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -390,13 +397,13 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          // (the EsqlSessionCCSUtils.updateExecutionInfoWithUnavailableClusters() method handles that case not the one tested here)
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*"));
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*", randomBoolean()));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*"));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", randomBoolean()));
 | 
	
		
			
				|  |  |              // remote2 is already marked as SKIPPED (simulating failed enrich policy lookup due to unavailable cluster)
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(
 | 
	
		
			
				|  |  | -                remote2Alias,
 | 
	
		
			
				|  |  | +                REMOTE2_ALIAS,
 | 
	
		
			
				|  |  |                  (k, v) -> new EsqlExecutionInfo.Cluster(
 | 
	
		
			
				|  |  | -                    remote2Alias,
 | 
	
		
			
				|  |  | +                    REMOTE2_ALIAS,
 | 
	
		
			
				|  |  |                      "mylogs1*,mylogs2*,logs*",
 | 
	
		
			
				|  |  |                      randomBoolean(),
 | 
	
		
			
				|  |  |                      EsqlExecutionInfo.Cluster.Status.SKIPPED
 | 
	
	
		
			
				|  | @@ -411,22 +418,22 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              // remote1 is unavailable
 | 
	
		
			
				|  |  |              var failure = new FieldCapabilitiesFailure(new String[] { "logs-a" }, new NoSeedNodeLeftException("unable to connect"));
 | 
	
		
			
				|  |  | -            Map<String, FieldCapabilitiesFailure> unavailableClusters = Map.of(remote1Alias, failure);
 | 
	
		
			
				|  |  | +            Map<String, FieldCapabilitiesFailure> unavailableClusters = Map.of(REMOTE1_ALIAS, failure);
 | 
	
		
			
				|  |  |              IndexResolution indexResolution = IndexResolution.valid(esIndex, esIndex.concreteIndices(), unavailableClusters);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlSessionCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution);
 | 
	
		
			
				|  |  | +            EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(localClusterAlias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(localCluster.getIndexExpression(), equalTo("logs*"));
 | 
	
		
			
				|  |  |              assertClusterStatusAndShardCounts(localCluster, EsqlExecutionInfo.Cluster.Status.RUNNING);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(remote1Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(REMOTE1_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getIndexExpression(), equalTo("*"));
 | 
	
		
			
				|  |  |              // since remote1 is in the unavailable Map (passed to IndexResolution.valid), it's status will not be changed
 | 
	
		
			
				|  |  |              // by updateExecutionInfoWithClustersWithNoMatchingIndices (it is handled in updateExecutionInfoWithUnavailableClusters)
 | 
	
		
			
				|  |  |              assertThat(remote1Cluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.RUNNING));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(remote2Alias);
 | 
	
		
			
				|  |  | +            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(REMOTE2_ALIAS);
 | 
	
		
			
				|  |  |              assertThat(remote2Cluster.getIndexExpression(), equalTo("mylogs1*,mylogs2*,logs*"));
 | 
	
		
			
				|  |  |              assertThat(remote2Cluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SKIPPED));
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -444,7 +451,7 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |                  )
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            Map<String, FieldCapabilitiesFailure> unavailableClusters = EsqlSessionCCSUtils.determineUnavailableRemoteClusters(failures);
 | 
	
		
			
				|  |  | +            Map<String, FieldCapabilitiesFailure> unavailableClusters = EsqlCCSUtils.determineUnavailableRemoteClusters(failures);
 | 
	
		
			
				|  |  |              assertThat(unavailableClusters.keySet(), equalTo(Set.of("remote1", "remote2")));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
	
		
			
				|  | @@ -454,7 +461,7 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |              failures.add(new FieldCapabilitiesFailure(new String[] { "remote2:mylogs1" }, new NoSuchRemoteClusterException("remote2")));
 | 
	
		
			
				|  |  |              failures.add(new FieldCapabilitiesFailure(new String[] { "remote2:mylogs1" }, new NoSeedNodeLeftException("no seed node")));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            Map<String, FieldCapabilitiesFailure> unavailableClusters = EsqlSessionCCSUtils.determineUnavailableRemoteClusters(failures);
 | 
	
		
			
				|  |  | +            Map<String, FieldCapabilitiesFailure> unavailableClusters = EsqlCCSUtils.determineUnavailableRemoteClusters(failures);
 | 
	
		
			
				|  |  |              assertThat(unavailableClusters.keySet(), equalTo(Set.of("remote2")));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
	
		
			
				|  | @@ -468,7 +475,7 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |                      new IllegalStateException("Unable to open any connections")
 | 
	
		
			
				|  |  |                  )
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  | -            Map<String, FieldCapabilitiesFailure> unavailableClusters = EsqlSessionCCSUtils.determineUnavailableRemoteClusters(failures);
 | 
	
		
			
				|  |  | +            Map<String, FieldCapabilitiesFailure> unavailableClusters = EsqlCCSUtils.determineUnavailableRemoteClusters(failures);
 | 
	
		
			
				|  |  |              assertThat(unavailableClusters.keySet(), equalTo(Set.of("remote2")));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
	
		
			
				|  | @@ -476,29 +483,28 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              List<FieldCapabilitiesFailure> failures = new ArrayList<>();
 | 
	
		
			
				|  |  |              failures.add(new FieldCapabilitiesFailure(new String[] { "remote1:mylogs1" }, new RuntimeException("foo")));
 | 
	
		
			
				|  |  | -            Map<String, FieldCapabilitiesFailure> unavailableClusters = EsqlSessionCCSUtils.determineUnavailableRemoteClusters(failures);
 | 
	
		
			
				|  |  | +            Map<String, FieldCapabilitiesFailure> unavailableClusters = EsqlCCSUtils.determineUnavailableRemoteClusters(failures);
 | 
	
		
			
				|  |  |              assertThat(unavailableClusters.keySet(), equalTo(Set.of()));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          // empty failures list
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              List<FieldCapabilitiesFailure> failures = new ArrayList<>();
 | 
	
		
			
				|  |  | -            Map<String, FieldCapabilitiesFailure> unavailableClusters = EsqlSessionCCSUtils.determineUnavailableRemoteClusters(failures);
 | 
	
		
			
				|  |  | +            Map<String, FieldCapabilitiesFailure> unavailableClusters = EsqlCCSUtils.determineUnavailableRemoteClusters(failures);
 | 
	
		
			
				|  |  |              assertThat(unavailableClusters.keySet(), equalTo(Set.of()));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      public void testUpdateExecutionInfoAtEndOfPlanning() {
 | 
	
		
			
				|  |  | -        String localClusterAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY;
 | 
	
		
			
				|  |  | -        String remote1Alias = "remote1";
 | 
	
		
			
				|  |  | -        String remote2Alias = "remote2";
 | 
	
		
			
				|  |  | +        String REMOTE1_ALIAS = "remote1";
 | 
	
		
			
				|  |  | +        String REMOTE2_ALIAS = "remote2";
 | 
	
		
			
				|  |  |          EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(true);
 | 
	
		
			
				|  |  | -        executionInfo.swapCluster(localClusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false));
 | 
	
		
			
				|  |  | +        executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  |          executionInfo.swapCluster(
 | 
	
		
			
				|  |  | -            remote1Alias,
 | 
	
		
			
				|  |  | -            (k, v) -> new EsqlExecutionInfo.Cluster(remote1Alias, "*", true, EsqlExecutionInfo.Cluster.Status.SKIPPED)
 | 
	
		
			
				|  |  | +            REMOTE1_ALIAS,
 | 
	
		
			
				|  |  | +            (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", true, EsqlExecutionInfo.Cluster.Status.SKIPPED)
 | 
	
		
			
				|  |  |          );
 | 
	
		
			
				|  |  | -        executionInfo.swapCluster(remote2Alias, (k, v) -> new EsqlExecutionInfo.Cluster(remote2Alias, "mylogs1,mylogs2,logs*", false));
 | 
	
		
			
				|  |  | +        executionInfo.swapCluster(REMOTE2_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "mylogs1,mylogs2,logs*", false));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          assertNull(executionInfo.planningTookTime());
 | 
	
		
			
				|  |  |          assertNull(executionInfo.overallTook());
 | 
	
	
		
			
				|  | @@ -506,7 +512,7 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |              Thread.sleep(1);
 | 
	
		
			
				|  |  |          } catch (InterruptedException e) {}
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        EsqlSessionCCSUtils.updateExecutionInfoAtEndOfPlanning(executionInfo);
 | 
	
		
			
				|  |  | +        EsqlCCSUtils.updateExecutionInfoAtEndOfPlanning(executionInfo);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          assertThat(executionInfo.planningTookTime().millis(), greaterThanOrEqualTo(0L));
 | 
	
		
			
				|  |  |          assertNull(executionInfo.overallTook());
 | 
	
	
		
			
				|  | @@ -517,7 +523,7 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          assertNull(localCluster.getTotalShards());
 | 
	
		
			
				|  |  |          assertNull(localCluster.getTook());
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(remote1Alias);
 | 
	
		
			
				|  |  | +        EsqlExecutionInfo.Cluster remote1Cluster = executionInfo.getCluster(REMOTE1_ALIAS);
 | 
	
		
			
				|  |  |          assertThat(remote1Cluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SKIPPED));
 | 
	
		
			
				|  |  |          assertThat(remote1Cluster.getTotalShards(), equalTo(0));
 | 
	
		
			
				|  |  |          assertThat(remote1Cluster.getSuccessfulShards(), equalTo(0));
 | 
	
	
		
			
				|  | @@ -526,7 +532,7 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          assertThat(remote1Cluster.getTook().millis(), greaterThanOrEqualTo(0L));
 | 
	
		
			
				|  |  |          assertThat(remote1Cluster.getTook().millis(), equalTo(executionInfo.planningTookTime().millis()));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(remote2Alias);
 | 
	
		
			
				|  |  | +        EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(REMOTE2_ALIAS);
 | 
	
		
			
				|  |  |          assertThat(remote2Cluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.RUNNING));
 | 
	
		
			
				|  |  |          assertNull(remote2Cluster.getTotalShards());
 | 
	
		
			
				|  |  |          assertNull(remote2Cluster.getTook());
 | 
	
	
		
			
				|  | @@ -534,7 +540,10 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      private void assertClusterStatusAndShardCounts(EsqlExecutionInfo.Cluster cluster, EsqlExecutionInfo.Cluster.Status status) {
 | 
	
		
			
				|  |  |          assertThat(cluster.getStatus(), equalTo(status));
 | 
	
		
			
				|  |  | -        assertNull(cluster.getTook());
 | 
	
		
			
				|  |  | +        if (cluster.getTook() != null) {
 | 
	
		
			
				|  |  | +            // It is also ok if it's null in some tests
 | 
	
		
			
				|  |  | +            assertThat(cluster.getTook().millis(), greaterThanOrEqualTo(0L));
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  |          if (status == EsqlExecutionInfo.Cluster.Status.RUNNING) {
 | 
	
		
			
				|  |  |              assertNull(cluster.getTotalShards());
 | 
	
		
			
				|  |  |              assertNull(cluster.getSuccessfulShards());
 | 
	
	
		
			
				|  | @@ -545,6 +554,11 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |              assertThat(cluster.getSuccessfulShards(), equalTo(0));
 | 
	
		
			
				|  |  |              assertThat(cluster.getSkippedShards(), equalTo(0));
 | 
	
		
			
				|  |  |              assertThat(cluster.getFailedShards(), equalTo(0));
 | 
	
		
			
				|  |  | +        } else if (status == EsqlExecutionInfo.Cluster.Status.PARTIAL) {
 | 
	
		
			
				|  |  | +            assertThat(cluster.getTotalShards(), equalTo(0));
 | 
	
		
			
				|  |  | +            assertThat(cluster.getSuccessfulShards(), equalTo(0));
 | 
	
		
			
				|  |  | +            assertThat(cluster.getSkippedShards(), equalTo(0));
 | 
	
		
			
				|  |  | +            assertThat(cluster.getFailedShards(), equalTo(0));
 | 
	
		
			
				|  |  |          } else {
 | 
	
		
			
				|  |  |              fail("Unexpected status: " + status);
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -560,35 +574,32 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      public void testReturnSuccessWithEmptyResult() {
 | 
	
		
			
				|  |  | -        String localClusterAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY;
 | 
	
		
			
				|  |  | -        String remote1Alias = "remote1";
 | 
	
		
			
				|  |  | -        String remote2Alias = "remote2";
 | 
	
		
			
				|  |  |          String remote3Alias = "remote3";
 | 
	
		
			
				|  |  |          NoClustersToSearchException noClustersException = new NoClustersToSearchException();
 | 
	
		
			
				|  |  |          Predicate<String> skipUnPredicate = s -> {
 | 
	
		
			
				|  |  | -            if (s.equals("remote2") || s.equals("remote3")) {
 | 
	
		
			
				|  |  | +            if (s.equals(REMOTE2_ALIAS) || s.equals("remote3")) {
 | 
	
		
			
				|  |  |                  return true;
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |              return false;
 | 
	
		
			
				|  |  |          };
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        EsqlExecutionInfo.Cluster localCluster = new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false);
 | 
	
		
			
				|  |  | -        EsqlExecutionInfo.Cluster remote1 = new EsqlExecutionInfo.Cluster(remote1Alias, "logs*", false);
 | 
	
		
			
				|  |  | -        EsqlExecutionInfo.Cluster remote2 = new EsqlExecutionInfo.Cluster(remote2Alias, "logs*", true);
 | 
	
		
			
				|  |  | +        EsqlExecutionInfo.Cluster localCluster = new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false);
 | 
	
		
			
				|  |  | +        EsqlExecutionInfo.Cluster remote1 = new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "logs*", false);
 | 
	
		
			
				|  |  | +        EsqlExecutionInfo.Cluster remote2 = new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "logs*", true);
 | 
	
		
			
				|  |  |          EsqlExecutionInfo.Cluster remote3 = new EsqlExecutionInfo.Cluster(remote3Alias, "logs*", true);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          // not a cross-cluster cluster search, so do not return empty result
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(skipUnPredicate, randomBoolean());
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> localCluster);
 | 
	
		
			
				|  |  | -            assertFalse(EsqlSessionCCSUtils.returnSuccessWithEmptyResult(executionInfo, noClustersException));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> localCluster);
 | 
	
		
			
				|  |  | +            assertFalse(EsqlCCSUtils.returnSuccessWithEmptyResult(executionInfo, noClustersException));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          // local cluster is present, so do not return empty result
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(skipUnPredicate, randomBoolean());
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(localClusterAlias, (k, v) -> localCluster);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> remote1);
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> localCluster);
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> remote1);
 | 
	
		
			
				|  |  |              // TODO: this logic will be added in the follow-on PR that handles missing indices
 | 
	
		
			
				|  |  |              // assertFalse(EsqlSessionCCSUtils.returnSuccessWithEmptyResult(executionInfo, noClustersException));
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -596,16 +607,16 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          // remote-only, one cluster is skip_unavailable=false, so do not return empty result
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(skipUnPredicate, randomBoolean());
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote1Alias, (k, v) -> remote1);
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote2Alias, (k, v) -> remote2);
 | 
	
		
			
				|  |  | -            assertFalse(EsqlSessionCCSUtils.returnSuccessWithEmptyResult(executionInfo, noClustersException));
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> remote1);
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE2_ALIAS, (k, v) -> remote2);
 | 
	
		
			
				|  |  | +            assertFalse(EsqlCCSUtils.returnSuccessWithEmptyResult(executionInfo, noClustersException));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          // remote-only, all clusters are skip_unavailable=true, so should return empty result with
 | 
	
		
			
				|  |  |          // NoSuchClustersException or "remote unavailable" type exception
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(skipUnPredicate, randomBoolean());
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote2Alias, (k, v) -> remote2);
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE2_ALIAS, (k, v) -> remote2);
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(remote3Alias, (k, v) -> remote3);
 | 
	
		
			
				|  |  |              Exception e = randomFrom(
 | 
	
		
			
				|  |  |                  new NoSuchRemoteClusterException("foo"),
 | 
	
	
		
			
				|  | @@ -613,23 +624,22 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |                  new NoSeedNodeLeftException("foo"),
 | 
	
		
			
				|  |  |                  new IllegalStateException("unknown host")
 | 
	
		
			
				|  |  |              );
 | 
	
		
			
				|  |  | -            assertTrue(EsqlSessionCCSUtils.returnSuccessWithEmptyResult(executionInfo, e));
 | 
	
		
			
				|  |  | +            assertTrue(EsqlCCSUtils.returnSuccessWithEmptyResult(executionInfo, e));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          // remote-only, all clusters are skip_unavailable=true, but exception is not "remote unavailable" so return false
 | 
	
		
			
				|  |  |          // Note: this functionality may change in follow-on PRs, so remove this test in that case
 | 
	
		
			
				|  |  |          {
 | 
	
		
			
				|  |  |              EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(skipUnPredicate, randomBoolean());
 | 
	
		
			
				|  |  | -            executionInfo.swapCluster(remote2Alias, (k, v) -> remote2);
 | 
	
		
			
				|  |  | +            executionInfo.swapCluster(REMOTE2_ALIAS, (k, v) -> remote2);
 | 
	
		
			
				|  |  |              executionInfo.swapCluster(remote3Alias, (k, v) -> remote3);
 | 
	
		
			
				|  |  | -            assertFalse(EsqlSessionCCSUtils.returnSuccessWithEmptyResult(executionInfo, new NullPointerException()));
 | 
	
		
			
				|  |  | +            assertFalse(EsqlCCSUtils.returnSuccessWithEmptyResult(executionInfo, new NullPointerException()));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      public void testUpdateExecutionInfoToReturnEmptyResult() {
 | 
	
		
			
				|  |  | -        String localClusterAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY;
 | 
	
		
			
				|  |  | -        String remote1Alias = "remote1";
 | 
	
		
			
				|  |  | -        String remote2Alias = "remote2";
 | 
	
		
			
				|  |  | +        String REMOTE1_ALIAS = "remote1";
 | 
	
		
			
				|  |  | +        String REMOTE2_ALIAS = "remote2";
 | 
	
		
			
				|  |  |          String remote3Alias = "remote3";
 | 
	
		
			
				|  |  |          ConnectTransportException transportEx = new ConnectTransportException(null, "foo");
 | 
	
		
			
				|  |  |          Predicate<String> skipUnPredicate = s -> {
 | 
	
	
		
			
				|  | @@ -639,9 +649,9 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |              return false;
 | 
	
		
			
				|  |  |          };
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        EsqlExecutionInfo.Cluster localCluster = new EsqlExecutionInfo.Cluster(localClusterAlias, "logs*", false);
 | 
	
		
			
				|  |  | -        EsqlExecutionInfo.Cluster remote1 = new EsqlExecutionInfo.Cluster(remote1Alias, "logs*", true);
 | 
	
		
			
				|  |  | -        EsqlExecutionInfo.Cluster remote2 = new EsqlExecutionInfo.Cluster(remote2Alias, "logs*", true);
 | 
	
		
			
				|  |  | +        EsqlExecutionInfo.Cluster localCluster = new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false);
 | 
	
		
			
				|  |  | +        EsqlExecutionInfo.Cluster remote1 = new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "logs*", true);
 | 
	
		
			
				|  |  | +        EsqlExecutionInfo.Cluster remote2 = new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "logs*", true);
 | 
	
		
			
				|  |  |          EsqlExecutionInfo.Cluster remote3 = new EsqlExecutionInfo.Cluster(remote3Alias, "logs*", true);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(skipUnPredicate, randomBoolean());
 | 
	
	
		
			
				|  | @@ -652,13 +662,13 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          assertNull(executionInfo.overallTook());
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        EsqlSessionCCSUtils.updateExecutionInfoToReturnEmptyResult(executionInfo, transportEx);
 | 
	
		
			
				|  |  | +        EsqlCCSUtils.updateExecutionInfoToReturnEmptyResult(executionInfo, transportEx);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          assertNotNull(executionInfo.overallTook());
 | 
	
		
			
				|  |  | -        assertThat(executionInfo.getCluster(localClusterAlias).getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL));
 | 
	
		
			
				|  |  | -        assertThat(executionInfo.getCluster(localClusterAlias).getFailures().size(), equalTo(0));
 | 
	
		
			
				|  |  | +        assertThat(executionInfo.getCluster(LOCAL_CLUSTER_ALIAS).getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL));
 | 
	
		
			
				|  |  | +        assertThat(executionInfo.getCluster(LOCAL_CLUSTER_ALIAS).getFailures().size(), equalTo(0));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        for (String remoteAlias : Set.of(remote1Alias, remote2Alias, remote3Alias)) {
 | 
	
		
			
				|  |  | +        for (String remoteAlias : Set.of(REMOTE1_ALIAS, REMOTE2_ALIAS, remote3Alias)) {
 | 
	
		
			
				|  |  |              assertThat(executionInfo.getCluster(remoteAlias).getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SKIPPED));
 | 
	
		
			
				|  |  |              List<ShardSearchFailure> remoteFailures = executionInfo.getCluster(remoteAlias).getFailures();
 | 
	
		
			
				|  |  |              assertThat(remoteFailures.size(), equalTo(1));
 | 
	
	
		
			
				|  | @@ -667,11 +677,11 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      public void testConcreteIndexRequested() {
 | 
	
		
			
				|  |  | -        assertThat(EsqlSessionCCSUtils.concreteIndexRequested("logs*"), equalTo(false));
 | 
	
		
			
				|  |  | -        assertThat(EsqlSessionCCSUtils.concreteIndexRequested("mylogs1,mylogs2,logs*"), equalTo(true));
 | 
	
		
			
				|  |  | -        assertThat(EsqlSessionCCSUtils.concreteIndexRequested("x*,logs"), equalTo(true));
 | 
	
		
			
				|  |  | -        assertThat(EsqlSessionCCSUtils.concreteIndexRequested("logs,metrics"), equalTo(true));
 | 
	
		
			
				|  |  | -        assertThat(EsqlSessionCCSUtils.concreteIndexRequested("*"), equalTo(false));
 | 
	
		
			
				|  |  | +        assertThat(EsqlCCSUtils.concreteIndexRequested("logs*"), equalTo(false));
 | 
	
		
			
				|  |  | +        assertThat(EsqlCCSUtils.concreteIndexRequested("mylogs1,mylogs2,logs*"), equalTo(true));
 | 
	
		
			
				|  |  | +        assertThat(EsqlCCSUtils.concreteIndexRequested("x*,logs"), equalTo(true));
 | 
	
		
			
				|  |  | +        assertThat(EsqlCCSUtils.concreteIndexRequested("logs,metrics"), equalTo(true));
 | 
	
		
			
				|  |  | +        assertThat(EsqlCCSUtils.concreteIndexRequested("*"), equalTo(false));
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      public void testCheckForCcsLicense() {
 | 
	
	
		
			
				|  | @@ -758,6 +768,64 @@ public class EsqlSessionCCSUtilsTests extends ESTestCase {
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    public void testShouldIgnoreRuntimeError() {
 | 
	
		
			
				|  |  | +        Predicate<String> skipUnPredicate = s -> s.equals(REMOTE1_ALIAS);
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(skipUnPredicate, true);
 | 
	
		
			
				|  |  | +        executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  | +        executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", true));
 | 
	
		
			
				|  |  | +        executionInfo.swapCluster(REMOTE2_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "mylogs1,mylogs2,logs*", false));
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        // remote1: skip_unavailable=true, so should ignore connect errors, but not others
 | 
	
		
			
				|  |  | +        assertThat(
 | 
	
		
			
				|  |  | +            shouldIgnoreRuntimeError(executionInfo, REMOTE1_ALIAS, new IllegalStateException("Unable to open any connections")),
 | 
	
		
			
				|  |  | +            is(true)
 | 
	
		
			
				|  |  | +        );
 | 
	
		
			
				|  |  | +        assertThat(shouldIgnoreRuntimeError(executionInfo, REMOTE1_ALIAS, new TaskCancelledException("task cancelled")), is(false));
 | 
	
		
			
				|  |  | +        assertThat(shouldIgnoreRuntimeError(executionInfo, REMOTE1_ALIAS, new ElasticsearchException("something is wrong")), is(false));
 | 
	
		
			
				|  |  | +        // remote2: skip_unavailable=false, so should not ignore any errors
 | 
	
		
			
				|  |  | +        assertThat(
 | 
	
		
			
				|  |  | +            shouldIgnoreRuntimeError(executionInfo, REMOTE2_ALIAS, new IllegalStateException("Unable to open any connections")),
 | 
	
		
			
				|  |  | +            is(false)
 | 
	
		
			
				|  |  | +        );
 | 
	
		
			
				|  |  | +        assertThat(shouldIgnoreRuntimeError(executionInfo, REMOTE2_ALIAS, new TaskCancelledException("task cancelled")), is(false));
 | 
	
		
			
				|  |  | +        // same for local
 | 
	
		
			
				|  |  | +        assertThat(
 | 
	
		
			
				|  |  | +            shouldIgnoreRuntimeError(executionInfo, LOCAL_CLUSTER_ALIAS, new IllegalStateException("Unable to open any connections")),
 | 
	
		
			
				|  |  | +            is(false)
 | 
	
		
			
				|  |  | +        );
 | 
	
		
			
				|  |  | +        assertThat(shouldIgnoreRuntimeError(executionInfo, LOCAL_CLUSTER_ALIAS, new TaskCancelledException("task cancelled")), is(false));
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    public void testSkipUnavailableListener() {
 | 
	
		
			
				|  |  | +        Predicate<String> skipUnPredicate = s -> s.equals(REMOTE1_ALIAS);
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        EsqlExecutionInfo executionInfo = new EsqlExecutionInfo(skipUnPredicate, true);
 | 
	
		
			
				|  |  | +        executionInfo.swapCluster(LOCAL_CLUSTER_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(LOCAL_CLUSTER_ALIAS, "logs*", false));
 | 
	
		
			
				|  |  | +        executionInfo.swapCluster(REMOTE1_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE1_ALIAS, "*", true));
 | 
	
		
			
				|  |  | +        executionInfo.swapCluster(REMOTE2_ALIAS, (k, v) -> new EsqlExecutionInfo.Cluster(REMOTE2_ALIAS, "mylogs1,mylogs2,logs*", false));
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        ActionListener<Void> expectResult = ActionListener.wrap(unused -> {}, (e) -> fail("Listener should not have failed"));
 | 
	
		
			
				|  |  | +        ActionListener<Void> expectFailure = ActionListener.wrap(unused -> fail("Listener should have failed"), (e) -> {});
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        // snip_unavailable=true but not connect exception, so should fail
 | 
	
		
			
				|  |  | +        skipUnavailableListener(expectFailure, executionInfo, REMOTE1_ALIAS, EsqlExecutionInfo.Cluster.Status.PARTIAL).onFailure(
 | 
	
		
			
				|  |  | +            new ElasticsearchException("something is wrong")
 | 
	
		
			
				|  |  | +        );
 | 
	
		
			
				|  |  | +        assertThat(executionInfo.getCluster(REMOTE1_ALIAS).getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.RUNNING));
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        // snip_unavailable=true, so should not fail
 | 
	
		
			
				|  |  | +        skipUnavailableListener(expectResult, executionInfo, REMOTE1_ALIAS, EsqlExecutionInfo.Cluster.Status.PARTIAL).onFailure(
 | 
	
		
			
				|  |  | +            new IllegalStateException("Unable to open any connections")
 | 
	
		
			
				|  |  | +        );
 | 
	
		
			
				|  |  | +        assertThat(executionInfo.getCluster(REMOTE1_ALIAS).getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.PARTIAL));
 | 
	
		
			
				|  |  | +        // snip_unavailable=false, so should fail
 | 
	
		
			
				|  |  | +        skipUnavailableListener(expectFailure, executionInfo, REMOTE2_ALIAS, EsqlExecutionInfo.Cluster.Status.PARTIAL).onFailure(
 | 
	
		
			
				|  |  | +            new IllegalStateException("Unable to open any connections")
 | 
	
		
			
				|  |  | +        );
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      private XPackLicenseStatus activeLicenseStatus(License.OperationMode operationMode) {
 | 
	
		
			
				|  |  |          return new XPackLicenseStatus(operationMode, true, null);
 | 
	
		
			
				|  |  |      }
 |