[SQL] Support Grouping Sets, Rollup and Cube to extend group by statement
Support Grouping Sets, Rollup and Cube to extend group by statement support GROUPING SETS syntax ``` SELECT a, b, SUM( c ) FROM tab1 GROUP BY GROUPING SETS ( (a, b), (a), (b), ( ) ); ``` cube or rollup like ``` SELECT a, b,c, SUM( d ) FROM tab1 GROUP BY ROLLUP|CUBE(a,b,c) ``` [ADD] support grouping functions in expr like grouping(a) + grouping(b) (#2039) [FIX] fix analyzer error in window function(#2039)
This commit is contained in:
@ -52,6 +52,7 @@
|
||||
#include "exprs/utility_functions.h"
|
||||
#include "exprs/json_functions.h"
|
||||
#include "exprs/hll_hash_function.h"
|
||||
#include "exprs/grouping_sets_functions.h"
|
||||
#include "exprs/timezone_db.h"
|
||||
#include "exprs/bitmap_function.h"
|
||||
#include "exprs/hll_function.h"
|
||||
@ -273,6 +274,7 @@ void init_daemon(int argc, char** argv, const std::vector<StorePath>& paths) {
|
||||
HllHashFunctions::init();
|
||||
ESFunctions::init();
|
||||
GeoFunctions::init();
|
||||
GroupingSetsFunctions::init();
|
||||
TimezoneDatabase::init();
|
||||
BitmapFunctions::init();
|
||||
HllFunctions::init();
|
||||
|
||||
@ -71,6 +71,7 @@ set(EXEC_FILES
|
||||
spill_sort_node.cc
|
||||
union_node.cpp
|
||||
union_node_ir.cpp
|
||||
repeat_node.cpp
|
||||
schema_scanner.cpp
|
||||
schema_scan_node.cpp
|
||||
schema_scanner/schema_tables_scanner.cpp
|
||||
|
||||
@ -50,6 +50,7 @@
|
||||
#include "exec/analytic_eval_node.h"
|
||||
#include "exec/select_node.h"
|
||||
#include "exec/union_node.h"
|
||||
#include "exec/repeat_node.h"
|
||||
#include "exec/assert_num_rows_node.h"
|
||||
#include "runtime/exec_env.h"
|
||||
#include "runtime/descriptors.h"
|
||||
@ -452,10 +453,14 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN
|
||||
*node = pool->add(new BrokerScanNode(pool, tnode, descs));
|
||||
return Status::OK();
|
||||
|
||||
case TPlanNodeType::REPEAT_NODE:
|
||||
*node = pool->add(new RepeatNode(pool, tnode, descs));
|
||||
return Status::OK();
|
||||
|
||||
case TPlanNodeType::ASSERT_NUM_ROWS_NODE:
|
||||
*node = pool->add(new AssertNumRowsNode(pool, tnode, descs));
|
||||
return Status::OK();
|
||||
|
||||
|
||||
default:
|
||||
map<int, const char*>::const_iterator i =
|
||||
_TPlanNodeType_VALUES_TO_NAMES.find(tnode.node_type);
|
||||
|
||||
223
be/src/exec/repeat_node.cpp
Normal file
223
be/src/exec/repeat_node.cpp
Normal file
@ -0,0 +1,223 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
#include "exec/repeat_node.h"
|
||||
|
||||
#include "exprs/expr.h"
|
||||
#include "runtime/raw_value.h"
|
||||
#include "runtime/row_batch.h"
|
||||
#include "runtime/runtime_state.h"
|
||||
#include "util/runtime_profile.h"
|
||||
#include "gutil/strings/join.h"
|
||||
|
||||
namespace doris {
|
||||
|
||||
RepeatNode::RepeatNode(ObjectPool* pool, const TPlanNode& tnode,
|
||||
const DescriptorTbl& descs)
|
||||
: ExecNode(pool, tnode, descs),
|
||||
_slot_id_set_list(tnode.repeat_node.slot_id_set_list),
|
||||
_repeat_id_list(tnode.repeat_node.repeat_id_list),
|
||||
_grouping_list(tnode.repeat_node.grouping_list),
|
||||
_output_tuple_id(tnode.repeat_node.output_tuple_id),
|
||||
_tuple_desc(nullptr),
|
||||
_child_row_batch(nullptr),
|
||||
_child_eos(false),
|
||||
_repeat_id_idx(0),
|
||||
_runtime_state(nullptr) {
|
||||
}
|
||||
|
||||
RepeatNode::~RepeatNode() {
|
||||
}
|
||||
|
||||
Status RepeatNode::prepare(RuntimeState* state) {
|
||||
SCOPED_TIMER(_runtime_profile->total_time_counter());
|
||||
RETURN_IF_ERROR(ExecNode::prepare(state));
|
||||
_runtime_state = state;
|
||||
_tuple_desc = state->desc_tbl().get_tuple_descriptor(_output_tuple_id);
|
||||
if (_tuple_desc == NULL) {
|
||||
return Status::InternalError("Failed to get tuple descriptor.");
|
||||
}
|
||||
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status RepeatNode::open(RuntimeState* state) {
|
||||
SCOPED_TIMER(_runtime_profile->total_time_counter());
|
||||
RETURN_IF_ERROR(ExecNode::open(state));
|
||||
RETURN_IF_CANCELLED(state);
|
||||
RETURN_IF_ERROR(child(0)->open(state));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
/**
|
||||
* copy the rows to new tuple based on repeat_id_idx and _repeat_id_list and fill in row_batch,
|
||||
* and then set grouping_id and other grouping function slot in child_row_batch
|
||||
* e.g. _repeat_id_list = [0, 3, 1, 2], _repeat_id_idx = 2, _grouping_list [[0, 3, 1, 2], [0, 1, 1, 0]],
|
||||
* row_batch tuple 0 ['a', 'b', 1] -> [['a', null, 1] tuple 1 [1, 1]]
|
||||
*/
|
||||
Status RepeatNode::get_repeated_batch(
|
||||
RowBatch* child_row_batch, int repeat_id_idx, RowBatch* row_batch) {
|
||||
|
||||
DCHECK(child_row_batch != nullptr);
|
||||
DCHECK_EQ(row_batch->num_rows(), 0);
|
||||
|
||||
// Fill all slots according to child
|
||||
MemPool* tuple_pool = row_batch->tuple_data_pool();
|
||||
const vector<TupleDescriptor*>& src_tuple_descs = child_row_batch->row_desc().tuple_descriptors();
|
||||
const vector<TupleDescriptor*>& dst_tuple_descs = row_batch->row_desc().tuple_descriptors();
|
||||
vector<Tuple*> dst_tuples(src_tuple_descs.size(), nullptr);
|
||||
for (Tuple* &tuple : dst_tuples) {
|
||||
void* tuple_buffer = tuple_pool->allocate(0);
|
||||
if (tuple_buffer == nullptr) {
|
||||
return Status::InternalError("Allocate memory for row batch failed.");
|
||||
}
|
||||
tuple = reinterpret_cast<Tuple*>(tuple_buffer);
|
||||
}
|
||||
for (int i = 0; i < child_row_batch->num_rows(); ++i) {
|
||||
int row_idx = row_batch->add_row();
|
||||
TupleRow* dst_row = row_batch->get_row(row_idx);
|
||||
TupleRow* src_row = child_row_batch->get_row(i);
|
||||
|
||||
auto src_it = src_tuple_descs.begin();
|
||||
auto dst_it = dst_tuple_descs.begin();
|
||||
for (int j = 0; src_it != src_tuple_descs.end() && dst_it != dst_tuple_descs.end(); ++src_it, ++dst_it, ++j) {
|
||||
Tuple* src_tuple = src_row->get_tuple(j);
|
||||
if (src_tuple == NULL) {
|
||||
continue;
|
||||
}
|
||||
|
||||
char* new_tuple = reinterpret_cast<char*>(dst_tuples[j]);
|
||||
new_tuple += (*dst_it)->byte_size();
|
||||
dst_tuples[j] = reinterpret_cast<Tuple*>(new_tuple);
|
||||
|
||||
dst_row->set_tuple(j, dst_tuples[j]);
|
||||
memset(dst_tuples[j], 0, (*dst_it)->num_null_bytes());
|
||||
src_tuple->deep_copy(dst_tuples[j], **dst_it, tuple_pool);
|
||||
for (int k = 0; k < (*src_it)->slots().size(); k++) {
|
||||
SlotDescriptor* src_slot_desc = (*src_it)->slots()[k];
|
||||
SlotDescriptor* dst_slot_desc = (*dst_it)->slots()[k];
|
||||
DCHECK_EQ(src_slot_desc->type().type, dst_slot_desc->type().type);
|
||||
DCHECK_EQ(src_slot_desc->col_name(), dst_slot_desc->col_name());
|
||||
// set null base on repeated list
|
||||
// the first element in _slot_id_set_list contain all slots, so find in the _slot_id_set_list[0]
|
||||
if (_slot_id_set_list[0].find(src_slot_desc->id()) != _slot_id_set_list[0].end()) {
|
||||
std::set<SlotId>& repeat_ids = _slot_id_set_list[repeat_id_idx];
|
||||
if (repeat_ids.find(src_slot_desc->id()) == repeat_ids.end()) {
|
||||
dst_tuples[j]->set_null(dst_slot_desc->null_indicator_offset());
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
row_batch->commit_last_row();
|
||||
}
|
||||
Tuple *tuple = nullptr;
|
||||
// Fill grouping ID to tuple
|
||||
for (int i = 0; i < child_row_batch->num_rows(); ++i) {
|
||||
int row_idx = i;
|
||||
TupleRow *row = row_batch->get_row(row_idx);
|
||||
|
||||
if (tuple == nullptr) {
|
||||
int size = row_batch->capacity() * _tuple_desc->byte_size();
|
||||
void *tuple_buffer = tuple_pool->allocate(size);
|
||||
if (tuple_buffer == nullptr) {
|
||||
return Status::InternalError("Allocate memory for row batch failed.");
|
||||
}
|
||||
tuple = reinterpret_cast<Tuple *>(tuple_buffer);
|
||||
} else {
|
||||
char *new_tuple = reinterpret_cast<char *>(tuple);
|
||||
new_tuple += _tuple_desc->byte_size();
|
||||
tuple = reinterpret_cast<Tuple *>(new_tuple);
|
||||
}
|
||||
|
||||
row->set_tuple(src_tuple_descs.size(), tuple);
|
||||
memset(tuple, 0, _tuple_desc->num_null_bytes());
|
||||
|
||||
for(size_t slot_idx = 0; slot_idx < _grouping_list.size(); ++slot_idx) {
|
||||
int64_t val = _grouping_list[slot_idx][repeat_id_idx];
|
||||
DCHECK_LT(slot_idx, _tuple_desc->slots().size()) << "TupleDescriptor: " << _tuple_desc->debug_string();
|
||||
const SlotDescriptor *slot_desc = _tuple_desc->slots()[slot_idx];
|
||||
tuple->set_not_null(slot_desc->null_indicator_offset());
|
||||
RawValue::write(&val, tuple, slot_desc, tuple_pool);
|
||||
}
|
||||
}
|
||||
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status RepeatNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {
|
||||
SCOPED_TIMER(_runtime_profile->total_time_counter());
|
||||
RETURN_IF_CANCELLED(state);
|
||||
DCHECK(_repeat_id_idx >= 0);
|
||||
for (const std::vector<int64_t>& v : _grouping_list) {
|
||||
DCHECK(_repeat_id_idx <= (int)v.size());
|
||||
}
|
||||
// current child has finished its repeat, get child's next batch
|
||||
if (_child_row_batch.get() == nullptr) {
|
||||
if (_child_eos) {
|
||||
*eos = true;
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
_child_row_batch.reset(
|
||||
new RowBatch(child(0)->row_desc(), state->batch_size(), mem_tracker()));
|
||||
RETURN_IF_ERROR(child(0)->get_next(state, _child_row_batch.get(), &_child_eos));
|
||||
|
||||
if (_child_row_batch->num_rows() <= 0) {
|
||||
_child_row_batch.reset(nullptr);
|
||||
*eos = true;
|
||||
return Status::OK();
|
||||
}
|
||||
}
|
||||
|
||||
DCHECK_EQ(row_batch->num_rows(), 0);
|
||||
RETURN_IF_ERROR(get_repeated_batch(_child_row_batch.get(), _repeat_id_idx, row_batch));
|
||||
_repeat_id_idx++;
|
||||
|
||||
int size = _repeat_id_list.size();
|
||||
if (_repeat_id_idx >= size) {
|
||||
_child_row_batch.reset(nullptr);
|
||||
_repeat_id_idx = 0;
|
||||
}
|
||||
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status RepeatNode::close(RuntimeState* state) {
|
||||
if (is_closed()) {
|
||||
return Status::OK();
|
||||
}
|
||||
_child_row_batch.reset(nullptr);
|
||||
RETURN_IF_ERROR(child(0)->close(state));
|
||||
return ExecNode::close(state);
|
||||
}
|
||||
|
||||
void RepeatNode::debug_string(int indentation_level, std::stringstream* out) const {
|
||||
*out << string(indentation_level * 2, ' ');
|
||||
*out << "RepeatNode(";
|
||||
*out << "repeat pattern: [" << JoinElements(_repeat_id_list, ",") << "]\n";
|
||||
*out << "add " << _grouping_list.size() << " columns. \n";
|
||||
*out << "added column values: ";
|
||||
for (const std::vector<int64_t> &v : _grouping_list ){
|
||||
*out << "[" << JoinElements(v, ",") << "] ";
|
||||
}
|
||||
*out << "\n";
|
||||
ExecNode::debug_string(indentation_level, out);
|
||||
*out << ")";
|
||||
}
|
||||
|
||||
}
|
||||
61
be/src/exec/repeat_node.h
Normal file
61
be/src/exec/repeat_node.h
Normal file
@ -0,0 +1,61 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "exec/exec_node.h"
|
||||
|
||||
namespace doris {
|
||||
|
||||
class Tuple;
|
||||
class RuntimeState;
|
||||
class RowBatch;
|
||||
|
||||
// repeat tuple of children and set given slots to null, this class generates tuple rows according to the given
|
||||
// _repeat_id_list, and sets the value of the slot corresponding to the grouping function according to _grouping_list
|
||||
class RepeatNode : public ExecNode {
|
||||
public:
|
||||
RepeatNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
|
||||
~RepeatNode();
|
||||
|
||||
virtual Status prepare(RuntimeState* state) override;
|
||||
virtual Status open(RuntimeState* state) override;
|
||||
virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
|
||||
virtual Status close(RuntimeState* state) override;
|
||||
|
||||
protected:
|
||||
virtual void debug_string(int indentation_level, std::stringstream* out) const override;
|
||||
|
||||
private:
|
||||
Status get_repeated_batch(RowBatch* child_row_batch, int repeat_id_idx, RowBatch* row_batch);
|
||||
|
||||
// Slot id set used to indicate those slots need to set to null.
|
||||
std::vector<std::set<SlotId>> _slot_id_set_list;
|
||||
// An integer bitmap list, it indicates the bit position of the exprs not null.
|
||||
std::vector<int64_t> _repeat_id_list;
|
||||
std::vector<std::vector<int64_t>> _grouping_list;
|
||||
// Tulple id used for output, it has new slots.
|
||||
TupleId _output_tuple_id;
|
||||
const TupleDescriptor* _tuple_desc;
|
||||
|
||||
std::unique_ptr<RowBatch> _child_row_batch;
|
||||
bool _child_eos;
|
||||
int _repeat_id_idx;
|
||||
RuntimeState* _runtime_state;
|
||||
};
|
||||
|
||||
}
|
||||
@ -66,4 +66,5 @@ add_library(Exprs
|
||||
agg_fn.cc
|
||||
new_agg_fn_evaluator.cc
|
||||
bitmap_function.cpp
|
||||
hll_function.cpp)
|
||||
hll_function.cpp
|
||||
grouping_sets_functions.cpp)
|
||||
|
||||
36
be/src/exprs/grouping_sets_functions.cpp
Normal file
36
be/src/exprs/grouping_sets_functions.cpp
Normal file
@ -0,0 +1,36 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
#include "exprs/grouping_sets_functions.h"
|
||||
|
||||
namespace doris {
|
||||
|
||||
void GroupingSetsFunctions::init() {
|
||||
}
|
||||
|
||||
doris_udf::BigIntVal GroupingSetsFunctions::grouping_id(
|
||||
doris_udf::FunctionContext* ctx, const doris_udf::BigIntVal& grouping_id) {
|
||||
return grouping_id;
|
||||
}
|
||||
|
||||
BigIntVal GroupingSetsFunctions::grouping(
|
||||
doris_udf::FunctionContext* ctx, const doris_udf::BigIntVal& grouping) {
|
||||
return grouping;
|
||||
}
|
||||
|
||||
} // doris
|
||||
|
||||
37
be/src/exprs/grouping_sets_functions.h
Normal file
37
be/src/exprs/grouping_sets_functions.h
Normal file
@ -0,0 +1,37 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "udf/udf.h"
|
||||
|
||||
namespace doris {
|
||||
|
||||
class Expr;
|
||||
|
||||
class GroupingSetsFunctions {
|
||||
public:
|
||||
static void init();
|
||||
|
||||
static doris_udf::BigIntVal grouping_id(
|
||||
doris_udf::FunctionContext* ctx, const doris_udf::BigIntVal& grouping_id);
|
||||
static doris_udf::BigIntVal grouping(
|
||||
doris_udf::FunctionContext* ctx, const doris_udf::BigIntVal& grouping);
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
510
docs/documentation/cn/internal/grouping_sets_design.md
Normal file
510
docs/documentation/cn/internal/grouping_sets_design.md
Normal file
@ -0,0 +1,510 @@
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one
|
||||
or more contributor license agreements. See the NOTICE file
|
||||
distributed with this work for additional information
|
||||
regarding copyright ownership. The ASF licenses this file
|
||||
to you under the Apache License, Version 2.0 (the
|
||||
"License"); you may not use this file except in compliance
|
||||
with the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing,
|
||||
software distributed under the License is distributed on an
|
||||
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
KIND, either express or implied. See the License for the
|
||||
specific language governing permissions and limitations
|
||||
under the License.
|
||||
-->
|
||||
|
||||
# GROUPING SETS 设计文档
|
||||
|
||||
## 1. GROUPING SETS 相关背景知识
|
||||
|
||||
### 1.1 GROUPING SETS 子句
|
||||
|
||||
GROUP BY GROUPING SETS 是对 GROUP BY 子句的扩展,它能够在一个 GROUP BY 子句中一次实现多个集合的分组。其结果等价于将多个相应 GROUP BY 子句进行 UNION 操作。
|
||||
|
||||
特别地,一个空的子集意味着将所有的行聚集到一个分组。
|
||||
GROUP BY 子句是只含有一个元素的 GROUP BY GROUPING SETS 的特例。
|
||||
|
||||
例如,GROUPING SETS 语句:
|
||||
|
||||
```
|
||||
SELECT k1, k2, SUM( k3 ) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k1), (k2), ( ) );
|
||||
```
|
||||
|
||||
其查询结果等价于:
|
||||
|
||||
```
|
||||
SELECT k1, k2, SUM( k3 ) FROM t GROUP BY k1, k2
|
||||
UNION
|
||||
SELECT k1, null, SUM( k3 ) FROM t GROUP BY k1
|
||||
UNION
|
||||
SELECT null, k2, SUM( k3 ) FROM t GROUP BY k2
|
||||
UNION
|
||||
SELECT null, null, SUM( k3 ) FROM t
|
||||
```
|
||||
|
||||
下面是一个实际数据的例子:
|
||||
|
||||
```
|
||||
mysql> SELECT * FROM t;
|
||||
+------+------+------+
|
||||
| k1 | k2 | k3 |
|
||||
+------+------+------+
|
||||
| a | A | 1 |
|
||||
| a | A | 2 |
|
||||
| a | B | 1 |
|
||||
| a | B | 3 |
|
||||
| b | A | 1 |
|
||||
| b | A | 4 |
|
||||
| b | B | 1 |
|
||||
| b | B | 5 |
|
||||
+------+------+------+
|
||||
8 rows in set (0.01 sec)
|
||||
|
||||
mysql> SELECT k1, k2, SUM(k3) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k2), (k1), ( ) );
|
||||
+------+------+-----------+
|
||||
| k1 | k2 | sum(`k3`) |
|
||||
+------+------+-----------+
|
||||
| b | B | 6 |
|
||||
| a | B | 4 |
|
||||
| a | A | 3 |
|
||||
| b | A | 5 |
|
||||
| NULL | B | 10 |
|
||||
| NULL | A | 8 |
|
||||
| a | NULL | 7 |
|
||||
| b | NULL | 11 |
|
||||
| NULL | NULL | 18 |
|
||||
+------+------+-----------+
|
||||
9 rows in set (0.06 sec)
|
||||
```
|
||||
|
||||
### 1.2 ROLLUP 子句
|
||||
|
||||
ROLLUP 是对 GROUPING SETS 的扩展。
|
||||
|
||||
```
|
||||
SELECT a, b,c, SUM( d ) FROM tab1 GROUP BY ROLLUP(a,b,c)
|
||||
```
|
||||
|
||||
这个 ROLLUP 等价于下面的 GROUPING SETS:
|
||||
|
||||
```
|
||||
GROUPING SETS (
|
||||
(a,b,c),
|
||||
( a, b ),
|
||||
( a),
|
||||
( )
|
||||
)
|
||||
```
|
||||
|
||||
### 1.3 CUBE 子句
|
||||
|
||||
CUBE 也是对 GROUPING SETS 的扩展。
|
||||
|
||||
```
|
||||
CUBE ( e1, e2, e3, ... )
|
||||
```
|
||||
|
||||
其含义是 GROUPING SETS 后面列表中的所有子集。
|
||||
|
||||
例如,CUBE ( a, b, c ) 等价于下面的 GROUPING SETS:
|
||||
|
||||
```
|
||||
GROUPING SETS (
|
||||
( a, b, c ),
|
||||
( a, b ),
|
||||
( a, c ),
|
||||
( a ),
|
||||
( b, c ),
|
||||
( b ),
|
||||
( c ),
|
||||
( )
|
||||
)
|
||||
```
|
||||
|
||||
### 1.4 GROUPING 和 GROUPING_ID 函数
|
||||
当我们没有统计某一列时,它的值显示为 NULL,这也可能是列本身就有 NULL 值,这就需要一种方法区分是没有统计还是值本来就是 NULL。为此引入 GROUPING 和 GROUPING_ID 函数。
|
||||
GROUPING(column:Column) 函数用于区分分组后的单个列是普通行和聚合行。如果是聚合行,则返回1,反之,则是0. GROUPING() 只能有一个参数列。
|
||||
|
||||
GROUPING_ID(column1, column2) 则根据指定的column 顺序,否则根据聚合的时候给的集合的元素顺序,计算出一个列列表的 bitmap 值,一个列如果是聚合列为0,否则为1. GROUPING_ID()函数返回位向量的十进制值。
|
||||
比如 [0 1 0] ->2 从下列第三个查询可以看到这种对应关系
|
||||
|
||||
例如,对于下面的表:
|
||||
|
||||
```
|
||||
mysql> select * from t;
|
||||
+------+------+------+
|
||||
| k1 | k2 | k3 |
|
||||
+------+------+------+
|
||||
| a | A | 1 |
|
||||
| a | A | 2 |
|
||||
| a | B | 1 |
|
||||
| a | B | 3 |
|
||||
| b | A | 1 |
|
||||
| b | A | 4 |
|
||||
| b | B | 1 |
|
||||
| b | B | 5 |
|
||||
+------+------+------+
|
||||
```
|
||||
|
||||
grouping sets 的结果如下:
|
||||
|
||||
```
|
||||
mysql> SELECT k1, k2, GROUPING(k1), GROUPING(k2), SUM(k3) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k2), (k1), ( ) );
|
||||
+------+------+----------------+----------------+-----------+
|
||||
| k1 | k2 | grouping(`k1`) | grouping(`k2`) | sum(`k3`) |
|
||||
+------+------+----------------+----------------+-----------+
|
||||
| a | A | 0 | 0 | 3 |
|
||||
| a | B | 0 | 0 | 4 |
|
||||
| a | NULL | 0 | 1 | 7 |
|
||||
| b | A | 0 | 0 | 5 |
|
||||
| b | B | 0 | 0 | 6 |
|
||||
| b | NULL | 0 | 1 | 11 |
|
||||
| NULL | A | 1 | 0 | 8 |
|
||||
| NULL | B | 1 | 0 | 10 |
|
||||
| NULL | NULL | 1 | 1 | 18 |
|
||||
+------+------+----------------+----------------+-----------+
|
||||
9 rows in set (0.02 sec)
|
||||
|
||||
mysql> SELECT k1, k2, GROUPING_ID(k1,k2), SUM(k3) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k2), (k1), ( ) );
|
||||
+------+------+-------------------------+-----------+
|
||||
| k1 | k2 | grouping_id(`k1`, `k2`) | sum(`k3`) |
|
||||
+------+------+-------------------------+-----------+
|
||||
| a | A | 0 | 3 |
|
||||
| a | B | 0 | 4 |
|
||||
| a | NULL | 1 | 7 |
|
||||
| b | A | 0 | 5 |
|
||||
| b | B | 0 | 6 |
|
||||
| b | NULL | 1 | 11 |
|
||||
| NULL | A | 2 | 8 |
|
||||
| NULL | B | 2 | 10 |
|
||||
| NULL | NULL | 3 | 18 |
|
||||
+------+------+-------------------------+-----------+
|
||||
9 rows in set (0.02 sec)
|
||||
|
||||
mysql> SELECT k1, k2, grouping(k1), grouping(k2), GROUPING_ID(k1,k2), SUM(k4) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k2), (k1), ( ) ) order by k1, k2;
|
||||
+------+------+----------------+----------------+-------------------------+-----------+
|
||||
| k1 | k2 | grouping(`k1`) | grouping(`k2`) | grouping_id(`k1`, `k2`) | sum(`k4`) |
|
||||
+------+------+----------------+----------------+-------------------------+-----------+
|
||||
| a | A | 0 | 0 | 0 | 3 |
|
||||
| a | B | 0 | 0 | 0 | 4 |
|
||||
| a | NULL | 0 | 1 | 1 | 7 |
|
||||
| b | A | 0 | 0 | 0 | 5 |
|
||||
| b | B | 0 | 0 | 0 | 6 |
|
||||
| b | NULL | 0 | 1 | 1 | 11 |
|
||||
| NULL | A | 1 | 0 | 2 | 8 |
|
||||
| NULL | B | 1 | 0 | 2 | 10 |
|
||||
| NULL | NULL | 1 | 1 | 3 | 18 |
|
||||
+------+------+----------------+----------------+-------------------------+-----------+
|
||||
9 rows in set (0.02 sec)
|
||||
|
||||
```
|
||||
|
||||
### 1.5 GROUPING SETS 的组合与嵌套
|
||||
|
||||
首先,一个 GROUP BY 子句本质上是一个 GROUPING SETS 的特例, 例如:
|
||||
|
||||
```
|
||||
GROUP BY a
|
||||
等同于
|
||||
GROUP BY GROUPING SETS((a))
|
||||
同样地,
|
||||
GROUP BY a,b,c
|
||||
等同于
|
||||
GROUP BY GROUPING SETS((a,b,c))
|
||||
```
|
||||
|
||||
同样的,CUBE 和 ROLLUP 也可以展开成 GROUPING SETS,因此 GROUP BY, CUBE, ROLLUP, GROUPING SETS 的各种组合和嵌套本质上就是 GROUPING SETS 的组合与嵌套。
|
||||
|
||||
对于 GROUPING SETS 的嵌套,语义上等价于将嵌套内的语句直接写到外面。(参考:<https://www.brytlyt.com/documentation/data-manipulation-dml/grouping-sets-rollup-cube/>),其中写道:
|
||||
|
||||
```
|
||||
The CUBE and ROLLUP constructs can be used either directly in the GROUP BY clause, or nested inside a GROUPING SETS clause. If one GROUPING SETS clause is nested inside another, the effect is the same as if all the elements of the inner clause had been written directly in the outer clause.
|
||||
```
|
||||
|
||||
对于多个 GROUPING SETS 的组合列表,很多数据库认为是叉乘(cross product)的关系。
|
||||
|
||||
例如:
|
||||
|
||||
```
|
||||
GROUP BY a, CUBE (b, c), GROUPING SETS ((d), (e))
|
||||
|
||||
等同于:
|
||||
|
||||
GROUP BY GROUPING SETS (
|
||||
(a, b, c, d), (a, b, c, e),
|
||||
(a, b, d), (a, b, e),
|
||||
(a, c, d), (a, c, e),
|
||||
(a, d), (a, e)
|
||||
)
|
||||
```
|
||||
|
||||
对于 GROUPING SETS 的组合与嵌套,各个数据库支持不太一样。例如 snowflake 不支持任何的组合和嵌套。
|
||||
(<https://docs.snowflake.net/manuals/sql-reference/constructs/group-by.html>)
|
||||
|
||||
Oracle 既支持组合,也支持嵌套。
|
||||
(<https://docs.oracle.com/cd/B19306_01/server.102/b14223/aggreg.htm#i1006842>)
|
||||
|
||||
Presto 支持组合,但不支持嵌套。
|
||||
(<https://prestodb.github.io/docs/current/sql/select.html>)
|
||||
|
||||
## 2. 设计目标
|
||||
|
||||
从语法上支持 GROUPING SETS, ROLLUP 和 CUBE。实现上述所述的1.1, 1.2, 1.3 1.4.
|
||||
|
||||
对于 1.5 GROUPING 函数 和 1.6 GROUPING SETS 的组合与嵌套 先不实现。
|
||||
|
||||
具体语法列出如下:
|
||||
|
||||
### 2.1 GROUPING SETS 语法
|
||||
|
||||
```
|
||||
SELECT ...
|
||||
FROM ...
|
||||
[ ... ]
|
||||
GROUP BY GROUPING SETS ( groupSet [ , groupSet [ , ... ] ] )
|
||||
[ ... ]
|
||||
|
||||
groupSet ::= { ( expr [ , expr [ , ... ] ] )}
|
||||
|
||||
<expr>
|
||||
各种表达式,包括列名.
|
||||
|
||||
```
|
||||
|
||||
### 2.2 ROLLUP 语法
|
||||
|
||||
```
|
||||
SELECT ...
|
||||
FROM ...
|
||||
[ ... ]
|
||||
GROUP BY ROLLUP ( expr [ , expr [ , ... ] ] )
|
||||
[ ... ]
|
||||
|
||||
<expr>
|
||||
各种表达式,包括列名.
|
||||
|
||||
```
|
||||
|
||||
### 2.3 CUBE 语法
|
||||
|
||||
```
|
||||
SELECT ...
|
||||
FROM ...
|
||||
[ ... ]
|
||||
GROUP BY CUBE ( expr [ , expr [ , ... ] ] )
|
||||
[ ... ]
|
||||
|
||||
<expr>
|
||||
各种表达式,包括列名.
|
||||
|
||||
```
|
||||
|
||||
## 3. 实现方案
|
||||
|
||||
### 3.1 整体思路
|
||||
|
||||
既然 GROUPING SET 子句逻辑上等价于多个相应 GROUP BY 子句的 UNION,可以通过扩展输入行(此输入行已经是通过下推条件过滤和投影后的), 在此基础上进行一个单一的 GROUP BY 操作来达到目的。
|
||||
|
||||
关键是怎样扩展输入行呢?下面举例说明:
|
||||
|
||||
例如,对应下面的语句:
|
||||
|
||||
```
|
||||
SELECT a, b FROM src GROUP BY a, b GROUPING SETS ((a, b), (a), (b), ());
|
||||
|
||||
```
|
||||
|
||||
假定 src 表的数据如下:
|
||||
|
||||
```
|
||||
1, 2
|
||||
3, 4
|
||||
|
||||
```
|
||||
|
||||
根据 GROUPING SETS 子句给出的列表,可以将输入行扩展为下面的 8 行 (GROUPING SETS集合数 * 行数, 同时为每行生成对应的 全列的GROUPING_ID: 和其他grouping 函数的值
|
||||
|
||||
```
|
||||
1, 2 (GROUPING_ID: a, b -> 00->0)
|
||||
1, null (GUPING_ID: a, null -> 01 -> 1)
|
||||
null, 2 (GROUPING_ID: null, b -> 10 -> 2)
|
||||
null, null (GROUPING_ID: null, null -> 11 -> 3)
|
||||
|
||||
3, 4 (GROUPING_ID: a, b -> 00 -> 0)
|
||||
3, null (GROUPING_ID: a, null -> 01 -> 1)
|
||||
null, 4 (GROUPING_ID: null, b -> 10 -> 2)
|
||||
null, null (GROUPING_ID: null, null -> 11 -> 3)
|
||||
|
||||
```
|
||||
|
||||
然后,将上面的 8 行数据作为输入,对 a, b, GROUPING_ID 进行 GROUP BY 操作即可。
|
||||
|
||||
### 3.2 具体例子验证说明
|
||||
|
||||
假设有一个 t 表,包含如下列和数据:
|
||||
|
||||
```
|
||||
mysql> select * from t;
|
||||
+------+------+------+
|
||||
| k1 | k2 | k3 |
|
||||
+------+------+------+
|
||||
| a | A | 1 |
|
||||
| a | A | 2 |
|
||||
| a | B | 1 |
|
||||
| a | B | 3 |
|
||||
| b | A | 1 |
|
||||
| b | A | 4 |
|
||||
| b | B | 1 |
|
||||
| b | B | 5 |
|
||||
+------+------+------+
|
||||
8 rows in set (0.01 sec)
|
||||
|
||||
```
|
||||
|
||||
对于如下的查询:
|
||||
|
||||
```
|
||||
SELECT k1, k2, GROUPING_ID(k1,k2), SUM(k3) FROM t GROUP BY GROUPING SETS ((k1, k2), (k1), (k2), ());
|
||||
|
||||
```
|
||||
|
||||
首先,对输入行进行扩展,每行数据扩展成 4 行 (GROUPING SETS子句的集合数目),同时增加 GROUPING_ID() 列 :
|
||||
|
||||
例如 a, A, 1 扩展后变成下面的 4 行:
|
||||
|
||||
```
|
||||
+------+------+------+-------------------------+
|
||||
| k1 | k2 | k3 | GROUPING_ID(`k1`, `k2`) |
|
||||
+------+------+------+-------------------------+
|
||||
| a | A | 1 | 0 |
|
||||
| a | NULL | 1 | 1 |
|
||||
| NULL | A | 1 | 2 |
|
||||
| NULL | NULL | 1 | 3 |
|
||||
+------+------+------+-------------------------+
|
||||
|
||||
```
|
||||
|
||||
最终, 全部扩展后的输入行如下(总共 32 行):
|
||||
|
||||
```
|
||||
+------+------+------+-------------------------+
|
||||
| k1 | k2 | k3 | GROUPING_ID(`k1`, `k2`) |
|
||||
+------+------+------+-------------------------+
|
||||
| a | A | 1 | 0 |
|
||||
| a | A | 2 | 0 |
|
||||
| a | B | 1 | 0 |
|
||||
| a | B | 3 | 0 |
|
||||
| b | A | 1 | 0 |
|
||||
| b | A | 4 | 0 |
|
||||
| b | B | 1 | 0 |
|
||||
| b | B | 5 | 0 |
|
||||
| a | NULL | 1 | 1 |
|
||||
| a | NULL | 1 | 1 |
|
||||
| a | NULL | 2 | 1 |
|
||||
| a | NULL | 3 | 1 |
|
||||
| b | NULL | 1 | 1 |
|
||||
| b | NULL | 1 | 1 |
|
||||
| b | NULL | 4 | 1 |
|
||||
| b | NULL | 5 | 1 |
|
||||
| NULL | A | 1 | 2 |
|
||||
| NULL | A | 1 | 2 |
|
||||
| NULL | A | 2 | 2 |
|
||||
| NULL | A | 4 | 2 |
|
||||
| NULL | B | 1 | 2 |
|
||||
| NULL | B | 1 | 2 |
|
||||
| NULL | B | 3 | 2 |
|
||||
| NULL | B | 5 | 2 |
|
||||
| NULL | NULL | 1 | 3 |
|
||||
| NULL | NULL | 1 | 3 |
|
||||
| NULL | NULL | 1 | 3 |
|
||||
| NULL | NULL | 1 | 3 |
|
||||
| NULL | NULL | 2 | 3 |
|
||||
| NULL | NULL | 3 | 3 |
|
||||
| NULL | NULL | 4 | 3 |
|
||||
| NULL | NULL | 5 | 3 |
|
||||
+------+------+------+-------------------------+
|
||||
32 rows in set.
|
||||
|
||||
```
|
||||
|
||||
现在对k1, k2, GROUPING_ID(`k1`, `k2`) 进行 GROUP BY:
|
||||
|
||||
```
|
||||
+------+------+-------------------------+-----------+
|
||||
| k1 | k2 | grouping_id(`k1`, `k2`) | sum(`k3`) |
|
||||
+------+------+-------------------------+-----------+
|
||||
| a | A | 0 | 3 |
|
||||
| a | B | 0 | 4 |
|
||||
| a | NULL | 1 | 7 |
|
||||
| b | A | 0 | 5 |
|
||||
| b | B | 0 | 6 |
|
||||
| b | NULL | 1 | 11 |
|
||||
| NULL | A | 2 | 8 |
|
||||
| NULL | B | 2 | 10 |
|
||||
| NULL | NULL | 3 | 18 |
|
||||
+------+------+-------------------------+-----------+
|
||||
9 rows in set (0.02 sec)
|
||||
|
||||
```
|
||||
|
||||
可以看到,其结果与对 GROUPING SETS 子句后每个子集进行 GROUP BY 后再进行 UNION 的结果一致。
|
||||
|
||||
```
|
||||
select k1, k2, sum(k3) from t group by k1, k2
|
||||
UNION ALL
|
||||
select NULL, k2, sum(k3) from t group by k2
|
||||
UNION ALL
|
||||
select k1, NULL, sum(k3) from t group by k1
|
||||
UNION ALL
|
||||
select NULL, NULL, sum(k3) from t;
|
||||
|
||||
+------+------+-----------+
|
||||
| k1 | k2 | sum(`k3`) |
|
||||
+------+------+-----------+
|
||||
| b | B | 6 |
|
||||
| b | A | 5 |
|
||||
| a | A | 3 |
|
||||
| a | B | 4 |
|
||||
| a | NULL | 7 |
|
||||
| b | NULL | 11 |
|
||||
| NULL | B | 10 |
|
||||
| NULL | A | 8 |
|
||||
| NULL | NULL | 18 |
|
||||
+------+------+-----------+
|
||||
9 rows in set (0.06 sec)
|
||||
|
||||
```
|
||||
|
||||
### 3.3 FE 规划阶段
|
||||
|
||||
#### 3.3.1 主要任务
|
||||
|
||||
1. 引入 GroupByClause 类,封装 Group By 相关信息,替换原有的 groupingExprs.
|
||||
2. 增加 Grouping Sets, Cube 和 RollUp 的语法支持和语法检查、错误处理和错误信息;
|
||||
3. 在 SelectStmt 类中增加 GroupByClause 成员;
|
||||
4. 引入 GroupingFunctionCallExpr 类,封装grouping 和grouping_id 函数调用
|
||||
5. 引入 VirtualSlot 类,封装grouping,grouping_id 生成的虚拟列和实际列的对应关系
|
||||
6. 增加虚拟列 GROUPING_ID 和其他grouping,grouping_id 函数对应的虚拟列,并将此列加入到原有的 groupingExprs 表达式列表中;
|
||||
7. 增加一个 PlanNode,考虑更通用的功能,命名为 RepeatNode。对于 GroupingSets 的聚合,在执行计划中插入 RepeatNode。
|
||||
|
||||
#### 3.3.2 Tuple
|
||||
|
||||
在 GroupByClause 类中为了将 GROUPING_ID 加到 groupingExprs 表达式列表中,需要创建 virtual SlotRef, 相应的,需要对这个 slot 创建一个 tuple, 叫 GROUPING_ID Tuple。
|
||||
|
||||
对于 RepeatNode 这个执行计划,其输入是子节点的所有 tuple, 输出的 tuple 除了 repeat 子节点的数据外,还需要填写 GROUPING_ID 和其他grouping,grouping_id 对应的虚拟列,因此。
|
||||
|
||||
|
||||
### 3.4 BE 查询执行阶段
|
||||
|
||||
主要任务:
|
||||
|
||||
1. 通过 RepeatNode 的执行类,增加扩展输入行的逻辑,其功能是在聚合之前将原有数据进行 repeat:对每行增加一列 GROUPING_ID, 然后按照 GroupingSets 中的集合数进行 repeat,并对对应列置为 null。根据grouping list设置新增虚拟列的值
|
||||
2. 实现 grouping_id() 和grouping() 函数。
|
||||
|
||||
|
||||
|
||||
|
||||
@ -0,0 +1,163 @@
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one
|
||||
or more contributor license agreements. See the NOTICE file
|
||||
distributed with this work for additional information
|
||||
regarding copyright ownership. The ASF licenses this file
|
||||
to you under the Apache License, Version 2.0 (the
|
||||
"License"); you may not use this file except in compliance
|
||||
with the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing,
|
||||
software distributed under the License is distributed on an
|
||||
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
KIND, either express or implied. See the License for the
|
||||
specific language governing permissions and limitations
|
||||
under the License.
|
||||
-->
|
||||
|
||||
# GROUP BY
|
||||
|
||||
## description
|
||||
|
||||
GROUP BY `GROUPING SETS` | `CUBE` | `ROLLUP` 是对 GROUP BY 子句的扩展,它能够在一个 GROUP BY 子句中实现多个集合的分组的聚合。其结果等价于将多个相应 GROUP BY 子句进行 UNION 操作。
|
||||
|
||||
GROUP BY 子句是只含有一个元素的 GROUP BY GROUPING SETS 的特例。
|
||||
例如,GROUPING SETS 语句:
|
||||
|
||||
```
|
||||
SELECT a, b, SUM( c ) FROM tab1 GROUP BY GROUPING SETS ( (a, b), (a), (b), ( ) );
|
||||
```
|
||||
|
||||
其查询结果等价于:
|
||||
|
||||
```
|
||||
SELECT a, b, SUM( c ) FROM tab1 GROUP BY a, b
|
||||
UNION
|
||||
SELECT a, null, SUM( c ) FROM tab1 GROUP BY a
|
||||
UNION
|
||||
SELECT null, b, SUM( c ) FROM tab1 GROUP BY b
|
||||
UNION
|
||||
SELECT null, null, SUM( c ) FROM tab1
|
||||
```
|
||||
|
||||
`GROUPING(expr)` 指示一个列是否为聚合列,如果是聚合列为0,否则为1
|
||||
|
||||
`GROUPING_ID(expr [ , expr [ , ... ] ])` 与GROUPING 类似, GROUPING_ID根据指定的column 顺序,计算出一个列列表的 bitmap 值,每一位为GROUPING的值. GROUPING_ID()函数返回位向量的十进制值。
|
||||
|
||||
### Syntax
|
||||
|
||||
```
|
||||
SELECT ...
|
||||
FROM ...
|
||||
[ ... ]
|
||||
GROUP BY [
|
||||
, ... |
|
||||
GROUPING SETS [, ...] ( groupSet [ , groupSet [ , ... ] ] ) |
|
||||
ROLLUP(expr [ , expr [ , ... ] ]) |
|
||||
expr [ , expr [ , ... ] ] WITH ROLLUP |
|
||||
CUBE(expr [ , expr [ , ... ] ]) |
|
||||
expr [ , expr [ , ... ] ] WITH CUBE
|
||||
]
|
||||
[ ... ]
|
||||
```
|
||||
|
||||
### Parameters
|
||||
|
||||
`groupSet` 表示 select list 中的列,别名或者表达式组成的集合 `groupSet ::= { ( expr [ , expr [ , ... ] ] )}`
|
||||
|
||||
`expr` 表示 select list 中的列,别名或者表达式
|
||||
|
||||
### Note
|
||||
|
||||
doris 支持类似PostgreSQL 语法, 语法实例如下
|
||||
|
||||
```
|
||||
SELECT a, b, SUM( c ) FROM tab1 GROUP BY GROUPING SETS ( (a, b), (a), (b), ( ) );
|
||||
SELECT a, b,c, SUM( d ) FROM tab1 GROUP BY ROLLUP(a,b,c)
|
||||
SELECT a, b,c, SUM( d ) FROM tab1 GROUP BY CUBE(a,b,c)
|
||||
```
|
||||
|
||||
`ROLLUP(a,b,c)` 等价于如下`GROUPING SETS` 语句
|
||||
|
||||
```
|
||||
GROUPING SETS (
|
||||
(a,b,c),
|
||||
( a, b ),
|
||||
( a),
|
||||
( )
|
||||
)
|
||||
```
|
||||
|
||||
`CUBE ( a, b, c )` 等价于如下`GROUPING SETS` 语句
|
||||
|
||||
```
|
||||
GROUPING SETS (
|
||||
( a, b, c ),
|
||||
( a, b ),
|
||||
( a, c ),
|
||||
( a ),
|
||||
( b, c ),
|
||||
( b ),
|
||||
( c ),
|
||||
( )
|
||||
)
|
||||
```
|
||||
|
||||
## example
|
||||
|
||||
下面是一个实际数据的例子
|
||||
|
||||
```
|
||||
> SELECT * FROM t;
|
||||
+------+------+------+
|
||||
| k1 | k2 | k3 |
|
||||
+------+------+------+
|
||||
| a | A | 1 |
|
||||
| a | A | 2 |
|
||||
| a | B | 1 |
|
||||
| a | B | 3 |
|
||||
| b | A | 1 |
|
||||
| b | A | 4 |
|
||||
| b | B | 1 |
|
||||
| b | B | 5 |
|
||||
+------+------+------+
|
||||
8 rows in set (0.01 sec)
|
||||
|
||||
> SELECT k1, k2, SUM(k3) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k2), (k1), ( ) );
|
||||
+------+------+-----------+
|
||||
| k1 | k2 | sum(`k3`) |
|
||||
+------+------+-----------+
|
||||
| b | B | 6 |
|
||||
| a | B | 4 |
|
||||
| a | A | 3 |
|
||||
| b | A | 5 |
|
||||
| NULL | B | 10 |
|
||||
| NULL | A | 8 |
|
||||
| a | NULL | 7 |
|
||||
| b | NULL | 11 |
|
||||
| NULL | NULL | 18 |
|
||||
+------+------+-----------+
|
||||
9 rows in set (0.06 sec)
|
||||
|
||||
> SELECT k1, k2, GROUPING_ID(k1,k2), SUM(k3) FROM t GROUP BY GROUPING SETS ((k1, k2), (k1), (k2), ());
|
||||
+------+------+---------------+----------------+
|
||||
| k1 | k2 | grouping_id(k1,k2) | sum(`k3`) |
|
||||
+------+------+---------------+----------------+
|
||||
| a | A | 0 | 3 |
|
||||
| a | B | 0 | 4 |
|
||||
| a | NULL | 1 | 7 |
|
||||
| b | A | 0 | 5 |
|
||||
| b | B | 0 | 6 |
|
||||
| b | NULL | 1 | 11 |
|
||||
| NULL | A | 2 | 8 |
|
||||
| NULL | B | 2 | 10 |
|
||||
| NULL | NULL | 3 | 18 |
|
||||
+------+------+---------------+----------------+
|
||||
9 rows in set (0.02 sec)
|
||||
```
|
||||
|
||||
## keyword
|
||||
|
||||
GROUP, GROUPING, GROUPING_ID, GROUPING_SETS, GROUPING SETS, CUBE, ROLLUP
|
||||
493
docs/documentation/en/internal/grouping_sets_design_EN.md
Normal file
493
docs/documentation/en/internal/grouping_sets_design_EN.md
Normal file
@ -0,0 +1,493 @@
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one
|
||||
or more contributor license agreements. See the NOTICE file
|
||||
distributed with this work for additional information
|
||||
regarding copyright ownership. The ASF licenses this file
|
||||
to you under the Apache License, Version 2.0 (the
|
||||
"License"); you may not use this file except in compliance
|
||||
with the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing,
|
||||
software distributed under the License is distributed on an
|
||||
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
KIND, either express or implied. See the License for the
|
||||
specific language governing permissions and limitations
|
||||
under the License.
|
||||
-->
|
||||
# GROUPING SETS DESIGN
|
||||
|
||||
## 1. GROUPING SETS Background
|
||||
|
||||
The `CUBE`, `ROLLUP`, and `GROUPING` `SETS` extensions to SQL make querying and reporting easier and faster. `CUBE`, `ROLLUP`, and grouping sets produce a single result set that is equivalent to a `UNION` `ALL` of differently grouped rows. `ROLLUP` calculates aggregations such as `SUM`, `COUNT`, `MAX`, `MIN`, and `AVG` at increasing levels of aggregation, from the most detailed up to a grand total. `CUBE` is an extension similar to `ROLLUP`, enabling a single statement to calculate all possible combinations of aggregations. The `CUBE`, `ROLLUP`, and the `GROUPING` `SETS` extension lets you specify just the groupings needed in the `GROUP` `BY` clause. This allows efficient analysis across multiple dimensions without performing a `CUBE` operation. Computing a `CUBE` creates a heavy processing load, so replacing cubes with grouping sets can significantly increase performance.
|
||||
To enhance performance, `CUBE`, `ROLLUP`, and `GROUPING SETS` can be parallelized: multiple processes can simultaneously execute all of these statements. These capabilities make aggregate calculations more efficient, thereby enhancing database performance, and scalability.
|
||||
|
||||
The three `GROUPING` functions help you identify the group each row belongs to and enable sorting subtotal rows and filtering results.
|
||||
|
||||
### 1.1 GROUPING SETS Syntax
|
||||
|
||||
`GROUPING SETS` syntax lets you define multiple groupings in the same query. `GROUP BY` computes all the groupings specified and combines them with `UNION ALL`. For example, consider the following statement:
|
||||
|
||||
```
|
||||
SELECT k1, k2, SUM( k3 ) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k1), (k2), ( ) );
|
||||
```
|
||||
|
||||
|
||||
This statement is equivalent to:
|
||||
|
||||
```
|
||||
SELECT k1, k2, SUM( k3 ) FROM t GROUP BY k1, k2
|
||||
UNION
|
||||
SELECT k1, null, SUM( k3 ) FROM t GROUP BY k1
|
||||
UNION
|
||||
SELECT null, k2, SUM( k3 ) FROM t GROUP BY k2
|
||||
UNION
|
||||
SELECT null, null, SUM( k3 ) FROM t
|
||||
```
|
||||
|
||||
This is an example of real query:
|
||||
|
||||
```
|
||||
mysql> SELECT * FROM t;
|
||||
+------+------+------+
|
||||
| k1 | k2 | k3 |
|
||||
+------+------+------+
|
||||
| a | A | 1 |
|
||||
| a | A | 2 |
|
||||
| a | B | 1 |
|
||||
| a | B | 3 |
|
||||
| b | A | 1 |
|
||||
| b | A | 4 |
|
||||
| b | B | 1 |
|
||||
| b | B | 5 |
|
||||
+------+------+------+
|
||||
8 rows in set (0.01 sec)
|
||||
|
||||
mysql> SELECT k1, k2, SUM(k3) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k2), (k1), ( ) );
|
||||
+------+------+-----------+
|
||||
| k1 | k2 | sum(`k3`) |
|
||||
+------+------+-----------+
|
||||
| b | B | 6 |
|
||||
| a | B | 4 |
|
||||
| a | A | 3 |
|
||||
| b | A | 5 |
|
||||
| NULL | B | 10 |
|
||||
| NULL | A | 8 |
|
||||
| a | NULL | 7 |
|
||||
| b | NULL | 11 |
|
||||
| NULL | NULL | 18 |
|
||||
+------+------+-----------+
|
||||
9 rows in set (0.06 sec)
|
||||
```
|
||||
|
||||
### 1.2 ROLLUP Syntax
|
||||
|
||||
`ROLLUP` enables a `SELECT` statement to calculate multiple levels of subtotals across a specified group of dimensions. It also calculates a grand total. `ROLLUP` is a simple extension to the `GROUP` `BY` clause, so its syntax is extremely easy to use. The `ROLLUP` extension is highly efficient, adding minimal overhead to a query.
|
||||
|
||||
`ROLLUP` appears in the `GROUP` `BY` clause in a `SELECT` statement. Its form is:
|
||||
|
||||
```
|
||||
SELECT a, b,c, SUM( d ) FROM tab1 GROUP BY ROLLUP(a,b,c)
|
||||
```
|
||||
|
||||
This statement is equivalent to GROUPING SETS as followed:
|
||||
|
||||
```
|
||||
GROUPING SETS (
|
||||
(a,b,c),
|
||||
( a, b ),
|
||||
( a),
|
||||
( )
|
||||
)
|
||||
```
|
||||
|
||||
### 1.3 CUBE Syntax
|
||||
|
||||
Like `ROLLUP` `CUBE` generates all the subtotals that could be calculated for a data cube with the specified dimensions.
|
||||
|
||||
```
|
||||
SELECT a, b,c, SUM( d ) FROM tab1 GROUP BY CUBE(a,b,c)
|
||||
```
|
||||
|
||||
e.g. CUBE ( a, b, c ) is equivalent to GROUPING SETS as followed:
|
||||
|
||||
```
|
||||
GROUPING SETS (
|
||||
( a, b, c ),
|
||||
( a, b ),
|
||||
( a, c ),
|
||||
( a ),
|
||||
( b, c ),
|
||||
( b ),
|
||||
( c ),
|
||||
( )
|
||||
)
|
||||
```
|
||||
|
||||
### 1.4 GROUPING and GROUPING_ID Function
|
||||
|
||||
Indicates whether a specified column expression in a `GROUP BY` list is aggregated or not. `GROUPING `returns 1 for aggregated or 0 for not aggregated in the result set. `GROUPING` can be used only in the `SELECT` list, `HAVING`, and `ORDER BY` clauses when `GROUP BY` is specified.
|
||||
|
||||
`GROUPING_ID` describes which of a list of expressions are grouped in a row produced by a `GROUP BY` query. The `GROUPING_ID` function simply returns the decimal equivalent of the binary value formed as a result of the concatenation of the values returned by the `GROUPING` functions.
|
||||
|
||||
Each `GROUPING_ID` argument must be an element of the `GROUP BY` list. `GROUPING_ID ()` returns an **integer** bitmap whose lowest N bits may be lit. A lit **bit** indicates the corresponding argument is not a grouping column for the given output row. The lowest-order **bit** corresponds to argument N, and the N-1th lowest-order **bit** corresponds to argument 1. If the column is a grouping column the bit is 0 else is 1.
|
||||
|
||||
For example:
|
||||
|
||||
```
|
||||
mysql> select * from t;
|
||||
+------+------+------+
|
||||
| k1 | k2 | k3 |
|
||||
+------+------+------+
|
||||
| a | A | 1 |
|
||||
| a | A | 2 |
|
||||
| a | B | 1 |
|
||||
| a | B | 3 |
|
||||
| b | A | 1 |
|
||||
| b | A | 4 |
|
||||
| b | B | 1 |
|
||||
| b | B | 5 |
|
||||
+------+------+------+
|
||||
```
|
||||
|
||||
grouping sets result:
|
||||
|
||||
```
|
||||
mysql> SELECT k1, k2, GROUPING(k1), GROUPING(k2), SUM(k3) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k2), (k1), ( ) );
|
||||
+------+------+----------------+----------------+-----------+
|
||||
| k1 | k2 | grouping(`k1`) | grouping(`k2`) | sum(`k3`) |
|
||||
+------+------+----------------+----------------+-----------+
|
||||
| a | A | 0 | 0 | 3 |
|
||||
| a | B | 0 | 0 | 4 |
|
||||
| a | NULL | 0 | 1 | 7 |
|
||||
| b | A | 0 | 0 | 5 |
|
||||
| b | B | 0 | 0 | 6 |
|
||||
| b | NULL | 0 | 1 | 11 |
|
||||
| NULL | A | 1 | 0 | 8 |
|
||||
| NULL | B | 1 | 0 | 10 |
|
||||
| NULL | NULL | 1 | 1 | 18 |
|
||||
+------+------+----------------+----------------+-----------+
|
||||
9 rows in set (0.02 sec)
|
||||
|
||||
mysql> SELECT k1, k2, GROUPING_ID(k1,k2), SUM(k3) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k2), (k1), ( ) );
|
||||
+------+------+-------------------------+-----------+
|
||||
| k1 | k2 | grouping_id(`k1`, `k2`) | sum(`k3`) |
|
||||
+------+------+-------------------------+-----------+
|
||||
| a | A | 0 | 3 |
|
||||
| a | B | 0 | 4 |
|
||||
| a | NULL | 1 | 7 |
|
||||
| b | A | 0 | 5 |
|
||||
| b | B | 0 | 6 |
|
||||
| b | NULL | 1 | 11 |
|
||||
| NULL | A | 2 | 8 |
|
||||
| NULL | B | 2 | 10 |
|
||||
| NULL | NULL | 3 | 18 |
|
||||
+------+------+-------------------------+-----------+
|
||||
9 rows in set (0.02 sec)
|
||||
|
||||
mysql> SELECT k1, k2, grouping(k1), grouping(k2), GROUPING_ID(k1,k2), SUM(k4) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k2), (k1), ( ) ) order by k1, k2;
|
||||
+------+------+----------------+----------------+-------------------------+-----------+
|
||||
| k1 | k2 | grouping(`k1`) | grouping(`k2`) | grouping_id(`k1`, `k2`) | sum(`k4`) |
|
||||
+------+------+----------------+----------------+-------------------------+-----------+
|
||||
| a | A | 0 | 0 | 0 | 3 |
|
||||
| a | B | 0 | 0 | 0 | 4 |
|
||||
| a | NULL | 0 | 1 | 1 | 7 |
|
||||
| b | A | 0 | 0 | 0 | 5 |
|
||||
| b | B | 0 | 0 | 0 | 6 |
|
||||
| b | NULL | 0 | 1 | 1 | 11 |
|
||||
| NULL | A | 1 | 0 | 2 | 8 |
|
||||
| NULL | B | 1 | 0 | 2 | 10 |
|
||||
| NULL | NULL | 1 | 1 | 3 | 18 |
|
||||
+------+------+----------------+----------------+-------------------------+-----------+
|
||||
9 rows in set (0.02 sec)
|
||||
|
||||
```
|
||||
### 1.5 Composition and nesting of GROUPING SETS
|
||||
|
||||
First of all, a GROUP BY clause is essentially a special case of GROUPING SETS, for example:
|
||||
|
||||
```
|
||||
GROUP BY a
|
||||
is equivalent to:
|
||||
GROUP BY GROUPING SETS((a))
|
||||
also,
|
||||
GROUP BY a,b,c
|
||||
is equivalent to:
|
||||
GROUP BY GROUPING SETS((a,b,c))
|
||||
```
|
||||
|
||||
Similarly, CUBE and ROLLUP can be expanded into GROUPING SETS, so the various combinations and nesting of GROUP BY, CUBE, ROLLUP, GROUPING SETS are essentially the combination and nesting of GROUPING SETS.
|
||||
|
||||
For GROUPING SETS nesting, it is semantically equivalent to writing the statements inside the nest directly outside. (ref:<https://www.brytlyt.com/documentation/data-manipulation-dml/grouping-sets-rollup-cube/>) mentions:
|
||||
|
||||
```
|
||||
The CUBE and ROLLUP constructs can be used either directly in the GROUP BY clause, or nested inside a GROUPING SETS clause. If one GROUPING SETS clause is nested inside another, the effect is the same as if all the elements of the inner clause had been written directly in the outer clause.
|
||||
```
|
||||
|
||||
For a combined list of multiple GROUPING SETS, many databases consider it a cross product relationship.
|
||||
|
||||
for example:
|
||||
|
||||
```
|
||||
GROUP BY a, CUBE (b, c), GROUPING SETS ((d), (e))
|
||||
|
||||
is equivalent to:
|
||||
|
||||
GROUP BY GROUPING SETS (
|
||||
(a, b, c, d), (a, b, c, e),
|
||||
(a, b, d), (a, b, e),
|
||||
(a, c, d), (a, c, e),
|
||||
(a, d), (a, e)
|
||||
)
|
||||
```
|
||||
|
||||
For the combination and nesting of GROUPING SETS, each database support is not the same. For example snowflake does not support any combination and nesting.
|
||||
(<https://docs.snowflake.net/manuals/sql-reference/constructs/group-by.html>)
|
||||
|
||||
Oracle supports both composition and nesting.
|
||||
(<https://docs.oracle.com/cd/B19306_01/server.102/b14223/aggreg.htm#i1006842>)
|
||||
|
||||
Presto supports composition, but not nesting.
|
||||
(<https://prestodb.github.io/docs/current/sql/select.html>)
|
||||
|
||||
## 2. Object
|
||||
|
||||
Support `GROUPING SETS`, `ROLLUP` and `CUBE ` syntax,impliments 1.1, 1.2, 1.3 1.4, 1.5
|
||||
|
||||
### 2.1 GROUPING SETS Syntax
|
||||
|
||||
```
|
||||
SELECT ...
|
||||
FROM ...
|
||||
[ ... ]
|
||||
GROUP BY GROUPING SETS ( groupSet [ , groupSet [ , ... ] ] )
|
||||
[ ... ]
|
||||
|
||||
groupSet ::= { ( expr [ , expr [ , ... ] ] )}
|
||||
|
||||
<expr>
|
||||
Expression,column name.
|
||||
```
|
||||
|
||||
### 2.2 ROLLUP Syntax
|
||||
|
||||
```
|
||||
SELECT ...
|
||||
FROM ...
|
||||
[ ... ]
|
||||
GROUP BY ROLLUP ( expr [ , expr [ , ... ] ] )
|
||||
[ ... ]
|
||||
|
||||
<expr>
|
||||
Expression,column name.
|
||||
```
|
||||
|
||||
### 2.3 CUBE Syntax
|
||||
|
||||
```
|
||||
SELECT ...
|
||||
FROM ...
|
||||
[ ... ]
|
||||
GROUP BY CUBE ( expr [ , expr [ , ... ] ] )
|
||||
[ ... ]
|
||||
|
||||
<expr>
|
||||
Expression,column name.
|
||||
```
|
||||
|
||||
## 3. Implementation
|
||||
|
||||
### 3.1 Overall Design Approaches
|
||||
|
||||
For `GROUPING SET` is equivalent to the `UNION` of `GROUP BY` . So we can expand input rows, and run an GROUP BY on these rows。
|
||||
|
||||
For example:
|
||||
|
||||
```
|
||||
SELECT a, b FROM src GROUP BY a, b GROUPING SETS ((a, b), (a), (b), ());
|
||||
```
|
||||
|
||||
Data in table src :
|
||||
|
||||
```
|
||||
1, 2
|
||||
3, 4
|
||||
```
|
||||
|
||||
Base on GROUPING SETS , we can expend the input to:
|
||||
|
||||
```
|
||||
1, 2 (GROUPING_ID: a, b -> 00 -> 0)
|
||||
1, null (GROUPING_ID: a, null -> 01 -> 1)
|
||||
null, 2 (GROUPING_ID: null, b -> 10 -> 2)
|
||||
null, null (GROUPING_ID: null, null -> 11 -> 3)
|
||||
|
||||
3, 4 (GROUPING_ID: a, b -> 00 -> 0)
|
||||
3, null (GROUPING_ID: a, null -> 01 -> 1)
|
||||
null, 4 (GROUPING_ID: null, b -> 10 -> 2)
|
||||
null, null (GROUPING_ID: null, null -> 11 -> 3)
|
||||
```
|
||||
|
||||
And then use those row as input, then GROUP BY a, b, GROUPING_ID
|
||||
|
||||
### 3.2 Example
|
||||
|
||||
Table t:
|
||||
|
||||
```
|
||||
mysql> select * from t;
|
||||
+------+------+------+
|
||||
| k1 | k2 | k3 |
|
||||
+------+------+------+
|
||||
| a | A | 1 |
|
||||
| a | A | 2 |
|
||||
| a | B | 1 |
|
||||
| a | B | 3 |
|
||||
| b | A | 1 |
|
||||
| b | A | 4 |
|
||||
| b | B | 1 |
|
||||
| b | B | 5 |
|
||||
+------+------+------+
|
||||
8 rows in set (0.01 sec)
|
||||
```
|
||||
|
||||
for the query:
|
||||
|
||||
```
|
||||
SELECT k1, k2, GROUPING_ID(k1,k2), SUM(k3) FROM t GROUP BY GROUPING SETS ((k1, k2), (k1), (k2), ());
|
||||
```
|
||||
|
||||
First,expand the input,every row expand into 4 rows ( the size of GROUPING SETS), and insert GROUPING_ID column
|
||||
|
||||
e.g. a, A, 1 expanded to:
|
||||
|
||||
```
|
||||
+------+------+------+-------------------------+
|
||||
| k1 | k2 | k3 | GROUPING_ID(`k1`, `k2`) |
|
||||
+------+------+------+-------------------------+
|
||||
| a | A | 1 | 0 |
|
||||
| a | NULL | 1 | 1 |
|
||||
| NULL | A | 1 | 2 |
|
||||
| NULL | NULL | 1 | 3 |
|
||||
+------+------+------+-------------------------+
|
||||
```
|
||||
|
||||
Finally, all rows expended as follows (32 rows):
|
||||
|
||||
```
|
||||
+------+------+------+-------------------------+
|
||||
| k1 | k2 | k3 | GROUPING_ID(`k1`, `k2`) |
|
||||
+------+------+------+-------------------------+
|
||||
| a | A | 1 | 0 |
|
||||
| a | A | 2 | 0 |
|
||||
| a | B | 1 | 0 |
|
||||
| a | B | 3 | 0 |
|
||||
| b | A | 1 | 0 |
|
||||
| b | A | 4 | 0 |
|
||||
| b | B | 1 | 0 |
|
||||
| b | B | 5 | 0 |
|
||||
| a | NULL | 1 | 1 |
|
||||
| a | NULL | 1 | 1 |
|
||||
| a | NULL | 2 | 1 |
|
||||
| a | NULL | 3 | 1 |
|
||||
| b | NULL | 1 | 1 |
|
||||
| b | NULL | 1 | 1 |
|
||||
| b | NULL | 4 | 1 |
|
||||
| b | NULL | 5 | 1 |
|
||||
| NULL | A | 1 | 2 |
|
||||
| NULL | A | 1 | 2 |
|
||||
| NULL | A | 2 | 2 |
|
||||
| NULL | A | 4 | 2 |
|
||||
| NULL | B | 1 | 2 |
|
||||
| NULL | B | 1 | 2 |
|
||||
| NULL | B | 3 | 2 |
|
||||
| NULL | B | 5 | 2 |
|
||||
| NULL | NULL | 1 | 3 |
|
||||
| NULL | NULL | 1 | 3 |
|
||||
| NULL | NULL | 1 | 3 |
|
||||
| NULL | NULL | 1 | 3 |
|
||||
| NULL | NULL | 2 | 3 |
|
||||
| NULL | NULL | 3 | 3 |
|
||||
| NULL | NULL | 4 | 3 |
|
||||
| NULL | NULL | 5 | 3 |
|
||||
+------+------+------+-------------------------+
|
||||
32 rows in set.
|
||||
```
|
||||
|
||||
now GROUP BY k1, k2, GROUPING_ID(k1,k2):
|
||||
|
||||
```
|
||||
+------+------+-------------------------+-----------+
|
||||
| k1 | k2 | grouping_id(`k1`, `k2`) | sum(`k3`) |
|
||||
+------+------+-------------------------+-----------+
|
||||
| a | A | 0 | 3 |
|
||||
| a | B | 0 | 4 |
|
||||
| a | NULL | 1 | 7 |
|
||||
| b | A | 0 | 5 |
|
||||
| b | B | 0 | 6 |
|
||||
| b | NULL | 1 | 11 |
|
||||
| NULL | A | 2 | 8 |
|
||||
| NULL | B | 2 | 10 |
|
||||
| NULL | NULL | 3 | 18 |
|
||||
+------+------+-------------------------+-----------+
|
||||
9 rows in set (0.02 sec)
|
||||
```
|
||||
|
||||
The result is equivalent to the UNION ALL
|
||||
|
||||
```
|
||||
select k1, k2, sum(k3) from t group by k1, k2
|
||||
UNION ALL
|
||||
select NULL, k2, sum(k3) from t group by k2
|
||||
UNION ALL
|
||||
select k1, NULL, sum(k3) from t group by k1
|
||||
UNION ALL
|
||||
select NULL, NULL, sum(k3) from t;
|
||||
|
||||
+------+------+-----------+
|
||||
| k1 | k2 | sum(`k3`) |
|
||||
+------+------+-----------+
|
||||
| b | B | 6 |
|
||||
| b | A | 5 |
|
||||
| a | A | 3 |
|
||||
| a | B | 4 |
|
||||
| a | NULL | 7 |
|
||||
| b | NULL | 11 |
|
||||
| NULL | B | 10 |
|
||||
| NULL | A | 8 |
|
||||
| NULL | NULL | 18 |
|
||||
+------+------+-----------+
|
||||
9 rows in set (0.06 sec)
|
||||
```
|
||||
|
||||
### 3.3 FE
|
||||
|
||||
#### 3.3.1 Tasks
|
||||
|
||||
1. Add GroupByClause, repalce groupingExprs.
|
||||
2. Add Grouping Sets, Cube and RollUp syntax.
|
||||
3. Add GroupByClause in SelectStmt.
|
||||
4. Add GroupingFunctionCallExpr, impliments grouping grouping_id function call
|
||||
5. Add VirtualSlot, generate the map of virtual slots and real slots
|
||||
6. add virtual column GROUPING_ID and other virtual columns generated by grouping and grouping_id, insert into groupingExprs,
|
||||
7. Add a PlanNode, name as RepeatNode. For GroupingSets aggregation insert RepeatNode to the plan.
|
||||
|
||||
#### 3.3.2 Tuple
|
||||
|
||||
In order to add GROUPING_ID to groupingExprs in GroupByClause, need to create virtual SlotRef, also, need tot create a tuple for this slot, named GROUPING\_\_ID Tuple.
|
||||
|
||||
For the plannode RepeatNode, it's input is all the tuple of it's children, It's output tuple is the repeat data and GROUPING_ID.
|
||||
|
||||
|
||||
#### 3.3.3 Expression and Function Substitution
|
||||
|
||||
expr -> if(bitand(pos, grouping_id)=0, expr, null) for expr in extension grouping clause
|
||||
grouping_id() -> grouping_id(grouping_id) for grouping_id function
|
||||
|
||||
### 3.4 BE
|
||||
|
||||
#### 3.4.1 Tasks
|
||||
|
||||
1. Add RepeatNode executor, expend the input data and append GROUPING_ID to every row
|
||||
2. Implements grouping_id() and grouping() function.
|
||||
@ -0,0 +1,161 @@
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one
|
||||
or more contributor license agreements. See the NOTICE file
|
||||
distributed with this work for additional information
|
||||
regarding copyright ownership. The ASF licenses this file
|
||||
to you under the Apache License, Version 2.0 (the
|
||||
"License"); you may not use this file except in compliance
|
||||
with the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing,
|
||||
software distributed under the License is distributed on an
|
||||
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
KIND, either express or implied. See the License for the
|
||||
specific language governing permissions and limitations
|
||||
under the License.
|
||||
-->
|
||||
|
||||
# GROUP BY
|
||||
|
||||
## description
|
||||
|
||||
GROUP BY `GROUPING SETS` | `CUBE` | `ROLLUP` is an extension to GROUP BY clause. This syntax lets you define multiple groupings in the same query. GROUPING SETS produce a single result set that is equivalent to a UNION ALL of differently grouped rows
|
||||
For example GROUPING SETS clause:
|
||||
|
||||
```
|
||||
SELECT a, b, SUM( c ) FROM tab1 GROUP BY GROUPING SETS ( (a, b), (a), (b), ( ) );
|
||||
```
|
||||
|
||||
This statement is equivalent to:
|
||||
|
||||
```
|
||||
SELECT a, b, SUM( c ) FROM tab1 GROUP BY a, b
|
||||
UNION
|
||||
SELECT a, null, SUM( c ) FROM tab1 GROUP BY a
|
||||
UNION
|
||||
SELECT null, b, SUM( c ) FROM tab1 GROUP BY b
|
||||
UNION
|
||||
SELECT null, null, SUM( c ) FROM tab1
|
||||
```
|
||||
|
||||
`GROUPING(expr)` indicates whether a specified column expression in a GROUP BY list is aggregated or not. GROUPING returns 1 for aggregated or 0 for not aggregated in the result set.
|
||||
|
||||
`GROUPING_ID(expr [ , expr [ , ... ] ])` describes which of a list of expressions are grouped in a row produced by a GROUP BY query. The GROUPING_ID function simply returns the decimal equivalent of the binary value formed as a result of the concatenation of the values returned by the GROUPING functions.
|
||||
|
||||
### Syntax
|
||||
|
||||
```
|
||||
SELECT ...
|
||||
FROM ...
|
||||
[ ... ]
|
||||
GROUP BY [
|
||||
, ... |
|
||||
GROUPING SETS [, ...] ( groupSet [ , groupSet [ , ... ] ] ) |
|
||||
ROLLUP(expr [ , expr [ , ... ] ]) |
|
||||
expr [ , expr [ , ... ] ] WITH ROLLUP |
|
||||
CUBE(expr [ , expr [ , ... ] ]) |
|
||||
expr [ , expr [ , ... ] ] WITH CUBE
|
||||
]
|
||||
[ ... ]
|
||||
```
|
||||
|
||||
### Parameters
|
||||
|
||||
`groupSet` is a set of expression or column or it's alias appearing in the query block’s SELECT list. `groupSet ::= { ( expr [ , expr [ , ... ] ] )}`
|
||||
|
||||
`expr` is expression or column or it's alias appearing in the query block’s SELECT list.
|
||||
|
||||
### Note
|
||||
|
||||
doris supports PostgreSQL like syntax, for example:
|
||||
|
||||
```
|
||||
SELECT a, b, SUM( c ) FROM tab1 GROUP BY GROUPING SETS ( (a, b), (a), (b), ( ) );
|
||||
SELECT a, b,c, SUM( d ) FROM tab1 GROUP BY ROLLUP(a,b,c)
|
||||
SELECT a, b,c, SUM( d ) FROM tab1 GROUP BY CUBE(a,b,c)
|
||||
```
|
||||
|
||||
`ROLLUP(a,b,c)` is equivalent to `GROUPING SETS` as follows:
|
||||
|
||||
```
|
||||
GROUPING SETS (
|
||||
(a,b,c),
|
||||
( a, b ),
|
||||
( a),
|
||||
( )
|
||||
)
|
||||
```
|
||||
|
||||
`CUBE ( a, b, c )` is equivalent to `GROUPING SETS` as follows:
|
||||
|
||||
```
|
||||
GROUPING SETS (
|
||||
( a, b, c ),
|
||||
( a, b ),
|
||||
( a, c ),
|
||||
( a ),
|
||||
( b, c ),
|
||||
( b ),
|
||||
( c ),
|
||||
( )
|
||||
)
|
||||
```
|
||||
|
||||
## example
|
||||
|
||||
This is a simple example
|
||||
|
||||
```
|
||||
> SELECT * FROM t;
|
||||
+------+------+------+
|
||||
| k1 | k2 | k3 |
|
||||
+------+------+------+
|
||||
| a | A | 1 |
|
||||
| a | A | 2 |
|
||||
| a | B | 1 |
|
||||
| a | B | 3 |
|
||||
| b | A | 1 |
|
||||
| b | A | 4 |
|
||||
| b | B | 1 |
|
||||
| b | B | 5 |
|
||||
+------+------+------+
|
||||
8 rows in set (0.01 sec)
|
||||
|
||||
> SELECT k1, k2, SUM(k3) FROM t GROUP BY GROUPING SETS ( (k1, k2), (k2), (k1), ( ) );
|
||||
+------+------+-----------+
|
||||
| k1 | k2 | sum(`k3`) |
|
||||
+------+------+-----------+
|
||||
| b | B | 6 |
|
||||
| a | B | 4 |
|
||||
| a | A | 3 |
|
||||
| b | A | 5 |
|
||||
| NULL | B | 10 |
|
||||
| NULL | A | 8 |
|
||||
| a | NULL | 7 |
|
||||
| b | NULL | 11 |
|
||||
| NULL | NULL | 18 |
|
||||
+------+------+-----------+
|
||||
9 rows in set (0.06 sec)
|
||||
|
||||
> SELECT k1, k2, GROUPING_ID(k1,k2), SUM(k3) FROM t GROUP BY GROUPING SETS ((k1, k2), (k1), (k2), ());
|
||||
+------+------+---------------+----------------+
|
||||
| k1 | k2 | grouping_id(k1,k2) | sum(`k3`) |
|
||||
+------+------+---------------+----------------+
|
||||
| a | A | 0 | 3 |
|
||||
| a | B | 0 | 4 |
|
||||
| a | NULL | 1 | 7 |
|
||||
| b | A | 0 | 5 |
|
||||
| b | B | 0 | 6 |
|
||||
| b | NULL | 1 | 11 |
|
||||
| NULL | A | 2 | 8 |
|
||||
| NULL | B | 2 | 10 |
|
||||
| NULL | NULL | 3 | 18 |
|
||||
+------+------+---------------+----------------+
|
||||
9 rows in set (0.02 sec)
|
||||
```
|
||||
|
||||
## keyword
|
||||
|
||||
GROUP, GROUPING, GROUPING_ID, GROUPING_SETS, GROUPING SETS, CUBE, ROLLUP
|
||||
@ -17,28 +17,31 @@
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import org.apache.doris.common.Version;
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
import java.math.BigDecimal;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.doris.analysis.UnionStmt.Qualifier;
|
||||
import org.apache.doris.analysis.UnionStmt.UnionOperand;
|
||||
import org.apache.doris.catalog.AccessPrivilege;
|
||||
import org.apache.doris.catalog.AggregateType;
|
||||
import org.apache.doris.catalog.Column;
|
||||
import org.apache.doris.catalog.KeysType;
|
||||
import org.apache.doris.catalog.PrimitiveType;
|
||||
import org.apache.doris.catalog.ScalarType;
|
||||
import org.apache.doris.catalog.Type;
|
||||
import org.apache.doris.catalog.View;
|
||||
import org.apache.doris.catalog.AggregateType;
|
||||
import org.apache.doris.analysis.PartitionKeyDesc;
|
||||
import org.apache.doris.analysis.UnionStmt.UnionOperand;
|
||||
import org.apache.doris.analysis.UnionStmt.Qualifier;
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
import org.apache.doris.common.Version;
|
||||
import org.apache.doris.mysql.MysqlPassword;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.math.BigDecimal;
|
||||
import java_cup.runtime.Symbol;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
import java_cup.runtime.Symbol;
|
||||
|
||||
// Commented by Zhao Chun
|
||||
// Now we have 2 shift/reduce conflict
|
||||
// between TIMESTAMP "20100101" and TIMESTAMP "alias"
|
||||
@ -194,35 +197,34 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_A
|
||||
KW_BACKEND, KW_BACKUP, KW_BETWEEN, KW_BEGIN, KW_BIGINT, KW_BITMAP, KW_BITMAP_UNION, KW_BOOLEAN, KW_BOTH, KW_BROKER, KW_BACKENDS, KW_BY,
|
||||
KW_CANCEL, KW_CASE, KW_CAST, KW_CHAIN, KW_CHAR, KW_CHARSET, KW_CLUSTER, KW_CLUSTERS,
|
||||
KW_COLLATE, KW_COLLATION, KW_COLUMN, KW_COLUMNS, KW_COMMENT, KW_COMMIT, KW_COMMITTED,
|
||||
KW_CONFIG, KW_CONNECTION, KW_CONNECTION_ID, KW_CONSISTENT, KW_COUNT, KW_CREATE, KW_CROSS, KW_CURRENT, KW_CURRENT_USER,
|
||||
KW_CONFIG, KW_CONNECTION, KW_CONNECTION_ID, KW_CONSISTENT, KW_COUNT, KW_CREATE, KW_CROSS, KW_CUBE, KW_CURRENT, KW_CURRENT_USER,
|
||||
KW_DATA, KW_DATABASE, KW_DATABASES, KW_DATE, KW_DATETIME, KW_DAY, KW_DECIMAL, KW_DECOMMISSION, KW_DEFAULT, KW_DESC, KW_DESCRIBE,
|
||||
KW_DELETE, KW_DISTINCT, KW_DISTINCTPC, KW_DISTINCTPCSA, KW_DISTRIBUTED, KW_DISTRIBUTION, KW_DYNAMIC, KW_BUCKETS, KW_DIV, KW_DOUBLE, KW_DROP, KW_DROPP, KW_DUPLICATE,
|
||||
KW_ELSE, KW_END, KW_ENGINE, KW_ENGINES, KW_ENTER, KW_ERRORS, KW_EVENTS, KW_EXISTS, KW_EXPORT, KW_EXTERNAL, KW_EXTRACT,
|
||||
KW_FALSE, KW_FOLLOWER, KW_FOLLOWING, KW_FREE, KW_FROM, KW_FILE, KW_FIRST, KW_FLOAT, KW_FOR, KW_FORMAT, KW_FRONTEND, KW_FRONTENDS, KW_FULL, KW_FUNCTION,
|
||||
KW_GLOBAL, KW_GRANT, KW_GRANTS, KW_GROUP,
|
||||
KW_GLOBAL, KW_GRANT, KW_GRANTS, KW_GROUP, KW_GROUPING,
|
||||
KW_HASH, KW_HAVING, KW_HELP,KW_HLL, KW_HLL_UNION, KW_HOUR, KW_HUB,
|
||||
KW_IDENTIFIED, KW_IF, KW_IN, KW_INDEX, KW_INDEXES, KW_INFILE,
|
||||
KW_INNER, KW_INSERT, KW_INT, KW_INTERMEDIATE, KW_INTERVAL, KW_INTO, KW_IS, KW_ISNULL, KW_ISOLATION,
|
||||
KW_JOIN,
|
||||
KW_KEY, KW_KILL,
|
||||
KW_LABEL, KW_LARGEINT, KW_LAST, KW_LEFT, KW_LESS, KW_LEVEL, KW_LIKE, KW_LIMIT, KW_LINK, KW_LOAD,
|
||||
KW_ROUTINE, KW_PAUSE, KW_RESUME, KW_STOP, KW_TASK,
|
||||
KW_LOCAL, KW_LOCATION,
|
||||
KW_MAX, KW_MAX_VALUE, KW_MERGE, KW_MIN, KW_MINUTE, KW_MIGRATE, KW_MIGRATIONS, KW_MODIFY, KW_MONTH,
|
||||
KW_NAME, KW_NAMES, KW_NEGATIVE, KW_NO, KW_NOT, KW_NULL, KW_NULLS,
|
||||
KW_OBSERVER, KW_OFFSET, KW_ON, KW_ONLY, KW_OPEN, KW_OR, KW_PIPE, KW_ORDER, KW_OUTER, KW_OVER,
|
||||
KW_PARTITION, KW_PARTITIONS, KW_PATH, KW_PRECEDING,
|
||||
KW_PASSWORD, KW_PLUGIN, KW_PLUGINS,
|
||||
KW_OBSERVER, KW_OFFSET, KW_ON, KW_ONLY, KW_OPEN, KW_OR, KW_ORDER, KW_OUTER, KW_OVER,
|
||||
KW_PARTITION, KW_PARTITIONS, KW_PASSWORD, KW_PATH, KW_PAUSE, KW_PIPE, KW_PRECEDING,
|
||||
KW_PLUGIN, KW_PLUGINS,
|
||||
KW_PRIMARY,
|
||||
KW_PROC, KW_PROCEDURE, KW_PROCESSLIST, KW_PROPERTIES, KW_PROPERTY,
|
||||
KW_QUERY, KW_QUOTA,
|
||||
KW_RANDOM, KW_RANGE, KW_READ, KW_RECOVER, KW_REGEXP, KW_RELEASE, KW_RENAME,
|
||||
KW_REPAIR, KW_REPEATABLE, KW_REPOSITORY, KW_REPOSITORIES, KW_REPLACE, KW_REPLACE_IF_NOT_NULL, KW_REPLICA, KW_RESOURCE, KW_RESTORE, KW_RETURNS, KW_REVOKE,
|
||||
KW_RIGHT, KW_ROLE, KW_ROLES, KW_ROLLBACK, KW_ROLLUP, KW_ROW, KW_ROWS,
|
||||
KW_SCHEMAS, KW_SECOND, KW_SELECT, KW_SEMI, KW_SERIALIZABLE, KW_SESSION, KW_SET, KW_SHOW,
|
||||
KW_SMALLINT, KW_SNAPSHOT, KW_SONAME, KW_SPLIT, KW_START, KW_STATUS, KW_STORAGE, KW_STRING,
|
||||
KW_REPAIR, KW_REPEATABLE, KW_REPOSITORY, KW_REPOSITORIES, KW_REPLACE, KW_REPLACE_IF_NOT_NULL, KW_REPLICA, KW_RESOURCE, KW_RESTORE, KW_RETURNS, KW_RESUME, KW_REVOKE,
|
||||
KW_RIGHT, KW_ROLE, KW_ROLES, KW_ROLLBACK, KW_ROLLUP, KW_ROUTINE, KW_ROW, KW_ROWS,
|
||||
KW_SCHEMAS, KW_SECOND, KW_SELECT, KW_SEMI, KW_SERIALIZABLE, KW_SESSION, KW_SET, KW_SETS, KW_SHOW,
|
||||
KW_SMALLINT, KW_SNAPSHOT, KW_SONAME, KW_SPLIT, KW_START, KW_STATUS, KW_STOP, KW_STORAGE, KW_STRING,
|
||||
KW_SUM, KW_SUPERUSER, KW_SYNC, KW_SYSTEM,
|
||||
KW_TABLE, KW_TABLES, KW_TABLET, KW_TERMINATED, KW_THAN, KW_THEN, KW_TIME, KW_TIMESTAMP, KW_TINYINT,
|
||||
KW_TABLE, KW_TABLES, KW_TABLET, KW_TASK, KW_TERMINATED, KW_THAN, KW_THEN, KW_TIME, KW_TIMESTAMP, KW_TINYINT,
|
||||
KW_TO, KW_TRANSACTION, KW_TRIGGERS, KW_TRIM, KW_TRUE, KW_TRUNCATE, KW_TYPE, KW_TYPES,
|
||||
KW_UNCOMMITTED, KW_UNBOUNDED, KW_UNION, KW_UNIQUE, KW_UNSIGNED, KW_USE, KW_USER, KW_USING,
|
||||
KW_VALUE, KW_VALUES, KW_VARCHAR, KW_VARIABLES, KW_VIEW, KW_MATERIALIZED,
|
||||
@ -306,7 +308,7 @@ nonterminal Expr where_clause;
|
||||
nonterminal Expr where_clause_without_null;
|
||||
nonterminal Predicate predicate, between_predicate, comparison_predicate,
|
||||
compound_predicate, in_predicate, like_predicate, exists_predicate;
|
||||
nonterminal ArrayList<Expr> group_by_clause, opt_partition_by_clause;
|
||||
nonterminal ArrayList<Expr> opt_partition_by_clause;
|
||||
nonterminal Expr having_clause;
|
||||
nonterminal ArrayList<OrderByElement> order_by_elements, order_by_clause;
|
||||
nonterminal OrderByElement order_by_element;
|
||||
@ -411,6 +413,10 @@ nonterminal ColumnPosition opt_col_pos;
|
||||
nonterminal AlterClause alter_system_clause, alter_cluster_clause, alter_table_clause;
|
||||
nonterminal List<AlterClause> alter_table_clause_list;
|
||||
|
||||
// grouping sets
|
||||
nonterminal List<ArrayList<Expr>> grouping_set_list;
|
||||
nonterminal ArrayList<Expr> grouping_set;
|
||||
nonterminal GroupByClause group_by_clause, grouping_elements;
|
||||
//
|
||||
nonterminal String keyword, ident, ident_or_text, variable_name, text_or_password,
|
||||
charset_name_or_default, old_or_new_charset_name_or_default, opt_collate,
|
||||
@ -2899,13 +2905,13 @@ select_stmt ::=
|
||||
| select_clause:selectList
|
||||
from_clause:fromClause
|
||||
where_clause:wherePredicate
|
||||
group_by_clause:groupingExprs
|
||||
group_by_clause:groupByClause
|
||||
having_clause:havingPredicate
|
||||
order_by_clause:orderByClause
|
||||
limit_clause:limitClause
|
||||
{:
|
||||
RESULT = new SelectStmt(selectList, fromClause, wherePredicate,
|
||||
groupingExprs, havingPredicate, orderByClause,
|
||||
groupByClause, havingPredicate, orderByClause,
|
||||
limitClause);
|
||||
:}
|
||||
| value_clause:valueClause order_by_clause:orderByClause limit_clause:limitClause
|
||||
@ -3327,9 +3333,52 @@ where_clause_without_null ::=
|
||||
{: RESULT = e; :}
|
||||
;
|
||||
|
||||
group_by_clause ::=
|
||||
KW_GROUP KW_BY expr_list:l
|
||||
grouping_set ::=
|
||||
LPAREN RPAREN
|
||||
{:
|
||||
ArrayList<Expr> list = Lists.newArrayList();
|
||||
RESULT = list;
|
||||
:}
|
||||
| LPAREN expr_list:l RPAREN
|
||||
{: RESULT = l; :}
|
||||
;
|
||||
|
||||
grouping_set_list ::=
|
||||
grouping_set:l
|
||||
{:
|
||||
List<ArrayList<Expr>> list = Lists.newArrayList();
|
||||
list.add(l);
|
||||
RESULT = list;
|
||||
:}
|
||||
| grouping_set_list:list COMMA grouping_set:l
|
||||
{:
|
||||
list.add(l);
|
||||
RESULT = list;
|
||||
:}
|
||||
;
|
||||
|
||||
grouping_elements ::=
|
||||
expr_list:l
|
||||
{:
|
||||
RESULT = new GroupByClause(l, GroupByClause.GroupingType.GROUP_BY);
|
||||
:}
|
||||
| KW_GROUPING KW_SETS LPAREN grouping_set_list:ls RPAREN
|
||||
{:
|
||||
RESULT = new GroupByClause(ls, GroupByClause.GroupingType.GROUPING_SETS);
|
||||
:}
|
||||
| KW_CUBE LPAREN expr_list:l RPAREN
|
||||
{:
|
||||
RESULT = new GroupByClause(l, GroupByClause.GroupingType.CUBE);
|
||||
:}
|
||||
| KW_ROLLUP LPAREN expr_list:l RPAREN
|
||||
{:
|
||||
RESULT = new GroupByClause(l, GroupByClause.GroupingType.ROLLUP);
|
||||
:}
|
||||
;
|
||||
|
||||
group_by_clause ::=
|
||||
KW_GROUP KW_BY grouping_elements:e
|
||||
{: RESULT = e; :}
|
||||
| /* empty */
|
||||
{: RESULT = null; :}
|
||||
;
|
||||
@ -3555,7 +3604,18 @@ function_call_expr ::=
|
||||
function_name:fn_name LPAREN RPAREN
|
||||
{: RESULT = new FunctionCallExpr(fn_name, new ArrayList<Expr>()); :}
|
||||
| function_name:fn_name LPAREN function_params:params RPAREN
|
||||
{: RESULT = new FunctionCallExpr(fn_name, params); :}
|
||||
{:
|
||||
if ("grouping".equalsIgnoreCase(fn_name.getFunction())) {
|
||||
if (params.exprs().size() > 1) {
|
||||
throw new AnalysisException("GROUPING requires exactly one column parameter.");
|
||||
}
|
||||
RESULT = new GroupingFunctionCallExpr(fn_name, params);
|
||||
} else if ("grouping_id".equalsIgnoreCase(fn_name.getFunction())) {
|
||||
RESULT = new GroupingFunctionCallExpr(fn_name, params);
|
||||
} else {
|
||||
RESULT = new FunctionCallExpr(fn_name, params);
|
||||
}
|
||||
:}
|
||||
;
|
||||
|
||||
exists_predicate ::=
|
||||
@ -4355,6 +4415,8 @@ keyword ::=
|
||||
{: RESULT = id; :}
|
||||
| KW_STOP:id
|
||||
{: RESULT = id; :}
|
||||
| KW_GROUPING:id
|
||||
{: RESULT = id; :}
|
||||
| KW_DYNAMIC:id
|
||||
{: RESULT = id; :}
|
||||
| KW_YEAR:id
|
||||
|
||||
@ -17,6 +17,18 @@
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Calendar;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.doris.catalog.Catalog;
|
||||
import org.apache.doris.catalog.Column;
|
||||
import org.apache.doris.catalog.Database;
|
||||
@ -41,6 +53,8 @@ import org.apache.doris.rewrite.ExprRewriter;
|
||||
import org.apache.doris.rewrite.FoldConstantsRule;
|
||||
import org.apache.doris.rewrite.NormalizeBinaryPredicatesRule;
|
||||
import org.apache.doris.thrift.TQueryGlobals;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
@ -51,23 +65,6 @@ import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Multimap;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Calendar;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
//import org.apache.doris.catalog.InlineView;
|
||||
|
||||
/**
|
||||
* Repository of analysis state for single select block.
|
||||
* <p/>
|
||||
@ -589,6 +586,28 @@ public class Analyzer {
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Register a virtual column, and it is not a real column exist in table,
|
||||
* so it does not need to resolve.
|
||||
*/
|
||||
public SlotDescriptor registerVirtualColumnRef(String colName, Type type, TupleDescriptor tupleDescriptor)
|
||||
throws AnalysisException {
|
||||
// Make column name case insensitive
|
||||
String key = colName;
|
||||
SlotDescriptor result = slotRefMap.get(key);
|
||||
if (result != null) {
|
||||
result.setMultiRef(true);
|
||||
return result;
|
||||
}
|
||||
|
||||
result = addSlotDescriptor(tupleDescriptor);
|
||||
Column col = new Column(colName, type);
|
||||
result.setColumn(col);
|
||||
result.setIsNullable(true);
|
||||
slotRefMap.put(key, result);
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Resolves column name in context of any of the registered table aliases.
|
||||
* Returns null if not found or multiple bindings to different tables exist,
|
||||
|
||||
@ -17,6 +17,9 @@
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.doris.catalog.Function;
|
||||
import org.apache.doris.catalog.FunctionSet;
|
||||
import org.apache.doris.catalog.PrimitiveType;
|
||||
@ -26,17 +29,15 @@ import org.apache.doris.common.AnalysisException;
|
||||
import org.apache.doris.thrift.TExprNode;
|
||||
import org.apache.doris.thrift.TExprNodeType;
|
||||
import org.apache.doris.thrift.TExprOpcode;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.util.List;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
public class ArithmeticExpr extends Expr {
|
||||
private static final Logger LOG = LogManager.getLogger(ArithmeticExpr.class);
|
||||
|
||||
|
||||
enum OperatorPosition {
|
||||
BINARY_INFIX,
|
||||
UNARY_PREFIX,
|
||||
@ -196,7 +197,7 @@ public class ArithmeticExpr extends Expr {
|
||||
private Type findCommonType(Type t1, Type t2) {
|
||||
PrimitiveType pt1 = t1.getPrimitiveType();
|
||||
PrimitiveType pt2 = t2.getPrimitiveType();
|
||||
|
||||
|
||||
if (pt1 == PrimitiveType.DOUBLE || pt2 == PrimitiveType.DOUBLE) {
|
||||
return Type.DOUBLE;
|
||||
} else if (pt1 == PrimitiveType.DECIMALV2 || pt2 == PrimitiveType.DECIMALV2) {
|
||||
@ -273,4 +274,9 @@ public class ArithmeticExpr extends Expr {
|
||||
"No match for '%s' with operand types %s and %s", toSql(), t1, t2));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Objects.hashCode(op);
|
||||
}
|
||||
}
|
||||
|
||||
@ -19,7 +19,6 @@ package org.apache.doris.analysis;
|
||||
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
import org.apache.doris.thrift.TExprNode;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
@ -99,4 +98,9 @@ public class BetweenPredicate extends Predicate {
|
||||
|
||||
@Override
|
||||
public Expr clone(ExprSubstitutionMap sMap) { return new BetweenPredicate(this); }
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Boolean.hashCode(isNotBetween);
|
||||
}
|
||||
}
|
||||
|
||||
@ -17,6 +17,11 @@
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.doris.catalog.Function;
|
||||
import org.apache.doris.catalog.FunctionSet;
|
||||
import org.apache.doris.catalog.PrimitiveType;
|
||||
@ -30,16 +35,11 @@ import org.apache.doris.common.io.Writable;
|
||||
import org.apache.doris.thrift.TExprNode;
|
||||
import org.apache.doris.thrift.TExprNodeType;
|
||||
import org.apache.doris.thrift.TExprOpcode;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* Most predicates with two operands..
|
||||
@ -552,5 +552,10 @@ public class BinaryPredicate extends Predicate implements Writable {
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Objects.hashCode(op);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -118,4 +118,9 @@ public class BoolLiteral extends LiteralExpr {
|
||||
literal.readFields(in);
|
||||
return literal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Boolean.hashCode(value);
|
||||
}
|
||||
}
|
||||
|
||||
@ -18,14 +18,17 @@
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.doris.catalog.Type;
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
import org.apache.doris.thrift.TExprNode;
|
||||
import org.apache.doris.thrift.TExprNodeType;
|
||||
import org.apache.doris.thrift.TExprOpcode;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* &&, ||, ! predicates.
|
||||
@ -219,4 +222,9 @@ public class CompoundPredicate extends Predicate {
|
||||
}
|
||||
return new BoolLiteral(compoundResult);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Objects.hashCode(op);
|
||||
}
|
||||
}
|
||||
|
||||
@ -17,6 +17,15 @@
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Date;
|
||||
import java.util.Objects;
|
||||
import java.util.TimeZone;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.doris.catalog.Catalog;
|
||||
import org.apache.doris.catalog.PrimitiveType;
|
||||
import org.apache.doris.catalog.Type;
|
||||
@ -26,9 +35,6 @@ import org.apache.doris.common.util.TimeUtils;
|
||||
import org.apache.doris.thrift.TDateLiteral;
|
||||
import org.apache.doris.thrift.TExprNode;
|
||||
import org.apache.doris.thrift.TExprNodeType;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.joda.time.DateTime;
|
||||
@ -37,13 +43,7 @@ import org.joda.time.LocalDateTime;
|
||||
import org.joda.time.format.DateTimeFormatter;
|
||||
import org.joda.time.format.DateTimeFormatterBuilder;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Date;
|
||||
import java.util.TimeZone;
|
||||
import java.util.regex.Pattern;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
public class DateLiteral extends LiteralExpr {
|
||||
private static final Logger LOG = LogManager.getLogger(DateLiteral.class);
|
||||
@ -616,4 +616,10 @@ public class DateLiteral extends LiteralExpr {
|
||||
private long minute;
|
||||
private long second;
|
||||
private long microsecond;
|
||||
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Objects.hashCode(unixTimestamp(TimeZone.getDefault()));
|
||||
}
|
||||
}
|
||||
|
||||
@ -37,6 +37,7 @@ import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.util.Objects;
|
||||
|
||||
public class DecimalLiteral extends LiteralExpr {
|
||||
private static final Logger LOG = LogManager.getLogger(DecimalLiteral.class);
|
||||
@ -233,4 +234,8 @@ public class DecimalLiteral extends LiteralExpr {
|
||||
return super.uncheckedCastTo(targetType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Objects.hashCode(value);
|
||||
}
|
||||
}
|
||||
|
||||
@ -17,10 +17,10 @@
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import org.apache.doris.thrift.TExprNode;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.doris.thrift.TExprNode;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
@ -68,5 +68,10 @@ public class ExistsPredicate extends Predicate {
|
||||
strBuilder.append(getChild(0).toSql());
|
||||
return strBuilder.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Boolean.hashCode(notExists);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -964,11 +964,16 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
// in group by clause, group by list need to remove duplicate exprs, the expr may be not not analyzed, the id
|
||||
// may be null
|
||||
if (id == null) {
|
||||
throw new UnsupportedOperationException("Expr.hashCode() is not implemented");
|
||||
} else {
|
||||
return id.asInt();
|
||||
int result = 31 * Objects.hashCode(type) + Objects.hashCode(opcode);
|
||||
for (Expr child : children) {
|
||||
result = 31 * result + Objects.hashCode(child);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
return id.asInt();
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@ -186,5 +186,10 @@ public class FloatLiteral extends LiteralExpr {
|
||||
literal.readFields(in);
|
||||
return literal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Double.hashCode(value);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -555,7 +555,8 @@ public class FunctionCallExpr extends Expr {
|
||||
} else {
|
||||
// now first find function in built-in functions
|
||||
if (Strings.isNullOrEmpty(fnName.getDb())) {
|
||||
fn = getBuiltinFunction(analyzer, fnName.getFunction(), collectChildReturnTypes(),
|
||||
Type[] childTypes = collectChildReturnTypes();
|
||||
fn = getBuiltinFunction(analyzer, fnName.getFunction(), childTypes,
|
||||
Function.CompareMode.IS_NONSTRICT_SUPERTYPE_OF);
|
||||
}
|
||||
|
||||
@ -638,7 +639,8 @@ public class FunctionCallExpr extends Expr {
|
||||
for (int i = 0; i < argTypes.length; ++i) {
|
||||
// For varargs, we must compare with the last type in callArgs.argTypes.
|
||||
int ix = Math.min(args.length - 1, i);
|
||||
if (!argTypes[i].matchesType(args[ix]) && !(argTypes[i].isDateType() && args[ix].isDateType())) {
|
||||
if (!argTypes[i].matchesType(args[ix]) && !(
|
||||
argTypes[i].isDateType() && args[ix].isDateType())) {
|
||||
uncheckedCastChild(args[ix], i);
|
||||
//if (argTypes[i] != args[ix]) castChild(args[ix], i);
|
||||
}
|
||||
@ -723,4 +725,13 @@ public class FunctionCallExpr extends Expr {
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = super.hashCode();
|
||||
result = 31 * result + Objects.hashCode(opcode);
|
||||
result = 31 * result + Objects.hashCode(fnName);
|
||||
result = 31 * result + Objects.hashCode(fnParams);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
@ -33,6 +33,7 @@ import org.apache.logging.log4j.Logger;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Class to represent a function name. Function names are specified as
|
||||
@ -203,4 +204,9 @@ public class FunctionName implements Writable {
|
||||
functionName.readFields(in);
|
||||
return functionName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * Objects.hashCode(db_) + Objects.hashCode(fn_);
|
||||
}
|
||||
}
|
||||
|
||||
@ -17,14 +17,15 @@
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import org.apache.doris.common.io.Writable;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.doris.common.io.Writable;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* Return value of the grammar production that parses function
|
||||
@ -106,4 +107,13 @@ public class FunctionParams implements Writable {
|
||||
params.readFields(in);
|
||||
return params;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = 31 * Boolean.hashCode(isStar) + Boolean.hashCode(isDistinct);
|
||||
for (Expr expr : exprs) {
|
||||
result = 31 * result + Objects.hashCode(expr);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
295
fe/src/main/java/org/apache/doris/analysis/GroupByClause.java
Normal file
295
fe/src/main/java/org/apache/doris/analysis/GroupByClause.java
Normal file
@ -0,0 +1,295 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicates;
|
||||
|
||||
import org.apache.commons.collections.CollectionUtils;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Wraps all information of group by clause. support normal GROUP BY clause and extended GROUP BY clause like
|
||||
* ROLLUP, GROUPING SETS, CUBE syntax like
|
||||
* SELECT a, b, SUM( c ) FROM tab1 GROUP BY GROUPING SETS ( (a, b), (a), (b), ( ) );
|
||||
* SELECT a, b,c, SUM( d ) FROM tab1 GROUP BY ROLLUP(a,b,c)
|
||||
* SELECT a, b,c, SUM( d ) FROM tab1 GROUP BY CUBE(a,b,c)
|
||||
* GROUP BY `GROUPING SETS` | `CUBE` | `ROLLUP` is an extension to GROUP BY clause.
|
||||
* This syntax lets you define multiple groupings in the same query.
|
||||
* GROUPING SETS produce a single result set that is equivalent to a UNION ALL of differently grouped rows.
|
||||
* In this class we produce the rule of generating rows base on the group by clause.
|
||||
*/
|
||||
public class GroupByClause implements ParseNode {
|
||||
private final static Logger LOG = LogManager.getLogger(GroupByClause.class);
|
||||
|
||||
// max num of distinct sets in grouping sets clause
|
||||
private final static int MAX_GROUPING_SETS_NUM = 16;
|
||||
// max num of distinct expressions
|
||||
private final static int MAX_GROUPING_SETS_EXPRESSION_NUM = 64;
|
||||
private boolean analyzed_ = false;
|
||||
private boolean exprGenerated = false;
|
||||
private GroupingType groupingType;
|
||||
private ArrayList<Expr> groupingExprs;
|
||||
private ArrayList<Expr> oriGroupingExprs;
|
||||
// reserve this info for toSQL
|
||||
private List<ArrayList<Expr>> groupingSetList;
|
||||
|
||||
public GroupByClause(List<ArrayList<Expr>> groupingSetList, GroupingType type) {
|
||||
this.groupingType = type;
|
||||
this.groupingSetList = groupingSetList;
|
||||
Preconditions.checkState(type == GroupingType.GROUPING_SETS);
|
||||
}
|
||||
|
||||
public GroupByClause(ArrayList<Expr> groupingExprs, GroupingType type) {
|
||||
this.groupingType = type;
|
||||
this.oriGroupingExprs = groupingExprs;
|
||||
this.groupingExprs = new ArrayList<>();
|
||||
this.groupingExprs.addAll(oriGroupingExprs);
|
||||
Preconditions.checkState(type != GroupingType.GROUPING_SETS);
|
||||
}
|
||||
|
||||
protected GroupByClause(GroupByClause other) {
|
||||
this.groupingType = other.groupingType;
|
||||
this.groupingExprs = (other.groupingExprs != null) ? Expr.cloneAndResetList(other.groupingExprs) : null;
|
||||
this.oriGroupingExprs =
|
||||
(other.oriGroupingExprs != null) ? Expr.cloneAndResetList(other.oriGroupingExprs) : null;
|
||||
|
||||
if (other.groupingSetList != null) {
|
||||
this.groupingSetList = new ArrayList<>();
|
||||
for (List<Expr> exprList : other.groupingSetList) {
|
||||
this.groupingSetList.add(Expr.cloneAndResetList(exprList));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public List<ArrayList<Expr>> getGroupingSetList() {
|
||||
return groupingSetList;
|
||||
}
|
||||
|
||||
public GroupingType getGroupingType() {
|
||||
return groupingType;
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
groupingExprs = new ArrayList<>();
|
||||
analyzed_ = false;
|
||||
exprGenerated = false;
|
||||
if (oriGroupingExprs != null) {
|
||||
Expr.resetList(oriGroupingExprs);
|
||||
groupingExprs.addAll(oriGroupingExprs);
|
||||
}
|
||||
if (groupingSetList != null) {
|
||||
for (List<Expr> s : groupingSetList) {
|
||||
for (Expr e : s) {
|
||||
if (e != null) {
|
||||
e.reset();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public ArrayList<Expr> getGroupingExprs() {
|
||||
if (!exprGenerated) {
|
||||
try {
|
||||
genGroupingExprs();
|
||||
} catch (AnalysisException e) {
|
||||
LOG.error("gen grouping expr error:", e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
return groupingExprs;
|
||||
}
|
||||
|
||||
// generate grouping exprs from group by, grouping sets, cube, rollup cluase
|
||||
public void genGroupingExprs() throws AnalysisException {
|
||||
if (exprGenerated) {
|
||||
return;
|
||||
}
|
||||
if (CollectionUtils.isNotEmpty(groupingExprs)) {
|
||||
// remove repeated element
|
||||
Set<Expr> groupingExprSet = new LinkedHashSet<>(groupingExprs);
|
||||
groupingExprs.clear();
|
||||
groupingExprs.addAll(groupingExprSet);
|
||||
}
|
||||
if (groupingType == GroupingType.CUBE || groupingType == GroupingType.ROLLUP) {
|
||||
if (CollectionUtils.isEmpty(groupingExprs)) {
|
||||
throw new AnalysisException(
|
||||
"The expresions in GROUPING CUBE or ROLLUP can not be empty");
|
||||
}
|
||||
} else if (groupingType == GroupingType.GROUPING_SETS) {
|
||||
if (CollectionUtils.isEmpty(groupingSetList)) {
|
||||
throw new AnalysisException("The expresions in GROUPINGING SETS can not be empty");
|
||||
}
|
||||
// collect all Expr elements
|
||||
Set<Expr> groupingExprSet = new LinkedHashSet<>();
|
||||
for (ArrayList<Expr> list : groupingSetList) {
|
||||
groupingExprSet.addAll(list);
|
||||
}
|
||||
groupingExprs = new ArrayList<>(groupingExprSet);
|
||||
}
|
||||
exprGenerated = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void analyze(Analyzer analyzer) throws AnalysisException {
|
||||
if (analyzed_) {
|
||||
return;
|
||||
}
|
||||
genGroupingExprs();
|
||||
|
||||
// disallow subqueries in the GROUP BY clause
|
||||
for (Expr expr : groupingExprs) {
|
||||
if (expr.contains(Predicates.instanceOf(Subquery.class))) {
|
||||
throw new AnalysisException(
|
||||
"Subqueries are not supported in the GROUP BY clause.");
|
||||
}
|
||||
}
|
||||
//TODO add the analysis for grouping and grouping_id functions
|
||||
for (Expr groupingExpr : groupingExprs) {
|
||||
groupingExpr.analyze(analyzer);
|
||||
if (groupingExpr.contains(Expr.isAggregatePredicate())) {
|
||||
// reference the original expr in the error msg
|
||||
throw new AnalysisException(
|
||||
"GROUP BY expression must not contain aggregate functions: "
|
||||
+ groupingExpr.toSql());
|
||||
}
|
||||
if (groupingExpr.contains(AnalyticExpr.class)) {
|
||||
// reference the original expr in the error msg
|
||||
throw new AnalysisException(
|
||||
"GROUP BY expression must not contain analytic expressions: "
|
||||
+ groupingExpr.toSql());
|
||||
}
|
||||
|
||||
if (groupingExpr.type.isHllType()) {
|
||||
throw new AnalysisException(
|
||||
"GROUP BY expression must not contain hll column: "
|
||||
+ groupingExpr.toSql());
|
||||
}
|
||||
}
|
||||
|
||||
if (groupingExprs != null && groupingExprs.size() > MAX_GROUPING_SETS_NUM) {
|
||||
throw new AnalysisException(
|
||||
"Too many sets in GROUP BY clause, it must be not more than "
|
||||
+ MAX_GROUPING_SETS_NUM);
|
||||
}
|
||||
analyzed_ = true;
|
||||
}
|
||||
|
||||
public boolean isGroupByExtension() {
|
||||
if (groupingType == GroupingType.GROUP_BY ||
|
||||
groupingType == GroupingType.GROUPING_SETS && (groupingSetList == null || groupingSetList.size() < 2)) {
|
||||
return false;
|
||||
} else {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toSql() {
|
||||
StringBuilder strBuilder = new StringBuilder();
|
||||
switch (groupingType) {
|
||||
case GROUP_BY:
|
||||
if (oriGroupingExprs != null) {
|
||||
for (int i = 0; i < oriGroupingExprs.size(); ++i) {
|
||||
strBuilder.append(oriGroupingExprs.get(i).toSql());
|
||||
strBuilder.append((i + 1 != oriGroupingExprs.size()) ? ", " : "");
|
||||
}
|
||||
}
|
||||
break;
|
||||
case GROUPING_SETS:
|
||||
if (groupingSetList != null) {
|
||||
strBuilder.append("GROUPING SETS (");
|
||||
boolean first = true;
|
||||
for (List<Expr> groupingExprs : groupingSetList) {
|
||||
if (first) {
|
||||
strBuilder.append("(");
|
||||
first = false;
|
||||
} else {
|
||||
strBuilder.append(", (");
|
||||
}
|
||||
for (int i = 0; i < groupingExprs.size(); ++i) {
|
||||
strBuilder.append(groupingExprs.get(i).toSql());
|
||||
strBuilder.append((i + 1 != groupingExprs.size()) ? ", " : "");
|
||||
}
|
||||
strBuilder.append(")");
|
||||
}
|
||||
strBuilder.append(")");
|
||||
}
|
||||
break;
|
||||
case CUBE:
|
||||
if (oriGroupingExprs != null) {
|
||||
strBuilder.append("CUBE (");
|
||||
for (int i = 0; i < oriGroupingExprs.size(); ++i) {
|
||||
strBuilder.append(oriGroupingExprs.get(i).toSql());
|
||||
strBuilder.append((i + 1 != oriGroupingExprs.size()) ? ", " : "");
|
||||
}
|
||||
strBuilder.append(")");
|
||||
}
|
||||
break;
|
||||
case ROLLUP:
|
||||
if (oriGroupingExprs != null) {
|
||||
strBuilder.append("ROLLUP (");
|
||||
for (int i = 0; i < oriGroupingExprs.size(); ++i) {
|
||||
strBuilder.append(oriGroupingExprs.get(i).toSql());
|
||||
strBuilder.append((i + 1 != oriGroupingExprs.size()) ? ", " : "");
|
||||
}
|
||||
strBuilder.append(")");
|
||||
}
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return strBuilder.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public GroupByClause clone() {
|
||||
return new GroupByClause(this);
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return CollectionUtils.isEmpty(groupingExprs);
|
||||
}
|
||||
|
||||
public void substituteGroupingExprs(Set<VirtualSlotRef> groupingSlots, ExprSubstitutionMap smap,
|
||||
Analyzer analyzer) {
|
||||
groupingExprs = Expr.substituteList(groupingExprs, smap, analyzer, true);
|
||||
for (VirtualSlotRef vs : groupingSlots) {
|
||||
vs.setRealSlots(Optional.ofNullable(Expr.substituteList(vs.getRealSlots(), smap, analyzer, true)).orElse(
|
||||
new ArrayList<>()).stream().map(e -> (SlotRef) e).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
public enum GroupingType {
|
||||
GROUP_BY,
|
||||
GROUPING_SETS,
|
||||
ROLLUP,
|
||||
CUBE
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,121 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import org.apache.doris.catalog.Function;
|
||||
import org.apache.doris.catalog.Type;
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* mapping the real slot to virtual slots, grouping(_id) function will use a virtual slot of BIGINT to substitute
|
||||
* real slots, and then set real slot to realChildren
|
||||
*/
|
||||
public class GroupingFunctionCallExpr extends FunctionCallExpr {
|
||||
private static final Logger LOG = LogManager.getLogger(FunctionCallExpr.class);
|
||||
private boolean childrenReseted = false;
|
||||
private List<Expr> realChildren;
|
||||
|
||||
public GroupingFunctionCallExpr(String functionName, List<Expr> params) {
|
||||
super(functionName, params);
|
||||
childrenReseted = false;
|
||||
}
|
||||
|
||||
public GroupingFunctionCallExpr(FunctionName functionName, FunctionParams params) {
|
||||
super(functionName, params);
|
||||
childrenReseted = false;
|
||||
}
|
||||
|
||||
public GroupingFunctionCallExpr(GroupingFunctionCallExpr other) {
|
||||
super(other);
|
||||
this.childrenReseted = other.childrenReseted;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Expr clone() {
|
||||
return new GroupingFunctionCallExpr(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void analyzeImpl(Analyzer analyzer) throws AnalysisException {
|
||||
if (children.size() < 1) {
|
||||
throw new AnalysisException("GROUPING functions required at least one parameters");
|
||||
}
|
||||
for (Expr expr : children) {
|
||||
if (expr instanceof SlotRef) {
|
||||
continue;
|
||||
} else {
|
||||
// throw new AnalysisException("GROUPING functions required columns as parameters");
|
||||
}
|
||||
}
|
||||
Type[] childTypes = new Type[1];
|
||||
childTypes[0] = Type.BIGINT;
|
||||
fn = getBuiltinFunction(analyzer, getFnName().getFunction(), childTypes,
|
||||
Function.CompareMode.IS_IDENTICAL);
|
||||
this.type = fn.getReturnType();
|
||||
}
|
||||
|
||||
// set child to virtual slot
|
||||
public void resetChild(VirtualSlotRef virtualSlot) {
|
||||
ArrayList<Expr> newChildren = new ArrayList<>();
|
||||
newChildren.add(virtualSlot);
|
||||
realChildren = new ArrayList<>();
|
||||
realChildren.addAll(children);
|
||||
children = newChildren;
|
||||
childrenReseted = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Expr reset() {
|
||||
if (childrenReseted) {
|
||||
children = new ArrayList<>();
|
||||
children.addAll(realChildren);
|
||||
}
|
||||
childrenReseted = false;
|
||||
realChildren = null;
|
||||
return super.reset();
|
||||
}
|
||||
|
||||
// get the origin children of the expr
|
||||
public List<Expr> getRealSlot() {
|
||||
if (childrenReseted) {
|
||||
List<Expr> result = new ArrayList<>();
|
||||
for (Expr expr : realChildren) {
|
||||
result.add(expr);
|
||||
}
|
||||
return result;
|
||||
} else if (isAnalyzed()) {
|
||||
List<Expr> result = new ArrayList<>();
|
||||
for (Expr expr : children) {
|
||||
result.add(expr);
|
||||
}
|
||||
return result;
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
public List<Expr> getRealChildren() {
|
||||
return realChildren;
|
||||
}
|
||||
}
|
||||
209
fe/src/main/java/org/apache/doris/analysis/GroupingInfo.java
Normal file
209
fe/src/main/java/org/apache/doris/analysis/GroupingInfo.java
Normal file
@ -0,0 +1,209 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import org.apache.doris.catalog.Table;
|
||||
import org.apache.doris.catalog.Type;
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.ListIterator;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class GroupingInfo {
|
||||
private VirtualSlotRef groupingIDSlot;
|
||||
private TupleDescriptor virtualTuple;
|
||||
private Set<VirtualSlotRef> groupingSlots;
|
||||
private List<BitSet> groupingIdList;
|
||||
private GroupByClause.GroupingType groupingType;
|
||||
|
||||
public GroupingInfo(Analyzer analyzer, GroupByClause.GroupingType groupingType) throws AnalysisException {
|
||||
this.groupingType = groupingType;
|
||||
groupingSlots = new LinkedHashSet<>();
|
||||
virtualTuple = analyzer.getDescTbl().createTupleDescriptor("VIRTUAL_TUPLE");
|
||||
String colName = "GROUPING__ID";
|
||||
groupingIDSlot = new VirtualSlotRef(colName, Type.BIGINT, virtualTuple, new ArrayList<>());
|
||||
groupingIDSlot.analyze(analyzer);
|
||||
groupingSlots.add(groupingIDSlot);
|
||||
}
|
||||
|
||||
public Set<VirtualSlotRef> getGroupingSlots() {
|
||||
return groupingSlots;
|
||||
}
|
||||
|
||||
public TupleDescriptor getVirtualTuple() {
|
||||
return virtualTuple;
|
||||
}
|
||||
|
||||
public VirtualSlotRef getGroupingIDSlot() {
|
||||
return groupingIDSlot;
|
||||
}
|
||||
|
||||
public List<BitSet> getGroupingIdList() {
|
||||
return groupingIdList;
|
||||
}
|
||||
|
||||
// generate virtual slots for grouping or grouping_id functions
|
||||
public VirtualSlotRef addGroupingSlots(List<Expr> realSlots, Analyzer analyzer) throws AnalysisException {
|
||||
String colName = realSlots.stream().map(expr -> expr.toSql()).collect(Collectors.joining(
|
||||
"_"));
|
||||
colName = "GROUPING_PREFIX_" + colName;
|
||||
VirtualSlotRef virtualSlot = new VirtualSlotRef(colName, Type.BIGINT, virtualTuple, realSlots);
|
||||
virtualSlot.analyze(analyzer);
|
||||
if (groupingSlots.contains(virtualSlot)) {
|
||||
for (VirtualSlotRef vs : groupingSlots) {
|
||||
if (vs.equals(virtualSlot)) {
|
||||
return vs;
|
||||
}
|
||||
}
|
||||
}
|
||||
groupingSlots.add(virtualSlot);
|
||||
return virtualSlot;
|
||||
}
|
||||
|
||||
// generate the bitmap that repeated node will repeat rows according to
|
||||
public void buildRepeat(ArrayList<Expr> groupingExprs, List<ArrayList<Expr>> groupingSetList) {
|
||||
groupingIdList = new ArrayList<>();
|
||||
BitSet bitSetAll = new BitSet();
|
||||
bitSetAll.set(0, groupingExprs.size(), true);
|
||||
groupingIdList.add(bitSetAll);
|
||||
switch (groupingType) {
|
||||
case CUBE:
|
||||
for (int i = 0; i < (1 << groupingExprs.size()) - 1; i++) {
|
||||
BitSet bitSet = new BitSet();
|
||||
for (int j = 0; j < groupingExprs.size(); j++) {
|
||||
if ((i & (1 << j)) > 0) {
|
||||
bitSet.set(j, true);
|
||||
}
|
||||
}
|
||||
groupingIdList.add(bitSet);
|
||||
}
|
||||
break;
|
||||
|
||||
case ROLLUP:
|
||||
for (int i = 0; i < groupingExprs.size(); i++) {
|
||||
BitSet bitSet = new BitSet();
|
||||
bitSet.set(0, i);
|
||||
groupingIdList.add(bitSet);
|
||||
}
|
||||
break;
|
||||
|
||||
case GROUPING_SETS:
|
||||
BitSet bitSetBase = new BitSet();
|
||||
bitSetBase.set(0, groupingExprs.size());
|
||||
for (ArrayList<Expr> list : groupingSetList) {
|
||||
BitSet bitSet = new BitSet();
|
||||
for (int i = 0; i < groupingExprs.size(); i++) {
|
||||
bitSet.set(i, list.contains(groupingExprs.get(i)));
|
||||
}
|
||||
if (!bitSet.equals(bitSetBase)) {
|
||||
if (!groupingIdList.contains(bitSet)) {
|
||||
groupingIdList.add(bitSet);
|
||||
}
|
||||
}
|
||||
}
|
||||
break;
|
||||
|
||||
default:
|
||||
Preconditions.checkState(false);
|
||||
}
|
||||
groupingExprs.addAll(groupingSlots);
|
||||
}
|
||||
|
||||
// generate grouping function's value
|
||||
public List<List<Long>> genGroupingList(ArrayList<Expr> groupingExprs) {
|
||||
List<List<Long>> groupingList = new ArrayList<>();
|
||||
BitSet base = groupingIdList.get(0);
|
||||
for (SlotRef slot : groupingSlots) {
|
||||
List<Long> glist = new ArrayList<>();
|
||||
for (BitSet bitSet : groupingIdList) {
|
||||
long l = 0L;
|
||||
// for all column, using for group by
|
||||
if ("GROUPING__ID".equals(slot.getColumnName())) {
|
||||
BitSet newBitSet = new BitSet();
|
||||
for (int i = 0; i < base.length(); ++i) {
|
||||
newBitSet.set(i, bitSet.get(base.length() - i - 1));
|
||||
}
|
||||
newBitSet.flip(0, base.length());
|
||||
newBitSet.and(base);
|
||||
for (int i = 0; i < newBitSet.length(); ++i) {
|
||||
l += newBitSet.get(i) ? (1L << i) : 0L;
|
||||
}
|
||||
} else {
|
||||
// for grouping[_id] functions
|
||||
int slotSize = ((VirtualSlotRef) slot).getRealSlots().size();
|
||||
for (int i = 0; i < slotSize; ++i) {
|
||||
int j = groupingExprs.indexOf(((VirtualSlotRef) slot).getRealSlots().get(i));
|
||||
l += bitSet.get(j) ? 0L : (1L << (slotSize - i - 1));
|
||||
}
|
||||
}
|
||||
glist.add(l);
|
||||
}
|
||||
groupingList.add(glist);
|
||||
}
|
||||
return groupingList;
|
||||
}
|
||||
|
||||
public void substituteGroupingFn(List<Expr> exprs, Analyzer analyzer) throws AnalysisException {
|
||||
if (groupingType == GroupByClause.GroupingType.GROUP_BY) {
|
||||
throw new AnalysisException("cannot use GROUPING functions without [grouping sets|rollup|cube] a"
|
||||
+ "clause or grouping sets only have one element.");
|
||||
}
|
||||
ListIterator<Expr> i = exprs.listIterator();
|
||||
while (i.hasNext()) {
|
||||
Expr expr = i.next();
|
||||
substituteGroupingFn(expr, analyzer);
|
||||
}
|
||||
}
|
||||
|
||||
public void substituteGroupingFn(Expr expr, Analyzer analyzer) throws AnalysisException {
|
||||
if (expr instanceof GroupingFunctionCallExpr) {
|
||||
// TODO(yangzhengguo) support expression in grouping functuons
|
||||
for (Expr child: expr.getChildren()) {
|
||||
if (!(child instanceof SlotRef)) {
|
||||
throw new AnalysisException("grouping functions only support column in current version.");
|
||||
// expr from inline view
|
||||
} else if (((SlotRef) child).getDesc().getParent().getTable().getType()
|
||||
== Table.TableType.INLINE_VIEW) {
|
||||
InlineViewRef ref = (InlineViewRef) ((SlotRef) child).getDesc().getParent().getRef();
|
||||
int colIndex = ref.getColLabels().indexOf(((SlotRef) child).getColumnName());
|
||||
if (colIndex != -1 && !(ref.getViewStmt().getResultExprs().get(colIndex) instanceof SlotRef)) {
|
||||
throw new AnalysisException("grouping functions only support column in current version.");
|
||||
}
|
||||
}
|
||||
}
|
||||
// if is substituted skip
|
||||
if (expr.getChildren().size() == 1 && expr.getChild(0) instanceof VirtualSlotRef) {
|
||||
return;
|
||||
}
|
||||
VirtualSlotRef vSlot = addGroupingSlots(((GroupingFunctionCallExpr) expr).getRealSlot(), analyzer);
|
||||
((GroupingFunctionCallExpr) expr).resetChild(vSlot);
|
||||
expr.analyze(analyzer);
|
||||
} else if (expr.getChildren().size() > 0) {
|
||||
for (Expr child : expr.getChildren()) {
|
||||
substituteGroupingFn(child, analyzer);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -335,4 +335,9 @@ public class IntLiteral extends LiteralExpr {
|
||||
literal.readFields(in);
|
||||
return literal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Long.hashCode(value);
|
||||
}
|
||||
}
|
||||
|
||||
@ -17,6 +17,15 @@
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.math.BigInteger;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.doris.catalog.PrimitiveType;
|
||||
import org.apache.doris.catalog.Type;
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
@ -25,17 +34,8 @@ import org.apache.doris.common.io.Text;
|
||||
import org.apache.doris.thrift.TExprNode;
|
||||
import org.apache.doris.thrift.TExprNodeType;
|
||||
import org.apache.doris.thrift.TLargeIntLiteral;
|
||||
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.math.BigInteger;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
// large int for the num that native types can not
|
||||
public class LargeIntLiteral extends LiteralExpr {
|
||||
@ -221,4 +221,9 @@ public class LargeIntLiteral extends LiteralExpr {
|
||||
largeIntLiteral.readFields(in);
|
||||
return largeIntLiteral;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Objects.hashCode(value);
|
||||
}
|
||||
}
|
||||
|
||||
@ -17,6 +17,10 @@
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.regex.PatternSyntaxException;
|
||||
|
||||
import org.apache.doris.catalog.Function;
|
||||
import org.apache.doris.catalog.FunctionSet;
|
||||
import org.apache.doris.catalog.ScalarFunction;
|
||||
@ -28,9 +32,6 @@ import org.apache.doris.thrift.TExprNodeType;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.regex.PatternSyntaxException;
|
||||
|
||||
public class LikePredicate extends Predicate {
|
||||
|
||||
enum Operator {
|
||||
@ -134,4 +135,9 @@ public class LikePredicate extends Predicate {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Objects.hashCode(op);
|
||||
}
|
||||
}
|
||||
|
||||
@ -47,6 +47,7 @@ import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import org.apache.commons.collections.CollectionUtils;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
@ -64,14 +65,14 @@ import java.util.Set;
|
||||
*/
|
||||
public class SelectStmt extends QueryStmt {
|
||||
private final static Logger LOG = LogManager.getLogger(SelectStmt.class);
|
||||
|
||||
|
||||
// ///////////////////////////////////////
|
||||
// BEGIN: Members that need to be reset()
|
||||
|
||||
protected SelectList selectList;
|
||||
private final ArrayList<String> colLabels; // lower case column labels
|
||||
protected final FromClause fromClause_;
|
||||
protected ArrayList<Expr> groupingExprs;
|
||||
protected GroupByClause groupByClause;
|
||||
private List<Expr> originalExpr;
|
||||
//
|
||||
private Expr havingClause; // original having clause
|
||||
@ -89,6 +90,9 @@ public class SelectStmt extends QueryStmt {
|
||||
|
||||
private ValueList valueList;
|
||||
|
||||
// if we have grouping extensions like cube or rollup or grouping sets
|
||||
private GroupingInfo groupingInfo;
|
||||
|
||||
// END: Members that need to be reset()
|
||||
// ///////////////////////////////////////
|
||||
|
||||
@ -108,7 +112,7 @@ public class SelectStmt extends QueryStmt {
|
||||
SelectList selectList,
|
||||
FromClause fromClause,
|
||||
Expr wherePredicate,
|
||||
ArrayList<Expr> groupingExprs,
|
||||
GroupByClause groupByClause,
|
||||
Expr havingPredicate,
|
||||
ArrayList<OrderByElement> orderByElements,
|
||||
LimitElement limitElement) {
|
||||
@ -120,13 +124,14 @@ public class SelectStmt extends QueryStmt {
|
||||
fromClause_ = fromClause;
|
||||
}
|
||||
this.whereClause = wherePredicate;
|
||||
this.groupingExprs = groupingExprs;
|
||||
this.groupByClause = groupByClause;
|
||||
this.havingClause = havingPredicate;
|
||||
|
||||
this.colLabels = Lists.newArrayList();
|
||||
this.havingPred = null;
|
||||
this.aggInfo = null;
|
||||
this.sortInfo = null;
|
||||
this.groupingInfo = null;
|
||||
}
|
||||
|
||||
protected SelectStmt(SelectStmt other) {
|
||||
@ -134,14 +139,15 @@ public class SelectStmt extends QueryStmt {
|
||||
selectList = other.selectList.clone();
|
||||
fromClause_ = other.fromClause_.clone();
|
||||
whereClause = (other.whereClause != null) ? other.whereClause.clone() : null;
|
||||
groupingExprs = (other.groupingExprs != null) ? Expr.cloneAndResetList(other.groupingExprs) : null;
|
||||
groupByClause = (other.groupByClause != null) ? other.groupByClause.clone() : null;
|
||||
havingClause = (other.havingClause != null) ? other.havingClause.clone() : null;
|
||||
|
||||
colLabels = Lists.newArrayList(other.colLabels);
|
||||
aggInfo = (other.aggInfo != null) ? other.aggInfo.clone() : null;
|
||||
analyticInfo = (other.analyticInfo != null) ? other.analyticInfo.clone() : null;
|
||||
sqlString_ = (other.sqlString_ != null) ? new String(other.sqlString_) : null;
|
||||
sqlString_ = (other.sqlString_ != null) ? other.sqlString_ : null;
|
||||
baseTblSmap = other.baseTblSmap.clone();
|
||||
groupingInfo = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -150,13 +156,20 @@ public class SelectStmt extends QueryStmt {
|
||||
selectList.reset();
|
||||
colLabels.clear();
|
||||
fromClause_.reset();
|
||||
if (whereClause != null) whereClause.reset();
|
||||
if (groupingExprs != null) Expr.resetList(groupingExprs);
|
||||
if (havingClause != null) havingClause.reset();
|
||||
if (whereClause != null) {
|
||||
whereClause.reset();
|
||||
}
|
||||
if (groupByClause != null) {
|
||||
groupByClause.reset();
|
||||
}
|
||||
if (havingClause != null) {
|
||||
havingClause.reset();
|
||||
}
|
||||
havingPred = null;
|
||||
aggInfo = null;
|
||||
analyticInfo = null;
|
||||
baseTblSmap.clear();
|
||||
groupingInfo = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -190,24 +203,33 @@ public class SelectStmt extends QueryStmt {
|
||||
return whereClause;
|
||||
}
|
||||
|
||||
public ArrayList<Expr> getGroupingExprs() {
|
||||
return groupingExprs;
|
||||
}
|
||||
|
||||
public void setWhereClause(Expr whereClause) {
|
||||
this.whereClause = whereClause;
|
||||
}
|
||||
|
||||
public AggregateInfo getAggInfo() {
|
||||
return aggInfo;
|
||||
}
|
||||
|
||||
public GroupingInfo getGroupingInfo() {
|
||||
return groupingInfo;
|
||||
}
|
||||
|
||||
public GroupByClause getGroupByClause() {
|
||||
return groupByClause;
|
||||
}
|
||||
|
||||
public AnalyticInfo getAnalyticInfo() {
|
||||
return analyticInfo;
|
||||
}
|
||||
|
||||
public boolean hasAnalyticInfo() {
|
||||
return analyticInfo != null;
|
||||
}
|
||||
public boolean hasHavingClause() { return havingClause != null; }
|
||||
|
||||
public boolean hasHavingClause() {
|
||||
return havingClause != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortInfo getSortInfo() {
|
||||
@ -249,12 +271,12 @@ public class SelectStmt extends QueryStmt {
|
||||
|
||||
// check auth
|
||||
if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), dbName,
|
||||
tblRef.getName().getTbl(),
|
||||
PrivPredicate.SELECT)) {
|
||||
tblRef.getName().getTbl(),
|
||||
PrivPredicate.SELECT)) {
|
||||
ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SELECT",
|
||||
ConnectContext.get().getQualifiedUser(),
|
||||
ConnectContext.get().getRemoteIP(),
|
||||
tblRef.getName().getTbl());
|
||||
ConnectContext.get().getQualifiedUser(),
|
||||
ConnectContext.get().getRemoteIP(),
|
||||
tblRef.getName().getTbl());
|
||||
}
|
||||
|
||||
dbs.put(dbName, db);
|
||||
@ -283,7 +305,9 @@ public class SelectStmt extends QueryStmt {
|
||||
}
|
||||
|
||||
public void analyze(Analyzer analyzer) throws AnalysisException, UserException {
|
||||
if (isAnalyzed()) return;
|
||||
if (isAnalyzed()) {
|
||||
return;
|
||||
}
|
||||
super.analyze(analyzer);
|
||||
|
||||
fromClause_.setNeedToSql(needToSql);
|
||||
@ -293,9 +317,10 @@ public class SelectStmt extends QueryStmt {
|
||||
// Skip this step when analyzing a WITH-clause because CollectionTableRefs
|
||||
// do not register collection slots in their parent in that context
|
||||
// (see CollectionTableRef.analyze()).
|
||||
if (!analyzer.isWithClause()) registerIsNotEmptyPredicates(analyzer);
|
||||
|
||||
// populate selectListExprs, aliasSMap, and colNames
|
||||
if (!analyzer.isWithClause()) {
|
||||
registerIsNotEmptyPredicates(analyzer);
|
||||
}
|
||||
// populate selectListExprs, aliasSMap, groupingSmap and colNames
|
||||
for (SelectListItem item : selectList.getItems()) {
|
||||
if (item.isStar()) {
|
||||
TableName tblName = item.getTblName();
|
||||
@ -309,10 +334,8 @@ public class SelectStmt extends QueryStmt {
|
||||
// of expr child and depth limits (toColumn() label may call toSql()).
|
||||
item.getExpr().analyze(analyzer);
|
||||
if (item.getExpr().contains(Predicates.instanceOf(Subquery.class))) {
|
||||
throw new AnalysisException(
|
||||
"Subqueries are not supported in the select list.");
|
||||
throw new AnalysisException("Subqueries are not supported in the select list.");
|
||||
}
|
||||
|
||||
resultExprs.add(item.getExpr());
|
||||
SlotRef aliasRef = new SlotRef(null, item.toColumnLabel());
|
||||
Expr existingAliasExpr = aliasSMap.get(aliasRef);
|
||||
@ -325,6 +348,11 @@ public class SelectStmt extends QueryStmt {
|
||||
colLabels.add(item.toColumnLabel());
|
||||
}
|
||||
}
|
||||
if (groupByClause != null && groupByClause.isGroupByExtension()) {
|
||||
groupingInfo = new GroupingInfo(analyzer, groupByClause.getGroupingType());
|
||||
groupingInfo.substituteGroupingFn(resultExprs, analyzer);
|
||||
}
|
||||
|
||||
if (valueList != null) {
|
||||
if (!fromInsert) {
|
||||
valueList.analyzeForSelect(analyzer);
|
||||
@ -360,6 +388,9 @@ public class SelectStmt extends QueryStmt {
|
||||
|
||||
if (whereClause != null) {
|
||||
whereClauseRewrite();
|
||||
if (checkGroupingFn(whereClause)) {
|
||||
throw new AnalysisException("grouping operations are not allowed in WHERE.");
|
||||
}
|
||||
whereClause.analyze(analyzer);
|
||||
if (whereClause.containsAggregate()) {
|
||||
ErrorReport.reportAnalysisException(ErrorCode.ERR_INVALID_GROUP_FUNC_USE);
|
||||
@ -375,6 +406,11 @@ public class SelectStmt extends QueryStmt {
|
||||
}
|
||||
|
||||
createSortInfo(analyzer);
|
||||
if (sortInfo != null && CollectionUtils.isNotEmpty(sortInfo.getOrderingExprs())) {
|
||||
if (groupingInfo != null) {
|
||||
groupingInfo.substituteGroupingFn(sortInfo.getOrderingExprs(), analyzer);
|
||||
}
|
||||
}
|
||||
analyzeAggregation(analyzer);
|
||||
createAnalyticInfo(analyzer);
|
||||
if (evaluateOrderBy) {
|
||||
@ -393,14 +429,16 @@ public class SelectStmt extends QueryStmt {
|
||||
}
|
||||
|
||||
if (aggInfo != null) {
|
||||
if (LOG.isDebugEnabled()) LOG.debug("post-analysis " + aggInfo.debugString());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("post-analysis " + aggInfo.debugString());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public List<TupleId> getTableRefIds() {
|
||||
List<TupleId> result = Lists.newArrayList();
|
||||
|
||||
for (TableRef ref: fromClause_) {
|
||||
for (TableRef ref : fromClause_) {
|
||||
result.add(ref.getId());
|
||||
}
|
||||
|
||||
@ -409,7 +447,7 @@ public class SelectStmt extends QueryStmt {
|
||||
|
||||
private void whereClauseRewrite() {
|
||||
if (whereClause instanceof IntLiteral) {
|
||||
if (((IntLiteral)whereClause).getLongValue() == 0) {
|
||||
if (((IntLiteral) whereClause).getLongValue() == 0) {
|
||||
whereClause = new BoolLiteral(false);
|
||||
} else {
|
||||
whereClause = new BoolLiteral(true);
|
||||
@ -422,15 +460,15 @@ public class SelectStmt extends QueryStmt {
|
||||
* in the parent scan of collection table refs. This is a performance optimization to
|
||||
* avoid the expensive processing of empty collections inside a subplan that would
|
||||
* yield an empty result set.
|
||||
*
|
||||
* <p>
|
||||
* For correctness purposes, the predicates are generated in cases where we can ensure
|
||||
* that they will be assigned only to the parent scan, and no other plan node.
|
||||
*
|
||||
* <p>
|
||||
* The conditions are as follows:
|
||||
* - collection table ref is relative and non-correlated
|
||||
* - collection table ref represents the rhs of an inner/cross/semi join
|
||||
* - collection table ref's parent tuple is not outer joined
|
||||
*
|
||||
* <p>
|
||||
* TODO: In some cases, it is possible to generate !empty() predicates for a correlated
|
||||
* table ref, but in general, that is not correct for non-trivial query blocks.
|
||||
* For example, if the block with the correlated ref has an aggregation then adding a
|
||||
@ -471,15 +509,15 @@ public class SelectStmt extends QueryStmt {
|
||||
// can also be safely evaluated below the join (picked up by getBoundPredicates()).
|
||||
// Such predicates will be marked twice and that is ok.
|
||||
List<Expr> unassigned =
|
||||
analyzer.getUnassignedConjuncts(getTableRefIds(), true);
|
||||
analyzer.getUnassignedConjuncts(getTableRefIds(), true);
|
||||
List<Expr> unassignedJoinConjuncts = Lists.newArrayList();
|
||||
for (Expr e: unassigned) {
|
||||
for (Expr e : unassigned) {
|
||||
if (analyzer.evalAfterJoin(e)) {
|
||||
unassignedJoinConjuncts.add(e);
|
||||
}
|
||||
}
|
||||
List<Expr> baseTblJoinConjuncts =
|
||||
Expr.trySubstituteList(unassignedJoinConjuncts, baseTblSmap, analyzer, false);
|
||||
Expr.trySubstituteList(unassignedJoinConjuncts, baseTblSmap, analyzer, false);
|
||||
analyzer.materializeSlots(baseTblJoinConjuncts);
|
||||
|
||||
if (evaluateOrderBy) {
|
||||
@ -507,12 +545,8 @@ public class SelectStmt extends QueryStmt {
|
||||
// before calling AggregateInfo.materializeRequiredSlots(), otherwise they won't
|
||||
// show up in AggregateInfo.getMaterializedAggregateExprs()
|
||||
ArrayList<Expr> havingConjuncts = Lists.newArrayList();
|
||||
if (havingPred != null) havingConjuncts.add(havingPred);
|
||||
// Ignore predicates bound to a group-by slot because those
|
||||
// are already evaluated below this agg node (e.g., in a scan).
|
||||
Set<SlotId> groupBySlots = Sets.newHashSet();
|
||||
for (int i = 0; i < aggInfo.getGroupingExprs().size(); ++i) {
|
||||
groupBySlots.add(aggInfo.getOutputTupleDesc().getSlots().get(i).getId());
|
||||
if (havingPred != null) {
|
||||
havingConjuncts.add(havingPred);
|
||||
}
|
||||
// Binding predicates are assigned to the final output tuple of the aggregation,
|
||||
// which is the tuple of the 2nd phase agg for distinct aggs.
|
||||
@ -569,7 +603,7 @@ public class SelectStmt extends QueryStmt {
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// can not get AST only with equal join, MayBe cross join can help
|
||||
fromClause_.clear();
|
||||
for (Pair<TableRef, Long> candidate : candidates) {
|
||||
@ -594,7 +628,6 @@ public class SelectStmt extends QueryStmt {
|
||||
fromClause_.add(firstRef);
|
||||
tableRefMap.remove(firstRef.getId());
|
||||
|
||||
|
||||
// reserve TupleId has been added successfully
|
||||
Set<TupleId> validTupleId = Sets.newHashSet();
|
||||
validTupleId.add(firstRef.getId());
|
||||
@ -613,6 +646,7 @@ public class SelectStmt extends QueryStmt {
|
||||
}
|
||||
TableRef candidateTableRef = tableRefMap.get(tid);
|
||||
if (candidateTableRef != null) {
|
||||
|
||||
// When sorting table according to the rows, you must ensure
|
||||
// that all tables On-conjuncts referenced has been added or
|
||||
// is being added.
|
||||
@ -626,7 +660,7 @@ public class SelectStmt extends QueryStmt {
|
||||
count--;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
if (count == 0) {
|
||||
fromClause_.add(candidateTableRef);
|
||||
validTupleId.add(tid);
|
||||
@ -671,32 +705,33 @@ public class SelectStmt extends QueryStmt {
|
||||
* Substitute all exprs (result of the analysis) of this select block referencing any
|
||||
* of our inlined views, including everything registered with the analyzer.
|
||||
* Expressions created during parsing (such as whereClause) are not touched.
|
||||
*
|
||||
* @throws AnalysisException
|
||||
*/
|
||||
public void seondSubstituteInlineViewExprs(ExprSubstitutionMap sMap) throws AnalysisException {
|
||||
// we might not have anything to substitute
|
||||
if (sMap.size() == 0) {
|
||||
return;
|
||||
}
|
||||
public void seondSubstituteInlineViewExprs(ExprSubstitutionMap sMap) throws AnalysisException {
|
||||
// we might not have anything to substitute
|
||||
if (sMap.size() == 0) {
|
||||
return;
|
||||
}
|
||||
|
||||
// select
|
||||
// Expr.substituteList(resultExprs, sMap);
|
||||
// select
|
||||
// Expr.substituteList(resultExprs, sMap);
|
||||
|
||||
// aggregation (group by and aggregation expr)
|
||||
if (aggInfo != null) {
|
||||
aggInfo.substitute(sMap, analyzer);
|
||||
}
|
||||
// aggregation (group by and aggregation expr)
|
||||
if (aggInfo != null) {
|
||||
aggInfo.substitute(sMap, analyzer);
|
||||
}
|
||||
|
||||
// having
|
||||
if (havingPred != null) {
|
||||
havingPred.substitute(sMap);
|
||||
}
|
||||
// having
|
||||
if (havingPred != null) {
|
||||
havingPred.substitute(sMap);
|
||||
}
|
||||
|
||||
// ordering
|
||||
//if (sortInfo != null) {
|
||||
// sortInfo.substitute(sMap);
|
||||
//}
|
||||
}
|
||||
// ordering
|
||||
//if (sortInfo != null) {
|
||||
// sortInfo.substitute(sMap);
|
||||
//}
|
||||
}
|
||||
|
||||
/**
|
||||
* Expand "*" select list item.
|
||||
@ -710,7 +745,7 @@ public class SelectStmt extends QueryStmt {
|
||||
if (analyzer.isSemiJoined(tableRef.getId())) {
|
||||
continue;
|
||||
}
|
||||
expandStar(new TableName(null,tableRef.getAlias()), tableRef.getDesc());
|
||||
expandStar(new TableName(null, tableRef.getAlias()), tableRef.getDesc());
|
||||
}
|
||||
}
|
||||
|
||||
@ -734,11 +769,11 @@ public class SelectStmt extends QueryStmt {
|
||||
private void expandStar(TableName tblName, TupleDescriptor desc) throws AnalysisException {
|
||||
for (Column col : desc.getTable().getBaseSchema()) {
|
||||
if (col.getDataType() == PrimitiveType.HLL && !fromInsert) {
|
||||
throw new AnalysisException (
|
||||
throw new AnalysisException(
|
||||
"hll only use in HLL_UNION_AGG or HLL_CARDINALITY , HLL_HASH and so on.");
|
||||
}
|
||||
if (col.getAggregationType() == AggregateType.BITMAP_UNION && !fromInsert) {
|
||||
throw new AnalysisException (
|
||||
throw new AnalysisException(
|
||||
"BITMAP_UNION agg column only use in TO_BITMAP or BITMAP_UNION , BITMAP_COUNT and so on.");
|
||||
}
|
||||
resultExprs.add(new SlotRef(tblName, col.getName()));
|
||||
@ -775,14 +810,16 @@ public class SelectStmt extends QueryStmt {
|
||||
}
|
||||
}
|
||||
|
||||
if (groupingExprs == null && !selectList.isDistinct()
|
||||
if (groupByClause == null && !selectList.isDistinct()
|
||||
&& !TreeNode.contains(resultExprs, Expr.isAggregatePredicate())
|
||||
&& (havingPred == null || !havingPred.contains(Expr.isAggregatePredicate()))
|
||||
&& (sortInfo == null || !TreeNode.contains(sortInfo.getOrderingExprs(),
|
||||
Expr.isAggregatePredicate()))) {
|
||||
// We're not computing aggregates but we still need to register the HAVING
|
||||
// clause which could, e.g., contain a constant expression evaluating to false.
|
||||
if (havingPred != null) analyzer.registerConjuncts(havingPred, true);
|
||||
if (havingPred != null) {
|
||||
analyzer.registerConjuncts(havingPred, true);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
@ -791,7 +828,7 @@ public class SelectStmt extends QueryStmt {
|
||||
throw new AnalysisException("Aggregation without a FROM clause is not allowed");
|
||||
}
|
||||
|
||||
if (selectList.isDistinct() && groupingExprs == null) {
|
||||
if (selectList.isDistinct() && groupByClause == null) {
|
||||
List<Expr> aggregateExpr = Lists.newArrayList();
|
||||
TreeNode.collect(resultExprs, Expr.isAggregatePredicate(), aggregateExpr);
|
||||
if (aggregateExpr.size() == resultExprs.size()) {
|
||||
@ -800,69 +837,21 @@ public class SelectStmt extends QueryStmt {
|
||||
}
|
||||
|
||||
if (selectList.isDistinct()
|
||||
&& (groupingExprs != null
|
||||
|| TreeNode.contains(resultExprs, Expr.isAggregatePredicate())
|
||||
|| (havingPred != null && havingPred.contains(Expr.isAggregatePredicate())))) {
|
||||
&& (groupByClause != null
|
||||
|| TreeNode.contains(resultExprs, Expr.isAggregatePredicate())
|
||||
|| (havingPred != null && havingPred.contains(Expr.isAggregatePredicate())))) {
|
||||
throw new AnalysisException("cannot combine SELECT DISTINCT with aggregate functions or GROUP BY");
|
||||
}
|
||||
|
||||
// disallow '*' and explicit GROUP BY (we can't group by '*', and if you need to
|
||||
// name all star-expanded cols in the group by clause you might as well do it
|
||||
// in the select list)
|
||||
if (groupingExprs != null ||
|
||||
if (groupByClause != null ||
|
||||
TreeNode.contains(resultExprs, Expr.isAggregatePredicate())) {
|
||||
for (SelectListItem item : selectList.getItems()) {
|
||||
if (item.isStar()) {
|
||||
throw new AnalysisException(
|
||||
"cannot combine '*' in select list with GROUP BY: " + item.toSql());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// disallow subqueries in the GROUP BY clause
|
||||
if (groupingExprs != null) {
|
||||
for (Expr expr: groupingExprs) {
|
||||
if (expr.contains(Predicates.instanceOf(Subquery.class))) {
|
||||
throw new AnalysisException(
|
||||
"Subqueries are not supported in the GROUP BY clause.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// analyze grouping exprs
|
||||
ArrayList<Expr> groupingExprsCopy = Lists.newArrayList();
|
||||
if (groupingExprs != null) {
|
||||
// make a deep copy here, we don't want to modify the original
|
||||
// exprs during analysis (in case we need to print them later)
|
||||
groupingExprsCopy = Expr.cloneList(groupingExprs);
|
||||
|
||||
substituteOrdinalsAliases(groupingExprsCopy, "GROUP BY", analyzer);
|
||||
|
||||
for (int i = 0; i < groupingExprsCopy.size(); ++i) {
|
||||
groupingExprsCopy.get(i).analyze(analyzer);
|
||||
if (groupingExprsCopy.get(i).contains(Expr.isAggregatePredicate())) {
|
||||
// reference the original expr in the error msg
|
||||
throw new AnalysisException(
|
||||
"GROUP BY expression must not contain aggregate functions: "
|
||||
+ groupingExprs.get(i).toSql());
|
||||
}
|
||||
if (groupingExprsCopy.get(i).contains(AnalyticExpr.class)) {
|
||||
// reference the original expr in the error msg
|
||||
throw new AnalysisException(
|
||||
"GROUP BY expression must not contain analytic expressions: "
|
||||
+ groupingExprsCopy.get(i).toSql());
|
||||
}
|
||||
|
||||
if (groupingExprsCopy.get(i).type.isHllType()) {
|
||||
throw new AnalysisException(
|
||||
"GROUP BY expression must not contain hll column: "
|
||||
+ groupingExprsCopy.get(i).toSql());
|
||||
}
|
||||
|
||||
if (groupingExprsCopy.get(i).type.isBitmapType()) {
|
||||
throw new AnalysisException(
|
||||
"GROUP BY expression must not contain bitmap column: "
|
||||
+ groupingExprsCopy.get(i).toSql());
|
||||
"cannot combine '*' in select list with GROUP BY: " + item.toSql());
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -890,24 +879,45 @@ public class SelectStmt extends QueryStmt {
|
||||
// i) There is no GROUP-BY clause, and
|
||||
// ii) Other DISTINCT aggregates are present.
|
||||
ExprSubstitutionMap countAllMap = createCountAllMap(aggExprs, analyzer);
|
||||
final ExprSubstitutionMap multiCountOrSumDistinctMap =
|
||||
final ExprSubstitutionMap multiCountOrSumDistinctMap =
|
||||
createSumOrCountMultiDistinctSMap(aggExprs, analyzer);
|
||||
countAllMap = ExprSubstitutionMap.compose(multiCountOrSumDistinctMap, countAllMap, analyzer);
|
||||
List<Expr> substitutedAggs =
|
||||
Expr.substituteList(aggExprs, countAllMap, analyzer, false);
|
||||
aggExprs.clear();
|
||||
TreeNode.collect(substitutedAggs, Expr.isAggregatePredicate(), aggExprs);
|
||||
createAggInfo(groupingExprsCopy, aggExprs, analyzer);
|
||||
|
||||
List<TupleId> groupingByTupleIds = new ArrayList<>();
|
||||
if (groupByClause != null) {
|
||||
// must do it before copying for createAggInfo()
|
||||
if (groupingInfo != null) {
|
||||
groupingByTupleIds.add(groupingInfo.getVirtualTuple().getId());
|
||||
}
|
||||
groupByClause.genGroupingExprs();
|
||||
if (groupingInfo != null) {
|
||||
GroupByClause.GroupingType groupingType = groupByClause.getGroupingType();
|
||||
if ((groupingType == GroupByClause.GroupingType.GROUPING_SETS && CollectionUtils
|
||||
.isNotEmpty(groupByClause.getGroupingSetList()))
|
||||
|| groupingType == GroupByClause.GroupingType.CUBE
|
||||
|| groupingType == GroupByClause.GroupingType.ROLLUP) {
|
||||
|
||||
}
|
||||
groupingInfo.buildRepeat(groupByClause.getGroupingExprs(), groupByClause.getGroupingSetList());
|
||||
}
|
||||
substituteOrdinalsAliases(groupByClause.getGroupingExprs(), "GROUP BY", analyzer);
|
||||
groupByClause.analyze(analyzer);
|
||||
createAggInfo(groupByClause.getGroupingExprs(), aggExprs, analyzer);
|
||||
} else {
|
||||
createAggInfo( new ArrayList<>(), aggExprs, analyzer);
|
||||
}
|
||||
|
||||
// combine avg smap with the one that produces the final agg output
|
||||
AggregateInfo finalAggInfo =
|
||||
aggInfo.getSecondPhaseDistinctAggInfo() != null
|
||||
? aggInfo.getSecondPhaseDistinctAggInfo()
|
||||
: aggInfo;
|
||||
|
||||
ExprSubstitutionMap combinedSmap =
|
||||
ExprSubstitutionMap.compose(countAllMap, finalAggInfo.getOutputSmap(), analyzer);
|
||||
|
||||
groupingByTupleIds.add(finalAggInfo.getOutputTupleId());
|
||||
ExprSubstitutionMap combinedSmap = ExprSubstitutionMap.compose(countAllMap, finalAggInfo.getOutputSmap(), analyzer);
|
||||
// change select list, having and ordering exprs to point to agg output. We need
|
||||
// to reanalyze the exprs at this point.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
@ -941,42 +951,42 @@ public class SelectStmt extends QueryStmt {
|
||||
|
||||
// check that all post-agg exprs point to agg output
|
||||
for (int i = 0; i < selectList.getItems().size(); ++i) {
|
||||
if (!resultExprs.get(i).isBound(finalAggInfo.getOutputTupleId())) {
|
||||
if (!resultExprs.get(i).isBoundByTupleIds(groupingByTupleIds)) {
|
||||
throw new AnalysisException(
|
||||
"select list expression not produced by aggregation output " + "(missing from " +
|
||||
"GROUP BY clause?): " + selectList.getItems().get(
|
||||
i).getExpr().toSql());
|
||||
"select list expression not produced by aggregation output " + "(missing from " +
|
||||
"GROUP BY clause?): " + selectList.getItems().get(i).getExpr().toSql());
|
||||
}
|
||||
}
|
||||
if (orderByElements != null) {
|
||||
for (int i = 0; i < orderByElements.size(); ++i) {
|
||||
if (!sortInfo.getOrderingExprs().get(i).isBound(finalAggInfo.getOutputTupleId())) {
|
||||
if (!sortInfo.getOrderingExprs().get(i).isBoundByTupleIds(groupingByTupleIds)) {
|
||||
throw new AnalysisException(
|
||||
"ORDER BY expression not produced by aggregation output " + "(missing from " +
|
||||
"GROUP BY clause?): " + orderByElements.get(
|
||||
i).getExpr().toSql());
|
||||
"ORDER BY expression not produced by aggregation output " + "(missing from " +
|
||||
"GROUP BY clause?): " + orderByElements.get(i).getExpr().toSql());
|
||||
}
|
||||
|
||||
if (sortInfo.getOrderingExprs().get(i).type.isHllType()) {
|
||||
throw new AnalysisException(
|
||||
"ORDER BY expression could not contain hll column.");
|
||||
throw new AnalysisException("ORDER BY expression could not contain hll column.");
|
||||
}
|
||||
|
||||
if (sortInfo.getOrderingExprs().get(i).type.isBitmapType()) {
|
||||
throw new AnalysisException(
|
||||
"ORDER BY expression could not contain bitmap column.");
|
||||
throw new AnalysisException("ORDER BY expression could not contain bitmap column.");
|
||||
}
|
||||
}
|
||||
}
|
||||
if (havingPred != null) {
|
||||
if (!havingPred.isBound(finalAggInfo.getOutputTupleId())) {
|
||||
if (!havingPred.isBoundByTupleIds(groupingByTupleIds)) {
|
||||
throw new AnalysisException(
|
||||
"HAVING clause not produced by aggregation output " + "(missing from GROUP BY " +
|
||||
"clause?): " + havingClause.toSql());
|
||||
"HAVING clause not produced by aggregation output " + "(missing from GROUP BY " +
|
||||
"clause?): " + havingClause.toSql());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void createGroupingInfo(Analyzer analyzer) throws AnalysisException {
|
||||
groupingInfo = new GroupingInfo(analyzer, groupByClause.getGroupingType());
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Build smap count_distinct->multi_count_distinct sum_distinct->multi_count_distinct
|
||||
@ -986,9 +996,9 @@ public class SelectStmt extends QueryStmt {
|
||||
ArrayList<FunctionCallExpr> aggExprs, Analyzer analyzer) throws AnalysisException {
|
||||
final List<FunctionCallExpr> distinctExprs = Lists.newArrayList();
|
||||
for (FunctionCallExpr aggExpr : aggExprs) {
|
||||
if (aggExpr.isDistinct()) {
|
||||
distinctExprs.add(aggExpr);
|
||||
}
|
||||
if (aggExpr.isDistinct()) {
|
||||
distinctExprs.add(aggExpr);
|
||||
}
|
||||
}
|
||||
final ExprSubstitutionMap result = new ExprSubstitutionMap();
|
||||
final boolean hasMultiDistinct = AggregateInfo.estimateIfContainsMultiDistinct(distinctExprs);
|
||||
@ -1010,9 +1020,9 @@ public class SelectStmt extends QueryStmt {
|
||||
final List<Expr> sumInputExprs = Lists.newArrayList(inputExpr.getChild(0).clone(null));
|
||||
final List<Expr> countInputExpr = Lists.newArrayList(inputExpr.getChild(0).clone(null));
|
||||
final FunctionCallExpr sumExpr = new FunctionCallExpr("MULTI_DISTINCT_SUM",
|
||||
new FunctionParams(inputExpr.isDistinct(), sumInputExprs));
|
||||
new FunctionParams(inputExpr.isDistinct(), sumInputExprs));
|
||||
final FunctionCallExpr countExpr = new FunctionCallExpr("MULTI_DISTINCT_COUNT",
|
||||
new FunctionParams(inputExpr.isDistinct(), countInputExpr));
|
||||
new FunctionParams(inputExpr.isDistinct(), countInputExpr));
|
||||
replaceExpr = new ArithmeticExpr(ArithmeticExpr.Operator.DIVIDE, sumExpr, countExpr);
|
||||
} else {
|
||||
throw new AnalysisException(inputExpr.getFnName() + " can't support multi distinct.");
|
||||
@ -1021,7 +1031,9 @@ public class SelectStmt extends QueryStmt {
|
||||
replaceExpr.analyze(analyzer);
|
||||
result.put(inputExpr, replaceExpr);
|
||||
}
|
||||
if (LOG.isDebugEnabled()) LOG.debug("multi distinct smap: {}", result.debugString());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("multi distinct smap: {}", result.debugString());
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
@ -1037,7 +1049,7 @@ public class SelectStmt extends QueryStmt {
|
||||
throws AnalysisException {
|
||||
ExprSubstitutionMap scalarCountAllMap = new ExprSubstitutionMap();
|
||||
|
||||
if (groupingExprs != null && !groupingExprs.isEmpty()) {
|
||||
if (groupByClause != null && !groupByClause.isEmpty()) {
|
||||
// There are grouping expressions, so no substitution needs to be done.
|
||||
return scalarCountAllMap;
|
||||
}
|
||||
@ -1062,7 +1074,7 @@ public class SelectStmt extends QueryStmt {
|
||||
|
||||
Iterable<FunctionCallExpr> countAllAggs =
|
||||
Iterables.filter(aggExprs, Predicates.and(isCountPred, isNotDistinctPred));
|
||||
for (FunctionCallExpr countAllAgg: countAllAggs) {
|
||||
for (FunctionCallExpr countAllAgg : countAllAggs) {
|
||||
// TODO(zc)
|
||||
// Replace COUNT(ALL) with zeroifnull(COUNT(ALL))
|
||||
ArrayList<Expr> zeroIfNullParam = Lists.newArrayList(countAllAgg.clone(), new IntLiteral(0, Type.BIGINT));
|
||||
@ -1110,8 +1122,8 @@ public class SelectStmt extends QueryStmt {
|
||||
return;
|
||||
}
|
||||
ExprSubstitutionMap rewriteSmap = new ExprSubstitutionMap();
|
||||
for (Expr expr: analyticExprs) {
|
||||
AnalyticExpr toRewrite = (AnalyticExpr)expr;
|
||||
for (Expr expr : analyticExprs) {
|
||||
AnalyticExpr toRewrite = (AnalyticExpr) expr;
|
||||
Expr newExpr = AnalyticExpr.rewrite(toRewrite);
|
||||
if (newExpr != null) {
|
||||
newExpr.analyze(analyzer);
|
||||
@ -1157,20 +1169,29 @@ public class SelectStmt extends QueryStmt {
|
||||
public void rewriteExprs(ExprRewriter rewriter) throws AnalysisException {
|
||||
Preconditions.checkState(isAnalyzed());
|
||||
selectList.rewriteExprs(rewriter, analyzer);
|
||||
for (TableRef ref: fromClause_) ref.rewriteExprs(rewriter, analyzer);
|
||||
for (TableRef ref : fromClause_) {
|
||||
ref.rewriteExprs(rewriter, analyzer);
|
||||
}
|
||||
if (whereClause != null) {
|
||||
whereClause = rewriter.rewrite(whereClause, analyzer);
|
||||
// Also rewrite exprs in the statements of subqueries.
|
||||
List<Subquery> subqueryExprs = Lists.newArrayList();
|
||||
whereClause.collect(Subquery.class, subqueryExprs);
|
||||
for (Subquery s: subqueryExprs) s.getStatement().rewriteExprs(rewriter);
|
||||
for (Subquery s : subqueryExprs) {
|
||||
s.getStatement().rewriteExprs(rewriter);
|
||||
}
|
||||
}
|
||||
if (havingClause != null) {
|
||||
havingClause = rewriter.rewrite(havingClause, analyzer);
|
||||
}
|
||||
if (groupingExprs != null) rewriter.rewriteList(groupingExprs, analyzer);
|
||||
if (groupByClause != null) {
|
||||
ArrayList<Expr> groupingExprs = groupByClause.getGroupingExprs();
|
||||
if (groupingExprs != null) {
|
||||
rewriter.rewriteList(groupingExprs, analyzer);
|
||||
}
|
||||
}
|
||||
if (orderByElements != null) {
|
||||
for (OrderByElement orderByElem: orderByElements) {
|
||||
for (OrderByElement orderByElem : orderByElements) {
|
||||
orderByElem.setExpr(rewriter.rewrite(orderByElem.getExpr(), analyzer));
|
||||
}
|
||||
}
|
||||
@ -1207,7 +1228,9 @@ public class SelectStmt extends QueryStmt {
|
||||
}
|
||||
|
||||
// From clause
|
||||
if (!fromClause_.isEmpty()) { strBuilder.append(fromClause_.toSql()); }
|
||||
if (!fromClause_.isEmpty()) {
|
||||
strBuilder.append(fromClause_.toSql());
|
||||
}
|
||||
|
||||
// Where clause
|
||||
if (whereClause != null) {
|
||||
@ -1215,12 +1238,9 @@ public class SelectStmt extends QueryStmt {
|
||||
strBuilder.append(whereClause.toSql());
|
||||
}
|
||||
// Group By clause
|
||||
if (groupingExprs != null) {
|
||||
if (groupByClause != null) {
|
||||
strBuilder.append(" GROUP BY ");
|
||||
for (int i = 0; i < groupingExprs.size(); ++i) {
|
||||
strBuilder.append(groupingExprs.get(i).toSql());
|
||||
strBuilder.append((i + 1 != groupingExprs.size()) ? ", " : "");
|
||||
}
|
||||
strBuilder.append(groupByClause.toSql());
|
||||
}
|
||||
// Having clause
|
||||
if (havingClause != null) {
|
||||
@ -1299,7 +1319,7 @@ public class SelectStmt extends QueryStmt {
|
||||
}
|
||||
} else {
|
||||
// to make sure the sortinfo's AnalyticExpr and resultExprs's AnalyticExpr analytic once
|
||||
if(item.getExpr() instanceof AnalyticExpr){
|
||||
if (item.getExpr() instanceof AnalyticExpr) {
|
||||
item.getExpr().analyze(analyzer);
|
||||
}
|
||||
if (item.getAlias() != null) {
|
||||
@ -1312,8 +1332,8 @@ public class SelectStmt extends QueryStmt {
|
||||
}
|
||||
}
|
||||
// substitute group by
|
||||
if (groupingExprs != null) {
|
||||
substituteOrdinalsAliases(groupingExprs, "GROUP BY", analyzer);
|
||||
if (groupByClause != null) {
|
||||
substituteOrdinalsAliases(groupByClause.getGroupingExprs(), "GROUP BY", analyzer);
|
||||
}
|
||||
// substitute having
|
||||
if (havingClause != null) {
|
||||
@ -1330,33 +1350,48 @@ public class SelectStmt extends QueryStmt {
|
||||
colLabels.addAll(newColLabels);
|
||||
}
|
||||
|
||||
public boolean hasWhereClause() { return whereClause != null; }
|
||||
public boolean hasAggInfo() { return aggInfo != null; }
|
||||
public boolean hasGroupByClause() { return groupingExprs != null; }
|
||||
public boolean hasWhereClause() {
|
||||
return whereClause != null;
|
||||
}
|
||||
|
||||
public boolean hasAggInfo() {
|
||||
return aggInfo != null;
|
||||
}
|
||||
|
||||
public boolean hasGroupByClause() {
|
||||
return groupByClause != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the stmt returns a single row. This can happen
|
||||
* in the following cases:
|
||||
* 1. select stmt with a 'limit 1' clause
|
||||
* 2. select stmt with an aggregate function and no group by.
|
||||
* 3. select stmt with no from clause.
|
||||
*
|
||||
* <p>
|
||||
* This function may produce false negatives because the cardinality of the
|
||||
* result set also depends on the data a stmt is processing.
|
||||
*/
|
||||
public boolean returnsSingleRow() {
|
||||
// limit 1 clause
|
||||
if (hasLimitClause() && getLimit() == 1) { return true; }
|
||||
if (hasLimitClause() && getLimit() == 1) {
|
||||
return true;
|
||||
}
|
||||
// No from clause (base tables or inline views)
|
||||
if (fromClause_.isEmpty()) { return true; }
|
||||
if (fromClause_.isEmpty()) {
|
||||
return true;
|
||||
}
|
||||
// Aggregation with no group by and no DISTINCT
|
||||
if (hasAggInfo() && !hasGroupByClause() && !selectList.isDistinct()) { return true; }
|
||||
if (hasAggInfo() && !hasGroupByClause() && !selectList.isDistinct()) {
|
||||
return true;
|
||||
}
|
||||
// In all other cases, return false.
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collectTableRefs(List<TableRef> tblRefs) {
|
||||
for (TableRef tblRef: fromClause_) {
|
||||
for (TableRef tblRef : fromClause_) {
|
||||
if (tblRef instanceof InlineViewRef) {
|
||||
InlineViewRef inlineViewRef = (InlineViewRef) tblRef;
|
||||
inlineViewRef.getViewStmt().collectTableRefs(tblRefs);
|
||||
@ -1366,4 +1401,16 @@ public class SelectStmt extends QueryStmt {
|
||||
}
|
||||
}
|
||||
|
||||
private boolean checkGroupingFn(Expr expr) {
|
||||
if (expr instanceof GroupingFunctionCallExpr) {
|
||||
return true;
|
||||
} else if (expr.getChildren().size() > 0) {
|
||||
for (Expr child : expr.getChildren()) {
|
||||
if (checkGroupingFn(child)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@ -44,7 +44,7 @@ public class SlotRef extends Expr {
|
||||
private String label;
|
||||
|
||||
// results of analysis
|
||||
private SlotDescriptor desc;
|
||||
protected SlotDescriptor desc;
|
||||
|
||||
// Only used write
|
||||
private SlotRef() {
|
||||
@ -200,7 +200,7 @@ public class SlotRef extends Expr {
|
||||
if (desc != null) {
|
||||
return desc.getId().hashCode();
|
||||
}
|
||||
return Objects.hashCode((tblName.toSql() + "." + label).toLowerCase());
|
||||
return Objects.hashCode((tblName == null ? "" : tblName.toSql() + "." + label).toLowerCase());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@ -877,9 +877,9 @@ public class StmtRewriter {
|
||||
// Update subquery's GROUP BY clause
|
||||
if (groupByExprs != null && !groupByExprs.isEmpty()) {
|
||||
if (stmt.hasGroupByClause()) {
|
||||
stmt.groupingExprs.addAll(groupByExprs);
|
||||
stmt.groupByClause.getGroupingExprs().addAll(groupByExprs);
|
||||
} else {
|
||||
stmt.groupingExprs = groupByExprs;
|
||||
stmt.groupByClause = new GroupByClause(groupByExprs, GroupByClause.GroupingType.GROUP_BY);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -36,6 +36,7 @@ import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class StringLiteral extends LiteralExpr {
|
||||
private static final Logger LOG = LogManager.getLogger(StringLiteral.class);
|
||||
@ -228,4 +229,9 @@ public class StringLiteral extends LiteralExpr {
|
||||
literal.readFields(in);
|
||||
return literal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 31 * super.hashCode() + Objects.hashCode(value);
|
||||
}
|
||||
}
|
||||
|
||||
107
fe/src/main/java/org/apache/doris/analysis/VirtualSlotRef.java
Normal file
107
fe/src/main/java/org/apache/doris/analysis/VirtualSlotRef.java
Normal file
@ -0,0 +1,107 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import org.apache.doris.catalog.Type;
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
|
||||
import org.apache.commons.collections.CollectionUtils;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* It like a SlotRef except that it is not a real column exist in table.
|
||||
*/
|
||||
public class VirtualSlotRef extends SlotRef {
|
||||
private static final Logger LOG = LogManager.getLogger(VirtualSlotRef.class);
|
||||
// results of analysis slot
|
||||
|
||||
private TupleDescriptor tupleDescriptor;
|
||||
private List<Expr> realSlots;
|
||||
|
||||
public VirtualSlotRef(String col, Type type, TupleDescriptor tupleDescriptor, List<Expr> realSlots) {
|
||||
super(null, col);
|
||||
super.type = type;
|
||||
this.tupleDescriptor = tupleDescriptor;
|
||||
this.realSlots = realSlots;
|
||||
}
|
||||
|
||||
protected VirtualSlotRef(VirtualSlotRef other) {
|
||||
super(other);
|
||||
if (other.realSlots != null) {
|
||||
realSlots = Expr.cloneList(other.realSlots);
|
||||
}
|
||||
tupleDescriptor = other.tupleDescriptor;
|
||||
}
|
||||
|
||||
public static VirtualSlotRef read(DataInput in) throws IOException {
|
||||
VirtualSlotRef virtualSlotRef = new VirtualSlotRef(null, Type.BIGINT, null, new ArrayList<>());
|
||||
virtualSlotRef.readFields(in);
|
||||
return virtualSlotRef;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
super.write(out);
|
||||
if (CollectionUtils.isEmpty(realSlots)) {
|
||||
out.writeInt(0);
|
||||
} else {
|
||||
out.writeInt(realSlots.size());
|
||||
for (Expr slotRef : realSlots) {
|
||||
slotRef.write(out);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
super.readFields(in);
|
||||
int realSlotsSize = in.readInt();
|
||||
if (realSlotsSize > 0) {
|
||||
for (int i = 0; i < realSlotsSize; i++) {
|
||||
realSlots.add(SlotRef.read(in));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public List<Expr> getRealSlots() {
|
||||
return realSlots;
|
||||
}
|
||||
|
||||
public void setRealSlots(List<Expr> realSlots) {
|
||||
this.realSlots = realSlots;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Expr clone() {
|
||||
return new VirtualSlotRef(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void analyzeImpl(Analyzer analyzer) throws AnalysisException {
|
||||
desc = analyzer.registerVirtualColumnRef(super.getColumnName(), type, tupleDescriptor);
|
||||
numDistinctValues = desc.getStats().getNumDistinctValues();
|
||||
}
|
||||
}
|
||||
@ -17,19 +17,20 @@
|
||||
|
||||
package org.apache.doris.catalog;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.doris.thrift.TColumnType;
|
||||
import org.apache.doris.thrift.TScalarType;
|
||||
import org.apache.doris.thrift.TTypeDesc;
|
||||
import org.apache.doris.thrift.TTypeNode;
|
||||
import org.apache.doris.thrift.TTypeNodeType;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.gson.annotations.SerializedName;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Describes a scalar type. For most types this class just wraps a PrimitiveType enum,
|
||||
* but for types like CHAR and DECIMAL, this class contain additional information.
|
||||
@ -274,7 +275,7 @@ public class ScalarType extends Type {
|
||||
public static ScalarType createVarcharType() {
|
||||
return DEFAULT_VARCHAR;
|
||||
}
|
||||
|
||||
|
||||
public static ScalarType createHllType() {
|
||||
ScalarType type = new ScalarType(PrimitiveType.HLL);
|
||||
type.len = MAX_HLL_LENGTH;
|
||||
@ -361,7 +362,7 @@ public class ScalarType extends Type {
|
||||
container.types.add(node);
|
||||
switch(type) {
|
||||
case VARCHAR:
|
||||
case CHAR:
|
||||
case CHAR:
|
||||
case HLL: {
|
||||
node.setType(TTypeNodeType.SCALAR);
|
||||
TScalarType scalarType = new TScalarType();
|
||||
@ -659,12 +660,12 @@ public class ScalarType extends Type {
|
||||
|
||||
if ((t1.isDecimal() || t1.isDecimalV2()) && t2.isDate()
|
||||
|| t1.isDate() && (t2.isDecimal() || t2.isDecimalV2())) {
|
||||
return INVALID;
|
||||
return INVALID;
|
||||
}
|
||||
|
||||
if (t1.isDecimalV2() || t2.isDecimalV2()) {
|
||||
return DECIMALV2;
|
||||
}
|
||||
}
|
||||
|
||||
if (t1.isDecimal() || t2.isDecimal()) {
|
||||
return DECIMAL;
|
||||
@ -777,4 +778,13 @@ public class ScalarType extends Type {
|
||||
}
|
||||
return thrift;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = 0;
|
||||
result = 31 * result + Objects.hashCode(type);
|
||||
result = 31 * result + precision;
|
||||
result = 31 * result + scale;
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
@ -17,29 +17,28 @@
|
||||
|
||||
package org.apache.doris.planner;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.doris.analysis.AggregateInfo;
|
||||
import org.apache.doris.analysis.Analyzer;
|
||||
import org.apache.doris.analysis.Expr;
|
||||
import org.apache.doris.analysis.FunctionCallExpr;
|
||||
import org.apache.doris.analysis.SlotId;
|
||||
//import org.apache.doris.thrift.TAggregateFunctionCall;
|
||||
import org.apache.doris.thrift.TExpr;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.thrift.TAggregationNode;
|
||||
import org.apache.doris.thrift.TExplainLevel;
|
||||
import org.apache.doris.thrift.TExpr;
|
||||
import org.apache.doris.thrift.TPlanNode;
|
||||
import org.apache.doris.thrift.TPlanNodeType;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
//import org.apache.doris.thrift.TAggregateFunctionCall;
|
||||
|
||||
/**
|
||||
* Aggregation computation.
|
||||
@ -128,11 +127,11 @@ public class AggregationNode extends PlanNode {
|
||||
// conjuncts bound by those grouping slots in createEquivConjuncts() (IMPALA-2089).
|
||||
// Those conjuncts cannot be redundant because our equivalence classes do not
|
||||
// capture dependencies with non-SlotRef exprs.
|
||||
Set<SlotId> groupBySlots = Sets.newHashSet();
|
||||
for (int i = 0; i < aggInfo.getGroupingExprs().size(); ++i) {
|
||||
if (aggInfo.getGroupingExprs().get(i).unwrapSlotRef(true) == null) continue;
|
||||
groupBySlots.add(aggInfo.getOutputTupleDesc().getSlots().get(i).getId());
|
||||
}
|
||||
// Set<SlotId> groupBySlots = Sets.newHashSet();
|
||||
// for (int i = 0; i < aggInfo.getGroupingExprs().size(); ++i) {
|
||||
// if (aggInfo.getGroupingExprs().get(i).unwrapSlotRef(true) == null) continue;
|
||||
// groupBySlots.add(aggInfo.getOutputTupleDesc().getSlots().get(i).getId());
|
||||
// }
|
||||
// ArrayList<Expr> bindingPredicates =
|
||||
// analyzer.getBoundPredicates(tupleIds.get(0), groupBySlots, true);
|
||||
ArrayList<Expr> bindingPredicates = Lists.newArrayList();
|
||||
|
||||
@ -223,6 +223,8 @@ public class DistributedPlanner {
|
||||
result = createAnalyticFragment(root, childFragments.get(0), fragments);
|
||||
} else if (root instanceof EmptySetNode) {
|
||||
result = new PlanFragment(ctx_.getNextFragmentId(), root, DataPartition.UNPARTITIONED);
|
||||
} else if (root instanceof RepeatNode) {
|
||||
result = createRepeatNodeFragment((RepeatNode) root, childFragments.get(0), fragments);
|
||||
} else if (root instanceof AssertNumRowsNode) {
|
||||
result = createAssertFragment(root, childFragments.get(0));
|
||||
} else {
|
||||
@ -774,6 +776,14 @@ public class DistributedPlanner {
|
||||
}
|
||||
}
|
||||
|
||||
private PlanFragment createRepeatNodeFragment(
|
||||
RepeatNode repeatNode, PlanFragment childFragment, ArrayList<PlanFragment> fragments)
|
||||
throws UserException {
|
||||
repeatNode.setNumInstances(childFragment.getPlanRoot().getNumInstances());
|
||||
childFragment.addPlanRoot(repeatNode);
|
||||
return childFragment;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a fragment that materializes the final result of an aggregation where
|
||||
* 'childFragment' is a partitioned fragment and 'node' is not part of a distinct
|
||||
|
||||
@ -17,6 +17,7 @@
|
||||
|
||||
package org.apache.doris.planner;
|
||||
|
||||
import org.apache.commons.collections.CollectionUtils;
|
||||
import org.apache.doris.analysis.Analyzer;
|
||||
import org.apache.doris.analysis.Expr;
|
||||
import org.apache.doris.common.UserException;
|
||||
@ -32,6 +33,7 @@ import org.apache.logging.log4j.LogManager;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* PlanFragments form a tree structure via their ExchangeNodes. A tree of fragments
|
||||
@ -213,10 +215,9 @@ public class PlanFragment extends TreeNode<PlanFragment> {
|
||||
StringBuilder str = new StringBuilder();
|
||||
Preconditions.checkState(dataPartition != null);
|
||||
str.append(" OUTPUT EXPRS:");
|
||||
if (outputExprs != null) {
|
||||
for (Expr e : outputExprs) {
|
||||
str.append(e.toSql() + " | ");
|
||||
}
|
||||
if (CollectionUtils.isNotEmpty(outputExprs)) {
|
||||
str.append(outputExprs.stream().map(Expr::toSql)
|
||||
.collect(Collectors.joining(" | ")));
|
||||
}
|
||||
str.append("\n");
|
||||
str.append(" PARTITION: " + dataPartition.getExplainString(explainLevel) + "\n");
|
||||
|
||||
197
fe/src/main/java/org/apache/doris/planner/RepeatNode.java
Normal file
197
fe/src/main/java/org/apache/doris/planner/RepeatNode.java
Normal file
@ -0,0 +1,197 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
package org.apache.doris.planner;
|
||||
|
||||
import org.apache.doris.analysis.Analyzer;
|
||||
import org.apache.doris.analysis.Expr;
|
||||
import org.apache.doris.analysis.GroupByClause;
|
||||
import org.apache.doris.analysis.GroupingFunctionCallExpr;
|
||||
import org.apache.doris.analysis.GroupingInfo;
|
||||
import org.apache.doris.analysis.SlotDescriptor;
|
||||
import org.apache.doris.analysis.SlotId;
|
||||
import org.apache.doris.analysis.SlotRef;
|
||||
import org.apache.doris.analysis.TupleDescriptor;
|
||||
import org.apache.doris.analysis.TupleId;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.thrift.TExplainLevel;
|
||||
import org.apache.doris.thrift.TPlanNode;
|
||||
import org.apache.doris.thrift.TPlanNodeType;
|
||||
import org.apache.doris.thrift.TRepeatNode;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import org.apache.commons.collections.CollectionUtils;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Used for grouping sets.
|
||||
* It will add some new rows and a column of groupingId according to grouping sets info.
|
||||
*/
|
||||
public class RepeatNode extends PlanNode {
|
||||
|
||||
private List<Set<Integer>> repeatSlotIdList;
|
||||
private TupleDescriptor outputTupleDesc;
|
||||
private List<List<Long>> groupingList;
|
||||
private GroupingInfo groupingInfo;
|
||||
private PlanNode input;
|
||||
private GroupByClause groupByClause;
|
||||
|
||||
protected RepeatNode(PlanNodeId id, PlanNode input, GroupingInfo groupingInfo, GroupByClause groupByClause) {
|
||||
super(id, input.getTupleIds(), "REPEAT_NODE");
|
||||
this.children.add(input);
|
||||
this.groupingInfo = groupingInfo;
|
||||
this.input = input;
|
||||
this.groupByClause = groupByClause;
|
||||
|
||||
}
|
||||
|
||||
// only for unittest
|
||||
protected RepeatNode(PlanNodeId id, PlanNode input, List<Set<SlotId>> repeatSlotIdList,
|
||||
TupleDescriptor outputTupleDesc, List<List<Long>> groupingList) {
|
||||
super(id, input.getTupleIds(), "REPEAT_NODE");
|
||||
this.children.add(input);
|
||||
this.repeatSlotIdList = buildIdSetList(repeatSlotIdList);
|
||||
this.groupingList = groupingList;
|
||||
this.outputTupleDesc = outputTupleDesc;
|
||||
tupleIds.add(outputTupleDesc.getId());
|
||||
}
|
||||
|
||||
private static List<Set<Integer>> buildIdSetList(List<Set<SlotId>> repeatSlotIdList) {
|
||||
List<Set<Integer>> slotIdList = new ArrayList<>();
|
||||
for (Set slotSet : repeatSlotIdList) {
|
||||
Set<Integer> intSet = new HashSet<>();
|
||||
for (Object slotId : slotSet) {
|
||||
intSet.add(((SlotId) slotId).asInt());
|
||||
}
|
||||
slotIdList.add(intSet);
|
||||
}
|
||||
return slotIdList;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void computeStats(Analyzer analyzer) {
|
||||
avgRowSize = 0;
|
||||
cardinality = 0;
|
||||
numNodes = 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(Analyzer analyzer) throws UserException {
|
||||
Preconditions.checkState(conjuncts.isEmpty());
|
||||
groupByClause.substituteGroupingExprs(groupingInfo.getGroupingSlots(), input.getOutputSmap(),
|
||||
analyzer);
|
||||
|
||||
for (Expr expr : groupByClause.getGroupingExprs()) {
|
||||
if (expr instanceof SlotRef || (expr instanceof GroupingFunctionCallExpr)) {
|
||||
continue;
|
||||
}
|
||||
// throw new AnalysisException("function or expr is not allowed in grouping sets clause.");
|
||||
}
|
||||
|
||||
// build new BitSet List for tupleDesc
|
||||
Set<SlotDescriptor> slotDescSet = new HashSet<>();
|
||||
for (TupleId tupleId : input.getTupleIds()) {
|
||||
TupleDescriptor tupleDescriptor = analyzer.getDescTbl().getTupleDesc(tupleId);
|
||||
slotDescSet.addAll(tupleDescriptor.getSlots());
|
||||
}
|
||||
|
||||
// build tupleDesc according to child's tupleDesc info
|
||||
outputTupleDesc = groupingInfo.getVirtualTuple();
|
||||
//set aggregate nullable
|
||||
for (Expr slot : groupByClause.getGroupingExprs()) {
|
||||
if (slot instanceof SlotRef) {
|
||||
((SlotRef) slot).getDesc().setIsNullable(true);
|
||||
}
|
||||
}
|
||||
outputTupleDesc.computeMemLayout();
|
||||
|
||||
List<Set<SlotId>> groupingIdList = new ArrayList<>();
|
||||
List<Expr> exprList = groupByClause.getGroupingExprs();
|
||||
Preconditions.checkState(exprList.size() >= 2);
|
||||
for (BitSet bitSet : Collections.unmodifiableList(groupingInfo.getGroupingIdList())) {
|
||||
Set<SlotId> slotIdSet = new HashSet<>();
|
||||
for (SlotDescriptor slotDesc : slotDescSet) {
|
||||
SlotId slotId = slotDesc.getId();
|
||||
if (slotId == null) {
|
||||
continue;
|
||||
}
|
||||
for (int i = 0; i < exprList.size(); i++) {
|
||||
if (exprList.get(i) instanceof SlotRef) {
|
||||
SlotRef slotRef = (SlotRef) (exprList.get(i));
|
||||
if (bitSet.get(i) && slotRef.getSlotId() == slotId) {
|
||||
slotIdSet.add(slotId);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
groupingIdList.add(slotIdSet);
|
||||
}
|
||||
this.repeatSlotIdList = buildIdSetList(groupingIdList);
|
||||
this.groupingList = groupingInfo.genGroupingList(groupByClause.getGroupingExprs());
|
||||
tupleIds.add(outputTupleDesc.getId());
|
||||
for (TupleId id : tupleIds) {
|
||||
analyzer.getTupleDesc(id).setIsMaterialized(true);
|
||||
}
|
||||
computeMemLayout(analyzer);
|
||||
computeStats(analyzer);
|
||||
createDefaultSmap(analyzer);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void toThrift(TPlanNode msg) {
|
||||
msg.node_type = TPlanNodeType.REPEAT_NODE;
|
||||
msg.repeat_node = new TRepeatNode(outputTupleDesc.getId().asInt(), repeatSlotIdList, groupingList.get(0),
|
||||
groupingList);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String debugString() {
|
||||
return Objects.toStringHelper(this).add("Repeat", repeatSlotIdList.size()).addValue(
|
||||
super.debugString()).toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getNodeExplainString(String detailPrefix, TExplainLevel detailLevel) {
|
||||
StringBuilder output = new StringBuilder();
|
||||
output.append(detailPrefix + "repeat: repeat ");
|
||||
output.append(repeatSlotIdList.size() - 1);
|
||||
output.append(" lines ");
|
||||
output.append(repeatSlotIdList);
|
||||
output.append("\n" );
|
||||
if (CollectionUtils.isNotEmpty(outputTupleDesc.getSlots())) {
|
||||
output.append(detailPrefix + "generate: ");
|
||||
output.append(outputTupleDesc.getSlots().stream().map(slot -> "`" + slot.getColumn().getName() + "`")
|
||||
.collect(Collectors.joining(", ")) + "\n");
|
||||
}
|
||||
return output.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumInstances() {
|
||||
return children.get(0).getNumInstances();
|
||||
}
|
||||
}
|
||||
@ -29,6 +29,8 @@ import org.apache.doris.analysis.DescriptorTable;
|
||||
import org.apache.doris.analysis.Expr;
|
||||
import org.apache.doris.analysis.ExprSubstitutionMap;
|
||||
import org.apache.doris.analysis.FunctionCallExpr;
|
||||
import org.apache.doris.analysis.GroupByClause;
|
||||
import org.apache.doris.analysis.GroupingInfo;
|
||||
import org.apache.doris.analysis.InPredicate;
|
||||
import org.apache.doris.analysis.InlineViewRef;
|
||||
import org.apache.doris.analysis.IsNullPredicate;
|
||||
@ -97,26 +99,26 @@ public class SingleNodePlanner {
|
||||
* - Migrate conjuncts from parent blocks into inline views and union operands.
|
||||
* In the bottom-up phase generate the plan tree for every query statement:
|
||||
* - Generate the plan for the FROM-clause of a select statement: The plan trees of
|
||||
* absolute and uncorrelated table refs are connected via JoinNodes. The relative
|
||||
* and correlated table refs are associated with one or more SubplanNodes.
|
||||
* absolute and uncorrelated table refs are connected via JoinNodes. The relative
|
||||
* and correlated table refs are associated with one or more SubplanNodes.
|
||||
* - A new SubplanNode is placed on top of an existing plan node whenever the tuples
|
||||
* materialized by that plan node enable evaluation of one or more relative or
|
||||
* correlated table refs, i.e., SubplanNodes are placed at the lowest possible point
|
||||
* in the plan, often right after a ScanNode materializing the (single) parent tuple.
|
||||
* materialized by that plan node enable evaluation of one or more relative or
|
||||
* correlated table refs, i.e., SubplanNodes are placed at the lowest possible point
|
||||
* in the plan, often right after a ScanNode materializing the (single) parent tuple.
|
||||
* - The right-hand side of each SubplanNode is a plan tree generated by joining a
|
||||
* SingularRowSrcTableRef with those applicable relative and correlated refs.
|
||||
* A SingularRowSrcTableRef represents the current row being processed by the
|
||||
* SubplanNode from its input (first child).
|
||||
* SingularRowSrcTableRef with those applicable relative and correlated refs.
|
||||
* A SingularRowSrcTableRef represents the current row being processed by the
|
||||
* SubplanNode from its input (first child).
|
||||
* - Connecting table ref plans via JoinNodes is done in a cost-based fashion
|
||||
* (join-order optimization). All materialized slots, including those of tuples
|
||||
* materialized inside a SubplanNode, must be known for an accurate estimate of row
|
||||
* sizes needed for cost-based join ordering.
|
||||
* (join-order optimization). All materialized slots, including those of tuples
|
||||
* materialized inside a SubplanNode, must be known for an accurate estimate of row
|
||||
* sizes needed for cost-based join ordering.
|
||||
* - The remaining aggregate/analytic/orderby portions of a select statement are added
|
||||
* on top of the FROM-clause plan.
|
||||
* on top of the FROM-clause plan.
|
||||
* - Whenever a new node is added to the plan tree, assign conjuncts that can be
|
||||
* evaluated at that node and compute the stats of that node (cardinality, etc.).
|
||||
* evaluated at that node and compute the stats of that node (cardinality, etc.).
|
||||
* - Apply combined expression substitution map of child plan nodes; if a plan node
|
||||
* re-maps its input, set a substitution map to be applied by parents.
|
||||
* re-maps its input, set a substitution map to be applied by parents.
|
||||
*/
|
||||
public PlanNode createSingleNodePlan() throws UserException, AnalysisException {
|
||||
QueryStmt queryStmt = ctx_.getQueryStmt();
|
||||
@ -162,7 +164,7 @@ public class SingleNodePlanner {
|
||||
// Constant selects do not have materialized tuples at this stage.
|
||||
Preconditions.checkState(stmt instanceof SelectStmt,
|
||||
"Only constant selects should have no materialized tuples");
|
||||
SelectStmt selectStmt = (SelectStmt)stmt;
|
||||
SelectStmt selectStmt = (SelectStmt) stmt;
|
||||
Preconditions.checkState(selectStmt.getTableRefs().isEmpty());
|
||||
tupleIds.add(createResultTupleDescriptor(selectStmt, "empty", analyzer).getId());
|
||||
}
|
||||
@ -183,8 +185,10 @@ public class SingleNodePlanner {
|
||||
private void unmarkCollectionSlots(QueryStmt stmt) {
|
||||
List<TableRef> tblRefs = Lists.newArrayList();
|
||||
stmt.collectTableRefs(tblRefs);
|
||||
for (TableRef ref: tblRefs) {
|
||||
if (!ref.isRelative()) continue;
|
||||
for (TableRef ref : tblRefs) {
|
||||
if (!ref.isRelative()) {
|
||||
continue;
|
||||
}
|
||||
// Preconditions.checkState(ref instanceof CollectionTableRef);
|
||||
// CollectionTableRef collTblRef = (CollectionTableRef) ref;
|
||||
// Expr collExpr = collTblRef.getCollectionExpr();
|
||||
@ -201,7 +205,9 @@ public class SingleNodePlanner {
|
||||
*/
|
||||
private PlanNode createQueryPlan(QueryStmt stmt, Analyzer analyzer, long defaultOrderByLimit)
|
||||
throws UserException, AnalysisException {
|
||||
if (analyzer.hasEmptyResultSet()) return createEmptyNode(stmt, analyzer);
|
||||
if (analyzer.hasEmptyResultSet()) {
|
||||
return createEmptyNode(stmt, analyzer);
|
||||
}
|
||||
|
||||
long newDefaultOrderByLimit = defaultOrderByLimit;
|
||||
if (newDefaultOrderByLimit == -1) {
|
||||
@ -236,7 +242,7 @@ public class SingleNodePlanner {
|
||||
// Avoid adding a sort node if the sort tuple has no materialized slots.
|
||||
boolean sortHasMaterializedSlots = false;
|
||||
if (stmt.evaluateOrderBy()) {
|
||||
for (SlotDescriptor sortSlotDesc:
|
||||
for (SlotDescriptor sortSlotDesc :
|
||||
stmt.getSortInfo().getSortTupleDescriptor().getSlots()) {
|
||||
if (sortSlotDesc.isMaterialized()) {
|
||||
sortHasMaterializedSlots = true;
|
||||
@ -277,7 +283,6 @@ public class SingleNodePlanner {
|
||||
return root;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* If there are unassigned conjuncts that are bound by tupleIds or if there are slot
|
||||
* equivalences for tupleIds that have not yet been enforced, returns a SelectNode on
|
||||
@ -304,16 +309,20 @@ public class SingleNodePlanner {
|
||||
private PlanNode addUnassignedConjuncts(
|
||||
Analyzer analyzer, List<TupleId> tupleIds, PlanNode root) throws UserException {
|
||||
// No point in adding SelectNode on top of an EmptyNode.
|
||||
if (root instanceof EmptySetNode) return root;
|
||||
if (root instanceof EmptySetNode) {
|
||||
return root;
|
||||
}
|
||||
Preconditions.checkNotNull(root);
|
||||
// Gather unassigned conjuncts and generate predicates to enfore
|
||||
// slot equivalences for each tuple id.
|
||||
List<Expr> conjuncts = analyzer.getUnassignedConjuncts(root);
|
||||
for (TupleId tid: tupleIds) {
|
||||
for (TupleId tid : tupleIds) {
|
||||
// TODO(zc)
|
||||
// analyzer.createEquivConjuncts(tid, conjuncts);
|
||||
}
|
||||
if (conjuncts.isEmpty()) return root;
|
||||
if (conjuncts.isEmpty()) {
|
||||
return root;
|
||||
}
|
||||
// evaluate conjuncts in SelectNode
|
||||
SelectNode selectNode = new SelectNode(ctx_.getNextNodeId(), root, conjuncts);
|
||||
// init() marks conjuncts as assigned
|
||||
@ -330,8 +339,8 @@ public class SingleNodePlanner {
|
||||
break;
|
||||
}
|
||||
|
||||
if (((OlapScanNode)root).getForceOpenPreAgg()) {
|
||||
((OlapScanNode)root).setIsPreAggregation(true, "");
|
||||
if (((OlapScanNode) root).getForceOpenPreAgg()) {
|
||||
((OlapScanNode) root).setIsPreAggregation(true, "");
|
||||
return;
|
||||
}
|
||||
|
||||
@ -352,7 +361,7 @@ public class SingleNodePlanner {
|
||||
" joinOp is full outer join or right outer join.";
|
||||
aggTableValidate = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!aggTableValidate) {
|
||||
break;
|
||||
@ -376,7 +385,8 @@ public class SingleNodePlanner {
|
||||
|
||||
if (analyzer.getTupleDesc(tupleId).getRef() != olapTableRef) {
|
||||
if (analyzer.getTupleDesc(tupleId).getTable() != null
|
||||
&& analyzer.getTupleDesc(tupleId).getTable().getType() == Table.TableType.OLAP) {
|
||||
&& analyzer.getTupleDesc(tupleId).getTable().getType()
|
||||
== Table.TableType.OLAP) {
|
||||
turnOffReason = "agg expr [" + aggExpr.debugString() + "] is not bound ["
|
||||
+ selectStmt.getTableRefs().get(0).toSql() + "]";
|
||||
aggTableValidate = false;
|
||||
@ -558,7 +568,8 @@ public class SingleNodePlanner {
|
||||
} else if (aggExpr.getFnName().getFunction().equalsIgnoreCase(FunctionSet.BITMAP_UNION)
|
||||
|| aggExpr.getFnName().getFunction().equalsIgnoreCase(FunctionSet.BITMAP_UNION_COUNT)) {
|
||||
if (col.getAggregationType() != AggregateType.BITMAP_UNION) {
|
||||
turnOffReason = "Aggregate Operator not match: BITMAP_UNION <--> " + col.getAggregationType();
|
||||
turnOffReason =
|
||||
"Aggregate Operator not match: BITMAP_UNION <--> " + col.getAggregationType();
|
||||
returnColumnValidate = false;
|
||||
break;
|
||||
}
|
||||
@ -655,7 +666,7 @@ public class SingleNodePlanner {
|
||||
for (TableRef tblRef : selectStmt.getTableRefs()) {
|
||||
rowTuples.addAll(tblRef.getMaterializedTupleIds());
|
||||
}
|
||||
|
||||
|
||||
if (analyzer.hasEmptySpjResultSet()) {
|
||||
final PlanNode emptySetNode = new EmptySetNode(ctx_.getNextNodeId(), rowTuples);
|
||||
emptySetNode.init(analyzer);
|
||||
@ -705,14 +716,36 @@ public class SingleNodePlanner {
|
||||
// add unassigned conjuncts before aggregation
|
||||
// (scenario: agg input comes from an inline view which wasn't able to
|
||||
// evaluate all Where clause conjuncts from this scope)
|
||||
if (!selectStmt.hasOrderByClause()) {
|
||||
if (!selectStmt.hasOrderByClause()) {
|
||||
root = addUnassignedConjuncts(analyzer, root);
|
||||
}
|
||||
}
|
||||
|
||||
// add aggregation, if required
|
||||
if (aggInfo != null) root = createAggregationPlan(selectStmt, analyzer, root);
|
||||
if (aggInfo != null) {
|
||||
// introduce repeatNode for group by extension
|
||||
GroupByClause groupByClause = selectStmt.getGroupByClause();
|
||||
if (groupByClause != null && groupByClause.isGroupByExtension()) {
|
||||
root = createRepeatNodePlan(selectStmt, analyzer, root);
|
||||
}
|
||||
|
||||
root = createAggregationPlan(selectStmt, analyzer, root);
|
||||
}
|
||||
|
||||
return root;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a new RepeatNode.
|
||||
*/
|
||||
private PlanNode createRepeatNodePlan(SelectStmt selectStmt, Analyzer analyzer,
|
||||
PlanNode root) throws UserException {
|
||||
GroupByClause groupByClause = selectStmt.getGroupByClause();
|
||||
GroupingInfo groupingInfo = selectStmt.getGroupingInfo();
|
||||
Preconditions.checkState(groupByClause != null && groupByClause.isGroupByExtension()
|
||||
&& groupingInfo != null);
|
||||
root = new RepeatNode(ctx_.getNextNodeId(), root, groupingInfo, groupByClause);
|
||||
root.init(analyzer);
|
||||
return root;
|
||||
}
|
||||
|
||||
@ -725,6 +758,7 @@ public class SingleNodePlanner {
|
||||
Preconditions.checkState(selectStmt.getAggInfo() != null);
|
||||
// add aggregation, if required
|
||||
AggregateInfo aggInfo = selectStmt.getAggInfo();
|
||||
// aggInfo.substitueGroupingExpr(analyzer);
|
||||
PlanNode newRoot = new AggregationNode(ctx_.getNextNodeId(), root, aggInfo);
|
||||
newRoot.init(analyzer);
|
||||
Preconditions.checkState(newRoot.hasValidStats());
|
||||
@ -881,14 +915,14 @@ public class SingleNodePlanner {
|
||||
/**
|
||||
* Returns plan tree for an inline view ref:
|
||||
* - predicates from the enclosing scope that can be evaluated directly within
|
||||
* the inline-view plan are pushed down
|
||||
* the inline-view plan are pushed down
|
||||
* - predicates that cannot be evaluated directly within the inline-view plan
|
||||
* but only apply to the inline view are evaluated in a SelectNode placed
|
||||
* on top of the inline view plan
|
||||
* but only apply to the inline view are evaluated in a SelectNode placed
|
||||
* on top of the inline view plan
|
||||
* - all slots that are referenced by predicates from the enclosing scope that cannot
|
||||
* be pushed down are marked as materialized (so that when computeMemLayout() is
|
||||
* called on the base table descriptors materialized by the inline view it has a
|
||||
* complete picture)
|
||||
* be pushed down are marked as materialized (so that when computeMemLayout() is
|
||||
* called on the base table descriptors materialized by the inline view it has a
|
||||
* complete picture)
|
||||
*/
|
||||
private PlanNode createInlineViewPlan(Analyzer analyzer, InlineViewRef inlineViewRef)
|
||||
throws UserException, AnalysisException {
|
||||
@ -924,7 +958,9 @@ public class SingleNodePlanner {
|
||||
analyzer.getTupleDesc(inlineViewRef.getId()).materializeSlots();
|
||||
UnionNode unionNode = new UnionNode(ctx_.getNextNodeId(),
|
||||
inlineViewRef.getMaterializedTupleIds().get(0));
|
||||
if (analyzer.hasEmptyResultSet()) return unionNode;
|
||||
if (analyzer.hasEmptyResultSet()) {
|
||||
return unionNode;
|
||||
}
|
||||
unionNode.setTblRefIds(Lists.newArrayList(inlineViewRef.getId()));
|
||||
unionNode.addConstExprList(selectStmt.getBaseTblResultExprs());
|
||||
unionNode.init(analyzer);
|
||||
@ -1021,13 +1057,14 @@ public class SingleNodePlanner {
|
||||
}
|
||||
// Non constant conjuncts
|
||||
unassignedConjuncts.removeAll(unassignedConstantConjuncts);
|
||||
migrateNonconstantConjuncts(inlineViewRef, unassignedConjuncts, analyzer);
|
||||
migrateNonconstantConjuncts(inlineViewRef, unassignedConjuncts, analyzer);
|
||||
migrateConstantConjuncts(inlineViewRef, unassignedConstantConjuncts);
|
||||
}
|
||||
|
||||
/**
|
||||
* For handling non-constant conjuncts. This only substitute conjunct's tuple with InlineView's
|
||||
* and register it in InlineView's Analyzer, whcih will be assigned by the next planning.
|
||||
*
|
||||
* @param inlineViewRef
|
||||
* @param unassignedConjuncts
|
||||
* @param analyzer
|
||||
@ -1036,7 +1073,7 @@ public class SingleNodePlanner {
|
||||
private void migrateNonconstantConjuncts(
|
||||
InlineViewRef inlineViewRef, List<Expr> unassignedConjuncts, Analyzer analyzer) throws AnalysisException {
|
||||
final List<Expr> preds = Lists.newArrayList();
|
||||
for (Expr e: unassignedConjuncts) {
|
||||
for (Expr e : unassignedConjuncts) {
|
||||
if (analyzer.canEvalPredicate(inlineViewRef.getId().asList(), e)) {
|
||||
preds.add(e);
|
||||
}
|
||||
@ -1076,7 +1113,9 @@ public class SingleNodePlanner {
|
||||
analyzer.markConjunctsAssigned(preds);
|
||||
// Unset the On-clause flag of the migrated conjuncts because the migrated conjuncts
|
||||
// apply to the post-join/agg/analytic result of the inline view.
|
||||
for (Expr e: viewPredicates) e.setIsOnClauseConjunct(false);
|
||||
for (Expr e : viewPredicates) {
|
||||
e.setIsOnClauseConjunct(false);
|
||||
}
|
||||
inlineViewRef.getAnalyzer().registerConjuncts(viewPredicates, inlineViewRef.getAllTupleIds());
|
||||
|
||||
// mark (fully resolve) slots referenced by remaining unassigned conjuncts as
|
||||
@ -1090,6 +1129,7 @@ public class SingleNodePlanner {
|
||||
/**
|
||||
* For handling constant conjuncts when migrating conjuncts to InlineViews. Constant conjuncts
|
||||
* should be assigned to query block from top to bottom, it will try to push down constant conjuncts.
|
||||
*
|
||||
* @param inlineViewRef
|
||||
* @param conjuncts
|
||||
* @throws AnalysisException
|
||||
@ -1108,7 +1148,7 @@ public class SingleNodePlanner {
|
||||
final Analyzer viewAnalyzer = inlineViewRef.getAnalyzer();
|
||||
viewAnalyzer.markConjunctsAssigned(conjuncts);
|
||||
if (stmt instanceof SelectStmt) {
|
||||
final SelectStmt select = (SelectStmt)stmt;
|
||||
final SelectStmt select = (SelectStmt) stmt;
|
||||
if (select.getAggInfo() != null) {
|
||||
viewAnalyzer.registerConjuncts(newConjuncts, select.getAggInfo().getOutputTupleId().asList());
|
||||
} else if (select.getTableRefs().size() > 1) {
|
||||
@ -1136,7 +1176,7 @@ public class SingleNodePlanner {
|
||||
}
|
||||
} else {
|
||||
Preconditions.checkArgument(stmt instanceof UnionStmt);
|
||||
final UnionStmt union = (UnionStmt)stmt;
|
||||
final UnionStmt union = (UnionStmt) stmt;
|
||||
viewAnalyzer.registerConjuncts(newConjuncts, union.getTupleId().asList());
|
||||
}
|
||||
}
|
||||
@ -1147,7 +1187,7 @@ public class SingleNodePlanner {
|
||||
return clones;
|
||||
}
|
||||
|
||||
/**
|
||||
/**
|
||||
* Get predicates can be migrated into an inline view.
|
||||
*/
|
||||
private List<Expr> getPushDownPredicatesForInlineView(
|
||||
@ -1165,7 +1205,7 @@ public class SingleNodePlanner {
|
||||
final List<Expr> candicatePredicates =
|
||||
Expr.substituteList(viewPredicates, inlineViewRef.getSmap(), analyzer, false);
|
||||
if (inlineViewRef.getViewStmt() instanceof UnionStmt) {
|
||||
final UnionStmt unionStmt = (UnionStmt)inlineViewRef.getViewStmt();
|
||||
final UnionStmt unionStmt = (UnionStmt) inlineViewRef.getViewStmt();
|
||||
for (int i = 0; i < candicatePredicates.size(); i++) {
|
||||
final Expr predicate = candicatePredicates.get(i);
|
||||
if (predicate.isBound(unionStmt.getTupleId())) {
|
||||
@ -1177,7 +1217,7 @@ public class SingleNodePlanner {
|
||||
return pushDownPredicates;
|
||||
}
|
||||
|
||||
final SelectStmt selectStmt = (SelectStmt)inlineViewRef.getViewStmt();
|
||||
final SelectStmt selectStmt = (SelectStmt) inlineViewRef.getViewStmt();
|
||||
if (selectStmt.hasAnalyticInfo()) {
|
||||
pushDownPredicates.addAll(getWindowsPushDownPredicates(candicatePredicates, viewPredicates,
|
||||
selectStmt.getAnalyticInfo(), pushDownFailedPredicates));
|
||||
@ -1189,6 +1229,7 @@ public class SingleNodePlanner {
|
||||
|
||||
/**
|
||||
* Get predicates which can be pushed down past Windows.
|
||||
*
|
||||
* @param predicates
|
||||
* @param viewPredicates
|
||||
* @param analyticInfo
|
||||
@ -1203,7 +1244,7 @@ public class SingleNodePlanner {
|
||||
final List<SlotId> partitionByIds = Lists.newArrayList();
|
||||
for (Expr expr : partitionExprs) {
|
||||
if (expr instanceof SlotRef) {
|
||||
final SlotRef slotRef = (SlotRef)expr;
|
||||
final SlotRef slotRef = (SlotRef) expr;
|
||||
partitionByIds.add(slotRef.getSlotId());
|
||||
}
|
||||
}
|
||||
@ -1227,12 +1268,12 @@ public class SingleNodePlanner {
|
||||
* Checks if conjuncts can be migrated into an inline view.
|
||||
*/
|
||||
private boolean canMigrateConjuncts(InlineViewRef inlineViewRef) {
|
||||
// TODO chenhao, remove evaluateOrderBy when SubQuery's default limit is removed.
|
||||
// TODO chenhao, remove evaluateOrderBy when SubQuery's default limit is removed.
|
||||
return inlineViewRef.getViewStmt().evaluateOrderBy() ? false :
|
||||
(!inlineViewRef.getViewStmt().hasLimit()
|
||||
&& !inlineViewRef.getViewStmt().hasOffset()
|
||||
&& (!(inlineViewRef.getViewStmt() instanceof SelectStmt)
|
||||
|| !((SelectStmt) inlineViewRef.getViewStmt()).hasAnalyticInfo()));
|
||||
&& !inlineViewRef.getViewStmt().hasOffset()
|
||||
&& (!(inlineViewRef.getViewStmt() instanceof SelectStmt)
|
||||
|| !((SelectStmt) inlineViewRef.getViewStmt()).hasAnalyticInfo()));
|
||||
}
|
||||
|
||||
/**
|
||||
@ -1305,7 +1346,7 @@ public class SingleNodePlanner {
|
||||
candidates = analyzer.getEqJoinConjuncts(lhsIds, rhsIds);
|
||||
if (candidates == null) {
|
||||
if (op.isOuterJoin() || op.isSemiAntiJoin()) {
|
||||
errMsg.setRef("non-equal " + op.toString() + " is not supported");
|
||||
errMsg.setRef("non-equal " + op.toString() + " is not supported");
|
||||
LOG.warn(errMsg);
|
||||
}
|
||||
LOG.debug("no candidates for join.");
|
||||
@ -1378,8 +1419,8 @@ public class SingleNodePlanner {
|
||||
|
||||
// only inner join can change to cross join
|
||||
if (innerRef.getJoinOp().isOuterJoin() || innerRef.getJoinOp().isSemiAntiJoin()) {
|
||||
throw new AnalysisException("non-equal " + innerRef.getJoinOp().toString()
|
||||
+ " is not supported");
|
||||
throw new AnalysisException("non-equal " + innerRef.getJoinOp().toString()
|
||||
+ " is not supported");
|
||||
}
|
||||
|
||||
// construct cross join node
|
||||
@ -1399,7 +1440,7 @@ public class SingleNodePlanner {
|
||||
// that can be evaluated by this join are assigned in createSelectPlan().
|
||||
ojConjuncts = analyzer.getUnassignedOjConjuncts(innerRef);
|
||||
analyzer.markConjunctsAssigned(ojConjuncts);
|
||||
} else if (innerRef.getJoinOp().isSemiAntiJoin()){
|
||||
} else if (innerRef.getJoinOp().isSemiAntiJoin()) {
|
||||
final List<TupleId> tupleIds = innerRef.getAllTupleIds();
|
||||
ojConjuncts = analyzer.getUnassignedConjuncts(tupleIds, false);
|
||||
analyzer.markConjunctsAssigned(ojConjuncts);
|
||||
@ -1415,16 +1456,16 @@ public class SingleNodePlanner {
|
||||
/**
|
||||
* Create a tree of PlanNodes for the given tblRef, which can be a BaseTableRef,
|
||||
* CollectionTableRef or an InlineViewRef.
|
||||
*
|
||||
* <p>
|
||||
* 'fastPartitionKeyScans' indicates whether to try to produce the slots with
|
||||
* metadata instead of table scans. Only applicable to BaseTableRef which is also
|
||||
* an HDFS table.
|
||||
*
|
||||
* <p>
|
||||
* Throws if a PlanNode.init() failed or if planning of the given
|
||||
* table ref is not implemented.
|
||||
*/
|
||||
private PlanNode createTableRefNode(Analyzer analyzer, TableRef tblRef)
|
||||
throws UserException, AnalysisException {
|
||||
throws UserException, AnalysisException {
|
||||
if (tblRef instanceof BaseTableRef) {
|
||||
return createScanNode(analyzer, tblRef);
|
||||
}
|
||||
@ -1447,7 +1488,7 @@ public class SingleNodePlanner {
|
||||
throws UserException, AnalysisException {
|
||||
UnionNode unionNode = new UnionNode(ctx_.getNextNodeId(), unionStmt.getTupleId(),
|
||||
unionStmt.getUnionResultExprs(), false);
|
||||
for (UnionStmt.UnionOperand op: unionOperands) {
|
||||
for (UnionStmt.UnionOperand op : unionOperands) {
|
||||
if (op.getAnalyzer().hasEmptyResultSet()) {
|
||||
unmarkCollectionSlots(op.getQueryStmt());
|
||||
continue;
|
||||
@ -1464,7 +1505,9 @@ public class SingleNodePlanner {
|
||||
// There may still be unassigned conjuncts if the operand has an order by + limit.
|
||||
// Place them into a SelectNode on top of the operand's plan.
|
||||
opPlan = addUnassignedConjuncts(analyzer, opPlan.getTupleIds(), opPlan);
|
||||
if (opPlan instanceof EmptySetNode) continue;
|
||||
if (opPlan instanceof EmptySetNode) {
|
||||
continue;
|
||||
}
|
||||
unionNode.addChild(opPlan, op.getQueryStmt().getResultExprs());
|
||||
}
|
||||
|
||||
@ -1481,18 +1524,18 @@ public class SingleNodePlanner {
|
||||
/**
|
||||
* Returns plan tree for unionStmt:
|
||||
* - distinctOperands' plan trees are collected in a single UnionNode
|
||||
* and duplicates removed via distinct aggregation
|
||||
* and duplicates removed via distinct aggregation
|
||||
* - the output of that plus the allOperands' plan trees are collected in
|
||||
* another UnionNode which materializes the result of unionStmt
|
||||
* another UnionNode which materializes the result of unionStmt
|
||||
* - if any of the union operands contains analytic exprs, we avoid pushing
|
||||
* predicates directly into the operands and instead evaluate them
|
||||
* *after* the final UnionNode (see createInlineViewPlan() for the reasoning)
|
||||
* TODO: optimize this by still pushing predicates into the union operands
|
||||
* that don't contain analytic exprs and evaluating the conjuncts in Select
|
||||
* directly above the AnalyticEvalNodes
|
||||
* predicates directly into the operands and instead evaluate them
|
||||
* *after* the final UnionNode (see createInlineViewPlan() for the reasoning)
|
||||
* TODO: optimize this by still pushing predicates into the union operands
|
||||
* that don't contain analytic exprs and evaluating the conjuncts in Select
|
||||
* directly above the AnalyticEvalNodes
|
||||
* TODO: Simplify the plan of unions with empty operands using an empty set node.
|
||||
* TODO: Simplify the plan of unions with only a single non-empty operand to not
|
||||
* use a union node (this is tricky because a union materializes a new tuple).
|
||||
* use a union node (this is tricky because a union materializes a new tuple).
|
||||
*/
|
||||
private PlanNode createUnionPlan(UnionStmt unionStmt, Analyzer analyzer, long defaultOrderByLimit)
|
||||
throws UserException, AnalysisException {
|
||||
@ -1510,7 +1553,7 @@ public class SingleNodePlanner {
|
||||
// the individual operands.
|
||||
// Do this prior to creating the operands' plan trees so they get a chance to
|
||||
// pick up propagated predicates.
|
||||
for (UnionStmt.UnionOperand op: unionStmt.getOperands()) {
|
||||
for (UnionStmt.UnionOperand op : unionStmt.getOperands()) {
|
||||
List<Expr> opConjuncts =
|
||||
Expr.substituteList(conjuncts, op.getSmap(), analyzer, false);
|
||||
boolean selectHasTableRef = true;
|
||||
@ -1578,14 +1621,15 @@ public class SingleNodePlanner {
|
||||
* which are not referenced by Statement outside. However, in some cases, in order to ensure The
|
||||
* correct execution, it is necessary to materialize the slots that are not needed by Statement
|
||||
* outside.
|
||||
*
|
||||
* @param tblRef
|
||||
* @param analyzer
|
||||
*/
|
||||
private void materializeTableResultForCrossJoinOrCountStar(TableRef tblRef, Analyzer analyzer) {
|
||||
if (tblRef instanceof BaseTableRef) {
|
||||
materializeBaseTableRefResultForCrossJoinOrCountStar((BaseTableRef)tblRef, analyzer);
|
||||
materializeBaseTableRefResultForCrossJoinOrCountStar((BaseTableRef) tblRef, analyzer);
|
||||
} else if (tblRef instanceof InlineViewRef) {
|
||||
materializeInlineViewResultExprForCrossJoinOrCountStar((InlineViewRef)tblRef, analyzer);
|
||||
materializeInlineViewResultExprForCrossJoinOrCountStar((InlineViewRef) tblRef, analyzer);
|
||||
} else {
|
||||
Preconditions.checkArgument(false);
|
||||
}
|
||||
@ -1593,6 +1637,7 @@ public class SingleNodePlanner {
|
||||
|
||||
/**
|
||||
* materialize BaseTableRef result' exprs for Cross Join or Count Star
|
||||
*
|
||||
* @param tblRef
|
||||
* @param analyzer
|
||||
*/
|
||||
@ -1615,6 +1660,7 @@ public class SingleNodePlanner {
|
||||
|
||||
/**
|
||||
* materialize InlineViewRef result'exprs for Cross Join or Count Star
|
||||
*
|
||||
* @param inlineView
|
||||
* @param analyzer
|
||||
*/
|
||||
@ -1673,7 +1719,7 @@ public class SingleNodePlanner {
|
||||
|
||||
/**
|
||||
* Entrance for push-down rules, it will execute possible push-down rules from top to down
|
||||
* and the planner will be responsible for assigning all predicates to PlanNode.
|
||||
* and the planner will be responsible for assigning all predicates to PlanNode.
|
||||
*/
|
||||
private void pushDownPredicates(Analyzer analyzer, SelectStmt stmt) throws AnalysisException {
|
||||
// Push down predicates according to the semantic requirements of SQL.
|
||||
@ -1686,7 +1732,7 @@ public class SingleNodePlanner {
|
||||
// TODO chenhao, remove isEvaluateOrderBy when SubQuery's default limit is removed.
|
||||
if (stmt.evaluateOrderBy() || stmt.getLimit() >= 0 || stmt.getOffset() > 0 || stmt.getSortInfo() == null) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
final List<Expr> predicates = getBoundPredicates(analyzer, stmt.getSortInfo().getSortTupleDescriptor());
|
||||
if (predicates.size() <= 0) {
|
||||
return;
|
||||
@ -1790,7 +1836,7 @@ public class SingleNodePlanner {
|
||||
|
||||
private void replacePredicateSlotRefWithSource(Expr parent, Expr predicate, int childIndex, Analyzer analyzer) {
|
||||
if (predicate instanceof SlotRef) {
|
||||
final SlotRef slotRef = (SlotRef)predicate;
|
||||
final SlotRef slotRef = (SlotRef) predicate;
|
||||
if (parent != null && childIndex >= 0) {
|
||||
final Expr newReplacedExpr = slotRef.getDesc().getSourceExprs().get(0).clone();
|
||||
parent.setChild(childIndex, newReplacedExpr);
|
||||
@ -1805,7 +1851,7 @@ public class SingleNodePlanner {
|
||||
|
||||
// Register predicates with Aggregation's output tuple id.
|
||||
private boolean putPredicatesOnAggregation(SelectStmt stmt, Analyzer analyzer,
|
||||
List<Expr> predicates) throws AnalysisException {
|
||||
List<Expr> predicates) throws AnalysisException {
|
||||
final AggregateInfo aggregateInfo = stmt.getAggInfo();
|
||||
if (aggregateInfo != null) {
|
||||
analyzer.registerConjuncts(predicates, aggregateInfo.getOutputTupleId());
|
||||
@ -1816,7 +1862,7 @@ public class SingleNodePlanner {
|
||||
|
||||
// Register predicates with Windows's tuple id.
|
||||
private boolean putPredicatesOnWindows(SelectStmt stmt, Analyzer analyzer,
|
||||
List<Expr> predicates) throws AnalysisException {
|
||||
List<Expr> predicates) throws AnalysisException {
|
||||
final AnalyticInfo analyticInfo = stmt.getAnalyticInfo();
|
||||
if (analyticInfo != null) {
|
||||
analyzer.registerConjuncts(predicates, analyticInfo.getOutputTupleId());
|
||||
@ -1826,7 +1872,8 @@ public class SingleNodePlanner {
|
||||
}
|
||||
|
||||
// Register predicates with TableRef's tuple id.
|
||||
private void putPredicatesOnFrom(SelectStmt stmt, Analyzer analyzer, List<Expr> predicates) throws AnalysisException {
|
||||
private void putPredicatesOnFrom(SelectStmt stmt, Analyzer analyzer, List<Expr> predicates)
|
||||
throws AnalysisException {
|
||||
final List<TupleId> tableTupleIds = Lists.newArrayList();
|
||||
for (TableRef tableRef : stmt.getTableRefs()) {
|
||||
tableTupleIds.add(tableRef.getId());
|
||||
@ -1842,7 +1889,7 @@ public class SingleNodePlanner {
|
||||
}
|
||||
|
||||
/**
|
||||
------------------------------------------------------------------------------
|
||||
* ------------------------------------------------------------------------------
|
||||
*/
|
||||
|
||||
private List<Expr> getBoundPredicates(Analyzer analyzer, TupleDescriptor tupleDesc) {
|
||||
|
||||
@ -418,7 +418,6 @@ public class StmtExecutor {
|
||||
StmtRewriter.rewrite(analyzer, parsedStmt);
|
||||
reAnalyze = true;
|
||||
}
|
||||
|
||||
if (reAnalyze) {
|
||||
// The rewrites should have no user-visible effect. Remember the original result
|
||||
// types and column labels to restore them after the rewritten stmt has been
|
||||
|
||||
@ -106,10 +106,11 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("between", new Integer(SqlParserSymbols.KW_BETWEEN));
|
||||
keywordMap.put("bigint", new Integer(SqlParserSymbols.KW_BIGINT));
|
||||
keywordMap.put("bitmap", new Integer(SqlParserSymbols.KW_BITMAP));
|
||||
keywordMap.put("bitmap_union", new Integer(SqlParserSymbols.KW_BITMAP_UNION));
|
||||
keywordMap.put("boolean", new Integer(SqlParserSymbols.KW_BOOLEAN));
|
||||
keywordMap.put("day", new Integer(SqlParserSymbols.KW_DAY));
|
||||
keywordMap.put("both", new Integer(SqlParserSymbols.KW_BOTH));
|
||||
keywordMap.put("broker", new Integer(SqlParserSymbols.KW_BROKER));
|
||||
keywordMap.put("buckets", new Integer(SqlParserSymbols.KW_BUCKETS));
|
||||
keywordMap.put("by", new Integer(SqlParserSymbols.KW_BY));
|
||||
keywordMap.put("cancel", new Integer(SqlParserSymbols.KW_CANCEL));
|
||||
keywordMap.put("case", new Integer(SqlParserSymbols.KW_CASE));
|
||||
@ -120,12 +121,6 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("charset", new Integer(SqlParserSymbols.KW_CHARSET));
|
||||
keywordMap.put("cluster", new Integer(SqlParserSymbols.KW_CLUSTER));
|
||||
keywordMap.put("clusters", new Integer(SqlParserSymbols.KW_CLUSTERS));
|
||||
keywordMap.put("free", new Integer(SqlParserSymbols.KW_FREE));
|
||||
keywordMap.put("system", new Integer(SqlParserSymbols.KW_SYSTEM));
|
||||
keywordMap.put("link", new Integer(SqlParserSymbols.KW_LINK));
|
||||
keywordMap.put("migrate", new Integer(SqlParserSymbols.KW_MIGRATE));
|
||||
keywordMap.put("enter", new Integer(SqlParserSymbols.KW_ENTER));
|
||||
keywordMap.put("migrations", new Integer(SqlParserSymbols.KW_MIGRATIONS));
|
||||
keywordMap.put("collate", new Integer(SqlParserSymbols.KW_COLLATE));
|
||||
keywordMap.put("collation", new Integer(SqlParserSymbols.KW_COLLATION));
|
||||
keywordMap.put("column", new Integer(SqlParserSymbols.KW_COLUMN));
|
||||
@ -140,6 +135,7 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("count", new Integer(SqlParserSymbols.KW_COUNT));
|
||||
keywordMap.put("create", new Integer(SqlParserSymbols.KW_CREATE));
|
||||
keywordMap.put("cross", new Integer(SqlParserSymbols.KW_CROSS));
|
||||
keywordMap.put("cube", new Integer(SqlParserSymbols.KW_CUBE));
|
||||
keywordMap.put("current", new Integer(SqlParserSymbols.KW_CURRENT));
|
||||
keywordMap.put("current_user", new Integer(SqlParserSymbols.KW_CURRENT_USER));
|
||||
keywordMap.put("data", new Integer(SqlParserSymbols.KW_DATA));
|
||||
@ -147,6 +143,7 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("databases", new Integer(SqlParserSymbols.KW_DATABASES));
|
||||
keywordMap.put("date", new Integer(SqlParserSymbols.KW_DATE));
|
||||
keywordMap.put("datetime", new Integer(SqlParserSymbols.KW_DATETIME));
|
||||
keywordMap.put("day", new Integer(SqlParserSymbols.KW_DAY));
|
||||
keywordMap.put("decimal", new Integer(SqlParserSymbols.KW_DECIMAL));
|
||||
keywordMap.put("decommission", new Integer(SqlParserSymbols.KW_DECOMMISSION));
|
||||
keywordMap.put("default", new Integer(SqlParserSymbols.KW_DEFAULT));
|
||||
@ -161,7 +158,6 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("distributed", new Integer(SqlParserSymbols.KW_DISTRIBUTED));
|
||||
keywordMap.put("distribution", new Integer(SqlParserSymbols.KW_DISTRIBUTION));
|
||||
keywordMap.put("dynamic", new Integer(SqlParserSymbols.KW_DYNAMIC));
|
||||
keywordMap.put("buckets", new Integer(SqlParserSymbols.KW_BUCKETS));
|
||||
keywordMap.put("div", new Integer(SqlParserSymbols.KW_DIV));
|
||||
keywordMap.put("double", new Integer(SqlParserSymbols.KW_DOUBLE));
|
||||
keywordMap.put("drop", new Integer(SqlParserSymbols.KW_DROP));
|
||||
@ -171,11 +167,12 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("end", new Integer(SqlParserSymbols.KW_END));
|
||||
keywordMap.put("engine", new Integer(SqlParserSymbols.KW_ENGINE));
|
||||
keywordMap.put("engines", new Integer(SqlParserSymbols.KW_ENGINES));
|
||||
keywordMap.put("enter", new Integer(SqlParserSymbols.KW_ENTER));
|
||||
keywordMap.put("errors", new Integer(SqlParserSymbols.KW_ERRORS));
|
||||
keywordMap.put("events", new Integer(SqlParserSymbols.KW_EVENTS));
|
||||
keywordMap.put("exists", new Integer(SqlParserSymbols.KW_EXISTS));
|
||||
keywordMap.put("explain", new Integer(SqlParserSymbols.KW_DESCRIBE));
|
||||
keywordMap.put("export", new Integer(SqlParserSymbols.KW_EXPORT));
|
||||
keywordMap.put("exists", new Integer(SqlParserSymbols.KW_EXISTS));
|
||||
keywordMap.put("external", new Integer(SqlParserSymbols.KW_EXTERNAL));
|
||||
keywordMap.put("extract", new Integer(SqlParserSymbols.KW_EXTRACT));
|
||||
keywordMap.put("false", new Integer(SqlParserSymbols.KW_FALSE));
|
||||
@ -186,7 +183,7 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("following", new Integer(SqlParserSymbols.KW_FOLLOWING));
|
||||
keywordMap.put("for", new Integer(SqlParserSymbols.KW_FOR));
|
||||
keywordMap.put("format", new Integer(SqlParserSymbols.KW_FORMAT));
|
||||
keywordMap.put("path", new Integer(SqlParserSymbols.KW_PATH));
|
||||
keywordMap.put("free", new Integer(SqlParserSymbols.KW_FREE));
|
||||
keywordMap.put("from", new Integer(SqlParserSymbols.KW_FROM));
|
||||
keywordMap.put("frontend", new Integer(SqlParserSymbols.KW_FRONTEND));
|
||||
keywordMap.put("frontends", new Integer(SqlParserSymbols.KW_FRONTENDS));
|
||||
@ -196,14 +193,14 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("grant", new Integer(SqlParserSymbols.KW_GRANT));
|
||||
keywordMap.put("grants", new Integer(SqlParserSymbols.KW_GRANTS));
|
||||
keywordMap.put("group", new Integer(SqlParserSymbols.KW_GROUP));
|
||||
keywordMap.put("grouping", new Integer(SqlParserSymbols.KW_GROUPING));
|
||||
keywordMap.put("hash", new Integer(SqlParserSymbols.KW_HASH));
|
||||
keywordMap.put("having", new Integer(SqlParserSymbols.KW_HAVING));
|
||||
keywordMap.put("help", new Integer(SqlParserSymbols.KW_HELP));
|
||||
keywordMap.put("hll", new Integer(SqlParserSymbols.KW_HLL));
|
||||
keywordMap.put("hll_union", new Integer(SqlParserSymbols.KW_HLL_UNION));
|
||||
keywordMap.put("bitmap_union", new Integer(SqlParserSymbols.KW_BITMAP_UNION));
|
||||
keywordMap.put("hour", new Integer(SqlParserSymbols.KW_HOUR));
|
||||
keywordMap.put("hub", new Integer(SqlParserSymbols.KW_HUB));
|
||||
keywordMap.put("hub", new Integer(SqlParserSymbols.KW_HUB));
|
||||
keywordMap.put("identified", new Integer(SqlParserSymbols.KW_IDENTIFIED));
|
||||
keywordMap.put("if", new Integer(SqlParserSymbols.KW_IF));
|
||||
keywordMap.put("in", new Integer(SqlParserSymbols.KW_IN));
|
||||
@ -231,17 +228,15 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("level", new Integer(SqlParserSymbols.KW_LEVEL));
|
||||
keywordMap.put("like", new Integer(SqlParserSymbols.KW_LIKE));
|
||||
keywordMap.put("limit", new Integer(SqlParserSymbols.KW_LIMIT));
|
||||
keywordMap.put("link", new Integer(SqlParserSymbols.KW_LINK));
|
||||
keywordMap.put("load", new Integer(SqlParserSymbols.KW_LOAD));
|
||||
keywordMap.put("routine", new Integer(SqlParserSymbols.KW_ROUTINE));
|
||||
keywordMap.put("pause", new Integer(SqlParserSymbols.KW_PAUSE));
|
||||
keywordMap.put("resume", new Integer(SqlParserSymbols.KW_RESUME));
|
||||
keywordMap.put("stop", new Integer(SqlParserSymbols.KW_STOP));
|
||||
keywordMap.put("task", new Integer(SqlParserSymbols.KW_TASK));
|
||||
keywordMap.put("local", new Integer(SqlParserSymbols.KW_LOCAL));
|
||||
keywordMap.put("location", new Integer(SqlParserSymbols.KW_LOCATION));
|
||||
keywordMap.put("max", new Integer(SqlParserSymbols.KW_MAX));
|
||||
keywordMap.put("maxvalue", new Integer(SqlParserSymbols.KW_MAX_VALUE));
|
||||
keywordMap.put("merge", new Integer(SqlParserSymbols.KW_MERGE));
|
||||
keywordMap.put("migrate", new Integer(SqlParserSymbols.KW_MIGRATE));
|
||||
keywordMap.put("migrations", new Integer(SqlParserSymbols.KW_MIGRATIONS));
|
||||
keywordMap.put("min", new Integer(SqlParserSymbols.KW_MIN));
|
||||
keywordMap.put("minute", new Integer(SqlParserSymbols.KW_MINUTE));
|
||||
keywordMap.put("modify", new Integer(SqlParserSymbols.KW_MODIFY));
|
||||
@ -264,11 +259,12 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("over", new Integer(SqlParserSymbols.KW_OVER));
|
||||
keywordMap.put("partition", new Integer(SqlParserSymbols.KW_PARTITION));
|
||||
keywordMap.put("partitions", new Integer(SqlParserSymbols.KW_PARTITIONS));
|
||||
keywordMap.put("preceding", new Integer(SqlParserSymbols.KW_PRECEDING));
|
||||
keywordMap.put("range", new Integer(SqlParserSymbols.KW_RANGE));
|
||||
keywordMap.put("password", new Integer(SqlParserSymbols.KW_PASSWORD));
|
||||
keywordMap.put("path", new Integer(SqlParserSymbols.KW_PATH));
|
||||
keywordMap.put("pause", new Integer(SqlParserSymbols.KW_PAUSE));
|
||||
keywordMap.put("plugin", new Integer(SqlParserSymbols.KW_PLUGIN));
|
||||
keywordMap.put("plugins", new Integer(SqlParserSymbols.KW_PLUGINS));
|
||||
keywordMap.put("preceding", new Integer(SqlParserSymbols.KW_PRECEDING));
|
||||
keywordMap.put("primary", new Integer(SqlParserSymbols.KW_PRIMARY));
|
||||
keywordMap.put("proc", new Integer(SqlParserSymbols.KW_PROC));
|
||||
keywordMap.put("procedure", new Integer(SqlParserSymbols.KW_PROCEDURE));
|
||||
@ -279,21 +275,23 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("quota", new Integer(SqlParserSymbols.KW_QUOTA));
|
||||
keywordMap.put("random", new Integer(SqlParserSymbols.KW_RANDOM));
|
||||
keywordMap.put("range", new Integer(SqlParserSymbols.KW_RANGE));
|
||||
keywordMap.put("range", new Integer(SqlParserSymbols.KW_RANGE));
|
||||
keywordMap.put("read", new Integer(SqlParserSymbols.KW_READ));
|
||||
keywordMap.put("release", new Integer(SqlParserSymbols.KW_RELEASE));
|
||||
keywordMap.put("real", new Integer(SqlParserSymbols.KW_DOUBLE));
|
||||
keywordMap.put("recover", new Integer(SqlParserSymbols.KW_RECOVER));
|
||||
keywordMap.put("regexp", new Integer(SqlParserSymbols.KW_REGEXP));
|
||||
keywordMap.put("release", new Integer(SqlParserSymbols.KW_RELEASE));
|
||||
keywordMap.put("rename", new Integer(SqlParserSymbols.KW_RENAME));
|
||||
keywordMap.put("repair", new Integer(SqlParserSymbols.KW_REPAIR));
|
||||
keywordMap.put("repeatable", new Integer(SqlParserSymbols.KW_REPEATABLE));
|
||||
keywordMap.put("replace", new Integer(SqlParserSymbols.KW_REPLACE));
|
||||
keywordMap.put("replace_if_not_null", new Integer(SqlParserSymbols.KW_REPLACE_IF_NOT_NULL));
|
||||
keywordMap.put("replica", new Integer(SqlParserSymbols.KW_REPLICA));
|
||||
keywordMap.put("repository", new Integer(SqlParserSymbols.KW_REPOSITORY));
|
||||
keywordMap.put("repositories", new Integer(SqlParserSymbols.KW_REPOSITORIES));
|
||||
keywordMap.put("repository", new Integer(SqlParserSymbols.KW_REPOSITORY));
|
||||
keywordMap.put("resource", new Integer(SqlParserSymbols.KW_RESOURCE));
|
||||
keywordMap.put("restore", new Integer(SqlParserSymbols.KW_RESTORE));
|
||||
keywordMap.put("resume", new Integer(SqlParserSymbols.KW_RESUME));
|
||||
keywordMap.put("returns", new Integer(SqlParserSymbols.KW_RETURNS));
|
||||
keywordMap.put("revoke", new Integer(SqlParserSymbols.KW_REVOKE));
|
||||
keywordMap.put("right", new Integer(SqlParserSymbols.KW_RIGHT));
|
||||
@ -302,6 +300,7 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("roles", new Integer(SqlParserSymbols.KW_ROLES));
|
||||
keywordMap.put("rollback", new Integer(SqlParserSymbols.KW_ROLLBACK));
|
||||
keywordMap.put("rollup", new Integer(SqlParserSymbols.KW_ROLLUP));
|
||||
keywordMap.put("routine", new Integer(SqlParserSymbols.KW_ROUTINE));
|
||||
keywordMap.put("row", new Integer(SqlParserSymbols.KW_ROW));
|
||||
keywordMap.put("rows", new Integer(SqlParserSymbols.KW_ROWS));
|
||||
keywordMap.put("schemas", new Integer(SqlParserSymbols.KW_SCHEMAS));
|
||||
@ -311,6 +310,7 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("serializable", new Integer(SqlParserSymbols.KW_SERIALIZABLE));
|
||||
keywordMap.put("session", new Integer(SqlParserSymbols.KW_SESSION));
|
||||
keywordMap.put("set", new Integer(SqlParserSymbols.KW_SET));
|
||||
keywordMap.put("sets", new Integer(SqlParserSymbols.KW_SETS));
|
||||
keywordMap.put("show", new Integer(SqlParserSymbols.KW_SHOW));
|
||||
keywordMap.put("smallint", new Integer(SqlParserSymbols.KW_SMALLINT));
|
||||
keywordMap.put("snapshot", new Integer(SqlParserSymbols.KW_SNAPSHOT));
|
||||
@ -318,14 +318,17 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("split", new Integer(SqlParserSymbols.KW_SPLIT));
|
||||
keywordMap.put("start", new Integer(SqlParserSymbols.KW_START));
|
||||
keywordMap.put("status", new Integer(SqlParserSymbols.KW_STATUS));
|
||||
keywordMap.put("stop", new Integer(SqlParserSymbols.KW_STOP));
|
||||
keywordMap.put("storage", new Integer(SqlParserSymbols.KW_STORAGE));
|
||||
keywordMap.put("string", new Integer(SqlParserSymbols.KW_STRING));
|
||||
keywordMap.put("sum", new Integer(SqlParserSymbols.KW_SUM));
|
||||
keywordMap.put("superuser", new Integer(SqlParserSymbols.KW_SUPERUSER));
|
||||
keywordMap.put("sync", new Integer(SqlParserSymbols.KW_SYNC));
|
||||
keywordMap.put("system", new Integer(SqlParserSymbols.KW_SYSTEM));
|
||||
keywordMap.put("table", new Integer(SqlParserSymbols.KW_TABLE));
|
||||
keywordMap.put("tables", new Integer(SqlParserSymbols.KW_TABLES));
|
||||
keywordMap.put("tablet", new Integer(SqlParserSymbols.KW_TABLET));
|
||||
keywordMap.put("task", new Integer(SqlParserSymbols.KW_TASK));
|
||||
keywordMap.put("terminated", new Integer(SqlParserSymbols.KW_TERMINATED));
|
||||
keywordMap.put("than", new Integer(SqlParserSymbols.KW_THAN));
|
||||
keywordMap.put("then", new Integer(SqlParserSymbols.KW_THEN));
|
||||
@ -358,6 +361,7 @@ import org.apache.doris.qe.SqlModeHelper;
|
||||
keywordMap.put("whitelist", new Integer(SqlParserSymbols.KW_WHITELIST));
|
||||
keywordMap.put("when", new Integer(SqlParserSymbols.KW_WHEN));
|
||||
keywordMap.put("where", new Integer(SqlParserSymbols.KW_WHERE));
|
||||
keywordMap.put("whitelist", new Integer(SqlParserSymbols.KW_WHITELIST));
|
||||
keywordMap.put("with", new Integer(SqlParserSymbols.KW_WITH));
|
||||
keywordMap.put("work", new Integer(SqlParserSymbols.KW_WORK));
|
||||
keywordMap.put("write", new Integer(SqlParserSymbols.KW_WRITE));
|
||||
|
||||
@ -383,4 +383,138 @@ public class AccessTestUtil {
|
||||
};
|
||||
return analyzer;
|
||||
}
|
||||
|
||||
public static Analyzer fetchTableAnalyzer() {
|
||||
Column column1 = new Column("k1", PrimitiveType.VARCHAR);
|
||||
Column column2 = new Column("k2", PrimitiveType.VARCHAR);
|
||||
Column column3 = new Column("k3", PrimitiveType.VARCHAR);
|
||||
Column column4 = new Column("k4", PrimitiveType.BIGINT);
|
||||
|
||||
MaterializedIndex index = new MaterializedIndex();
|
||||
new Expectations(index) {
|
||||
{
|
||||
index.getId();
|
||||
minTimes = 0;
|
||||
result = 30000L;
|
||||
}
|
||||
};
|
||||
|
||||
Partition partition = Deencapsulation.newInstance(Partition.class);
|
||||
new Expectations(partition) {
|
||||
{
|
||||
partition.getBaseIndex();
|
||||
minTimes = 0;
|
||||
result = index;
|
||||
|
||||
partition.getIndex(30000L);
|
||||
minTimes = 0;
|
||||
result = index;
|
||||
}
|
||||
};
|
||||
|
||||
OlapTable table = new OlapTable();
|
||||
new Expectations(table) {
|
||||
{
|
||||
table.getBaseSchema();
|
||||
minTimes = 0;
|
||||
result = Lists.newArrayList(column1, column2, column3, column4);
|
||||
|
||||
table.getPartition(40000L);
|
||||
minTimes = 0;
|
||||
result = partition;
|
||||
|
||||
table.getColumn("k1");
|
||||
minTimes = 0;
|
||||
result = column1;
|
||||
|
||||
table.getColumn("k2");
|
||||
minTimes = 0;
|
||||
result = column2;
|
||||
|
||||
table.getColumn("k3");
|
||||
minTimes = 0;
|
||||
result = column3;
|
||||
|
||||
table.getColumn("k4");
|
||||
minTimes = 0;
|
||||
result = column4;
|
||||
}
|
||||
};
|
||||
|
||||
Database db = new Database();
|
||||
|
||||
new Expectations(db) {
|
||||
{
|
||||
db.getTable("t");
|
||||
minTimes = 0;
|
||||
result = table;
|
||||
|
||||
db.getTable("emptyTable");
|
||||
minTimes = 0;
|
||||
result = null;
|
||||
|
||||
db.getTableNamesWithLock();
|
||||
minTimes = 0;
|
||||
result = Sets.newHashSet("t");
|
||||
|
||||
db.getTables();
|
||||
minTimes = 0;
|
||||
result = Lists.newArrayList(table);
|
||||
|
||||
db.readLock();
|
||||
minTimes = 0;
|
||||
|
||||
db.readUnlock();
|
||||
minTimes = 0;
|
||||
|
||||
db.getFullName();
|
||||
minTimes = 0;
|
||||
result = "testDb";
|
||||
}
|
||||
};
|
||||
Catalog catalog = fetchBlockCatalog();
|
||||
Analyzer analyzer = new Analyzer(catalog, new ConnectContext(null));
|
||||
new Expectations(analyzer) {
|
||||
{
|
||||
analyzer.getDefaultDb();
|
||||
minTimes = 0;
|
||||
result = "testDb";
|
||||
|
||||
analyzer.getTable((TableName) any);
|
||||
minTimes = 0;
|
||||
result = table;
|
||||
|
||||
analyzer.getQualifiedUser();
|
||||
minTimes = 0;
|
||||
result = "testUser";
|
||||
|
||||
analyzer.getCatalog();
|
||||
minTimes = 0;
|
||||
result = catalog;
|
||||
|
||||
analyzer.getClusterName();
|
||||
minTimes = 0;
|
||||
result = "testCluster";
|
||||
|
||||
analyzer.incrementCallDepth();
|
||||
minTimes = 0;
|
||||
result = 1;
|
||||
|
||||
analyzer.decrementCallDepth();
|
||||
minTimes = 0;
|
||||
result = 0;
|
||||
|
||||
analyzer.getCallDepth();
|
||||
minTimes = 0;
|
||||
result = 1;
|
||||
|
||||
analyzer.getContext();
|
||||
minTimes = 0;
|
||||
result = new ConnectContext(null);
|
||||
|
||||
}
|
||||
};
|
||||
return analyzer;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -0,0 +1,285 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import com.google.common.collect.Multimap;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.BitSet;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
public class GroupByClauseTest {
|
||||
|
||||
private Analyzer analyzer;
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
Analyzer analyzerBase = AccessTestUtil.fetchTableAnalyzer();
|
||||
analyzer = new Analyzer(analyzerBase.getCatalog(), analyzerBase.getContext());
|
||||
try {
|
||||
Field f = analyzer.getClass().getDeclaredField("tupleByAlias");
|
||||
f.setAccessible(true);
|
||||
Multimap<String, TupleDescriptor> tupleByAlias = ArrayListMultimap.create();
|
||||
TupleDescriptor td = new TupleDescriptor(new TupleId(0));
|
||||
td.setTable(analyzerBase.getTable(new TableName("testdb", "t")));
|
||||
tupleByAlias.put("testdb.t", td);
|
||||
f.set(analyzer, tupleByAlias);
|
||||
} catch (NoSuchFieldException e) {
|
||||
e.printStackTrace();
|
||||
} catch (IllegalAccessException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupingSets() {
|
||||
List<ArrayList<Expr>> groupingExprsList = new ArrayList<>();
|
||||
ArrayList<Expr> groupByExprs = new ArrayList<>();
|
||||
String[][] colsLists = {
|
||||
{"k3", "k1"},
|
||||
{"k2", "k3", "k2"},
|
||||
{"k1", "k3"},
|
||||
{"k4"},
|
||||
{"k1", "k2", "k3", "k4"}
|
||||
};
|
||||
for (String[] colsList : colsLists) {
|
||||
ArrayList<Expr> exprList = new ArrayList<>();
|
||||
for (String col : colsList) {
|
||||
exprList.add(new SlotRef(new TableName("testdb", "t"), col));
|
||||
}
|
||||
groupingExprsList.add(exprList);
|
||||
}
|
||||
String[] groupByCols = {"k1", "k2", "k3", "k4"};
|
||||
for (String col : groupByCols) {
|
||||
groupByExprs.add(new SlotRef(new TableName("testdb", "t"), col));
|
||||
}
|
||||
GroupByClause groupByClause = new GroupByClause(groupingExprsList,
|
||||
GroupByClause.GroupingType.GROUPING_SETS);
|
||||
GroupingInfo groupingInfo = null;
|
||||
try {
|
||||
groupingInfo = new GroupingInfo(analyzer, GroupByClause.GroupingType.GROUPING_SETS);
|
||||
groupByClause.genGroupingExprs();
|
||||
groupingInfo.buildRepeat(groupByClause.getGroupingExprs(), groupByClause.getGroupingSetList());
|
||||
groupByClause.analyze(analyzer);
|
||||
} catch (AnalysisException execption) {
|
||||
execption.printStackTrace();
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
Assert.assertEquals(5, groupByClause.getGroupingExprs().size());
|
||||
|
||||
Assert.assertEquals("GROUPING SETS ((`testdb`.`t`.`k3`, `testdb`.`t`.`k1`), (`testdb`.`t`.`k2`, `testdb`.`t`"
|
||||
+ ".`k3`, `testdb`.`t`.`k2`), (`testdb`.`t`.`k1`, `testdb`.`t`.`k3`), (`testdb`.`t`.`k4`), (`testdb`"
|
||||
+ ".`t`.`k1`, `testdb`.`t`.`k2`, `testdb`.`t`.`k3`, `testdb`.`t`.`k4`))", groupByClause.toSql());
|
||||
List<BitSet> bitSetList = groupingInfo.getGroupingIdList();
|
||||
bitSetList.remove(0);
|
||||
|
||||
{
|
||||
String[] answer = {"{0, 1}", "{0, 2}", "{3}"};
|
||||
Set<String> answerSet = new HashSet<String>(Arrays.asList(answer));
|
||||
Set<String> resultSet = new HashSet<>();
|
||||
for (BitSet aBitSetList : bitSetList) {
|
||||
String s = aBitSetList.toString();
|
||||
resultSet.add(s);
|
||||
}
|
||||
Assert.assertEquals(answerSet, resultSet);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollUp() {
|
||||
ArrayList<Expr> groupingExprs = new ArrayList<>();
|
||||
String[] cols = {"k2", "k3", "k4", "k3"};
|
||||
for (String col : cols) {
|
||||
Expr expr = new SlotRef(new TableName("testdb", "t"), col);
|
||||
groupingExprs.add(expr);
|
||||
}
|
||||
|
||||
GroupByClause groupByClause =
|
||||
new GroupByClause(
|
||||
Expr.cloneList(groupingExprs),
|
||||
GroupByClause.GroupingType.ROLLUP);
|
||||
GroupingInfo groupingInfo = null;
|
||||
try {
|
||||
groupingInfo = new GroupingInfo(analyzer, GroupByClause.GroupingType.ROLLUP);
|
||||
groupByClause.genGroupingExprs();
|
||||
groupingInfo.buildRepeat(groupByClause.getGroupingExprs(), groupByClause.getGroupingSetList());
|
||||
groupByClause.analyze(analyzer);
|
||||
} catch (AnalysisException execption) {
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
Assert.assertEquals(4, groupByClause.getGroupingExprs().size());
|
||||
Assert.assertEquals("ROLLUP (`testdb`.`t`.`k2`, `testdb`.`t`.`k3`, "
|
||||
+ "`testdb`.`t`.`k4`, `testdb`.`t`.`k3`)", groupByClause.toSql());
|
||||
List<BitSet> bitSetList = groupingInfo.getGroupingIdList();
|
||||
bitSetList.remove(0);
|
||||
|
||||
{
|
||||
String[] answer = {"{}", "{0}", "{0, 1}"};
|
||||
Set<String> answerSet = new HashSet<String>(Arrays.asList(answer));
|
||||
Set<String> resultSet = new HashSet<>();
|
||||
for (BitSet aBitSetList : bitSetList) {
|
||||
String s = aBitSetList.toString();
|
||||
resultSet.add(s);
|
||||
}
|
||||
Assert.assertEquals(answerSet, resultSet);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCube() {
|
||||
ArrayList<Expr> groupingExprs = new ArrayList<>();
|
||||
String[] cols = {"k1", "k2", "k3", "k1"};
|
||||
for (String col : cols) {
|
||||
Expr expr = new SlotRef(new TableName("testdb", "t"), col);
|
||||
groupingExprs.add(expr);
|
||||
}
|
||||
|
||||
GroupByClause groupByClause = new GroupByClause(Expr.cloneList(groupingExprs),
|
||||
GroupByClause.GroupingType.CUBE);
|
||||
GroupingInfo groupingInfo = null;
|
||||
try {
|
||||
groupingInfo = new GroupingInfo(analyzer, GroupByClause.GroupingType.CUBE);
|
||||
groupByClause.genGroupingExprs();
|
||||
groupingInfo.buildRepeat(groupByClause.getGroupingExprs(), groupByClause.getGroupingSetList());
|
||||
groupByClause.analyze(analyzer);
|
||||
} catch (AnalysisException exception) {
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
Assert.assertEquals("CUBE (`testdb`.`t`.`k1`, `testdb`.`t`.`k2`, "
|
||||
+ "`testdb`.`t`.`k3`, `testdb`.`t`.`k1`)", groupByClause.toSql());
|
||||
Assert.assertEquals(4, groupByClause.getGroupingExprs().size());
|
||||
|
||||
List<BitSet> bitSetList = groupingInfo.getGroupingIdList();
|
||||
bitSetList.remove(0);
|
||||
|
||||
{
|
||||
String[] answer = {"{}", "{1}", "{0}", "{0, 1}", "{2}", "{1, 2}", "{0, 2}"};
|
||||
Set<String> answerSet = new HashSet<String>(Arrays.asList(answer));
|
||||
Set<String> resultSet = new HashSet<>();
|
||||
for (BitSet aBitSetList : bitSetList) {
|
||||
String s = aBitSetList.toString();
|
||||
resultSet.add(s);
|
||||
}
|
||||
|
||||
Assert.assertEquals(answerSet, resultSet);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupBy() {
|
||||
ArrayList<Expr> groupingExprs = new ArrayList<>();
|
||||
String[] cols = {"k2", "k2", "k3", "k1"};
|
||||
for (String col : cols) {
|
||||
Expr expr = new SlotRef(new TableName("testdb", "t"), col);
|
||||
groupingExprs.add(expr);
|
||||
}
|
||||
|
||||
GroupByClause groupByClause = new GroupByClause(Expr.cloneList(groupingExprs),
|
||||
GroupByClause.GroupingType.GROUP_BY);
|
||||
try {
|
||||
groupByClause.analyze(analyzer);
|
||||
} catch (AnalysisException execption) {
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
Assert.assertEquals("`testdb`.`t`.`k2`, `testdb`.`t`.`k2`, `testdb`.`t`.`k3`, `testdb`.`t`.`k1`", groupByClause.toSql());
|
||||
Assert.assertEquals(3, groupByClause.getGroupingExprs().size());
|
||||
groupingExprs.remove(0);
|
||||
Assert.assertEquals(groupByClause.getGroupingExprs(), groupingExprs);
|
||||
}
|
||||
@Test
|
||||
public void testReset() {
|
||||
ArrayList<Expr> groupingExprs = new ArrayList<>();
|
||||
String[] cols = {"k2", "k2", "k3", "k1"};
|
||||
for (String col : cols) {
|
||||
Expr expr = new SlotRef(new TableName("testdb", "t"), col);
|
||||
groupingExprs.add(expr);
|
||||
}
|
||||
|
||||
GroupByClause groupByClause = new GroupByClause(Expr.cloneList(groupingExprs),
|
||||
GroupByClause.GroupingType.GROUP_BY);
|
||||
try {
|
||||
groupByClause.analyze(analyzer);
|
||||
} catch (AnalysisException execption) {
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
try {
|
||||
groupByClause.reset();
|
||||
} catch (Exception e) {
|
||||
Assert.fail("reset throw exceptions!" + e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetTuple() throws AnalysisException {
|
||||
ArrayList<Expr> groupingExprs = new ArrayList<>();
|
||||
String[] cols = {"k1", "k2", "k3", "k1"};
|
||||
for (String col : cols) {
|
||||
Expr expr = new SlotRef(new TableName("testdb", "t"), col);
|
||||
groupingExprs.add(expr);
|
||||
}
|
||||
GroupByClause groupByClause = new GroupByClause(Expr.cloneList(groupingExprs),
|
||||
GroupByClause.GroupingType.GROUP_BY);
|
||||
try {
|
||||
groupByClause.analyze(analyzer);
|
||||
} catch (AnalysisException exception) {
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGenGroupingList() throws AnalysisException {
|
||||
ArrayList<Expr> groupingExprs = new ArrayList<>();
|
||||
String[] cols = {"k1", "k2", "k3"};
|
||||
for (String col : cols) {
|
||||
Expr expr = new SlotRef(new TableName("testdb", "t"), col);
|
||||
groupingExprs.add(expr);
|
||||
}
|
||||
|
||||
GroupByClause groupByClause = new GroupByClause(Expr.cloneList(groupingExprs),
|
||||
GroupByClause.GroupingType.CUBE);
|
||||
List<Expr> slots = new ArrayList<>();
|
||||
for (String col : cols) {
|
||||
SlotRef expr = new SlotRef(new TableName("testdb", "t"), col);
|
||||
slots.add(expr);
|
||||
}
|
||||
GroupingInfo groupingInfo = null;
|
||||
try {
|
||||
groupingInfo = new GroupingInfo(analyzer, GroupByClause.GroupingType.CUBE);
|
||||
groupingInfo.addGroupingSlots(slots, analyzer);
|
||||
groupByClause.genGroupingExprs();
|
||||
groupingInfo.buildRepeat(groupByClause.getGroupingExprs(), groupByClause.getGroupingSetList());
|
||||
groupByClause.analyze(analyzer);
|
||||
} catch (AnalysisException exception) {
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
List<List<Long>> list = groupingInfo.genGroupingList(groupByClause.getGroupingExprs());
|
||||
Assert.assertEquals(2, list.size());
|
||||
Assert.assertEquals(list.get(0), list.get(1));
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,113 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
package org.apache.doris.analysis;
|
||||
|
||||
import org.apache.doris.catalog.Type;
|
||||
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import com.google.common.collect.Multimap;
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class VirtualSlotRefTest {
|
||||
private Analyzer analyzer;
|
||||
private List<Expr> slots;
|
||||
private TupleDescriptor virtualTuple;
|
||||
private VirtualSlotRef virtualSlot;
|
||||
DataOutputStream dos;
|
||||
File file;
|
||||
DataInputStream dis;
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException {
|
||||
Analyzer analyzerBase = AccessTestUtil.fetchTableAnalyzer();
|
||||
analyzer = new Analyzer(analyzerBase.getCatalog(), analyzerBase.getContext());
|
||||
String[] cols = {"k1", "k2", "k3"};
|
||||
slots = new ArrayList<>();
|
||||
for (String col : cols) {
|
||||
SlotRef expr = new SlotRef(new TableName("testdb", "t"), col);
|
||||
slots.add(expr);
|
||||
}
|
||||
try {
|
||||
Field f = analyzer.getClass().getDeclaredField("tupleByAlias");
|
||||
f.setAccessible(true);
|
||||
Multimap<String, TupleDescriptor> tupleByAlias = ArrayListMultimap.create();
|
||||
TupleDescriptor td = new TupleDescriptor(new TupleId(0));
|
||||
td.setTable(analyzerBase.getTable(new TableName("testdb", "t")));
|
||||
tupleByAlias.put("testdb.t", td);
|
||||
f.set(analyzer, tupleByAlias);
|
||||
} catch (NoSuchFieldException e) {
|
||||
e.printStackTrace();
|
||||
} catch (IllegalAccessException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
virtualTuple = analyzer.getDescTbl().createTupleDescriptor("VIRTUAL_TUPLE");
|
||||
virtualSlot = new VirtualSlotRef("colName", Type.BIGINT, virtualTuple, slots);
|
||||
file = new File("./virtualSlot");
|
||||
file.createNewFile();
|
||||
dos = new DataOutputStream(new FileOutputStream(file));
|
||||
dis = new DataInputStream(new FileInputStream(file));
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
dis.close();
|
||||
dos.close();
|
||||
file.delete();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void read() throws IOException {
|
||||
virtualSlot.write(dos);
|
||||
virtualSlot.setRealSlots(slots);
|
||||
VirtualSlotRef v = VirtualSlotRef.read(dis);
|
||||
Assert.assertEquals(3, v.getRealSlots().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClone() {
|
||||
Expr v = virtualSlot.clone();
|
||||
Assert.assertTrue(v instanceof VirtualSlotRef);
|
||||
Assert.assertTrue(((VirtualSlotRef) v).getRealSlots().get(0).equals(virtualSlot.getRealSlots().get(0)));
|
||||
Assert.assertFalse(((VirtualSlotRef) v).getRealSlots().get(0) == virtualSlot.getRealSlots().get(0));
|
||||
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void analyzeImpl() {
|
||||
try {
|
||||
virtualSlot.analyzeImpl(analyzer);
|
||||
} catch (Exception e) {
|
||||
Assert.fail("analyze throw exception");
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,97 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
package org.apache.doris.planner;
|
||||
|
||||
import org.apache.doris.analysis.AccessTestUtil;
|
||||
import org.apache.doris.analysis.Analyzer;
|
||||
import org.apache.doris.analysis.DescriptorTable;
|
||||
import org.apache.doris.analysis.SlotId;
|
||||
import org.apache.doris.analysis.SlotRef;
|
||||
import org.apache.doris.analysis.TableName;
|
||||
import org.apache.doris.analysis.TupleDescriptor;
|
||||
import org.apache.doris.analysis.TupleId;
|
||||
import org.apache.doris.thrift.TExplainLevel;
|
||||
import org.apache.doris.thrift.TPlanNode;
|
||||
import org.apache.doris.thrift.TPlanNodeType;
|
||||
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import com.google.common.collect.Multimap;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
public class RepeatNodeTest {
|
||||
private Analyzer analyzer;
|
||||
private RepeatNode node;
|
||||
private TupleDescriptor virtualTuple;
|
||||
private List<Set<SlotId>> groupingIdList = new ArrayList<>();
|
||||
private List<List<Long>> groupingList = new ArrayList<>();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
Analyzer analyzerBase = AccessTestUtil.fetchTableAnalyzer();
|
||||
analyzer = new Analyzer(analyzerBase.getCatalog(), analyzerBase.getContext());
|
||||
String[] cols = {"k1", "k2", "k3"};
|
||||
List<SlotRef> slots = new ArrayList<>();
|
||||
for (String col : cols) {
|
||||
SlotRef expr = new SlotRef(new TableName("testdb", "t"), col);
|
||||
slots.add(expr);
|
||||
}
|
||||
try {
|
||||
Field f = analyzer.getClass().getDeclaredField("tupleByAlias");
|
||||
f.setAccessible(true);
|
||||
Multimap<String, TupleDescriptor> tupleByAlias = ArrayListMultimap.create();
|
||||
TupleDescriptor td = new TupleDescriptor(new TupleId(0));
|
||||
td.setTable(analyzerBase.getTable(new TableName("testdb", "t")));
|
||||
tupleByAlias.put("testdb.t", td);
|
||||
f.set(analyzer, tupleByAlias);
|
||||
} catch (NoSuchFieldException e) {
|
||||
e.printStackTrace();
|
||||
} catch (IllegalAccessException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
virtualTuple = analyzer.getDescTbl().createTupleDescriptor("VIRTUAL_TUPLE");
|
||||
groupingList.add(Arrays.asList(0L, 7L, 3L, 5L, 1L, 6L, 2L, 4L));
|
||||
groupingList.add(Arrays.asList(0L, 7L, 3L, 5L, 1L, 6L, 2L, 4L));
|
||||
DescriptorTable descTable = new DescriptorTable();
|
||||
TupleDescriptor tuple = descTable.createTupleDescriptor("DstTable");
|
||||
node = new RepeatNode(new PlanNodeId(1),
|
||||
new OlapScanNode(new PlanNodeId(0), tuple, "null"), groupingIdList, virtualTuple, groupingList);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNornal() {
|
||||
try {
|
||||
TPlanNode msg = new TPlanNode();
|
||||
node.toThrift(msg);
|
||||
node.getNodeExplainString("", TExplainLevel.NORMAL);
|
||||
node.debugString();
|
||||
Assert.assertEquals(TPlanNodeType.REPEAT_NODE, msg.node_type);
|
||||
} catch (Exception e) {
|
||||
Assert.fail("throw exceptions");
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -699,7 +699,11 @@ visible_functions = [
|
||||
[['ST_Contains'], 'BOOLEAN', ['VARCHAR', 'VARCHAR'],
|
||||
'_ZN5doris12GeoFunctions11st_containsEPN9doris_udf15FunctionContextERKNS1_9StringValES6_',
|
||||
'_ZN5doris12GeoFunctions19st_contains_prepareEPN9doris_udf15FunctionContextENS2_18FunctionStateScopeE',
|
||||
'_ZN5doris12GeoFunctions17st_contains_closeEPN9doris_udf15FunctionContextENS2_18FunctionStateScopeE']
|
||||
'_ZN5doris12GeoFunctions17st_contains_closeEPN9doris_udf15FunctionContextENS2_18FunctionStateScopeE'],
|
||||
# grouping sets functions
|
||||
[['grouping_id'], 'BIGINT', ['BIGINT'],
|
||||
'_ZN5doris21GroupingSetsFunctions11grouping_idEPN9doris_udf15FunctionContextERKNS1_9BigIntValE'],
|
||||
[['grouping'], 'BIGINT', ['BIGINT'], '_ZN5doris21GroupingSetsFunctions8groupingEPN9doris_udf15FunctionContextERKNS1_9BigIntValE'],
|
||||
]
|
||||
|
||||
invisible_functions = [
|
||||
|
||||
@ -46,6 +46,7 @@ enum TPlanNodeType {
|
||||
UNION_NODE,
|
||||
ES_SCAN_NODE,
|
||||
ES_HTTP_SCAN_NODE,
|
||||
REPEAT_NODE
|
||||
ASSERT_NUM_ROWS_NODE
|
||||
}
|
||||
|
||||
@ -386,6 +387,17 @@ struct TAggregationNode {
|
||||
6: optional bool use_streaming_preaggregation
|
||||
}
|
||||
|
||||
struct TRepeatNode {
|
||||
// Tulple id used for output, it has new slots.
|
||||
1: required Types.TTupleId output_tuple_id
|
||||
// Slot id set used to indicate those slots need to set to null.
|
||||
2: required list<set<Types.TSlotId>> slot_id_set_list
|
||||
// An integer bitmap list, it indicates the bit position of the exprs not null.
|
||||
3: required list<i64> repeat_id_list
|
||||
// A list of integer list, it indicates the position of the grouping virtual slot.
|
||||
4: required list<list<i64>> grouping_list
|
||||
}
|
||||
|
||||
struct TPreAggregationNode {
|
||||
1: required list<Exprs.TExpr> group_exprs
|
||||
2: required list<Exprs.TExpr> aggregate_exprs
|
||||
@ -613,7 +625,8 @@ struct TPlanNode {
|
||||
28: optional TUnionNode union_node
|
||||
29: optional TBackendResourceProfile resource_profile
|
||||
30: optional TEsScanNode es_scan_node
|
||||
31: optional TAssertNumRowsNode assert_num_rows_node
|
||||
31: optional TRepeatNode repeat_node
|
||||
32: optional TAssertNumRowsNode assert_num_rows_node
|
||||
}
|
||||
|
||||
// A flattened representation of a tree of PlanNodes, obtained by depth-first
|
||||
|
||||
Reference in New Issue
Block a user