-
Notifications
You must be signed in to change notification settings - Fork 5.3k
/
LocalExecutionPlanner.java
2073 lines (1775 loc) · 100 KB
/
LocalExecutionPlanner.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
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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 com.facebook.presto.sql.planner;
import com.facebook.presto.Session;
import com.facebook.presto.SystemSessionProperties;
import com.facebook.presto.execution.SharedBuffer;
import com.facebook.presto.execution.TaskManagerConfig;
import com.facebook.presto.index.IndexManager;
import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.metadata.Signature;
import com.facebook.presto.operator.AggregationOperator.AggregationOperatorFactory;
import com.facebook.presto.operator.CursorProcessor;
import com.facebook.presto.operator.DeleteOperator.DeleteOperatorFactory;
import com.facebook.presto.operator.DriverFactory;
import com.facebook.presto.operator.ExchangeClientSupplier;
import com.facebook.presto.operator.ExchangeOperator.ExchangeOperatorFactory;
import com.facebook.presto.operator.FilterAndProjectOperator;
import com.facebook.presto.operator.FilterFunction;
import com.facebook.presto.operator.FilterFunctions;
import com.facebook.presto.operator.GenericCursorProcessor;
import com.facebook.presto.operator.GenericPageProcessor;
import com.facebook.presto.operator.HashAggregationOperator.HashAggregationOperatorFactory;
import com.facebook.presto.operator.HashBuilderOperator.HashBuilderOperatorFactory;
import com.facebook.presto.operator.HashPartitionMaskOperator.HashPartitionMaskOperatorFactory;
import com.facebook.presto.operator.HashSemiJoinOperator.HashSemiJoinOperatorFactory;
import com.facebook.presto.operator.InMemoryExchange;
import com.facebook.presto.operator.JoinOperatorFactory;
import com.facebook.presto.operator.LimitOperator.LimitOperatorFactory;
import com.facebook.presto.operator.LocalPlannerAware;
import com.facebook.presto.operator.LookupJoinOperators;
import com.facebook.presto.operator.LookupOuterOperator.OuterLookupSourceSupplier;
import com.facebook.presto.operator.LookupSourceSupplier;
import com.facebook.presto.operator.MarkDistinctOperator.MarkDistinctOperatorFactory;
import com.facebook.presto.operator.MetadataDeleteOperator.MetadataDeleteOperatorFactory;
import com.facebook.presto.operator.NestedLoopJoinPagesSupplier;
import com.facebook.presto.operator.OperatorFactory;
import com.facebook.presto.operator.OrderByOperator.OrderByOperatorFactory;
import com.facebook.presto.operator.OutputFactory;
import com.facebook.presto.operator.PageProcessor;
import com.facebook.presto.operator.ParallelHashBuilder;
import com.facebook.presto.operator.PartitionFunction;
import com.facebook.presto.operator.PartitionedOutputOperator.PartitionedOutputFactory;
import com.facebook.presto.operator.ProjectionFunction;
import com.facebook.presto.operator.ProjectionFunctions;
import com.facebook.presto.operator.RowNumberOperator;
import com.facebook.presto.operator.SampleOperator.SampleOperatorFactory;
import com.facebook.presto.operator.ScanFilterAndProjectOperator;
import com.facebook.presto.operator.SetBuilderOperator.SetBuilderOperatorFactory;
import com.facebook.presto.operator.SetBuilderOperator.SetSupplier;
import com.facebook.presto.operator.SourceOperatorFactory;
import com.facebook.presto.operator.TableScanOperator.TableScanOperatorFactory;
import com.facebook.presto.operator.TaskOutputOperator.TaskOutputFactory;
import com.facebook.presto.operator.TopNOperator.TopNOperatorFactory;
import com.facebook.presto.operator.TopNRowNumberOperator;
import com.facebook.presto.operator.ValuesOperator.ValuesOperatorFactory;
import com.facebook.presto.operator.WindowFunctionDefinition;
import com.facebook.presto.operator.WindowOperator.WindowOperatorFactory;
import com.facebook.presto.operator.aggregation.AccumulatorFactory;
import com.facebook.presto.operator.index.DynamicTupleFilterFactory;
import com.facebook.presto.operator.index.FieldSetFilteringRecordSet;
import com.facebook.presto.operator.index.IndexBuildDriverFactoryProvider;
import com.facebook.presto.operator.index.IndexJoinLookupStats;
import com.facebook.presto.operator.index.IndexLookupSourceSupplier;
import com.facebook.presto.operator.index.IndexSourceOperator;
import com.facebook.presto.operator.window.FrameInfo;
import com.facebook.presto.operator.window.WindowFunctionSupplier;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ConnectorIndex;
import com.facebook.presto.spi.PageBuilder;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.RecordSet;
import com.facebook.presto.spi.block.SortOrder;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.split.MappedRecordSet;
import com.facebook.presto.split.PageSinkManager;
import com.facebook.presto.split.PageSourceProvider;
import com.facebook.presto.sql.gen.ExpressionCompiler;
import com.facebook.presto.sql.parser.SqlParser;
import com.facebook.presto.sql.planner.optimizations.IndexJoinOptimizer;
import com.facebook.presto.sql.planner.plan.AggregationNode;
import com.facebook.presto.sql.planner.plan.AggregationNode.Step;
import com.facebook.presto.sql.planner.plan.DeleteNode;
import com.facebook.presto.sql.planner.plan.DistinctLimitNode;
import com.facebook.presto.sql.planner.plan.FilterNode;
import com.facebook.presto.sql.planner.plan.IndexJoinNode;
import com.facebook.presto.sql.planner.plan.IndexSourceNode;
import com.facebook.presto.sql.planner.plan.JoinNode;
import com.facebook.presto.sql.planner.plan.LimitNode;
import com.facebook.presto.sql.planner.plan.MarkDistinctNode;
import com.facebook.presto.sql.planner.plan.MetadataDeleteNode;
import com.facebook.presto.sql.planner.plan.OutputNode;
import com.facebook.presto.sql.planner.plan.PlanNode;
import com.facebook.presto.sql.planner.plan.PlanVisitor;
import com.facebook.presto.sql.planner.plan.ProjectNode;
import com.facebook.presto.sql.planner.plan.RemoteSourceNode;
import com.facebook.presto.sql.planner.plan.RowNumberNode;
import com.facebook.presto.sql.planner.plan.SampleNode;
import com.facebook.presto.sql.planner.plan.SemiJoinNode;
import com.facebook.presto.sql.planner.plan.SortNode;
import com.facebook.presto.sql.planner.plan.TableFinishNode;
import com.facebook.presto.sql.planner.plan.TableScanNode;
import com.facebook.presto.sql.planner.plan.TableWriterNode;
import com.facebook.presto.sql.planner.plan.TableWriterNode.DeleteHandle;
import com.facebook.presto.sql.planner.plan.TopNNode;
import com.facebook.presto.sql.planner.plan.TopNRowNumberNode;
import com.facebook.presto.sql.planner.plan.UnionNode;
import com.facebook.presto.sql.planner.plan.UnnestNode;
import com.facebook.presto.sql.planner.plan.ValuesNode;
import com.facebook.presto.sql.planner.plan.WindowNode;
import com.facebook.presto.sql.planner.plan.WindowNode.Frame;
import com.facebook.presto.sql.relational.ConstantExpression;
import com.facebook.presto.sql.relational.InputReferenceExpression;
import com.facebook.presto.sql.relational.RowExpression;
import com.facebook.presto.sql.relational.SqlToRowExpressionTranslator;
import com.facebook.presto.sql.tree.BooleanLiteral;
import com.facebook.presto.sql.tree.Expression;
import com.facebook.presto.sql.tree.ExpressionTreeRewriter;
import com.facebook.presto.sql.tree.FunctionCall;
import com.facebook.presto.sql.tree.QualifiedNameReference;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableMap.Builder;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableSetMultimap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Multimap;
import com.google.common.collect.Ordering;
import com.google.common.collect.SetMultimap;
import com.google.common.primitives.Ints;
import io.airlift.log.Logger;
import io.airlift.slice.Slice;
import io.airlift.units.DataSize;
import javax.annotation.Nullable;
import javax.inject.Inject;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalInt;
import java.util.Set;
import java.util.function.Function;
import static com.facebook.presto.SystemSessionProperties.getTaskAggregationConcurrency;
import static com.facebook.presto.SystemSessionProperties.getTaskHashBuildConcurrency;
import static com.facebook.presto.SystemSessionProperties.getTaskJoinConcurrency;
import static com.facebook.presto.SystemSessionProperties.getTaskWriterCount;
import static com.facebook.presto.metadata.FunctionKind.SCALAR;
import static com.facebook.presto.operator.DistinctLimitOperator.DistinctLimitOperatorFactory;
import static com.facebook.presto.operator.InMemoryExchangeSourceOperator.InMemoryExchangeSourceOperatorFactory.createBroadcastDistribution;
import static com.facebook.presto.operator.InMemoryExchangeSourceOperator.InMemoryExchangeSourceOperatorFactory.createRandomDistribution;
import static com.facebook.presto.operator.NestedLoopBuildOperator.NestedLoopBuildOperatorFactory;
import static com.facebook.presto.operator.NestedLoopJoinOperator.NestedLoopJoinOperatorFactory;
import static com.facebook.presto.operator.TableFinishOperator.TableFinishOperatorFactory;
import static com.facebook.presto.operator.TableFinishOperator.TableFinisher;
import static com.facebook.presto.operator.TableWriterOperator.TableWriterOperatorFactory;
import static com.facebook.presto.operator.UnnestOperator.UnnestOperatorFactory;
import static com.facebook.presto.operator.WindowFunctionDefinition.window;
import static com.facebook.presto.spi.StandardErrorCode.COMPILER_ERROR;
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.BooleanType.BOOLEAN;
import static com.facebook.presto.spi.type.TypeUtils.writeNativeValue;
import static com.facebook.presto.sql.analyzer.ExpressionAnalyzer.getExpressionTypes;
import static com.facebook.presto.sql.analyzer.ExpressionAnalyzer.getExpressionTypesFromInput;
import static com.facebook.presto.sql.planner.plan.JoinNode.Type.FULL;
import static com.facebook.presto.sql.planner.plan.JoinNode.Type.RIGHT;
import static com.facebook.presto.sql.planner.plan.TableWriterNode.CreateHandle;
import static com.facebook.presto.sql.planner.plan.TableWriterNode.InsertHandle;
import static com.facebook.presto.sql.planner.plan.TableWriterNode.WriterTarget;
import static com.facebook.presto.util.ImmutableCollectors.toImmutableList;
import static com.facebook.presto.util.ImmutableCollectors.toImmutableSet;
import static com.google.common.base.Functions.forMap;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.collect.Iterables.concat;
import static java.lang.Boolean.TRUE;
import static java.lang.String.format;
import static java.util.Collections.singleton;
import static java.util.Objects.requireNonNull;
public class LocalExecutionPlanner
{
private static final Logger log = Logger.get(LocalExecutionPlanner.class);
private final Metadata metadata;
private final SqlParser sqlParser;
private final PageSourceProvider pageSourceProvider;
private final IndexManager indexManager;
private final PageSinkManager pageSinkManager;
private final ExchangeClientSupplier exchangeClientSupplier;
private final ExpressionCompiler compiler;
private final boolean interpreterEnabled;
private final DataSize maxIndexMemorySize;
private final IndexJoinLookupStats indexJoinLookupStats;
private final DataSize maxPartialAggregationMemorySize;
@Inject
public LocalExecutionPlanner(
Metadata metadata,
SqlParser sqlParser,
PageSourceProvider pageSourceProvider,
IndexManager indexManager,
PageSinkManager pageSinkManager,
ExchangeClientSupplier exchangeClientSupplier,
ExpressionCompiler compiler,
IndexJoinLookupStats indexJoinLookupStats,
CompilerConfig compilerConfig,
TaskManagerConfig taskManagerConfig)
{
requireNonNull(compilerConfig, "compilerConfig is null");
this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null");
this.indexManager = requireNonNull(indexManager, "indexManager is null");
this.exchangeClientSupplier = exchangeClientSupplier;
this.metadata = requireNonNull(metadata, "metadata is null");
this.sqlParser = requireNonNull(sqlParser, "sqlParser is null");
this.pageSinkManager = requireNonNull(pageSinkManager, "pageSinkManager is null");
this.compiler = requireNonNull(compiler, "compiler is null");
this.indexJoinLookupStats = requireNonNull(indexJoinLookupStats, "indexJoinLookupStats is null");
this.maxIndexMemorySize = requireNonNull(taskManagerConfig, "taskManagerConfig is null").getMaxIndexMemoryUsage();
this.maxPartialAggregationMemorySize = taskManagerConfig.getMaxPartialAggregationMemoryUsage();
interpreterEnabled = compilerConfig.isInterpreterEnabled();
}
public LocalExecutionPlan plan(
Session session,
PlanNode plan,
List<Symbol> outputLayout,
Map<Symbol, Type> types,
Optional<PartitionFunctionBinding> partitionFunctionBinding,
SharedBuffer sharedBuffer,
boolean singleNode,
boolean allowLocalParallel)
{
if (!partitionFunctionBinding.isPresent()) {
return plan(session, plan, outputLayout, types, new TaskOutputFactory(sharedBuffer), singleNode, allowLocalParallel);
}
PartitionFunctionBinding functionBinding = partitionFunctionBinding.get();
// We can convert the symbols directly into channels, because the root must be a sink and therefore the layout is fixed
List<Integer> partitionChannels;
List<Type> partitionChannelTypes;
if (functionBinding.getHashColumn().isPresent()) {
partitionChannels = ImmutableList.of(outputLayout.indexOf(functionBinding.getHashColumn().get()));
partitionChannelTypes = ImmutableList.of(BIGINT);
}
else {
partitionChannels = functionBinding.getPartitioningColumns().stream()
.map(outputLayout::indexOf)
.collect(toImmutableList());
partitionChannelTypes = functionBinding.getPartitioningColumns().stream()
.map(types::get)
.collect(toImmutableList());
}
PartitionFunction partitionFunction = functionBinding.getFunctionHandle().createPartitionFunction(functionBinding, partitionChannelTypes);
OptionalInt nullChannel = OptionalInt.empty();
if (functionBinding.isReplicateNulls()) {
checkArgument(functionBinding.getPartitioningColumns().size() == 1);
nullChannel = OptionalInt.of(outputLayout.indexOf(Iterables.getOnlyElement(functionBinding.getPartitioningColumns())));
}
return plan(
session,
plan,
outputLayout,
types,
new PartitionedOutputFactory(partitionFunction, partitionChannels, nullChannel, sharedBuffer),
singleNode,
allowLocalParallel);
}
public LocalExecutionPlan plan(Session session,
PlanNode plan,
List<Symbol> outputLayout,
Map<Symbol, Type> types,
OutputFactory outputOperatorFactory,
boolean singleNode,
boolean allowLocalParallel)
{
LocalExecutionPlanContext context = new LocalExecutionPlanContext(session, types, singleNode, allowLocalParallel);
PhysicalOperation physicalOperation = enforceLayout(outputLayout, context, plan.accept(new Visitor(session), context));
DriverFactory driverFactory = new DriverFactory(
context.isInputDriver(),
true,
ImmutableList.<OperatorFactory>builder()
.addAll(physicalOperation.getOperatorFactories())
.add(outputOperatorFactory.createOutputOperator(context.getNextOperatorId(), physicalOperation.getTypes()))
.build(),
context.getDriverInstanceCount());
context.addDriverFactory(driverFactory);
addLookupOuterDrivers(context);
// notify operator factories that planning has completed
context.getDriverFactories().stream()
.map(DriverFactory::getOperatorFactories)
.flatMap(List::stream)
.filter(LocalPlannerAware.class::isInstance)
.map(LocalPlannerAware.class::cast)
.forEach(LocalPlannerAware::localPlannerComplete);
return new LocalExecutionPlan(context.getDriverFactories());
}
private static void addLookupOuterDrivers(LocalExecutionPlanContext context)
{
// For an outer join on the lookup side (RIGHT or FULL) add an additional
// driver to output the unused rows in the lookup source
for (DriverFactory factory : context.getDriverFactories()) {
List<OperatorFactory> operatorFactories = factory.getOperatorFactories();
for (int i = 0; i < operatorFactories.size(); i++) {
OperatorFactory operatorFactory = operatorFactories.get(i);
if (!(operatorFactory instanceof JoinOperatorFactory)) {
continue;
}
JoinOperatorFactory lookupJoin = (JoinOperatorFactory) operatorFactory;
Optional<OperatorFactory> outerOperatorFactory = lookupJoin.createOuterOperatorFactory();
if (outerOperatorFactory.isPresent()) {
// Add a new driver to output the unmatched rows in an outer join.
// We duplicate all of the factories above the JoinOperator (the ones reading from the joins),
// and replace the JoinOperator with the OuterOperator (the one that produces unmatched rows).
ImmutableList.Builder<OperatorFactory> newOperators = ImmutableList.builder();
newOperators.add(outerOperatorFactory.get());
operatorFactories.subList(i + 1, operatorFactories.size()).stream()
.map(OperatorFactory::duplicate)
.forEach(newOperators::add);
context.addDriverFactory(new DriverFactory(false, factory.isOutputDriver(), newOperators.build()));
}
}
}
}
private PhysicalOperation enforceLayout(List<Symbol> outputLayout, LocalExecutionPlanContext context, PhysicalOperation physicalOperation)
{
// are the symbols of the source in the same order as the sink expects?
boolean projectionMatchesOutput = physicalOperation.getLayout()
.entrySet().stream()
.sorted(Ordering.<Integer>natural().onResultOf(Map.Entry::getValue))
.map(Map.Entry::getKey)
.collect(toImmutableList())
.equals(outputLayout);
if (!projectionMatchesOutput) {
IdentityProjectionInfo projectionInfo = computeIdentityProjectionInfo(outputLayout, physicalOperation.getLayout(), context.getTypes());
List<RowExpression> projections = projectionInfo.getProjections();
OperatorFactory operatorFactory = new FilterAndProjectOperator.FilterAndProjectOperatorFactory(
context.getNextOperatorId(),
compiler.compilePageProcessor(trueExpression(), projections),
projections.stream()
.map(RowExpression::getType)
.collect(toImmutableList()));
// NOTE: the generated output layout may not be completely accurate if the same field was projected as multiple inputs.
// However, this should not affect the operation of the sink.
physicalOperation = new PhysicalOperation(operatorFactory, projectionInfo.getOutputLayout(), physicalOperation);
}
return physicalOperation;
}
private static RowExpression trueExpression()
{
return new ConstantExpression(TRUE, BOOLEAN);
}
private static class LocalExecutionPlanContext
{
private final Session session;
private final Map<Symbol, Type> types;
private final boolean singleNode;
private final boolean allowLocalParallel;
private final List<DriverFactory> driverFactories;
private final Optional<IndexSourceContext> indexSourceContext;
private int nextOperatorId;
private boolean inputDriver = true;
private int driverInstanceCount = 1;
public LocalExecutionPlanContext(Session session, Map<Symbol, Type> types, boolean singleNode, boolean allowLocalParallel)
{
this(session, types, singleNode, allowLocalParallel, new ArrayList<>(), Optional.empty());
}
private LocalExecutionPlanContext(
Session session,
Map<Symbol, Type> types,
boolean singleNode,
boolean allowLocalParallel,
List<DriverFactory> driverFactories,
Optional<IndexSourceContext> indexSourceContext)
{
this.session = session;
this.types = types;
this.singleNode = singleNode;
this.allowLocalParallel = allowLocalParallel;
this.driverFactories = driverFactories;
this.indexSourceContext = indexSourceContext;
}
public void addDriverFactory(DriverFactory driverFactory)
{
driverFactories.add(requireNonNull(driverFactory, "driverFactory is null"));
}
private List<DriverFactory> getDriverFactories()
{
return ImmutableList.copyOf(driverFactories);
}
public Session getSession()
{
return session;
}
public Map<Symbol, Type> getTypes()
{
return types;
}
public Optional<IndexSourceContext> getIndexSourceContext()
{
return indexSourceContext;
}
private int getNextOperatorId()
{
return nextOperatorId++;
}
private boolean isInputDriver()
{
return inputDriver;
}
private void setInputDriver(boolean inputDriver)
{
this.inputDriver = inputDriver;
}
public LocalExecutionPlanContext createSubContext()
{
checkState(!indexSourceContext.isPresent(), "index build plan can not have sub-contexts");
return new LocalExecutionPlanContext(session, types, singleNode, allowLocalParallel, driverFactories, indexSourceContext);
}
public LocalExecutionPlanContext createIndexSourceSubContext(IndexSourceContext indexSourceContext)
{
return new LocalExecutionPlanContext(session, types, true, false, driverFactories, Optional.of(indexSourceContext));
}
public boolean isSingleNode()
{
return singleNode;
}
public boolean isAllowLocalParallel()
{
return allowLocalParallel;
}
public int getDriverInstanceCount()
{
return driverInstanceCount;
}
public void setDriverInstanceCount(int driverInstanceCount)
{
checkArgument(driverInstanceCount > 0, "driverInstanceCount must be > 0");
this.driverInstanceCount = driverInstanceCount;
}
}
private static class IndexSourceContext
{
private final SetMultimap<Symbol, Integer> indexLookupToProbeInput;
public IndexSourceContext(SetMultimap<Symbol, Integer> indexLookupToProbeInput)
{
this.indexLookupToProbeInput = ImmutableSetMultimap.copyOf(requireNonNull(indexLookupToProbeInput, "indexLookupToProbeInput is null"));
}
private SetMultimap<Symbol, Integer> getIndexLookupToProbeInput()
{
return indexLookupToProbeInput;
}
}
public static class LocalExecutionPlan
{
private final List<DriverFactory> driverFactories;
public LocalExecutionPlan(List<DriverFactory> driverFactories)
{
this.driverFactories = ImmutableList.copyOf(requireNonNull(driverFactories, "driverFactories is null"));
}
public List<DriverFactory> getDriverFactories()
{
return driverFactories;
}
}
private class Visitor
extends PlanVisitor<LocalExecutionPlanContext, PhysicalOperation>
{
private final Session session;
private Visitor(Session session)
{
this.session = session;
}
@Override
public PhysicalOperation visitRemoteSource(RemoteSourceNode node, LocalExecutionPlanContext context)
{
List<Type> types = getSourceOperatorTypes(node, context.getTypes());
OperatorFactory operatorFactory = new ExchangeOperatorFactory(context.getNextOperatorId(), node.getId(), exchangeClientSupplier, types);
return new PhysicalOperation(operatorFactory, makeLayout(node));
}
@Override
public PhysicalOperation visitOutput(OutputNode node, LocalExecutionPlanContext context)
{
return node.getSource().accept(this, context);
}
@Override
public PhysicalOperation visitRowNumber(RowNumberNode node, LocalExecutionPlanContext context)
{
PhysicalOperation source = node.getSource().accept(this, context);
List<Symbol> partitionBySymbols = node.getPartitionBy();
List<Integer> partitionChannels = getChannelsForSymbols(partitionBySymbols, source.getLayout());
List<Type> partitionTypes = partitionChannels.stream()
.map(channel -> source.getTypes().get(channel))
.collect(toImmutableList());
ImmutableList.Builder<Integer> outputChannels = ImmutableList.builder();
for (int i = 0; i < source.getTypes().size(); i++) {
outputChannels.add(i);
}
// compute the layout of the output from the window operator
ImmutableMap.Builder<Symbol, Integer> outputMappings = ImmutableMap.builder();
outputMappings.putAll(source.getLayout());
// row number function goes in the last channel
int channel = source.getTypes().size();
outputMappings.put(node.getRowNumberSymbol(), channel);
Optional<Integer> hashChannel = node.getHashSymbol().map(channelGetter(source));
OperatorFactory operatorFactory = new RowNumberOperator.RowNumberOperatorFactory(
context.getNextOperatorId(),
source.getTypes(),
outputChannels.build(),
partitionChannels,
partitionTypes,
node.getMaxRowCountPerPartition(),
hashChannel,
10_000);
return new PhysicalOperation(operatorFactory, outputMappings.build(), source);
}
@Override
public PhysicalOperation visitTopNRowNumber(TopNRowNumberNode node, LocalExecutionPlanContext context)
{
PhysicalOperation source = node.getSource().accept(this, context);
List<Symbol> partitionBySymbols = node.getPartitionBy();
List<Integer> partitionChannels = getChannelsForSymbols(partitionBySymbols, source.getLayout());
List<Type> partitionTypes = partitionChannels.stream()
.map(channel -> source.getTypes().get(channel))
.collect(toImmutableList());
List<Symbol> orderBySymbols = node.getOrderBy();
List<Integer> sortChannels = getChannelsForSymbols(orderBySymbols, source.getLayout());
List<SortOrder> sortOrder = orderBySymbols.stream()
.map(symbol -> node.getOrderings().get(symbol))
.collect(toImmutableList());
ImmutableList.Builder<Integer> outputChannels = ImmutableList.builder();
for (int i = 0; i < source.getTypes().size(); i++) {
outputChannels.add(i);
}
// compute the layout of the output from the window operator
ImmutableMap.Builder<Symbol, Integer> outputMappings = ImmutableMap.builder();
outputMappings.putAll(source.getLayout());
if (!node.isPartial() || !partitionChannels.isEmpty()) {
// row number function goes in the last channel
int channel = source.getTypes().size();
outputMappings.put(node.getRowNumberSymbol(), channel);
}
Optional<Integer> hashChannel = node.getHashSymbol().map(channelGetter(source));
OperatorFactory operatorFactory = new TopNRowNumberOperator.TopNRowNumberOperatorFactory(
context.getNextOperatorId(),
source.getTypes(),
outputChannels.build(),
partitionChannels,
partitionTypes,
sortChannels,
sortOrder,
node.getMaxRowCountPerPartition(),
node.isPartial(),
hashChannel,
1000);
return new PhysicalOperation(operatorFactory, makeLayout(node), source);
}
@Override
public PhysicalOperation visitWindow(WindowNode node, LocalExecutionPlanContext context)
{
PhysicalOperation source = node.getSource().accept(this, context);
List<Symbol> partitionBySymbols = node.getPartitionBy();
List<Symbol> orderBySymbols = node.getOrderBy();
List<Integer> partitionChannels = ImmutableList.copyOf(getChannelsForSymbols(partitionBySymbols, source.getLayout()));
List<Integer> preGroupedChannels = ImmutableList.copyOf(getChannelsForSymbols(ImmutableList.copyOf(node.getPrePartitionedInputs()), source.getLayout()));
List<Integer> sortChannels = getChannelsForSymbols(orderBySymbols, source.getLayout());
List<SortOrder> sortOrder = orderBySymbols.stream()
.map(symbol -> node.getOrderings().get(symbol))
.collect(toImmutableList());
Optional<Integer> frameStartChannel = Optional.empty();
Optional<Integer> frameEndChannel = Optional.empty();
Frame frame = node.getFrame();
if (frame.getStartValue().isPresent()) {
frameStartChannel = Optional.of(source.getLayout().get(frame.getStartValue().get()));
}
if (frame.getEndValue().isPresent()) {
frameEndChannel = Optional.of(source.getLayout().get(frame.getEndValue().get()));
}
ImmutableList.Builder<Integer> outputChannels = ImmutableList.builder();
for (int i = 0; i < source.getTypes().size(); i++) {
outputChannels.add(i);
}
ImmutableList.Builder<WindowFunctionDefinition> windowFunctionsBuilder = ImmutableList.builder();
ImmutableList.Builder<Symbol> windowFunctionOutputSymbolsBuilder = ImmutableList.builder();
for (Map.Entry<Symbol, FunctionCall> entry : node.getWindowFunctions().entrySet()) {
ImmutableList.Builder<Integer> arguments = ImmutableList.builder();
for (Expression argument : entry.getValue().getArguments()) {
Symbol argumentSymbol = Symbol.fromQualifiedName(((QualifiedNameReference) argument).getName());
arguments.add(source.getLayout().get(argumentSymbol));
}
Symbol symbol = entry.getKey();
Signature signature = node.getSignatures().get(symbol);
WindowFunctionSupplier windowFunctionSupplier = metadata.getFunctionRegistry().getWindowFunctionImplementation(signature);
Type type = metadata.getType(signature.getReturnType());
windowFunctionsBuilder.add(window(windowFunctionSupplier, type, arguments.build()));
windowFunctionOutputSymbolsBuilder.add(symbol);
}
List<Symbol> windowFunctionOutputSymbols = windowFunctionOutputSymbolsBuilder.build();
List<WindowFunctionDefinition> windowFunctions = windowFunctionsBuilder.build();
// compute the layout of the output from the window operator
ImmutableMap.Builder<Symbol, Integer> outputMappings = ImmutableMap.builder();
for (Symbol symbol : node.getSource().getOutputSymbols()) {
outputMappings.put(symbol, source.getLayout().get(symbol));
}
// window functions go in remaining channels starting after the last channel from the source operator, one per channel
int channel = source.getTypes().size();
for (Symbol symbol : windowFunctionOutputSymbols) {
outputMappings.put(symbol, channel);
channel++;
}
OperatorFactory operatorFactory = new WindowOperatorFactory(
context.getNextOperatorId(),
source.getTypes(),
outputChannels.build(),
windowFunctions,
partitionChannels,
preGroupedChannels,
sortChannels,
sortOrder,
node.getPreSortedOrderPrefix(),
new FrameInfo(frame.getType(), frame.getStartType(), frameStartChannel, frame.getEndType(), frameEndChannel),
10_000);
return new PhysicalOperation(operatorFactory, outputMappings.build(), source);
}
@Override
public PhysicalOperation visitTopN(TopNNode node, LocalExecutionPlanContext context)
{
PhysicalOperation source = node.getSource().accept(this, context);
List<Symbol> orderBySymbols = node.getOrderBy();
List<Integer> sortChannels = new ArrayList<>();
List<SortOrder> sortOrders = new ArrayList<>();
for (Symbol symbol : orderBySymbols) {
sortChannels.add(source.getLayout().get(symbol));
sortOrders.add(node.getOrderings().get(symbol));
}
OperatorFactory operator = new TopNOperatorFactory(
context.getNextOperatorId(),
source.getTypes(),
(int) node.getCount(),
sortChannels,
sortOrders,
node.isPartial());
return new PhysicalOperation(operator, source.getLayout(), source);
}
@Override
public PhysicalOperation visitSort(SortNode node, LocalExecutionPlanContext context)
{
PhysicalOperation source = node.getSource().accept(this, context);
List<Symbol> orderBySymbols = node.getOrderBy();
List<Integer> orderByChannels = getChannelsForSymbols(orderBySymbols, source.getLayout());
ImmutableList.Builder<SortOrder> sortOrder = ImmutableList.builder();
for (Symbol symbol : orderBySymbols) {
sortOrder.add(node.getOrderings().get(symbol));
}
ImmutableList.Builder<Integer> outputChannels = ImmutableList.builder();
for (int i = 0; i < source.getTypes().size(); i++) {
outputChannels.add(i);
}
OperatorFactory operator = new OrderByOperatorFactory(
context.getNextOperatorId(),
source.getTypes(),
outputChannels.build(),
10_000,
orderByChannels,
sortOrder.build());
return new PhysicalOperation(operator, source.getLayout(), source);
}
@Override
public PhysicalOperation visitLimit(LimitNode node, LocalExecutionPlanContext context)
{
PhysicalOperation source = node.getSource().accept(this, context);
OperatorFactory operatorFactory = new LimitOperatorFactory(context.getNextOperatorId(), source.getTypes(), node.getCount());
return new PhysicalOperation(operatorFactory, source.getLayout(), source);
}
@Override
public PhysicalOperation visitDistinctLimit(DistinctLimitNode node, LocalExecutionPlanContext context)
{
PhysicalOperation source = node.getSource().accept(this, context);
Optional<Integer> hashChannel = node.getHashSymbol().map(channelGetter(source));
List<Integer> distinctChannels = getChannelsForSymbols(node.getDistinctSymbols(), source.getLayout());
OperatorFactory operatorFactory = new DistinctLimitOperatorFactory(
context.getNextOperatorId(),
source.getTypes(),
distinctChannels,
node.getLimit(),
hashChannel);
return new PhysicalOperation(operatorFactory, source.getLayout(), source);
}
@Override
public PhysicalOperation visitAggregation(AggregationNode node, LocalExecutionPlanContext context)
{
if (node.getGroupBy().isEmpty()) {
PhysicalOperation source = node.getSource().accept(this, context);
return planGlobalAggregation(context.getNextOperatorId(), node, source);
}
if (node.getStep() == Step.INTERMEDIATE) {
LocalExecutionPlanContext intermediateContext = context.createSubContext();
intermediateContext.setInputDriver(context.isInputDriver());
PhysicalOperation source = node.getSource().accept(this, intermediateContext);
InMemoryExchange exchange = new InMemoryExchange(source.getTypes());
List<OperatorFactory> factories = ImmutableList.<OperatorFactory>builder()
.addAll(source.getOperatorFactories())
.add(exchange.createSinkFactory(intermediateContext.getNextOperatorId()))
.build();
context.addDriverFactory(new DriverFactory(intermediateContext.isInputDriver(), false, factories));
OperatorFactory exchangeSource = createRandomDistribution(context.getNextOperatorId(), exchange);
source = new PhysicalOperation(exchangeSource, source.getLayout());
return planGroupByAggregation(node, source, context.getNextOperatorId(), Optional.empty());
}
int aggregationConcurrency = getTaskAggregationConcurrency(session);
if (node.getStep() == Step.PARTIAL || !context.isAllowLocalParallel() || context.getDriverInstanceCount() > 1 || aggregationConcurrency <= 1) {
PhysicalOperation source = node.getSource().accept(this, context);
return planGroupByAggregation(node, source, context.getNextOperatorId(), Optional.empty());
}
// create context for parallel operators
LocalExecutionPlanContext parallelContext = context.createSubContext();
parallelContext.setDriverInstanceCount(aggregationConcurrency);
// create context for source operators
LocalExecutionPlanContext sourceContext = parallelContext.createSubContext();
parallelContext.setInputDriver(false);
// plan aggregation source
PhysicalOperation source = node.getSource().accept(this, sourceContext);
// add a broadcast exchange which copies every page into all parallel workers
InMemoryExchange exchange = new InMemoryExchange(source.getTypes(), aggregationConcurrency);
// finish source operator
List<OperatorFactory> factories = ImmutableList.<OperatorFactory>builder()
.addAll(source.getOperatorFactories())
.add(exchange.createSinkFactory(sourceContext.getNextOperatorId()))
.build();
parallelContext.addDriverFactory(new DriverFactory(sourceContext.isInputDriver(), false, factories));
// add broadcast exchange as first parallel operator
OperatorFactory exchangeSource = createBroadcastDistribution(parallelContext.getNextOperatorId(), exchange);
source = new PhysicalOperation(exchangeSource, source.getLayout());
// mask each parallel driver to only see one partition of groups
HashPartitionMaskOperatorFactory hashPartitionMask = new HashPartitionMaskOperatorFactory(
parallelContext.getNextOperatorId(),
aggregationConcurrency,
exchangeSource.getTypes(),
getChannelsForSymbols(ImmutableList.copyOf(node.getMasks().values()), source.getLayout()),
getChannelsForSymbols(ImmutableList.copyOf(node.getGroupBy()), source.getLayout()),
node.getHashSymbol().map(channelGetter(source)));
int defaultMaskChannel = hashPartitionMask.getDefaultMaskChannel();
source = new PhysicalOperation(hashPartitionMask, source.getLayout(), source);
// plan aggregation
PhysicalOperation operation = planGroupByAggregation(node, source, parallelContext.getNextOperatorId(), Optional.of(defaultMaskChannel));
// merge parallel tasks back into a single stream
operation = addInMemoryExchange(context, operation, parallelContext);
return operation;
}
@Override
public PhysicalOperation visitMarkDistinct(MarkDistinctNode node, LocalExecutionPlanContext context)
{
PhysicalOperation source = node.getSource().accept(this, context);
List<Integer> channels = getChannelsForSymbols(node.getDistinctSymbols(), source.getLayout());
Optional<Integer> hashChannel = node.getHashSymbol().map(channelGetter(source));
MarkDistinctOperatorFactory operator = new MarkDistinctOperatorFactory(context.getNextOperatorId(), source.getTypes(), channels, hashChannel);
return new PhysicalOperation(operator, makeLayout(node), source);
}
@Override
public PhysicalOperation visitSample(SampleNode node, LocalExecutionPlanContext context)
{
// For system sample, the splits are already filtered out, so no specific action needs to be taken here
if (node.getSampleType() == SampleNode.Type.SYSTEM) {
return node.getSource().accept(this, context);
}
if (node.getSampleType() == SampleNode.Type.POISSONIZED) {
PhysicalOperation source = node.getSource().accept(this, context);
OperatorFactory operatorFactory = new SampleOperatorFactory(context.getNextOperatorId(), node.getSampleRatio(), node.isRescaled(), source.getTypes());
checkState(node.getSampleWeightSymbol().isPresent(), "sample weight symbol missing");
return new PhysicalOperation(operatorFactory, makeLayout(node), source);
}
throw new UnsupportedOperationException("not yet implemented: " + node);
}
@Override
public PhysicalOperation visitFilter(FilterNode node, LocalExecutionPlanContext context)
{
PlanNode sourceNode = node.getSource();
Expression filterExpression = node.getPredicate();
List<Symbol> outputSymbols = node.getOutputSymbols();
List<Expression> projectionExpressions = outputSymbols.stream()
.map(Symbol::toQualifiedNameReference)
.collect(toImmutableList());
return visitScanFilterAndProject(context, sourceNode, filterExpression, projectionExpressions, outputSymbols);
}
@Override
public PhysicalOperation visitProject(ProjectNode node, LocalExecutionPlanContext context)
{
PlanNode sourceNode;
Expression filterExpression;
if (node.getSource() instanceof FilterNode) {
FilterNode filterNode = (FilterNode) node.getSource();
sourceNode = filterNode.getSource();
filterExpression = filterNode.getPredicate();
}
else {
sourceNode = node.getSource();
filterExpression = BooleanLiteral.TRUE_LITERAL;
}
List<Expression> projectionExpressions = node.getExpressions();
List<Symbol> outputSymbols = node.getOutputSymbols();
return visitScanFilterAndProject(context, sourceNode, filterExpression, projectionExpressions, outputSymbols);
}
private PhysicalOperation visitScanFilterAndProject(
LocalExecutionPlanContext context,
PlanNode sourceNode,
Expression filterExpression,
List<Expression> projectionExpressions,
List<Symbol> outputSymbols)
{
// if source is a table scan we fold it directly into the filter and project
// otherwise we plan it as a normal operator
Map<Symbol, Integer> sourceLayout;
Map<Integer, Type> sourceTypes;
List<ColumnHandle> columns = null;
PhysicalOperation source = null;
if (sourceNode instanceof TableScanNode) {
TableScanNode tableScanNode = (TableScanNode) sourceNode;
// extract the column handles and channel to type mapping
sourceLayout = new LinkedHashMap<>();
sourceTypes = new LinkedHashMap<>();
columns = new ArrayList<>();
int channel = 0;
for (Symbol symbol : tableScanNode.getOutputSymbols()) {
columns.add(tableScanNode.getAssignments().get(symbol));
Integer input = channel;
sourceLayout.put(symbol, input);
Type type = requireNonNull(context.getTypes().get(symbol), format("No type for symbol %s", symbol));
sourceTypes.put(input, type);
channel++;
}
}
else {
// plan source
source = sourceNode.accept(this, context);
sourceLayout = source.getLayout();
sourceTypes = getInputTypes(source.getLayout(), source.getTypes());
}
// build output mapping
ImmutableMap.Builder<Symbol, Integer> outputMappingsBuilder = ImmutableMap.builder();
for (int i = 0; i < outputSymbols.size(); i++) {
Symbol symbol = outputSymbols.get(i);
outputMappingsBuilder.put(symbol, i);
}
Map<Symbol, Integer> outputMappings = outputMappingsBuilder.build();
// compiler uses inputs instead of symbols, so rewrite the expressions first
SymbolToInputRewriter symbolToInputRewriter = new SymbolToInputRewriter(sourceLayout);
Expression rewrittenFilter = ExpressionTreeRewriter.rewriteWith(symbolToInputRewriter, filterExpression);
List<Expression> rewrittenProjections = new ArrayList<>();
for (Expression projection : projectionExpressions) {
rewrittenProjections.add(ExpressionTreeRewriter.rewriteWith(symbolToInputRewriter, projection));
}