[Bug](pipelineX) Fix runtime filter on pipelineX (#24141)

This commit is contained in:
Gabriel
2023-09-10 20:53:54 +08:00
committed by GitHub
parent a05003fbe1
commit ff92b5bc29
45 changed files with 85 additions and 112 deletions

View File

@ -98,6 +98,11 @@ bool ExchangeSinkLocalState::transfer_large_data_by_brpc() const {
Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) {
RETURN_IF_ERROR(PipelineXSinkLocalState<>::init(state, info));
_sender_id = info.sender_id;
return Status::OK();
}
Status ExchangeSinkLocalState::open(RuntimeState* state) {
RETURN_IF_ERROR(PipelineXSinkLocalState<>::open(state));
_broadcast_pb_blocks.resize(config::num_broadcast_buffer);
_broadcast_pb_block_idx = 0;
auto& p = _parent->cast<ExchangeSinkOperatorX>();

View File

@ -77,6 +77,7 @@ public:
_serializer(this) {}
Status init(RuntimeState* state, LocalSinkStateInfo& info) override;
Status open(RuntimeState* state) override;
Status close(RuntimeState* state) override;
Status serialize_block(vectorized::Block* src, PBlock* dest, int num_receivers = 1);

View File

@ -45,7 +45,10 @@ ExchangeLocalState::ExchangeLocalState(RuntimeState* state, OperatorXBase* paren
Status ExchangeLocalState::init(RuntimeState* state, LocalStateInfo& info) {
RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info));
stream_recvr = info.recvr;
auto& p = _parent->cast<ExchangeSourceOperatorX>();
stream_recvr = state->exec_env()->vstream_mgr()->create_recvr(
state, p.input_row_desc(), state->fragment_instance_id(), p.id(), p.num_senders(),
profile(), p.is_merging(), p.sub_plan_query_statistics_recvr());
RETURN_IF_ERROR(_parent->cast<ExchangeSourceOperatorX>()._vsort_exec_exprs.clone(
state, vsort_exec_exprs));
return Status::OK();

View File

@ -80,7 +80,6 @@ public:
Status close(RuntimeState* state) override;
bool is_source() const override { return true; }
bool need_to_create_exch_recv() const override { return true; }
RowDescriptor input_row_desc() const { return _input_row_desc; }

View File

@ -52,10 +52,18 @@ bool ResultSinkOperator::can_write() {
Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) {
RETURN_IF_ERROR(PipelineXSinkLocalState<>::init(state, info));
auto& p = _parent->cast<ResultSinkOperatorX>();
auto fragment_instance_id = state->fragment_instance_id();
// create sender
_sender = info.sender;
std::shared_ptr<BufferControlBlock> sender = nullptr;
RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(
state->fragment_instance_id(), vectorized::RESULT_SINK_BUFFER_SIZE, &_sender, true,
state->execution_timeout()));
return Status::OK();
}
Status ResultSinkLocalState::open(RuntimeState* state) {
RETURN_IF_ERROR(PipelineXSinkLocalState<>::open(state));
auto& p = _parent->cast<ResultSinkOperatorX>();
_output_vexpr_ctxs.resize(p._output_vexpr_ctxs.size());
for (size_t i = 0; i < _output_vexpr_ctxs.size(); i++) {
RETURN_IF_ERROR(p._output_vexpr_ctxs[i]->clone(state, _output_vexpr_ctxs[i]));

View File

@ -50,7 +50,7 @@ public:
: PipelineXSinkLocalState<>(parent, state) {}
Status init(RuntimeState* state, LocalSinkStateInfo& info) override;
Status open(RuntimeState* state) override;
Status close(RuntimeState* state) override;
private:
@ -74,8 +74,6 @@ public:
bool can_write(RuntimeState* state) override;
[[nodiscard]] bool need_to_create_result_sender() const override { return true; }
private:
friend class ResultSinkLocalState;

View File

@ -25,14 +25,12 @@ namespace doris::pipeline {
struct LocalStateInfo {
const std::vector<TScanRangeParams> scan_ranges;
Dependency* dependency;
std::shared_ptr<vectorized::VDataStreamRecvr> recvr;
};
// This struct is used only for initializing local sink state.
struct LocalSinkStateInfo {
const int sender_id;
Dependency* dependency;
std::shared_ptr<BufferControlBlock> sender;
};
class PipelineXLocalStateBase {
@ -232,7 +230,6 @@ public:
}
[[nodiscard]] virtual bool is_source() const override { return false; }
[[nodiscard]] virtual bool need_to_create_exch_recv() const { return false; }
Status get_next_after_projects(RuntimeState* state, vectorized::Block* block,
SourceState& source_state);
@ -345,6 +342,8 @@ public:
virtual ~PipelineXSinkLocalStateBase() {}
virtual Status init(RuntimeState* state, LocalSinkStateInfo& info) = 0;
virtual Status open(RuntimeState* state) { return Status::OK(); }
virtual Status close(RuntimeState* state) = 0;
virtual std::string debug_string(int indentation_level) const;
@ -402,8 +401,6 @@ public:
virtual Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) = 0;
[[nodiscard]] virtual bool need_to_create_result_sender() const { return false; }
template <class TARGET>
TARGET& cast() {
DCHECK(dynamic_cast<TARGET*>(this))

View File

@ -299,33 +299,9 @@ Status PipelineXFragmentContext::_build_pipeline_tasks(
std::map<PipelineId, PipelineXTask*> pipeline_id_to_task;
for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) {
auto scan_ranges = find_with_default(local_params.per_node_scan_ranges,
_pipelines[pip_idx]->operator_xs().front()->id(),
no_scan_ranges);
std::shared_ptr<BufferControlBlock> sender = nullptr;
if (_pipelines[pip_idx]->sink_x()->need_to_create_result_sender()) {
// create sender
RETURN_IF_ERROR(_runtime_states[i]->exec_env()->result_mgr()->create_sender(
_runtime_states[i]->fragment_instance_id(),
vectorized::RESULT_SINK_BUFFER_SIZE, &sender, true,
_runtime_states[i]->execution_timeout()));
}
std::shared_ptr<vectorized::VDataStreamRecvr> recvr = nullptr;
if (_pipelines[pip_idx]->operator_xs().front()->need_to_create_exch_recv()) {
auto* src =
(ExchangeSourceOperatorX*)_pipelines[pip_idx]->operator_xs().front().get();
recvr = _runtime_states[i]->exec_env()->vstream_mgr()->create_recvr(
_runtime_states[i].get(), src->input_row_desc(),
_runtime_states[i]->fragment_instance_id(), src->id(), src->num_senders(),
_runtime_profile.get(), src->is_merging(),
src->sub_plan_query_statistics_recvr());
}
auto task = std::make_unique<PipelineXTask>(
_pipelines[pip_idx], _total_tasks++, _runtime_states[i].get(), this,
_pipelines[pip_idx]->pipeline_profile(), scan_ranges, local_params.sender_id,
sender, recvr);
auto task = std::make_unique<PipelineXTask>(_pipelines[pip_idx], _total_tasks++,
_runtime_states[i].get(), this,
_pipelines[pip_idx]->pipeline_profile());
pipeline_id_to_task.insert({_pipelines[pip_idx]->id(), task.get()});
RETURN_IF_ERROR(task->prepare(_runtime_states[i].get()));
_runtime_profile->add_child(_pipelines[pip_idx]->pipeline_profile(), true, nullptr);
@ -361,6 +337,19 @@ Status PipelineXFragmentContext::_build_pipeline_tasks(
pipeline_id_to_task[dep]->get_downstream_dependency());
}
}
auto scan_ranges = find_with_default(local_params.per_node_scan_ranges,
_pipelines[pip_idx]->operator_xs().front()->id(),
no_scan_ranges);
for (auto& op : _pipelines[pip_idx]->operator_xs()) {
LocalStateInfo info {scan_ranges, task->get_upstream_dependency(op->id())};
RETURN_IF_ERROR(op->setup_local_state(_runtime_states[i].get(), info));
}
LocalSinkStateInfo info {local_params.sender_id,
task->get_downstream_dependency().get()};
RETURN_IF_ERROR(_pipelines[pip_idx]->sink_x()->setup_local_state(
_runtime_states[i].get(), info));
}
{

View File

@ -44,19 +44,12 @@ namespace doris::pipeline {
PipelineXTask::PipelineXTask(PipelinePtr& pipeline, uint32_t index, RuntimeState* state,
PipelineFragmentContext* fragment_context,
RuntimeProfile* parent_profile,
const std::vector<TScanRangeParams>& scan_ranges, const int sender_id,
std::shared_ptr<BufferControlBlock>& sender,
std::shared_ptr<vectorized::VDataStreamRecvr>& recvr)
RuntimeProfile* parent_profile)
: PipelineTask(pipeline, index, state, fragment_context, parent_profile),
_scan_ranges(scan_ranges),
_operators(pipeline->operator_xs()),
_source(_operators.front()),
_root(_operators.back()),
_sink(pipeline->sink_shared_pointer()),
_sender_id(sender_id),
_sender(sender),
_recvr(recvr) {
_sink(pipeline->sink_shared_pointer()) {
_pipeline_task_watcher.start();
_sink->get_dependency(_downstream_dependency);
}
@ -98,27 +91,11 @@ Status PipelineXTask::_open() {
SCOPED_TIMER(_task_profile->total_time_counter());
SCOPED_CPU_TIMER(_task_cpu_timer);
SCOPED_TIMER(_open_timer);
if (!_init_local_state) {
Status st = Status::OK();
for (auto& o : _operators) {
Dependency* dep = _upstream_dependency.find(o->id()) == _upstream_dependency.end()
? (Dependency*)nullptr
: _upstream_dependency.find(o->id())->second.get();
LocalStateInfo info {_scan_ranges, dep, _recvr};
Status cur_st = o->setup_local_state(_state, info);
if (!cur_st.ok()) {
st = cur_st;
}
}
LocalSinkStateInfo info {_sender_id, _downstream_dependency.get(), _sender};
RETURN_IF_ERROR(_sink->setup_local_state(_state, info));
_dry_run = _sink->should_dry_run(_state);
RETURN_IF_ERROR(st);
_init_local_state = true;
}
_dry_run = _sink->should_dry_run(_state);
for (auto& o : _operators) {
RETURN_IF_ERROR(_state->get_local_state(o->id())->open(_state));
}
RETURN_IF_ERROR(_state->get_sink_local_state(_sink->id())->open(_state));
_opened = true;
return Status::OK();
}

View File

@ -50,10 +50,7 @@ class PriorityTaskQueue;
class PipelineXTask : public PipelineTask {
public:
PipelineXTask(PipelinePtr& pipeline, uint32_t index, RuntimeState* state,
PipelineFragmentContext* fragment_context, RuntimeProfile* parent_profile,
const std::vector<TScanRangeParams>& scan_ranges, const int sender_id,
std::shared_ptr<BufferControlBlock>& sender,
std::shared_ptr<vectorized::VDataStreamRecvr>& recvr);
PipelineFragmentContext* fragment_context, RuntimeProfile* parent_profile);
Status prepare(RuntimeState* state) override;
@ -110,26 +107,25 @@ public:
_upstream_dependency.insert({upstream_dependency->id(), upstream_dependency});
}
Dependency* get_upstream_dependency(int id) {
return _upstream_dependency.find(id) == _upstream_dependency.end()
? (Dependency*)nullptr
: _upstream_dependency.find(id)->second.get();
}
private:
using DependencyMap = std::map<int, DependencySPtr>;
Status _open() override;
const std::vector<TScanRangeParams> _scan_ranges;
OperatorXs _operators; // left is _source, right is _root
OperatorXPtr _source;
OperatorXPtr _root;
DataSinkOperatorXPtr _sink;
const int _sender_id;
DependencyMap _upstream_dependency;
DependencySPtr _downstream_dependency;
std::shared_ptr<BufferControlBlock> _sender;
std::shared_ptr<vectorized::VDataStreamRecvr> _recvr;
bool _dry_run = false;
bool _init_local_state = false;
};
} // namespace doris::pipeline

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ SUM(lo_extendedprice*lo_discount) AS
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS
REVENUE
FROM lineorder, date
WHERE lo_orderdate = d_datekey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ SUM(lo_extendedprice*lo_discount) AS
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS
REVENUE
FROM lineorder, date
WHERE lo_orderdate = d_datekey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ SUM(lo_extendedprice*lo_discount) AS
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS
REVENUE
FROM lineorder, date
WHERE lo_orderdate = d_datekey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ SUM(lo_revenue), d_year, p_brand
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand
FROM lineorder, date, part, supplier
WHERE lo_orderdate = d_datekey
AND lo_partkey = p_partkey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ SUM(lo_revenue), d_year, p_brand
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand
FROM lineorder, date, part, supplier
WHERE lo_orderdate = d_datekey
AND lo_partkey = p_partkey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ SUM(lo_revenue), d_year, p_brand
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand
FROM lineorder, date, part, supplier
WHERE lo_orderdate = d_datekey
AND lo_partkey = p_partkey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ c_nation, s_nation, d_year,
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_nation, s_nation, d_year,
SUM(lo_revenue) AS REVENUE
FROM customer, lineorder, supplier, date
WHERE lo_custkey = c_custkey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ c_city, s_city, d_year, sum(lo_revenue)
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, sum(lo_revenue)
AS REVENUE
FROM customer, lineorder, supplier, date
WHERE lo_custkey = c_custkey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ c_city, s_city, d_year, SUM(lo_revenue)
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, SUM(lo_revenue)
AS REVENUE
FROM customer, lineorder, supplier, date
WHERE lo_custkey = c_custkey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ c_city, s_city, d_year, SUM(lo_revenue)
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, SUM(lo_revenue)
AS REVENUE
FROM customer, lineorder, supplier, date
WHERE lo_custkey = c_custkey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ d_year, c_nation,
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, c_nation,
SUM(lo_revenue - lo_supplycost) AS PROFIT
FROM date, customer, supplier, part, lineorder
WHERE lo_custkey = c_custkey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ d_year, s_nation, p_category,
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, s_nation, p_category,
SUM(lo_revenue - lo_supplycost) AS PROFIT
FROM date, customer, supplier, part, lineorder
WHERE lo_custkey = c_custkey

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ d_year, s_city, p_brand,
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, s_city, p_brand,
SUM(lo_revenue - lo_supplycost) AS PROFIT
FROM date, customer, supplier, part, lineorder
WHERE lo_custkey = c_custkey

View File

@ -1,5 +1,5 @@
-- tables: lineitem
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
l_returnflag,
l_linestatus,
sum(l_quantity) AS sum_qty,

View File

@ -1,5 +1,5 @@
-- tables: part,supplier,partsupp,nation,region
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
s_acctbal,
s_name,
n_name,

View File

@ -1,5 +1,5 @@
-- tables: customer,orders,lineitem
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
l_orderkey,
sum(l_extendedprice * (1 - l_discount)) AS revenue,
o_orderdate,

View File

@ -1,5 +1,5 @@
-- tables: orders,lineitem
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
o_orderpriority,
count(*) AS order_count
FROM orders

View File

@ -1,5 +1,5 @@
-- tables: customer,orders,lineitem,supplier,nation,region
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
n_name,
sum(l_extendedprice * (1 - l_discount)) AS revenue
FROM

View File

@ -1,6 +1,6 @@
-- tables: lineitem
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ sum(l_extendedprice * l_discount) AS revenue
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice * l_discount) AS revenue
FROM
lineitem
WHERE

