1515import org .elasticsearch .cluster .metadata .IndexMetadata ;
1616import org .elasticsearch .common .Priority ;
1717import org .elasticsearch .common .Strings ;
18+ import org .elasticsearch .common .UUIDs ;
1819import org .elasticsearch .common .settings .Settings ;
1920import org .elasticsearch .compute .lucene .DataPartitioning ;
2021import org .elasticsearch .compute .operator .DriverProfile ;
3334import org .elasticsearch .xpack .esql .plugin .QueryPragmas ;
3435
3536import java .io .IOException ;
37+ import java .util .Collection ;
3638import java .util .HashMap ;
39+ import java .util .HashSet ;
3740import java .util .List ;
3841import java .util .Locale ;
3942import java .util .Map ;
4043import java .util .Set ;
4144import java .util .concurrent .CountDownLatch ;
4245import java .util .concurrent .TimeUnit ;
4346import java .util .stream .Collectors ;
47+ import java .util .stream .Stream ;
4448
4549import static org .elasticsearch .test .hamcrest .ElasticsearchAssertions .assertAcked ;
4650import static org .elasticsearch .xpack .esql .EsqlTestUtils .getValuesList ;
5054import static org .hamcrest .Matchers .greaterThan ;
5155import static org .hamcrest .Matchers .greaterThanOrEqualTo ;
5256import static org .hamcrest .Matchers .hasSize ;
57+ import static org .hamcrest .Matchers .in ;
5358import static org .hamcrest .Matchers .instanceOf ;
5459import static org .hamcrest .Matchers .is ;
5560import static org .hamcrest .Matchers .lessThanOrEqualTo ;
@@ -838,6 +843,108 @@ private static void assertClusterMetadataInResponse(EsqlQueryResponse resp, bool
838843 }
839844 }
840845
846+ public void testPartialResults () throws Exception {
847+ class Cluster {
848+ final int okShards = randomIntBetween (1 , 5 );
849+ final int failingShards = randomIntBetween (1 , 5 );
850+ Set <String > okIds ;
851+ }
852+ Cluster local = new Cluster ();
853+ Cluster remote1 = new Cluster ();
854+ Cluster remote2 = new Cluster ();
855+
856+ local .okIds = populateIndex (LOCAL_CLUSTER , "ok-local" , local .okShards );
857+ populateIndexWithFailingFields (LOCAL_CLUSTER , "fail-local" , local .failingShards );
858+
859+ remote1 .okIds = populateIndex (REMOTE_CLUSTER_1 , "ok-cluster1" , remote1 .okShards );
860+ populateIndexWithFailingFields (REMOTE_CLUSTER_1 , "fail-cluster1" , remote1 .failingShards );
861+
862+ remote2 .okIds = populateIndex (REMOTE_CLUSTER_2 , "ok-cluster2" , remote2 .okShards );
863+ populateIndexWithFailingFields (REMOTE_CLUSTER_2 , "fail-cluster2" , remote2 .failingShards );
864+ // allow_partial_results = false
865+ {
866+ EsqlQueryRequest request = new EsqlQueryRequest ();
867+ request .query ("FROM ok*,fail*,*:ok*,*:fail* | KEEP id, fail_me" );
868+ request .allowPartialResults (false );
869+ IllegalStateException error = expectThrows (IllegalStateException .class , () -> runQuery (request ).close ());
870+ assertThat (error .getMessage (), containsString ("Accessing failing field" ));
871+ }
872+ // allow_partial_results = true
873+ {
874+ EsqlQueryRequest request = new EsqlQueryRequest ();
875+ request .query ("FROM ok*,fail*,*:ok*,*:fail* | KEEP id, fail_me" );
876+ request .allowPartialResults (true );
877+ request .includeCCSMetadata (randomBoolean ());
878+ try (var resp = runQuery (request )) {
879+ assertTrue (resp .isPartial ());
880+ Set <String > allIds = Stream .of (local .okIds , remote1 .okIds , remote2 .okIds )
881+ .flatMap (Collection ::stream )
882+ .collect (Collectors .toSet ());
883+ List <List <Object >> rows = getValuesList (resp );
884+ assertThat (rows .size (), lessThanOrEqualTo (allIds .size ()));
885+ Set <String > returnedIds = new HashSet <>();
886+ for (List <Object > row : rows ) {
887+ assertThat (row .size (), equalTo (2 ));
888+ String id = (String ) row .get (0 );
889+ assertTrue (returnedIds .add (id ));
890+ assertThat (id , is (in (allIds )));
891+ }
892+ if (request .includeCCSMetadata ()) {
893+ EsqlExecutionInfo .Cluster localInfo = resp .getExecutionInfo ().getCluster (LOCAL_CLUSTER );
894+ assertThat (localInfo .getTotalShards (), equalTo (local .okShards + local .failingShards ));
895+ assertThat (localInfo .getSuccessfulShards (), lessThanOrEqualTo (local .okShards ));
896+ assertThat (localInfo .getStatus (), equalTo (EsqlExecutionInfo .Cluster .Status .PARTIAL ));
897+
898+ EsqlExecutionInfo .Cluster remote1Info = resp .getExecutionInfo ().getCluster (REMOTE_CLUSTER_1 );
899+ assertThat (remote1Info .getTotalShards (), equalTo (remote1 .okShards + remote1 .failingShards ));
900+ assertThat (remote1Info .getSuccessfulShards (), lessThanOrEqualTo (remote1 .okShards ));
901+ assertThat (localInfo .getStatus (), equalTo (EsqlExecutionInfo .Cluster .Status .PARTIAL ));
902+
903+ EsqlExecutionInfo .Cluster remote2Info = resp .getExecutionInfo ().getCluster (REMOTE_CLUSTER_2 );
904+ assertThat (remote2Info .getTotalShards (), equalTo (remote2 .okShards + remote1 .failingShards ));
905+ assertThat (remote2Info .getSuccessfulShards (), lessThanOrEqualTo (remote2 .okShards ));
906+ assertThat (localInfo .getStatus (), equalTo (EsqlExecutionInfo .Cluster .Status .PARTIAL ));
907+ }
908+ }
909+ }
910+ // allow_partial_results = true
911+ {
912+ EsqlQueryRequest request = new EsqlQueryRequest ();
913+ request .query ("FROM ok*,cluster-a:ok*,remote-b:fail* | KEEP id, fail_me" );
914+ request .allowPartialResults (true );
915+ request .includeCCSMetadata (randomBoolean ());
916+ try (var resp = runQuery (request )) {
917+ assertTrue (resp .isPartial ());
918+ Set <String > allIds = Stream .of (local .okIds , remote1 .okIds ).flatMap (Collection ::stream ).collect (Collectors .toSet ());
919+ List <List <Object >> rows = getValuesList (resp );
920+ assertThat (rows .size (), equalTo (allIds .size ()));
921+ Set <String > returnedIds = new HashSet <>();
922+ for (List <Object > row : rows ) {
923+ assertThat (row .size (), equalTo (2 ));
924+ String id = (String ) row .get (0 );
925+ assertTrue (returnedIds .add (id ));
926+ }
927+ assertThat (returnedIds , equalTo (allIds ));
928+ if (request .includeCCSMetadata ()) {
929+ EsqlExecutionInfo .Cluster localInfo = resp .getExecutionInfo ().getCluster (LOCAL_CLUSTER );
930+ assertThat (localInfo .getTotalShards (), equalTo (local .okShards ));
931+ assertThat (localInfo .getSuccessfulShards (), equalTo (local .okShards ));
932+ assertThat (localInfo .getStatus (), equalTo (EsqlExecutionInfo .Cluster .Status .SUCCESSFUL ));
933+
934+ EsqlExecutionInfo .Cluster remote1Info = resp .getExecutionInfo ().getCluster (REMOTE_CLUSTER_1 );
935+ assertThat (remote1Info .getTotalShards (), equalTo (remote1 .okShards ));
936+ assertThat (remote1Info .getSuccessfulShards (), equalTo (remote1 .okShards ));
937+ assertThat (remote1Info .getStatus (), equalTo (EsqlExecutionInfo .Cluster .Status .SUCCESSFUL ));
938+
939+ EsqlExecutionInfo .Cluster remote2Info = resp .getExecutionInfo ().getCluster (REMOTE_CLUSTER_2 );
940+ assertThat (remote2Info .getTotalShards (), equalTo (remote1 .failingShards ));
941+ assertThat (remote2Info .getSuccessfulShards (), equalTo (0 ));
942+ assertThat (remote1Info .getStatus (), equalTo (EsqlExecutionInfo .Cluster .Status .PARTIAL ));
943+ }
944+ }
945+ }
946+ }
947+
841948 void waitForNoInitializingShards (Client client , TimeValue timeout , String ... indices ) {
842949 ClusterHealthResponse resp = client .admin ()
843950 .cluster ()
@@ -951,7 +1058,7 @@ Map<String, Object> setupFailClusters() throws IOException {
9511058 populateLocalIndices (LOCAL_INDEX , numShardsLocal );
9521059
9531060 int numShardsRemote = randomIntBetween (1 , 3 );
954- populateRemoteIndicesFail (REMOTE_CLUSTER_1 , REMOTE_INDEX , numShardsRemote );
1061+ populateIndexWithFailingFields (REMOTE_CLUSTER_1 , REMOTE_INDEX , numShardsRemote );
9551062
9561063 Map <String , Object > clusterInfo = new HashMap <>();
9571064 clusterInfo .put ("local.num_shards" , numShardsLocal );
@@ -962,8 +1069,29 @@ Map<String, Object> setupFailClusters() throws IOException {
9621069 return clusterInfo ;
9631070 }
9641071
965- void populateRemoteIndicesFail (String clusterAlias , String indexName , int numShards ) throws IOException {
966- Client remoteClient = client (clusterAlias );
1072+ protected Set <String > populateIndex (String clusterAlias , String indexName , int numShards ) {
1073+ Client localClient = client (clusterAlias );
1074+ assertAcked (
1075+ localClient .admin ()
1076+ .indices ()
1077+ .prepareCreate (indexName )
1078+ .setSettings (Settings .builder ().put ("index.number_of_shards" , numShards ))
1079+ .setMapping ("id" , "type=keyword" , "tag" , "type=keyword" , "v" , "type=long" , "const" , "type=long" )
1080+ );
1081+ Set <String > ids = new HashSet <>();
1082+ int numDocs = between (1 , 100 );
1083+ String tag = Strings .isEmpty (clusterAlias ) ? "local" : clusterAlias ;
1084+ for (int i = 0 ; i < numDocs ; i ++) {
1085+ String id = UUIDs .randomBase64UUID ();
1086+ localClient .prepareIndex (indexName ).setSource ("id" , id , "tag" , tag , "v" , i ).get ();
1087+ ids .add (id );
1088+ }
1089+ localClient .admin ().indices ().prepareRefresh (indexName ).get ();
1090+ return ids ;
1091+ }
1092+
1093+ Set <String > populateIndexWithFailingFields (String clusterAlias , String indexName , int numShards ) throws IOException {
1094+ Client client = client (clusterAlias );
9671095 XContentBuilder mapping = JsonXContent .contentBuilder ().startObject ();
9681096 mapping .startObject ("runtime" );
9691097 {
@@ -975,16 +1103,29 @@ void populateRemoteIndicesFail(String clusterAlias, String indexName, int numSha
9751103 mapping .endObject ();
9761104 }
9771105 mapping .endObject ();
1106+ mapping .startObject ("properties" );
1107+ {
1108+ mapping .startObject ("id" ).field ("type" , "keyword" ).endObject ();
1109+ mapping .startObject ("tag" ).field ("type" , "keyword" ).endObject ();
1110+ }
1111+ mapping .endObject ();
9781112 assertAcked (
979- remoteClient .admin ()
1113+ client .admin ()
9801114 .indices ()
9811115 .prepareCreate (indexName )
9821116 .setSettings (Settings .builder ().put ("index.number_of_shards" , numShards ))
9831117 .setMapping (mapping .endObject ())
9841118 );
985-
986- remoteClient .prepareIndex (indexName ).setSource ("id" , 0 ).get ();
987- remoteClient .admin ().indices ().prepareRefresh (indexName ).get ();
1119+ Set <String > ids = new HashSet <>();
1120+ String tag = clusterAlias .isEmpty () ? "local" : clusterAlias ;
1121+ int numDocs = between (1 , 100 );
1122+ for (int i = 0 ; i < numDocs ; i ++) {
1123+ String id = UUIDs .base64UUID ();
1124+ client .prepareIndex (indexName ).setSource ("id" , id , "tag" , tag , "v" , i ).get ();
1125+ ids .add (id );
1126+ }
1127+ client .admin ().indices ().prepareRefresh (indexName ).get ();
1128+ return ids ;
9881129 }
9891130
9901131}
0 commit comments