[pipelineX](fix) Fix broadcast dependency hanging (#24740)

This commit is contained in:
Gabriel
2023-09-22 12:24:32 +08:00
committed by GitHub
parent d82b603b53
commit 034582bb64
62 changed files with 107 additions and 64 deletions

View File

@ -118,7 +118,9 @@ Status DataQueue::get_block_from_queue(std::unique_ptr<vectorized::Block>* outpu
_cur_bytes_in_queue[_flag_queue_idx] -= (*output_block)->allocated_bytes();
_cur_blocks_nums_in_queue[_flag_queue_idx] -= 1;
if (_dependency) {
_dependency->block_reading();
if (!_is_finished[_flag_queue_idx]) {
_dependency->block_reading();
}
_dependency->set_ready_for_write();
}
} else {
@ -150,6 +152,7 @@ void DataQueue::push_block(std::unique_ptr<vectorized::Block> block, int child_i
}
void DataQueue::set_finish(int child_idx) {
std::lock_guard<std::mutex> l(*_queue_blocks_lock[child_idx]);
_is_finished[child_idx] = true;
if (_dependency) {
_dependency->set_ready_for_read();
@ -157,6 +160,7 @@ void DataQueue::set_finish(int child_idx) {
}
void DataQueue::set_canceled(int child_idx) {
std::lock_guard<std::mutex> l(*_queue_blocks_lock[child_idx]);
DCHECK(!_is_finished[child_idx]);
_is_canceled[child_idx] = true;
_is_finished[child_idx] = true;

View File

@ -49,11 +49,12 @@ namespace vectorized {
void BroadcastPBlockHolder::unref() noexcept {
DCHECK_GT(_ref_count._value, 0);
_ref_count._value.fetch_sub(1);
if (_dep && _ref_count._value == 0) {
auto old_value = _ref_count._value.fetch_sub(1);
if (_dep && old_value == 1) {
_dep->return_available_block();
}
}
} // namespace vectorized
namespace pipeline {
@ -169,7 +170,8 @@ Status ExchangeSinkBuffer<Parent>::add_block(TransmitInfo<Parent>&& request) {
}
template <typename Parent>
Status ExchangeSinkBuffer<Parent>::add_block(BroadcastTransmitInfo<Parent>&& request) {
Status ExchangeSinkBuffer<Parent>::add_block(BroadcastTransmitInfo<Parent>&& request,
[[maybe_unused]] bool* sent) {
if (_is_finishing) {
return Status::OK();
}
@ -178,6 +180,9 @@ Status ExchangeSinkBuffer<Parent>::add_block(BroadcastTransmitInfo<Parent>&& req
return Status::EndOfFile("receiver eof");
}
bool send_now = false;
if (sent) {
*sent = true;
}
request.block_holder->ref();
{
std::unique_lock<std::mutex> lock(*_instance_to_package_queue_mutex[ins_id.lo]);

View File

@ -174,7 +174,7 @@ public:
void register_sink(TUniqueId);
Status add_block(TransmitInfo<Parent>&& request);
Status add_block(BroadcastTransmitInfo<Parent>&& request);
Status add_block(BroadcastTransmitInfo<Parent>&& request, [[maybe_unused]] bool* sent);
bool can_write() const;
bool is_pending_finish();
void close();

View File

@ -340,7 +340,6 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block
} else {
vectorized::BroadcastPBlockHolder* block_holder = nullptr;
RETURN_IF_ERROR(local_state.get_next_available_buffer(&block_holder));
local_state._broadcast_dependency->take_available_block();
{
SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
bool serialized = false;
@ -356,18 +355,22 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block
block_holder->get_block()->Clear();
}
Status status;
bool sent = false;
for (auto channel : local_state.channels) {
if (!channel->is_receiver_eof()) {
if (channel->is_local()) {
status = channel->send_local_block(&cur_block);
} else {
SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
status = channel->send_block(block_holder,
status = channel->send_block(block_holder, &sent,
source_state == SourceState::FINISHED);
}
HANDLE_CHANNEL_STATUS(state, channel, status);
}
}
if (sent) {
local_state._broadcast_dependency->take_available_block();
}
cur_block.clear_column_data();
local_state._serializer.get_block()->set_muatable_columns(
cur_block.mutate_columns());

View File

@ -82,6 +82,11 @@ public:
virtual ~BroadcastDependency() = default;
[[nodiscard]] WriteDependency* write_blocked_by() override {
if (config::enable_fuzzy_mode && _available_block == 0 &&
_write_dependency_watcher.elapsed_time() > SLOW_DEPENDENCY_THRESHOLD) {
LOG(WARNING) << "========Dependency may be blocked by some reasons: " << name() << " "
<< id();
}
return _available_block > 0 ? nullptr : this;
}

View File

@ -57,6 +57,11 @@ public:
: Dependency(id, "DataDependency"), _sender_queue(sender_queue), _always_done(false) {}
void* shared_state() override { return nullptr; }
[[nodiscard]] Dependency* read_blocked_by() override {
if (config::enable_fuzzy_mode && _sender_queue->should_wait() &&
_read_dependency_watcher.elapsed_time() > SLOW_DEPENDENCY_THRESHOLD) {
LOG(WARNING) << "========Dependency may be blocked by some reasons: " << name() << " "
<< id();
}
return _sender_queue->should_wait() ? this : nullptr;
}

View File

@ -101,6 +101,11 @@ public:
if (_scanner_ctx->get_num_running_scanners() == 0 && _scanner_ctx->should_be_scheduled()) {
_scanner_ctx->reschedule_scanner_ctx();
}
if (config::enable_fuzzy_mode && !_ready_for_read &&
_read_dependency_watcher.elapsed_time() > SLOW_DEPENDENCY_THRESHOLD) {
LOG(WARNING) << "========Dependency may be blocked by some reasons: " << name() << " "
<< id();
}
return _ready_for_read ? nullptr : this;
}

View File

@ -33,6 +33,8 @@ namespace pipeline {
class Dependency;
using DependencySPtr = std::shared_ptr<Dependency>;
static constexpr auto SLOW_DEPENDENCY_THRESHOLD = 10 * 1000L * 1000L * 1000L;
class Dependency : public std::enable_shared_from_this<Dependency> {
public:
Dependency(int id, std::string name) : _id(id), _name(name), _ready_for_read(false) {}
@ -57,7 +59,14 @@ public:
}
// Which dependency current pipeline task is blocked by. `nullptr` if this dependency is ready.
[[nodiscard]] virtual Dependency* read_blocked_by() { return _ready_for_read ? nullptr : this; }
[[nodiscard]] virtual Dependency* read_blocked_by() {
if (config::enable_fuzzy_mode && !_ready_for_read &&
_read_dependency_watcher.elapsed_time() > SLOW_DEPENDENCY_THRESHOLD) {
LOG(WARNING) << "========Dependency may be blocked by some reasons: " << name() << " "
<< id();
}
return _ready_for_read ? nullptr : this;
}
// Notify downstream pipeline tasks this dependency is ready.
virtual void set_ready_for_read() {
@ -118,6 +127,11 @@ public:
}
[[nodiscard]] virtual WriteDependency* write_blocked_by() {
if (config::enable_fuzzy_mode && !_ready_for_write &&
_write_dependency_watcher.elapsed_time() > SLOW_DEPENDENCY_THRESHOLD) {
LOG(WARNING) << "========Dependency may be blocked by some reasons: " << name() << " "
<< id();
}
return _ready_for_write ? nullptr : this;
}

View File

@ -538,7 +538,7 @@ Status VDataStreamSender::send(RuntimeState* state, Block* block, bool eos) {
status = channel->send_local_block(&cur_block);
} else {
SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
status = channel->send_block(block_holder, eos);
status = channel->send_block(block_holder, nullptr, eos);
}
HANDLE_CHANNEL_STATUS(state, channel, status);
}

View File

@ -275,7 +275,8 @@ public:
// if batch is nullptr, send the eof packet
virtual Status send_block(PBlock* block, bool eos = false);
virtual Status send_block(BroadcastPBlockHolder* block, bool eos = false) {
virtual Status send_block(BroadcastPBlockHolder* block, [[maybe_unused]] bool* sent,
bool eos = false) {
return Status::InternalError("Send BroadcastPBlockHolder is not allowed!");
}
@ -493,7 +494,8 @@ public:
return Status::OK();
}
Status send_block(BroadcastPBlockHolder* block, bool eos = false) override {
Status send_block(BroadcastPBlockHolder* block, [[maybe_unused]] bool* sent,
bool eos = false) override {
COUNTER_UPDATE(Channel<Parent>::_parent->blocks_sent_counter(), 1);
if (eos) {
if (_eos_send) {
@ -503,7 +505,7 @@ public:
}
}
if (eos || block->get_block()->column_metas_size()) {
RETURN_IF_ERROR(_buffer->add_block({this, block, eos}));
RETURN_IF_ERROR(_buffer->add_block({this, block, eos}, sent));
}
return Status::OK();
}

View File

@ -61,7 +61,7 @@ suite("test_distinct_streaming_agg_operator") {
select k6 from baseall group by k6 order by k6;
"""
sql"""set experimental_enable_pipeline_x_engine=false; """
sql"""set experimental_enable_pipeline_x_engine=true; """
qt_pipelineX_1 """
select * from ( select k1,k2,k3 from baseall union select k1,k2,k3 from baseall) as t ORDER BY 1, 2,3;

View File

@ -64,7 +64,7 @@ suite("test_partition_sort_operator") {
ORDER BY 1, 2,3;
"""
sql"""set experimental_enable_pipeline_x_engine=false; """
sql"""set experimental_enable_pipeline_x_engine=true; """
qt_pipelineX_1 """
select * from (select k6,k2,row_number() over(partition by k6 order by k2) as num from baseall) as res where num < 5

View File

@ -70,7 +70,7 @@ suite("test_repeat_operator") {
ORDER BY k1, k2,k3;
"""
sql"""set experimental_enable_pipeline_x_engine=false; """
sql"""set experimental_enable_pipeline_x_engine=true; """
qt_pipelineX """
SELECT k1, k2

View File

@ -80,7 +80,7 @@ suite("test_union_operator") {
"""
sql"""set experimental_enable_pipeline_x_engine=false,parallel_pipeline_task_num = 8;; """
sql"""set experimental_enable_pipeline_x_engine=true,parallel_pipeline_task_num = 8;; """
qt_pipelineX """
SELECT count(*)

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue
FROM lineorder_flat
WHERE
LO_ORDERDATE >= 19930101

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q1.2
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue
FROM lineorder_flat
WHERE
LO_ORDERDATE >= 19940101

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q1.3
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue
FROM lineorder_flat
WHERE
weekofyear(LO_ORDERDATE) = 6

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q2.1
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR,
P_BRAND
FROM lineorder_flat

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q2.2
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR,
P_BRAND
FROM lineorder_flat

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q2.3
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR,
P_BRAND
FROM lineorder_flat

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q3.1
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
C_NATION,
S_NATION, (LO_ORDERDATE DIV 10000) AS YEAR,
SUM(LO_REVENUE) AS revenue

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q3.2
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
C_CITY,
S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR,
SUM(LO_REVENUE) AS revenue

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q3.3
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
C_CITY,
S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR,
SUM(LO_REVENUE) AS revenue

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q3.4
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
C_CITY,
S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR,
SUM(LO_REVENUE) AS revenue

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q4.1
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ (LO_ORDERDATE DIV 10000) AS YEAR,
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR,
C_NATION,
SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit
FROM lineorder_flat

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q4.2
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ (LO_ORDERDATE DIV 10000) AS YEAR,
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR,
S_NATION,
P_CATEGORY,
SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
--Q4.3
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ (LO_ORDERDATE DIV 10000) AS YEAR,
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR,
S_CITY,
P_BRAND,
SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit

View File

@ -15,7 +15,7 @@
-- specific language governing permissions and limitations
-- under the License.
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ 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=false) */ 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=false) */ 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=false) */ 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=false) */ 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=false) */ 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=false) */ 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=false) */ 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=false) */ 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=false) */ 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=false) */ 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=false) */ 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=false) */ 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=false) */
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=false) */
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=false) */
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=false) */
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=false) */
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=false) */ 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=false) */
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=false) */
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=false) */
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=false) */
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=false) */
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=false) */
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=false) */
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=false) */ 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=false) */
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=false) */
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=false) */ 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=false) */
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=false) */ 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=false) */
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=false) */
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=false) */
SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */
cntrycode,
count(*) AS numcust,
sum(c_acctbal) AS totacctbal