View File

@ -1,5 +1,5 @@
-- tables: supplier,lineitem,orders,customer,nation
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
supp_nation,
cust_nation,
l_year,

View File

@ -1,5 +1,5 @@
-- tables: part,supplier,lineitem,orders,customer,nation,region
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
o_year,
sum(CASE
WHEN nation = 'BRAZIL'

View File

@ -1,5 +1,5 @@
-- tables: part,supplier,lineitem,partsupp,orders,nation
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
nation,
o_year,
sum(amount) AS sum_profit

View File

@ -1,5 +1,5 @@
-- tables: customer,orders,lineitem,nation
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
c_custkey,
c_name,
sum(l_extendedprice * (1 - l_discount)) AS revenue,

View File

@ -1,5 +1,5 @@
-- tables: partsupp,supplier,nation
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
ps_partkey,
sum(ps_supplycost * ps_availqty) AS value
FROM

View File

@ -1,5 +1,5 @@
-- tables: orders,lineitem
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
l_shipmode,
sum(CASE
WHEN o_orderpriority = '1-URGENT'

View File

@ -1,5 +1,5 @@
-- tables: customer
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
c_count,
count(*) AS custdist
FROM (

View File

@ -1,5 +1,5 @@
-- tables: lineitem,part
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ 100.00 * sum(CASE
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ 100.00 * sum(CASE
WHEN p_type LIKE 'PROMO%'
THEN l_extendedprice * (1 - l_discount)
ELSE 0

View File

@ -1,4 +1,4 @@
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
s_suppkey,
s_name,
s_address,

View File

@ -1,5 +1,5 @@
-- tables: partsupp,part,supplier
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
p_brand,
p_type,
p_size,

View File

@ -1,5 +1,5 @@
-- tables: lineitem,part
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ sum(l_extendedprice) / 7.0 AS avg_yearly
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice) / 7.0 AS avg_yearly
FROM
lineitem,
part

View File

@ -1,5 +1,5 @@
-- tables: customer,orders,lineitem
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
c_name,
c_custkey,
o_orderkey,

View File

@ -1,5 +1,5 @@
-- tables: lineitem,part
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */ sum(l_extendedprice * (1 - l_discount)) AS revenue
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice * (1 - l_discount)) AS revenue
FROM
lineitem,
part

View File

@ -1,5 +1,5 @@
-- tables: supplier,nation,partsupp,lineitem,part
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
s_name,
s_address
FROM

View File

@ -1,5 +1,5 @@
-- tables: supplier,lineitem,orders,nation
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
s_name,
count(*) AS numwait
FROM

View File

@ -1,5 +1,5 @@
-- tables: orders,customer
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, runtime_filter_mode=OFF) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
cntrycode,
count(*) AS numcust,
sum(c_acctbal) AS totacctbal