[pipelineX](fix) Fix broadcast dependency hanging (#24740)
This commit is contained in:
@ -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;
|
||||
|
||||
@ -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]);
|
||||
|
||||
@ -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();
|
||||
|
||||
@ -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());
|
||||
|
||||
@ -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;
|
||||
}
|
||||
|
||||
|
||||
@ -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;
|
||||
}
|
||||
|
||||
|
||||
@ -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;
|
||||
}
|
||||
|
||||
|
||||
@ -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;
|
||||
}
|
||||
|
||||
|
||||
@ -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);
|
||||
}
|
||||
|
||||
@ -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();
|
||||
}
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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(*)
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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'
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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'
|
||||
|
||||
@ -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 (
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
Reference in New Issue
Block a user