Compare commits

...

149 Commits

Author SHA1 Message Date
yan.zhang 045d90a389 Remove iceberg column stats (#4699)
Set ndv=1 is a dangerous behaviour, and maybe optimizer will give you a unexpected plan.
2022-04-07 14:32:56 +08:00
stdpain 28233b0e2e support a faster unique id implement (#4701)
(cherry picked from commit 1b4c54a2d2)
2022-04-06 23:45:36 +08:00
liuyehcf 35573be541 Modify the statistical method of WaitTime of Exchange (#4733) 2022-04-06 16:26:15 +08:00
liuyehcf f0ef8923c3 Add ExecutionTotalTime (#4714) 2022-04-06 16:26:15 +08:00
liuyehcf 6aa5e81a59 Pipeline profile prune (#4695) 2022-04-06 16:26:15 +08:00
liuyehcf bae0287b4a Update profile merge strategy (#4604) 2022-04-06 16:26:15 +08:00
liuyehcf af0259fd88 Fix use destroyed thrift obj (#4641) 2022-04-06 16:26:15 +08:00
liuyehcf f5335c9bea Fix NPE of CTE (#4562) 2022-04-06 16:26:15 +08:00
liuyehcf b0905c7fc8 Fix uninitialized sql_sort_keys (#4631) 2022-04-06 16:26:15 +08:00
liuyehcf a06cae9553 Pipeline Profile add more metrics (#4443) 2022-04-06 16:26:15 +08:00
liuyehcf 62c2375434 Add round/decimal tests (#4561) 2022-04-06 16:26:15 +08:00
liuyehcf 939185b990 Profile add metrics (#4414) 2022-04-06 16:26:15 +08:00
mergify[bot] 8ee336b6c2
Fix meta filelds duplication issue for hudi table (#4732) (#4750)
(cherry picked from commit b5d958a9cd)

Co-authored-by: miomiocat <284487410@qq.com>
2022-04-06 15:49:54 +08:00
stdpain 03813be0fd
[cherry-pick][branch-2.2] Add memory limit check when evaluate expression (#4368) (#4825) 2022-04-06 15:29:11 +08:00
mergify[bot] 2a8d9afbd9
fix #4818: getTypeSize of INVALID_TYPE (#4820) (#4822)
(cherry picked from commit 48fcb5290f)

Co-authored-by: Murphy <96611012+mofeiatwork@users.noreply.github.com>
2022-04-06 13:57:58 +08:00
ricky c8e32d164b
Be crash caused by invalid json root (#4726) (#4790)
The pull request fixes the crash problem caused by an invalid JSON root. Because the JsonPath is refactored in branch-2.2, this bug only happens in branch-2.1. Nevertheless, merge this fix to branch-2.2 to make it more robust.
2022-04-06 10:35:15 +08:00
mergify[bot] 88eb86af21
Only TResultSinkType.MYSQL_PROTOCAL ResultSink can use pipeline (#4729) (#4752) 2022-04-03 00:15:18 +08:00
stdpain 3a891ce57d Fix get wrong function id in UDTF (#4706)
fn.fid for UDF/UDAF/UDTF was always 0. we should use fn.id to load or
save jar libs

(cherry picked from commit ad593d4d6c)
2022-04-02 15:47:21 +08:00
wyb 35e3df87b7
Bugfix: copy the last row in array column replace aggregator (#4709) (#4744)
Currently, the column in replace aggregator ArrayState is the same as the column in the aggregate iterator.
When processing the next chunk in the aggregate iterator, the previous column will be overwritten or reset.
So copy it when processing the last row in the array column replace the aggregator.
2022-04-02 15:03:46 +08:00
miomiocat 1b3d7a335f
Support cleaning meta cache and fix null partition value exception for hudi table (#4599) (#4731) 2022-04-02 14:55:58 +08:00
mergify[bot] 5b5819df96
Skip zero length file in hdfs scan node (#4700) (#4747)
* skip zero length file

* skip zero length file at frontend

(cherry picked from commit 43691b70ac)

Co-authored-by: yan.zhang <dirtysalt1987@gmail.com>
2022-04-02 14:44:21 +08:00
gengjun-git 7c3e4bebc1
BugFix: schema change will not store the tablet to TabletInvertedIndex (#4725) (#4739)
Put tablet to TabletInvertedIndex in onFinished;
2022-04-02 13:52:09 +08:00
mergify[bot] 4ebd4f5a2f
Don't obtain hms event id when the hive_incremental_sync is false (#4190) (#4435)
(cherry picked from commit a9cdb810a2)

Co-authored-by: stephen <91597003+stephen-shelby@users.noreply.github.com>
2022-04-02 11:54:12 +08:00
ricky 4844a5bf2a
Refactor: move JsonParser to separate file (#4586) (#4692) 2022-04-02 10:21:00 +08:00
mergify[bot] e3fa553ac1
fixup merge_partial_bloom_filters (#4685) (#4704) 2022-04-01 20:33:48 +08:00
miomiocat 66b5604d2a
Support create table like statement for iceberg and hudi table (#4607) (#4697) 2022-04-01 16:32:53 +08:00
zihe.liu 95199420c2
Rehash for local shuffle to avoid data skew (#4629) (#4684) 2022-04-01 10:05:52 +08:00
mergify[bot] 4f9bf40d15
Skip checksum for json (#4673) (#4679)
(cherry picked from commit e1b1d4481e)

Co-authored-by: Murphy <96611012+mofeiatwork@users.noreply.github.com>
2022-03-31 22:32:55 +08:00
stdpain 5b569fec0f Fix thirdparty compile error in docker (#4517)
(cherry picked from commit 57475c4a17)
2022-03-31 21:23:22 +08:00
stdpain dd3cc161f3 Add fast fail when local dictionary size exceed max size (#4652)
(cherry picked from commit e6f4333f4a)
2022-03-31 21:22:40 +08:00
mergify[bot] 4cc0174f71
Support sort_nulls_last mode (#4296) (#4653)
(cherry picked from commit 6a1f5a650d)

Co-authored-by: Seaven <seaven_7@qq.com>
2022-03-31 20:06:36 +08:00
Pslydhh 3ff6a5f847
Provide matched num_rows when array's columns is empty (#4442) (#4658)
Provide right num_rows when array's columns is empty.
2022-03-31 19:11:06 +08:00
stdpain 043e7a4a64 fix UDTF crash in pipeline engine (#4634)
TableFunction::open has not been called in pipeline engine, which will
cause BE crash in pipeline engine

(cherry picked from commit d338bbe0fc)
2022-03-31 19:08:34 +08:00
stdpain ce50b2f1fa Add fast fail when collecting dictionary (#4624)
(cherry picked from commit 00af1f2c9d)
2022-03-31 19:07:41 +08:00
kangkaisen 21c68147d7
cherry-pick to 2.2 branch (#4649)
* Return query status for statistic executor (#4636)

* All methods in CacheDictManager shouldn't be blocked (#4621)
2022-03-31 17:17:09 +08:00
ricky be8011705e Expand array in json root (#4302)
```
{
	"data": [{
		"key": 1
	}, {
		"key": 2
	}]
}
```
This PR adds support for the parse of json showed above, with json root `$.data` and `strip_outer_array=true`.

(cherry picked from commit a00c3b42a4)
2022-03-31 16:14:50 +08:00
satanson 00b8b04272
Branch 2.2 fixup local runtime bloom filter (#4617) 2022-03-31 10:06:21 +08:00
Pslydhh 509d797a12 Fix for onlynull input column (#4145) 2022-03-30 21:22:33 +08:00
xueyan.li 87131b4fd5
[cherry-pick]mv bugfixs to branch 2.2 (#4611)
* Forbid the "select * from" when create materialized view (#4143)

* Fix bug create materialized view cause double as key column (#4554)

We should prohibit creating the key column as double or float when creating the materialized view.
In order to prevent FE from failing, we do not check the replay
2022-03-30 19:28:53 +08:00
mergify[bot] 17e117ec0c
fix wrong order of destructions in pipeline engine (#4558) (#4593)
(cherry picked from commit 0bbc79b044)

Co-authored-by: stdpain <34912776+stdpain@users.noreply.github.com>
2022-03-30 15:46:00 +08:00
gengjun-git 856e4a4fcb
BugFix: loading to external olap table causes be crash (#4574) (#4591)
Throws exception when response status is not OK.
2022-03-30 15:37:16 +08:00
wulei 26bce27ab1
The fe node get crashed when receive beginRemoteTxn request if it is … (#4418) (#4585) (#4589)
* The fe node get crashed when receive beginRemoteTxn request if it is not leader.

===========cause=================
When an external table is created and the host property set to a
follower/observer node of the source cluster. If we insert some values
into this external table, the target fe node will receive
beginRemoteTxn rpc, and get crashed because of wal writing
is needed for beginRemoteTxn, which is forbidden in non-leader node.

==========solution===============
For non-leader fe node, it will forward beginRemoteTxn/commitRemoteTxn/
abortRemoteTxn to leader node and return back response from leader.

* return failed response rather than throws exception if forward request failed

Co-authored-by: wulei <87756338+wuleistarrocks@users.noreply.github.com>

Co-authored-by: gengjun-git <gengjun@dorisdb.com>
2022-03-30 15:36:45 +08:00
mergify[bot] 754ef5d80e
Fix missing gperftools/profile.h in thirdparty (#4539) (#4596)
if gperftools compile with FLAGS `-std=c11` or `-std=c99`. gperftools
won't install profile.h

(cherry picked from commit 1917f183b9)

Co-authored-by: stdpain <34912776+stdpain@users.noreply.github.com>
2022-03-30 15:36:11 +08:00
sevev c3c59ac9b6 BugFix: Be crash using shared tablet schema 2022-03-30 14:16:41 +08:00
stdpain 002bbea831 Fix nested expression (#4423)
(cherry picked from commit e30e4c5c40)
2022-03-30 13:53:21 +08:00
laotan332 f6f6228644
Fix be crash when enable enable_filter_unused_columns_in_scan_stage in pipeline engine (#4563) (#4571) 2022-03-30 10:29:18 +08:00
eyes_on_me 20a823371e
Fix result column size error when handling const column in truncate_decimal128 (#4550) (#4577) 2022-03-30 09:58:31 +08:00
Pslydhh 5d92609b74
Remove checks for array_difference([]) and array_difference(null) (#4520) (#4549) 2022-03-30 09:45:48 +08:00
mergify[bot] b842d87fca
Fix: make json type support case-when expression (#4533) (#4564)
(cherry picked from commit ce032c6e21)

Co-authored-by: Murphy <96611012+mofeiatwork@users.noreply.github.com>
2022-03-30 09:38:46 +08:00
trueeyu c57793fa5f
Fix the mem statistics bug of column pool (#4499) (#4507)
* When release large binary column, the bytes vector is already swap, but the mem statistics is not updated.
* the column_pool_test.cpp is not from apache doris, so change the license
2022-03-29 18:31:49 +08:00
Murphy 19bfddced9 Fix statistic of json type (#4513)
(cherry picked from commit d385444cc4)
2022-03-29 15:34:35 +08:00
Seaven 0939be9ad0
Fix date formatter different from be (#4136) (#4484) 2022-03-29 15:29:21 +08:00
miomiocat 61b1f1f717
Support refresh external command and getting metadata from hudi meta client (#4467) (#4537)
* Support refresh external command and getting metadata from hudi meta client
2022-03-29 14:54:24 +08:00
trueeyu 825b88a997 Fix the mem leak of table function node (#4450) 2022-03-29 13:34:46 +08:00
stdpain faef3ec1ff add UDF error check in pipeline engine (#4441)
(cherry picked from commit 259706cdb1)
2022-03-29 11:21:18 +08:00
ABing bae3581a36
fix the bug of mv creation failure (#4512)
Originally when create mv by 'create materialized view lo_mv_1 as select lo_orderdate, count(lo_LINENUMBER)
from lineorder group by lo_orderdate' and the field in table lineorder is lo_linenumber, then the creation
will fail because of the case of name. Fix it by use the original name defined in table schema and make it
case independent.
2022-03-29 10:24:38 +08:00
HangyuanLiu e20c7efdb3 user define table function analyze (#4455) 2022-03-29 10:22:08 +08:00
satanson 546abd9a24
Use unqualified user and role name to choose resource_groups (#4494) (#4519) 2022-03-29 08:29:15 +08:00
trueeyu 86bfebbcd6
Fix the bug of not init _table_function (#4187) (#4476) 2022-03-28 17:06:54 +08:00
xueyan.li 470b7d9424 Fix bug NULL_TYPE cause NPE in CTAS (#4487) 2022-03-28 16:51:59 +08:00
Murphy 7df2947e62 Add test case for CTAS decimal type (#4057) 2022-03-28 16:51:59 +08:00
Youngwb 1fbf56797e
[Cherry-pick][branch-2.2] fix bugs (#4478)
* Fix aggregate node not set local aggregation when global aggregate node and scan node is in the same plan fragment (#4382)

* Fix Fe UT (#4444)
2022-03-28 14:27:11 +08:00
ZiChen Qian 418eef7d24 [cherry-pick] [branch-2.2] BugFix: Use convert_schema_to_format_v2, not convert_schema, except the rowset writer adapter. 2022-03-28 14:07:42 +08:00
miomiocat 442094630a
Fix a data type error of creating hudi table due to typo mistake (#4473) 2022-03-28 13:10:59 +08:00
mergify[bot] 69f962e042
Fix: extract object field in array (#4403) (#4424)
(cherry picked from commit 28501c5637)

Co-authored-by: Murphy <96611012+mofeiatwork@users.noreply.github.com>
2022-03-25 12:34:45 +08:00
Pslydhh a174e4b2cf
Cherry exchange and array fix to2.2 (#4416)
* Avoid inconsistent when send eos quickly (#4360)

* Add more check for array_difference function (#4402)

Co-authored-by: kangkaisen <kangkaisen@apache.org>
2022-03-25 10:47:12 +08:00
Pslydhh 1798319cd4
Fix the return type of array_contains/array_position (#4222) (#4260) 2022-03-25 10:36:51 +08:00
stdpain 8becf94713
[cherry-pick][branch-2.2] Only apply limit prune tablet rule in duplicate keys (#4237) (#4272) 2022-03-25 10:32:02 +08:00
satanson a703e16ce5
Merge null column of result column with null columns of operands (#4406) (#4419) 2022-03-25 10:28:57 +08:00
stephen a1ae0dc173
IcebergScanNode support split files (#4411) 2022-03-24 21:05:44 +08:00
satanson 3f4d55f268
Fixup role match in classifier of resource_group (#4386) (#4407) 2022-03-24 18:39:04 +08:00
satanson 3ad75b3f6f
String-typed runtime in-filter referene freed key column of hash table (#4274) (#4389) 2022-03-24 16:28:25 +08:00
stephen 624a8b9e13
cherry-pick:Add session variable enable_hive_column_stats (#4388) 2022-03-24 14:02:03 +08:00
zhangqiang 18ed45b700 BugFix: Rowset id cache leak (#4333)
Rowset id cache will leak in some scenarios:
  1. allocate rowset id success, but create rowset writer failed
  2. create rowset writer success, but no data written
Anyway, if allocate rowset id success but does not generate rowset_ptr, we don't release the rowset_id, resulting in a memory leak. In addition, when we do load data, we create a delta_writer for each tablet in advance, but only a few of the delta_writer may have data written to them. This speeds up memory leak and output many useless rollback logs in be.INFO. This pr will call release_rowset_id in deconstruct function of delta_writer if the rowset is not built and reduce the rollback log.
2022-03-24 13:04:59 +08:00
zhangqiang 7958e33f14 BugFix: stack overflow in copy_file (#4347)
The stack memory limit set by the operating system can be changed using `ulimit -s `, and allocating memory greater or equal to the stack memory limit will cause be crash. If the stack memory limit is no more than 1MB, be will crash because try to allocate 1MB stack memory in function copy_files. We should not allocate large memory in the stack; allocate memory from the heap to solve this problem. This code is duplicated with function copy_file in utils/file_utils.cpp, merge the two copy_file function in utils/file_utils.cpp and storage/utils.cpp
2022-03-24 13:04:59 +08:00
liuyehcf e8e7b0d6cd Allow array type perform is null perdicate (#4369) 2022-03-24 10:58:53 +08:00
liuyehcf fe6fd4e02e Predicate check array type (#4332) 2022-03-24 10:58:53 +08:00
liuyehcf 8017be1cdb Simplify profile of OLAP_SCAN (#4341) 2022-03-24 10:58:53 +08:00
liuyehcf a5f01533af Profile add some metrics (#4235) 2022-03-24 10:58:53 +08:00
Youngwb ac2c40e46c Fix canColocateJoin not check orders of predicate columns when tables are the same (#4362) 2022-03-23 19:53:38 +08:00
Youngwb f937da3cc5 Remove unused search bushy tree for greedy join reorder (#4303) 2022-03-23 19:53:38 +08:00
Youngwb fa5093c826 Fix aggregate cannot generate one stage plan with array type params (#4343) 2022-03-23 19:53:38 +08:00
satanson 6f7be1fde0 Fix up resource group (#4323) 2022-03-23 19:38:57 +08:00
HangyuanLiu 77811eb00c Fix the bug that the dp search space becomes larger by using BitSet as the key of dp reorder (#4338) 2022-03-23 19:34:44 +08:00
HangyuanLiu edefbefeac Support aggregate expression in select statement without from clause (#4132)
Support aggregate expression in  select statement without from clause (#4132)
2022-03-23 19:34:44 +08:00
HangyuanLiu 7bc7c72724 Group by support array,json const expression (#4345) 2022-03-23 19:34:44 +08:00
satanson 73d574d55e
Fixup: decimal * decimal(0,0) report error (#4359) (#4370) 2022-03-23 19:33:23 +08:00
laotan332 65bc6485f7
resource_iso: more_accurate_cpu_use_for_wg (#4219) (#4354) 2022-03-23 17:13:15 +08:00
miomiocat 5849f223af
[cherry-pick][branch-2.2] cherry-pick hudi related PRs (#4312) 2022-03-23 15:51:31 +08:00
satanson c0de225b97
Refactor decimal multiplication (#4211) (#4331) 2022-03-22 19:05:20 +08:00
Youngwb 4976100f3b Fix type conversion error in PhysicalScanOperator (#4315)
(cherry picked from commit ae1fa12f7e)
2022-03-22 16:28:03 +08:00
mergify[bot] 15afb9c8fd
Support top-level array (#4283) (#4309)
(cherry picked from commit 202f0df122)

Co-authored-by: Murphy <96611012+mofeiatwork@users.noreply.github.com>
2022-03-22 10:46:03 +08:00
Youngwb c16172de6e
Disable colocate/bucket join when table with empty partition (#4266) (#4299) 2022-03-21 20:41:05 +08:00
HangyuanLiu 927e2d0d35 percentile_approx_raw support null (#4220) 2022-03-21 15:07:42 +08:00
HangyuanLiu d2d31fa43e Insert Statement support explain in new Parser (#4205) 2022-03-21 15:07:42 +08:00
HangyuanLiu 0228e8c3c5 Add invisible variable enableSQLDigest (#4188) 2022-03-21 15:07:42 +08:00
liuyehcf b61b70b766 Support decimal round (#4090) 2022-03-19 20:33:31 +08:00
liuyehcf b31303cc7c Fix type 'SHFFULE' to 'SHUFFLE' (#4191) 2022-03-19 20:33:31 +08:00
liuyehcf 49822f4ad9 Truncate support decimal (#3993) 2022-03-19 20:33:31 +08:00
liuyehcf ec83736b69 Rename local variables; (#3971)
Rename local variables
2022-03-19 20:33:31 +08:00
liuyehcf f7bc22a0cf Fix the problem of truncate_decimal (#3920) 2022-03-19 20:33:31 +08:00
Pslydhh 09f39a668f Add TRY_CATCH_BAD_ALLOC to pipeline (#4194) 2022-03-19 17:02:55 +08:00
satanson 09fd68e9f7 Refactor Operator::close,set_finishing,set_finished,set_canceled (#4179) 2022-03-19 17:02:55 +08:00
Murphy 7e8e50f9a3
Disable join on json type (#4255) 2022-03-19 15:44:28 +08:00
stdpain 762e842da8 Fix BE start crash if BE couldn't found timezone (#4221) 2022-03-19 14:25:05 +08:00
stdpain 33e6c23d7a limit scanner concurrency when scan too many columns (#4140)
1. limit scan range for each tablet
    each tablet could only split with `num_segments` scanner
2. limit scan concurrency to reduce memory usage
2022-03-19 14:25:05 +08:00
stdpain 4a48c0f81b Fix BE crash in graceful exit (#4209) 2022-03-19 14:25:05 +08:00
trueeyu 1af241b3d5
Add try catch bad alloc for hash join build and probe (#3981) (#4214)
Add try catch bad alloc for hash join build and probe
2022-03-19 10:25:09 +08:00
Murphy e65c1cbb4d
[Cherrypick] Fix crash of sha2 function with null parameter (#4224) 2022-03-19 09:17:15 +08:00
zhangqiang d71f7c4684
BugFix: error memory limitation of schema change (#4115) (#4223)
The type of memory_limitation_per_thread_for_schema_change is int32 and upper limit is 2GB.
When memory_limitation_per_thread_for_schema_change is set to more than 4GB, the memory limitation of schema change will overflow and maybe be set to 0.
Change the type of memory limitation to uint64
2022-03-18 20:01:55 +08:00
Youngwb 9cb7a89616 Fix enforcer use wrong group and child group when merge group (#4180) 2022-03-18 17:42:56 +08:00
Youngwb aaad10ac3f Fix join reorder projection not deal with columnOperator to columnOperator map (#4056) 2022-03-18 17:42:56 +08:00
Youngwb 861f7b1a9b Disable compute extra cost with unknown statistics (#3959) 2022-03-18 17:42:56 +08:00
satanson 016dae8496 Fixup: mistaken partition exprs of local shuffle interpolated into left side of bucket shuffle join when SlotRef one same bucket key appear more than once in join eq_conjunctions (#4204) 2022-03-18 14:57:05 +08:00
Murphy f0805b8e08 fix error message of json group by (#4200) 2022-03-17 20:31:48 +08:00
HangyuanLiu 8413c2bb10 fix ut 2022-03-16 17:26:21 +08:00
HangyuanLiu 9009350f08 Support PrivilegeChecker in new parser and fix create analyze fail bug #4098 2022-03-16 17:26:21 +08:00
HangyuanLiu fbc08e2ac7 Support parse special function which name is keyword in new Parser (#4105)
* Support parse special function which name is keyword in new Parser
2022-03-16 17:26:21 +08:00
HangyuanLiu d1de6422b2 Support select tablet syntax in new parser (#4061) 2022-03-16 17:26:21 +08:00
HangyuanLiu c7169a55d0 Fix union resolve bug (#4070) 2022-03-16 17:26:21 +08:00
HangyuanLiu 5be1ac411e Fix set miss sort operator bug in new parser(#4058) 2022-03-16 17:26:21 +08:00
HangyuanLiu f6f17ddf08 Fix compile about AnalyzeFunctionTest (#3967) 2022-03-16 17:26:21 +08:00
HangyuanLiu 3baa1a7577 Fix show stmt with predicate bug(#3957) 2022-03-16 17:26:21 +08:00
HangyuanLiu 8ac29d520d fix timestamp-expr (#3938) 2022-03-16 17:26:21 +08:00
HangyuanLiu d1ffbeed81 Refactor Parser code and remove unused code (#3881) 2022-03-16 17:26:21 +08:00
stdpain 2217eb0610 Fix low cardinality opmize crash in pipeline when enable cache table … (#3915)
when we rewrite decoded slot descriptor, we add the new slot descritpor
to runtime_state.object_pool. but this slot descriptor may be accessed
in other thread. which will cause a use-after-free
2022-03-15 20:36:15 +08:00
stdpain d4b1ab9392 Downgrading libmariadb to avoid problems accessing other MYSQL protocol databases (#3966) (#3975)
Fix the problem `mysql set character set failed.` when access TiDB
2022-03-15 20:36:15 +08:00
stdpain 264f4c4654 fix NPE when collect global dictionary 2022-03-15 20:36:15 +08:00
stdpain 1518ba19ff avoid download other Dir files in get_log_file interface (#4006) 2022-03-15 20:36:15 +08:00
stdpain 8349597c60 fix thirdparty compile problems in docker (#3976) 2022-03-15 20:36:15 +08:00
stdpain 8d8ce962f8 fix compile problem for other platform (#3951) 2022-03-15 20:36:15 +08:00
Murphy 23b4eff36d
Fix ddl bugs of json type (#4144) 2022-03-15 20:35:55 +08:00
Murphy 3c7ac28b2d
fix null check in functions (#4123) (#4146) 2022-03-15 20:35:39 +08:00
xueyan.li 070b92b3a3
Do not send migrate task when BE only have 1 storage medium (#4002) (#4121)
when be report tablet info to fe.
If we know that this BE must have no other medium, then only write a log as a reminder, not to send a task.
2022-03-15 13:03:42 +08:00
zihe.liu 1ede12b724
Fix: finish analytic sink after reached limit (#4003) (#4113) 2022-03-15 12:46:32 +08:00
Seaven 6404372320 Forbidden non-number type cast reduce (#4055) 2022-03-14 21:06:27 +08:00
Seaven 030bc2b101 Fix date cast to datetime bug when datetime is null 2022-03-14 21:06:27 +08:00
Seaven d444f7af7a Fix sql split bug (#3956) 2022-03-14 21:06:27 +08:00
ricky 475351d825 Persist meta once in publish version group (#3841)
Flush the WAL of RocksDB after saving the meta. There are two flush operations during load.
One is the commit, and the other is the publish. To alleviate the I/O operations, we batch the flush operations.
2022-03-14 21:05:04 +08:00
satanson f6d91a35c5 Analytic function needs func_version (#4078) 2022-03-14 20:48:29 +08:00
satanson 26d25bb373 fixup: QueryContext is removed accidently (#4060) 2022-03-14 20:48:29 +08:00
satanson 15078f53af use ConcurrentNavigableMap instead of List to for Coordinator.backendExecStates (#3958) 2022-03-14 20:48:29 +08:00
satanson 0bb1de651f use decimal128 instead decimal64 for aggr sum (#3944) 2022-03-14 20:48:29 +08:00
ricky 76dcf80796 BugFix: fatal error when the delta writer is busy (#3878)
This PR makes retry when the AsyncDeltaWriter is busy to avoid the bthread's fatal error.
2022-03-14 20:16:36 +08:00
yan.zhang 8b69672b42
Optimize small read on ORC(small row group index/stripe/file) (#3962) (#4091)
This PR is to optimization read small orc file and small stripe. And what's more, we can put previous optimization on row group index into this framework.

Handle Small File
=======
I set up a test environment of a lot of small hdfs files. All of them are small files under 2MB.

```
-rw-r-----   3 hadoop hadoop    1332877 2022-03-09 12:10 /user/zya/tpch_100g/lineitem_smallfile_zlib/012314_0
-rw-r-----   3 hadoop hadoop    1328418 2022-03-09 12:11 /user/zya/tpch_100g/lineitem_smallfile_zlib/012315_0
-rw-r-----   3 hadoop hadoop    1312973 2022-03-09 12:11 /user/zya/tpch_100g/lineitem_smallfile_zlib/012316_0
-rw-r-----   3 hadoop hadoop    1304832 2022-03-09 12:11 /user/zya/tpch_100g/lineitem_smallfile_zlib/012317_0
-rw-r-----   3 hadoop hadoop     878983 2022-03-09 12:10 /user/zya/tpch_100g/lineitem_smallfile_zlib/012318_0
```

Running query
> select count(*) from lineitem_smallfile;

You can see although IOBytes increased, but IOCounte decreased. And average bytes per IO is increase from (629.1MB / 257.963K ~= 2.69KB) to (16.72GB / 12.319K ~= 1.35MB)

Before optimization

```
         HDFS_SCAN_NODE (id=0):(Active: 8s861ms[8861568366ns], % non-child: 98.73%)
             - BytesRead: 0.00
             - BytesReadDataNodeCache: 0.00
             - BytesReadFromDisk: 629.10 MB
             - BytesReadLocal: 0.00
             - BytesReadRemote: 629.10 MB
             - BytesReadShortCircuit: 0.00
             - BytesTotalRead: 629.10 MB
             - IoCounter: 257.063K (257063)
             - IoTime: 6m10s
```

After optimization
```
          HDFS_SCAN_NODE (id=0):(Active: 4s322ms[4322893106ns], % non-child: 97.17%)
             - Table: lineitem_smallfile
             - Predicates:
             - PredicatesPartition:
             - BytesRead: 16.72 GB
             - ColumnConvertTime: 1s250ms
             - ColumnReadTime: 16s355ms
             - ExprFilterTime: 8.39ms
             - HdfsIO: 0ns
               - TotalBytesRead: 17.957812729B (17957812729)
               - TotalLocalBytesRead: 0
               - TotalShortCircuitBytesRead: 0
               - TotalZeroCopyBytesRead: 0
             - IoCounter: 12.319K (12319) // <=== exact file number
             - IoTime: 2m17s
```

Handle Stripe Size
=======
I set up a test environment of a single file but with small stripe size. You can see a stripe size is abot 700K, and there are 350 stripes in a single file.

> java -jar ~/installed/orc-tools-1.7.0-SNAPSHOT-uber.jar meta test_many_columns.orc

```
  Stripe 348:
  Stripe 349:
  Stripe 350:
Stripes:
  Stripe: offset: 3 data: 722325 rows: 2864 tail: 1861 index: 9095
  Stripe: offset: 733284 data: 763098 rows: 2864 tail: 1907 index: 9096
  Stripe: offset: 1507385 data: 776631 rows: 2864 tail: 1880 index: 9097
```

Normally we don't have such small stripe size. Usually stripe size is about 256MB. But there are some bugs in SparkSQL may use small stripe size.
- https://community.cloudera.com/t5/Support-Questions/Spark-ORC-Stripe-Size/td-p/189844
- https://issues.apache.org/jira/browse/HIVE-13232

Running query
> select max(c1) from test_many_columns;

You can see IOCounter is cut from 2451 down to 351.

Before optimization

```
          HDFS_SCAN_NODE (id=0):(Active: 2s771ms[2771719221ns], % non-child: 99.20%)
             - BytesRead: 0.00
             - BytesReadDataNodeCache: 0.00
             - BytesReadFromDisk: 1.92 MB
             - BytesReadLocal: 0.00
             - BytesReadRemote: 1.92 MB
             - BytesReadShortCircuit: 0.00
             - BytesTotalRead: 1.92 MB
             - ColumnConvertTime: 20.341ms
             - ColumnReadTime: 1s814ms
             - ExprFilterTime: 0ns
             - GroupChunkRead: 0ns
             - GroupDictDecode: 0ns
             - GroupDictFilter: 0ns
             - IoCounter: 2.451K (2451)
             - IoTime: 1s772ms
```

After optimization

```
          HDFS_SCAN_NODE (id=0):(Active: 697.474ms[697474842ns], % non-child: 98.24%)
             - Table: test_many_columns
             - Predicates:
             - PredicatesPartition:
             - BytesRead: 255.89 MB
             - ColumnConvertTime: 20.537ms
             - ColumnReadTime: 676.219ms
             - ExprFilterTime: 15.637us
             - HdfsIO: 0ns
               - TotalBytesRead: 268.315441M (268315441)
               - TotalLocalBytesRead: 0
               - TotalShortCircuitBytesRead: 0
               - TotalZeroCopyBytesRead: 0
             - IoCounter: 351 // <== exact stripe size + file footer read.
             - IoTime: 612.488ms
```

And if we combine reads of stripes, we can save more IOCounter

```
          HDFS_SCAN_NODE (id=0):(Active: 643.709ms[643709229ns], % non-child: 97.69%)
             - Table: test_many_columns
             - Predicates:
             - PredicatesPartition:
             - BytesRead: 349.31 MB
             - ColumnConvertTime: 22.263ms
             - ColumnReadTime: 623.502ms
             - ExprFilterTime: 11.235us
             - HdfsIO: 0ns
               - TotalBytesRead: 366.282951M (366282951)
               - TotalLocalBytesRead: 0
               - TotalShortCircuitBytesRead: 0
               - TotalZeroCopyBytesRead: 0
             - IoCounter: 176
             - IoTime: 564.540ms
 ```
2022-03-14 18:17:38 +08:00
Pslydhh 2dc8d3c37b
Fix crash of pipeline caused by QueryContext destruct early (#4085) 2022-03-14 16:38:52 +08:00
ZiChen Qian 6ce5b2b0ad PrimaryKey table is forbidden to backup/restore (#3985)
Backup/restore does not support the PrimaryKey table. So the two operations are forbidden.
2022-03-14 15:25:42 +08:00
xueyan.li 9f502d111c
Support multi-character row delimiter during load (#3900) (#3992)
The pull request supports multi-character row delimiter. After that, the '<br>' can be used as a row delimiter.
2022-03-11 15:48:39 +08:00
yan.zhang 88c4e0918f
Fix be crash caused by #3912 and #3825 (#3934) (#4054)
* be will crash with #3912 since we should init config after 
```
Aws::SDKOptions aws_sdk_options;  Aws::InitAPI(aws_sdk_options); 
```
being called. we have called the two apis in starrocks_main.cpp, so we use lazy singleton instance for AWS ClientConfiguration to avoid be crash during starting
* iceberg table has no partitions, after #3825 be will crash when we query iceberg table.We should check has_partition in _find_and_insert_hdfs_file and _init_partition_values_map

Co-authored-by: zhoukang <zhoukang199191@gmail.com>
2022-03-11 14:59:48 +08:00
525 changed files with 11712 additions and 8389 deletions

View File

@ -326,7 +326,9 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
endif()
if ("${CMAKE_BUILD_TARGET_ARCH}" STREQUAL "x86" OR "${CMAKE_BUILD_TARGET_ARCH}" STREQUAL "x86_64")
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -msse4.2")
if (USE_SSE4_2)
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -msse4.2")
endif()
if (USE_AVX2)
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -mavx2")
endif()

View File

@ -64,6 +64,7 @@ namespace starrocks {
const uint32_t TASK_FINISH_MAX_RETRY = 3;
const uint32_t PUBLISH_VERSION_MAX_RETRY = 3;
const uint32_t PUBLISH_VERSION_SUBMIT_MAX_RETRY = 10;
const size_t PUBLISH_VERSION_BATCH_SIZE = 10;
std::atomic_ulong TaskWorkerPool::_s_report_version(time(nullptr) * 10000);
std::mutex TaskWorkerPool::_s_task_signatures_locks[TTaskType::type::NUM_TASK_TYPE];
@ -707,7 +708,8 @@ void* TaskWorkerPool::_push_worker_thread_callback(void* arg_this) {
}
Status TaskWorkerPool::_publish_version_in_parallel(void* arg_this, std::unique_ptr<ThreadPool>& threadpool,
const TPublishVersionRequest publish_version_req, size_t* tablet_n,
const TPublishVersionRequest publish_version_req,
std::set<TTabletId>* tablet_ids, size_t* tablet_n,
std::vector<TTabletId>* error_tablet_ids) {
TaskWorkerPool* worker_pool_this = (TaskWorkerPool*)arg_this;
int64_t transaction_id = publish_version_req.transaction_id;
@ -797,8 +799,8 @@ Status TaskWorkerPool::_publish_version_in_parallel(void* arg_this, std::unique_
error_status = status;
}
}
tablet_ids->insert(tablet_info.tablet_id);
}
*tablet_n += tablet_infos.size();
}
return error_status;
}
@ -814,12 +816,16 @@ void* TaskWorkerPool::_publish_version_worker_thread_callback(void* arg_this) {
// The ideal queue size of threadpool should be larger than the maximum number of tablet of a partition.
// But it seems that there's no limit for the number of tablets of a partition.
// Since a large queue size brings a little overhead, a big one is chosen here.
.set_max_queue_size(256)
.set_max_queue_size(2048)
.build(&threadpool);
assert(st.ok());
std::vector<TAgentTaskRequest> task_requests;
std::vector<TFinishTaskRequest> finish_task_requests;
std::set<TTabletId> tablet_ids;
std::vector<TabletSharedPtr> tablets;
while (true) {
TAgentTaskRequest agent_task_req;
TPublishVersionRequest publish_version_req;
{
std::unique_lock l(worker_pool_this->_worker_thread_lock);
while (worker_pool_this->_tasks.empty() && !(worker_pool_this->_stopped)) {
@ -829,54 +835,86 @@ void* TaskWorkerPool::_publish_version_worker_thread_callback(void* arg_this) {
break;
}
agent_task_req = worker_pool_this->_tasks.front();
publish_version_req = agent_task_req.publish_version_req;
worker_pool_this->_tasks.pop_front();
}
StarRocksMetrics::instance()->publish_task_request_total.increment(1);
LOG(INFO) << "get publish version task, signature:" << agent_task_req.signature;
std::vector<TTabletId> error_tablet_ids;
uint32_t retry_time = 0;
Status status;
size_t tablet_n = 0;
while (retry_time < PUBLISH_VERSION_MAX_RETRY) {
error_tablet_ids.clear();
status = _publish_version_in_parallel(arg_this, threadpool, publish_version_req, &tablet_n,
&error_tablet_ids);
if (status.ok()) {
break;
} else {
LOG(WARNING) << "publish version error, retry. [transaction_id=" << publish_version_req.transaction_id
<< ", error_tablets_size=" << error_tablet_ids.size() << "]";
++retry_time;
SleepFor(MonoDelta::FromSeconds(1));
while (!worker_pool_this->_tasks.empty() && task_requests.size() < PUBLISH_VERSION_BATCH_SIZE) {
// collect some publish version tasks as a group.
task_requests.push_back(worker_pool_this->_tasks.front());
worker_pool_this->_tasks.pop_front();
}
}
TFinishTaskRequest finish_task_request;
if (!status.ok()) {
StarRocksMetrics::instance()->publish_task_failed_total.increment(1);
// if publish failed, return failed, FE will ignore this error and
// check error tablet ids and FE will also republish this task
LOG(WARNING) << "Fail to publish version. signature:" << agent_task_req.signature
<< " related tablet num: " << tablet_n;
finish_task_request.__set_error_tablet_ids(error_tablet_ids);
} else {
LOG(INFO) << "publish_version success. signature:" << agent_task_req.signature
<< " related tablet num: " << tablet_n;
for (size_t i = 0; i < task_requests.size(); ++i) {
auto& publish_version_task = task_requests[i];
StarRocksMetrics::instance()->publish_task_request_total.increment(1);
LOG(INFO) << "get publish version task, signature:" << publish_version_task.signature << " index: " << i
<< " group size: " << task_requests.size();
auto& publish_version_req = publish_version_task.publish_version_req;
std::vector<TTabletId> error_tablet_ids;
uint32_t retry_time = 0;
Status status;
size_t tablet_n = 0;
while (retry_time < PUBLISH_VERSION_MAX_RETRY) {
error_tablet_ids.clear();
status = _publish_version_in_parallel(arg_this, threadpool, publish_version_req, &tablet_ids, &tablet_n,
&error_tablet_ids);
if (status.ok()) {
break;
} else {
LOG(WARNING) << "publish version error, retry. [transaction_id="
<< publish_version_req.transaction_id
<< ", error_tablets_size=" << error_tablet_ids.size() << "]";
++retry_time;
SleepFor(MonoDelta::FromSeconds(1));
}
}
TFinishTaskRequest finish_task_request;
if (!status.ok()) {
StarRocksMetrics::instance()->publish_task_failed_total.increment(1);
// if publish failed, return failed, FE will ignore this error and
// check error tablet ids and FE will also republish this task
LOG(WARNING) << "Fail to publish version. signature:" << publish_version_task.signature
<< " related tablet num: " << tablet_n;
finish_task_request.__set_error_tablet_ids(error_tablet_ids);
} else {
LOG(INFO) << "publish_version success. signature:" << publish_version_task.signature
<< " related tablet num: " << tablet_n;
}
status.to_thrift(&finish_task_request.task_status);
finish_task_request.__set_backend(worker_pool_this->_backend);
finish_task_request.__set_task_type(publish_version_task.task_type);
finish_task_request.__set_signature(publish_version_task.signature);
finish_task_request.__set_report_version(_s_report_version);
finish_task_requests.emplace_back(std::move(finish_task_request));
}
status.to_thrift(&finish_task_request.task_status);
finish_task_request.__set_backend(worker_pool_this->_backend);
finish_task_request.__set_task_type(agent_task_req.task_type);
finish_task_request.__set_signature(agent_task_req.signature);
finish_task_request.__set_report_version(_s_report_version);
// persist all related meta once in a group.
tablets.reserve(tablet_ids.size());
for (const auto tablet_id : tablet_ids) {
TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id);
if (tablet != nullptr) {
tablets.push_back(tablet);
}
}
worker_pool_this->_finish_task(finish_task_request);
worker_pool_this->_remove_task_info(agent_task_req.task_type, agent_task_req.signature);
auto st = StorageEngine::instance()->txn_manager()->persist_tablet_related_txns(tablets);
if (!st.ok()) {
LOG(WARNING) << "failed to persist transactions, tablets num: " << tablets.size() << " err: " << st;
}
tablet_ids.clear();
tablets.clear();
// notify FE when all tasks of group have been finished.
for (auto& finish_task_request : finish_task_requests) {
worker_pool_this->_finish_task(finish_task_request);
worker_pool_this->_remove_task_info(finish_task_request.task_type, finish_task_request.signature);
}
task_requests.clear();
finish_task_requests.clear();
}
threadpool->shutdown();
return (void*)nullptr;

View File

@ -99,7 +99,8 @@ private:
static void* _drop_tablet_worker_thread_callback(void* arg_this);
static void* _push_worker_thread_callback(void* arg_this);
static Status _publish_version_in_parallel(void* arg_this, std::unique_ptr<ThreadPool>& threadpool,
const TPublishVersionRequest publish_version_req, size_t* tablet_n,
const TPublishVersionRequest publish_version_req,
std::set<TTabletId>* tablet_ids, size_t* tablet_n,
std::vector<TTabletId>* error_tablet_ids);
static void* _publish_version_worker_thread_callback(void* arg_this);
static void* _clear_transaction_task_worker_thread_callback(void* arg_this);

View File

@ -4,10 +4,15 @@
#include <cstdint>
#if defined(__x86_64__)
#ifdef __SSE4_2__
#include <nmmintrin.h>
#endif
#ifdef __SSE2__
#include <emmintrin.h>
#include <xmmintrin.h>
#endif
#include "util/hash_util.hpp"
#include "util/slice.h"
#include "util/unaligned_access.h"
@ -103,6 +108,9 @@ public:
};
static uint32_t crc_hash_32(const void* data, int32_t bytes, uint32_t hash) {
#if defined(__x86_64__) && !defined(__SSE4_2__)
return crc32(hash, (const unsigned char*)data, bytes);
#else
uint32_t words = bytes / sizeof(uint32_t);
bytes = bytes % 4 /*sizeof(uint32_t)*/;
@ -134,9 +142,13 @@ static uint32_t crc_hash_32(const void* data, int32_t bytes, uint32_t hash) {
// for anyone who only uses the first several bits of the hash.
hash = (hash << 16u) | (hash >> 16u);
return hash;
#endif
}
static uint64_t crc_hash_64(const void* data, int32_t length, uint64_t hash) {
#if defined(__x86_64__) && !defined(__SSE4_2__)
return crc32(hash, (const unsigned char*)data, length);
#else
if (UNLIKELY(length < 8)) {
return crc_hash_32(data, length, hash);
}
@ -145,7 +157,7 @@ static uint64_t crc_hash_64(const void* data, int32_t length, uint64_t hash) {
auto* p = reinterpret_cast<const uint8_t*>(data);
auto* end = reinterpret_cast<const uint8_t*>(data) + length;
while (words--) {
#if defined(__x86_64__)
#if defined(__x86_64__) && defined(__SSE4_2__)
hash = _mm_crc32_u64(hash, unaligned_load<uint64_t>(p));
#elif defined(__aarch64__)
hash = __crc32cd(hash, unaligned_load<uint32_t>(p));
@ -165,6 +177,7 @@ static uint64_t crc_hash_64(const void* data, int32_t length, uint64_t hash) {
#endif
p += sizeof(uint64_t);
return hash;
#endif
}
// TODO: 0x811C9DC5 is not prime number
@ -255,8 +268,10 @@ public:
};
inline uint64_t crc_hash_uint64(uint64_t value, uint64_t seed) {
#if defined(__x86_64__)
#if defined(__x86_64__) && defined(__SSE4_2__)
return _mm_crc32_u64(seed, value);
#elif defined(__x86_64__)
return crc32(seed, (const unsigned char*)&value, sizeof(uint64_t));
#elif defined(__aarch64__)
return __crc32cd(seed, value);
#else
@ -265,9 +280,12 @@ inline uint64_t crc_hash_uint64(uint64_t value, uint64_t seed) {
}
inline uint64_t crc_hash_uint128(uint64_t value0, uint64_t value1, uint64_t seed) {
#if defined(__x86_64__)
#if defined(__x86_64__) && defined(__SSE4_2__)
uint64_t hash = _mm_crc32_u64(seed, value0);
hash = _mm_crc32_u64(hash, value1);
#elif defined(__x86_64__)
hash = crc32(seed, (const unsigned char*)&value0, sizeof(uint64_t));
hash = crc32(hash, (const unsigned char*)&value1, sizeof(uint64_t));
#elif defined(__aarch64__)
uint64_t hash = __crc32cd(seed, value0);
hash = __crc32cd(hash, value1);

View File

@ -172,7 +172,11 @@ class CACHELINE_ALIGNED ColumnPool {
freed_bytes += release_column_if_large(_curr_free.ptrs[i], limit);
ASAN_POISON_MEMORY_REGION(_curr_free.ptrs[i], sizeof(T));
}
_curr_free.bytes -= freed_bytes;
if (freed_bytes > 0) {
_curr_free.bytes -= freed_bytes;
tls_thread_status.mem_consume(freed_bytes);
_pool->mem_tracker()->release(freed_bytes);
}
}
static inline void delete_local_pool(void* arg) { delete (LocalPool*)arg; }

View File

@ -36,7 +36,7 @@ public:
bool is_null(size_t index) const override { return _data->is_null(0); }
bool only_null() const override { return _data->is_nullable(); }
bool only_null() const override { return _data->is_null(0); }
bool has_null() const override { return _data->has_null(); }

View File

@ -175,6 +175,7 @@ CONF_String(local_library_dir, "${UDF_RUNTIME_DIR}");
CONF_mInt32(doris_scanner_thread_pool_thread_num, "48");
// Number of olap scanner thread pool size.
CONF_Int32(doris_scanner_thread_pool_queue_size, "102400");
CONF_mDouble(scan_use_query_mem_ratio, "0.25");
// Number of etl thread pool size.
CONF_Int32(etl_thread_pool_size, "8");
CONF_Int32(udf_thread_pool_size, "1");
@ -665,6 +666,7 @@ CONF_String(object_storage_endpoint, "");
CONF_Int64(object_storage_max_connection, "102400");
CONF_Bool(enable_orc_late_materialization, "true");
CONF_Int32(orc_file_cache_max_size, "2097152");
// default: 16MB
CONF_mInt64(experimental_s3_max_single_part_size, "16777216");

View File

@ -113,6 +113,8 @@ public:
return Status(TStatusCode::DATA_QUALITY_ERROR, msg);
}
static Status GlobalDictError(const Slice& msg) { return Status(TStatusCode::GLOBAL_DICT_ERROR, msg); }
bool ok() const { return _state == nullptr; }
bool is_cancelled() const { return code() == TStatusCode::CANCELLED; }

21
be/src/env/env_s3.cpp vendored
View File

@ -123,13 +123,6 @@ bool operator==(const Aws::Client::ClientConfiguration& lhs, const Aws::Client::
class S3ClientFactory {
public:
// We cached config here and make a deep copy each time.Since aws sdk has changed the
// Aws::Client::ClientConfiguration default constructor to search for the region
// (where as before 1.8 it has been hard coded default of "us-east-1").
// Part of that change is looking through the ec2 metadata, which can take a long time.
// For more details, please refer https://github.com/aws/aws-sdk-cpp/issues/1440
static Aws::Client::ClientConfiguration s_config;
using ClientConfiguration = Aws::Client::ClientConfiguration;
using S3Client = Aws::S3::S3Client;
using S3ClientPtr = std::shared_ptr<S3Client>;
@ -148,6 +141,16 @@ public:
S3ClientPtr new_client(const ClientConfiguration& config);
static ClientConfiguration& getClientConfig() {
// We cached config here and make a deep copy each time.Since aws sdk has changed the
// Aws::Client::ClientConfiguration default constructor to search for the region
// (where as before 1.8 it has been hard coded default of "us-east-1").
// Part of that change is looking through the ec2 metadata, which can take a long time.
// For more details, please refer https://github.com/aws/aws-sdk-cpp/issues/1440
static ClientConfiguration instance;
return instance;
}
private:
S3ClientFactory();
@ -161,8 +164,6 @@ private:
Random _rand;
};
Aws::Client::ClientConfiguration S3ClientFactory::s_config;
S3ClientFactory::S3ClientFactory() : _items(0), _rand((int)::time(NULL)) {}
S3ClientFactory::S3ClientPtr S3ClientFactory::new_client(const ClientConfiguration& config) {
@ -196,7 +197,7 @@ S3ClientFactory::S3ClientPtr S3ClientFactory::new_client(const ClientConfigurati
}
static std::shared_ptr<Aws::S3::S3Client> new_s3client(const S3URI& uri) {
Aws::Client::ClientConfiguration config = S3ClientFactory::s_config;
Aws::Client::ClientConfiguration config = S3ClientFactory::getClientConfig();
config.scheme = Aws::Http::Scheme::HTTP; // TODO: use the scheme in uri
if (!uri.endpoint().empty()) {
config.endpointOverride = uri.endpoint();

View File

@ -94,6 +94,7 @@ set(EXEC_FILES
vectorized/hdfs_scanner_parquet.cpp
vectorized/hdfs_scanner_text.cpp
vectorized/json_scanner.cpp
vectorized/json_parser.cpp
vectorized/project_node.cpp
vectorized/dict_decode_node.cpp
vectorized/repeat_node.cpp
@ -195,8 +196,8 @@ set(EXEC_FILES
# simdjson Runtime Implement Dispatch: https://github.com/simdjson/simdjson/blob/master/doc/implementation-selection.md#runtime-cpu-detection
# Unset architecture-specific flags to avoid breaking implement runtime dispatch.
if ("${CMAKE_BUILD_TARGET_ARCH}" STREQUAL "x86" OR "${CMAKE_BUILD_TARGET_ARCH}" STREQUAL "x86_64")
set_source_files_properties(vectorized/json_scanner.cpp PROPERTIES COMPILE_FLAGS -mno-avx)
set_source_files_properties(vectorized/json_scanner.cpp PROPERTIES COMPILE_FLAGS -mno-avx2)
set_source_files_properties(vectorized/json_scanner.cpp PROPERTIES COMPILE_FLAGS "-mno-avx -mno-avx2")
set_source_files_properties(vectorized/json_parser.cpp PROPERTIES COMPILE_FLAGS "-mno-avx -mno-avx2")
endif()
set(EXEC_FILES

View File

@ -311,12 +311,8 @@ public:
return _begin_scan_keys.size();
}
void set_begin_include(bool begin_include) { _begin_include = begin_include; }
bool begin_include() const { return _begin_include; }
void set_end_include(bool end_include) { _end_include = end_include; }
bool end_include() const { return _end_include; }
void set_is_convertible(bool is_convertible) { _is_convertible = is_convertible; }

View File

@ -17,7 +17,7 @@ void AggregateBlockingSinkOperator::close(RuntimeState* state) {
Operator::close(state);
}
void AggregateBlockingSinkOperator::set_finishing(RuntimeState* state) {
Status AggregateBlockingSinkOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
if (!_aggregator->is_none_group_by_exprs()) {
@ -45,6 +45,7 @@ void AggregateBlockingSinkOperator::set_finishing(RuntimeState* state) {
COUNTER_SET(_aggregator->input_row_count(), _aggregator->num_input_rows());
_aggregator->sink_complete();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> AggregateBlockingSinkOperator::pull_chunk(RuntimeState* state) {

View File

@ -20,7 +20,7 @@ public:
bool has_output() const override { return false; }
bool need_input() const override { return !is_finished(); }
bool is_finished() const override { return _is_finished || _aggregator->is_finished(); }
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
Status prepare(RuntimeState* state) override;
void close(RuntimeState* state) override;

View File

@ -14,8 +14,8 @@ bool AggregateBlockingSourceOperator::is_finished() const {
return _aggregator->is_sink_complete() && _aggregator->is_ht_eos();
}
void AggregateBlockingSourceOperator::set_finished(RuntimeState* state) {
_aggregator->set_finished();
Status AggregateBlockingSourceOperator::set_finished(RuntimeState* state) {
return _aggregator->set_finished();
}
void AggregateBlockingSourceOperator::close(RuntimeState* state) {

View File

@ -22,7 +22,7 @@ public:
bool has_output() const override;
bool is_finished() const override;
void set_finished(RuntimeState* state) override;
Status set_finished(RuntimeState* state) override;
void close(RuntimeState* state) override;

View File

@ -17,7 +17,7 @@ void AggregateDistinctBlockingSinkOperator::close(RuntimeState* state) {
Operator::close(state);
}
void AggregateDistinctBlockingSinkOperator::set_finishing(RuntimeState* state) {
Status AggregateDistinctBlockingSinkOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
COUNTER_SET(_aggregator->hash_table_size(), (int64_t)_aggregator->hash_set_variant().size());
@ -38,6 +38,7 @@ void AggregateDistinctBlockingSinkOperator::set_finishing(RuntimeState* state) {
COUNTER_SET(_aggregator->input_row_count(), _aggregator->num_input_rows());
_aggregator->sink_complete();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> AggregateDistinctBlockingSinkOperator::pull_chunk(RuntimeState* state) {

View File

@ -22,7 +22,7 @@ public:
bool has_output() const override { return false; }
bool need_input() const override { return !is_finished(); }
bool is_finished() const override { return _is_finished || _aggregator->is_finished(); }
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
Status prepare(RuntimeState* state) override;
void close(RuntimeState* state) override;

View File

@ -14,8 +14,8 @@ bool AggregateDistinctBlockingSourceOperator::is_finished() const {
return _aggregator->is_sink_complete() && _aggregator->is_ht_eos();
}
void AggregateDistinctBlockingSourceOperator::set_finished(RuntimeState* state) {
_aggregator->set_finished();
Status AggregateDistinctBlockingSourceOperator::set_finished(RuntimeState* state) {
return _aggregator->set_finished();
}
void AggregateDistinctBlockingSourceOperator::close(RuntimeState* state) {

View File

@ -22,7 +22,7 @@ public:
bool has_output() const override;
bool is_finished() const override;
void set_finished(RuntimeState* state) override;
Status set_finished(RuntimeState* state) override;
void close(RuntimeState* state) override;

View File

@ -17,7 +17,7 @@ void AggregateDistinctStreamingSinkOperator::close(RuntimeState* state) {
Operator::close(state);
}
void AggregateDistinctStreamingSinkOperator::set_finishing(RuntimeState* state) {
Status AggregateDistinctStreamingSinkOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
if (_aggregator->hash_set_variant().size() == 0) {
@ -25,6 +25,7 @@ void AggregateDistinctStreamingSinkOperator::set_finishing(RuntimeState* state)
}
_aggregator->sink_complete();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> AggregateDistinctStreamingSinkOperator::pull_chunk(RuntimeState* state) {

View File

@ -22,7 +22,7 @@ public:
bool has_output() const override { return false; }
bool need_input() const override { return !is_finished(); }
bool is_finished() const override { return _is_finished || _aggregator->is_finished(); }
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
Status prepare(RuntimeState* state) override;
void close(RuntimeState* state) override;

View File

@ -29,8 +29,8 @@ bool AggregateDistinctStreamingSourceOperator::is_finished() const {
return _aggregator->is_sink_complete() && _aggregator->is_chunk_buffer_empty() && _aggregator->is_ht_eos();
}
void AggregateDistinctStreamingSourceOperator::set_finished(RuntimeState* state) {
_aggregator->set_finished();
Status AggregateDistinctStreamingSourceOperator::set_finished(RuntimeState* state) {
return _aggregator->set_finished();
}
void AggregateDistinctStreamingSourceOperator::close(RuntimeState* state) {

View File

@ -22,7 +22,7 @@ public:
bool has_output() const override;
bool is_finished() const override;
void set_finished(RuntimeState* state) override;
Status set_finished(RuntimeState* state) override;
void close(RuntimeState* state) override;

View File

@ -17,7 +17,7 @@ void AggregateStreamingSinkOperator::close(RuntimeState* state) {
Operator::close(state);
}
void AggregateStreamingSinkOperator::set_finishing(RuntimeState* state) {
Status AggregateStreamingSinkOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
if (_aggregator->hash_map_variant().size() == 0) {
@ -25,6 +25,7 @@ void AggregateStreamingSinkOperator::set_finishing(RuntimeState* state) {
}
_aggregator->sink_complete();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> AggregateStreamingSinkOperator::pull_chunk(RuntimeState* state) {

View File

@ -20,7 +20,7 @@ public:
bool has_output() const override { return false; }
bool need_input() const override { return !is_finished(); }
bool is_finished() const override { return _is_finished || _aggregator->is_finished(); }
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
Status prepare(RuntimeState* state) override;
void close(RuntimeState* state) override;

View File

@ -43,12 +43,13 @@ bool AggregateStreamingSourceOperator::is_finished() const {
return _is_finished;
}
void AggregateStreamingSourceOperator::set_finishing(RuntimeState* state) {
Status AggregateStreamingSourceOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
return Status::OK();
}
void AggregateStreamingSourceOperator::set_finished(RuntimeState* state) {
_aggregator->set_finished();
Status AggregateStreamingSourceOperator::set_finished(RuntimeState* state) {
return _aggregator->set_finished();
}
void AggregateStreamingSourceOperator::close(RuntimeState* state) {

View File

@ -21,8 +21,8 @@ public:
bool has_output() const override;
bool is_finished() const override;
void set_finishing(RuntimeState* state) override;
void set_finished(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
Status set_finished(RuntimeState* state) override;
void close(RuntimeState* state) override;

View File

@ -18,8 +18,9 @@ bool RepeatOperator::is_finished() const {
return _is_finished && _repeat_times_last >= _repeat_times_required;
}
void RepeatOperator::set_finishing(RuntimeState* state) {
Status RepeatOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
return Status::OK();
}
bool RepeatOperator::has_output() const {

View File

@ -43,7 +43,7 @@ public:
return _repeat_times_last >= _repeat_times_required;
}
bool is_finished() const override;
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
Status prepare(RuntimeState* state) override;

View File

@ -2,6 +2,7 @@
#include "analytic_sink_operator.h"
#include "runtime/current_thread.h"
#include "runtime/runtime_state.h"
namespace starrocks::pipeline {
@ -43,11 +44,12 @@ void AnalyticSinkOperator::close(RuntimeState* state) {
Operator::close(state);
}
void AnalyticSinkOperator::set_finishing(RuntimeState* state) {
Status AnalyticSinkOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
_analytor->input_eos() = true;
_process_by_partition_if_necessary();
RETURN_IF_ERROR(_process_by_partition_if_necessary());
_analytor->sink_complete();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> AnalyticSinkOperator::pull_chunk(RuntimeState* state) {
@ -68,21 +70,22 @@ Status AnalyticSinkOperator::push_chunk(RuntimeState* state, const vectorized::C
// For performance, we do this special handle.
// In future, if need, we could remove this if else easily.
if (j == 0) {
_analytor->append_column(chunk_size, _analytor->agg_intput_columns()[i][j].get(), column);
TRY_CATCH_BAD_ALLOC(
_analytor->append_column(chunk_size, _analytor->agg_intput_columns()[i][j].get(), column));
} else {
_analytor->agg_intput_columns()[i][j]->append(*column, 0, column->size());
TRY_CATCH_BAD_ALLOC(_analytor->agg_intput_columns()[i][j]->append(*column, 0, column->size()));
}
}
}
for (size_t i = 0; i < _analytor->partition_ctxs().size(); i++) {
ColumnPtr column = _analytor->partition_ctxs()[i]->evaluate(chunk.get());
_analytor->append_column(chunk_size, _analytor->partition_columns()[i].get(), column);
TRY_CATCH_BAD_ALLOC(_analytor->append_column(chunk_size, _analytor->partition_columns()[i].get(), column));
}
for (size_t i = 0; i < _analytor->order_ctxs().size(); i++) {
ColumnPtr column = _analytor->order_ctxs()[i]->evaluate(chunk.get());
_analytor->append_column(chunk_size, _analytor->order_columns()[i].get(), column);
TRY_CATCH_BAD_ALLOC(_analytor->append_column(chunk_size, _analytor->order_columns()[i].get(), column));
}
_analytor->input_chunks().emplace_back(std::move(chunk));
@ -92,6 +95,10 @@ Status AnalyticSinkOperator::push_chunk(RuntimeState* state, const vectorized::C
Status AnalyticSinkOperator::_process_by_partition_if_necessary() {
while (_analytor->has_output()) {
if (_analytor->reached_limit()) {
return Status::OK();
}
int64_t found_partition_end = _analytor->find_partition_end();
// Only process after all the data in a partition is reached
if (!_analytor->is_partition_finished(found_partition_end)) {
@ -114,9 +121,6 @@ Status AnalyticSinkOperator::_process_by_partition_if_necessary() {
vectorized::ChunkPtr chunk;
RETURN_IF_ERROR(_analytor->output_result_chunk(&chunk));
_analytor->offer_chunk_to_buffer(chunk);
if (_analytor->reached_limit()) {
return Status::OK();
}
}
}
return Status::OK();

View File

@ -17,8 +17,8 @@ public:
bool has_output() const override { return false; }
bool need_input() const override { return !is_finished(); }
bool is_finished() const override { return _is_finished || _analytor->is_finished(); }
void set_finishing(RuntimeState* state) override;
bool is_finished() const override { return _is_finished || _analytor->reached_limit() || _analytor->is_finished(); }
Status set_finishing(RuntimeState* state) override;
Status prepare(RuntimeState* state) override;
void close(RuntimeState* state) override;

View File

@ -12,8 +12,8 @@ bool AnalyticSourceOperator::is_finished() const {
return _analytor->is_sink_complete() && _analytor->is_chunk_buffer_empty();
}
void AnalyticSourceOperator::set_finished(RuntimeState* state) {
_analytor->set_finished();
Status AnalyticSourceOperator::set_finished(RuntimeState* state) {
return _analytor->set_finished();
}
void AnalyticSourceOperator::close(RuntimeState* state) {

View File

@ -19,7 +19,7 @@ public:
bool has_output() const override;
bool is_finished() const override;
void set_finished(RuntimeState* state) override;
Status set_finished(RuntimeState* state) override;
void close(RuntimeState* state) override;

View File

@ -60,8 +60,9 @@ Status AssertNumRowsOperator::push_chunk(RuntimeState* state, const vectorized::
return Status::OK();
}
void AssertNumRowsOperator::set_finishing(RuntimeState* state) {
Status AssertNumRowsOperator::set_finishing(RuntimeState* state) {
_input_finished = true;
return Status::OK();
}
bool AssertNumRowsOperator::is_finished() const {

View File

@ -39,7 +39,7 @@ public:
bool need_input() const override;
Status push_chunk(RuntimeState* state, const vectorized::ChunkPtr& chunk) override;
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
bool is_finished() const override;
private:

View File

@ -12,12 +12,14 @@
namespace starrocks {
class RuntimeState;
class RuntimeProfile;
namespace pipeline {
class ChunkSource {
public:
ChunkSource(MorselPtr&& morsel) : _morsel(std::move(morsel)) {}
ChunkSource(RuntimeProfile* runtime_profile, MorselPtr&& morsel)
: _runtime_profile(runtime_profile), _morsel(std::move(morsel)){};
virtual ~ChunkSource() = default;
@ -40,8 +42,10 @@ public:
virtual Status buffer_next_batch_chunks_blocking_for_workgroup(size_t chunk_size, bool& can_finish,
size_t* num_read_chunks, int worker_id,
workgroup::WorkGroupPtr running_wg) = 0;
virtual int64_t last_spent_cpu_time_ns() { return 0; }
protected:
RuntimeProfile* _runtime_profile;
// The morsel will own by pipeline driver
MorselPtr _morsel;
};

View File

@ -48,7 +48,10 @@ public:
}
// When the output operator is finished, the context can be finished regardless of other running operators.
void set_finished() { _is_finished.store(true, std::memory_order_release); }
Status set_finished() {
_is_finished.store(true, std::memory_order_release);
return Status::OK();
}
// Predicate whether the context is finished, which is used to notify
// non-output operators to be finished early.
@ -59,4 +62,4 @@ protected:
std::atomic<bool> _is_finished = false;
};
} // namespace starrocks::pipeline
} // namespace starrocks::pipeline

View File

@ -67,9 +67,15 @@ public:
return _is_finished && _is_curr_probe_chunk_finished();
}
void set_finishing(RuntimeState* state) override { _is_finished = true; }
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
return Status::OK();
}
void set_finished(RuntimeState* state) override { _cross_join_context->set_finished(); }
Status set_finished(RuntimeState* state) override {
_cross_join_context->set_finished();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -4,6 +4,7 @@
#include "column/chunk.h"
#include "column/column_helper.h"
#include "runtime/current_thread.h"
using namespace starrocks::vectorized;
@ -22,9 +23,9 @@ Status CrossJoinRightSinkOperator::push_chunk(RuntimeState* state, const vectori
// merge chunks from right table.
size_t col_number = chunk->num_columns();
for (size_t col = 0; col < col_number; ++col) {
_cross_join_context->get_build_chunk(_driver_sequence)
->get_column_by_index(col)
->append(*(chunk->get_column_by_index(col).get()), 0, row_number);
TRY_CATCH_BAD_ALLOC(_cross_join_context->get_build_chunk(_driver_sequence)
->get_column_by_index(col)
->append(*(chunk->get_column_by_index(col).get()), 0, row_number));
}
}
}

View File

@ -34,10 +34,11 @@ public:
bool is_finished() const override { return _is_finished || _cross_join_context->is_finished(); }
void set_finishing(RuntimeState* state) override {
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
// Used to notify cross_join_left_operator.
_cross_join_context->finish_one_right_sinker();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -8,7 +8,7 @@
namespace starrocks::pipeline {
Status DictDecodeOperator::prepare(RuntimeState* state) {
Operator::prepare(state);
RETURN_IF_ERROR(Operator::prepare(state));
return Status::OK();
}

View File

@ -41,7 +41,10 @@ public:
bool is_finished() const override { return _is_finished && _cur_chunk == nullptr; }
void set_finishing(RuntimeState* state) override { _is_finished = true; }
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -38,9 +38,10 @@ bool ExchangeMergeSortSourceOperator::is_finished() const {
}
}
void ExchangeMergeSortSourceOperator::set_finishing(RuntimeState* state) {
Status ExchangeMergeSortSourceOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
return _stream_recvr->close();
_stream_recvr->close();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> ExchangeMergeSortSourceOperator::pull_chunk(RuntimeState* state) {

View File

@ -36,7 +36,7 @@ public:
bool is_finished() const override;
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -41,13 +41,12 @@ public:
// how much tuple data is getting accumulated before being sent; it only applies
// when data is added via add_row() and not sent directly via send_batch().
Channel(ExchangeSinkOperator* parent, const TNetworkAddress& brpc_dest, const TUniqueId& fragment_instance_id,
PlanNodeId dest_node_id, int32_t num_shuffles, int32_t channel_id, bool enable_exchange_pass_through,
PlanNodeId dest_node_id, int32_t num_shuffles, bool enable_exchange_pass_through,
PassThroughChunkBuffer* pass_through_chunk_buffer)
: _parent(parent),
_brpc_dest_addr(brpc_dest),
_fragment_instance_id(fragment_instance_id),
_dest_node_id(dest_node_id),
_channel_id(channel_id),
_enable_exchange_pass_through(enable_exchange_pass_through),
_pass_through_context(pass_through_chunk_buffer, fragment_instance_id, dest_node_id),
_chunks(num_shuffles) {}
@ -102,7 +101,6 @@ private:
const TNetworkAddress _brpc_dest_addr;
const TUniqueId _fragment_instance_id;
const PlanNodeId _dest_node_id;
const int32_t _channel_id;
const bool _enable_exchange_pass_through;
PassThroughContext _pass_through_context;
@ -116,7 +114,7 @@ private:
// always be 1
std::vector<std::unique_ptr<vectorized::Chunk>> _chunks;
PTransmitChunkParamsPtr _chunk_request;
size_t _current_request_bytes;
size_t _current_request_bytes = 0;
bool _is_inited = false;
bool _use_pass_through = false;
@ -302,10 +300,8 @@ ExchangeSinkOperator::ExchangeSinkOperator(OperatorFactory* factory, int32_t id,
if (fragment_instance_id.lo == -1 && pseudo_channel.has_value()) {
_channels.emplace_back(pseudo_channel.value());
} else {
const auto channel_id = _channels.size();
_channels.emplace_back(new Channel(this, destination.brpc_server, fragment_instance_id, dest_node_id,
_num_shuffles, channel_id, enable_exchange_pass_through,
pass_through_chunk_buffer));
_num_shuffles, enable_exchange_pass_through, pass_through_chunk_buffer));
if (fragment_instance_id.lo == -1) {
pseudo_channel = _channels.back();
}
@ -342,7 +338,7 @@ Status ExchangeSinkOperator::prepare(RuntimeState* state) {
_unique_metrics->add_info_string("PartType", _TPartitionType_VALUES_TO_NAMES.at(_part_type));
if (_part_type == TPartitionType::HASH_PARTITIONED ||
_part_type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED) {
_part_type == TPartitionType::BUCKET_SHUFFLE_HASH_PARTITIONED) {
_partitions_columns.resize(_partition_expr_ctxs.size());
}
@ -352,21 +348,12 @@ Status ExchangeSinkOperator::prepare(RuntimeState* state) {
srand(reinterpret_cast<uint64_t>(this));
std::shuffle(_channel_indices.begin(), _channel_indices.end(), std::mt19937(std::random_device()()));
_bytes_sent_counter = ADD_COUNTER(_unique_metrics, "BytesSent", TUnit::BYTES);
_bytes_pass_through_counter = ADD_COUNTER(_unique_metrics, "BytesPassThrough", TUnit::BYTES);
_uncompressed_bytes_counter = ADD_COUNTER(_unique_metrics, "UncompressedBytes", TUnit::BYTES);
_ignore_rows = ADD_COUNTER(_unique_metrics, "IgnoreRows", TUnit::UNIT);
_serialize_batch_timer = ADD_TIMER(_unique_metrics, "SerializeBatchTime");
_shuffle_hash_timer = ADD_TIMER(_unique_metrics, "ShuffleHashTimer");
_serialize_chunk_timer = ADD_TIMER(_unique_metrics, "SerializeChunkTime");
_shuffle_hash_timer = ADD_TIMER(_unique_metrics, "ShuffleHashTime");
_compress_timer = ADD_TIMER(_unique_metrics, "CompressTime");
_send_request_timer = ADD_TIMER(_unique_metrics, "SendRequestTime");
_wait_response_timer = ADD_TIMER(_unique_metrics, "WaitResponseTime");
_overall_throughput = _unique_metrics->add_derived_counter(
"OverallThroughput", TUnit::BYTES_PER_SECOND,
[capture0 = _bytes_sent_counter, capture1 = _total_timer] {
return RuntimeProfile::units_per_second(capture0, capture1);
},
"");
for (auto& _channel : _channels) {
RETURN_IF_ERROR(_channel->init(state));
}
@ -390,8 +377,9 @@ bool ExchangeSinkOperator::pending_finish() const {
return !_buffer->is_finished();
}
void ExchangeSinkOperator::set_cancelled(RuntimeState* state) {
Status ExchangeSinkOperator::set_cancelled(RuntimeState* state) {
_buffer->cancel_one_sinker();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> ExchangeSinkOperator::pull_chunk(RuntimeState* state) {
@ -452,7 +440,7 @@ Status ExchangeSinkOperator::push_chunk(RuntimeState* state, const vectorized::C
_curr_random_channel_idx = (_curr_random_channel_idx + 1) % _channels.size();
}
} else if (_part_type == TPartitionType::HASH_PARTITIONED ||
_part_type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED) {
_part_type == TPartitionType::BUCKET_SHUFFLE_HASH_PARTITIONED) {
// hash-partition batch's rows across channels
const auto num_channels = _channels.size();
{
@ -482,7 +470,8 @@ Status ExchangeSinkOperator::push_chunk(RuntimeState* state, const vectorized::C
for (size_t i = 0; i < num_rows; ++i) {
auto channel_id = _hash_values[i] % num_channels;
auto driver_sequence = _hash_values[i] % _num_shuffles;
// Note that xorshift32 rehash must be applied for both local shuffle and exchange sink here.
auto driver_sequence = HashUtil::xorshift32(_hash_values[i]) % _num_shuffles;
_channel_ids[i] = channel_id;
_driver_sequences[i] = driver_sequence;
_channel_row_idx_start_points[channel_id * _num_shuffles + driver_sequence]++;
@ -522,7 +511,7 @@ Status ExchangeSinkOperator::push_chunk(RuntimeState* state, const vectorized::C
return Status::OK();
}
void ExchangeSinkOperator::set_finishing(RuntimeState* state) {
Status ExchangeSinkOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
if (_chunk_request != nullptr) {
@ -539,9 +528,13 @@ void ExchangeSinkOperator::set_finishing(RuntimeState* state) {
for (auto& _channel : _channels) {
_channel->close(state, _fragment_ctx);
}
_buffer->set_finishing();
return Status::OK();
}
void ExchangeSinkOperator::close(RuntimeState* state) {
_buffer->update_profile(_unique_metrics.get());
Operator::close(state);
}
@ -549,7 +542,7 @@ Status ExchangeSinkOperator::serialize_chunk(const vectorized::Chunk* src, Chunk
int num_receivers) {
VLOG_ROW << "[ExchangeSinkOperator] serializing " << src->num_rows() << " rows";
{
SCOPED_TIMER(_serialize_batch_timer);
SCOPED_TIMER(_serialize_chunk_timer);
// We only serialize chunk meta for first chunk
if (*is_first_chunk) {
StatusOr<ChunkPB> res = serde::ProtobufChunkSerde::serialize(*src);
@ -567,8 +560,8 @@ Status ExchangeSinkOperator::serialize_chunk(const vectorized::Chunk* src, Chunk
const size_t uncompressed_size = dst->uncompressed_size();
if (_compress_codec != nullptr && _compress_codec->exceed_max_input_size(uncompressed_size)) {
return Status::InternalError("The input size for compression should be less than " +
_compress_codec->max_input_size());
return Status::InternalError(strings::Substitute("The input size for compression should be less than $0",
_compress_codec->max_input_size()));
}
// try compress the ChunkPB data
@ -596,7 +589,6 @@ Status ExchangeSinkOperator::serialize_chunk(const vectorized::Chunk* src, Chunk
size_t chunk_size = dst->data().size();
VLOG_ROW << "chunk data size " << chunk_size;
COUNTER_UPDATE(_bytes_sent_counter, chunk_size * num_receivers);
COUNTER_UPDATE(_uncompressed_bytes_counter, uncompressed_size * num_receivers);
return Status::OK();
}
@ -637,7 +629,7 @@ Status ExchangeSinkOperatorFactory::prepare(RuntimeState* state) {
RETURN_IF_ERROR(OperatorFactory::prepare(state));
if (_part_type == TPartitionType::HASH_PARTITIONED ||
_part_type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED) {
_part_type == TPartitionType::BUCKET_SHUFFLE_HASH_PARTITIONED) {
RETURN_IF_ERROR(Expr::prepare(_partition_expr_ctxs, state));
RETURN_IF_ERROR(Expr::open(_partition_expr_ctxs, state));
}

View File

@ -51,9 +51,9 @@ public:
bool pending_finish() const override;
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
void set_cancelled(RuntimeState* state) override;
Status set_cancelled(RuntimeState* state) override;
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;
@ -115,18 +115,11 @@ private:
CompressionTypePB _compress_type = CompressionTypePB::NO_COMPRESSION;
const BlockCompressionCodec* _compress_codec = nullptr;
RuntimeProfile::Counter* _serialize_batch_timer = nullptr;
RuntimeProfile::Counter* _serialize_chunk_timer = nullptr;
RuntimeProfile::Counter* _shuffle_hash_timer = nullptr;
RuntimeProfile::Counter* _compress_timer = nullptr;
RuntimeProfile::Counter* _bytes_sent_counter = nullptr;
RuntimeProfile::Counter* _bytes_pass_through_counter = nullptr;
RuntimeProfile::Counter* _uncompressed_bytes_counter = nullptr;
RuntimeProfile::Counter* _ignore_rows = nullptr;
RuntimeProfile::Counter* _send_request_timer = nullptr;
RuntimeProfile::Counter* _wait_response_timer = nullptr;
// Throughput per total time spent in sender
RuntimeProfile::Counter* _overall_throughput = nullptr;
std::atomic<bool> _is_finished = false;
std::atomic<bool> _is_cancelled = false;

View File

@ -24,10 +24,11 @@ bool ExchangeSourceOperator::is_finished() const {
return _stream_recvr->is_finished();
}
void ExchangeSourceOperator::set_finishing(RuntimeState* state) {
Status ExchangeSourceOperator::set_finishing(RuntimeState* state) {
_is_finishing = true;
_stream_recvr->short_circuit_for_pipeline(_driver_sequence);
static_cast<ExchangeSourceOperatorFactory*>(_factory)->close_stream_recvr();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> ExchangeSourceOperator::pull_chunk(RuntimeState* state) {

View File

@ -23,7 +23,7 @@ public:
bool is_finished() const override;
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -32,6 +32,17 @@ Status PartitionExchanger::Partitioner::partition_chunk(const vectorized::ChunkP
}
}
// When the local exchange is the successor of ExchangeSourceOperator,
// the data flow is `exchange sink -> exchange source -> local exchange`.
// `exchange sink -> exchange source` (phase 1) determines which fragment instance to deliver each row,
// while `exchange source -> local exchange` (phase 2) determines which pipeline driver to deliver each row.
// To avoid hash two times and data skew due to hash one time, phase 1 hashes one time
// and phase 2 applies xorshift32 on the hash value.
// Note that xorshift32 rehash must be applied for both local shuffle here and exchange sink.
for (int32_t i = 0; i < num_rows; ++i) {
_hash_values[i] = HashUtil::xorshift32(_hash_values[i]);
}
// Compute row indexes for each channel.
_partition_row_indexes_start_points.assign(num_partitions + 1, 0);
for (int32_t i = 0; i < num_rows; ++i) {

View File

@ -20,9 +20,10 @@ StatusOr<vectorized::ChunkPtr> LocalExchangeSinkOperator::pull_chunk(RuntimeStat
return Status::InternalError("Shouldn't call pull_chunk from local exchange sink.");
}
void LocalExchangeSinkOperator::set_finishing(RuntimeState* state) {
Status LocalExchangeSinkOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
_exchanger->finish(state);
return Status::OK();
}
Status LocalExchangeSinkOperator::push_chunk(RuntimeState* state, const vectorized::ChunkPtr& chunk) {

View File

@ -32,7 +32,7 @@ public:
// In either case, LocalExchangeSinkOperator is finished.
bool is_finished() const override { return _is_finished || _exchanger->is_all_sources_finished(); }
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -49,7 +49,7 @@ bool LocalExchangeSourceOperator::has_output() const {
(_is_finished && _partition_rows_num > 0);
}
void LocalExchangeSourceOperator::set_finished(RuntimeState* state) {
Status LocalExchangeSourceOperator::set_finished(RuntimeState* state) {
std::lock_guard<std::mutex> l(_chunk_lock);
_is_finished = true;
// Compute out the number of rows of the _full_chunk_queue.
@ -66,6 +66,7 @@ void LocalExchangeSourceOperator::set_finished(RuntimeState* state) {
// Subtract the number of rows of buffered chunks from row_count of _memory_manager and make it unblocked.
_memory_manager->update_row_count(-(full_rows_num + _partition_rows_num));
_partition_rows_num = 0;
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> LocalExchangeSourceOperator::pull_chunk(RuntimeState* state) {

View File

@ -41,10 +41,11 @@ public:
bool is_finished() const override;
void set_finished(RuntimeState* state) override;
void set_finishing(RuntimeState* state) override {
Status set_finished(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override {
std::lock_guard<std::mutex> l(_chunk_lock);
_is_finished = true;
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -182,11 +182,12 @@ Status MultiCastLocalExchangeSourceOperator::prepare(RuntimeState* state) {
return Status::OK();
}
void MultiCastLocalExchangeSourceOperator::set_finishing(RuntimeState* state) {
Status MultiCastLocalExchangeSourceOperator::set_finishing(RuntimeState* state) {
if (!_is_finished) {
_is_finished = true;
_exchanger->close_source_operator(_mcast_consumer_index);
}
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> MultiCastLocalExchangeSourceOperator::pull_chunk(RuntimeState* state) {
@ -215,11 +216,12 @@ bool MultiCastLocalExchangeSinkOperator::need_input() const {
return _exchanger->can_push_chunk();
}
void MultiCastLocalExchangeSinkOperator::set_finishing(RuntimeState* state) {
Status MultiCastLocalExchangeSinkOperator::set_finishing(RuntimeState* state) {
if (!_is_finished) {
_is_finished = true;
_exchanger->close_sink_operator();
}
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> MultiCastLocalExchangeSinkOperator::pull_chunk(RuntimeState* state) {

View File

@ -90,7 +90,7 @@ public:
bool is_finished() const override { return _is_finished; }
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;
@ -139,7 +139,7 @@ public:
bool is_finished() const override { return _is_finished; }
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -2,6 +2,10 @@
#include "exec/pipeline/exchange/sink_buffer.h"
#include <chrono>
#include "util/time.h"
namespace starrocks::pipeline {
SinkBuffer::SinkBuffer(FragmentContext* fragment_ctx, const std::vector<TPlanFragmentDestination>& destinations,
@ -31,6 +35,7 @@ SinkBuffer::SinkBuffer(FragmentContext* fragment_ctx, const std::vector<TPlanFra
_buffers[instance_id.lo] = std::queue<TransmitChunkInfo, std::list<TransmitChunkInfo>>();
_num_finished_rpcs[instance_id.lo] = 0;
_num_in_flight_rpcs[instance_id.lo] = 0;
_network_times[instance_id.lo] = TimeTrace{};
_mutexes[instance_id.lo] = std::make_unique<std::mutex>();
PUniqueId finst_id;
@ -64,7 +69,7 @@ SinkBuffer::~SinkBuffer() {
}
}
void SinkBuffer::add_request(const TransmitChunkInfo& request) {
void SinkBuffer::add_request(TransmitChunkInfo& request) {
DCHECK(_num_remaining_eos > 0);
if (_is_finishing) {
// Once the request is added to SinkBuffer, its ownership will also be transferred,
@ -72,6 +77,10 @@ void SinkBuffer::add_request(const TransmitChunkInfo& request) {
request.params->release_finst_id();
return;
}
if (!request.attachment.empty()) {
_bytes_enqueued += request.attachment.size();
_request_enqueued++;
}
{
auto& instance_id = request.fragment_instance_id;
std::lock_guard<std::mutex> l(*_mutexes[instance_id.lo]);
@ -89,7 +98,21 @@ bool SinkBuffer::is_full() const {
for (auto& [_, buffer] : _buffers) {
buffer_size += buffer.size();
}
return buffer_size > max_buffer_size;
bool is_full = buffer_size > max_buffer_size;
if (is_full && _last_full_timestamp == -1) {
_last_full_timestamp = MonotonicNanos();
}
if (!is_full && _last_full_timestamp != -1) {
_full_time += (MonotonicNanos() - _last_full_timestamp);
_last_full_timestamp = -1;
}
return is_full;
}
void SinkBuffer::set_finishing() {
_pending_timestamp = MonotonicNanos();
}
bool SinkBuffer::is_finished() const {
@ -100,14 +123,68 @@ bool SinkBuffer::is_finished() const {
return _num_sending_rpc == 0 && _total_in_flight_rpc == 0;
}
// When all the ExchangeSinkOperator shared this SinkBuffer are cancelled,
// the rest chunk request and EOS request needn't be sent anymore.
void SinkBuffer::update_profile(RuntimeProfile* profile) {
bool flag = false;
if (!_is_profile_updated.compare_exchange_strong(flag, true)) {
return;
}
auto* network_timer = ADD_TIMER(profile, "NetworkTime");
auto* wait_timer = ADD_TIMER(profile, "WaitTime");
COUNTER_SET(network_timer, _network_time());
// WaitTime consists two parts
// 1. buffer full time
// 2. pending finish time
COUNTER_UPDATE(wait_timer, _full_time);
COUNTER_UPDATE(wait_timer, MonotonicNanos() - _pending_timestamp);
auto* bytes_sent_counter = ADD_COUNTER(profile, "BytesSent", TUnit::BYTES);
auto* request_sent_counter = ADD_COUNTER(profile, "RequestSent", TUnit::UNIT);
COUNTER_SET(bytes_sent_counter, _bytes_sent);
COUNTER_SET(request_sent_counter, _request_sent);
if (_bytes_enqueued - _bytes_sent > 0) {
auto* bytes_unsent_counter = ADD_COUNTER(profile, "BytesUnsent", TUnit::BYTES);
auto* request_unsent_counter = ADD_COUNTER(profile, "RequestUnsent", TUnit::UNIT);
COUNTER_SET(bytes_unsent_counter, _bytes_enqueued - _bytes_sent);
COUNTER_SET(request_unsent_counter, _request_enqueued - _request_sent);
}
profile->add_derived_counter(
"OverallThroughput", TUnit::BYTES_PER_SECOND,
[bytes_sent_counter, network_timer] {
return RuntimeProfile::units_per_second(bytes_sent_counter, network_timer);
},
"");
}
int64_t SinkBuffer::_network_time() {
int64_t max = 0;
for (auto& [_, time_trace] : _network_times) {
double average_concurrency =
static_cast<double>(time_trace.accumulated_concurrency) / std::max(1, time_trace.times);
int64_t average_accumulated_time =
static_cast<int64_t>(time_trace.accumulated_time / std::max(1.0, average_concurrency));
if (average_accumulated_time > max) {
max = average_accumulated_time;
}
}
return max;
}
void SinkBuffer::cancel_one_sinker() {
if (--_num_uncancelled_sinkers == 0) {
_is_finishing = true;
}
}
void SinkBuffer::_update_network_time(const TUniqueId& instance_id, const int64_t send_timestamp,
const int64_t receive_timestamp) {
int32_t concurrency = _num_in_flight_rpcs[instance_id.lo];
_network_times[instance_id.lo].update(receive_timestamp - send_timestamp, concurrency);
}
void SinkBuffer::_process_send_window(const TUniqueId& instance_id, const int64_t sequence) {
// Both sender side and receiver side can tolerate disorder of tranmission
// if receiver side is not ExchangeMergeSortSourceOperator
@ -202,8 +279,14 @@ void SinkBuffer::_try_to_send_rpc(const TUniqueId& instance_id) {
request.params->set_allocated_finst_id(&_instance_id2finst_id[instance_id.lo]);
request.params->set_sequence(_request_seqs[instance_id.lo]++);
auto* closure =
new DisposableClosure<PTransmitChunkResult, ClosureContext>({instance_id, request.params->sequence()});
if (!request.attachment.empty()) {
_bytes_sent += request.attachment.size();
_request_sent++;
}
auto* closure = new DisposableClosure<PTransmitChunkResult, ClosureContext>(
{instance_id, request.params->sequence(), GetCurrentTimeNanos()});
closure->addFailedHandler([this](const ClosureContext& ctx) noexcept {
_is_finishing = true;
{
@ -229,6 +312,7 @@ void SinkBuffer::_try_to_send_rpc(const TUniqueId& instance_id) {
} else {
std::lock_guard<std::mutex> l(*_mutexes[ctx.instance_id.lo]);
_process_send_window(ctx.instance_id, ctx.sequence);
_update_network_time(ctx.instance_id, ctx.send_timestamp, result.receive_timestamp());
_try_to_send_rpc(ctx.instance_id);
}
--_total_in_flight_rpc;

View File

@ -24,7 +24,7 @@ namespace starrocks::pipeline {
using PTransmitChunkParamsPtr = std::shared_ptr<PTransmitChunkParams>;
struct TransmitChunkInfo {
// For BUCKET_SHFFULE_HASH_PARTITIONED, multiple channels may be related to
// For BUCKET_SHUFFLE_HASH_PARTITIONED, multiple channels may be related to
// a same exchange source fragment instance, so we should use fragment_instance_id
// of the destination as the key of destination instead of channel_id.
TUniqueId fragment_instance_id;
@ -36,6 +36,27 @@ struct TransmitChunkInfo {
struct ClosureContext {
TUniqueId instance_id;
int64_t sequence;
int64_t send_timestamp;
};
// TimeTrace is introduced to estimate time more accurately.
// For every update
// 1. times will be increased by 1.
// 2. sample time will be accumulated to accumulated_time.
// 3. sample concurrency will be accumulated to accumulated_concurrency.
// So we can get the average time of each direction by
// `average_concurrency = accumulated_concurrency / times`
// `average_time = accumulated_time / average_concurrency`
struct TimeTrace {
int32_t times = 0;
int64_t accumulated_time = 0;
int32_t accumulated_concurrency = 0;
void update(int64_t time, int32_t concurrency) {
times++;
accumulated_time += time;
accumulated_concurrency += concurrency;
}
};
// TODO(hcf) how to export brpc error
@ -45,15 +66,22 @@ public:
bool is_dest_merge, size_t num_sinkers);
~SinkBuffer();
void add_request(const TransmitChunkInfo& request);
void add_request(TransmitChunkInfo& request);
bool is_full() const;
void set_finishing();
bool is_finished() const;
// Add counters to the given profile
void update_profile(RuntimeProfile* profile);
// When all the ExchangeSinkOperator shared this SinkBuffer are cancelled,
// the rest chunk request and EOS request needn't be sent anymore.
void cancel_one_sinker();
private:
void _update_network_time(const TUniqueId& instance_id, const int64_t send_timestamp,
const int64_t receive_timestamp);
// Update the discontinuous acked window, here are the invariants:
// all acks received with sequence from [0, _max_continuous_acked_seqs[x]]
// not all the acks received with sequence from [_max_continuous_acked_seqs[x]+1, _request_seqs[x]]
@ -61,6 +89,14 @@ private:
void _process_send_window(const TUniqueId& instance_id, const int64_t sequence);
void _try_to_send_rpc(const TUniqueId& instance_id);
// Roughly estimate network time which is defined as the time between sending a and receiving a packet,
// and the processing time of both sides are excluded
// For each destination, we may send multiply packages at the same time, and the time is
// related to the degree of concurrency, so the network_time will be calculated as
// `accumulated_network_time / average_concurrency`
// And we just pick the maximum accumulated_network_time among all destination
int64_t _network_time();
FragmentContext* _fragment_ctx;
const MemTracker* _mem_tracker;
const int32_t _brpc_timeout_ms;
@ -91,6 +127,7 @@ private:
phmap::flat_hash_map<int64_t, std::queue<TransmitChunkInfo, std::list<TransmitChunkInfo>>> _buffers;
phmap::flat_hash_map<int64_t, int32_t> _num_finished_rpcs;
phmap::flat_hash_map<int64_t, int32_t> _num_in_flight_rpcs;
phmap::flat_hash_map<int64_t, TimeTrace> _network_times;
phmap::flat_hash_map<int64_t, std::unique_ptr<std::mutex>> _mutexes;
// True means that SinkBuffer needn't input chunk and send chunk anymore,
@ -106,6 +143,16 @@ private:
std::atomic<bool> _is_finishing = false;
std::atomic<int32_t> _num_sending_rpc = 0;
// RuntimeProfile counters
std::atomic_bool _is_profile_updated = false;
std::atomic<int64_t> _bytes_enqueued = 0;
std::atomic<int64_t> _request_enqueued = 0;
std::atomic<int64_t> _bytes_sent = 0;
std::atomic<int64_t> _request_sent = 0;
int64_t _pending_timestamp = -1;
mutable int64_t _last_full_timestamp = -1;
mutable int64_t _full_time = 0;
}; // namespace starrocks::pipeline
} // namespace starrocks::pipeline

View File

@ -53,8 +53,8 @@ public:
RuntimeState* runtime_state() const { return _runtime_state.get(); }
std::shared_ptr<RuntimeState> runtime_state_ptr() { return _runtime_state; }
void set_runtime_state(std::shared_ptr<RuntimeState>&& runtime_state) { _runtime_state = std::move(runtime_state); }
ExecNode* plan() const { return _plan; }
void set_plan(ExecNode* plan) { _plan = plan; }
ExecNode*& plan() { return _plan; }
Pipelines& pipelines() { return _pipelines; }
void set_pipelines(Pipelines&& pipelines) { _pipelines = std::move(pipelines); }
Drivers& drivers() { return _drivers; }

View File

@ -41,8 +41,10 @@ static void setup_profile_hierarchy(RuntimeState* runtime_state, const PipelineP
static void setup_profile_hierarchy(const PipelinePtr& pipeline, const DriverPtr& driver) {
pipeline->runtime_profile()->add_child(driver->runtime_profile(), true, nullptr);
auto* counter = pipeline->runtime_profile()->add_counter("DegreeOfParallelism", TUnit::UNIT);
counter->set(static_cast<int64_t>(pipeline->source_operator_factory()->degree_of_parallelism()));
auto* dop_counter = ADD_COUNTER(pipeline->runtime_profile(), "DegreeOfParallelism", TUnit::UNIT);
COUNTER_SET(dop_counter, static_cast<int64_t>(pipeline->source_operator_factory()->degree_of_parallelism()));
auto* total_dop_counter = ADD_COUNTER(pipeline->runtime_profile(), "TotalDegreeOfParallelism", TUnit::UNIT);
COUNTER_SET(total_dop_counter, dop_counter->value());
auto& operators = driver->operators();
for (int32_t i = operators.size() - 1; i >= 0; --i) {
auto& curr_op = operators[i];
@ -145,8 +147,11 @@ Status FragmentExecutor::prepare(ExecEnv* exec_env, const TExecPlanFragmentParam
std::make_unique<RuntimeState>(query_id, fragment_instance_id, query_options, query_globals, exec_env));
}
auto* runtime_state = _fragment_ctx->runtime_state();
int func_version = request.__isset.func_version ? request.func_version : 2;
runtime_state->set_func_version(func_version);
runtime_state->init_mem_trackers(query_id, wg != nullptr ? wg->mem_tracker() : nullptr);
runtime_state->set_be_number(backend_num);
runtime_state->set_query_ctx(_query_ctx);
// RuntimeFilterWorker::open_query is idempotent
if (params.__isset.runtime_filter_params && params.runtime_filter_params.id_to_prober_params.size() != 0) {
@ -175,13 +180,12 @@ Status FragmentExecutor::prepare(ExecEnv* exec_env, const TExecPlanFragmentParam
}
runtime_state->set_desc_tbl(desc_tbl);
// Set up plan
ExecNode* plan = nullptr;
RETURN_IF_ERROR(ExecNode::create_tree(runtime_state, obj_pool, fragment.plan, *desc_tbl, &plan));
RETURN_IF_ERROR(ExecNode::create_tree(runtime_state, obj_pool, fragment.plan, *desc_tbl, &_fragment_ctx->plan()));
ExecNode* plan = _fragment_ctx->plan();
plan->push_down_join_runtime_filter_recursively(runtime_state);
std::vector<TupleSlotMapping> empty_mappings;
plan->push_down_tuple_slot_mappings(runtime_state, empty_mappings);
runtime_state->set_fragment_root_id(plan->id());
_fragment_ctx->set_plan(plan);
// Set up global dict
if (request.fragment.__isset.query_global_dicts) {
@ -236,8 +240,8 @@ Status FragmentExecutor::prepare(ExecEnv* exec_env, const TExecPlanFragmentParam
const auto& pipeline = pipelines[n];
// DOP(degree of parallelism) of Pipeline's SourceOperator determines the Pipeline's DOP.
const auto degree_of_parallelism = pipeline->source_operator_factory()->degree_of_parallelism();
LOG(INFO) << "Pipeline " << pipeline->to_readable_string() << " parallel=" << degree_of_parallelism
<< " fragment_instance_id=" << print_id(params.fragment_instance_id);
VLOG_ROW << "Pipeline " << pipeline->to_readable_string() << " parallel=" << degree_of_parallelism
<< " fragment_instance_id=" << print_id(params.fragment_instance_id);
const bool is_root = pipeline->is_root();
// If pipeline's SourceOperator is with morsels, a MorselQueue is added to the SourceOperator.
// at present, only OlapScanOperator need a MorselQueue attached.
@ -339,9 +343,14 @@ void FragmentExecutor::_decompose_data_sink_to_operator(RuntimeState* runtime_st
bool is_pipeline_level_shuffle = false;
int32_t dest_dop = -1;
if (sender->get_partition_type() == TPartitionType::HASH_PARTITIONED ||
sender->get_partition_type() == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED) {
is_pipeline_level_shuffle = true;
sender->get_partition_type() == TPartitionType::BUCKET_SHUFFLE_HASH_PARTITIONED) {
dest_dop = t_stream_sink.dest_dop;
// UNPARTITIONED mode will be performed if both num of destination and dest dop is 1
// So we only enable pipeline level shuffle when num of destination or dest dop is greater than 1
if (sender->destinations().size() > 1 || dest_dop > 1) {
is_pipeline_level_shuffle = true;
}
DCHECK_GT(dest_dop, 0);
}
@ -406,9 +415,10 @@ void FragmentExecutor::_decompose_data_sink_to_operator(RuntimeState* runtime_st
// sink op
auto sink_buffer = std::make_shared<SinkBuffer>(_fragment_ctx, sender->destinations(), is_dest_merge, dop);
auto sink_op = std::make_shared<ExchangeSinkOperatorFactory>(
context->next_operator_id(), -1, sink_buffer, sender->get_partition_type(), sender->destinations(),
is_pipeline_level_shuffle, dest_dop, sender->sender_id(), sender->get_dest_node_id(),
sender->get_partition_exprs(), sender->get_enable_exchange_pass_through(), _fragment_ctx);
context->next_operator_id(), t_stream_sink.dest_node_id, sink_buffer, sender->get_partition_type(),
sender->destinations(), is_pipeline_level_shuffle, dest_dop, sender->sender_id(),
sender->get_dest_node_id(), sender->get_partition_exprs(),
sender->get_enable_exchange_pass_through(), _fragment_ctx);
ops.emplace_back(source_op);
ops.emplace_back(sink_op);

View File

@ -50,26 +50,32 @@ StatusOr<vectorized::ChunkPtr> HashJoinBuildOperator::pull_chunk(RuntimeState* s
return Status::NotSupported(msg);
}
void HashJoinBuildOperator::set_finishing(RuntimeState* state) {
Status HashJoinBuildOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
_join_builder->build_ht(state);
RETURN_IF_ERROR(_join_builder->build_ht(state));
size_t merger_index = _driver_sequence;
// Broadcast Join only has one build operator.
DCHECK(_distribution_mode != TJoinDistributionMode::BROADCAST || _driver_sequence == 0);
_join_builder->create_runtime_filters(state);
RETURN_IF_ERROR(_join_builder->create_runtime_filters(state));
auto ht_row_count = _join_builder->get_ht_row_count();
auto& partial_in_filters = _join_builder->get_runtime_in_filters();
auto& partial_bloom_filter_build_params = _join_builder->get_runtime_bloom_filter_build_params();
auto& partial_bloom_filters = _join_builder->get_runtime_bloom_filters();
// retain string-typed key columns to avoid premature de-allocation when both probe side and build side
// PipelineDrivers finalization before in-filers is merged.
((HashJoinBuildOperatorFactory*)_factory)
->retain_string_key_columns(_driver_sequence, _join_builder->string_key_columns());
// add partial filters generated by this HashJoinBuildOperator to PartialRuntimeFilterMerger to merge into a
// total one.
auto status = _partial_rf_merger->add_partial_filters(merger_index, ht_row_count, std::move(partial_in_filters),
std::move(partial_bloom_filter_build_params),
std::move(partial_bloom_filters));
if (status.ok() && status.value()) {
if (!status.ok()) {
return status.status();
} else if (status.value()) {
auto&& in_filters = _partial_rf_merger->get_total_in_filters();
auto&& bloom_filters = _partial_rf_merger->get_total_bloom_filters();
@ -87,6 +93,7 @@ void HashJoinBuildOperator::set_finishing(RuntimeState* state) {
for (auto& read_only_join_prober : _read_only_join_probers) {
read_only_join_prober->enter_probe_phase();
}
return Status::OK();
}
HashJoinBuildOperatorFactory::HashJoinBuildOperatorFactory(
@ -109,11 +116,17 @@ void HashJoinBuildOperatorFactory::close(RuntimeState* state) {
}
OperatorPtr HashJoinBuildOperatorFactory::create(int32_t degree_of_parallelism, int32_t driver_sequence) {
if (_string_key_columns.empty()) {
_string_key_columns.resize(degree_of_parallelism);
}
return std::make_shared<HashJoinBuildOperator>(this, _id, _name, _plan_node_id,
_hash_joiner_factory->create_builder(driver_sequence),
_hash_joiner_factory->get_read_only_probers(), driver_sequence,
_partial_rf_merger.get(), _distribution_mode);
}
void HashJoinBuildOperatorFactory::retain_string_key_columns(int32_t driver_sequence, vectorized::Columns&& columns) {
_string_key_columns[driver_sequence] = std::move(columns);
}
} // namespace pipeline
} // namespace starrocks

View File

@ -35,7 +35,7 @@ public:
}
bool need_input() const override { return !is_finished(); }
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
bool is_finished() const override { return _is_finished || _join_builder->is_finished(); }
Status push_chunk(RuntimeState* state, const vectorized::ChunkPtr& chunk) override;
@ -66,11 +66,12 @@ public:
Status prepare(RuntimeState* state) override;
void close(RuntimeState* state) override;
OperatorPtr create(int32_t degree_of_parallelism, int32_t driver_sequence) override;
void retain_string_key_columns(int32_t driver_sequence, vectorized::Columns&& columns);
private:
HashJoinerFactoryPtr _hash_joiner_factory;
std::unique_ptr<PartialRuntimeFilterMerger> _partial_rf_merger;
std::vector<vectorized::Columns> _string_key_columns;
const TJoinDistributionMode::type _distribution_mode;
};

View File

@ -55,14 +55,16 @@ StatusOr<vectorized::ChunkPtr> HashJoinProbeOperator::pull_chunk(RuntimeState* s
return _join_prober->pull_chunk(state);
}
void HashJoinProbeOperator::set_finishing(RuntimeState* state) {
Status HashJoinProbeOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
_join_prober->enter_post_probe_phase();
return Status::OK();
}
void HashJoinProbeOperator::set_finished(RuntimeState* state) {
Status HashJoinProbeOperator::set_finished(RuntimeState* state) {
_join_prober->enter_eos_phase();
_join_builder->set_prober_finished();
return Status::OK();
}
bool HashJoinProbeOperator::is_ready() const {

View File

@ -27,8 +27,8 @@ public:
bool need_input() const override;
bool is_finished() const override;
void set_finishing(RuntimeState* state) override;
void set_finished(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
Status set_finished(RuntimeState* state) override;
bool is_ready() const override;
std::string get_name() const override {

View File

@ -19,7 +19,10 @@ public:
bool is_finished() const override { return (_is_finished || _limit == 0) && _cur_chunk == nullptr; }
void set_finishing(RuntimeState* state) override { _is_finished = true; }
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -23,13 +23,13 @@
namespace starrocks::pipeline {
using namespace vectorized;
OlapChunkSource::OlapChunkSource(MorselPtr&& morsel, ScanOperator* op, vectorized::OlapScanNode* scan_node)
: ChunkSource(std::move(morsel)),
OlapChunkSource::OlapChunkSource(RuntimeProfile* runtime_profile, MorselPtr&& morsel, ScanOperator* op,
vectorized::OlapScanNode* scan_node)
: ChunkSource(runtime_profile, std::move(morsel)),
_scan_node(scan_node),
_limit(scan_node->limit()),
_runtime_in_filters(op->runtime_in_filters()),
_runtime_bloom_filters(op->runtime_bloom_filters()),
_runtime_profile(op->unique_metrics()) {
_runtime_bloom_filters(op->runtime_bloom_filters()) {
_conjunct_ctxs = scan_node->conjunct_ctxs();
_conjunct_ctxs.insert(_conjunct_ctxs.end(), _runtime_in_filters.begin(), _runtime_in_filters.end());
ScanMorsel* scan_morsel = (ScanMorsel*)_morsel.get();
@ -42,6 +42,14 @@ Status OlapChunkSource::prepare(RuntimeState* state) {
const TupleDescriptor* tuple_desc = state->desc_tbl().get_tuple_descriptor(thrift_olap_scan_node.tuple_id);
_slots = &tuple_desc->slots();
_runtime_profile->add_info_string("Table", tuple_desc->table_desc()->name());
if (thrift_olap_scan_node.__isset.rollup_name) {
_runtime_profile->add_info_string("Rollup", thrift_olap_scan_node.rollup_name);
}
if (thrift_olap_scan_node.__isset.sql_predicates) {
_runtime_profile->add_info_string("Predicates", thrift_olap_scan_node.sql_predicates);
}
_init_counter(state);
_dict_optimize_parser.set_mutable_dict_maps(state->mutable_query_global_dict_map());
@ -77,45 +85,43 @@ void OlapChunkSource::_init_counter(RuntimeState* state) {
_bytes_read_counter = ADD_COUNTER(_runtime_profile, "BytesRead", TUnit::BYTES);
_rows_read_counter = ADD_COUNTER(_runtime_profile, "RowsRead", TUnit::UNIT);
_scan_profile = _runtime_profile->create_child("SCAN", true, false);
_create_seg_iter_timer = ADD_TIMER(_runtime_profile, "CreateSegmentIter");
_create_seg_iter_timer = ADD_TIMER(_scan_profile, "CreateSegmentIter");
_read_compressed_counter = ADD_COUNTER(_runtime_profile, "CompressedBytesRead", TUnit::BYTES);
_read_uncompressed_counter = ADD_COUNTER(_runtime_profile, "UncompressedBytesRead", TUnit::BYTES);
_read_compressed_counter = ADD_COUNTER(_scan_profile, "CompressedBytesRead", TUnit::BYTES);
_read_uncompressed_counter = ADD_COUNTER(_scan_profile, "UncompressedBytesRead", TUnit::BYTES);
_raw_rows_counter = ADD_COUNTER(_scan_profile, "RawRowsRead", TUnit::UNIT);
_read_pages_num_counter = ADD_COUNTER(_scan_profile, "ReadPagesNum", TUnit::UNIT);
_cached_pages_num_counter = ADD_COUNTER(_scan_profile, "CachedPagesNum", TUnit::UNIT);
_pushdown_predicates_counter = ADD_COUNTER(_scan_profile, "PushdownPredicates", TUnit::UNIT);
_raw_rows_counter = ADD_COUNTER(_runtime_profile, "RawRowsRead", TUnit::UNIT);
_read_pages_num_counter = ADD_COUNTER(_runtime_profile, "ReadPagesNum", TUnit::UNIT);
_cached_pages_num_counter = ADD_COUNTER(_runtime_profile, "CachedPagesNum", TUnit::UNIT);
_pushdown_predicates_counter = ADD_COUNTER(_runtime_profile, "PushdownPredicates", TUnit::UNIT);
// SegmentInit
_seg_init_timer = ADD_TIMER(_scan_profile, "SegmentInit");
_bi_filter_timer = ADD_CHILD_TIMER(_scan_profile, "BitmapIndexFilter", "SegmentInit");
_bi_filtered_counter = ADD_CHILD_COUNTER(_scan_profile, "BitmapIndexFilterRows", TUnit::UNIT, "SegmentInit");
_bf_filtered_counter = ADD_CHILD_COUNTER(_scan_profile, "BloomFilterFilterRows", TUnit::UNIT, "SegmentInit");
_zm_filtered_counter = ADD_CHILD_COUNTER(_scan_profile, "ZoneMapIndexFilterRows", TUnit::UNIT, "SegmentInit");
_sk_filtered_counter = ADD_CHILD_COUNTER(_scan_profile, "ShortKeyFilterRows", TUnit::UNIT, "SegmentInit");
_seg_init_timer = ADD_TIMER(_runtime_profile, "SegmentInit");
_bi_filter_timer = ADD_CHILD_TIMER(_runtime_profile, "BitmapIndexFilter", "SegmentInit");
_bi_filtered_counter = ADD_CHILD_COUNTER(_runtime_profile, "BitmapIndexFilterRows", TUnit::UNIT, "SegmentInit");
_bf_filtered_counter = ADD_CHILD_COUNTER(_runtime_profile, "BloomFilterFilterRows", TUnit::UNIT, "SegmentInit");
_zm_filtered_counter = ADD_CHILD_COUNTER(_runtime_profile, "ZoneMapIndexFilterRows", TUnit::UNIT, "SegmentInit");
_sk_filtered_counter = ADD_CHILD_COUNTER(_runtime_profile, "ShortKeyFilterRows", TUnit::UNIT, "SegmentInit");
// SegmentRead
_block_load_timer = ADD_TIMER(_scan_profile, "SegmentRead");
_block_fetch_timer = ADD_CHILD_TIMER(_scan_profile, "BlockFetch", "SegmentRead");
_block_load_counter = ADD_CHILD_COUNTER(_scan_profile, "BlockFetchCount", TUnit::UNIT, "SegmentRead");
_block_seek_timer = ADD_CHILD_TIMER(_scan_profile, "BlockSeek", "SegmentRead");
_block_seek_counter = ADD_CHILD_COUNTER(_scan_profile, "BlockSeekCount", TUnit::UNIT, "SegmentRead");
_pred_filter_timer = ADD_CHILD_TIMER(_scan_profile, "PredFilter", "SegmentRead");
_pred_filter_counter = ADD_CHILD_COUNTER(_scan_profile, "PredFilterRows", TUnit::UNIT, "SegmentRead");
_del_vec_filter_counter = ADD_CHILD_COUNTER(_scan_profile, "DelVecFilterRows", TUnit::UNIT, "SegmentRead");
_chunk_copy_timer = ADD_CHILD_TIMER(_scan_profile, "ChunkCopy", "SegmentRead");
_decompress_timer = ADD_CHILD_TIMER(_scan_profile, "DecompressT", "SegmentRead");
_index_load_timer = ADD_CHILD_TIMER(_scan_profile, "IndexLoad", "SegmentRead");
_rowsets_read_count = ADD_CHILD_COUNTER(_scan_profile, "RowsetsReadCount", TUnit::UNIT, "SegmentRead");
_segments_read_count = ADD_CHILD_COUNTER(_scan_profile, "SegmentsReadCount", TUnit::UNIT, "SegmentRead");
_block_load_timer = ADD_TIMER(_runtime_profile, "SegmentRead");
_block_fetch_timer = ADD_CHILD_TIMER(_runtime_profile, "BlockFetch", "SegmentRead");
_block_load_counter = ADD_CHILD_COUNTER(_runtime_profile, "BlockFetchCount", TUnit::UNIT, "SegmentRead");
_block_seek_timer = ADD_CHILD_TIMER(_runtime_profile, "BlockSeek", "SegmentRead");
_block_seek_counter = ADD_CHILD_COUNTER(_runtime_profile, "BlockSeekCount", TUnit::UNIT, "SegmentRead");
_pred_filter_timer = ADD_CHILD_TIMER(_runtime_profile, "PredFilter", "SegmentRead");
_pred_filter_counter = ADD_CHILD_COUNTER(_runtime_profile, "PredFilterRows", TUnit::UNIT, "SegmentRead");
_del_vec_filter_counter = ADD_CHILD_COUNTER(_runtime_profile, "DelVecFilterRows", TUnit::UNIT, "SegmentRead");
_chunk_copy_timer = ADD_CHILD_TIMER(_runtime_profile, "ChunkCopy", "SegmentRead");
_decompress_timer = ADD_CHILD_TIMER(_runtime_profile, "DecompressT", "SegmentRead");
_index_load_timer = ADD_CHILD_TIMER(_runtime_profile, "IndexLoad", "SegmentRead");
_rowsets_read_count = ADD_CHILD_COUNTER(_runtime_profile, "RowsetsReadCount", TUnit::UNIT, "SegmentRead");
_segments_read_count = ADD_CHILD_COUNTER(_runtime_profile, "SegmentsReadCount", TUnit::UNIT, "SegmentRead");
_total_columns_data_page_count =
ADD_CHILD_COUNTER(_scan_profile, "TotalColumnsDataPageCount", TUnit::UNIT, "SegmentRead");
ADD_CHILD_COUNTER(_runtime_profile, "TotalColumnsDataPageCount", TUnit::UNIT, "SegmentRead");
// IOTime
_io_timer = ADD_TIMER(_scan_profile, "IOTime");
_io_timer = ADD_TIMER(_runtime_profile, "IOTime");
}
Status OlapChunkSource::_build_scan_range(RuntimeState* state) {
@ -163,7 +169,7 @@ Status OlapChunkSource::_init_reader_params(const std::vector<OlapScanRange*>& k
bool skip_aggregation = thrift_olap_scan_node.is_preaggregation;
_params.reader_type = READER_QUERY;
_params.skip_aggregation = skip_aggregation;
_params.profile = _scan_profile;
_params.profile = _runtime_profile;
_params.runtime_state = _runtime_state;
_params.use_page_cache = !config::disable_storage_page_cache;
// Improve for select * from table limit x, x is small
@ -288,7 +294,7 @@ Status OlapChunkSource::_init_olap_reader(RuntimeState* runtime_state) {
}
if (!_not_push_down_conjuncts.empty() || !_not_push_down_predicates.empty()) {
_expr_filter_timer = ADD_TIMER(_scan_profile, "ExprFilterTime");
_expr_filter_timer = ADD_TIMER(_runtime_profile, "ExprFilterTime");
}
DCHECK(_params.global_dictmaps != nullptr);
@ -328,7 +334,7 @@ Status OlapChunkSource::buffer_next_batch_chunks_blocking(size_t batch_size, boo
for (size_t i = 0; i < batch_size && !can_finish; ++i) {
ChunkUniquePtr chunk(
ChunkHelper::new_chunk_pooled(_prj_iter->encoded_schema(), _runtime_state->chunk_size(), true));
ChunkHelper::new_chunk_pooled(_prj_iter->output_schema(), _runtime_state->chunk_size(), true));
_status = _read_chunk_from_storage(_runtime_state, chunk.get());
if (!_status.ok()) {
// end of file is normal case, need process chunk
@ -356,7 +362,7 @@ Status OlapChunkSource::buffer_next_batch_chunks_blocking_for_workgroup(size_t b
SCOPED_RAW_TIMER(&time_spent);
ChunkUniquePtr chunk(
ChunkHelper::new_chunk_pooled(_prj_iter->encoded_schema(), _runtime_state->chunk_size(), true));
ChunkHelper::new_chunk_pooled(_prj_iter->output_schema(), _runtime_state->chunk_size(), true));
_status = _read_chunk_from_storage(_runtime_state, chunk.get());
if (!_status.ok()) {
// end of file is normal case, need process chunk
@ -417,6 +423,7 @@ Status OlapChunkSource::_read_chunk_from_storage(RuntimeState* state, vectorized
if (Status status = _prj_iter->get_next(chunk); !status.ok()) {
return status;
}
TRY_CATCH_ALLOC_SCOPE_START()
for (auto slot : _query_slots) {
size_t column_index = chunk->schema()->get_field_index_by_name(slot->col_name());
@ -436,6 +443,8 @@ Status OlapChunkSource::_read_chunk_from_storage(RuntimeState* state, vectorized
ExecNode::eval_conjuncts(_not_push_down_conjuncts, chunk);
DCHECK_CHUNK(chunk);
}
TRY_CATCH_ALLOC_SCOPE_END()
} while (chunk->num_rows() == 0);
_update_realtime_counter(chunk);
// Improve for select * from table limit x, x is small
@ -464,6 +473,14 @@ void OlapChunkSource::_update_realtime_counter(vectorized::Chunk* chunk) {
_num_rows_read += chunk->num_rows();
}
int64_t OlapChunkSource::last_spent_cpu_time_ns() {
int64_t time_ns = _last_spent_cpu_time_ns;
_last_spent_cpu_time_ns += _reader->stats().decompress_ns;
_last_spent_cpu_time_ns += _reader->stats().vec_cond_ns;
_last_spent_cpu_time_ns += _reader->stats().del_filter_ns;
return _last_spent_cpu_time_ns - time_ns;
}
void OlapChunkSource::_update_counter() {
COUNTER_UPDATE(_create_seg_iter_timer, _reader->stats().create_segment_iter_ns);
COUNTER_UPDATE(_rows_read_counter, _num_rows_read);
@ -512,16 +529,16 @@ void OlapChunkSource::_update_counter() {
StarRocksMetrics::instance()->query_scan_rows.increment(_raw_rows_read);
if (_reader->stats().decode_dict_ns > 0) {
RuntimeProfile::Counter* c = ADD_TIMER(_scan_profile, "DictDecode");
RuntimeProfile::Counter* c = ADD_TIMER(_runtime_profile, "DictDecode");
COUNTER_UPDATE(c, _reader->stats().decode_dict_ns);
}
if (_reader->stats().late_materialize_ns > 0) {
RuntimeProfile::Counter* c = ADD_TIMER(_scan_profile, "LateMaterialize");
RuntimeProfile::Counter* c = ADD_TIMER(_runtime_profile, "LateMaterialize");
COUNTER_UPDATE(c, _reader->stats().late_materialize_ns);
}
if (_reader->stats().del_filter_ns > 0) {
RuntimeProfile::Counter* c1 = ADD_TIMER(_scan_profile, "DeleteFilter");
RuntimeProfile::Counter* c2 = ADD_COUNTER(_scan_profile, "DeleteFilterRows", TUnit::UNIT);
RuntimeProfile::Counter* c1 = ADD_TIMER(_runtime_profile, "DeleteFilter");
RuntimeProfile::Counter* c2 = ADD_COUNTER(_runtime_profile, "DeleteFilterRows", TUnit::UNIT);
COUNTER_UPDATE(c1, _reader->stats().del_filter_ns);
COUNTER_UPDATE(c2, _reader->stats().rows_del_filtered);
}

View File

@ -30,7 +30,8 @@ namespace pipeline {
class ScanOperator;
class OlapChunkSource final : public ChunkSource {
public:
OlapChunkSource(MorselPtr&& morsel, ScanOperator* op, vectorized::OlapScanNode* scan_node);
OlapChunkSource(RuntimeProfile* runtime_profile, MorselPtr&& morsel, ScanOperator* op,
vectorized::OlapScanNode* scan_node);
~OlapChunkSource() override = default;
@ -49,6 +50,7 @@ public:
Status buffer_next_batch_chunks_blocking(size_t chunk_size, bool& can_finish) override;
Status buffer_next_batch_chunks_blocking_for_workgroup(size_t chunk_size, bool& can_finish, size_t* num_read_chunks,
int worker_id, workgroup::WorkGroupPtr running_wg) override;
int64_t last_spent_cpu_time_ns() override;
private:
// Yield scan io task when maximum time in nano-seconds has spent in current execution round.
@ -115,11 +117,11 @@ private:
int64_t _raw_rows_read = 0;
int64_t _compressed_bytes_read = 0;
RuntimeProfile* _runtime_profile;
RuntimeProfile::Counter* _bytes_read_counter = nullptr;
RuntimeProfile::Counter* _rows_read_counter = nullptr;
RuntimeProfile* _scan_profile = nullptr;
int64_t _last_spent_cpu_time_ns = 0;
RuntimeProfile::Counter* _expr_filter_timer = nullptr;
RuntimeProfile::Counter* _scan_timer = nullptr;
RuntimeProfile::Counter* _create_seg_iter_timer = nullptr;

View File

@ -80,9 +80,10 @@ Status OlapScanOperator::_capture_tablet_rowsets() {
return Status::OK();
}
ChunkSourcePtr OlapScanOperator::create_chunk_source(MorselPtr morsel) {
ChunkSourcePtr OlapScanOperator::create_chunk_source(MorselPtr morsel, int32_t chunk_source_index) {
vectorized::OlapScanNode* olap_scan_node = down_cast<vectorized::OlapScanNode*>(_scan_node);
return std::make_shared<OlapChunkSource>(std::move(morsel), this, olap_scan_node);
return std::make_shared<OlapChunkSource>(_chunk_source_profiles[chunk_source_index].get(), std::move(morsel), this,
olap_scan_node);
}
} // namespace starrocks::pipeline

View File

@ -34,7 +34,7 @@ public:
Status do_prepare(RuntimeState* state) override;
void do_close(RuntimeState* state) override;
ChunkSourcePtr create_chunk_source(MorselPtr morsel) override;
ChunkSourcePtr create_chunk_source(MorselPtr morsel, int32_t chunk_source_index) override;
private:
Status _capture_tablet_rowsets();

View File

@ -46,7 +46,7 @@ public:
// data inside is processed.
// It's one of the stages of the operator life cycleprepare -> finishing -> finished -> [cancelled] -> closed)
// This method will be exactly invoked once in the whole life cycle
virtual void set_finishing(RuntimeState* state) {}
virtual Status set_finishing(RuntimeState* state) { return Status::OK(); }
// set_finished is used to shutdown both input and output stream of a operator and after its invocation
// buffered data inside the operator is cleared.
@ -58,13 +58,13 @@ public:
// set_finishing function.
// It's one of the stages of the operator life cycleprepare -> finishing -> finished -> [cancelled] -> closed)
// This method will be exactly invoked once in the whole life cycle
virtual void set_finished(RuntimeState* state) {}
virtual Status set_finished(RuntimeState* state) { return Status::OK(); }
// It's one of the stages of the operator life cycleprepare -> finishing -> finished -> [cancelled] -> closed)
// - When the fragment exits abnormally, the stage operator will become to CANCELLED between FINISHED and CLOSE.
// - When the fragment exits normally, there isn't CANCELLED stage for the drivers.
// Sometimes, the operator need to realize it is cancelled to stop earlier than normal, such as ExchangeSink.
virtual void set_cancelled(RuntimeState* state) {}
virtual Status set_cancelled(RuntimeState* state) { return Status::OK(); }
// when local runtime filters are ready, the operator should bound its corresponding runtime in-filters.
virtual void set_precondition_ready(RuntimeState* state);

View File

@ -34,12 +34,6 @@ Status PipelineDriver::prepare(RuntimeState* runtime_state) {
_first_input_empty_timer = ADD_CHILD_TIMER(_runtime_profile, "FirstInputEmptyTime", "InputEmptyTime");
_followup_input_empty_timer = ADD_CHILD_TIMER(_runtime_profile, "FollowupInputEmptyTime", "InputEmptyTime");
_output_full_timer = ADD_CHILD_TIMER(_runtime_profile, "OutputFullTime", "PendingTime");
_local_rf_waiting_set_counter = ADD_COUNTER(_runtime_profile, "LocalRfWaitingSet", TUnit::UNIT);
_schedule_counter = ADD_COUNTER(_runtime_profile, "ScheduleCounter", TUnit::UNIT);
_schedule_effective_counter = ADD_COUNTER(_runtime_profile, "ScheduleEffectiveCounter", TUnit::UNIT);
_schedule_rows_per_chunk = ADD_COUNTER(_runtime_profile, "ScheduleAccumulatedRowsPerChunk", TUnit::UNIT);
_schedule_accumulated_chunks_moved = ADD_COUNTER(_runtime_profile, "ScheduleAccumulatedChunkMoved", TUnit::UNIT);
DCHECK(_state == DriverState::NOT_READY);
// fill OperatorWithDependency instances into _dependencies from _operators.
@ -64,7 +58,9 @@ Status PipelineDriver::prepare(RuntimeState* runtime_state) {
std::max(_global_rf_wait_timeout_ns, global_rf_collector->wait_timeout_ms() * 1000L * 1000L);
}
}
_local_rf_waiting_set_counter->set((int64_t)all_local_rf_set.size());
if (!all_local_rf_set.empty()) {
_runtime_profile->add_info_string("LocalRfWaitingSet", strings::Substitute("$0", all_local_rf_set.size()));
}
_local_rf_holders = fragment_ctx()->runtime_filter_hub()->gather_holders(all_local_rf_set);
source_operator()->add_morsel_queue(_morsel_queue.get());
@ -118,9 +114,9 @@ StatusOr<DriverState> PipelineDriver::process(RuntimeState* runtime_state, int w
if (curr_op->is_finished()) {
if (i == 0) {
// For source operators
_mark_operator_finishing(curr_op, runtime_state);
RETURN_IF_ERROR(_mark_operator_finishing(curr_op, runtime_state));
}
_mark_operator_finishing(next_op, runtime_state);
RETURN_IF_ERROR(_mark_operator_finishing(next_op, runtime_state));
new_first_unfinished = i + 1;
continue;
}
@ -180,9 +176,9 @@ StatusOr<DriverState> PipelineDriver::process(RuntimeState* runtime_state, int w
if (curr_op->is_finished()) {
if (i == 0) {
// For source operators
_mark_operator_finishing(curr_op, runtime_state);
RETURN_IF_ERROR(_mark_operator_finishing(curr_op, runtime_state));
}
_mark_operator_finishing(next_op, runtime_state);
RETURN_IF_ERROR(_mark_operator_finishing(next_op, runtime_state));
new_first_unfinished = i + 1;
continue;
}
@ -202,7 +198,7 @@ StatusOr<DriverState> PipelineDriver::process(RuntimeState* runtime_state, int w
}
// close finished operators and update _first_unfinished index
for (auto i = _first_unfinished; i < new_first_unfinished; ++i) {
_mark_operator_finished(_operators[i], runtime_state);
RETURN_IF_ERROR(_mark_operator_finished(_operators[i], runtime_state));
}
_first_unfinished = new_first_unfinished;
@ -304,10 +300,6 @@ void PipelineDriver::finalize(RuntimeState* runtime_state, DriverState state) {
COUNTER_UPDATE(_total_timer, _total_timer_sw->elapsed_time());
COUNTER_UPDATE(_schedule_timer, _total_timer->value() - _active_timer->value() - _pending_timer->value());
COUNTER_UPDATE(_schedule_counter, driver_acct().get_schedule_times());
COUNTER_UPDATE(_schedule_effective_counter, driver_acct().get_schedule_effective_times());
COUNTER_UPDATE(_schedule_rows_per_chunk, driver_acct().get_rows_per_chunk());
COUNTER_UPDATE(_schedule_accumulated_chunks_moved, driver_acct().get_accumulated_chunks_moved());
_update_overhead_timer();
// last root driver cancel the all drivers' execution and notify FE the
@ -328,7 +320,11 @@ void PipelineDriver::finalize(RuntimeState* runtime_state, DriverState state) {
_fragment_ctx->destroy_pass_through_chunk_buffer();
auto status = _fragment_ctx->final_status();
auto fragment_id = _fragment_ctx->fragment_instance_id();
_query_ctx->count_down_fragments();
if (_query_ctx->count_down_fragments()) {
auto query_id = _query_ctx->query_id();
DCHECK(!this->is_still_pending_finish());
QueryContextManager::instance()->remove(query_id);
}
}
}
@ -388,72 +384,73 @@ bool PipelineDriver::_check_fragment_is_canceled(RuntimeState* runtime_state) {
return false;
}
void PipelineDriver::_mark_operator_finishing(OperatorPtr& op, RuntimeState* state) {
Status PipelineDriver::_mark_operator_finishing(OperatorPtr& op, RuntimeState* state) {
auto& op_state = _operator_stages[op->get_id()];
if (op_state >= OperatorStage::FINISHING) {
return;
return Status::OK();
}
VLOG_ROW << strings::Substitute("[Driver] finishing operator [driver=$0] [operator=$1]", to_readable_string(),
op->get_name());
{
SCOPED_TIMER(op->_finishing_timer);
op->set_finishing(state);
op_state = OperatorStage::FINISHING;
return op->set_finishing(state);
}
op_state = OperatorStage::FINISHING;
}
void PipelineDriver::_mark_operator_finished(OperatorPtr& op, RuntimeState* state) {
_mark_operator_finishing(op, state);
Status PipelineDriver::_mark_operator_finished(OperatorPtr& op, RuntimeState* state) {
RETURN_IF_ERROR(_mark_operator_finishing(op, state));
auto& op_state = _operator_stages[op->get_id()];
if (op_state >= OperatorStage::FINISHED) {
return;
return Status::OK();
}
VLOG_ROW << strings::Substitute("[Driver] finished operator [driver=$0] [operator=$1]", to_readable_string(),
op->get_name());
{
SCOPED_TIMER(op->_finished_timer);
op->set_finished(state);
op_state = OperatorStage::FINISHED;
return op->set_finished(state);
}
op_state = OperatorStage::FINISHED;
}
void PipelineDriver::_mark_operator_cancelled(OperatorPtr& op, RuntimeState* state) {
_mark_operator_finished(op, state);
Status PipelineDriver::_mark_operator_cancelled(OperatorPtr& op, RuntimeState* state) {
RETURN_IF_ERROR(_mark_operator_finished(op, state));
auto& op_state = _operator_stages[op->get_id()];
if (op_state >= OperatorStage::CANCELLED) {
return;
return Status::OK();
}
VLOG_ROW << strings::Substitute("[Driver] cancelled operator [driver=$0] [operator=$1]", to_readable_string(),
op->get_name());
op->set_cancelled(state);
op_state = OperatorStage::CANCELLED;
return op->set_cancelled(state);
}
void PipelineDriver::_mark_operator_closed(OperatorPtr& op, RuntimeState* state) {
Status PipelineDriver::_mark_operator_closed(OperatorPtr& op, RuntimeState* state) {
if (_fragment_ctx->is_canceled()) {
_mark_operator_cancelled(op, state);
RETURN_IF_ERROR(_mark_operator_cancelled(op, state));
} else {
_mark_operator_finished(op, state);
RETURN_IF_ERROR(_mark_operator_finished(op, state));
}
auto& op_state = _operator_stages[op->get_id()];
if (op_state >= OperatorStage::CLOSED) {
return;
return Status::OK();
}
VLOG_ROW << strings::Substitute("[Driver] close operator [driver=$0] [operator=$1]", to_readable_string(),
op->get_name());
{
SCOPED_TIMER(op->_close_timer);
op_state = OperatorStage::CLOSED;
op->close(state);
}
COUNTER_UPDATE(op->_total_timer, op->_pull_timer->value() + op->_push_timer->value() +
op->_finishing_timer->value() + op->_finished_timer->value() +
op->_close_timer->value());
op_state = OperatorStage::CLOSED;
return Status::OK();
}
} // namespace starrocks::pipeline

View File

@ -219,10 +219,10 @@ public:
Operators& operators() { return _operators; }
SourceOperator* source_operator() { return down_cast<SourceOperator*>(_operators.front().get()); }
RuntimeProfile* runtime_profile() { return _runtime_profile.get(); }
// drivers that waits for runtime filters' readiness must be marked PRECONDITION_NOT_READY and put into
// PipelineDriverPoller.
void mark_precondition_not_ready();
// drivers in PRECONDITION_BLOCK state must be marked READY after its dependent runtime-filters or hash tables
// are finished.
void mark_precondition_ready(RuntimeState* runtime_state);
@ -355,10 +355,10 @@ private:
// check whether fragment is cancelled. It is used before pull_chunk and push_chunk.
bool _check_fragment_is_canceled(RuntimeState* runtime_state);
void _mark_operator_finishing(OperatorPtr& op, RuntimeState* runtime_state);
void _mark_operator_finished(OperatorPtr& op, RuntimeState* runtime_state);
void _mark_operator_cancelled(OperatorPtr& op, RuntimeState* runtime_state);
void _mark_operator_closed(OperatorPtr& op, RuntimeState* runtime_state);
Status _mark_operator_finishing(OperatorPtr& op, RuntimeState* runtime_state);
Status _mark_operator_finished(OperatorPtr& op, RuntimeState* runtime_state);
Status _mark_operator_cancelled(OperatorPtr& op, RuntimeState* runtime_state);
Status _mark_operator_closed(OperatorPtr& op, RuntimeState* runtime_state);
void _close_operators(RuntimeState* runtime_state);
// Update metrics when the driver yields.
@ -416,12 +416,6 @@ private:
RuntimeProfile::Counter* _first_input_empty_timer = nullptr;
RuntimeProfile::Counter* _followup_input_empty_timer = nullptr;
RuntimeProfile::Counter* _output_full_timer = nullptr;
RuntimeProfile::Counter* _local_rf_waiting_set_counter = nullptr;
RuntimeProfile::Counter* _schedule_counter = nullptr;
RuntimeProfile::Counter* _schedule_effective_counter = nullptr;
RuntimeProfile::Counter* _schedule_rows_per_chunk = nullptr;
RuntimeProfile::Counter* _schedule_accumulated_chunks_moved = nullptr;
MonotonicStopWatch* _total_timer_sw = nullptr;
MonotonicStopWatch* _pending_timer_sw = nullptr;
@ -431,4 +425,4 @@ private:
};
} // namespace pipeline
} // namespace starrocks
} // namespace starrocks

View File

@ -25,7 +25,7 @@ void GlobalDriverExecutor::initialize(int num_threads) {
_blocked_driver_poller->start();
_num_threads_setter.set_actual_num(num_threads);
for (auto i = 0; i < num_threads; ++i) {
_thread_pool->submit_func([this]() { this->worker_thread(); });
_thread_pool->submit_func([this]() { this->_worker_thread(); });
}
}
@ -35,21 +35,16 @@ void GlobalDriverExecutor::change_num_threads(int32_t num_threads) {
return;
}
for (int i = old_num_threads; i < num_threads; ++i) {
_thread_pool->submit_func([this]() { this->worker_thread(); });
_thread_pool->submit_func([this]() { this->_worker_thread(); });
}
}
void GlobalDriverExecutor::finalize_driver(DriverRawPtr driver, RuntimeState* runtime_state, DriverState state) {
void GlobalDriverExecutor::_finalize_driver(DriverRawPtr driver, RuntimeState* runtime_state, DriverState state) {
DCHECK(driver);
driver->finalize(runtime_state, state);
if (driver->query_ctx()->is_finished()) {
auto query_id = driver->query_ctx()->query_id();
DCHECK(!driver->is_still_pending_finish());
QueryContextManager::instance()->remove(query_id);
}
}
void GlobalDriverExecutor::worker_thread() {
void GlobalDriverExecutor::_worker_thread() {
const int worker_id = _next_id++;
while (true) {
if (_num_threads_setter.should_shrink()) {
@ -78,13 +73,13 @@ void GlobalDriverExecutor::worker_thread() {
driver->set_driver_state(DriverState::PENDING_FINISH);
_blocked_driver_poller->add_blocked_driver(driver);
} else {
finalize_driver(driver, runtime_state, DriverState::CANCELED);
_finalize_driver(driver, runtime_state, DriverState::CANCELED);
}
continue;
}
// a blocked driver is canceled because of fragment cancellation or query expiration.
if (driver->is_finished()) {
finalize_driver(driver, runtime_state, driver->driver_state());
_finalize_driver(driver, runtime_state, driver->driver_state());
continue;
}
@ -103,7 +98,7 @@ void GlobalDriverExecutor::worker_thread() {
driver->set_driver_state(DriverState::PENDING_FINISH);
_blocked_driver_poller->add_blocked_driver(driver);
} else {
finalize_driver(driver, runtime_state, DriverState::INTERNAL_ERROR);
_finalize_driver(driver, runtime_state, DriverState::INTERNAL_ERROR);
}
continue;
}
@ -117,7 +112,7 @@ void GlobalDriverExecutor::worker_thread() {
case FINISH:
case CANCELED:
case INTERNAL_ERROR: {
finalize_driver(driver, runtime_state, driver_state);
_finalize_driver(driver, runtime_state, driver_state);
break;
}
case INPUT_EMPTY:
@ -153,7 +148,7 @@ void GlobalDriverExecutor::submit(DriverRawPtr driver) {
}
void GlobalDriverExecutor::report_exec_state(FragmentContext* fragment_ctx, const Status& status, bool done) {
update_profile_by_level(fragment_ctx, done);
_update_profile_by_level(fragment_ctx, done);
auto params = ExecStateReporter::create_report_exec_status_params(fragment_ctx, status, done);
auto fe_addr = fragment_ctx->fe_addr();
auto exec_env = fragment_ctx->runtime_state()->exec_env();
@ -177,7 +172,7 @@ void GlobalDriverExecutor::report_exec_state(FragmentContext* fragment_ctx, cons
this->_exec_state_reporter->submit(std::move(report_task));
}
void GlobalDriverExecutor::update_profile_by_level(FragmentContext* fragment_ctx, bool done) {
void GlobalDriverExecutor::_update_profile_by_level(FragmentContext* fragment_ctx, bool done) {
if (!done) {
return;
}
@ -204,7 +199,7 @@ void GlobalDriverExecutor::update_profile_by_level(FragmentContext* fragment_ctx
continue;
}
remove_non_core_metrics(fragment_ctx, driver_profiles);
_remove_non_core_metrics(fragment_ctx, driver_profiles);
RuntimeProfile::merge_isomorphic_profiles(driver_profiles);
// all the isomorphic profiles will merged into the first profile
@ -212,12 +207,15 @@ void GlobalDriverExecutor::update_profile_by_level(FragmentContext* fragment_ctx
// use the name of pipeline' profile as pipeline driver's
merged_driver_profile->set_name(pipeline_profile->name());
merged_driver_profiles.push_back(merged_driver_profile);
// add all the info string and counters of the pipeline's profile
// to the pipeline driver's profile
merged_driver_profile->copy_all_counters_from(pipeline_profile);
merged_driver_profile->copy_all_info_strings_from(pipeline_profile);
merged_driver_profile->copy_all_counters_from(pipeline_profile);
_simplify_common_metrics(merged_driver_profile);
merged_driver_profiles.push_back(merged_driver_profile);
}
// remove pipeline's profile from the hierarchy
@ -228,8 +226,8 @@ void GlobalDriverExecutor::update_profile_by_level(FragmentContext* fragment_ctx
}
}
void GlobalDriverExecutor::remove_non_core_metrics(FragmentContext* fragment_ctx,
std::vector<RuntimeProfile*>& driver_profiles) {
void GlobalDriverExecutor::_remove_non_core_metrics(FragmentContext* fragment_ctx,
std::vector<RuntimeProfile*>& driver_profiles) {
if (fragment_ctx->profile_level() > TPipelineProfileLevel::CORE_METRICS) {
return;
}
@ -242,17 +240,34 @@ void GlobalDriverExecutor::remove_non_core_metrics(FragmentContext* fragment_ctx
for (auto* operator_profile : operator_profiles) {
RuntimeProfile* common_metrics = operator_profile->get_child("CommonMetrics");
DCHECK(common_metrics != nullptr);
common_metrics->remove_counters(std::set<std::string>{"OperatorTotalTime"});
operator_profile->remove_childs();
if (common_metrics != nullptr) {
common_metrics->remove_counters(std::set<std::string>{"OperatorTotalTime"});
}
common_metrics->reset_parent();
operator_profile->add_child(common_metrics, true, nullptr);
RuntimeProfile* unique_metrics = operator_profile->get_child("UniqueMetrics");
DCHECK(unique_metrics != nullptr);
unique_metrics->remove_counters(std::set<std::string>{"ScanTime", "WaitTime"});
}
}
}
void GlobalDriverExecutor::_simplify_common_metrics(RuntimeProfile* driver_profile) {
std::vector<RuntimeProfile*> operator_profiles;
driver_profile->get_children(&operator_profiles);
for (auto* operator_profile : operator_profiles) {
RuntimeProfile* common_metrics = operator_profile->get_child("CommonMetrics");
DCHECK(common_metrics != nullptr);
// Remove runtime filter related counters if it's value is 0
static std::string counter_names[] = {
"RuntimeInFilterNum", "RuntimeBloomFilterNum", "JoinRuntimeFilterInputRows",
"JoinRuntimeFilterOutputRows", "JoinRuntimeFilterEvaluate", "JoinRuntimeFilterTime",
"ConjunctsInputRows", "ConjunctsOutputRows", "ConjunctsEvaluate"};
for (auto& name : counter_names) {
auto* counter = common_metrics->get_counter(name);
if (counter != nullptr && counter->value() == 0) {
common_metrics->remove_counter(name);
}
}
}
}
} // namespace starrocks::pipeline

View File

@ -48,10 +48,11 @@ public:
void report_exec_state(FragmentContext* fragment_ctx, const Status& status, bool done) override;
private:
void worker_thread();
void finalize_driver(DriverRawPtr driver, RuntimeState* runtime_state, DriverState state);
void update_profile_by_level(FragmentContext* fragment_ctx, bool done);
void remove_non_core_metrics(FragmentContext* fragment_ctx, std::vector<RuntimeProfile*>& driver_profiles);
void _worker_thread();
void _finalize_driver(DriverRawPtr driver, RuntimeState* runtime_state, DriverState state);
void _update_profile_by_level(FragmentContext* fragment_ctx, bool done);
void _remove_non_core_metrics(FragmentContext* fragment_ctx, std::vector<RuntimeProfile*>& driver_profiles);
void _simplify_common_metrics(RuntimeProfile* driver_profile);
private:
LimitSetter _num_threads_setter;

View File

@ -284,6 +284,7 @@ void DriverQueueWithWorkGroup::_put_back(const DriverRawPtr driver) {
int64_t new_vruntime_ns = std::min(min_wg->vruntime_ns() - _ideal_runtime_ns(wg) / 2,
min_wg->real_runtime_ns() / int64_t(wg->cpu_limit()));
wg->set_vruntime_ns(std::max(wg->vruntime_ns(), new_vruntime_ns));
wg->update_last_real_runtime_ns(wg->real_runtime_ns());
int64_t diff_real_runtime_ns = wg->real_runtime_ns() - origin_real_runtime_ns;
workgroup::WorkGroupManager::instance()->increment_cpu_runtime_ns(diff_real_runtime_ns);

View File

@ -7,6 +7,7 @@
#include "column/nullable_column.h"
#include "exprs/expr.h"
#include "exprs/vectorized/column_ref.h"
#include "runtime/current_thread.h"
#include "runtime/runtime_state.h"
namespace starrocks::pipeline {
@ -23,8 +24,10 @@ StatusOr<vectorized::ChunkPtr> ProjectOperator::pull_chunk(RuntimeState* state)
}
Status ProjectOperator::push_chunk(RuntimeState* state, const vectorized::ChunkPtr& chunk) {
TRY_CATCH_ALLOC_SCOPE_START()
for (size_t i = 0; i < _common_sub_column_ids.size(); ++i) {
chunk->append_column(_common_sub_expr_ctxs[i]->evaluate(chunk.get()), _common_sub_column_ids[i]);
RETURN_IF_HAS_ERROR(_common_sub_expr_ctxs);
}
using namespace vectorized;
@ -52,6 +55,7 @@ Status ProjectOperator::push_chunk(RuntimeState* state, const vectorized::ChunkP
NullableColumn::create(result_columns[i], NullColumn::create(result_columns[i]->size(), 0));
}
}
RETURN_IF_HAS_ERROR(_expr_ctxs);
}
_cur_chunk = std::make_shared<vectorized::Chunk>();
@ -60,6 +64,7 @@ Status ProjectOperator::push_chunk(RuntimeState* state, const vectorized::ChunkP
}
eval_runtime_bloom_filters(_cur_chunk.get());
DCHECK_CHUNK(_cur_chunk);
TRY_CATCH_ALLOC_SCOPE_END()
return Status::OK();
}

View File

@ -33,7 +33,10 @@ public:
bool is_finished() const override { return _is_finished && _cur_chunk == nullptr; }
void set_finishing(RuntimeState* state) override { _is_finished = true; }
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -30,6 +30,7 @@ void QueryContext::cancel(const Status& status) {
static constexpr size_t QUERY_CONTEXT_MAP_SLOT_NUM = 64;
static constexpr size_t QUERY_CONTEXT_MAP_SLOT_NUM_MASK = (1 << 6) - 1;
QueryContextManager::QueryContextManager()
: _mutexes(QUERY_CONTEXT_MAP_SLOT_NUM),
_context_maps(QUERY_CONTEXT_MAP_SLOT_NUM),
@ -132,7 +133,7 @@ void QueryContextManager::remove(const TUniqueId& query_id) {
// the query context is really dead, so just cleanup
if (it->second->is_dead()) {
context_map.erase(it);
} else {
} else if (it->second->is_finished()) {
// although all of active fragments of the query context terminates, but some fragments maybe comes too late
// in the future, so extend the lifetime of query context and wait for some time till fragments on wire have
// vanished
@ -143,4 +144,14 @@ void QueryContextManager::remove(const TUniqueId& query_id) {
}
}
void QueryContextManager::clear() {
std::vector<std::unique_lock<std::shared_mutex>> locks;
locks.reserve(_mutexes.size());
for (int i = 0; i < _mutexes.size(); ++i) {
locks.emplace_back(_mutexes[i]);
}
_second_chance_maps.clear();
_context_maps.clear();
}
} // namespace starrocks::pipeline

View File

@ -45,7 +45,7 @@ public:
// now time point pass by deadline point.
bool is_expired() {
auto now = duration_cast<milliseconds>(steady_clock::now().time_since_epoch()).count();
return now > _deadline;
return is_finished() && now > _deadline;
}
bool is_dead() { return _num_active_fragments == 0 && _num_fragments == _total_fragments; }
@ -95,6 +95,8 @@ public:
QueryContext* get_or_register(const TUniqueId& query_id);
QueryContextPtr get(const TUniqueId& query_id);
void remove(const TUniqueId& query_id);
// used for graceful exit
void clear();
private:
std::vector<std::shared_mutex> _mutexes;

View File

@ -43,7 +43,10 @@ public:
bool is_finished() const override { return _is_finished && !_fetch_data_result; }
void set_finishing(RuntimeState* state) override { _is_finished = true; }
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -21,31 +21,31 @@ using RuntimeBloomFilterProbeDescriptorPtr = RuntimeBloomFilterProbeDescriptor*;
using RuntimeBloomFilterRunningContext = starrocks::vectorized::JoinRuntimeFilter::RunningContext;
using RuntimeInFilterPtr = RuntimeInFilter*;
using RuntimeBloomFilterPtr = RuntimeBloomFilter*;
using RuntimeInFilters = std::list<RuntimeInFilterPtr>;
using RuntimeBloomFilters = std::list<RuntimeBloomFilterPtr>;
using RuntimeInFilters = std::vector<RuntimeInFilterPtr>;
using RuntimeInFilterList = std::list<RuntimeInFilterPtr>;
using RuntimeBloomFilters = std::vector<RuntimeBloomFilterPtr>;
using RuntimeBloomFilterList = std::list<RuntimeBloomFilterPtr>;
struct RuntimeFilterCollector;
using RuntimeFilterCollectorPtr = std::unique_ptr<RuntimeFilterCollector>;
using RuntimeFilterProbeCollector = starrocks::vectorized::RuntimeFilterProbeCollector;
using Predicate = starrocks::Predicate;
struct RuntimeBloomFilterBuildParam;
using RuntimeBloomFilterBuildParams = std::list<RuntimeBloomFilterBuildParam>;
using OptRuntimeBloomFilterBuildParams = std::vector<std::optional<RuntimeBloomFilterBuildParam>>;
// Parameters used to build runtime bloom-filters.
struct RuntimeBloomFilterBuildParam {
RuntimeBloomFilterBuildParam(bool eq_null, const ColumnPtr& column, size_t ht_row_count)
: eq_null(eq_null), column(column), ht_row_count(ht_row_count) {}
RuntimeBloomFilterBuildParam(bool eq_null, const ColumnPtr& column) : eq_null(eq_null), column(column) {}
bool eq_null;
ColumnPtr column;
size_t ht_row_count;
};
// RuntimeFilterCollector contains runtime in-filters and bloom-filters, it is stored in RuntimeFilerHub
// and every HashJoinBuildOperatorFactory has its corresponding RuntimeFilterCollector.
struct RuntimeFilterCollector {
RuntimeFilterCollector(RuntimeInFilters&& in_filters, RuntimeBloomFilters&& bloom_filters)
RuntimeFilterCollector(RuntimeInFilterList&& in_filters, RuntimeBloomFilterList&& bloom_filters)
: _in_filters(std::move(in_filters)), _bloom_filters(std::move(bloom_filters)) {}
RuntimeBloomFilters& get_bloom_filters() { return _bloom_filters; }
RuntimeInFilters& get_in_filters() { return _in_filters; }
RuntimeBloomFilterList& get_bloom_filters() { return _bloom_filters; }
RuntimeInFilterList& get_in_filters() { return _in_filters; }
// In-filters are constructed by a node and may be pushed down to its descendant node.
// Different tuple id and slot id between descendant and ancestor nodes may be referenced to the same column,
@ -83,9 +83,9 @@ struct RuntimeFilterCollector {
private:
// local runtime in-filter
RuntimeInFilters _in_filters;
RuntimeInFilterList _in_filters;
// global/local runtime bloom-filter(including max-min filter)
RuntimeBloomFilters _bloom_filters;
RuntimeBloomFilterList _bloom_filters;
};
class RuntimeFilterHolder {
@ -113,7 +113,7 @@ public:
get_holder(id)->set_collector(std::move(collector));
}
RuntimeBloomFilters& get_bloom_filters(TPlanNodeId id) {
RuntimeBloomFilterList& get_bloom_filters(TPlanNodeId id) {
return get_holder(id)->get_collector()->get_bloom_filters();
}
@ -205,10 +205,9 @@ public:
// HashJoinBuildOperator call add_partial_filters to gather partial runtime filters. the last HashJoinBuildOperator
// will merge partial runtime filters into total one finally.
StatusOr<bool> add_partial_filters(
size_t idx, size_t ht_row_count, std::list<ExprContext*>&& partial_in_filters,
std::list<RuntimeBloomFilterBuildParam>&& partial_bloom_filter_build_params,
std::list<vectorized::RuntimeFilterBuildDescriptor*>&& bloom_filter_descriptors) {
StatusOr<bool> add_partial_filters(size_t idx, size_t ht_row_count, RuntimeInFilters&& partial_in_filters,
OptRuntimeBloomFilterBuildParams&& partial_bloom_filter_build_params,
RuntimeBloomFilters&& bloom_filter_descriptors) {
DCHECK(idx < _partial_bloom_filter_build_params.size());
// both _ht_row_counts, _partial_in_filters, _partial_bloom_filter_build_params are reserved beforehand,
// each HashJoinBuildOperator mutates its corresponding slot indexed by driver_sequence, so concurrent
@ -219,18 +218,22 @@ public:
// merge
if (1 == _num_active_builders--) {
_bloom_filter_descriptors = std::move(bloom_filter_descriptors);
merge_in_filters();
merge_bloom_filters();
merge_local_in_filters();
merge_local_bloom_filters();
return true;
}
return false;
}
RuntimeInFilters get_total_in_filters() { return _partial_in_filters[0]; }
RuntimeInFilterList get_total_in_filters() {
return std::list(_partial_in_filters[0].begin(), _partial_in_filters[0].end());
}
RuntimeBloomFilters get_total_bloom_filters() { return _bloom_filter_descriptors; }
RuntimeBloomFilterList get_total_bloom_filters() {
return std::list(_bloom_filter_descriptors.begin(), _bloom_filter_descriptors.end());
}
Status merge_in_filters() {
Status merge_local_in_filters() {
bool can_merge_in_filters = true;
size_t num_rows = 0;
ssize_t k = -1;
@ -263,17 +266,21 @@ public:
_partial_in_filters.resize(k + 1);
auto& total_in_filters = _partial_in_filters[0];
for (auto i = 1; i < _partial_in_filters.size(); ++i) {
auto& in_filters = _partial_in_filters[i];
auto total_in_filter_it = total_in_filters.begin();
auto in_filter_it = in_filters.begin();
while (total_in_filter_it != total_in_filters.end()) {
auto& total_in_filter = *(total_in_filter_it++);
auto& in_filter = *(in_filter_it++);
if (total_in_filter == nullptr || in_filter == nullptr) {
total_in_filter = nullptr;
continue;
}
const auto num_in_filters = total_in_filters.size();
for (auto i = 0; i < num_in_filters; ++i) {
auto& total_in_filter = total_in_filters[i];
if (total_in_filter == nullptr) {
continue;
}
auto can_merge = std::all_of(_partial_in_filters.begin() + 1, _partial_in_filters.end(),
[i](auto& in_filters) { return in_filters[i] != nullptr; });
if (!can_merge) {
total_in_filter = nullptr;
continue;
}
for (int j = 1; j < _partial_in_filters.size(); ++j) {
auto& in_filter = _partial_in_filters[j][i];
DCHECK(in_filter != nullptr);
auto* total_in_filter_pred = down_cast<Predicate*>(total_in_filter->root());
auto* in_filter_pred = down_cast<Predicate*>(in_filter->root());
RETURN_IF_ERROR(total_in_filter_pred->merge(in_filter_pred));
@ -284,7 +291,7 @@ public:
return Status::OK();
}
Status merge_bloom_filters() {
Status merge_local_bloom_filters() {
if (_partial_bloom_filter_build_params.empty()) {
return Status::OK();
}
@ -307,13 +314,46 @@ public:
desc->set_runtime_filter(filter);
}
for (auto& params : _partial_bloom_filter_build_params) {
auto desc_it = _bloom_filter_descriptors.begin();
auto param_it = params.begin();
while (param_it != params.end()) {
auto& desc = *(desc_it++);
auto& param = *(param_it++);
if (desc->runtime_filter() == nullptr || param.column == nullptr) {
const auto& num_bloom_filters = _bloom_filter_descriptors.size();
// remove empty params that generated in two cases:
// 1. the corresponding HashJoinProbeOperator is finished in short-circuit style because HashJoinBuildOperator
// above this operator has constructed an empty hash table.
// 2. the HashJoinBuildOperator is finished in advance because the fragment instance is canceled
_partial_bloom_filter_build_params.erase(
std::remove_if(_partial_bloom_filter_build_params.begin(), _partial_bloom_filter_build_params.end(),
[](auto& opt_params) { return opt_params.empty(); }),
_partial_bloom_filter_build_params.end());
// there is no non-empty params, set all runtime filter to nullptr
if (_partial_bloom_filter_build_params.empty()) {
for (auto& desc : _bloom_filter_descriptors) {
desc->set_runtime_filter(nullptr);
}
return Status::OK();
}
// all params must have the same size as num_bloom_filters
DCHECK(std::all_of(_partial_bloom_filter_build_params.begin(), _partial_bloom_filter_build_params.end(),
[&num_bloom_filters](auto& opt_params) { return opt_params.size() == num_bloom_filters; }));
for (auto i = 0; i < num_bloom_filters; ++i) {
auto& desc = _bloom_filter_descriptors[i];
if (desc->runtime_filter() == nullptr) {
continue;
}
auto can_merge =
std::all_of(_partial_bloom_filter_build_params.begin(), _partial_bloom_filter_build_params.end(),
[i](auto& opt_params) { return opt_params[i].has_value(); });
if (!can_merge) {
desc->set_runtime_filter(nullptr);
continue;
}
for (auto& opt_params : _partial_bloom_filter_build_params) {
auto& opt_param = opt_params[i];
DCHECK(opt_param.has_value());
auto& param = opt_param.value();
if (param.column == nullptr || param.column->empty()) {
continue;
}
auto status = vectorized::RuntimeFilterHelper::fill_runtime_bloom_filter(
@ -321,6 +361,7 @@ public:
vectorized::kHashJoinKeyColumnOffset, param.eq_null);
if (!status.ok()) {
desc->set_runtime_filter(nullptr);
break;
}
}
}
@ -333,7 +374,7 @@ private:
std::atomic<size_t> _num_active_builders;
std::vector<size_t> _ht_row_counts;
std::vector<RuntimeInFilters> _partial_in_filters;
std::vector<RuntimeBloomFilterBuildParams> _partial_bloom_filter_build_params;
std::vector<OptRuntimeBloomFilterBuildParams> _partial_bloom_filter_build_params;
RuntimeBloomFilters _bloom_filter_descriptors;
};

View File

@ -13,15 +13,18 @@
namespace starrocks::pipeline {
using starrocks::workgroup::WorkGroupManager;
// ========== ScanOperator ==========
ScanOperator::ScanOperator(OperatorFactory* factory, int32_t id, ScanNode* scan_node)
: SourceOperator(factory, id, "olap_scan", scan_node->id()),
_scan_node(scan_node),
_chunk_source_profiles(MAX_IO_TASKS_PER_OP),
_is_io_task_running(MAX_IO_TASKS_PER_OP),
_chunk_sources(MAX_IO_TASKS_PER_OP) {}
_chunk_sources(MAX_IO_TASKS_PER_OP) {
for (auto i = 0; i < MAX_IO_TASKS_PER_OP; i++) {
_chunk_source_profiles[i] = std::make_shared<RuntimeProfile>(strings::Substitute("ChunkSource$0", i));
}
}
Status ScanOperator::prepare(RuntimeState* state) {
RETURN_IF_ERROR(SourceOperator::prepare(state));
@ -120,8 +123,10 @@ bool ScanOperator::is_finished() const {
return true;
}
void ScanOperator::set_finishing(RuntimeState* state) {
Status ScanOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
_merge_chunk_source_profiles();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> ScanOperator::pull_chunk(RuntimeState* state) {
@ -175,12 +180,12 @@ Status ScanOperator::_trigger_next_scan(RuntimeState* state, int chunk_source_in
workgroup::ScanTask task = workgroup::ScanTask(_workgroup, [this, state, chunk_source_index](int worker_id) {
{
SCOPED_THREAD_LOCAL_MEM_TRACKER_SETTER(state->instance_mem_tracker());
size_t num_read_chunks = 0;
_chunk_sources[chunk_source_index]->buffer_next_batch_chunks_blocking_for_workgroup(
_buffer_size, _is_finished, &num_read_chunks, worker_id, _workgroup);
// TODO (by laotan332): More detailed information is needed
_workgroup->incr_period_scaned_chunk_num(num_read_chunks);
_workgroup->increment_real_runtime_ns(_chunk_sources[chunk_source_index]->last_spent_cpu_time_ns());
}
_num_running_io_tasks--;
@ -236,7 +241,7 @@ Status ScanOperator::_pickup_morsel(RuntimeState* state, int chunk_source_index)
if (maybe_morsel.has_value()) {
auto morsel = std::move(maybe_morsel.value());
DCHECK(morsel);
_chunk_sources[chunk_source_index] = create_chunk_source(std::move(morsel));
_chunk_sources[chunk_source_index] = create_chunk_source(std::move(morsel), chunk_source_index);
auto status = _chunk_sources[chunk_source_index]->prepare(state);
if (!status.ok()) {
_chunk_sources[chunk_source_index] = nullptr;
@ -249,6 +254,19 @@ Status ScanOperator::_pickup_morsel(RuntimeState* state, int chunk_source_index)
return Status::OK();
}
void ScanOperator::_merge_chunk_source_profiles() {
std::vector<RuntimeProfile*> profiles(_chunk_source_profiles.size());
for (auto i = 0; i < _chunk_source_profiles.size(); i++) {
profiles[i] = _chunk_source_profiles[i].get();
}
RuntimeProfile::merge_isomorphic_profiles(profiles);
RuntimeProfile* merged_profile = profiles[0];
_unique_metrics->copy_all_info_strings_from(merged_profile);
_unique_metrics->copy_all_counters_from(merged_profile);
}
// ========== ScanOperatorFactory ==========
ScanOperatorFactory::ScanOperatorFactory(int32_t id, ScanNode* scan_node)

View File

@ -28,7 +28,7 @@ public:
bool is_finished() const override;
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;
@ -38,21 +38,29 @@ public:
// interface for different scan node
virtual Status do_prepare(RuntimeState* state) = 0;
virtual void do_close(RuntimeState* state) = 0;
virtual ChunkSourcePtr create_chunk_source(MorselPtr morsel) = 0;
protected:
ScanNode* _scan_node = nullptr;
virtual ChunkSourcePtr create_chunk_source(MorselPtr morsel, int32_t chunk_source_index) = 0;
private:
static constexpr int MAX_IO_TASKS_PER_OP = 4;
const size_t _buffer_size = config::pipeline_io_buffer_size;
// This method is only invoked when current morsel is reached eof
// and all cached chunk of this morsel has benn read out
Status _pickup_morsel(RuntimeState* state, int chunk_source_index);
Status _trigger_next_scan(RuntimeState* state, int chunk_source_index);
Status _try_to_trigger_next_scan(RuntimeState* state);
void _merge_chunk_source_profiles();
protected:
ScanNode* _scan_node = nullptr;
// ScanOperator may do parallel scan, so each _chunk_sources[i] needs to hold
// a profile indenpendently, to be more specificly, _chunk_sources[i] will go through
// many ChunkSourcePtr in the entire life time, all these ChunkSources of _chunk_sources[i]
// should share one profile because these ChunkSources are serial in timeline.
// And all these parallel profiles will be merged to ScanOperator's profile at the end.
std::vector<std::shared_ptr<RuntimeProfile>> _chunk_source_profiles;
private:
static constexpr int MAX_IO_TASKS_PER_OP = 4;
const size_t _buffer_size = config::pipeline_io_buffer_size;
bool _is_finished = false;

View File

@ -23,7 +23,10 @@ public:
bool need_input() const override;
bool is_finished() const override { return _is_finished && !_curr_chunk && !_pre_output_chunk; }
void set_finishing(RuntimeState* state) override { _is_finished = true; }
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -39,9 +39,10 @@ public:
bool is_finished() const override { return _is_finished || _except_ctx->is_finished(); }
void set_finishing(RuntimeState* state) override {
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
_except_ctx->finish_build_ht();
return Status::OK();
}
Status prepare(RuntimeState* state) override;

View File

@ -28,7 +28,7 @@ public:
return _except_ctx->is_dependency_finished(_dependency_index) && _except_ctx->is_output_finished();
}
void set_finished(RuntimeState* state) override { _except_ctx->set_finished(); }
Status set_finished(RuntimeState* state) override { return _except_ctx->set_finished(); }
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -36,9 +36,10 @@ public:
return _except_ctx->is_dependency_finished(_dependency_index) && (_is_finished || _except_ctx->is_ht_empty());
}
void set_finishing(RuntimeState* state) override {
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
_except_ctx->finish_probe_ht();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -42,9 +42,10 @@ public:
bool is_finished() const override { return _is_finished || _intersect_ctx->is_finished(); }
void set_finishing(RuntimeState* state) override {
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
_intersect_ctx->finish_build_ht();
return Status::OK();
}
Status prepare(RuntimeState* state) override;

View File

@ -17,4 +17,4 @@ void IntersectOutputSourceOperatorFactory::close(RuntimeState* state) {
SourceOperatorFactory::close(state);
}
} // namespace starrocks::pipeline
} // namespace starrocks::pipeline

View File

@ -29,7 +29,10 @@ public:
return _intersect_ctx->is_dependency_finished(_dependency_index) && _intersect_ctx->is_output_finished();
}
void set_finished(RuntimeState* state) override { _intersect_ctx->set_finished(); }
Status set_finished(RuntimeState* state) override {
_intersect_ctx->set_finished();
return Status::OK();
}
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;

View File

@ -28,8 +28,7 @@ Status IntersectProbeSinkOperatorFactory::prepare(RuntimeState* state) {
void IntersectProbeSinkOperatorFactory::close(RuntimeState* state) {
Expr::close(_dst_exprs, state);
OperatorFactory::close(state);
}
} // namespace starrocks::pipeline
} // namespace starrocks::pipeline

View File

@ -38,9 +38,10 @@ public:
(_is_finished || _intersect_ctx->is_ht_empty());
}
void set_finishing(RuntimeState* state) override {
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
_intersect_ctx->finish_probe_ht();
return Status::OK();
}
void close(RuntimeState* state) override;

View File

@ -26,6 +26,7 @@ StatusOr<vectorized::ChunkPtr> UnionConstSourceOperator::pull_chunk(starrocks::R
DCHECK_EQ(_const_expr_lists[_next_processed_row_index + row_i].size(), columns_count);
ColumnPtr src_column = _const_expr_lists[_next_processed_row_index + row_i][col_i]->evaluate(nullptr);
RETURN_IF_HAS_ERROR(_const_expr_lists[_next_processed_row_index + row_i]);
auto cur_row_dst_column =
vectorized::ColumnHelper::move_column(dst_slot->type(), dst_slot->is_nullable(), src_column, 1);
dst_column->append(*cur_row_dst_column, 0, 1);

View File

@ -34,9 +34,6 @@ public:
bool is_finished() const override { return !has_output(); };
// finish is noop.
void set_finishing(RuntimeState* state) override{};
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;
private:

View File

@ -43,7 +43,10 @@ public:
bool is_finished() const override { return _is_finished && _dst_chunk == nullptr; }
void set_finishing(RuntimeState* state) override { _is_finished = true; }
Status set_finishing(RuntimeState* state) override {
_is_finished = true;
return Status::OK();
}
Status push_chunk(RuntimeState* state, const ChunkPtr& src_chunk) override;

View File

@ -22,12 +22,13 @@ StatusOr<vectorized::ChunkPtr> LocalMergeSortSourceOperator::pull_chunk(RuntimeS
return _sort_context->pull_chunk();
}
void LocalMergeSortSourceOperator::set_finishing(RuntimeState* state) {
Status LocalMergeSortSourceOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
return Status::OK();
}
void LocalMergeSortSourceOperator::set_finished(RuntimeState* state) {
_sort_context->set_finished();
Status LocalMergeSortSourceOperator::set_finished(RuntimeState* state) {
return _sort_context->set_finished();
}
bool LocalMergeSortSourceOperator::has_output() const {

View File

@ -35,8 +35,8 @@ public:
void add_morsel(Morsel* morsel) {}
void set_finishing(RuntimeState* state) override;
void set_finished(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
Status set_finished(RuntimeState* state) override;
private:
bool _is_finished = false;

View File

@ -14,6 +14,7 @@
#include "exec/vectorized/chunks_sorter_full_sort.h"
#include "exec/vectorized/chunks_sorter_topn.h"
#include "exprs/expr.h"
#include "runtime/current_thread.h"
#include "runtime/exec_env.h"
#include "runtime/runtime_state.h"
@ -22,6 +23,7 @@ using namespace starrocks::vectorized;
namespace starrocks::pipeline {
Status PartitionSortSinkOperator::prepare(RuntimeState* state) {
RETURN_IF_ERROR(Operator::prepare(state));
_chunks_sorter->setup_runtime(_unique_metrics.get());
return Status::OK();
}
@ -37,17 +39,18 @@ StatusOr<vectorized::ChunkPtr> PartitionSortSinkOperator::pull_chunk(RuntimeStat
Status PartitionSortSinkOperator::push_chunk(RuntimeState* state, const vectorized::ChunkPtr& chunk) {
vectorized::ChunkPtr materialize_chunk = ChunksSorter::materialize_chunk_before_sort(
chunk.get(), _materialized_tuple_desc, _sort_exec_exprs, _order_by_types);
RETURN_IF_ERROR(_chunks_sorter->update(state, materialize_chunk));
TRY_CATCH_BAD_ALLOC(RETURN_IF_ERROR(_chunks_sorter->update(state, materialize_chunk)));
return Status::OK();
}
void PartitionSortSinkOperator::set_finishing(RuntimeState* state) {
_chunks_sorter->finish(state);
Status PartitionSortSinkOperator::set_finishing(RuntimeState* state) {
RETURN_IF_ERROR(_chunks_sorter->finish(state));
// Current partition sort is ended, and
// the last call will drive LocalMergeSortSourceOperator to work.
_sort_context->finish_partition(_chunks_sorter->get_partition_rows());
_is_finished = true;
return Status::OK();
}
Status PartitionSortSinkOperatorFactory::prepare(RuntimeState* state) {
@ -68,16 +71,16 @@ OperatorPtr PartitionSortSinkOperatorFactory::create(int32_t degree_of_paralleli
if (_limit <= ChunksSorter::USE_HEAP_SORTER_LIMIT_SZ) {
chunks_sorter = std::make_unique<HeapChunkSorter>(
runtime_state(), &(_sort_exec_exprs.lhs_ordering_expr_ctxs()), &_is_asc_order, &_is_null_first,
_offset, _limit, SIZE_OF_CHUNK_FOR_TOPN);
_sort_keys, _offset, _limit, SIZE_OF_CHUNK_FOR_TOPN);
} else {
chunks_sorter = std::make_unique<ChunksSorterTopn>(
runtime_state(), &(_sort_exec_exprs.lhs_ordering_expr_ctxs()), &_is_asc_order, &_is_null_first,
_offset, _limit, SIZE_OF_CHUNK_FOR_TOPN);
_sort_keys, _offset, _limit, SIZE_OF_CHUNK_FOR_TOPN);
}
} else {
chunks_sorter = std::make_unique<vectorized::ChunksSorterFullSort>(
runtime_state(), &(_sort_exec_exprs.lhs_ordering_expr_ctxs()), &_is_asc_order, &_is_null_first,
SIZE_OF_CHUNK_FOR_FULL_SORT);
_sort_keys, SIZE_OF_CHUNK_FOR_FULL_SORT);
}
auto sort_context = _sort_context_factory->create(driver_sequence);

View File

@ -63,7 +63,7 @@ public:
Status push_chunk(RuntimeState* state, const vectorized::ChunkPtr& chunk) override;
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
private:
bool _is_finished = false;
@ -89,7 +89,7 @@ public:
PartitionSortSinkOperatorFactory(
int32_t id, int32_t plan_node_id, std::shared_ptr<SortContextFactory> sort_context_factory,
SortExecExprs& sort_exec_exprs, std::vector<bool> is_asc_order, std::vector<bool> is_null_first,
int64_t offset, int64_t limit, const std::vector<OrderByType>& order_by_types,
const std::string& sort_keys, int64_t offset, int64_t limit, const std::vector<OrderByType>& order_by_types,
TupleDescriptor* materialized_tuple_desc, const RowDescriptor& parent_node_row_desc,
const RowDescriptor& parent_node_child_row_desc, const std::vector<ExprContext*>& analytic_partition_exprs)
: OperatorFactory(id, "partition_sort_sink", plan_node_id),
@ -97,6 +97,7 @@ public:
_sort_exec_exprs(sort_exec_exprs),
_is_asc_order(is_asc_order),
_is_null_first(is_null_first),
_sort_keys(sort_keys),
_offset(offset),
_limit(limit),
_order_by_types(order_by_types),
@ -118,6 +119,7 @@ private:
SortExecExprs& _sort_exec_exprs;
std::vector<bool> _is_asc_order;
std::vector<bool> _is_null_first;
const std::string _sort_keys;
int64_t _offset;
int64_t _limit;
const std::vector<OrderByType>& _order_by_types;

View File

@ -4,6 +4,13 @@
namespace starrocks::pipeline {
void TableFunctionOperator::close(RuntimeState* state) {
if (_table_function != nullptr && _table_function_state != nullptr) {
_table_function->close(state, _table_function_state);
}
Operator::close(state);
}
bool TableFunctionOperator::has_output() const {
return _input_chunk != nullptr && (_remain_repeat_times > 0 || _input_chunk_index < _input_chunk->num_rows());
}
@ -16,8 +23,9 @@ bool TableFunctionOperator::is_finished() const {
return _is_finished && !has_output();
}
void TableFunctionOperator::set_finishing(RuntimeState* state) {
Status TableFunctionOperator::set_finishing(RuntimeState* state) {
_is_finished = true;
return Status::OK();
}
Status TableFunctionOperator::prepare(RuntimeState* state) {
@ -58,7 +66,8 @@ Status TableFunctionOperator::prepare(RuntimeState* state) {
return_types.emplace_back(return_type.type);
}
_table_function = vectorized::get_table_function(table_function_name, arg_types, return_types);
_table_function =
vectorized::get_table_function(table_function_name, arg_types, return_types, table_fn.binary_type);
if (_table_function == nullptr) {
return Status::InternalError("can't find table function " + table_function_name);
}
@ -69,7 +78,8 @@ Status TableFunctionOperator::prepare(RuntimeState* state) {
_remain_repeat_times = 0;
_table_function_exec_timer = ADD_TIMER(_unique_metrics, "TableFunctionTime");
return _table_function->prepare(_table_function_state);
RETURN_IF_ERROR(_table_function->prepare(_table_function_state));
return _table_function->open(state, _table_function_state);
}
StatusOr<vectorized::ChunkPtr> TableFunctionOperator::pull_chunk(RuntimeState* state) {

View File

@ -2,8 +2,6 @@
#pragma once
#include "column/column_helper.h"
#include "column/type_traits.h"
#include "column/vectorized_fwd.h"
#include "exec/pipeline/operator.h"
#include "exprs/expr.h"
@ -21,13 +19,15 @@ public:
Status prepare(RuntimeState* state) override;
void close(RuntimeState* state) override;
bool has_output() const override;
bool need_input() const override;
bool is_finished() const override;
void set_finishing(RuntimeState* state) override;
Status set_finishing(RuntimeState* state) override;
StatusOr<vectorized::ChunkPtr> pull_chunk(RuntimeState* state) override;
@ -38,7 +38,7 @@ private:
void _process_table_function();
const TPlanNode& _tnode;
const vectorized::TableFunction* _table_function;
const vectorized::TableFunction* _table_function = nullptr;
//Slots of output by table function
std::vector<SlotId> _fn_result_slots;
@ -52,15 +52,15 @@ private:
//Input chunk currently being processed
vectorized::ChunkPtr _input_chunk;
//The current chunk is processed to which row
size_t _input_chunk_index;
size_t _input_chunk_index = 0;
//The current outer line needs to be repeated several times
size_t _remain_repeat_times;
size_t _remain_repeat_times = 0;
//table function result
std::pair<vectorized::Columns, vectorized::ColumnPtr> _table_function_result;
//table function return result end ?
bool _table_function_result_eos;
bool _table_function_result_eos = false;
//table function param and return offset
vectorized::TableFunctionState* _table_function_state;
vectorized::TableFunctionState* _table_function_state = nullptr;
//Profile
RuntimeProfile::Counter* _table_function_exec_timer = nullptr;

Some files were not shown because too many files have changed in this diff Show More