|
26 | 26 | import org.elasticsearch.search.SearchHit;
|
27 | 27 | import org.elasticsearch.search.SearchHits;
|
28 | 28 | import org.elasticsearch.search.SearchModule;
|
| 29 | +import org.elasticsearch.search.aggregations.Aggregations; |
| 30 | +import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation; |
29 | 31 | import org.elasticsearch.test.ESTestCase;
|
30 | 32 | import org.elasticsearch.test.client.NoOpClient;
|
31 | 33 | import org.elasticsearch.xcontent.DeprecationHandler;
|
|
35 | 37 | import org.elasticsearch.xcontent.json.JsonXContent;
|
36 | 38 | import org.elasticsearch.xpack.core.transform.TransformDeprecations;
|
37 | 39 | import org.elasticsearch.xpack.core.transform.transforms.SettingsConfig;
|
| 40 | +import org.elasticsearch.xpack.core.transform.transforms.SettingsConfigTests; |
38 | 41 | import org.elasticsearch.xpack.core.transform.transforms.SourceConfig;
|
39 | 42 | import org.elasticsearch.xpack.core.transform.transforms.pivot.AggregationConfig;
|
40 | 43 | import org.elasticsearch.xpack.core.transform.transforms.pivot.AggregationConfigTests;
|
41 | 44 | import org.elasticsearch.xpack.core.transform.transforms.pivot.GroupConfig;
|
42 | 45 | import org.elasticsearch.xpack.core.transform.transforms.pivot.GroupConfigTests;
|
43 | 46 | import org.elasticsearch.xpack.core.transform.transforms.pivot.PivotConfig;
|
| 47 | +import org.elasticsearch.xpack.core.transform.transforms.pivot.PivotConfigTests; |
44 | 48 | import org.elasticsearch.xpack.spatial.SpatialPlugin;
|
45 | 49 | import org.elasticsearch.xpack.transform.Transform;
|
46 | 50 | import org.elasticsearch.xpack.transform.transforms.Function;
|
|
51 | 55 | import java.io.IOException;
|
52 | 56 | import java.util.ArrayList;
|
53 | 57 | import java.util.Collections;
|
| 58 | +import java.util.HashMap; |
54 | 59 | import java.util.List;
|
| 60 | +import java.util.Map; |
55 | 61 | import java.util.Set;
|
56 | 62 | import java.util.concurrent.CountDownLatch;
|
57 | 63 | import java.util.concurrent.TimeUnit;
|
|
63 | 69 | import static org.hamcrest.CoreMatchers.is;
|
64 | 70 | import static org.hamcrest.Matchers.contains;
|
65 | 71 | import static org.hamcrest.Matchers.containsString;
|
| 72 | +import static org.hamcrest.Matchers.empty; |
66 | 73 | import static org.hamcrest.Matchers.equalTo;
|
67 | 74 | import static org.hamcrest.Matchers.nullValue;
|
| 75 | +import static org.mockito.Mockito.mock; |
| 76 | +import static org.mockito.Mockito.when; |
68 | 77 |
|
69 | 78 | public class PivotTests extends ESTestCase {
|
70 | 79 |
|
@@ -206,6 +215,60 @@ public void testGetPerformanceCriticalFields() throws IOException {
|
206 | 215 | assertThat(pivot.getPerformanceCriticalFields(), contains("field-A", "field-B", "field-C"));
|
207 | 216 | }
|
208 | 217 |
|
| 218 | + public void testPreviewForEmptyAggregation() throws Exception { |
| 219 | + Function pivot = new Pivot( |
| 220 | + PivotConfigTests.randomPivotConfig(), |
| 221 | + SettingsConfigTests.randomSettingsConfig(), |
| 222 | + Version.CURRENT, |
| 223 | + Collections.emptySet() |
| 224 | + ); |
| 225 | + |
| 226 | + CountDownLatch latch = new CountDownLatch(1); |
| 227 | + final AtomicReference<Exception> exceptionHolder = new AtomicReference<>(); |
| 228 | + final AtomicReference<List<Map<String, Object>>> responseHolder = new AtomicReference<>(); |
| 229 | + |
| 230 | + Client emptyAggregationClient = new MyMockClientWithEmptyAggregation("empty aggregation test for preview"); |
| 231 | + pivot.preview(emptyAggregationClient, null, new HashMap<>(), new SourceConfig("test"), null, 1, ActionListener.wrap(r -> { |
| 232 | + responseHolder.set(r); |
| 233 | + latch.countDown(); |
| 234 | + }, e -> { |
| 235 | + exceptionHolder.set(e); |
| 236 | + latch.countDown(); |
| 237 | + })); |
| 238 | + assertTrue(latch.await(100, TimeUnit.MILLISECONDS)); |
| 239 | + emptyAggregationClient.close(); |
| 240 | + |
| 241 | + assertThat(exceptionHolder.get(), is(nullValue())); |
| 242 | + assertThat(responseHolder.get(), is(empty())); |
| 243 | + } |
| 244 | + |
| 245 | + public void testPreviewForCompositeAggregation() throws Exception { |
| 246 | + Function pivot = new Pivot( |
| 247 | + PivotConfigTests.randomPivotConfig(), |
| 248 | + SettingsConfigTests.randomSettingsConfig(), |
| 249 | + Version.CURRENT, |
| 250 | + Collections.emptySet() |
| 251 | + ); |
| 252 | + |
| 253 | + CountDownLatch latch = new CountDownLatch(1); |
| 254 | + final AtomicReference<Exception> exceptionHolder = new AtomicReference<>(); |
| 255 | + final AtomicReference<List<Map<String, Object>>> responseHolder = new AtomicReference<>(); |
| 256 | + |
| 257 | + Client compositeAggregationClient = new MyMockClientWithCompositeAggregation("composite aggregation test for preview"); |
| 258 | + pivot.preview(compositeAggregationClient, null, new HashMap<>(), new SourceConfig("test"), null, 1, ActionListener.wrap(r -> { |
| 259 | + responseHolder.set(r); |
| 260 | + latch.countDown(); |
| 261 | + }, e -> { |
| 262 | + exceptionHolder.set(e); |
| 263 | + latch.countDown(); |
| 264 | + })); |
| 265 | + assertTrue(latch.await(100, TimeUnit.MILLISECONDS)); |
| 266 | + compositeAggregationClient.close(); |
| 267 | + |
| 268 | + assertThat(exceptionHolder.get(), is(nullValue())); |
| 269 | + assertThat(responseHolder.get(), is(empty())); |
| 270 | + } |
| 271 | + |
209 | 272 | private class MyMockClient extends NoOpClient {
|
210 | 273 | MyMockClient(String testName) {
|
211 | 274 | super(testName);
|
@@ -262,6 +325,44 @@ protected <Request extends ActionRequest, Response extends ActionResponse> void
|
262 | 325 | }
|
263 | 326 | }
|
264 | 327 |
|
| 328 | + private class MyMockClientWithEmptyAggregation extends NoOpClient { |
| 329 | + MyMockClientWithEmptyAggregation(String testName) { |
| 330 | + super(testName); |
| 331 | + } |
| 332 | + |
| 333 | + @SuppressWarnings("unchecked") |
| 334 | + @Override |
| 335 | + protected <Request extends ActionRequest, Response extends ActionResponse> void doExecute( |
| 336 | + ActionType<Response> action, |
| 337 | + Request request, |
| 338 | + ActionListener<Response> listener |
| 339 | + ) { |
| 340 | + SearchResponse response = mock(SearchResponse.class); |
| 341 | + when(response.getAggregations()).thenReturn(new Aggregations(Collections.emptyList())); |
| 342 | + listener.onResponse((Response) response); |
| 343 | + } |
| 344 | + } |
| 345 | + |
| 346 | + private class MyMockClientWithCompositeAggregation extends NoOpClient { |
| 347 | + MyMockClientWithCompositeAggregation(String testName) { |
| 348 | + super(testName); |
| 349 | + } |
| 350 | + |
| 351 | + @SuppressWarnings("unchecked") |
| 352 | + @Override |
| 353 | + protected <Request extends ActionRequest, Response extends ActionResponse> void doExecute( |
| 354 | + ActionType<Response> action, |
| 355 | + Request request, |
| 356 | + ActionListener<Response> listener |
| 357 | + ) { |
| 358 | + SearchResponse response = mock(SearchResponse.class); |
| 359 | + CompositeAggregation compositeAggregation = mock(CompositeAggregation.class); |
| 360 | + when(response.getAggregations()).thenReturn(new Aggregations(Collections.singletonList(compositeAggregation))); |
| 361 | + when(compositeAggregation.getBuckets()).thenReturn(new ArrayList<>()); |
| 362 | + listener.onResponse((Response) response); |
| 363 | + } |
| 364 | + } |
| 365 | + |
265 | 366 | private PivotConfig getValidPivotConfig() throws IOException {
|
266 | 367 | return new PivotConfig(GroupConfigTests.randomGroupConfig(), getValidAggregationConfig(), null);
|
267 | 368 | }
|
|
0 commit comments