-
Notifications
You must be signed in to change notification settings - Fork 2.8k
/
TestSourcePartitionedScheduler.java
730 lines (627 loc) · 32.9 KB
/
TestSourcePartitionedScheduler.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.trino.execution.scheduler;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import io.airlift.units.Duration;
import io.trino.Session;
import io.trino.client.NodeVersion;
import io.trino.connector.CatalogName;
import io.trino.cost.StatsAndCosts;
import io.trino.execution.DynamicFilterConfig;
import io.trino.execution.MockRemoteTaskFactory;
import io.trino.execution.MockRemoteTaskFactory.MockRemoteTask;
import io.trino.execution.NodeTaskMap;
import io.trino.execution.RemoteTask;
import io.trino.execution.SqlStageExecution;
import io.trino.execution.StageId;
import io.trino.execution.TableInfo;
import io.trino.execution.buffer.OutputBuffers.OutputBufferId;
import io.trino.failuredetector.NoOpFailureDetector;
import io.trino.metadata.InMemoryNodeManager;
import io.trino.metadata.InternalNode;
import io.trino.metadata.InternalNodeManager;
import io.trino.metadata.Metadata;
import io.trino.metadata.QualifiedObjectName;
import io.trino.server.DynamicFilterService;
import io.trino.spi.QueryId;
import io.trino.spi.connector.ConnectorPartitionHandle;
import io.trino.spi.connector.ConnectorSplit;
import io.trino.spi.connector.ConnectorSplitSource;
import io.trino.spi.connector.DynamicFilter;
import io.trino.spi.connector.FixedSplitSource;
import io.trino.spi.predicate.TupleDomain;
import io.trino.spi.type.TypeOperators;
import io.trino.split.ConnectorAwareSplitSource;
import io.trino.split.SplitSource;
import io.trino.sql.DynamicFilters;
import io.trino.sql.planner.Partitioning;
import io.trino.sql.planner.PartitioningScheme;
import io.trino.sql.planner.PlanFragment;
import io.trino.sql.planner.StageExecutionPlan;
import io.trino.sql.planner.Symbol;
import io.trino.sql.planner.SymbolAllocator;
import io.trino.sql.planner.plan.DynamicFilterId;
import io.trino.sql.planner.plan.FilterNode;
import io.trino.sql.planner.plan.JoinNode;
import io.trino.sql.planner.plan.PlanFragmentId;
import io.trino.sql.planner.plan.PlanNodeId;
import io.trino.sql.planner.plan.RemoteSourceNode;
import io.trino.sql.planner.plan.TableScanNode;
import io.trino.testing.TestingMetadata.TestingColumnHandle;
import io.trino.testing.TestingSession;
import io.trino.testing.TestingSplit;
import io.trino.util.FinalizerService;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
import java.net.URI;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.function.Supplier;
import static com.google.common.base.Preconditions.checkArgument;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.trino.SessionTestUtils.TEST_SESSION;
import static io.trino.execution.buffer.OutputBuffers.BufferType.PARTITIONED;
import static io.trino.execution.buffer.OutputBuffers.createInitialEmptyOutputBuffers;
import static io.trino.execution.scheduler.ScheduleResult.BlockedReason.SPLIT_QUEUES_FULL;
import static io.trino.execution.scheduler.SourcePartitionedScheduler.newSourcePartitionedSchedulerAsStageScheduler;
import static io.trino.metadata.MetadataManager.createTestMetadataManager;
import static io.trino.operator.StageExecutionDescriptor.ungroupedExecution;
import static io.trino.spi.StandardErrorCode.NO_NODES_AVAILABLE;
import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED;
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static io.trino.sql.DynamicFilters.createDynamicFilterExpression;
import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION;
import static io.trino.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION;
import static io.trino.sql.planner.plan.ExchangeNode.Type.REPLICATE;
import static io.trino.sql.planner.plan.JoinNode.Type.INNER;
import static io.trino.testing.TestingHandles.TEST_TABLE_HANDLE;
import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy;
import static java.lang.Integer.min;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static java.util.concurrent.Executors.newScheduledThreadPool;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;
public class TestSourcePartitionedScheduler
{
public static final OutputBufferId OUT = new OutputBufferId(0);
private static final CatalogName CONNECTOR_ID = TEST_TABLE_HANDLE.getCatalogName();
private static final QueryId QUERY_ID = new QueryId("query");
private static final DynamicFilterId DYNAMIC_FILTER_ID = new DynamicFilterId("filter1");
private final ExecutorService queryExecutor = newCachedThreadPool(daemonThreadsNamed("stageExecutor-%s"));
private final ScheduledExecutorService scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed("stageScheduledExecutor-%s"));
private final InMemoryNodeManager nodeManager = new InMemoryNodeManager();
private final FinalizerService finalizerService = new FinalizerService();
private final Metadata metadata = createTestMetadataManager();
private final TypeOperators typeOperators = new TypeOperators();
private final Session session = TestingSession.testSessionBuilder().build();
public TestSourcePartitionedScheduler()
{
nodeManager.addNode(CONNECTOR_ID,
new InternalNode("other1", URI.create("http://127.0.0.1:11"), NodeVersion.UNKNOWN, false),
new InternalNode("other2", URI.create("http://127.0.0.1:12"), NodeVersion.UNKNOWN, false),
new InternalNode("other3", URI.create("http://127.0.0.1:13"), NodeVersion.UNKNOWN, false));
}
@BeforeClass
public void setUp()
{
finalizerService.start();
}
@AfterClass(alwaysRun = true)
public void destroyExecutor()
{
queryExecutor.shutdownNow();
scheduledExecutor.shutdownNow();
finalizerService.destroy();
}
@Test
public void testScheduleNoSplits()
{
StageExecutionPlan plan = createPlan(createFixedSplitSource(0, TestingSplit::createRemoteSplit));
NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService);
SqlStageExecution stage = createSqlStageExecution(plan, nodeTaskMap);
StageScheduler scheduler = getSourcePartitionedScheduler(plan, stage, nodeManager, nodeTaskMap, 1);
ScheduleResult scheduleResult = scheduler.schedule();
assertEquals(scheduleResult.getNewTasks().size(), 1);
assertEffectivelyFinished(scheduleResult, scheduler);
stage.abort();
}
@Test
public void testScheduleSplitsOneAtATime()
{
StageExecutionPlan plan = createPlan(createFixedSplitSource(60, TestingSplit::createRemoteSplit));
NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService);
SqlStageExecution stage = createSqlStageExecution(plan, nodeTaskMap);
StageScheduler scheduler = getSourcePartitionedScheduler(plan, stage, nodeManager, nodeTaskMap, 1);
for (int i = 0; i < 60; i++) {
ScheduleResult scheduleResult = scheduler.schedule();
// only finishes when last split is fetched
if (i == 59) {
assertEffectivelyFinished(scheduleResult, scheduler);
}
else {
assertFalse(scheduleResult.isFinished());
}
// never blocks
assertTrue(scheduleResult.getBlocked().isDone());
// first three splits create new tasks
assertEquals(scheduleResult.getNewTasks().size(), i < 3 ? 1 : 0);
assertEquals(stage.getAllTasks().size(), i < 3 ? i + 1 : 3);
assertPartitionedSplitCount(stage, min(i + 1, 60));
}
for (RemoteTask remoteTask : stage.getAllTasks()) {
assertEquals(remoteTask.getPartitionedSplitCount(), 20);
}
stage.abort();
}
@Test
public void testScheduleSplitsBatched()
{
StageExecutionPlan plan = createPlan(createFixedSplitSource(60, TestingSplit::createRemoteSplit));
NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService);
SqlStageExecution stage = createSqlStageExecution(plan, nodeTaskMap);
StageScheduler scheduler = getSourcePartitionedScheduler(plan, stage, nodeManager, nodeTaskMap, 7);
for (int i = 0; i <= (60 / 7); i++) {
ScheduleResult scheduleResult = scheduler.schedule();
// finishes when last split is fetched
if (i == (60 / 7)) {
assertEffectivelyFinished(scheduleResult, scheduler);
}
else {
assertFalse(scheduleResult.isFinished());
}
// never blocks
assertTrue(scheduleResult.getBlocked().isDone());
// first three splits create new tasks
assertEquals(scheduleResult.getNewTasks().size(), i == 0 ? 3 : 0);
assertEquals(stage.getAllTasks().size(), 3);
assertPartitionedSplitCount(stage, min((i + 1) * 7, 60));
}
for (RemoteTask remoteTask : stage.getAllTasks()) {
assertEquals(remoteTask.getPartitionedSplitCount(), 20);
}
stage.abort();
}
@Test
public void testScheduleSplitsBlock()
{
StageExecutionPlan plan = createPlan(createFixedSplitSource(80, TestingSplit::createRemoteSplit));
NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService);
SqlStageExecution stage = createSqlStageExecution(plan, nodeTaskMap);
StageScheduler scheduler = getSourcePartitionedScheduler(plan, stage, nodeManager, nodeTaskMap, 1);
// schedule first 60 splits, which will cause the scheduler to block
for (int i = 0; i <= 60; i++) {
ScheduleResult scheduleResult = scheduler.schedule();
assertFalse(scheduleResult.isFinished());
// blocks at 20 per node
assertEquals(scheduleResult.getBlocked().isDone(), i != 60);
// first three splits create new tasks
assertEquals(scheduleResult.getNewTasks().size(), i < 3 ? 1 : 0);
assertEquals(stage.getAllTasks().size(), i < 3 ? i + 1 : 3);
assertPartitionedSplitCount(stage, min(i + 1, 60));
}
for (RemoteTask remoteTask : stage.getAllTasks()) {
assertEquals(remoteTask.getPartitionedSplitCount(), 20);
}
// todo rewrite MockRemoteTask to fire a tate transition when splits are cleared, and then validate blocked future completes
// drop the 20 splits from one node
((MockRemoteTask) stage.getAllTasks().get(0)).clearSplits();
// schedule remaining 20 splits
for (int i = 0; i < 20; i++) {
ScheduleResult scheduleResult = scheduler.schedule();
// finishes when last split is fetched
if (i == 19) {
assertEffectivelyFinished(scheduleResult, scheduler);
}
else {
assertFalse(scheduleResult.isFinished());
}
// does not block again
assertTrue(scheduleResult.getBlocked().isDone());
// no additional tasks will be created
assertEquals(scheduleResult.getNewTasks().size(), 0);
assertEquals(stage.getAllTasks().size(), 3);
// we dropped 20 splits so start at 40 and count to 60
assertPartitionedSplitCount(stage, min(i + 41, 60));
}
for (RemoteTask remoteTask : stage.getAllTasks()) {
assertEquals(remoteTask.getPartitionedSplitCount(), 20);
}
stage.abort();
}
@Test
public void testScheduleSlowSplitSource()
{
QueuedSplitSource queuedSplitSource = new QueuedSplitSource(TestingSplit::createRemoteSplit);
StageExecutionPlan plan = createPlan(queuedSplitSource);
NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService);
SqlStageExecution stage = createSqlStageExecution(plan, nodeTaskMap);
StageScheduler scheduler = getSourcePartitionedScheduler(plan, stage, nodeManager, nodeTaskMap, 1);
// schedule with no splits - will block
ScheduleResult scheduleResult = scheduler.schedule();
assertFalse(scheduleResult.isFinished());
assertFalse(scheduleResult.getBlocked().isDone());
assertEquals(scheduleResult.getNewTasks().size(), 0);
assertEquals(stage.getAllTasks().size(), 0);
queuedSplitSource.addSplits(1);
assertTrue(scheduleResult.getBlocked().isDone());
}
@Test
public void testNoNodes()
{
assertTrinoExceptionThrownBy(() -> {
NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService);
InMemoryNodeManager nodeManager = new InMemoryNodeManager();
NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap));
StageExecutionPlan plan = createPlan(createFixedSplitSource(20, TestingSplit::createRemoteSplit));
SqlStageExecution stage = createSqlStageExecution(plan, nodeTaskMap);
StageScheduler scheduler = newSourcePartitionedSchedulerAsStageScheduler(
stage,
Iterables.getOnlyElement(plan.getSplitSources().keySet()),
Iterables.getOnlyElement(plan.getSplitSources().values()),
new DynamicSplitPlacementPolicy(nodeScheduler.createNodeSelector(session, Optional.of(CONNECTOR_ID)), stage::getAllTasks),
2,
new DynamicFilterService(metadata, typeOperators, new DynamicFilterConfig()),
() -> false);
scheduler.schedule();
}).hasErrorCode(NO_NODES_AVAILABLE);
}
@Test
public void testBalancedSplitAssignment()
{
// use private node manager so we can add a node later
InMemoryNodeManager nodeManager = new InMemoryNodeManager();
nodeManager.addNode(CONNECTOR_ID,
new InternalNode("other1", URI.create("http://127.0.0.1:11"), NodeVersion.UNKNOWN, false),
new InternalNode("other2", URI.create("http://127.0.0.1:12"), NodeVersion.UNKNOWN, false),
new InternalNode("other3", URI.create("http://127.0.0.1:13"), NodeVersion.UNKNOWN, false));
NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService);
// Schedule 15 splits - there are 3 nodes, each node should get 5 splits
StageExecutionPlan firstPlan = createPlan(createFixedSplitSource(15, TestingSplit::createRemoteSplit));
SqlStageExecution firstStage = createSqlStageExecution(firstPlan, nodeTaskMap);
StageScheduler firstScheduler = getSourcePartitionedScheduler(firstPlan, firstStage, nodeManager, nodeTaskMap, 200);
ScheduleResult scheduleResult = firstScheduler.schedule();
assertEffectivelyFinished(scheduleResult, firstScheduler);
assertTrue(scheduleResult.getBlocked().isDone());
assertEquals(scheduleResult.getNewTasks().size(), 3);
assertEquals(firstStage.getAllTasks().size(), 3);
for (RemoteTask remoteTask : firstStage.getAllTasks()) {
assertEquals(remoteTask.getPartitionedSplitCount(), 5);
}
// Add new node
InternalNode additionalNode = new InternalNode("other4", URI.create("http://127.0.0.1:14"), NodeVersion.UNKNOWN, false);
nodeManager.addNode(CONNECTOR_ID, additionalNode);
// Schedule 5 splits in another query. Since the new node does not have any splits, all 5 splits are assigned to the new node
StageExecutionPlan secondPlan = createPlan(createFixedSplitSource(5, TestingSplit::createRemoteSplit));
SqlStageExecution secondStage = createSqlStageExecution(secondPlan, nodeTaskMap);
StageScheduler secondScheduler = getSourcePartitionedScheduler(secondPlan, secondStage, nodeManager, nodeTaskMap, 200);
scheduleResult = secondScheduler.schedule();
assertEffectivelyFinished(scheduleResult, secondScheduler);
assertTrue(scheduleResult.getBlocked().isDone());
assertEquals(scheduleResult.getNewTasks().size(), 1);
assertEquals(secondStage.getAllTasks().size(), 1);
RemoteTask task = secondStage.getAllTasks().get(0);
assertEquals(task.getPartitionedSplitCount(), 5);
firstStage.abort();
secondStage.abort();
}
@Test
public void testNewTaskScheduledWhenChildStageBufferIsUnderutilized()
{
NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService);
// use private node manager so we can add a node later
InMemoryNodeManager nodeManager = new InMemoryNodeManager();
nodeManager.addNode(CONNECTOR_ID,
new InternalNode("other1", URI.create("http://127.0.0.1:11"), NodeVersion.UNKNOWN, false),
new InternalNode("other2", URI.create("http://127.0.0.1:12"), NodeVersion.UNKNOWN, false),
new InternalNode("other3", URI.create("http://127.0.0.1:13"), NodeVersion.UNKNOWN, false));
NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap, new Duration(0, SECONDS)));
StageExecutionPlan plan = createPlan(createFixedSplitSource(500, TestingSplit::createRemoteSplit));
SqlStageExecution stage = createSqlStageExecution(plan, nodeTaskMap);
// setting under utilized child output buffer
StageScheduler scheduler = newSourcePartitionedSchedulerAsStageScheduler(
stage,
Iterables.getOnlyElement(plan.getSplitSources().keySet()),
Iterables.getOnlyElement(plan.getSplitSources().values()),
new DynamicSplitPlacementPolicy(nodeScheduler.createNodeSelector(session, Optional.of(CONNECTOR_ID)), stage::getAllTasks),
500,
new DynamicFilterService(metadata, typeOperators, new DynamicFilterConfig()),
() -> false);
// the queues of 3 running nodes should be full
ScheduleResult scheduleResult = scheduler.schedule();
assertEquals(scheduleResult.getBlockedReason().get(), SPLIT_QUEUES_FULL);
assertEquals(scheduleResult.getNewTasks().size(), 3);
assertEquals(scheduleResult.getSplitsScheduled(), 300);
// new node added - the pending splits should go to it since the child tasks are not blocked
nodeManager.addNode(CONNECTOR_ID, new InternalNode("other4", URI.create("http://127.0.0.4:14"), NodeVersion.UNKNOWN, false));
scheduleResult = scheduler.schedule();
assertEquals(scheduleResult.getBlockedReason().get(), SPLIT_QUEUES_FULL); // split queue is full but still the source task creation isn't blocked
assertEquals(scheduleResult.getNewTasks().size(), 1);
assertEquals(scheduleResult.getSplitsScheduled(), 100);
}
@Test
public void testNoNewTaskScheduledWhenChildStageBufferIsOverutilized()
{
NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService);
// use private node manager so we can add a node later
InMemoryNodeManager nodeManager = new InMemoryNodeManager();
nodeManager.addNode(CONNECTOR_ID,
new InternalNode("other1", URI.create("http://127.0.0.1:11"), NodeVersion.UNKNOWN, false),
new InternalNode("other2", URI.create("http://127.0.0.1:12"), NodeVersion.UNKNOWN, false),
new InternalNode("other3", URI.create("http://127.0.0.1:13"), NodeVersion.UNKNOWN, false));
NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap, new Duration(0, SECONDS)));
StageExecutionPlan plan = createPlan(createFixedSplitSource(400, TestingSplit::createRemoteSplit));
SqlStageExecution stage = createSqlStageExecution(plan, nodeTaskMap);
// setting over utilized child output buffer
StageScheduler scheduler = newSourcePartitionedSchedulerAsStageScheduler(
stage,
Iterables.getOnlyElement(plan.getSplitSources().keySet()),
Iterables.getOnlyElement(plan.getSplitSources().values()),
new DynamicSplitPlacementPolicy(nodeScheduler.createNodeSelector(session, Optional.of(CONNECTOR_ID)), stage::getAllTasks),
400,
new DynamicFilterService(metadata, typeOperators, new DynamicFilterConfig()),
() -> true);
// the queues of 3 running nodes should be full
ScheduleResult scheduleResult = scheduler.schedule();
assertEquals(scheduleResult.getBlockedReason().get(), SPLIT_QUEUES_FULL);
assertEquals(scheduleResult.getNewTasks().size(), 3);
assertEquals(scheduleResult.getSplitsScheduled(), 300);
// new node added but 1 child's output buffer is overutilized - so lockdown the tasks
nodeManager.addNode(CONNECTOR_ID, new InternalNode("other4", URI.create("http://127.0.0.4:14"), NodeVersion.UNKNOWN, false));
scheduleResult = scheduler.schedule();
assertEquals(scheduleResult.getBlockedReason().get(), SPLIT_QUEUES_FULL);
assertEquals(scheduleResult.getNewTasks().size(), 0);
assertEquals(scheduleResult.getSplitsScheduled(), 0);
}
@Test
public void testDynamicFiltersUnblockedOnBlockedBuildSource()
{
StageExecutionPlan plan = createPlan(createBlockedSplitSource());
NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService);
SqlStageExecution stage = createSqlStageExecution(plan, nodeTaskMap);
NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap));
DynamicFilterService dynamicFilterService = new DynamicFilterService(metadata, typeOperators, new DynamicFilterConfig());
dynamicFilterService.registerQuery(
QUERY_ID,
TEST_SESSION,
ImmutableSet.of(DYNAMIC_FILTER_ID),
ImmutableSet.of(DYNAMIC_FILTER_ID),
ImmutableSet.of(DYNAMIC_FILTER_ID));
StageScheduler scheduler = newSourcePartitionedSchedulerAsStageScheduler(
stage,
Iterables.getOnlyElement(plan.getSplitSources().keySet()),
Iterables.getOnlyElement(plan.getSplitSources().values()),
new DynamicSplitPlacementPolicy(nodeScheduler.createNodeSelector(session, Optional.of(CONNECTOR_ID)), stage::getAllTasks),
2,
dynamicFilterService,
() -> true);
SymbolAllocator symbolAllocator = new SymbolAllocator();
Symbol symbol = symbolAllocator.newSymbol("DF_SYMBOL1", BIGINT);
DynamicFilter dynamicFilter = dynamicFilterService.createDynamicFilter(
QUERY_ID,
ImmutableList.of(new DynamicFilters.Descriptor(DYNAMIC_FILTER_ID, symbol.toSymbolReference())),
ImmutableMap.of(symbol, new TestingColumnHandle("probeColumnA")),
symbolAllocator.getTypes());
// make sure dynamic filter is initially blocked
assertFalse(dynamicFilter.isBlocked().isDone());
// make sure dynamic filter is unblocked due to build side source tasks being blocked
ScheduleResult scheduleResult = scheduler.schedule();
assertTrue(dynamicFilter.isBlocked().isDone());
// make sure that an extra task for collecting dynamic filters was created
assertEquals(scheduleResult.getNewTasks().size(), 1);
assertEquals(scheduleResult.getSplitsScheduled(), 0);
}
private static void assertPartitionedSplitCount(SqlStageExecution stage, int expectedPartitionedSplitCount)
{
assertEquals(stage.getAllTasks().stream().mapToInt(RemoteTask::getPartitionedSplitCount).sum(), expectedPartitionedSplitCount);
}
private static void assertEffectivelyFinished(ScheduleResult scheduleResult, StageScheduler scheduler)
{
if (scheduleResult.isFinished()) {
assertTrue(scheduleResult.getBlocked().isDone());
return;
}
assertTrue(scheduleResult.getBlocked().isDone());
ScheduleResult nextScheduleResult = scheduler.schedule();
assertTrue(nextScheduleResult.isFinished());
assertTrue(nextScheduleResult.getBlocked().isDone());
assertEquals(nextScheduleResult.getNewTasks().size(), 0);
assertEquals(nextScheduleResult.getSplitsScheduled(), 0);
}
private StageScheduler getSourcePartitionedScheduler(
StageExecutionPlan plan,
SqlStageExecution stage,
InternalNodeManager nodeManager,
NodeTaskMap nodeTaskMap,
int splitBatchSize)
{
NodeSchedulerConfig nodeSchedulerConfig = new NodeSchedulerConfig()
.setIncludeCoordinator(false)
.setMaxSplitsPerNode(20)
.setMaxPendingSplitsPerTask(0);
NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, nodeSchedulerConfig, nodeTaskMap));
PlanNodeId sourceNode = Iterables.getOnlyElement(plan.getSplitSources().keySet());
SplitSource splitSource = Iterables.getOnlyElement(plan.getSplitSources().values());
SplitPlacementPolicy placementPolicy = new DynamicSplitPlacementPolicy(nodeScheduler.createNodeSelector(session, Optional.of(splitSource.getCatalogName())), stage::getAllTasks);
return newSourcePartitionedSchedulerAsStageScheduler(
stage,
sourceNode,
splitSource,
placementPolicy,
splitBatchSize,
new DynamicFilterService(metadata, typeOperators, new DynamicFilterConfig()),
() -> false);
}
private static StageExecutionPlan createPlan(ConnectorSplitSource splitSource)
{
Symbol symbol = new Symbol("column");
Symbol buildSymbol = new Symbol("buildColumn");
// table scan with splitCount splits
PlanNodeId tableScanNodeId = new PlanNodeId("plan_id");
TableScanNode tableScan = TableScanNode.newInstance(
tableScanNodeId,
TEST_TABLE_HANDLE,
ImmutableList.of(symbol),
ImmutableMap.of(symbol, new TestingColumnHandle("column")),
false,
Optional.empty());
FilterNode filterNode = new FilterNode(
new PlanNodeId("filter_node_id"),
tableScan,
createDynamicFilterExpression(createTestMetadataManager(), DYNAMIC_FILTER_ID, VARCHAR, symbol.toSymbolReference()));
RemoteSourceNode remote = new RemoteSourceNode(new PlanNodeId("remote_id"), new PlanFragmentId("plan_fragment_id"), ImmutableList.of(buildSymbol), Optional.empty(), REPLICATE);
PlanFragment testFragment = new PlanFragment(
new PlanFragmentId("plan_id"),
new JoinNode(new PlanNodeId("join_id"),
INNER,
filterNode,
remote,
ImmutableList.of(),
tableScan.getOutputSymbols(),
remote.getOutputSymbols(),
false,
Optional.empty(),
Optional.empty(),
Optional.empty(),
Optional.empty(),
Optional.empty(),
ImmutableMap.of(DYNAMIC_FILTER_ID, buildSymbol),
Optional.empty()),
ImmutableMap.of(symbol, VARCHAR),
SOURCE_DISTRIBUTION,
ImmutableList.of(tableScanNodeId),
new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol)),
ungroupedExecution(),
StatsAndCosts.empty(),
Optional.empty());
return new StageExecutionPlan(
testFragment,
ImmutableMap.of(tableScanNodeId, new ConnectorAwareSplitSource(CONNECTOR_ID, splitSource)),
ImmutableList.of(),
ImmutableMap.of(tableScanNodeId, new TableInfo(new QualifiedObjectName("test", "test", "test"), TupleDomain.all())));
}
private static ConnectorSplitSource createBlockedSplitSource()
{
return new ConnectorSplitSource()
{
@Override
public CompletableFuture<ConnectorSplitBatch> getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize)
{
return new CompletableFuture<>();
}
@Override
public void close()
{
}
@Override
public boolean isFinished()
{
return false;
}
};
}
private static ConnectorSplitSource createFixedSplitSource(int splitCount, Supplier<ConnectorSplit> splitFactory)
{
ImmutableList.Builder<ConnectorSplit> splits = ImmutableList.builder();
for (int i = 0; i < splitCount; i++) {
splits.add(splitFactory.get());
}
return new FixedSplitSource(splits.build());
}
private SqlStageExecution createSqlStageExecution(StageExecutionPlan tableScanPlan, NodeTaskMap nodeTaskMap)
{
StageId stageId = new StageId(QUERY_ID, 0);
SqlStageExecution stage = SqlStageExecution.createSqlStageExecution(stageId,
tableScanPlan.getFragment(),
tableScanPlan.getTables(),
new MockRemoteTaskFactory(queryExecutor, scheduledExecutor),
TEST_SESSION,
true,
nodeTaskMap,
queryExecutor,
new NoOpFailureDetector(),
new DynamicFilterService(metadata, typeOperators, new DynamicFilterConfig()),
new SplitSchedulerStats());
stage.setOutputBuffers(createInitialEmptyOutputBuffers(PARTITIONED)
.withBuffer(OUT, 0)
.withNoMoreBufferIds());
return stage;
}
private static class QueuedSplitSource
implements ConnectorSplitSource
{
private final Supplier<ConnectorSplit> splitFactory;
private final LinkedBlockingQueue<ConnectorSplit> queue = new LinkedBlockingQueue<>();
private CompletableFuture<?> notEmptyFuture = new CompletableFuture<>();
private boolean closed;
public QueuedSplitSource(Supplier<ConnectorSplit> splitFactory)
{
this.splitFactory = requireNonNull(splitFactory, "splitFactory is null");
}
synchronized void addSplits(int count)
{
if (closed) {
return;
}
for (int i = 0; i < count; i++) {
queue.add(splitFactory.get());
notEmptyFuture.complete(null);
}
}
@Override
public CompletableFuture<ConnectorSplitBatch> getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize)
{
checkArgument(partitionHandle.equals(NOT_PARTITIONED), "partitionHandle must be NOT_PARTITIONED");
return notEmptyFuture
.thenApply(x -> getBatch(maxSize))
.thenApply(splits -> new ConnectorSplitBatch(splits, isFinished()));
}
private synchronized List<ConnectorSplit> getBatch(int maxSize)
{
// take up to maxSize elements from the queue
List<ConnectorSplit> elements = new ArrayList<>(maxSize);
queue.drainTo(elements, maxSize);
// if the queue is empty and the current future is finished, create a new one so
// a new readers can be notified when the queue has elements to read
if (queue.isEmpty() && !closed) {
if (notEmptyFuture.isDone()) {
notEmptyFuture = new CompletableFuture<>();
}
}
return ImmutableList.copyOf(elements);
}
@Override
public synchronized boolean isFinished()
{
return closed && queue.isEmpty();
}
@Override
public synchronized void close()
{
closed = true;
}
}
}