diff --git a/be/src/benchmarks/row-batch-serialize-benchmark.cc b/be/src/benchmarks/row-batch-serialize-benchmark.cc index b55d04b4a..40421ce7f 100644 --- a/be/src/benchmarks/row-batch-serialize-benchmark.cc +++ b/be/src/benchmarks/row-batch-serialize-benchmark.cc @@ -22,7 +22,6 @@ #include "common/init.h" #include "runtime/mem-tracker.h" #include "runtime/raw-value.h" -#include "runtime/row-batch.h" #include "runtime/string-value.h" #include "runtime/tuple-row.h" #include "service/fe-support.h" @@ -46,47 +45,33 @@ // fixed-length slot handling. The small tuples with few slots emphasizes per-tuple // dedup performance rather than per-slot serialization/deserialization performance. // -// serialize: Function Rate (iters/ms) Comparison -// ---------------------------------------------------------------------- -// ser_no_dups_baseline 17.43 1X -// ser_no_dups 17.33 0.9944X -// ser_no_dups_full 14.1 0.8092X +// Machine Info: Intel(R) Xeon(R) Platinum 8375C CPU @ 2.90GHz +// serialize: 10%ile 50%ile 90%ile 10%ile 50%ile 90%ile +// (relative) (relative) (relative) +// ------------------------------------------------------------------------------- +// ser_no_dups_base 18.6 18.8 18.9 1X 1X 1X +// ser_no_dups 18.5 18.5 18.8 0.998X 0.988X 0.991X +// ser_no_dups_full 14.7 14.8 14.8 0.793X 0.79X 0.783X // -// ser_adjacent_dups_baseline 26.65 1X -// ser_adjacent_dups 63.98 2.4X -// ser_adjacent_dups_full 55.88 2.096X +// ser_adj_dups_base 28.2 28.4 28.8 1X 1X 1X +// ser_adj_dups 68.9 69.1 69.8 2.44X 2.43X 2.43X +// ser_adj_dups_full 56.2 56.7 57.1 1.99X 2X 1.99X // -// ser_dups_baseline 19.26 1X -// ser_dups 19.55 1.015X -// ser_dups_full 32.4 1.682X +// ser_dups_base 20.7 20.9 20.9 1X 1X 1X +// ser_dups 20.6 20.8 20.9 0.994X 0.995X 1X +// ser_dups_full 39.8 40 40.5 1.93X 1.92X 1.94X // -// deserialize: Function Rate (iters/ms) Comparison -// ---------------------------------------------------------------------- -// deser_no_dups_baseline 64.94 1X -// deser_no_dups 69.24 1.066X +// deserialize: 10%ile 50%ile 90%ile 10%ile 50%ile 90%ile +// (relative) (relative) (relative) +// ------------------------------------------------------------------------------- +// deser_no_dups_base 75.9 76.6 77 1X 1X 1X +// deser_no_dups 74.9 75.6 76 0.987X 0.987X 0.987X // -// deser_adjacent_dups_baseline 112 1X -// deser_adjacent_dups 207.4 1.852X -// -// deser_dups_baseline 114.8 1X -// deser_dups 208.5 1.817X -// -// Earlier results with LossyHashTable -// serialize: Function Rate (iters/ms) Comparison -// ---------------------------------------------------------------------- -// ser_no_dups_lossy 15.93 0.9139X -// ser_adjacent_dups_lossy 58.21 2.184X -// ser_dups_lossy 50.46 2.62X -// -// Earlier results with boost::unordered_map -// serialize: Function Rate (iters/ms) Comparison -// ---------------------------------------------------------------------- -// ser_no_dups_full 8.73 0.5582X -// -// ser_adjacent_dups_full 38.7 1.634X -// -// ser_dups_full 27.5 1.54X +// deser_adj_dups_base 127 128 129 1X 1X 1X +// deser_adj_dups 179 193 195 1.41X 1.51X 1.51X // +// deser_dups_base 128 128 129 1X 1X 1X +// deser_dups 165 190 193 1.29X 1.48X 1.49X using namespace impala; @@ -97,21 +82,25 @@ namespace impala { // For computing tuple mem layouts. static scoped_ptr fe; +static std::shared_ptr char_mem_tracker_allocator; // Friend class with access to RowBatch internals class RowBatchSerializeBaseline { public: - // Copy of baseline version without dedup logic - static int Serialize(RowBatch* batch, TRowBatch* output_batch) { - output_batch->row_tuples.clear(); - output_batch->tuple_offsets.clear(); - output_batch->compression_type = THdfsCompression::NONE; - - output_batch->num_rows = batch->num_rows_; - batch->row_desc_->ToThrift(&output_batch->row_tuples); - output_batch->tuple_offsets.reserve(batch->num_rows_ * batch->num_tuples_per_row_); + // Copy of baseline version without dedup logic + static int Serialize(RowBatch* batch, OutboundRowBatch* output_batch) { + RowBatchHeaderPB* header = &output_batch->header_; + output_batch->tuple_offsets_.clear(); + header->set_num_rows(batch->num_rows_); + header->set_compression_type(CompressionTypePB::NONE); + output_batch->tuple_offsets_.reserve(batch->num_rows_ * batch->num_tuples_per_row_); int64_t size = TotalByteSize(batch); + + header->set_uncompressed_size(size); + + output_batch->tuple_data_.resize(size); + SerializeInternal(batch, size, output_batch); if (size > 0) { @@ -125,37 +114,38 @@ class RowBatchSerializeBaseline { int64_t compressed_size = compressor.MaxOutputLen(size); DCHECK_GT(compressed_size, 0); - if (batch->compression_scratch_.size() < compressed_size) { - batch->compression_scratch_.resize(compressed_size); + if (output_batch->compression_scratch_.size() < compressed_size) { + output_batch->compression_scratch_.resize(compressed_size); } + uint8_t* input = const_cast( - reinterpret_cast(output_batch->tuple_data.c_str())); + reinterpret_cast(output_batch->tuple_data_.data())); uint8_t* compressed_output = const_cast( - reinterpret_cast(batch->compression_scratch_.c_str())); - status = - compressor.ProcessBlock(true, size, input, &compressed_size, &compressed_output); + reinterpret_cast(output_batch->compression_scratch_.data())); + status = compressor.ProcessBlock( + true, size, input, &compressed_size, &compressed_output); DCHECK(status.ok()) << status.GetDetail(); if (LIKELY(compressed_size < size)) { - batch->compression_scratch_.resize(compressed_size); - output_batch->tuple_data.swap(batch->compression_scratch_); - output_batch->compression_type = THdfsCompression::LZ4; + output_batch->compression_scratch_.resize(compressed_size); + output_batch->tuple_data_.swap(output_batch->compression_scratch_); } - VLOG_ROW << "uncompressed size: " << size << ", compressed size: " << compressed_size; + VLOG_ROW << "uncompressed size: " << size + << ", compressed size: " << compressed_size; } return RowBatch::GetDeserializedSize(*output_batch); } // Copy of baseline version without dedup logic - static void SerializeInternal(RowBatch* batch, int64_t size, TRowBatch* output_batch) { - DCHECK_LE(size, output_batch->tuple_data.max_size()); - output_batch->tuple_data.resize(size); - output_batch->uncompressed_size = size; + static void SerializeInternal( + RowBatch* batch, int64_t size, OutboundRowBatch* output_batch) { + DCHECK_LE(size, output_batch->tuple_data_.max_size()); + output_batch->tuple_data_.resize(size); // Copy tuple data of unique tuples, including strings, into output_batch (converting // string pointers into offsets in the process). int offset = 0; // current offset into output_batch->tuple_data - char* tuple_data = const_cast(output_batch->tuple_data.c_str()); + char* tuple_data = const_cast(output_batch->tuple_data_.data()); for (int i = 0; i < batch->num_rows_; ++i) { vector::const_iterator desc = @@ -164,11 +154,11 @@ class RowBatchSerializeBaseline { Tuple* tuple = batch->GetRow(i)->GetTuple(j); if (tuple == NULL) { // NULLs are encoded as -1 - output_batch->tuple_offsets.push_back(-1); + output_batch->tuple_offsets_.push_back(-1); continue; } // Record offset before creating copy (which increments offset and tuple_data) - output_batch->tuple_offsets.push_back(offset); + output_batch->tuple_offsets_.push_back(offset); tuple->DeepCopy(**desc, &tuple_data, &offset, /* convert_ptrs */ true); DCHECK_LE(offset, size); } @@ -190,15 +180,15 @@ class RowBatchSerializeBaseline { } // Copy of baseline version without dedup logic - static void Deserialize(RowBatch* batch, const TRowBatch& input_batch) { - batch->num_rows_ = input_batch.num_rows; - batch->capacity_ = batch->num_rows_; + static void Deserialize(RowBatch* batch, const OutboundRowBatch& input_batch) { + batch->num_rows_ = input_batch.header()->num_rows(); + batch->capacity_ = batch->num_rows_; uint8_t* tuple_data; - if (input_batch.compression_type != THdfsCompression::NONE) { + if (input_batch.header()->compression_type() != CompressionTypePB::NONE) { // Decompress tuple data into data pool uint8_t* compressed_data = const_cast( - reinterpret_cast(input_batch.tuple_data.c_str())); - size_t compressed_size = input_batch.tuple_data.size(); + reinterpret_cast(input_batch.tuple_data_.data())); + size_t compressed_size = input_batch.tuple_data_.size(); Lz4Decompressor decompressor(nullptr, false); Status status = decompressor.Init(); @@ -206,7 +196,7 @@ class RowBatchSerializeBaseline { auto compressor_cleanup = MakeScopeExitTrigger([&decompressor]() { decompressor.Close(); }); - int64_t uncompressed_size = input_batch.uncompressed_size; + int64_t uncompressed_size = input_batch.header()->uncompressed_size(); DCHECK_NE(uncompressed_size, -1) << "RowBatch decompression failed"; tuple_data = batch->tuple_data_pool()->Allocate(uncompressed_size); status = decompressor.ProcessBlock( @@ -214,14 +204,14 @@ class RowBatchSerializeBaseline { DCHECK(status.ok()) << "RowBatch decompression failed."; } else { // Tuple data uncompressed, copy directly into data pool - tuple_data = batch->tuple_data_pool()->Allocate(input_batch.tuple_data.size()); - memcpy(tuple_data, input_batch.tuple_data.c_str(), input_batch.tuple_data.size()); + tuple_data = batch->tuple_data_pool()->Allocate(input_batch.tuple_data_.size()); + memcpy(tuple_data, input_batch.tuple_data_.data(), input_batch.tuple_data_.size()); } // Convert input_batch.tuple_offsets into pointers int tuple_idx = 0; - for (vector::const_iterator offset = input_batch.tuple_offsets.begin(); - offset != input_batch.tuple_offsets.end(); ++offset) {/// + for (vector::const_iterator offset = input_batch.tuple_offsets_.begin(); + offset != input_batch.tuple_offsets_.end(); ++offset) { /// if (*offset == -1) { batch->tuple_ptrs_[tuple_idx++] = NULL; } else { @@ -292,21 +282,21 @@ class RowBatchSerializeBenchmark { static void TestSerialize(int batch_size, void* data) { SerializeArgs* args = reinterpret_cast(data); for (int iter = 0; iter < batch_size; ++iter) { - TRowBatch trow_batch; - ABORT_IF_ERROR(args->batch->Serialize(&trow_batch, args->full_dedup)); + OutboundRowBatch row_batch(char_mem_tracker_allocator); + ABORT_IF_ERROR(args->batch->Serialize(&row_batch, args->full_dedup)); } } static void TestSerializeBaseline(int batch_size, void* data) { RowBatch* batch = reinterpret_cast(data); for (int iter = 0; iter < batch_size; ++iter) { - TRowBatch trow_batch; - RowBatchSerializeBaseline::Serialize(batch, &trow_batch); + OutboundRowBatch row_batch(char_mem_tracker_allocator); + RowBatchSerializeBaseline::Serialize(batch, &row_batch); } } struct DeserializeArgs { - TRowBatch* trow_batch; + OutboundRowBatch* row_batch; RowDescriptor* row_desc; MemTracker* tracker; }; @@ -314,7 +304,7 @@ class RowBatchSerializeBenchmark { static void TestDeserialize(int batch_size, void* data) { struct DeserializeArgs* args = reinterpret_cast(data); for (int iter = 0; iter < batch_size; ++iter) { - RowBatch deserialized_batch(args->row_desc, *args->trow_batch, args->tracker); + RowBatch deserialized_batch(args->row_desc, *args->row_batch, args->tracker); } } @@ -322,95 +312,102 @@ class RowBatchSerializeBenchmark { struct DeserializeArgs* args = reinterpret_cast(data); for (int iter = 0; iter < batch_size; ++iter) { RowBatch deserialized_batch( - args->row_desc, args->trow_batch->num_rows, args->tracker); - RowBatchSerializeBaseline::Deserialize(&deserialized_batch, *args->trow_batch); + args->row_desc, args->row_batch->header()->num_rows(), args->tracker); + RowBatchSerializeBaseline::Deserialize(&deserialized_batch, *args->row_batch); } } static void Run() { - MemTracker tracker; - MemPool mem_pool(&tracker); + cout << Benchmark::GetMachineInfo() << endl; + + char_mem_tracker_allocator.reset( + new CharMemTrackerAllocator(std::make_shared())); + shared_ptr tracker = char_mem_tracker_allocator->mem_tracker(); + MemPool mem_pool(tracker.get()); ObjectPool obj_pool; DescriptorTblBuilder builder(fe.get(), &obj_pool); builder.DeclareTuple() << TYPE_INT << TYPE_STRING; DescriptorTbl* desc_tbl = builder.Build(); vector nullable_tuples(1, false); - vector tuple_id(1, (TTupleId) 0); + vector tuple_id(1, (TTupleId)0); RowDescriptor row_desc(*desc_tbl, tuple_id, nullable_tuples); - RowBatch* no_dup_batch = obj_pool.Add(new RowBatch(&row_desc, NUM_ROWS, &tracker)); + RowBatch* no_dup_batch = + obj_pool.Add(new RowBatch(&row_desc, NUM_ROWS, tracker.get())); FillBatch(no_dup_batch, 12345, 1, -1); - TRowBatch no_dup_tbatch; - ABORT_IF_ERROR(no_dup_batch->Serialize(&no_dup_tbatch)); + OutboundRowBatch no_dup_row_batch(char_mem_tracker_allocator); + ABORT_IF_ERROR(no_dup_batch->Serialize(&no_dup_row_batch)); RowBatch* adjacent_dup_batch = - obj_pool.Add(new RowBatch(&row_desc, NUM_ROWS, &tracker)); + obj_pool.Add(new RowBatch(&row_desc, NUM_ROWS, tracker.get())); FillBatch(adjacent_dup_batch, 12345, 5, -1); - TRowBatch adjacent_dup_tbatch; - ABORT_IF_ERROR(adjacent_dup_batch->Serialize(&adjacent_dup_tbatch, false)); + OutboundRowBatch adjacent_dup_row_batch(char_mem_tracker_allocator); + ABORT_IF_ERROR(adjacent_dup_batch->Serialize(&adjacent_dup_row_batch, false)); - RowBatch* dup_batch = obj_pool.Add(new RowBatch(&row_desc, NUM_ROWS, &tracker)); + RowBatch* dup_batch = obj_pool.Add(new RowBatch(&row_desc, NUM_ROWS, tracker.get())); // Non-adjacent duplicates. FillBatch(dup_batch, 12345, 1, NUM_ROWS / 5); - TRowBatch dup_tbatch; - ABORT_IF_ERROR(dup_batch->Serialize(&dup_tbatch, true)); + OutboundRowBatch dup_row_batch(char_mem_tracker_allocator); + ABORT_IF_ERROR(dup_batch->Serialize(&dup_row_batch, true)); int baseline; Benchmark ser_suite("serialize"); - baseline = ser_suite.AddBenchmark("ser_no_dups_baseline", TestSerializeBaseline, - no_dup_batch, -1); - struct SerializeArgs no_dup_ser_args = { no_dup_batch, false }; - struct SerializeArgs no_dup_ser_full_args = { no_dup_batch, true }; + baseline = ser_suite.AddBenchmark( + "ser_no_dups_base", TestSerializeBaseline, no_dup_batch, -1); + struct SerializeArgs no_dup_ser_args = {no_dup_batch, false}; + struct SerializeArgs no_dup_ser_full_args = {no_dup_batch, true}; ser_suite.AddBenchmark("ser_no_dups", TestSerialize, &no_dup_ser_args, baseline); - ser_suite.AddBenchmark("ser_no_dups_full", - TestSerialize, &no_dup_ser_full_args, baseline); + ser_suite.AddBenchmark( + "ser_no_dups_full", TestSerialize, &no_dup_ser_full_args, baseline); - baseline = ser_suite.AddBenchmark("ser_adjacent_dups_baseline", - TestSerializeBaseline, adjacent_dup_batch, -1); - struct SerializeArgs adjacent_dup_ser_args = { adjacent_dup_batch, false }; - struct SerializeArgs adjacent_dup_ser_full_args = { adjacent_dup_batch, true }; - ser_suite.AddBenchmark("ser_adjacent_dups", - TestSerialize, &adjacent_dup_ser_args, baseline); - ser_suite.AddBenchmark("ser_adjacent_dups_full", - TestSerialize, &adjacent_dup_ser_full_args, baseline); + baseline = ser_suite.AddBenchmark( + "ser_adj_dups_base", TestSerializeBaseline, adjacent_dup_batch, -1); + struct SerializeArgs adjacent_dup_ser_args = {adjacent_dup_batch, false}; + struct SerializeArgs adjacent_dup_ser_full_args = {adjacent_dup_batch, true}; + ser_suite.AddBenchmark( + "ser_adj_dups", TestSerialize, &adjacent_dup_ser_args, baseline); + ser_suite.AddBenchmark( + "ser_adj_dups_full", TestSerialize, &adjacent_dup_ser_full_args, baseline); - baseline = ser_suite.AddBenchmark("ser_dups_baseline", - TestSerializeBaseline, dup_batch, -1); - struct SerializeArgs dup_ser_args = { dup_batch, false }; - struct SerializeArgs dup_ser_full_args = { dup_batch, true }; + baseline = + ser_suite.AddBenchmark("ser_dups_base", TestSerializeBaseline, dup_batch, -1); + struct SerializeArgs dup_ser_args = {dup_batch, false}; + struct SerializeArgs dup_ser_full_args = {dup_batch, true}; ser_suite.AddBenchmark("ser_dups", TestSerialize, &dup_ser_args, baseline); ser_suite.AddBenchmark("ser_dups_full", TestSerialize, &dup_ser_full_args, baseline); cout << ser_suite.Measure() << endl; Benchmark deser_suite("deserialize"); - struct DeserializeArgs no_dup_deser_args = { &no_dup_tbatch, &row_desc, &tracker }; - baseline = deser_suite.AddBenchmark("deser_no_dups_baseline", - TestDeserializeBaseline, &no_dup_deser_args, -1); - deser_suite.AddBenchmark("deser_no_dups", - TestDeserialize, &no_dup_deser_args, baseline); + struct DeserializeArgs no_dup_deser_args = { + &no_dup_row_batch, &row_desc, tracker.get()}; + baseline = deser_suite.AddBenchmark( + "deser_no_dups_base", TestDeserializeBaseline, &no_dup_deser_args, -1); + deser_suite.AddBenchmark( + "deser_no_dups", TestDeserialize, &no_dup_deser_args, baseline); - struct DeserializeArgs adjacent_dup_deser_args = { &adjacent_dup_tbatch, &row_desc, - &tracker }; - baseline = deser_suite.AddBenchmark("deser_adjacent_dups_baseline", - TestDeserializeBaseline, &adjacent_dup_deser_args, -1); - deser_suite.AddBenchmark("deser_adjacent_dups", - TestDeserialize, &adjacent_dup_deser_args, baseline); + struct DeserializeArgs adjacent_dup_deser_args = { + &adjacent_dup_row_batch, &row_desc, tracker.get()}; + baseline = deser_suite.AddBenchmark( + "deser_adj_dups_base", TestDeserializeBaseline, &adjacent_dup_deser_args, -1); + deser_suite.AddBenchmark( + "deser_adj_dups", TestDeserialize, &adjacent_dup_deser_args, baseline); - struct DeserializeArgs dup_deser_args = { &dup_tbatch, &row_desc, &tracker }; - baseline = deser_suite.AddBenchmark("deser_dups_baseline", - TestDeserializeBaseline, &dup_deser_args, -1); + struct DeserializeArgs dup_deser_args = {&dup_row_batch, &row_desc, tracker.get()}; + baseline = deser_suite.AddBenchmark( + "deser_dups_base", TestDeserializeBaseline, &dup_deser_args, -1); deser_suite.AddBenchmark("deser_dups", TestDeserialize, &dup_deser_args, baseline); cout << deser_suite.Measure() << endl; + mem_pool.FreeAll(); } }; -} +} // namespace impala int main(int argc, char** argv) { - impala::InitCommonRuntime(argc, argv, true); + impala::InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST); InitFeSupport(); fe.reset(new Frontend()); RowBatchSerializeBenchmark::Run(); diff --git a/be/src/runtime/krpc-data-stream-sender.cc b/be/src/runtime/krpc-data-stream-sender.cc index 6b724abf2..d17c8e46d 100644 --- a/be/src/runtime/krpc-data-stream-sender.cc +++ b/be/src/runtime/krpc-data-stream-sender.cc @@ -161,7 +161,7 @@ class KrpcDataStreamSender::Channel : public CacheLineAligned { // Initializes the channel. // Returns OK if successful, error indication otherwise. - Status Init(RuntimeState* state); + Status Init(RuntimeState* state, std::shared_ptr allocator); // Serializes the given row batch and send it to the destination. If the preceding // RPC is in progress, this function may block until the previous RPC finishes. @@ -233,7 +233,7 @@ class KrpcDataStreamSender::Channel : public CacheLineAligned { // TODO: rethink whether to keep per-channel buffers vs having all buffers in the // datastream sender and sharing them across all channels. These buffers are not used in // "UNPARTITIONED" scheme. - OutboundRowBatch outbound_batches_[NUM_OUTBOUND_BATCHES]; + std::vector outbound_batches_; // Index into 'outbound_batches_' for the next available OutboundRowBatch to serialize // into. This is read and written by the main execution thread. @@ -366,7 +366,8 @@ class KrpcDataStreamSender::Channel : public CacheLineAligned { const char* rpc_name, int64_t total_time_ns, const kudu::Status& err); }; -Status KrpcDataStreamSender::Channel::Init(RuntimeState* state) { +Status KrpcDataStreamSender::Channel::Init( + RuntimeState* state, std::shared_ptr allocator) { // TODO: take into account of var-len data at runtime. int capacity = max(1, parent_->per_channel_buffer_size_ / max(row_desc_->GetRowSize(), 1)); @@ -374,6 +375,11 @@ Status KrpcDataStreamSender::Channel::Init(RuntimeState* state) { // Create a DataStreamService proxy to the destination. RETURN_IF_ERROR(DataStreamService::GetProxy(address_, hostname_, &proxy_)); + + // Init outbound_batches_. + for (int i = 0; i < NUM_OUTBOUND_BATCHES; ++i) { + outbound_batches_.emplace_back(allocator); + } return Status::OK(); } @@ -709,6 +715,7 @@ void KrpcDataStreamSender::Channel::Teardown(RuntimeState* state) { while (rpc_in_flight_) rpc_done_cv_.wait(l); } batch_.reset(); + outbound_batches_.clear(); } KrpcDataStreamSender::KrpcDataStreamSender(TDataSinkId sink_id, int sender_id, @@ -755,9 +762,8 @@ Status KrpcDataStreamSender::Prepare( RETURN_IF_ERROR(DataSink::Prepare(state, parent_mem_tracker)); state_ = state; SCOPED_TIMER(profile_->total_time_counter()); - RETURN_IF_ERROR(ScalarExprEvaluator::Create(partition_exprs_, state, - state->obj_pool(), expr_perm_pool_.get(), expr_results_pool_.get(), - &partition_expr_evals_)); + RETURN_IF_ERROR(ScalarExprEvaluator::Create(partition_exprs_, state, state->obj_pool(), + expr_perm_pool_.get(), expr_results_pool_.get(), &partition_expr_evals_)); serialize_batch_timer_ = ADD_TIMER(profile(), "SerializeBatchTime"); rpc_retry_counter_ = ADD_COUNTER(profile(), "RpcRetry", TUnit::UNIT); rpc_failure_counter_ = ADD_COUNTER(profile(), "RpcFailure", TUnit::UNIT); @@ -774,9 +780,19 @@ Status KrpcDataStreamSender::Prepare( eos_sent_counter_ = ADD_COUNTER(profile(), "EosSent", TUnit::UNIT); uncompressed_bytes_counter_ = ADD_COUNTER(profile(), "UncompressedRowBatchSize", TUnit::BYTES); - total_sent_rows_counter_= ADD_COUNTER(profile(), "RowsSent", TUnit::UNIT); + total_sent_rows_counter_ = ADD_COUNTER(profile(), "RowsSent", TUnit::UNIT); + + outbound_rb_mem_tracker_.reset( + new MemTracker(-1, "RowBatchSerialization", mem_tracker_.get())); + char_mem_tracker_allocator_.reset( + new CharMemTrackerAllocator(outbound_rb_mem_tracker_)); + + for (int i = 0; i < NUM_OUTBOUND_BATCHES; ++i) { + outbound_batches_.emplace_back(char_mem_tracker_allocator_); + } + for (int i = 0; i < channels_.size(); ++i) { - RETURN_IF_ERROR(channels_[i]->Init(state)); + RETURN_IF_ERROR(channels_[i]->Init(state, char_mem_tracker_allocator_)); } return Status::OK(); } @@ -1079,6 +1095,12 @@ void KrpcDataStreamSender::Close(RuntimeState* state) { for (int i = 0; i < channels_.size(); ++i) { channels_[i]->Teardown(state); } + + outbound_batches_.clear(); + if (outbound_rb_mem_tracker_.get() != nullptr) { + outbound_rb_mem_tracker_->Close(); + } + ScalarExprEvaluator::Close(partition_expr_evals_, state); profile()->StopPeriodicCounters(); DataSink::Close(state); @@ -1101,4 +1123,3 @@ int64_t KrpcDataStreamSender::GetNumDataBytesSent() const { } } // namespace impala - diff --git a/be/src/runtime/krpc-data-stream-sender.h b/be/src/runtime/krpc-data-stream-sender.h index 7bc74cb48..99a7dca86 100644 --- a/be/src/runtime/krpc-data-stream-sender.h +++ b/be/src/runtime/krpc-data-stream-sender.h @@ -19,15 +19,16 @@ #ifndef IMPALA_RUNTIME_KRPC_DATA_STREAM_SENDER_H #define IMPALA_RUNTIME_KRPC_DATA_STREAM_SENDER_H -#include #include +#include -#include "exec/data-sink.h" #include "codegen/impala-ir.h" #include "common/global-types.h" #include "common/object-pool.h" #include "common/status.h" +#include "exec/data-sink.h" #include "exprs/scalar-expr.h" +#include "runtime/mem-tracker.h" #include "runtime/row-batch.h" #include "util/runtime-profile.h" @@ -206,7 +207,7 @@ class KrpcDataStreamSender : public DataSink { /// a RowBatchHeaderPB and buffers for the serialized tuple offsets and data. Used only /// when the partitioning strategy is UNPARTITIONED. static const int NUM_OUTBOUND_BATCHES = 2; - OutboundRowBatch outbound_batches_[NUM_OUTBOUND_BATCHES]; + std::vector outbound_batches_; /// If true, this sender has called FlushFinal() successfully. /// Not valid to call Send() anymore. @@ -261,6 +262,11 @@ class KrpcDataStreamSender : public DataSink { /// Identifier of the destination plan node. PlanNodeId dest_node_id_; + /// Memory tracker from Parent Memory Tracker for tracking memory of OutBoundRowBatch + /// serialization + std::shared_ptr outbound_rb_mem_tracker_; + std::shared_ptr char_mem_tracker_allocator_; + /// Used for Kudu partitioning to round-robin rows that don't correspond to a partition /// or when errors are encountered. int next_unknown_partition_; diff --git a/be/src/runtime/mem-tracker.h b/be/src/runtime/mem-tracker.h index ea2fecb35..1d20428cd 100644 --- a/be/src/runtime/mem-tracker.h +++ b/be/src/runtime/mem-tracker.h @@ -570,4 +570,53 @@ class PoolMemTrackerRegistry { /// automatically destroy itself as part of process teardown, which could cause races. SpinLock pool_to_mem_trackers_lock_; }; + +// An std::allocator that manipulates a MemTracker during allocation +// and deallocation. +// This is copied from src/kudu/util/mem_tracker.h +template > +class MemTrackerAllocator : public Alloc { + public: + typedef typename Alloc::pointer pointer; + typedef typename Alloc::const_pointer const_pointer; + typedef typename Alloc::size_type size_type; + + explicit MemTrackerAllocator(std::shared_ptr mem_tracker) + : mem_tracker_(std::move(mem_tracker)) {} + + // This constructor is used for rebinding. + template + MemTrackerAllocator(const MemTrackerAllocator& allocator) + : Alloc(allocator), mem_tracker_(allocator.mem_tracker()) {} + + ~MemTrackerAllocator() {} + + pointer allocate(size_type n, const_pointer hint = 0) { + // Ideally we'd use TryConsume() here to enforce the tracker's limit. + // However, that means throwing bad_alloc if the limit is exceeded, and + // it's not clear that the rest of Kudu can handle that. + mem_tracker_->Consume(n * sizeof(T)); + return Alloc::allocate(n, hint); + } + + void deallocate(pointer p, size_type n) { + Alloc::deallocate(p, n); + mem_tracker_->Release(n * sizeof(T)); + } + + // This allows an allocator to be used for a different type. + template + struct rebind { + typedef MemTrackerAllocator::other> other; + }; + + const std::shared_ptr& mem_tracker() const { return mem_tracker_; } + + private: + std::shared_ptr mem_tracker_; +}; + +typedef MemTrackerAllocator CharMemTrackerAllocator; +typedef std::basic_string, CharMemTrackerAllocator> + TrackedString; } diff --git a/be/src/runtime/row-batch-serialize-test.cc b/be/src/runtime/row-batch-serialize-test.cc index 83836f8db..bfd88c3cf 100644 --- a/be/src/runtime/row-batch-serialize-test.cc +++ b/be/src/runtime/row-batch-serialize-test.cc @@ -46,7 +46,8 @@ const int NULL_VALUE_PERCENT = 10; class RowBatchSerializeTest : public testing::Test { protected: ObjectPool pool_; - scoped_ptr tracker_; + std::shared_ptr tracker_; + std::shared_ptr char_mem_tracker_allocator_; scoped_ptr test_env_; RuntimeState* runtime_state_ = nullptr; @@ -57,11 +58,13 @@ class RowBatchSerializeTest : public testing::Test { test_env_.reset(new TestEnv); ASSERT_OK(test_env_->Init()); tracker_.reset(new MemTracker()); + char_mem_tracker_allocator_.reset(new CharMemTrackerAllocator(tracker_)); ASSERT_OK(test_env_->CreateQueryState(1234, &dummy_query_opts_, &runtime_state_)); } virtual void TearDown() { pool_.Clear(); + tracker_->Close(); tracker_.reset(); test_env_.reset(); runtime_state_ = nullptr; @@ -77,10 +80,10 @@ class RowBatchSerializeTest : public testing::Test { bool print_batches, bool full_dedup = false) { if (print_batches) cout << PrintBatch(batch) << endl; - TRowBatch trow_batch; - RETURN_IF_ERROR(batch->Serialize(&trow_batch, full_dedup)); + OutboundRowBatch row_batch(char_mem_tracker_allocator_); + RETURN_IF_ERROR(batch->Serialize(&row_batch, full_dedup)); - RowBatch deserialized_batch(&row_desc, trow_batch, tracker_.get()); + RowBatch deserialized_batch(&row_desc, row_batch, tracker_.get()); if (print_batches) cout << PrintBatch(&deserialized_batch) << endl; EXPECT_EQ(batch->num_rows(), deserialized_batch.num_rows()); @@ -88,7 +91,8 @@ class RowBatchSerializeTest : public testing::Test { TupleRow* row = batch->GetRow(row_idx); TupleRow* deserialized_row = deserialized_batch.GetRow(row_idx); - for (int tuple_idx = 0; tuple_idx < row_desc.tuple_descriptors().size(); ++tuple_idx) { + for (int tuple_idx = 0; tuple_idx < row_desc.tuple_descriptors().size(); + ++tuple_idx) { TupleDescriptor* tuple_desc = row_desc.tuple_descriptors()[tuple_idx]; Tuple* tuple = row->GetTuple(tuple_idx); Tuple* deserialized_tuple = deserialized_row->GetTuple(tuple_idx); @@ -608,14 +612,14 @@ void RowBatchSerializeTest::TestDupRemoval(bool full_dedup) { vector tuples; CreateTuples(tuple_desc, batch->tuple_data_pool(), num_distinct_tuples, 0, 10, &tuples); AddTuplesToRowBatch(num_rows, tuples, repeats, batch); - TRowBatch trow_batch; - EXPECT_OK(batch->Serialize(&trow_batch, full_dedup)); + OutboundRowBatch row_batch(char_mem_tracker_allocator_); + EXPECT_OK(batch->Serialize(&row_batch, full_dedup)); // Serialized data should only have one copy of each tuple. int64_t total_byte_size = 0; // Total size without duplication for (int i = 0; i < tuples.size(); ++i) { total_byte_size += tuples[i]->TotalByteSize(tuple_desc); } - EXPECT_EQ(total_byte_size, trow_batch.uncompressed_size); + EXPECT_EQ(total_byte_size, row_batch.header()->uncompressed_size()); TestRowBatch(row_desc, batch, false, full_dedup); } @@ -746,15 +750,16 @@ TEST_F(RowBatchSerializeTest, DedupPathologicalFull) { // Full dedup should be automatically enabled because of row batch structure. EXPECT_TRUE(UseFullDedup(batch)); LOG(INFO) << "Serializing row batch"; - TRowBatch trow_batch; - EXPECT_OK(batch->Serialize(&trow_batch)); - LOG(INFO) << "Serialized batch size: " << trow_batch.tuple_data.size(); - LOG(INFO) << "Serialized batch uncompressed size: " << trow_batch.uncompressed_size; + OutboundRowBatch row_batch(char_mem_tracker_allocator_); + EXPECT_OK(batch->Serialize(&row_batch)); + LOG(INFO) << "Serialized batch size: " << row_batch.TupleDataAsSlice().size(); + LOG(INFO) << "Serialized batch uncompressed size: " + << row_batch.header()->uncompressed_size(); LOG(INFO) << "Serialized batch expected size: " << total_byte_size; // Serialized data should only have one copy of each tuple. - EXPECT_EQ(total_byte_size, trow_batch.uncompressed_size); + EXPECT_EQ(total_byte_size, row_batch.header()->uncompressed_size()); LOG(INFO) << "Deserializing row batch"; - RowBatch deserialized_batch(&row_desc, trow_batch, tracker_.get()); + RowBatch deserialized_batch(&row_desc, row_batch, tracker_.get()); LOG(INFO) << "Verifying row batch"; // Need to do special verification: comparing all duplicate strings is too slow. EXPECT_EQ(batch->num_rows(), deserialized_batch.num_rows()); diff --git a/be/src/runtime/row-batch.cc b/be/src/runtime/row-batch.cc index aed2e1167..c7fc4c4f3 100644 --- a/be/src/runtime/row-batch.cc +++ b/be/src/runtime/row-batch.cc @@ -61,20 +61,15 @@ RowBatch::RowBatch(const RowDescriptor* row_desc, int capacity, MemTracker* mem_ DCHECK(tuple_ptrs_ != NULL); } -// TODO: we want our input_batch's tuple_data to come from our (not yet implemented) -// global runtime memory segment; how do we get thrift to allocate it from there? -// maybe change line (in Data_types.cc generated from Data.thrift) -// xfer += iprot->readString(this->tuple_data[_i9]); -// to allocated string data in special mempool -// (change via python script that runs over Data_types.cc) -RowBatch::RowBatch( - const RowDescriptor* row_desc, const TRowBatch& input_batch, MemTracker* mem_tracker) - : num_rows_(input_batch.num_rows), - capacity_(input_batch.num_rows), +RowBatch::RowBatch(const RowDescriptor* row_desc, const OutboundRowBatch& input_batch, + MemTracker* mem_tracker) + : num_rows_(input_batch.header()->num_rows()), + capacity_(input_batch.header()->num_rows()), flush_mode_(FlushMode::NO_FLUSH_RESOURCES), needs_deep_copy_(false), - num_tuples_per_row_(input_batch.row_tuples.size()), - tuple_ptrs_size_(capacity_ * num_tuples_per_row_ * sizeof(Tuple*)), + num_tuples_per_row_(input_batch.header()->num_tuples_per_row()), + tuple_ptrs_size_( + input_batch.header()->num_rows() * num_tuples_per_row_ * sizeof(Tuple*)), attached_buffer_bytes_(0), tuple_data_pool_(mem_tracker), row_desc_(row_desc), @@ -82,26 +77,21 @@ RowBatch::RowBatch( DCHECK(mem_tracker_ != nullptr); DCHECK_EQ(num_tuples_per_row_, row_desc_->tuple_descriptors().size()); DCHECK_GT(tuple_ptrs_size_, 0); - kudu::Slice input_tuple_data = - kudu::Slice(input_batch.tuple_data.c_str(), input_batch.tuple_data.size()); - kudu::Slice input_tuple_offsets = kudu::Slice( - reinterpret_cast(input_batch.tuple_offsets.data()), - input_batch.tuple_offsets.size() * sizeof(int32_t)); - const THdfsCompression::type& compression_type = input_batch.compression_type; - DCHECK(compression_type == THdfsCompression::NONE || - compression_type == THdfsCompression::LZ4) - << "Unexpected compression type: " << input_batch.compression_type; + const CompressionTypePB& compression_type = input_batch.header()->compression_type(); + DCHECK(compression_type == CompressionTypePB::NONE + || compression_type == CompressionTypePB::LZ4) + << "Unexpected compression type: " << input_batch.header()->compression_type(); mem_tracker_->Consume(tuple_ptrs_size_); tuple_ptrs_ = reinterpret_cast(malloc(tuple_ptrs_size_)); DCHECK(tuple_ptrs_ != nullptr) << "Failed to allocate tuple pointers"; - const uint64_t uncompressed_size = input_batch.uncompressed_size; + const uint64_t uncompressed_size = input_batch.header()->uncompressed_size(); uint8_t* tuple_data = tuple_data_pool_.Allocate(uncompressed_size); DCHECK(tuple_data != nullptr) << "Failed to allocate tuple data"; - Deserialize(input_tuple_offsets, input_tuple_data, uncompressed_size, - compression_type == THdfsCompression::LZ4, tuple_data); + Deserialize(input_batch.TupleOffsetsAsSlice(), input_batch.TupleDataAsSlice(), + uncompressed_size, compression_type == CompressionTypePB::LZ4, tuple_data); } RowBatch::RowBatch(const RowDescriptor* row_desc, const RowBatchHeaderPB& header, @@ -229,67 +219,58 @@ RowBatch::~RowBatch() { tuple_ptrs_ = nullptr; } -Status RowBatch::Serialize(TRowBatch* output_batch) { +Status RowBatch::Serialize(OutboundRowBatch* output_batch) { return Serialize(output_batch, UseFullDedup()); } -Status RowBatch::Serialize(TRowBatch* output_batch, bool full_dedup) { - // why does Thrift not generate a Clear() function? - output_batch->row_tuples.clear(); - output_batch->tuple_offsets.clear(); - int64_t uncompressed_size; - bool is_compressed; - RETURN_IF_ERROR(Serialize(full_dedup, &output_batch->tuple_offsets, - &output_batch->tuple_data, &uncompressed_size, &is_compressed)); - // TODO: max_size() is much larger than the amount of memory we could feasibly - // allocate. Need better way to detect problem. - DCHECK_LE(uncompressed_size, output_batch->tuple_data.max_size()); - output_batch->__set_num_rows(num_rows_); - output_batch->__set_uncompressed_size(uncompressed_size); - output_batch->__set_compression_type( - is_compressed ? THdfsCompression::LZ4 : THdfsCompression::NONE); - row_desc_->ToThrift(&output_batch->row_tuples); - return Status::OK(); -} - -Status RowBatch::Serialize(OutboundRowBatch* output_batch) { - int64_t uncompressed_size; +Status RowBatch::Serialize(OutboundRowBatch* output_batch, bool full_dedup) { bool is_compressed; output_batch->tuple_offsets_.clear(); - RETURN_IF_ERROR(Serialize(UseFullDedup(), &output_batch->tuple_offsets_, - &output_batch->tuple_data_, &uncompressed_size, &is_compressed)); + + DedupMap distinct_tuples; + int64_t size; + + // As part of the serialization process we deduplicate tuples to avoid serializing a + // Tuple multiple times for the RowBatch. By default we only detect duplicate tuples + // in adjacent rows only. If full deduplication is enabled, we will build a + // map to detect non-adjacent duplicates. Building this map comes with significant + // overhead, so is only worthwhile in the uncommon case of many non-adjacent duplicates. + if (full_dedup) { + RETURN_IF_ERROR(distinct_tuples.Init(num_rows_ * num_tuples_per_row_ * 2, 0)); + size = TotalByteSize(&distinct_tuples); + distinct_tuples.Clear(); // Reuse allocated hash table. + } else { + size = TotalByteSize(nullptr); + } + + // The maximum uncompressed RowBatch size that can be serialized is INT_MAX. This + // is because the tuple offsets are int32s and will overflow for a larger size. + if (size > numeric_limits::max()) { + return Status(TErrorCode::ROW_BATCH_TOO_LARGE, size, numeric_limits::max()); + } + output_batch->tuple_data_.resize(size); + + RETURN_IF_ERROR(Serialize( + full_dedup ? &distinct_tuples : nullptr, output_batch, &is_compressed, size)); // Initialize the RowBatchHeaderPB RowBatchHeaderPB* header = &output_batch->header_; header->Clear(); header->set_num_rows(num_rows_); header->set_num_tuples_per_row(row_desc_->tuple_descriptors().size()); - header->set_uncompressed_size(uncompressed_size); + header->set_uncompressed_size(size); header->set_compression_type( is_compressed ? CompressionTypePB::LZ4 : CompressionTypePB::NONE); return Status::OK(); } -Status RowBatch::Serialize(bool full_dedup, vector* tuple_offsets, - string* tuple_data, int64_t* uncompressed_size, bool* is_compressed) { - // As part of the serialization process we deduplicate tuples to avoid serializing a - // Tuple multiple times for the RowBatch. By default we only detect duplicate tuples - // in adjacent rows only. If full deduplication is enabled, we will build a - // map to detect non-adjacent duplicates. Building this map comes with significant - // overhead, so is only worthwhile in the uncommon case of many non-adjacent duplicates. - int64_t size; - if (full_dedup) { - // Maps from tuple to offset of its serialized data in output_batch->tuple_data. - DedupMap distinct_tuples; - RETURN_IF_ERROR(distinct_tuples.Init(num_rows_ * num_tuples_per_row_ * 2, 0)); - size = TotalByteSize(&distinct_tuples); - distinct_tuples.Clear(); // Reuse allocated hash table. - RETURN_IF_ERROR(SerializeInternal(size, &distinct_tuples, tuple_offsets, tuple_data)); - } else { - size = TotalByteSize(nullptr); - RETURN_IF_ERROR(SerializeInternal(size, nullptr, tuple_offsets, tuple_data)); - } - *uncompressed_size = size; +Status RowBatch::Serialize(DedupMap* distinct_tuples, OutboundRowBatch* output_batch, + bool* is_compressed, int64_t size) { + char* tuple_data = const_cast(output_batch->tuple_data_.data()); + std::vector* tuple_offsets = &output_batch->tuple_offsets_; + + RETURN_IF_ERROR(SerializeInternal(size, distinct_tuples, tuple_offsets, tuple_data)); + *is_compressed = false; if (size > 0) { @@ -306,18 +287,18 @@ Status RowBatch::Serialize(bool full_dedup, vector* tuple_offsets, return Status(TErrorCode::LZ4_COMPRESSION_INPUT_TOO_LARGE, size); } DCHECK_GT(compressed_size, 0); - if (compression_scratch_.size() < compressed_size) { - compression_scratch_.resize(compressed_size); + if (output_batch->compression_scratch_.size() < compressed_size) { + output_batch->compression_scratch_.resize(compressed_size); } - uint8_t* input = - const_cast(reinterpret_cast(tuple_data->c_str())); + + uint8_t* input = reinterpret_cast(tuple_data); uint8_t* compressed_output = const_cast( - reinterpret_cast(compression_scratch_.c_str())); + reinterpret_cast(output_batch->compression_scratch_.data())); RETURN_IF_ERROR( compressor.ProcessBlock(true, size, input, &compressed_size, &compressed_output)); if (LIKELY(compressed_size < size)) { - compression_scratch_.resize(compressed_size); - tuple_data->swap(compression_scratch_); + output_batch->compression_scratch_.resize(compressed_size); + output_batch->tuple_data_.swap(output_batch->compression_scratch_); *is_compressed = true; } VLOG_ROW << "uncompressed size: " << size << ", compressed size: " << compressed_size; @@ -340,24 +321,14 @@ bool RowBatch::UseFullDedup() { } Status RowBatch::SerializeInternal(int64_t size, DedupMap* distinct_tuples, - vector* tuple_offsets, string* tuple_data_str) { + vector* tuple_offsets, char* tuple_data) { DCHECK(distinct_tuples == nullptr || distinct_tuples->size() == 0); - // The maximum uncompressed RowBatch size that can be serialized is INT_MAX. This - // is because the tuple offsets are int32s and will overflow for a larger size. - if (size > numeric_limits::max()) { - return Status(TErrorCode::ROW_BATCH_TOO_LARGE, size, numeric_limits::max()); - } - - // TODO: track memory usage - // TODO: detect if serialized size is too large to allocate and return proper error. - tuple_data_str->resize(size); tuple_offsets->reserve(num_rows_ * num_tuples_per_row_); // Copy tuple data of unique tuples, including strings, into output_batch (converting // string pointers into offsets in the process). int offset = 0; // current offset into output_batch->tuple_data - char* tuple_data = const_cast(tuple_data_str->c_str()); for (int i = 0; i < num_rows_; ++i) { vector::const_iterator desc = @@ -460,17 +431,6 @@ void RowBatch::SetMemTracker(MemTracker* new_tracker) { mem_tracker_ = new_tracker; } -int64_t RowBatch::GetDeserializedSize(const TRowBatch& batch) { - return batch.uncompressed_size + batch.tuple_offsets.size() * sizeof(Tuple*); -} - -int64_t RowBatch::GetSerializedSize(const TRowBatch& batch) { - int64_t result = batch.tuple_data.size(); - result += batch.row_tuples.size() * sizeof(TTupleId); - result += batch.tuple_offsets.size() * sizeof(int32_t); - return result; -} - int64_t RowBatch::GetDeserializedSize(const RowBatchHeaderPB& header, const kudu::Slice& tuple_offsets) { DCHECK_EQ(tuple_offsets.size() % sizeof(int32_t), 0); @@ -580,4 +540,4 @@ void RowBatch::VLogRows(const string& context) { } } -} +} // namespace impala diff --git a/be/src/runtime/row-batch.h b/be/src/runtime/row-batch.h index 1687043e2..3620b36e5 100644 --- a/be/src/runtime/row-batch.h +++ b/be/src/runtime/row-batch.h @@ -20,6 +20,7 @@ #include #include + #include #include "codegen/impala-ir.h" @@ -30,6 +31,7 @@ #include "runtime/bufferpool/buffer-pool.h" #include "runtime/descriptors.h" #include "runtime/mem-pool.h" +#include "runtime/mem-tracker.h" namespace kudu { class Slice; @@ -41,7 +43,6 @@ template class FixedSizeHashTable; class MemTracker; class RowBatchSerializeTest; class RuntimeState; -class TRowBatch; class Tuple; class TupleRow; class TupleDescriptor; @@ -50,6 +51,9 @@ class TupleDescriptor; /// for holding the tuple offsets and tuple data. class OutboundRowBatch { public: + OutboundRowBatch(std::shared_ptr allocator) + : tuple_data_(*allocator.get()), compression_scratch_(*allocator.get()) {} + const RowBatchHeaderPB* header() const { return &header_; } /// Returns the serialized tuple offsets' vector as a kudu::Slice. @@ -68,7 +72,7 @@ class OutboundRowBatch { tuple_data_.length()); } - /// Returns true if the header has been intialized and ready to be sent. + /// Returns true if the header has been initialized and ready to be sent. /// This entails setting some fields initialized in RowBatch::Serialize(). bool IsInitialized() const { return header_.has_num_rows() && header_.has_uncompressed_size() && @@ -77,6 +81,7 @@ class OutboundRowBatch { private: friend class RowBatch; + friend class RowBatchSerializeBaseline; /// The serialized header which contains the meta-data of the row batch such as the /// number of rows and compression scheme used etc. @@ -87,7 +92,12 @@ class OutboundRowBatch { vector tuple_offsets_; /// Contains the actual data of all the tuples. The data could be compressed. - std::string tuple_data_; + TrackedString tuple_data_; + + /// Contains the compression scratch for the compressed data in serialization. + /// The compression_scratch_ will be swapped with tuple_data_ if the compressed data + /// is shorter. + TrackedString compression_scratch_; }; /// A RowBatch encapsulates a batch of rows, each composed of a number of tuples. @@ -104,7 +114,7 @@ class OutboundRowBatch { /// used. /// TODO: IMPALA-4179: simplify the ownership transfer model. /// -/// In order to minimize memory allocations, RowBatches and TRowBatches or +/// In order to minimize memory allocations, RowBatches or /// OutboundRowBatch that have been serialized and sent over the wire should be reused /// (this prevents compression_scratch_ from being needlessly reallocated). /// @@ -137,13 +147,11 @@ class RowBatch { /// tracker cannot be NULL. RowBatch(const RowDescriptor* row_desc, int capacity, MemTracker* tracker); - /// Populate a row batch from a serialized thrift input_batch by copying - /// input_batch's tuple_data into the row batch's mempool and converting all - /// offsets in the data back into pointers. - /// TODO: figure out how to transfer the data from input_batch to this RowBatch - /// (so that we don't need to make yet another copy) - RowBatch(const RowDescriptor* row_desc, const TRowBatch& input_batch, - MemTracker* tracker); + /// Construct a row batch from a serialized protobuf input_batch and protobuf header by + /// copying input_batch's tuple_data_ into the row batch's mempool and converting all + /// offsets in the data back into pointers + RowBatch(const RowDescriptor* row_desc, const OutboundRowBatch& input_batch, + MemTracker* mem_tracker); /// Creates a row batch from the protobuf row batch header, decompress / copy /// 'input_tuple_data' into a buffer and convert all offsets in 'input_tuple_offsets' @@ -376,13 +384,10 @@ class RowBatch { /// whether tuple_data is compressed. If an in-flight row is present in this row batch, /// it is ignored. This function does not Reset(). Status Serialize(OutboundRowBatch* output_batch); - Status Serialize(TRowBatch* output_batch); /// Utility function: returns total byte size of a batch in either serialized or /// deserialized form. If a row batch is compressed, its serialized size can be much /// less than the deserialized size. - static int64_t GetSerializedSize(const TRowBatch& batch); - static int64_t GetDeserializedSize(const TRowBatch& batch); static int64_t GetSerializedSize(const OutboundRowBatch& batch); static int64_t GetDeserializedSize(const OutboundRowBatch& batch); static int64_t GetDeserializedSize(const RowBatchHeaderPB& header, @@ -458,25 +463,33 @@ class RowBatch { bool UseFullDedup(); /// Overload for testing that allows the test to force the deduplication level. - Status Serialize(TRowBatch* output_batch, bool full_dedup); + Status Serialize(OutboundRowBatch* output_batch, bool full_dedup); - /// Shared implementation between thrift and protobuf to serialize this row batch. + typedef FixedSizeHashTable DedupMap; + + /// Implementation for protobuf to serialize this row batch. + /// + /// 'distinct_tuples': pointer to an empty DedupMap. Should not be null if full + /// deduplication is used. + /// 'output_batch': output_batch's tuple_offsets and tuple_data will be modified in the + /// following ways: + /// - 'tuple_data': Updated to hold the serialized tuples' + /// data. If 'is_compressed' is true, this is LZ4 + /// compressed. The tuple_data_length_ is also updated + /// accordingly. + /// - 'tuple_offsets': Updated to contain offsets of all tuples into + /// 'tuple_data' upon return. There are a total of + /// num_rows * num_tuples_per_row offsets. An offset + /// of -1 records a NULL. /// - /// 'full_dedup': true if full deduplication is used. - /// 'tuple_offsets': Updated to contain offsets of all tuples into 'tuple_data' upon - /// return. There are a total of num_rows * num_tuples_per_row offsets. - /// An offset of -1 records a NULL. - /// 'tuple_data': Updated to hold the serialized tuples' data. If 'is_compressed' - /// is true, this is LZ4 compressed. - /// 'uncompressed_size': Updated with the uncompressed size of 'tuple_data'. /// 'is_compressed': true if compression is applied on 'tuple_data'. + /// 'size': Expected size of serialized row batch data. /// /// Returns error status if serialization failed. Returns OK otherwise. - /// TODO: clean this up once the thrift RPC implementation is removed. - Status Serialize(bool full_dedup, vector* tuple_offsets, string* tuple_data, - int64_t* uncompressed_size, bool* is_compressed); + Status Serialize(DedupMap* distinct_tuples, OutboundRowBatch* output_batch, + bool* is_compressed, int64_t size); - /// Shared implementation between thrift and protobuf to deserialize a row batch. + /// Implementation for protobuf to deserialize a row batch. /// /// 'input_tuple_offsets': an int32_t array of tuples; offsets into 'input_tuple_data'. /// Used for populating the tuples in the row batch with actual pointers. @@ -489,14 +502,10 @@ class RowBatch { /// 'is_compressed': True if 'input_tuple_data' is compressed. /// /// 'tuple_data': buffer of 'uncompressed_size' bytes for holding tuple data. - /// - /// TODO: clean this up once the thrift RPC implementation is removed. void Deserialize(const kudu::Slice& input_tuple_offsets, const kudu::Slice& input_tuple_data, int64_t uncompressed_size, bool is_compressed, uint8_t* tuple_data); - typedef FixedSizeHashTable DedupMap; - /// The total size of all data represented in this row batch (tuples and referenced /// string and collection data). This is the size of the row batch after removing all /// gaps in the auxiliary and deduplicated tuples (i.e. the smallest footprint for the @@ -505,7 +514,7 @@ class RowBatch { int64_t TotalByteSize(DedupMap* distinct_tuples); Status SerializeInternal(int64_t size, DedupMap* distinct_tuples, - vector* tuple_offsets, string* tuple_data); + vector* tuple_offsets, char* tuple_data); /// All members below need to be handled in RowBatch::AcquireState() @@ -562,16 +571,6 @@ class RowBatch { /// The BufferInfo for the 'tuple_ptrs_' which are allocated from the buffer pool. std::unique_ptr tuple_ptrs_info_; - - /// String to write compressed tuple data to in Serialize(). - /// This is a string so we can swap() with the string in the serialized row batch - /// (i.e. TRowBatch or OutboundRowBatch) we're serializing to (we don't compress - /// directly into the serialized row batch in case the compressed data is longer than - /// the uncompressed data). Swapping avoids copying data to the serialized row batch - /// and avoids excess memory allocations: since we reuse the serialized row batches, and - /// assuming all row batches are roughly the same size, all strings will eventually be - /// allocated to the right size. - std::string compression_scratch_; }; } diff --git a/common/thrift/Results.thrift b/common/thrift/Results.thrift index 54aae75a7..ff5ffd4ff 100644 --- a/common/thrift/Results.thrift +++ b/common/thrift/Results.thrift @@ -22,30 +22,6 @@ include "Data.thrift" include "Types.thrift" include "CatalogObjects.thrift" -// Serialized, self-contained version of a RowBatch (in be/src/runtime/row-batch.h). -struct TRowBatch { - // total number of rows contained in this batch - 1: required i32 num_rows - - // row composition - 2: required list row_tuples - - // There are a total of num_rows * num_tuples_per_row offsets - // pointing into tuple_data. - // An offset of -1 records a NULL. - 3: list tuple_offsets - - // binary tuple data - // TODO: figure out how we can avoid copying the data during TRowBatch construction - 4: string tuple_data - - // Indicates the type of compression used - 5: required CatalogObjects.THdfsCompression compression_type - - // Indicates the uncompressed size - 6:i32 uncompressed_size -} - struct TResultSetMetadata { 1: required list columns } diff --git a/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java b/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java index 3cffe685e..06b8f31b0 100644 --- a/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java +++ b/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java @@ -57,9 +57,39 @@ public class DataStreamSink extends DataSink { return "EXCHANGE SENDER"; } + /** + * This method estimate total buffer size needed for outbound_batches_ in + * KrpcDataStreamSender. The total buffer size follow this formula: + * + * buffer_size = num_channels * 2 * (tuple_buffer_length + compressed_buffer_length) + * + * This method estimate that both tuple_buffer_length and compressed_buffer_length are + * equal to avgOutboundRowBatchSize. If outputPartiton_ is partitioned, all of the + * channel's OutboundRowBatches are used. Otherwise, only a pair of OutboundRowBatches + * in KrpcDataStreamSender class are used. + */ + private long estimateOutboundRowBatchBuffers(TQueryOptions queryOptions) { + int numChannels = + outputPartition_.isPartitioned() ? exchNode_.getFragment().getNumInstances() : 1; + long rowBatchSize = queryOptions.isSetBatch_size() && queryOptions.batch_size > 0 ? + queryOptions.batch_size : + PlanNode.DEFAULT_ROWBATCH_SIZE; + long avgOutboundRowBatchSize = Math.min( + (long) Math.ceil(rowBatchSize * exchNode_.getAvgSerializedRowSize(exchNode_)), + PlanNode.ROWBATCH_MAX_MEM_USAGE); + // Each channel has 2 OutboundRowBatch (KrpcDataStreamSender::NUM_OUTBOUND_BATCHES). + int outboundBatchesPerChannel = 2; + // Each OutboundRowBatch has 2 TrackedString, tuple_data_ and compressed_scratch_. + int bufferPerOutboundBatch = 2; + long bufferSize = numChannels * outboundBatchesPerChannel * bufferPerOutboundBatch + * avgOutboundRowBatchSize; + return bufferSize; + } + @Override public void computeResourceProfile(TQueryOptions queryOptions) { - resourceProfile_ = ResourceProfile.noReservation(0); + long estimatedMem = estimateOutboundRowBatchBuffers(queryOptions); + resourceProfile_ = ResourceProfile.noReservation(estimatedMem); } @Override diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java b/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java index 7974c48d4..c9bba8fdf 100644 --- a/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java +++ b/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java @@ -555,8 +555,8 @@ public class PlannerTestBase extends FrontendTestBase { String planDiff = TestUtils.compareOutput( Lists.newArrayList(explainStr.split("\n")), expectedPlan, true, resultFilters); if (!planDiff.isEmpty()) { - errorLog.append(String.format( - "\nSection %s of query:\n%s\n\n%s", section, query, planDiff)); + errorLog.append(String.format("\nSection %s of query at line %d:\n%s\n\n%s", + section, testCase.getStartingLineNum(), query, planDiff)); // Append the VERBOSE explain plan because it contains details about // tuples/sizes/cardinality for easier debugging. String verbosePlan = getVerboseExplainPlan(queryCtx); diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test index 42715d638..b5f1b7330 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test @@ -5,7 +5,7 @@ from tpch_parquet.customer inner join tpch_parquet.nation on c_nationkey = n_nationkey ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=49.97MB Threads=5 -Per-Host Resource Estimates: Memory=115MB +Per-Host Resource Estimates: Memory=117MB Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.customer INNER JOIN tpch_parquet.nation ON c_nationkey = n_nationkey @@ -21,7 +21,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Host Resources: mem-estimate=41.95MB mem-reservation=33.94MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=43.25MB mem-reservation=33.94MB thread-reservation=2 runtime-filters-memory=1.00MB 02:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: c_nationkey = n_nationkey | fk/pk conjuncts: c_nationkey = n_nationkey @@ -36,7 +36,7 @@ Per-Host Resources: mem-estimate=41.95MB mem-reservation=33.94MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.44MB mem-reservation=32.00KB thread-reservation=2 | 01:SCAN HDFS [tpch_parquet.nation, RANDOM] | HDFS partitions=1/1 files=1 size=3.04KB | stored statistics: @@ -65,7 +65,7 @@ from tpch_parquet.lineitem left join tpch_parquet.orders on l_orderkey = o_orderkey ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=126.00MB Threads=5 -Per-Host Resource Estimates: Memory=534MB +Per-Host Resource Estimates: Memory=535MB Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.lineitem LEFT OUTER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey @@ -95,7 +95,7 @@ Per-Host Resources: mem-estimate=382.84MB mem-reservation=86.00MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=40.68MB mem-reservation=24.00MB thread-reservation=2 | 01:SCAN HDFS [tpch_parquet.orders, RANDOM] | HDFS partitions=1/1 files=2 size=54.21MB | stored statistics: @@ -152,7 +152,7 @@ Per-Host Resources: mem-estimate=153.47MB mem-reservation=92.00MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=40.05MB mem-reservation=4.00MB thread-reservation=2 | 01:SCAN HDFS [tpch_parquet.orders, RANDOM] | HDFS partitions=1/1 files=2 size=54.21MB | stored statistics: @@ -237,7 +237,7 @@ Per-Host Resources: mem-estimate=123.37MB mem-reservation=81.00MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=40.29MB mem-reservation=8.00MB thread-reservation=2 | 01:SCAN HDFS [tpch_parquet.orders, RANDOM] | HDFS partitions=1/1 files=2 size=54.21MB | stored statistics: @@ -254,7 +254,7 @@ Per-Host Resources: mem-estimate=123.37MB mem-reservation=81.00MB thread-reserva | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=81.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=81.14MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM] HDFS partitions=1/1 files=3 size=193.98MB runtime filters: RF000[bloom] -> l_orderkey @@ -272,7 +272,7 @@ select distinct * from tpch_parquet.lineitem ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=136.00MB Threads=4 -Per-Host Resource Estimates: Memory=1012MB +Per-Host Resource Estimates: Memory=1015MB Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 @@ -300,7 +300,7 @@ Per-Host Resources: mem-estimate=473.84MB mem-reservation=46.00MB thread-reserva | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=427.37MB mem-reservation=74.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=430.12MB mem-reservation=74.00MB thread-reservation=2 01:AGGREGATE [STREAMING] | group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment | mem-estimate=347.37MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -354,7 +354,7 @@ Per-Host Resources: mem-estimate=86.40MB mem-reservation=48.00MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=137.23MB mem-reservation=50.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=137.61MB mem-reservation=50.00MB thread-reservation=2 01:AGGREGATE [STREAMING] | output: group_concat(l_linestatus, ',') | group by: l_orderkey, l_partkey @@ -395,7 +395,7 @@ PLAN-ROOT SINK | in pipelines: 01(GETNEXT) | F01:PLAN FRAGMENT [HASH(int_col)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=40.04MB mem-reservation=40.00MB thread-reservation=1 +Per-Host Resources: mem-estimate=40.05MB mem-reservation=40.00MB thread-reservation=1 02:ANALYTIC | functions: max(tinyint_col) | partition by: int_col @@ -415,7 +415,7 @@ Per-Host Resources: mem-estimate=40.04MB mem-reservation=40.00MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2 +Per-Host Resources: mem-estimate=16.11MB mem-reservation=32.00KB thread-reservation=2 00:SCAN HDFS [functional.alltypes, RANDOM] HDFS partitions=24/24 files=24 size=478.45KB stored statistics: diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test index ec49a1388..cb58de723 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test @@ -22,7 +22,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=44.00MB Threads=3 -Per-Host Resource Estimates: Memory=191MB +Per-Host Resource Estimates: Memory=192MB Analyzed query: SELECT * FROM tpch_parquet.lineitem F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 @@ -37,7 +37,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=80.92MB mem-reservation=40.00MB thread-reservation=2 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM] HDFS partitions=1/1 files=3 size=193.99MB stored statistics: @@ -49,7 +49,7 @@ Per-Host Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=44.00MB Threads=2 -Per-Host Resource Estimates: Memory=191MB +Per-Host Resource Estimates: Memory=192MB Analyzed query: SELECT * FROM tpch_parquet.lineitem F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 @@ -64,7 +64,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=80.92MB mem-reservation=40.00MB thread-reservation=1 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM] HDFS partitions=1/1 files=3 size=193.99MB stored statistics: @@ -115,7 +115,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=80.17MB mem-reservation=4.00MB thread-reservation=2 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM] HDFS partitions=1/1 files=3 size=193.99MB stored statistics: @@ -142,7 +142,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=80.17MB mem-reservation=4.00MB thread-reservation=1 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM] HDFS partitions=1/1 files=3 size=193.99MB stored statistics: @@ -648,7 +648,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=12.00MB Threads=3 -Per-Host Resource Estimates: Memory=199MB +Per-Host Resource Estimates: Memory=200MB Analyzed query: SELECT * FROM tpch.lineitem F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 @@ -663,7 +663,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=88.92MB mem-reservation=8.00MB thread-reservation=2 00:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB stored statistics: @@ -675,7 +675,7 @@ Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservati in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=20.00MB Threads=3 -Per-Host Resource Estimates: Memory=287MB +Per-Host Resource Estimates: Memory=289MB Analyzed query: SELECT * FROM tpch.lineitem F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 @@ -690,7 +690,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=88.92MB mem-reservation=8.00MB thread-reservation=1 00:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB stored statistics: @@ -740,7 +740,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=88.17MB mem-reservation=8.00MB thread-reservation=2 00:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB stored statistics: @@ -752,7 +752,7 @@ Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservati in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=20.00MB Threads=3 -Per-Host Resource Estimates: Memory=286MB +Per-Host Resource Estimates: Memory=287MB Analyzed query: SELECT l_comment FROM tpch.lineitem F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 @@ -767,7 +767,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=88.17MB mem-reservation=8.00MB thread-reservation=1 00:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB stored statistics: @@ -804,7 +804,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=4.03MB Threads=3 -Per-Host Resource Estimates: Memory=20MB +Per-Host Resource Estimates: Memory=21MB Codegen disabled by planner Analyzed query: SELECT * FROM functional.alltypes @@ -820,7 +820,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2 +Per-Host Resources: mem-estimate=16.36MB mem-reservation=32.00KB thread-reservation=2 00:SCAN HDFS [functional.alltypes, RANDOM] HDFS partitions=24/24 files=24 size=478.45KB stored statistics: @@ -849,7 +849,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +Per-Instance Resources: mem-estimate=16.36MB mem-reservation=32.00KB thread-reservation=1 00:SCAN HDFS [functional.alltypes, RANDOM] HDFS partitions=24/24 files=24 size=478.45KB stored statistics: @@ -890,7 +890,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=12.00MB Threads=3 -Per-Host Resource Estimates: Memory=74MB +Per-Host Resource Estimates: Memory=75MB WARNING: The following tables are missing relevant table and/or column statistics. tpch_avro.orders Analyzed query: SELECT * FROM tpch_avro.orders @@ -975,7 +975,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=12.00MB Threads=3 -Per-Host Resource Estimates: Memory=74MB +Per-Host Resource Estimates: Memory=75MB WARNING: The following tables are missing relevant table and/or column statistics. tpch_avro.orders Analyzed query: SELECT * FROM tpch_avro.orders @@ -992,7 +992,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=64.00MB mem-reservation=8.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=64.36MB mem-reservation=8.00MB thread-reservation=2 00:SCAN HDFS [tpch_avro.orders, RANDOM] HDFS partitions=1/1 files=3 size=156.92MB stored statistics: @@ -1046,7 +1046,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=32.32MB mem-reservation=8.00MB thread-reservation=2 00:SCAN HDFS [tpch_rc.customer, RANDOM] HDFS partitions=1/1 files=1 size=22.47MB stored statistics: @@ -1075,7 +1075,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=32.32MB mem-reservation=8.00MB thread-reservation=1 00:SCAN HDFS [tpch_rc.customer, RANDOM] HDFS partitions=1/1 files=1 size=22.47MB stored statistics: @@ -1112,7 +1112,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=12.00MB Threads=3 -Per-Host Resource Estimates: Memory=26MB +Per-Host Resource Estimates: Memory=27MB WARNING: The following tables are missing relevant table and/or column statistics. tpcds_seq_snap.web_returns Analyzed query: SELECT * FROM tpcds_seq_snap.web_returns @@ -1129,7 +1129,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=16.42MB mem-reservation=8.00MB thread-reservation=2 00:SCAN HDFS [tpcds_seq_snap.web_returns, RANDOM] HDFS partitions=1/1 files=1 size=6.61MB stored statistics: @@ -1141,7 +1141,7 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservati in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=12.00MB Threads=2 -Per-Host Resource Estimates: Memory=26MB +Per-Host Resource Estimates: Memory=27MB WARNING: The following tables are missing relevant table and/or column statistics. tpcds_seq_snap.web_returns Analyzed query: SELECT * FROM tpcds_seq_snap.web_returns @@ -1158,7 +1158,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=16.42MB mem-reservation=8.00MB thread-reservation=1 00:SCAN HDFS [tpcds_seq_snap.web_returns, RANDOM] HDFS partitions=1/1 files=1 size=6.61MB stored statistics: @@ -1325,7 +1325,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=4.09MB Threads=3 -Per-Host Resource Estimates: Memory=26MB +Per-Host Resource Estimates: Memory=27MB WARNING: The following tables are missing relevant table and/or column statistics. functional.alltypesmixedformat Analyzed query: SELECT * FROM functional.alltypesmixedformat @@ -1342,7 +1342,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=2 +Per-Host Resources: mem-estimate=16.33MB mem-reservation=88.00KB thread-reservation=2 00:SCAN HDFS [functional.alltypesmixedformat, RANDOM] HDFS partitions=4/4 files=4 size=66.33KB stored statistics: @@ -1355,7 +1355,7 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=4.17MB Threads=3 -Per-Host Resource Estimates: Memory=42MB +Per-Host Resource Estimates: Memory=43MB WARNING: The following tables are missing relevant table and/or column statistics. functional.alltypesmixedformat Analyzed query: SELECT * FROM functional.alltypesmixedformat @@ -1372,7 +1372,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=4 -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=1 +Per-Instance Resources: mem-estimate=16.33MB mem-reservation=88.00KB thread-reservation=1 00:SCAN HDFS [functional.alltypesmixedformat, RANDOM] HDFS partitions=4/4 files=4 size=66.33KB stored statistics: @@ -1425,7 +1425,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Host Resources: mem-estimate=256.00KB mem-reservation=0B thread-reservation=1 +Per-Host Resources: mem-estimate=592.00KB mem-reservation=0B thread-reservation=1 00:SCAN HBASE [functional_hbase.alltypes] stored statistics: table: rows=unavailable @@ -1452,7 +1452,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Instance Resources: mem-estimate=256.00KB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=592.00KB mem-reservation=0B thread-reservation=1 00:SCAN HBASE [functional_hbase.alltypes] stored statistics: table: rows=unavailable @@ -1500,7 +1500,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=4.00KB mem-reservation=0B thread-reservation=1 +Per-Host Resources: mem-estimate=376.00KB mem-reservation=0B thread-reservation=1 00:SCAN HBASE [functional_hbase.alltypessmall] stored statistics: table: rows=100 @@ -1526,7 +1526,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=4.00KB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=376.00KB mem-reservation=0B thread-reservation=1 00:SCAN HBASE [functional_hbase.alltypessmall] stored statistics: table: rows=100 @@ -1556,7 +1556,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=4.00MB Threads=2 -Per-Host Resource Estimates: Memory=1.00GB +Per-Host Resource Estimates: Memory=1.01GB WARNING: The following tables are missing relevant table and/or column statistics. functional.alltypes_datasource Analyzed query: SELECT * FROM functional.alltypes_datasource @@ -1580,7 +1580,7 @@ Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B thread-reservation=1 in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=4.00MB Threads=2 -Per-Host Resource Estimates: Memory=1.00GB +Per-Host Resource Estimates: Memory=1.01GB WARNING: The following tables are missing relevant table and/or column statistics. functional.alltypes_datasource Analyzed query: SELECT * FROM functional.alltypes_datasource @@ -1646,7 +1646,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=12.00MB Threads=3 -Per-Host Resource Estimates: Memory=199MB +Per-Host Resource Estimates: Memory=200MB Analyzed query: SELECT * FROM tpch.lineitem UNION ALL SELECT * FROM tpch.lineitem @@ -1662,7 +1662,7 @@ PLAN-ROOT SINK | in pipelines: 01(GETNEXT), 02(GETNEXT) | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=88.92MB mem-reservation=8.00MB thread-reservation=2 00:UNION | pass-through-operands: all | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -1690,7 +1690,7 @@ Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservati in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=20.00MB Threads=3 -Per-Host Resource Estimates: Memory=287MB +Per-Host Resource Estimates: Memory=289MB Analyzed query: SELECT * FROM tpch.lineitem UNION ALL SELECT * FROM tpch.lineitem @@ -1706,7 +1706,7 @@ PLAN-ROOT SINK | in pipelines: 01(GETNEXT), 02(GETNEXT) | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=88.92MB mem-reservation=8.00MB thread-reservation=1 00:UNION | pass-through-operands: all | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -1797,7 +1797,7 @@ Per-Host Resources: mem-estimate=25.01MB mem-reservation=17.00MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=120.06MB mem-reservation=38.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=120.30MB mem-reservation=38.00MB thread-reservation=2 01:AGGREGATE [STREAMING] | output: count(*) | group by: l_orderkey @@ -1846,7 +1846,7 @@ Per-Instance Resources: mem-estimate=25.01MB mem-reservation=17.00MB thread-rese | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=120.06MB mem-reservation=38.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=120.30MB mem-reservation=38.00MB thread-reservation=1 01:AGGREGATE [STREAMING] | output: count(*) | group by: l_orderkey @@ -2414,7 +2414,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=55.00MB Threads=5 -Per-Host Resource Estimates: Memory=591MB +Per-Host Resource Estimates: Memory=592MB Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN tpch.orders ON l_orderkey = o_orderkey @@ -2445,7 +2445,7 @@ Per-Host Resources: mem-estimate=391.84MB mem-reservation=43.00MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=88.68MB mem-reservation=8.00MB thread-reservation=2 | 01:SCAN HDFS [tpch.orders, RANDOM] | HDFS partitions=1/1 files=1 size=162.56MB | stored statistics: @@ -2468,7 +2468,7 @@ Per-Host Resources: mem-estimate=391.84MB mem-reservation=43.00MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=98.00MB Threads=5 -Per-Host Resource Estimates: Memory=681MB +Per-Host Resource Estimates: Memory=685MB Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN tpch.orders ON l_orderkey = o_orderkey @@ -2485,7 +2485,7 @@ PLAN-ROOT SINK | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=89.60MB mem-reservation=8.00MB thread-reservation=1 02:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=00 | hash predicates: l_orderkey = o_orderkey @@ -2508,7 +2508,7 @@ Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=88.68MB mem-reservation=8.00MB thread-reservation=1 | 01:SCAN HDFS [tpch.orders, RANDOM] | HDFS partitions=1/1 files=1 size=162.56MB | stored statistics: @@ -2575,7 +2575,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=56.00MB Threads=6 -Per-Host Resource Estimates: Memory=408MB +Per-Host Resource Estimates: Memory=413MB Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN /* +shuffle */ tpch.orders ON l_orderkey = o_orderkey @@ -2606,7 +2606,7 @@ Per-Host Resources: mem-estimate=119.53MB mem-reservation=35.00MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=90.05MB mem-reservation=8.00MB thread-reservation=2 | 01:SCAN HDFS [tpch.orders, RANDOM] | HDFS partitions=1/1 files=1 size=162.56MB | stored statistics: @@ -2623,7 +2623,7 @@ Per-Host Resources: mem-estimate=119.53MB mem-reservation=35.00MB thread-reserva | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=89.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=91.75MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB 00:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB runtime filters: RF000[bloom] -> l_orderkey @@ -2636,7 +2636,7 @@ Per-Host Resources: mem-estimate=89.00MB mem-reservation=9.00MB thread-reservati in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=99.00MB Threads=8 -Per-Host Resource Estimates: Memory=520MB +Per-Host Resource Estimates: Memory=539MB Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN /* +shuffle */ tpch.orders ON l_orderkey = o_orderkey @@ -2652,7 +2652,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=12.98MB mem-reservation=0B thread-reservation=1 02:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=00 | hash predicates: l_orderkey = o_orderkey @@ -2675,7 +2675,7 @@ Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservati | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=92.10MB mem-reservation=8.00MB thread-reservation=1 | 01:SCAN HDFS [tpch.orders, RANDOM] | HDFS partitions=1/1 files=1 size=162.56MB | stored statistics: @@ -2693,7 +2693,7 @@ Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservati | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=93.50MB mem-reservation=8.00MB thread-reservation=1 00:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB runtime filters: RF000[bloom] -> l_orderkey @@ -2745,7 +2745,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=20.00MB Threads=5 -Per-Host Resource Estimates: Memory=542MB +Per-Host Resource Estimates: Memory=544MB Analyzed query: SELECT * FROM tpch.lineitem, tpch.orders F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 @@ -2760,7 +2760,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=342.84MB mem-reservation=8.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=344.44MB mem-reservation=8.00MB thread-reservation=2 02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] | mem-estimate=244.49MB mem-reservation=0B thread-reservation=0 | tuple-ids=0,1 row-size=402B cardinality=9.00T @@ -2772,7 +2772,7 @@ Per-Host Resources: mem-estimate=342.84MB mem-reservation=8.00MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=88.68MB mem-reservation=8.00MB thread-reservation=2 | 01:SCAN HDFS [tpch.orders, RANDOM] | HDFS partitions=1/1 files=1 size=162.56MB | stored statistics: @@ -2794,7 +2794,7 @@ Per-Host Resources: mem-estimate=342.84MB mem-reservation=8.00MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=28.00MB Threads=5 -Per-Host Resource Estimates: Memory=631MB +Per-Host Resource Estimates: Memory=635MB Analyzed query: SELECT * FROM tpch.lineitem, tpch.orders F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 @@ -2809,7 +2809,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=89.60MB mem-reservation=8.00MB thread-reservation=1 02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] | join table id: 00 | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -2828,7 +2828,7 @@ Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=88.68MB mem-reservation=8.00MB thread-reservation=1 | 01:SCAN HDFS [tpch.orders, RANDOM] | HDFS partitions=1/1 files=1 size=162.56MB | stored statistics: @@ -2958,7 +2958,7 @@ PLAN-ROOT SINK | in pipelines: 01(GETNEXT) | F01:PLAN FRAGMENT [HASH(int_col)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=10.04MB mem-reservation=10.00MB thread-reservation=1 +Per-Host Resources: mem-estimate=10.05MB mem-reservation=10.00MB thread-reservation=1 02:ANALYTIC | functions: max(tinyint_col) | partition by: int_col @@ -2978,7 +2978,7 @@ Per-Host Resources: mem-estimate=10.04MB mem-reservation=10.00MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2 +Per-Host Resources: mem-estimate=16.11MB mem-reservation=32.00KB thread-reservation=2 00:SCAN HDFS [functional.alltypes, RANDOM] HDFS partitions=24/24 files=24 size=478.45KB stored statistics: @@ -2991,7 +2991,7 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=24.06MB Threads=5 -Per-Host Resource Estimates: Memory=56MB +Per-Host Resource Estimates: Memory=57MB Codegen disabled by planner Analyzed query: SELECT max(tinyint_col) OVER (PARTITION BY int_col) FROM functional.alltypes @@ -3028,7 +3028,7 @@ Per-Instance Resources: mem-estimate=10.06MB mem-reservation=10.00MB thread-rese | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +Per-Instance Resources: mem-estimate=16.21MB mem-reservation=32.00KB thread-reservation=1 00:SCAN HDFS [functional.alltypes, RANDOM] HDFS partitions=24/24 files=24 size=478.45KB stored statistics: @@ -3394,7 +3394,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=157.00MB Threads=12 -Per-Host Resource Estimates: Memory=531MB +Per-Host Resource Estimates: Memory=532MB Analyzed query: SELECT DISTINCT l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment FROM tpch_parquet.lineitem INNER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey WHERE l_tax > CAST(10 AS DECIMAL(3,0)) UNION ALL SELECT @@ -3416,7 +3416,7 @@ PLAN-ROOT SINK | in pipelines: 14(GETNEXT), 05(GETNEXT), 08(GETNEXT) | F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=151.47MB mem-reservation=60.00MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=151.74MB mem-reservation=60.00MB thread-reservation=2 runtime-filters-memory=2.00MB 00:UNION | pass-through-operands: 14 | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -3437,7 +3437,7 @@ Per-Host Resources: mem-estimate=151.47MB mem-reservation=60.00MB thread-reserva | | | in pipelines: 09(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=40.05MB mem-reservation=4.00MB thread-reservation=2 | | 09:SCAN HDFS [tpch_parquet.orders, RANDOM] | | HDFS partitions=1/1 files=2 size=54.21MB | | stored statistics: @@ -3476,7 +3476,7 @@ Per-Host Resources: mem-estimate=151.47MB mem-reservation=60.00MB thread-reserva | | | in pipelines: 06(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=40.13MB mem-reservation=8.00MB thread-reservation=2 | | 06:SCAN HDFS [tpch_parquet.orders, RANDOM] | | HDFS partitions=1/1 files=2 size=54.21MB | | predicates: o_orderpriority = '2-HIGH' @@ -3534,7 +3534,7 @@ Per-Host Resources: mem-estimate=67.98MB mem-reservation=52.00MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=40.14MB mem-reservation=4.00MB thread-reservation=2 | 02:SCAN HDFS [tpch_parquet.orders, RANDOM] | HDFS partitions=1/1 files=2 size=54.21MB | stored statistics: @@ -3551,7 +3551,7 @@ Per-Host Resources: mem-estimate=67.98MB mem-reservation=52.00MB thread-reservat | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=81.00MB mem-reservation=25.00MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=81.92MB mem-reservation=25.00MB thread-reservation=2 runtime-filters-memory=1.00MB 01:SCAN HDFS [tpch_parquet.lineitem, RANDOM] HDFS partitions=1/1 files=3 size=193.99MB predicates: l_tax > CAST(10 AS DECIMAL(3,0)) @@ -3567,7 +3567,7 @@ Per-Host Resources: mem-estimate=81.00MB mem-reservation=25.00MB thread-reservat in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=227.00MB Threads=10 -Per-Host Resource Estimates: Memory=583MB +Per-Host Resource Estimates: Memory=586MB Analyzed query: SELECT DISTINCT l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment FROM tpch_parquet.lineitem INNER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey WHERE l_tax > CAST(10 AS DECIMAL(3,0)) UNION ALL SELECT @@ -3590,7 +3590,7 @@ PLAN-ROOT SINK | F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=80.00MB mem-reservation=24.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=80.28MB mem-reservation=24.00MB thread-reservation=1 00:UNION | pass-through-operands: 14 | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -3619,7 +3619,7 @@ Per-Instance Resources: mem-estimate=80.00MB mem-reservation=24.00MB thread-rese | | | in pipelines: 09(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Instance Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=40.05MB mem-reservation=4.00MB thread-reservation=1 | | 09:SCAN HDFS [tpch_parquet.orders, RANDOM] | | HDFS partitions=1/1 files=2 size=54.21MB | | stored statistics: @@ -3666,7 +3666,7 @@ Per-Instance Resources: mem-estimate=80.00MB mem-reservation=24.00MB thread-rese | | | in pipelines: 06(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Instance Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=40.13MB mem-reservation=8.00MB thread-reservation=1 | | 06:SCAN HDFS [tpch_parquet.orders, RANDOM] | | HDFS partitions=1/1 files=2 size=54.21MB | | predicates: o_orderpriority = '2-HIGH' @@ -3703,7 +3703,7 @@ Per-Instance Resources: mem-estimate=80.00MB mem-reservation=24.00MB thread-rese | in pipelines: 01(GETNEXT) | F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=27.23MB mem-reservation=17.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=28.06MB mem-reservation=17.00MB thread-reservation=1 04:AGGREGATE [STREAMING] | group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment | mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0 @@ -3732,7 +3732,7 @@ Per-Instance Resources: mem-estimate=27.23MB mem-reservation=17.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=40.14MB mem-reservation=4.00MB thread-reservation=1 | 02:SCAN HDFS [tpch_parquet.orders, RANDOM] | HDFS partitions=1/1 files=2 size=54.21MB | stored statistics: @@ -3750,7 +3750,7 @@ Per-Instance Resources: mem-estimate=27.23MB mem-reservation=17.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=80.00MB mem-reservation=24.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=80.92MB mem-reservation=24.00MB thread-reservation=1 01:SCAN HDFS [tpch_parquet.lineitem, RANDOM] HDFS partitions=1/1 files=3 size=193.99MB predicates: l_tax > CAST(10 AS DECIMAL(3,0)) @@ -3904,7 +3904,7 @@ PLAN-ROOT SINK in pipelines: 02(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=217.75MB Threads=11 -Per-Host Resource Estimates: Memory=544MB +Per-Host Resource Estimates: Memory=546MB Analyzed query: SELECT c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity) FROM tpch.customer, tpch.orders, tpch.lineitem LEFT SEMI JOIN (SELECT l_orderkey FROM tpch.lineitem GROUP BY l_orderkey HAVING sum(l_quantity) @@ -3976,7 +3976,7 @@ Per-Host Resources: mem-estimate=144.65MB mem-reservation=109.75MB thread-reserv | | in pipelines: 03(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=139.51MB mem-reservation=42.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=139.84MB mem-reservation=42.00MB thread-reservation=2 | 04:AGGREGATE [STREAMING] | | output: sum(l_quantity) | | group by: l_orderkey @@ -4008,7 +4008,7 @@ Per-Host Resources: mem-estimate=144.65MB mem-reservation=109.75MB thread-reserv | | in pipelines: 00(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.16MB mem-reservation=8.00MB thread-reservation=2 | 00:SCAN HDFS [tpch.customer, RANDOM] | HDFS partitions=1/1 files=1 size=23.08MB | stored statistics: @@ -4033,7 +4033,7 @@ Per-Host Resources: mem-estimate=144.65MB mem-reservation=109.75MB thread-reserv | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=90.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=90.59MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB | 01:SCAN HDFS [tpch.orders, RANDOM] | HDFS partitions=1/1 files=1 size=162.56MB | runtime filters: RF000[bloom] -> o_orderkey, RF002[bloom] -> o_custkey @@ -4051,7 +4051,7 @@ Per-Host Resources: mem-estimate=144.65MB mem-reservation=109.75MB thread-reserv | in pipelines: 02(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=90.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=90.23MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB 02:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB runtime filters: RF000[bloom] -> tpch.lineitem.l_orderkey, RF004[bloom] -> l_orderkey @@ -4064,7 +4064,7 @@ Per-Host Resources: mem-estimate=90.00MB mem-reservation=10.00MB thread-reservat in pipelines: 02(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=322.75MB Threads=16 -Per-Host Resource Estimates: Memory=830MB +Per-Host Resource Estimates: Memory=839MB Analyzed query: SELECT c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity) FROM tpch.customer, tpch.orders, tpch.lineitem LEFT SEMI JOIN (SELECT l_orderkey FROM tpch.lineitem GROUP BY l_orderkey HAVING sum(l_quantity) @@ -4107,7 +4107,7 @@ Per-Instance Resources: mem-estimate=27.56MB mem-reservation=17.00MB thread-rese | in pipelines: 02(GETNEXT) | F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=20.12MB mem-reservation=9.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=22.37MB mem-reservation=9.00MB thread-reservation=1 08:AGGREGATE [STREAMING] | output: sum(l_quantity) | group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice @@ -4144,7 +4144,7 @@ Per-Instance Resources: mem-estimate=20.12MB mem-reservation=9.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=122.00MB mem-reservation=42.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=122.66MB mem-reservation=42.00MB thread-reservation=1 | 04:AGGREGATE [STREAMING] | | output: sum(l_quantity) | | group by: l_orderkey @@ -4184,7 +4184,7 @@ Per-Instance Resources: mem-estimate=20.12MB mem-reservation=9.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.16MB mem-reservation=8.00MB thread-reservation=1 | 00:SCAN HDFS [tpch.customer, RANDOM] | HDFS partitions=1/1 files=1 size=23.08MB | stored statistics: @@ -4218,7 +4218,7 @@ Per-Instance Resources: mem-estimate=20.12MB mem-reservation=9.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=89.17MB mem-reservation=8.00MB thread-reservation=1 | 01:SCAN HDFS [tpch.orders, RANDOM] | HDFS partitions=1/1 files=1 size=162.56MB | runtime filters: RF000[bloom] -> o_orderkey, RF002[bloom] -> o_custkey @@ -4237,7 +4237,7 @@ Per-Instance Resources: mem-estimate=20.12MB mem-reservation=9.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=88.47MB mem-reservation=8.00MB thread-reservation=1 02:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB runtime filters: RF000[bloom] -> tpch.lineitem.l_orderkey, RF004[bloom] -> l_orderkey @@ -4392,7 +4392,7 @@ WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey) | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=88.59MB mem-reservation=8.00MB thread-reservation=2 00:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB stored statistics: @@ -4404,7 +4404,7 @@ Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservati in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=40.00MB Threads=4 -Per-Host Resource Estimates: Memory=285MB +Per-Host Resource Estimates: Memory=288MB Analyzed query: SELECT l_comment, l_partkey FROM tpch.lineitem F01:PLAN FRAGMENT [HASH(l_partkey)] hosts=3 instances=6 @@ -4426,7 +4426,7 @@ WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey) | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=89.18MB mem-reservation=8.00MB thread-reservation=1 00:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB stored statistics: @@ -4480,7 +4480,7 @@ WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey) | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=88.59MB mem-reservation=8.00MB thread-reservation=2 00:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB stored statistics: @@ -4492,7 +4492,7 @@ Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservati in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=16.00MB Threads=4 -Per-Host Resource Estimates: Memory=285MB +Per-Host Resource Estimates: Memory=288MB Analyzed query: SELECT l_comment, l_partkey FROM tpch.lineitem F01:PLAN FRAGMENT [HASH(l_partkey)] hosts=3 instances=6 @@ -4508,7 +4508,7 @@ WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey) | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=89.18MB mem-reservation=8.00MB thread-reservation=1 00:SCAN HDFS [tpch.lineitem, RANDOM] HDFS partitions=1/1 files=1 size=718.94MB stored statistics: @@ -4636,7 +4636,7 @@ Per-Host Resources: mem-estimate=20.16MB mem-reservation=1.94MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=345.94MB mem-reservation=85.94MB thread-reservation=2 +Per-Host Resources: mem-estimate=346.57MB mem-reservation=85.94MB thread-reservation=2 09:AGGREGATE [STREAMING] | group by: c_name, o1.o_orderkey, o2.o_orderstatus | mem-estimate=128.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -4701,7 +4701,7 @@ Per-Host Resources: mem-estimate=345.94MB mem-reservation=85.94MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=179.75MB Threads=5 -Per-Host Resource Estimates: Memory=814MB +Per-Host Resource Estimates: Memory=816MB Analyzed query: SELECT DISTINCT c_name, v.o_orderkey, v.o_orderstatus FROM tpch_nested_parquet.customer c, (SELECT DISTINCT o1.o_orderkey, o2.o_orderstatus FROM c.c_orders o1 INNER JOIN c.c_orders o2 ON o1.o_orderkey = o2.o_orderkey @@ -4732,7 +4732,7 @@ Per-Instance Resources: mem-estimate=20.21MB mem-reservation=1.94MB thread-reser | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=4 -Per-Instance Resources: mem-estimate=345.94MB mem-reservation=85.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=346.78MB mem-reservation=85.94MB thread-reservation=1 09:AGGREGATE [STREAMING] | group by: c_name, o1.o_orderkey, o2.o_orderstatus | mem-estimate=128.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -4893,7 +4893,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=108.00MB Threads=3 -Per-Host Resource Estimates: Memory=247MB +Per-Host Resource Estimates: Memory=249MB Analyzed query: SELECT * FROM tpch_nested_parquet.customer c, (SELECT *, row_number() OVER (ORDER BY o_totalprice ASC) rnum_price, row_number() OVER (ORDER BY o_orderdate ASC) rnum_date, row_number() OVER (ORDER BY @@ -4911,7 +4911,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=136.00MB mem-reservation=104.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=137.48MB mem-reservation=104.00MB thread-reservation=2 01:SUBPLAN | mem-estimate=0B mem-reservation=0B thread-reservation=0 | tuple-ids=12,7,0 row-size=366B cardinality=1.50M @@ -4987,7 +4987,7 @@ Per-Host Resources: mem-estimate=136.00MB mem-reservation=104.00MB thread-reserv in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=212.00MB Threads=3 -Per-Host Resource Estimates: Memory=383MB +Per-Host Resource Estimates: Memory=386MB Analyzed query: SELECT * FROM tpch_nested_parquet.customer c, (SELECT *, row_number() OVER (ORDER BY o_totalprice ASC) rnum_price, row_number() OVER (ORDER BY o_orderdate ASC) rnum_date, row_number() OVER (ORDER BY @@ -5005,7 +5005,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=4 -Per-Instance Resources: mem-estimate=136.00MB mem-reservation=104.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=137.48MB mem-reservation=104.00MB thread-reservation=1 01:SUBPLAN | mem-estimate=0B mem-reservation=0B thread-reservation=0 | tuple-ids=12,7,0 row-size=366B cardinality=1.50M @@ -5235,7 +5235,7 @@ Per-Host Resources: mem-estimate=133.40MB mem-reservation=59.00MB thread-reserva | | | | in pipelines: 03(GETNEXT) | | | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | | Per-Host Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=40.09MB mem-reservation=4.00MB thread-reservation=2 | | | 03:SCAN HDFS [tpch_parquet.orders t4, RANDOM] | | | HDFS partitions=1/1 files=2 size=54.21MB | | | stored statistics: @@ -5252,7 +5252,7 @@ Per-Host Resources: mem-estimate=133.40MB mem-reservation=59.00MB thread-reserva | | | in pipelines: 02(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=41.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=41.09MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 02:SCAN HDFS [tpch_parquet.orders t3, RANDOM] | | HDFS partitions=1/1 files=2 size=54.21MB | | runtime filters: RF004[bloom] -> t3.o_orderkey @@ -5270,7 +5270,7 @@ Per-Host Resources: mem-estimate=133.40MB mem-reservation=59.00MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=41.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=41.09MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 01:SCAN HDFS [tpch_parquet.orders t2, RANDOM] | HDFS partitions=1/1 files=2 size=54.21MB | runtime filters: RF002[bloom] -> t2.o_orderkey @@ -5294,7 +5294,7 @@ Per-Host Resources: mem-estimate=133.40MB mem-reservation=59.00MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=182.00MB Threads=9 -Per-Host Resource Estimates: Memory=455MB +Per-Host Resource Estimates: Memory=457MB Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders t1 INNER JOIN (SELECT /* +straight_join */ t2.o_orderkey k2, k3, k4 FROM tpch_parquet.orders t2 INNER JOIN (SELECT /* +straight_join */ t3.o_orderkey k3, @@ -5315,7 +5315,7 @@ PLAN-ROOT SINK | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=40.82MB mem-reservation=24.00MB thread-reservation=1 06:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=00 | hash predicates: t1.o_orderkey = t3.o_orderkey @@ -5338,7 +5338,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=5.75MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=5.89MB mem-reservation=0B thread-reservation=1 | 05:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=01 | | hash predicates: t2.o_orderkey = t3.o_orderkey @@ -5377,7 +5377,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-rese | | | | in pipelines: 03(GETNEXT) | | | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | | Per-Instance Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=40.09MB mem-reservation=4.00MB thread-reservation=1 | | | 03:SCAN HDFS [tpch_parquet.orders t4, RANDOM] | | | HDFS partitions=1/1 files=2 size=54.21MB | | | stored statistics: @@ -5395,7 +5395,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-rese | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=40.09MB mem-reservation=4.00MB thread-reservation=1 | | 02:SCAN HDFS [tpch_parquet.orders t3, RANDOM] | | HDFS partitions=1/1 files=2 size=54.21MB | | runtime filters: RF004[bloom] -> t3.o_orderkey @@ -5414,7 +5414,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-rese | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=40.09MB mem-reservation=4.00MB thread-reservation=1 | 01:SCAN HDFS [tpch_parquet.orders t2, RANDOM] | HDFS partitions=1/1 files=2 size=54.21MB | runtime filters: RF002[bloom] -> t2.o_orderkey @@ -5542,7 +5542,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Host Resources: mem-estimate=97.55MB mem-reservation=32.00KB thread-reservation=2 +Per-Host Resources: mem-estimate=98.09MB mem-reservation=32.00KB thread-reservation=2 06:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] | mem-estimate=71.53MB mem-reservation=0B thread-reservation=0 | tuple-ids=0,1,2,3 row-size=121B cardinality=156.25M @@ -5554,7 +5554,7 @@ Per-Host Resources: mem-estimate=97.55MB mem-reservation=32.00KB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=20.79MB mem-reservation=8.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=20.88MB mem-reservation=8.00KB thread-reservation=2 | 05:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] | | mem-estimate=2.38MB mem-reservation=0B thread-reservation=0 | | tuple-ids=1,2,3 row-size=12B cardinality=6.25M @@ -5566,7 +5566,7 @@ Per-Host Resources: mem-estimate=97.55MB mem-reservation=32.00KB thread-reservat | | | in pipelines: 02(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.16MB mem-reservation=8.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=16.23MB mem-reservation=8.00KB thread-reservation=2 | | 04:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] | | | mem-estimate=78.12KB mem-reservation=0B thread-reservation=0 | | | tuple-ids=2,3 row-size=10B cardinality=250.00K @@ -5578,7 +5578,7 @@ Per-Host Resources: mem-estimate=97.55MB mem-reservation=32.00KB thread-reservat | | | | in pipelines: 03(GETNEXT) | | | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=16.05MB mem-reservation=128.00KB thread-reservation=2 | | | 03:SCAN HDFS [tpch_parquet.supplier t4, RANDOM] | | | HDFS partitions=1/1 files=1 size=883.03KB | | | stored statistics: @@ -5639,7 +5639,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +Per-Instance Resources: mem-estimate=16.54MB mem-reservation=32.00KB thread-reservation=1 06:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] | join table id: 00 | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -5658,7 +5658,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=8.00KB thread-reservation=1 | 05:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] | | join table id: 01 | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -5677,7 +5677,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-rese | | | in pipelines: 02(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.07MB mem-reservation=8.00KB thread-reservation=1 | | 04:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] | | | join table id: 02 | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -5696,7 +5696,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-rese | | | | in pipelines: 03(GETNEXT) | | | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=128.00KB thread-reservation=1 | | | 03:SCAN HDFS [tpch_parquet.supplier t4, RANDOM] | | | HDFS partitions=1/1 files=1 size=883.03KB | | | stored statistics: @@ -5923,7 +5923,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=79.94MB Threads=4 -Per-Host Resource Estimates: Memory=441MB +Per-Host Resource Estimates: Memory=445MB Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem LIMIT CAST(5 AS TINYINT) @@ -5940,7 +5940,7 @@ PLAN-ROOT SINK | in pipelines: 03(GETNEXT) | F01:PLAN FRAGMENT [HASH(tpch_parquet.lineitem.l_orderkey,tpch_parquet.lineitem.l_partkey,tpch_parquet.lineitem.l_suppkey,tpch_parquet.lineitem.l_linenumber,tpch_parquet.lineitem.l_quantity,tpch_parquet.lineitem.l_extendedprice,tpch_parquet.lineitem.l_discount,tpch_parquet.lineitem.l_tax,tpch_parquet.lineitem.l_returnflag,tpch_parquet.lineitem.l_linestatus,tpch_parquet.lineitem.l_shipdate,tpch_parquet.lineitem.l_commitdate,tpch_parquet.lineitem.l_receiptdate,tpch_parquet.lineitem.l_shipinstruct,tpch_parquet.lineitem.l_shipmode,tpch_parquet.lineitem.l_comment)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=10.92MB mem-reservation=1.94MB thread-reservation=1 03:AGGREGATE [FINALIZE] | group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment | limit: 5 @@ -5954,7 +5954,7 @@ Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=427.37MB mem-reservation=74.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=430.12MB mem-reservation=74.00MB thread-reservation=2 01:AGGREGATE [STREAMING] | group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment | limit: 5 @@ -6016,7 +6016,7 @@ INSERT INTO KUDU [functional_kudu.tinyinttable] | in pipelines: | F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -Per-Host Resources: mem-estimate=0B mem-reservation=0B thread-reservation=1 +Per-Host Resources: mem-estimate=20.00KB mem-reservation=0B thread-reservation=1 00:UNION constant-operands=1 mem-estimate=0B mem-reservation=0B thread-reservation=0 diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/result-spooling.test b/testdata/workloads/functional-planner/queries/PlannerTest/result-spooling.test index 10b5a6644..df28b3fc0 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/result-spooling.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/result-spooling.test @@ -190,7 +190,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=2 +Per-Host Resources: mem-estimate=16.05MB mem-reservation=8.00KB thread-reservation=2 00:SCAN HDFS [functional.dimtbl, RANDOM] HDFS partitions=1/1 files=1 size=171B stored statistics: @@ -241,4 +241,4 @@ Per-Host Resources: mem-estimate=16.02MB mem-reservation=8.00KB thread-reservati mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 tuple-ids=0 row-size=0B cardinality=10 in pipelines: 00(GETNEXT) -==== \ No newline at end of file +==== diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test index 63d87b62b..df589a1b2 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test @@ -4,7 +4,7 @@ from tpch_parquet.customer inner join tpch_parquet.nation on c_nationkey = n_nationkey ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=22.97MB Threads=5 -Per-Host Resource Estimates: Memory=100MB +Per-Host Resource Estimates: Memory=102MB Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.customer INNER JOIN tpch_parquet.nation ON c_nationkey = n_nationkey @@ -20,7 +20,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Host Resources: mem-estimate=26.95MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=28.25MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB 02:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: c_nationkey = n_nationkey | fk/pk conjuncts: c_nationkey = n_nationkey @@ -35,7 +35,7 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=18.94MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.44MB mem-reservation=32.00KB thread-reservation=2 | 01:SCAN HDFS [tpch_parquet.nation, RANDOM] | HDFS partitions=1/1 files=1 size=3.04KB | stored statistics: @@ -58,7 +58,7 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=18.94MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=25.91MB Threads=4 -Per-Host Resource Estimates: Memory=103MB +Per-Host Resource Estimates: Memory=105MB Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.customer INNER JOIN tpch_parquet.nation ON c_nationkey = n_nationkey @@ -75,7 +75,7 @@ PLAN-ROOT SINK | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=25.31MB mem-reservation=16.00MB thread-reservation=1 02:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=00 | hash predicates: c_nationkey = n_nationkey @@ -98,7 +98,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.44MB mem-reservation=32.00KB thread-reservation=1 | 01:SCAN HDFS [tpch_parquet.nation, RANDOM] | HDFS partitions=1/1 files=1 size=3.04KB | stored statistics: @@ -126,7 +126,7 @@ from tpch_parquet.lineitem left join tpch_parquet.orders on l_orderkey = o_orderkey ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=102.00MB Threads=5 -Per-Host Resource Estimates: Memory=534MB +Per-Host Resource Estimates: Memory=535MB Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.lineitem LEFT OUTER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey @@ -156,7 +156,7 @@ Per-Host Resources: mem-estimate=382.84MB mem-reservation=74.00MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=40.68MB mem-reservation=24.00MB thread-reservation=2 | 01:SCAN HDFS [tpch_parquet.orders, RANDOM] | HDFS partitions=1/1 files=2 size=54.21MB | stored statistics: @@ -178,7 +178,7 @@ Per-Host Resources: mem-estimate=382.84MB mem-reservation=74.00MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=136.00MB Threads=4 -Per-Host Resource Estimates: Memory=534MB +Per-Host Resource Estimates: Memory=536MB Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.lineitem LEFT OUTER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey @@ -194,7 +194,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=81.60MB mem-reservation=40.00MB thread-reservation=1 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST] | hash-table-id=00 | hash predicates: l_orderkey = o_orderkey @@ -216,7 +216,7 @@ Per-Instance Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=40.68MB mem-reservation=24.00MB thread-reservation=1 | 01:SCAN HDFS [tpch_parquet.orders, RANDOM] | HDFS partitions=1/1 files=2 size=54.21MB | stored statistics: @@ -243,7 +243,7 @@ from tpch_parquet.orders join /*+shuffle*/ tpch_parquet.customer on o_custkey = c_custkey ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=80.00MB Threads=6 -Per-Host Resource Estimates: Memory=231MB +Per-Host Resource Estimates: Memory=234MB Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders INNER JOIN /* +shuffle */ tpch_parquet.customer ON o_custkey = c_custkey @@ -274,7 +274,7 @@ Per-Host Resources: mem-estimate=55.56MB mem-reservation=35.00MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=25.73MB mem-reservation=16.00MB thread-reservation=2 | 01:SCAN HDFS [tpch_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=12.34MB | stored statistics: @@ -291,7 +291,7 @@ Per-Host Resources: mem-estimate=55.56MB mem-reservation=35.00MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -Per-Host Resources: mem-estimate=41.00MB mem-reservation=25.00MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=42.37MB mem-reservation=25.00MB thread-reservation=2 runtime-filters-memory=1.00MB 00:SCAN HDFS [tpch_parquet.orders, RANDOM] HDFS partitions=1/1 files=2 size=54.21MB runtime filters: RF000[bloom] -> o_custkey @@ -304,7 +304,7 @@ Per-Host Resources: mem-estimate=41.00MB mem-reservation=25.00MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=80.00MB Threads=5 -Per-Host Resource Estimates: Memory=231MB +Per-Host Resource Estimates: Memory=236MB Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders INNER JOIN /* +shuffle */ tpch_parquet.customer ON o_custkey = c_custkey @@ -320,7 +320,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F02:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=2 -Per-Instance Resources: mem-estimate=10.34MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=11.89MB mem-reservation=0B thread-reservation=1 02:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=00 | hash predicates: o_custkey = c_custkey @@ -343,7 +343,7 @@ Per-Instance Resources: mem-estimate=10.34MB mem-reservation=0B thread-reservati | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=25.73MB mem-reservation=16.00MB thread-reservation=1 | 01:SCAN HDFS [tpch_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=12.34MB | stored statistics: @@ -361,7 +361,7 @@ Per-Instance Resources: mem-estimate=10.34MB mem-reservation=0B thread-reservati | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=41.37MB mem-reservation=24.00MB thread-reservation=1 00:SCAN HDFS [tpch_parquet.orders, RANDOM] HDFS partitions=1/1 files=2 size=54.21MB runtime filters: RF000[bloom] -> o_custkey @@ -379,7 +379,7 @@ from tpch_parquet.orders join /*+broadcast*/ tpch_parquet.customer on o_custkey = c_custkey ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=79.00MB Threads=5 -Per-Host Resource Estimates: Memory=220MB +Per-Host Resource Estimates: Memory=221MB Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders INNER JOIN /* +broadcast */ tpch_parquet.customer ON o_custkey = c_custkey @@ -410,7 +410,7 @@ Per-Host Resources: mem-estimate=85.34MB mem-reservation=59.00MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=24.87MB mem-reservation=16.00MB thread-reservation=2 | 01:SCAN HDFS [tpch_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=12.34MB | stored statistics: @@ -433,7 +433,7 @@ Per-Host Resources: mem-estimate=85.34MB mem-reservation=59.00MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=114.00MB Threads=4 -Per-Host Resource Estimates: Memory=255MB +Per-Host Resource Estimates: Memory=257MB Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders INNER JOIN /* +broadcast */ tpch_parquet.customer ON o_custkey = c_custkey @@ -450,7 +450,7 @@ PLAN-ROOT SINK | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=41.55MB mem-reservation=24.00MB thread-reservation=1 02:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=00 | hash predicates: o_custkey = c_custkey @@ -473,7 +473,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.87MB mem-reservation=16.00MB thread-reservation=1 | 01:SCAN HDFS [tpch_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=12.34MB | stored statistics: @@ -533,7 +533,7 @@ Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.09MB thread-reservati | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.33MB mem-reservation=88.00KB thread-reservation=2 | 01:SCAN HDFS [functional_parquet.alltypestiny, RANDOM] | HDFS partitions=4/4 files=4 size=11.92KB | stored statistics: @@ -557,7 +557,7 @@ Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.09MB thread-reservati in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=72.34MB Threads=6 -Per-Host Resource Estimates: Memory=2.07GB +Per-Host Resource Estimates: Memory=2.08GB WARNING: The following tables are missing relevant table and/or column statistics. functional_parquet.alltypestiny Analyzed query: SELECT /* +straight_join */ * FROM functional_parquet.alltypes @@ -575,7 +575,7 @@ PLAN-ROOT SINK | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=1 +Per-Instance Resources: mem-estimate=16.66MB mem-reservation=88.00KB thread-reservation=1 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST] | hash-table-id=00 | hash predicates: alltypes.id = alltypestiny.id @@ -597,7 +597,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=4 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.33MB mem-reservation=88.00KB thread-reservation=1 | 01:SCAN HDFS [functional_parquet.alltypestiny, RANDOM] | HDFS partitions=4/4 files=4 size=11.92KB | stored statistics: @@ -642,7 +642,7 @@ PLAN-ROOT SINK | in pipelines: 03(GETNEXT) | F01:PLAN FRAGMENT [HASH(c_nationkey)] hosts=1 instances=1 -Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=10.05MB mem-reservation=1.94MB thread-reservation=1 03:AGGREGATE [FINALIZE] | output: avg:merge(c_acctbal) | group by: c_nationkey @@ -656,7 +656,7 @@ Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Host Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=34.05MB mem-reservation=4.00MB thread-reservation=2 01:AGGREGATE [STREAMING] | output: avg(c_acctbal) | group by: c_nationkey @@ -691,7 +691,7 @@ PLAN-ROOT SINK | in pipelines: 03(GETNEXT) | F01:PLAN FRAGMENT [HASH(c_nationkey)] hosts=1 instances=1 -Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=10.05MB mem-reservation=1.94MB thread-reservation=1 03:AGGREGATE [FINALIZE] | output: avg:merge(c_acctbal) | group by: c_nationkey @@ -705,7 +705,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=34.05MB mem-reservation=4.00MB thread-reservation=1 01:AGGREGATE [STREAMING] | output: avg(c_acctbal) | group by: c_nationkey @@ -786,7 +786,7 @@ Per-Host Resources: mem-estimate=111.37MB mem-reservation=69.00MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=40.29MB mem-reservation=8.00MB thread-reservation=2 | 01:SCAN HDFS [tpch_parquet.orders, RANDOM] | HDFS partitions=1/1 files=2 size=54.21MB | stored statistics: @@ -803,7 +803,7 @@ Per-Host Resources: mem-estimate=111.37MB mem-reservation=69.00MB thread-reserva | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=81.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=81.14MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM] HDFS partitions=1/1 files=3 size=193.98MB runtime filters: RF000[bloom] -> l_orderkey @@ -816,7 +816,7 @@ Per-Host Resources: mem-estimate=81.00MB mem-reservation=5.00MB thread-reservati in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=120.00MB Threads=6 -Per-Host Resource Estimates: Memory=414MB +Per-Host Resource Estimates: Memory=415MB Analyzed query: SELECT /* +straight_join */ l_orderkey, o_orderstatus, count(*) FROM tpch_parquet.lineitem INNER JOIN tpch_parquet.orders ON o_orderkey = l_orderkey GROUP BY l_orderkey, o_orderstatus HAVING count(*) = CAST(1 AS @@ -849,7 +849,7 @@ Per-Instance Resources: mem-estimate=71.23MB mem-reservation=34.00MB thread-rese | in pipelines: 00(GETNEXT) | F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=66.32MB mem-reservation=34.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=66.70MB mem-reservation=34.00MB thread-reservation=1 03:AGGREGATE [STREAMING] | output: count(*) | group by: l_orderkey, o_orderstatus @@ -879,7 +879,7 @@ Per-Instance Resources: mem-estimate=66.32MB mem-reservation=34.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=40.29MB mem-reservation=8.00MB thread-reservation=1 | 01:SCAN HDFS [tpch_parquet.orders, RANDOM] | HDFS partitions=1/1 files=2 size=54.21MB | stored statistics: @@ -897,7 +897,7 @@ Per-Instance Resources: mem-estimate=66.32MB mem-reservation=34.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=80.14MB mem-reservation=4.00MB thread-reservation=1 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM] HDFS partitions=1/1 files=3 size=193.98MB runtime filters: RF000[bloom] -> l_orderkey @@ -914,7 +914,7 @@ select distinct * from tpch_parquet.lineitem ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=112.00MB Threads=4 -Per-Host Resource Estimates: Memory=1012MB +Per-Host Resource Estimates: Memory=1015MB Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 @@ -942,7 +942,7 @@ Per-Host Resources: mem-estimate=473.84MB mem-reservation=34.00MB thread-reserva | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=427.37MB mem-reservation=74.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=430.12MB mem-reservation=74.00MB thread-reservation=2 01:AGGREGATE [STREAMING] | group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment | mem-estimate=347.37MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -960,7 +960,7 @@ Per-Host Resources: mem-estimate=427.37MB mem-reservation=74.00MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=112.00MB Threads=3 -Per-Host Resource Estimates: Memory=1012MB +Per-Host Resource Estimates: Memory=1015MB Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 @@ -988,7 +988,7 @@ Per-Instance Resources: mem-estimate=473.84MB mem-reservation=34.00MB thread-res | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=427.37MB mem-reservation=74.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=430.12MB mem-reservation=74.00MB thread-reservation=1 01:AGGREGATE [STREAMING] | group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment | mem-estimate=347.37MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -1029,7 +1029,7 @@ PLAN-ROOT SINK | in pipelines: 03(GETNEXT) | F01:PLAN FRAGMENT [HASH(string_col)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=128.07MB mem-reservation=34.00MB thread-reservation=1 +Per-Host Resources: mem-estimate=128.09MB mem-reservation=34.00MB thread-reservation=1 03:AGGREGATE [FINALIZE] | output: count:merge(*) | group by: string_col @@ -1043,7 +1043,7 @@ Per-Host Resources: mem-estimate=128.07MB mem-reservation=34.00MB thread-reserva | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.01MB thread-reservation=2 +Per-Host Resources: mem-estimate=144.28MB mem-reservation=34.01MB thread-reservation=2 01:AGGREGATE [STREAMING] | output: count(*) | group by: string_col @@ -1063,7 +1063,7 @@ Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.01MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=140.02MB Threads=5 -Per-Host Resource Estimates: Memory=554MB +Per-Host Resource Estimates: Memory=555MB WARNING: The following tables are missing relevant table and/or column statistics. functional_parquet.alltypestiny Analyzed query: SELECT string_col, count(*) FROM functional_parquet.alltypestiny @@ -1095,7 +1095,7 @@ Per-Instance Resources: mem-estimate=128.09MB mem-reservation=34.00MB thread-res | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=4 -Per-Instance Resources: mem-estimate=144.00MB mem-reservation=34.01MB thread-reservation=1 +Per-Instance Resources: mem-estimate=144.38MB mem-reservation=34.01MB thread-reservation=1 01:AGGREGATE [STREAMING] | output: count(*) | group by: string_col diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test index 6e26ce0a0..65cb845ac 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test @@ -172,7 +172,7 @@ PLAN-ROOT SINK in pipelines: 05(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=49.20MB Threads=16 -Per-Host Resource Estimates: Memory=408MB +Per-Host Resource Estimates: Memory=409MB F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -208,7 +208,7 @@ Per-Host Resources: mem-estimate=44.92MB mem-reservation=13.62MB thread-reservat | | in pipelines: 24(GETNEXT) | | | F08:PLAN FRAGMENT [HASH(ctr2.ctr_store_sk)] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.09MB mem-reservation=1.94MB thread-reservation=1 | 24:AGGREGATE [FINALIZE] | | output: avg:merge(ctr_total_return) | | group by: ctr2.ctr_store_sk @@ -243,7 +243,7 @@ Per-Host Resources: mem-estimate=44.92MB mem-reservation=13.62MB thread-reservat | | in pipelines: 06(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=108.95MB mem-reservation=8.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=109.05MB mem-reservation=8.94MB thread-reservation=2 runtime-filters-memory=1.00MB | 09:AGGREGATE [STREAMING] | | output: sum(sr_return_amt) | | group by: sr_customer_sk, sr_store_sk @@ -265,7 +265,7 @@ Per-Host Resources: mem-estimate=44.92MB mem-reservation=13.62MB thread-reservat | | | in pipelines: 07(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT) @@ -304,7 +304,7 @@ Per-Host Resources: mem-estimate=44.92MB mem-reservation=13.62MB thread-reservat | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.09MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB | 04:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_state = 'TN' @@ -347,7 +347,7 @@ Per-Host Resources: mem-estimate=44.92MB mem-reservation=13.62MB thread-reservat | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=110.95MB mem-reservation=10.94MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=111.05MB mem-reservation=10.94MB thread-reservation=2 runtime-filters-memory=3.00MB | 03:AGGREGATE [STREAMING] | | output: sum(sr_return_amt) | | group by: sr_customer_sk, sr_store_sk @@ -369,7 +369,7 @@ Per-Host Resources: mem-estimate=44.92MB mem-reservation=13.62MB thread-reservat | | | in pipelines: 01(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT) @@ -451,7 +451,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 24(GETNEXT) | | | F08:PLAN FRAGMENT [HASH(ctr2.ctr_store_sk)] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.09MB mem-reservation=1.94MB thread-reservation=1 | 24:AGGREGATE [FINALIZE] | | output: avg:merge(ctr_total_return) | | group by: ctr2.ctr_store_sk @@ -487,7 +487,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=34.00MB mem-reservation=6.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=34.11MB mem-reservation=6.00MB thread-reservation=1 | 09:AGGREGATE [STREAMING] | | output: sum(sr_return_amt) | | group by: sr_customer_sk, sr_store_sk @@ -517,7 +517,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | | in pipelines: 07(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT) @@ -565,7 +565,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_state = 'TN' @@ -617,7 +617,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=34.00MB mem-reservation=6.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=34.11MB mem-reservation=6.00MB thread-reservation=1 | 03:AGGREGATE [STREAMING] | | output: sum(sr_return_amt) | | group by: sr_customer_sk, sr_store_sk @@ -647,7 +647,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | | in pipelines: 01(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q02.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q02.test index 18805793f..6beab3f94 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q02.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q02.test @@ -253,7 +253,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=53.69MB Threads=15 -Per-Host Resource Estimates: Memory=432MB +Per-Host Resource Estimates: Memory=433MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.87MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -288,7 +288,7 @@ Per-Host Resources: mem-estimate=21.88MB mem-reservation=13.81MB thread-reservat | | in pipelines: 24(GETNEXT) | | | F10:PLAN FRAGMENT [HASH(d_week_seq)] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=13.70MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=14.48MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB | 15:HASH JOIN [INNER JOIN, PARTITIONED] | | hash predicates: d_week_seq = date_dim.d_week_seq | | fk/pk conjuncts: none @@ -303,7 +303,7 @@ Per-Host Resources: mem-estimate=21.88MB mem-reservation=13.81MB thread-reservat | | | in pipelines: 14(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.14MB mem-reservation=256.00KB thread-reservation=2 | | 14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2002 AS INT) @@ -352,7 +352,7 @@ Per-Host Resources: mem-estimate=21.88MB mem-reservation=13.81MB thread-reservat | | | in pipelines: 11(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=49.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=49.12MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | | 11:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF007[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF006[bloom] -> tpcds_parquet.date_dim.d_week_seq @@ -406,7 +406,7 @@ Per-Host Resources: mem-estimate=21.88MB mem-reservation=13.81MB thread-reservat | | in pipelines: 06(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.14MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB | 06:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT) @@ -456,7 +456,7 @@ Per-Host Resources: mem-estimate=113.67MB mem-reservation=12.75MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=50.00MB mem-reservation=2.50MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=50.12MB mem-reservation=2.50MB thread-reservation=2 runtime-filters-memory=2.00MB | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF001[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF003[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF000[bloom] -> tpcds_parquet.date_dim.d_week_seq, RF002[bloom] -> tpcds_parquet.date_dim.d_week_seq @@ -497,7 +497,7 @@ Per-Host Resources: mem-estimate=113.67MB mem-reservation=12.75MB thread-reserva in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=65.19MB Threads=14 -Per-Host Resource Estimates: Memory=252MB +Per-Host Resource Estimates: Memory=256MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.87MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -540,7 +540,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reser | | in pipelines: 24(GETNEXT) | | | F10:PLAN FRAGMENT [HASH(d_week_seq)] hosts=3 instances=3 -| Per-Instance Resources: mem-estimate=10.74MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=11.55MB mem-reservation=1.94MB thread-reservation=1 | 15:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=01 | | hash predicates: d_week_seq = date_dim.d_week_seq @@ -563,7 +563,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reser | | | in pipelines: 14(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.14MB mem-reservation=256.00KB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2002 AS INT) @@ -591,7 +591,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reser | | | F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=58.00MB mem-reservation=7.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=59.41MB mem-reservation=7.00MB thread-reservation=1 | 13:AGGREGATE [STREAMING] | | output: sum(CASE WHEN (d_day_name = 'Sunday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN sales_price ELSE NULL END) | | group by: d_week_seq @@ -622,7 +622,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reser | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=512.00KB thread-reservation=1 | | 11:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF007[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF006[bloom] -> tpcds_parquet.date_dim.d_week_seq @@ -685,7 +685,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reser | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.14MB mem-reservation=256.00KB thread-reservation=1 | 06:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT) @@ -714,7 +714,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reser | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=58.00MB mem-reservation=7.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=59.41MB mem-reservation=7.00MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(CASE WHEN (d_day_name = 'Sunday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN sales_price ELSE NULL END) | group by: d_week_seq @@ -745,7 +745,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=7.00MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF001[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF003[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF000[bloom] -> tpcds_parquet.date_dim.d_week_seq, RF002[bloom] -> tpcds_parquet.date_dim.d_week_seq diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test index 10d437fc8..66213f4ef 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test @@ -99,7 +99,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=16.56MB Threads=9 -Per-Host Resource Estimates: Memory=175MB +Per-Host Resource Estimates: Memory=177MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -134,7 +134,7 @@ Per-Host Resources: mem-estimate=10.22MB mem-reservation=1.94MB thread-reservati | in pipelines: 01(GETNEXT) | F03:PLAN FRAGMENT [HASH(store_sales.ss_sold_date_sk)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=13.28MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=13.85MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=1.00MB 05:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: dt.d_year, item.i_brand, item.i_brand_id @@ -156,7 +156,7 @@ Per-Host Resources: mem-estimate=13.28MB mem-reservation=4.94MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.19MB mem-reservation=512.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.date_dim dt, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: dt.d_moy = CAST(12 AS INT) @@ -176,7 +176,7 @@ Per-Host Resources: mem-estimate=13.28MB mem-reservation=4.94MB thread-reservati | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=36.74MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=2.00MB 03:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: store_sales.ss_item_sk = item.i_item_sk | fk/pk conjuncts: store_sales.ss_item_sk = item.i_item_sk @@ -191,7 +191,7 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.19MB mem-reservation=256.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: item.i_manufact_id = CAST(436 AS INT) @@ -218,7 +218,7 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=27.38MB Threads=12 -Per-Host Resource Estimates: Memory=123MB +Per-Host Resource Estimates: Memory=129MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -253,7 +253,7 @@ Per-Instance Resources: mem-estimate=10.38MB mem-reservation=1.94MB thread-reser | in pipelines: 01(GETNEXT) | F03:PLAN FRAGMENT [HASH(store_sales.ss_sold_date_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=10.46MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=11.77MB mem-reservation=2.00MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: dt.d_year, item.i_brand, item.i_brand_id @@ -283,7 +283,7 @@ Per-Instance Resources: mem-estimate=10.46MB mem-reservation=2.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.38MB mem-reservation=512.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.date_dim dt, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: dt.d_moy = CAST(12 AS INT) @@ -304,7 +304,7 @@ Per-Instance Resources: mem-estimate=10.46MB mem-reservation=2.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=17.60MB mem-reservation=1.00MB thread-reservation=1 03:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=01 | hash predicates: store_sales.ss_item_sk = item.i_item_sk @@ -327,7 +327,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: item.i_manufact_id = CAST(436 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test index e1af9fb00..8e3b4cdba 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test @@ -559,7 +559,7 @@ PLAN-ROOT SINK in pipelines: 22(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=546.88MB Threads=50 -Per-Host Resource Estimates: Memory=2.60GB +Per-Host Resource Estimates: Memory=2.61GB F37:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -596,7 +596,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | in pipelines: 82(GETNEXT) | | | F36:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1 +| Per-Host Resources: mem-estimate=27.90MB mem-reservation=17.00MB thread-reservation=1 | 35:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=82 row-size=44B cardinality=148.00K @@ -637,7 +637,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | in pipelines: 36(GETNEXT) | | | | | F33:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=129.23MB mem-reservation=8.00MB thread-reservation=2 | | 36:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -654,7 +654,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | in pipelines: 37(GETNEXT) | | | F31:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=194.95MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=195.25MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB | 39:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_sold_date_sk = d_date_sk | | fk/pk conjuncts: ws_sold_date_sk = d_date_sk @@ -669,7 +669,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | in pipelines: 38(GETNEXT) | | | | | F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 38:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -717,7 +717,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | in pipelines: 75(GETNEXT) | | | | | F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | Per-Host Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1 +| | Per-Host Resources: mem-estimate=45.07MB mem-reservation=34.00MB thread-reservation=1 | | 14:UNION | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | | tuple-ids=40 row-size=44B cardinality=29.46K @@ -759,7 +759,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | | in pipelines: 15(GETNEXT) | | | | | | | F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | | Per-Host Resources: mem-estimate=130.84MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | | 15:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -777,7 +777,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | in pipelines: 16(GETNEXT) | | | | | F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | Per-Host Resources: mem-estimate=290.95MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=291.41MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 18:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: cs_sold_date_sk = d_date_sk | | | fk/pk conjuncts: cs_sold_date_sk = d_date_sk @@ -792,7 +792,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | | in pipelines: 17(GETNEXT) | | | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -831,7 +831,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | in pipelines: 68(GETNEXT) | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | Per-Host Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1 +| | Per-Host Resources: mem-estimate=45.07MB mem-reservation=34.00MB thread-reservation=1 | | 00:UNION | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | | tuple-ids=12 row-size=44B cardinality=58.90K @@ -873,7 +873,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | | in pipelines: 01(GETNEXT) | | | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | | Per-Host Resources: mem-estimate=130.84MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | | 01:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -891,7 +891,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | in pipelines: 02(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | Per-Host Resources: mem-estimate=82.95MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=83.41MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 04:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: ss_sold_date_sk = d_date_sk | | | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -906,7 +906,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | | in pipelines: 03(GETNEXT) | | | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -938,7 +938,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | in pipelines: 57(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=48.16MB mem-reservation=36.94MB thread-reservation=1 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=49.14MB mem-reservation=36.94MB thread-reservation=1 runtime-filters-memory=1.00MB | 42:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: customer_id = customer_id | | fk/pk conjuncts: assumed fk/pk @@ -953,7 +953,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | in pipelines: 62(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1 +| | Per-Host Resources: mem-estimate=27.53MB mem-reservation=17.00MB thread-reservation=1 | | 28:UNION | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | | tuple-ids=68 row-size=44B cardinality=14.80K @@ -995,7 +995,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | | in pipelines: 29(GETNEXT) | | | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | | Per-Host Resources: mem-estimate=130.23MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | | 29:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1013,7 +1013,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | in pipelines: 30(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=194.95MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=195.25MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 32:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: ws_sold_date_sk = d_date_sk | | | fk/pk conjuncts: ws_sold_date_sk = d_date_sk @@ -1028,7 +1028,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | | in pipelines: 31(GETNEXT) | | | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -1093,7 +1093,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | in pipelines: 08(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=130.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=131.84MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB | | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF013[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> tpcds_parquet.customer.c_customer_id, RF012[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1111,7 +1111,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | in pipelines: 09(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=82.95MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=83.41MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB | 11:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_sold_date_sk = d_date_sk | | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -1126,7 +1126,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | | | in pipelines: 10(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -1158,7 +1158,7 @@ Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservat | in pipelines: 52(GETNEXT) | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1 +Per-Host Resources: mem-estimate=45.07MB mem-reservation=34.00MB thread-reservation=1 21:UNION | mem-estimate=0B mem-reservation=0B thread-reservation=0 | tuple-ids=54 row-size=44B cardinality=294.63K @@ -1199,7 +1199,7 @@ Per-Host Resources: mem-estimate=39.77MB mem-reservation=26.50MB thread-reservat | | in pipelines: 22(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=130.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=131.84MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB | 22:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1217,7 +1217,7 @@ Per-Host Resources: mem-estimate=39.77MB mem-reservation=26.50MB thread-reservat | in pipelines: 23(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=291.95MB mem-reservation=19.94MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=292.41MB mem-reservation=19.94MB thread-reservation=2 runtime-filters-memory=2.00MB 25:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: cs_sold_date_sk = d_date_sk | fk/pk conjuncts: cs_sold_date_sk = d_date_sk @@ -1232,7 +1232,7 @@ Per-Host Resources: mem-estimate=291.95MB mem-reservation=19.94MB thread-reserva | | in pipelines: 24(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -1258,7 +1258,7 @@ Per-Host Resources: mem-estimate=291.95MB mem-reservation=19.94MB thread-reserva in pipelines: 23(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=644.44MB Threads=57 -Per-Host Resource Estimates: Memory=1.12GB +Per-Host Resource Estimates: Memory=1.16GB F37:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -1303,7 +1303,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | in pipelines: 82(GETNEXT) | | | F36:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=27.90MB mem-reservation=17.00MB thread-reservation=1 | 35:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=82 row-size=44B cardinality=148.00K @@ -1322,7 +1322,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | in pipelines: 37(GETNEXT) | | | F34:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=19.34MB mem-reservation=17.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=20.69MB mem-reservation=17.00MB thread-reservation=1 | 41:AGGREGATE [STREAMING] | | output: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))) | | group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year @@ -1352,7 +1352,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | in pipelines: 36(GETNEXT) | | | | | F33:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=17.23MB mem-reservation=8.00MB thread-reservation=1 | | 36:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -1370,7 +1370,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | F31:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.31MB mem-reservation=16.00MB thread-reservation=1 | 39:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=02 | | hash predicates: ws_sold_date_sk = d_date_sk @@ -1393,7 +1393,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | in pipelines: 38(GETNEXT) | | | | | F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 38:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -1457,7 +1457,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | in pipelines: 75(GETNEXT) | | | | | F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | Per-Instance Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=45.07MB mem-reservation=34.00MB thread-reservation=1 | | 14:UNION | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | | tuple-ids=40 row-size=44B cardinality=29.46K @@ -1477,7 +1477,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | in pipelines: 16(GETNEXT) | | | | | F28:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3 -| | Per-Instance Resources: mem-estimate=20.11MB mem-reservation=17.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=22.14MB mem-reservation=17.00MB thread-reservation=1 | | 20:AGGREGATE [STREAMING] | | | output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))) | | | group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year @@ -1508,7 +1508,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | | | | | F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=17.84MB mem-reservation=8.00MB thread-reservation=1 | | | 15:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1527,7 +1527,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | | | F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=48.47MB mem-reservation=16.00MB thread-reservation=1 | | 18:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=06 | | | hash predicates: cs_sold_date_sk = d_date_sk @@ -1550,7 +1550,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | | in pipelines: 17(GETNEXT) | | | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -1597,7 +1597,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | in pipelines: 68(GETNEXT) | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| | Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=45.58MB mem-reservation=34.00MB thread-reservation=1 | | 00:UNION | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | | tuple-ids=12 row-size=44B cardinality=58.90K @@ -1617,7 +1617,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | in pipelines: 02(GETNEXT) | | | | | F21:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -| | Per-Instance Resources: mem-estimate=23.23MB mem-reservation=17.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=27.28MB mem-reservation=17.00MB thread-reservation=1 | | 06:AGGREGATE [STREAMING] | | | output: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / CAST(2 AS DECIMAL(3,0))) | | | group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year @@ -1648,7 +1648,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=19.68MB mem-reservation=8.00MB thread-reservation=1 | | | 01:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1667,7 +1667,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.94MB mem-reservation=4.00MB thread-reservation=1 | | 04:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=09 | | | hash predicates: ss_sold_date_sk = d_date_sk @@ -1690,7 +1690,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | | in pipelines: 03(GETNEXT) | | | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -1722,7 +1722,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | in pipelines: 57(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=46.71MB mem-reservation=34.00MB thread-reservation=1 | 42:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=10 | | hash predicates: customer_id = customer_id @@ -1745,7 +1745,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | in pipelines: 62(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Instance Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=27.53MB mem-reservation=17.00MB thread-reservation=1 | | 28:UNION | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | | tuple-ids=68 row-size=44B cardinality=14.80K @@ -1765,7 +1765,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | in pipelines: 30(GETNEXT) | | | | | F15:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2 -| | Per-Instance Resources: mem-estimate=19.34MB mem-reservation=17.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=20.69MB mem-reservation=17.00MB thread-reservation=1 | | 34:AGGREGATE [STREAMING] | | | output: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))) | | | group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year @@ -1796,7 +1796,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=17.23MB mem-reservation=8.00MB thread-reservation=1 | | | 29:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1815,7 +1815,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=32.31MB mem-reservation=16.00MB thread-reservation=1 | | 32:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=12 | | | hash predicates: ws_sold_date_sk = d_date_sk @@ -1838,7 +1838,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | | in pipelines: 31(GETNEXT) | | | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -1881,7 +1881,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | in pipelines: 09(GETNEXT) | | | F09:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=23.23MB mem-reservation=17.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=27.28MB mem-reservation=17.00MB thread-reservation=1 | 13:AGGREGATE [STREAMING] | | output: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / CAST(2 AS DECIMAL(3,0))) | | group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year @@ -1912,7 +1912,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=19.68MB mem-reservation=8.00MB thread-reservation=1 | | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF013[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> tpcds_parquet.customer.c_customer_id, RF012[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1931,7 +1931,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.94MB mem-reservation=4.00MB thread-reservation=1 | 11:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=14 | | hash predicates: ss_sold_date_sk = d_date_sk @@ -1954,7 +1954,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | | | in pipelines: 10(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -1986,7 +1986,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio | in pipelines: 52(GETNEXT) | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=45.07MB mem-reservation=34.00MB thread-reservation=1 21:UNION | mem-estimate=0B mem-reservation=0B thread-reservation=0 | tuple-ids=54 row-size=44B cardinality=294.63K @@ -2005,7 +2005,7 @@ Per-Instance Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-rese | in pipelines: 23(GETNEXT) | F03:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=20.11MB mem-reservation=17.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=22.14MB mem-reservation=17.00MB thread-reservation=1 27:AGGREGATE [STREAMING] | output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))) | group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year @@ -2036,7 +2036,7 @@ Per-Instance Resources: mem-estimate=20.11MB mem-reservation=17.00MB thread-rese | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.84MB mem-reservation=8.00MB thread-reservation=1 | 22:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -2055,7 +2055,7 @@ Per-Instance Resources: mem-estimate=20.11MB mem-reservation=17.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=48.47MB mem-reservation=16.00MB thread-reservation=1 25:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=16 | hash predicates: cs_sold_date_sk = d_date_sk @@ -2078,7 +2078,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-rese | | in pipelines: 24(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test index 64ca9d5f1..43187aafd 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test @@ -405,7 +405,7 @@ PLAN-ROOT SINK in pipelines: 02(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=79.59MB Threads=25 -Per-Host Resource Estimates: Memory=923MB +Per-Host Resource Estimates: Memory=930MB F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -454,7 +454,7 @@ Per-Host Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reservati | in pipelines: 33(GETNEXT), 37(GETNEXT), 43(GETNEXT) | F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=43.29MB mem-reservation=7.94MB thread-reservation=1 27:AGGREGATE [STREAMING] | Class 0 | output: sum(sales), sum(returns), sum(profit) @@ -487,7 +487,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati | | in pipelines: 18(GETNEXT), 20(GETNEXT) | | | F16:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=146.10MB mem-reservation=17.88MB thread-reservation=2 runtime-filters-memory=4.00MB +| Per-Host Resources: mem-estimate=147.00MB mem-reservation=17.88MB thread-reservation=2 runtime-filters-memory=4.00MB | 26:AGGREGATE [STREAMING] | | output: sum(sales_price), sum(profit), sum(return_amt), sum(net_loss) | | group by: web_site_id @@ -509,7 +509,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 22(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | 22:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -537,7 +537,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 23(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.14MB mem-reservation=16.00KB thread-reservation=2 | | 23:SCAN HDFS [tpcds_parquet.web_site, RANDOM] | | HDFS partitions=1/1 files=1 size=11.91KB | | stored statistics: @@ -567,7 +567,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati | | | | in pipelines: 19(GETNEXT) | | | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=81.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | | Per-Host Resources: mem-estimate=81.25MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | | 19:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.66MB | | | runtime filters: RF009[min_max] -> tpcds_parquet.web_returns.wr_returned_date_sk, RF008[bloom] -> tpcds_parquet.web_returns.wr_returned_date_sk @@ -585,7 +585,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 20(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=99.00MB mem-reservation=7.00MB thread-reservation=2 runtime-filters-memory=3.00MB +| | Per-Host Resources: mem-estimate=99.19MB mem-reservation=7.00MB thread-reservation=2 runtime-filters-memory=3.00MB | | 20:SCAN HDFS [tpcds_parquet.web_sales, RANDOM] | | HDFS partitions=1/1 files=2 size=45.09MB | | runtime filters: RF014[min_max] -> ws_item_sk, RF015[min_max] -> ws_order_number, RF011[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF012[bloom] -> ws_item_sk, RF013[bloom] -> ws_order_number, RF010[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk @@ -621,7 +621,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati | | in pipelines: 10(GETNEXT), 11(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=208.48MB mem-reservation=16.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=209.00MB mem-reservation=16.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 16:AGGREGATE [STREAMING] | | output: sum(sales_price), sum(profit), sum(return_amt), sum(net_loss) | | group by: cp_catalog_page_id @@ -643,7 +643,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 12(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | 12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -671,7 +671,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 13(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.14MB mem-reservation=256.00KB thread-reservation=2 | | 13:SCAN HDFS [tpcds_parquet.catalog_page, RANDOM] | | HDFS partitions=1/1 files=1 size=739.17KB | | stored statistics: @@ -722,7 +722,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati | in pipelines: 02(GETNEXT), 03(GETNEXT) | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=112.10MB mem-reservation=11.88MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=113.00MB mem-reservation=11.88MB thread-reservation=2 runtime-filters-memory=2.00MB 08:AGGREGATE [STREAMING] | output: sum(sales_price), sum(profit), sum(return_amt), sum(net_loss) | group by: s_store_id @@ -744,7 +744,7 @@ Per-Host Resources: mem-estimate=112.10MB mem-reservation=11.88MB thread-reserva | | in pipelines: 05(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.14MB mem-reservation=16.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -769,7 +769,7 @@ Per-Host Resources: mem-estimate=112.10MB mem-reservation=11.88MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -812,7 +812,7 @@ Per-Host Resources: mem-estimate=112.10MB mem-reservation=11.88MB thread-reserva in pipelines: 02(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=121.78MB Threads=24 -Per-Host Resource Estimates: Memory=524MB +Per-Host Resource Estimates: Memory=545MB F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -861,7 +861,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reser | in pipelines: 33(GETNEXT), 37(GETNEXT), 43(GETNEXT) | F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=45.96MB mem-reservation=7.94MB thread-reservation=1 27:AGGREGATE [STREAMING] | Class 0 | output: sum(sales), sum(returns), sum(profit) @@ -895,7 +895,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser | | | F16:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=42.00MB mem-reservation=10.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=44.25MB mem-reservation=10.00MB thread-reservation=1 | 26:AGGREGATE [STREAMING] | | output: sum(sales_price), sum(profit), sum(return_amt), sum(net_loss) | | group by: web_site_id @@ -925,7 +925,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser | | | in pipelines: 22(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 22:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -961,7 +961,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser | | | in pipelines: 23(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1 | | 23:SCAN HDFS [tpcds_parquet.web_site, RANDOM] | | HDFS partitions=1/1 files=1 size=11.91KB | | stored statistics: @@ -1000,7 +1000,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser | | | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.25MB mem-reservation=2.00MB thread-reservation=1 | | | 19:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.66MB | | | runtime filters: RF009[min_max] -> tpcds_parquet.web_returns.wr_returned_date_sk, RF008[bloom] -> tpcds_parquet.web_returns.wr_returned_date_sk @@ -1019,7 +1019,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| | Per-Instance Resources: mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=32.19MB mem-reservation=4.00MB thread-reservation=1 | | 20:SCAN HDFS [tpcds_parquet.web_sales, RANDOM] | | HDFS partitions=1/1 files=2 size=45.09MB | | runtime filters: RF014[min_max] -> ws_item_sk, RF015[min_max] -> ws_order_number, RF011[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF012[bloom] -> ws_item_sk, RF013[bloom] -> ws_order_number, RF010[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk @@ -1056,7 +1056,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser | | | F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=58.00MB mem-reservation=11.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=60.25MB mem-reservation=11.00MB thread-reservation=1 | 16:AGGREGATE [STREAMING] | | output: sum(sales_price), sum(profit), sum(return_amt), sum(net_loss) | | group by: cp_catalog_page_id @@ -1086,7 +1086,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser | | | in pipelines: 12(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -1122,7 +1122,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser | | | in pipelines: 13(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.14MB mem-reservation=256.00KB thread-reservation=1 | | 13:SCAN HDFS [tpcds_parquet.catalog_page, RANDOM] | | HDFS partitions=1/1 files=1 size=739.17KB | | stored statistics: @@ -1174,7 +1174,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=34.00MB mem-reservation=6.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=36.25MB mem-reservation=6.00MB thread-reservation=1 08:AGGREGATE [STREAMING] | output: sum(sales_price), sum(profit), sum(return_amt), sum(net_loss) | group by: s_store_id @@ -1204,7 +1204,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -1237,7 +1237,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q06.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q06.test index b1252f681..33b24c0eb 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q06.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q06.test @@ -187,7 +187,7 @@ PLAN-ROOT SINK in pipelines: 02(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=48.25MB Threads=19 -Per-Host Resource Estimates: Memory=352MB +Per-Host Resource Estimates: Memory=353MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -245,7 +245,7 @@ Per-Host Resources: mem-estimate=70.86MB mem-reservation=29.06MB thread-reservat | | in pipelines: 26(GETNEXT) | | | F09:PLAN FRAGMENT [HASH(j.i_category)] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.12MB mem-reservation=1.94MB thread-reservation=1 | 26:AGGREGATE [FINALIZE] | | output: avg:merge(j.i_current_price) | | group by: j.i_category @@ -259,7 +259,7 @@ Per-Host Resources: mem-estimate=70.86MB mem-reservation=29.06MB thread-reservat | | in pipelines: 07(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=42.00MB mem-reservation=2.12MB thread-reservation=2 +| Per-Host Resources: mem-estimate=42.12MB mem-reservation=2.12MB thread-reservation=2 | 08:AGGREGATE [STREAMING] | | output: avg(j.i_current_price) | | group by: j.i_category @@ -290,7 +290,7 @@ Per-Host Resources: mem-estimate=70.86MB mem-reservation=29.06MB thread-reservat | | in pipelines: 22(GETNEXT) | | | F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=48.00KB mem-reservation=0B thread-reservation=1 | 23:EXCHANGE [UNPARTITIONED] | | limit: 1 | | mem-estimate=16.00KB mem-reservation=0B thread-reservation=0 @@ -298,7 +298,7 @@ Per-Host Resources: mem-estimate=70.86MB mem-reservation=29.06MB thread-reservat | | in pipelines: 22(GETNEXT) | | | F06:PLAN FRAGMENT [HASH((d_month_seq))] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.03MB mem-reservation=1.94MB thread-reservation=1 | 22:AGGREGATE [FINALIZE] | | group by: (d_month_seq) | | limit: 1 @@ -312,7 +312,7 @@ Per-Host Resources: mem-estimate=70.86MB mem-reservation=29.06MB thread-reservat | | in pipelines: 05(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=58.00MB mem-reservation=2.25MB thread-reservation=2 +| Per-Host Resources: mem-estimate=58.03MB mem-reservation=2.25MB thread-reservation=2 | 06:AGGREGATE [STREAMING] | | group by: (d_month_seq) | | limit: 1 @@ -347,7 +347,7 @@ Per-Host Resources: mem-estimate=70.86MB mem-reservation=29.06MB thread-reservat | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=49.00MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=49.13MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB | 04:SCAN HDFS [tpcds_parquet.item i, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> i.i_category, RF000[bloom] -> i.i_category @@ -373,7 +373,7 @@ Per-Host Resources: mem-estimate=70.86MB mem-reservation=29.06MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.05MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | 03:SCAN HDFS [tpcds_parquet.date_dim d, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF003[min_max] -> d.d_month_seq, RF002[bloom] -> d.d_month_seq @@ -399,7 +399,7 @@ Per-Host Resources: mem-estimate=70.86MB mem-reservation=29.06MB thread-reservat | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.09MB mem-reservation=256.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.customer_address a, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -424,7 +424,7 @@ Per-Host Resources: mem-estimate=70.86MB mem-reservation=29.06MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.05MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 01:SCAN HDFS [tpcds_parquet.customer c, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF009[min_max] -> c.c_current_addr_sk, RF008[bloom] -> c.c_current_addr_sk @@ -449,7 +449,7 @@ Per-Host Resources: mem-estimate=70.86MB mem-reservation=29.06MB thread-reservat in pipelines: 02(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=76.25MB Threads=20 -Per-Host Resource Estimates: Memory=267MB +Per-Host Resource Estimates: Memory=269MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -486,7 +486,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.61MB mem-reservation=3.00MB thread-reservation=1 15:AGGREGATE [STREAMING] | output: count(*) | group by: a.ca_state @@ -516,7 +516,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 26(GETNEXT) | | | F09:PLAN FRAGMENT [HASH(j.i_category)] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.12MB mem-reservation=1.94MB thread-reservation=1 | 26:AGGREGATE [FINALIZE] | | output: avg:merge(j.i_current_price) | | group by: j.i_category @@ -530,7 +530,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 07(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.12MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.12MB mem-reservation=2.12MB thread-reservation=1 | 08:AGGREGATE [STREAMING] | | output: avg(j.i_current_price) | | group by: j.i_category @@ -569,7 +569,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 22(GETNEXT) | | | F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=48.00KB mem-reservation=0B thread-reservation=1 | 23:EXCHANGE [UNPARTITIONED] | | limit: 1 | | mem-estimate=16.00KB mem-reservation=0B thread-reservation=0 @@ -577,7 +577,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 22(GETNEXT) | | | F06:PLAN FRAGMENT [HASH((d_month_seq))] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.03MB mem-reservation=1.94MB thread-reservation=1 | 22:AGGREGATE [FINALIZE] | | group by: (d_month_seq) | | limit: 1 @@ -591,7 +591,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.25MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.03MB mem-reservation=2.25MB thread-reservation=1 | 06:AGGREGATE [STREAMING] | | group by: (d_month_seq) | | limit: 1 @@ -635,7 +635,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.13MB mem-reservation=256.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.item i, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> i.i_category, RF000[bloom] -> i.i_category @@ -670,7 +670,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim d, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF003[min_max] -> d.d_month_seq, RF002[bloom] -> d.d_month_seq @@ -704,7 +704,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.customer_address a, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -738,7 +738,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=1.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.customer c, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF009[min_max] -> c.c_current_addr_sk, RF008[bloom] -> c.c_current_addr_sk diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test index 32254b569..ab0ac7aee 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test @@ -141,7 +141,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=39.28MB Threads=12 -Per-Host Resource Estimates: Memory=336MB +Per-Host Resource Estimates: Memory=337MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -198,7 +198,7 @@ Per-Host Resources: mem-estimate=145.85MB mem-reservation=24.31MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.16MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -223,7 +223,7 @@ Per-Host Resources: mem-estimate=145.85MB mem-reservation=24.31MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=32.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.13MB mem-reservation=32.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | predicates: (p_channel_email = 'N' OR p_channel_event = 'N') @@ -249,7 +249,7 @@ Per-Host Resources: mem-estimate=145.85MB mem-reservation=24.31MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.22MB mem-reservation=8.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'W', cd_gender = 'F', cd_education_status = 'Primary' @@ -277,7 +277,7 @@ Per-Host Resources: mem-estimate=145.85MB mem-reservation=24.31MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT) @@ -304,7 +304,7 @@ Per-Host Resources: mem-estimate=145.85MB mem-reservation=24.31MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=65.53MB Threads=13 -Per-Host Resource Estimates: Memory=183MB +Per-Host Resource Estimates: Memory=187MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -340,7 +340,7 @@ Per-Instance Resources: mem-estimate=10.54MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=27.50MB mem-reservation=6.00MB thread-reservation=1 09:AGGREGATE [STREAMING] | output: avg(CAST(ss_quantity AS BIGINT)), avg(ss_list_price), avg(ss_coupon_amt), avg(ss_sales_price) | group by: i_item_id @@ -370,7 +370,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -403,7 +403,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.13MB mem-reservation=32.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | predicates: (p_channel_email = 'N' OR p_channel_event = 'N') @@ -437,7 +437,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.22MB mem-reservation=8.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'W', cd_gender = 'F', cd_education_status = 'Primary' @@ -473,7 +473,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test index dfe3d66a7..0d6e55e92 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test @@ -224,7 +224,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=35.96MB Threads=15 -Per-Host Resource Estimates: Memory=290MB +Per-Host Resource Estimates: Memory=291MB F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -259,7 +259,7 @@ Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=50.86MB mem-reservation=11.81MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=51.24MB mem-reservation=11.81MB thread-reservation=2 runtime-filters-memory=3.00MB 13:AGGREGATE [STREAMING] | output: sum(ss_net_profit) | group by: s_store_name @@ -324,7 +324,7 @@ Per-Host Resources: mem-estimate=50.86MB mem-reservation=11.81MB thread-reservat | | | | in pipelines: 06(GETNEXT) | | | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.08MB mem-reservation=512.00KB thread-reservation=2 | | | 06:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | predicates: c_preferred_cust_flag = 'Y' @@ -361,7 +361,7 @@ Per-Host Resources: mem-estimate=50.86MB mem-reservation=11.81MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=26.00MB mem-reservation=2.12MB thread-reservation=2 +| Per-Host Resources: mem-estimate=26.06MB mem-reservation=2.12MB thread-reservation=2 | 04:AGGREGATE [STREAMING] | | group by: substring(ca_zip, CAST(1 AS BIGINT), CAST(5 AS BIGINT)) | | mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0 @@ -394,7 +394,7 @@ Per-Host Resources: mem-estimate=50.86MB mem-reservation=11.81MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=49.00MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=49.16MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | runtime filters: RF000[bloom] -> substring(s_zip, 1, 2) @@ -420,7 +420,7 @@ Per-Host Resources: mem-estimate=50.86MB mem-reservation=11.81MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT), d_qoy = CAST(2 AS INT) @@ -447,7 +447,7 @@ Per-Host Resources: mem-estimate=50.86MB mem-reservation=11.81MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=54.46MB Threads=16 -Per-Host Resource Estimates: Memory=228MB +Per-Host Resource Estimates: Memory=230MB F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -483,7 +483,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.85MB mem-reservation=3.00MB thread-reservation=1 13:AGGREGATE [STREAMING] | output: sum(ss_net_profit) | group by: s_store_name @@ -543,7 +543,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.50MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=26.09MB mem-reservation=2.50MB thread-reservation=1 | | 08:AGGREGATE [STREAMING] | | | output: count(*) | | | group by: substring(ca_zip, CAST(1 AS BIGINT), CAST(5 AS BIGINT)) @@ -573,7 +573,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | in pipelines: 06(GETNEXT) | | | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.08MB mem-reservation=512.00KB thread-reservation=1 | | | 06:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | predicates: c_preferred_cust_flag = 'Y' @@ -610,7 +610,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.12MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.06MB mem-reservation=2.12MB thread-reservation=1 | 04:AGGREGATE [STREAMING] | | group by: substring(ca_zip, CAST(1 AS BIGINT), CAST(5 AS BIGINT)) | | mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0 @@ -652,7 +652,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.16MB mem-reservation=24.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | runtime filters: RF000[bloom] -> substring(s_zip, 1, 2) @@ -686,7 +686,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT), d_qoy = CAST(2 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q09.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q09.test index 5692647fc..23f759df3 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q09.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q09.test @@ -462,7 +462,7 @@ PLAN-ROOT SINK | | in pipelines: 89(GETNEXT) | | | F30:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 89:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -507,7 +507,7 @@ PLAN-ROOT SINK | | in pipelines: 86(GETNEXT) | | | F28:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 86:AGGREGATE [FINALIZE] | | output: avg:merge(ss_ext_discount_amt) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -552,7 +552,7 @@ PLAN-ROOT SINK | | in pipelines: 83(GETNEXT) | | | F26:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 83:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -597,7 +597,7 @@ PLAN-ROOT SINK | | in pipelines: 80(GETNEXT) | | | F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 80:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -642,7 +642,7 @@ PLAN-ROOT SINK | | in pipelines: 77(GETNEXT) | | | F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 77:AGGREGATE [FINALIZE] | | output: avg:merge(ss_ext_discount_amt) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -687,7 +687,7 @@ PLAN-ROOT SINK | | in pipelines: 74(GETNEXT) | | | F20:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 74:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -732,7 +732,7 @@ PLAN-ROOT SINK | | in pipelines: 71(GETNEXT) | | | F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 71:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -777,7 +777,7 @@ PLAN-ROOT SINK | | in pipelines: 68(GETNEXT) | | | F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 68:AGGREGATE [FINALIZE] | | output: avg:merge(ss_ext_discount_amt) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -822,7 +822,7 @@ PLAN-ROOT SINK | | in pipelines: 65(GETNEXT) | | | F14:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 65:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -867,7 +867,7 @@ PLAN-ROOT SINK | | in pipelines: 62(GETNEXT) | | | F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 62:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -912,7 +912,7 @@ PLAN-ROOT SINK | | in pipelines: 59(GETNEXT) | | | F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 59:AGGREGATE [FINALIZE] | | output: avg:merge(ss_ext_discount_amt) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -957,7 +957,7 @@ PLAN-ROOT SINK | | in pipelines: 56(GETNEXT) | | | F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 56:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1002,7 +1002,7 @@ PLAN-ROOT SINK | | in pipelines: 53(GETNEXT) | | | F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 53:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1047,7 +1047,7 @@ PLAN-ROOT SINK | | in pipelines: 50(GETNEXT) | | | F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 50:AGGREGATE [FINALIZE] | | output: avg:merge(ss_ext_discount_amt) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1068,7 +1068,7 @@ PLAN-ROOT SINK | | in pipelines: 04(GETNEXT), 03(OPEN) | | | 03:SCAN HDFS [tpcds_parquet.store_sales, RANDOM] -| HDFS partitions=1824/1824 files=1824 size=200.96MB +| HDFS partitions=1824/1824 files=1824 size=199.88MB | predicates: ss_quantity <= CAST(20 AS INT), ss_quantity >= CAST(1 AS INT) | stored statistics: | table: rows=2.88M size=200.96MB @@ -1092,7 +1092,7 @@ PLAN-ROOT SINK | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.reason, RANDOM] | HDFS partitions=1/1 files=1 size=1.92KB | predicates: r_reason_sk = CAST(1 AS INT) @@ -1140,7 +1140,7 @@ Per-Host Resources: mem-estimate=16.02MB mem-reservation=128.00KB thread-reserva in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=25.26MB Threads=61 -Per-Host Resource Estimates: Memory=501MB +Per-Host Resource Estimates: Memory=502MB F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -1165,7 +1165,7 @@ PLAN-ROOT SINK | | in pipelines: 89(GETNEXT) | | | F30:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 89:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1217,7 +1217,7 @@ PLAN-ROOT SINK | | in pipelines: 86(GETNEXT) | | | F28:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 86:AGGREGATE [FINALIZE] | | output: avg:merge(ss_ext_discount_amt) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1269,7 +1269,7 @@ PLAN-ROOT SINK | | in pipelines: 83(GETNEXT) | | | F26:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 83:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1321,7 +1321,7 @@ PLAN-ROOT SINK | | in pipelines: 80(GETNEXT) | | | F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 80:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1373,7 +1373,7 @@ PLAN-ROOT SINK | | in pipelines: 77(GETNEXT) | | | F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 77:AGGREGATE [FINALIZE] | | output: avg:merge(ss_ext_discount_amt) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1425,7 +1425,7 @@ PLAN-ROOT SINK | | in pipelines: 74(GETNEXT) | | | F20:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 74:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1477,7 +1477,7 @@ PLAN-ROOT SINK | | in pipelines: 71(GETNEXT) | | | F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 71:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1529,7 +1529,7 @@ PLAN-ROOT SINK | | in pipelines: 68(GETNEXT) | | | F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 68:AGGREGATE [FINALIZE] | | output: avg:merge(ss_ext_discount_amt) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1581,7 +1581,7 @@ PLAN-ROOT SINK | | in pipelines: 65(GETNEXT) | | | F14:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 65:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1633,7 +1633,7 @@ PLAN-ROOT SINK | | in pipelines: 62(GETNEXT) | | | F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 62:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1685,7 +1685,7 @@ PLAN-ROOT SINK | | in pipelines: 59(GETNEXT) | | | F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 59:AGGREGATE [FINALIZE] | | output: avg:merge(ss_ext_discount_amt) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1737,7 +1737,7 @@ PLAN-ROOT SINK | | in pipelines: 56(GETNEXT) | | | F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 56:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1789,7 +1789,7 @@ PLAN-ROOT SINK | | in pipelines: 53(GETNEXT) | | | F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 53:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1841,7 +1841,7 @@ PLAN-ROOT SINK | | in pipelines: 50(GETNEXT) | | | F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 50:AGGREGATE [FINALIZE] | | output: avg:merge(ss_ext_discount_amt) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1893,7 +1893,7 @@ PLAN-ROOT SINK | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.reason, RANDOM] | HDFS partitions=1/1 files=1 size=1.92KB | predicates: r_reason_sk = CAST(1 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test index 99f1487b4..e0e47ee32 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test @@ -248,7 +248,7 @@ PLAN-ROOT SINK in pipelines: 07(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=52.62MB Threads=19 -Per-Host Resource Estimates: Memory=519MB +Per-Host Resource Estimates: Memory=520MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -311,7 +311,7 @@ Per-Host Resources: mem-estimate=17.34MB mem-reservation=7.88MB thread-reservati | | | in pipelines: 02(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=141.61MB mem-reservation=20.44MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=142.08MB mem-reservation=20.44MB thread-reservation=2 runtime-filters-memory=2.00MB | | 14:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: c.c_current_addr_sk = ca.ca_address_sk | | | fk/pk conjuncts: c.c_current_addr_sk = ca.ca_address_sk @@ -326,7 +326,7 @@ Per-Host Resources: mem-estimate=17.34MB mem-reservation=7.88MB thread-reservati | | | | in pipelines: 01(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.13MB mem-reservation=512.00KB thread-reservation=2 | | | 01:SCAN HDFS [tpcds_parquet.customer_address ca, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.16MB | | | predicates: ca_county IN ('Rush County', 'Toole County', 'Jefferson County', 'Dona Ana County', 'La Porte County') @@ -354,7 +354,7 @@ Per-Host Resources: mem-estimate=17.34MB mem-reservation=7.88MB thread-reservati | | | | in pipelines: 00(GETNEXT) | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=49.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | | Per-Host Resources: mem-estimate=49.06MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | | 00:SCAN HDFS [tpcds_parquet.customer c, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF011[min_max] -> c.c_current_addr_sk, RF010[bloom] -> c.c_current_addr_sk @@ -383,7 +383,7 @@ Per-Host Resources: mem-estimate=17.34MB mem-reservation=7.88MB thread-reservati | | in pipelines: 03(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=19.95MB mem-reservation=4.44MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=20.27MB mem-reservation=4.44MB thread-reservation=2 runtime-filters-memory=2.00MB | 05:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_sold_date_sk = d_date_sk | | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -398,7 +398,7 @@ Per-Host Resources: mem-estimate=17.34MB mem-reservation=7.88MB thread-reservati | | | in pipelines: 04(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2002 AS INT), d_moy <= CAST(4 AS INT), d_moy >= CAST(1 AS INT) @@ -430,7 +430,7 @@ Per-Host Resources: mem-estimate=17.34MB mem-reservation=7.88MB thread-reservati | in pipelines: 07(GETNEXT), 10(GETNEXT) | F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=100.95MB mem-reservation=8.94MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=101.05MB mem-reservation=8.94MB thread-reservation=2 runtime-filters-memory=3.00MB 06:UNION | mem-estimate=0B mem-reservation=0B thread-reservation=0 | tuple-ids=10 row-size=4B cardinality=140.03K @@ -450,7 +450,7 @@ Per-Host Resources: mem-estimate=100.95MB mem-reservation=8.94MB thread-reservat | | | in pipelines: 11(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 11:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2002 AS INT), d_moy <= CAST(4 AS INT), d_moy >= CAST(1 AS INT) @@ -489,7 +489,7 @@ Per-Host Resources: mem-estimate=100.95MB mem-reservation=8.94MB thread-reservat | | in pipelines: 08(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2002 AS INT), d_moy <= CAST(4 AS INT), d_moy >= CAST(1 AS INT) @@ -515,7 +515,7 @@ Per-Host Resources: mem-estimate=100.95MB mem-reservation=8.94MB thread-reservat in pipelines: 07(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=75.31MB Threads=19 -Per-Host Resource Estimates: Memory=253MB +Per-Host Resource Estimates: Memory=257MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -550,7 +550,7 @@ Per-Instance Resources: mem-estimate=10.10MB mem-reservation=1.94MB thread-reser | in pipelines: 07(GETNEXT), 10(GETNEXT) | F10:PLAN FRAGMENT [HASH(c.c_customer_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=10.20MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=11.31MB mem-reservation=2.00MB thread-reservation=1 17:AGGREGATE [STREAMING] | output: count(*) | group by: cd_gender, cd_marital_status, cd_education_status, cd_purchase_estimate, cd_credit_rating, cd_dep_count, cd_dep_employed_count, cd_dep_college_count @@ -595,7 +595,7 @@ Per-Instance Resources: mem-estimate=10.20MB mem-reservation=2.00MB thread-reser | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=17.65MB mem-reservation=8.00MB thread-reservation=1 | | 14:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=02 | | | hash predicates: c.c_current_addr_sk = ca.ca_address_sk @@ -618,7 +618,7 @@ Per-Instance Resources: mem-estimate=10.20MB mem-reservation=2.00MB thread-reser | | | | in pipelines: 01(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.13MB mem-reservation=512.00KB thread-reservation=1 | | | 01:SCAN HDFS [tpcds_parquet.customer_address ca, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.16MB | | | predicates: ca_county IN ('Rush County', 'Toole County', 'Jefferson County', 'Dona Ana County', 'La Porte County') @@ -655,7 +655,7 @@ Per-Instance Resources: mem-estimate=10.20MB mem-reservation=2.00MB thread-reser | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=2.00MB thread-reservation=1 | | | 00:SCAN HDFS [tpcds_parquet.customer c, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF011[min_max] -> c.c_current_addr_sk, RF010[bloom] -> c.c_current_addr_sk @@ -685,7 +685,7 @@ Per-Instance Resources: mem-estimate=10.20MB mem-reservation=2.00MB thread-reser | | | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.33MB mem-reservation=512.00KB thread-reservation=1 | 05:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=04 | | hash predicates: ss_sold_date_sk = d_date_sk @@ -708,7 +708,7 @@ Per-Instance Resources: mem-estimate=10.20MB mem-reservation=2.00MB thread-reser | | | in pipelines: 04(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2002 AS INT), d_moy <= CAST(4 AS INT), d_moy >= CAST(1 AS INT) @@ -741,7 +741,7 @@ Per-Instance Resources: mem-estimate=10.20MB mem-reservation=2.00MB thread-reser | F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=48.09MB mem-reservation=4.00MB thread-reservation=1 06:UNION | mem-estimate=0B mem-reservation=0B thread-reservation=0 | tuple-ids=10 row-size=4B cardinality=140.03K @@ -769,7 +769,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reser | | | in pipelines: 11(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 11:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2002 AS INT), d_moy <= CAST(4 AS INT), d_moy >= CAST(1 AS INT) @@ -816,7 +816,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 08(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2002 AS INT), d_moy <= CAST(4 AS INT), d_moy >= CAST(1 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test index 7faaf0db6..52546e6ea 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test @@ -376,7 +376,7 @@ PLAN-ROOT SINK in pipelines: 08(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=351.31MB Threads=34 -Per-Host Resource Estimates: Memory=1.37GB +Per-Host Resource Estimates: Memory=1.38GB F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -413,7 +413,7 @@ Per-Host Resources: mem-estimate=16.11MB mem-reservation=9.62MB thread-reservati | | in pipelines: 54(GETNEXT) | | | F24:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1 +| Per-Host Resources: mem-estimate=27.90MB mem-reservation=17.00MB thread-reservation=1 | 21:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=38 row-size=44B cardinality=148.00K @@ -454,7 +454,7 @@ Per-Host Resources: mem-estimate=16.11MB mem-reservation=9.62MB thread-reservati | | | in pipelines: 24(GETNEXT) | | | | | F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -482,7 +482,7 @@ Per-Host Resources: mem-estimate=16.11MB mem-reservation=9.62MB thread-reservati | | | in pipelines: 22(GETNEXT) | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=129.23MB mem-reservation=8.00MB thread-reservation=2 | | 22:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -499,7 +499,7 @@ Per-Host Resources: mem-estimate=16.11MB mem-reservation=9.62MB thread-reservati | | in pipelines: 23(GETNEXT) | | | F19:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=129.16MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 23:SCAN HDFS [tpcds_parquet.web_sales, RANDOM] | HDFS partitions=1/1 files=2 size=45.09MB | runtime filters: RF019[min_max] -> ws_sold_date_sk, RF021[min_max] -> ws_bill_customer_sk, RF018[bloom] -> ws_sold_date_sk @@ -525,7 +525,7 @@ Per-Host Resources: mem-estimate=16.11MB mem-reservation=9.62MB thread-reservati | | in pipelines: 47(GETNEXT) | | | F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1 +| Per-Host Resources: mem-estimate=45.07MB mem-reservation=34.00MB thread-reservation=1 | 00:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=8 row-size=44B cardinality=58.90K @@ -567,7 +567,7 @@ Per-Host Resources: mem-estimate=16.11MB mem-reservation=9.62MB thread-reservati | | | in pipelines: 01(GETNEXT) | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=130.84MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 01:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -585,7 +585,7 @@ Per-Host Resources: mem-estimate=16.11MB mem-reservation=9.62MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=51.95MB mem-reservation=5.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=52.31MB mem-reservation=5.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 04:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_sold_date_sk = d_date_sk | | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -600,7 +600,7 @@ Per-Host Resources: mem-estimate=16.11MB mem-reservation=9.62MB thread-reservati | | | in pipelines: 03(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -632,7 +632,7 @@ Per-Host Resources: mem-estimate=16.11MB mem-reservation=9.62MB thread-reservati | in pipelines: 36(GETNEXT) | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=48.16MB mem-reservation=36.94MB thread-reservation=1 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=49.14MB mem-reservation=36.94MB thread-reservation=1 runtime-filters-memory=1.00MB 28:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: customer_id = customer_id | fk/pk conjuncts: assumed fk/pk @@ -647,7 +647,7 @@ Per-Host Resources: mem-estimate=48.16MB mem-reservation=36.94MB thread-reservat | | in pipelines: 41(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1 +| Per-Host Resources: mem-estimate=27.53MB mem-reservation=17.00MB thread-reservation=1 | 14:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=28 row-size=44B cardinality=14.80K @@ -689,7 +689,7 @@ Per-Host Resources: mem-estimate=48.16MB mem-reservation=36.94MB thread-reservat | | | in pipelines: 17(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -717,7 +717,7 @@ Per-Host Resources: mem-estimate=48.16MB mem-reservation=36.94MB thread-reservat | | | in pipelines: 15(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=130.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=131.23MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB | | 15:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id @@ -735,7 +735,7 @@ Per-Host Resources: mem-estimate=48.16MB mem-reservation=36.94MB thread-reservat | | in pipelines: 16(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=130.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=130.16MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB | 16:SCAN HDFS [tpcds_parquet.web_sales, RANDOM] | HDFS partitions=1/1 files=2 size=45.09MB | runtime filters: RF011[min_max] -> ws_sold_date_sk, RF013[min_max] -> ws_bill_customer_sk, RF010[bloom] -> ws_sold_date_sk, RF012[bloom] -> ws_bill_customer_sk @@ -787,7 +787,7 @@ Per-Host Resources: mem-estimate=58.24MB mem-reservation=43.50MB thread-reservat | | in pipelines: 08(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=131.00MB mem-reservation=11.00MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=132.84MB mem-reservation=11.00MB thread-reservation=2 runtime-filters-memory=3.00MB | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id @@ -805,7 +805,7 @@ Per-Host Resources: mem-estimate=58.24MB mem-reservation=43.50MB thread-reservat | in pipelines: 09(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=51.95MB mem-reservation=5.94MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=52.31MB mem-reservation=5.94MB thread-reservation=2 runtime-filters-memory=2.00MB 11:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_sold_date_sk = d_date_sk | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -820,7 +820,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=5.94MB thread-reservati | | in pipelines: 10(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -847,7 +847,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=5.94MB thread-reservati in pipelines: 09(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=443.00MB Threads=44 -Per-Host Resource Estimates: Memory=811MB +Per-Host Resource Estimates: Memory=854MB F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -892,7 +892,7 @@ Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservatio | | in pipelines: 54(GETNEXT) | | | F24:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=28.46MB mem-reservation=17.00MB thread-reservation=1 | 21:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=38 row-size=44B cardinality=148.00K @@ -911,7 +911,7 @@ Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservatio | | in pipelines: 23(GETNEXT) | | | F21:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=22.53MB mem-reservation=17.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=23.88MB mem-reservation=17.00MB thread-reservation=1 | 27:AGGREGATE [STREAMING] | | output: sum(ws_ext_list_price - ws_ext_discount_amt) | | group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year @@ -941,7 +941,7 @@ Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservatio | | | in pipelines: 24(GETNEXT) | | | | | F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -977,7 +977,7 @@ Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservatio | | | in pipelines: 22(GETNEXT) | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=17.23MB mem-reservation=8.00MB thread-reservation=1 | | 22:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -995,7 +995,7 @@ Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservatio | | | F19:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.16MB mem-reservation=8.00MB thread-reservation=1 | 23:SCAN HDFS [tpcds_parquet.web_sales, RANDOM] | HDFS partitions=1/1 files=2 size=45.09MB | runtime filters: RF019[min_max] -> ws_sold_date_sk, RF021[min_max] -> ws_bill_customer_sk, RF018[bloom] -> ws_sold_date_sk @@ -1029,7 +1029,7 @@ Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservatio | | in pipelines: 47(GETNEXT) | | | F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=46.14MB mem-reservation=34.00MB thread-reservation=1 | 00:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=8 row-size=44B cardinality=58.90K @@ -1049,7 +1049,7 @@ Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservatio | | in pipelines: 02(GETNEXT) | | | F15:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=21.68MB mem-reservation=17.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=25.74MB mem-reservation=17.00MB thread-reservation=1 | 06:AGGREGATE [STREAMING] | | output: sum(ss_ext_list_price - ss_ext_discount_amt) | | group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year @@ -1080,7 +1080,7 @@ Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservatio | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=19.68MB mem-reservation=8.00MB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1099,7 +1099,7 @@ Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservatio | | | F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.75MB mem-reservation=2.00MB thread-reservation=1 | 04:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=05 | | hash predicates: ss_sold_date_sk = d_date_sk @@ -1122,7 +1122,7 @@ Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservatio | | | in pipelines: 03(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -1154,7 +1154,7 @@ Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservatio | in pipelines: 36(GETNEXT) | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=48.41MB mem-reservation=34.00MB thread-reservation=1 28:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=06 | hash predicates: customer_id = customer_id @@ -1177,7 +1177,7 @@ Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-rese | | in pipelines: 41(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=27.53MB mem-reservation=17.00MB thread-reservation=1 | 14:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=28 row-size=44B cardinality=14.80K @@ -1197,7 +1197,7 @@ Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-rese | | in pipelines: 16(GETNEXT) | | | F08:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=22.53MB mem-reservation=17.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=23.88MB mem-reservation=17.00MB thread-reservation=1 | 20:AGGREGATE [STREAMING] | | output: sum(ws_ext_list_price - ws_ext_discount_amt) | | group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year @@ -1227,7 +1227,7 @@ Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-rese | | | in pipelines: 17(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -1264,7 +1264,7 @@ Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-rese | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=17.23MB mem-reservation=8.00MB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1283,7 +1283,7 @@ Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-rese | | | F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.16MB mem-reservation=8.00MB thread-reservation=1 | 16:SCAN HDFS [tpcds_parquet.web_sales, RANDOM] | HDFS partitions=1/1 files=2 size=45.09MB | runtime filters: RF011[min_max] -> ws_sold_date_sk, RF013[min_max] -> ws_bill_customer_sk, RF010[bloom] -> ws_sold_date_sk, RF012[bloom] -> ws_bill_customer_sk @@ -1313,7 +1313,7 @@ Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-rese | in pipelines: 09(GETNEXT) | F03:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=21.68MB mem-reservation=17.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=25.74MB mem-reservation=17.00MB thread-reservation=1 13:AGGREGATE [STREAMING] | output: sum(ss_ext_list_price - ss_ext_discount_amt) | group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year @@ -1344,7 +1344,7 @@ Per-Instance Resources: mem-estimate=21.68MB mem-reservation=17.00MB thread-rese | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=19.68MB mem-reservation=8.00MB thread-reservation=1 | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1363,7 +1363,7 @@ Per-Instance Resources: mem-estimate=21.68MB mem-reservation=17.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=16.75MB mem-reservation=2.00MB thread-reservation=1 11:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=10 | hash predicates: ss_sold_date_sk = d_date_sk @@ -1386,7 +1386,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 10(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test index 9f2a9b054..1bc98d160 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test @@ -115,7 +115,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=105.81MB Threads=8 -Per-Host Resource Estimates: Memory=329MB +Per-Host Resource Estimates: Memory=330MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -163,7 +163,7 @@ Per-Host Resources: mem-estimate=58.39MB mem-reservation=50.00MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -Per-Host Resources: mem-estimate=138.19MB mem-reservation=48.81MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=138.39MB mem-reservation=48.81MB thread-reservation=2 runtime-filters-memory=2.00MB 05:AGGREGATE [STREAMING] | output: sum(ws_ext_sales_price) | group by: i_item_id, i_item_desc, i_category, i_class, i_current_price @@ -185,7 +185,7 @@ Per-Host Resources: mem-estimate=138.19MB mem-reservation=48.81MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '1999-03-24', CAST(d_date AS DATE) >= DATE '1999-02-22' @@ -213,7 +213,7 @@ Per-Host Resources: mem-estimate=138.19MB mem-reservation=48.81MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=96.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=96.76MB mem-reservation=2.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category IN ('Sports', 'Books', 'Home') @@ -239,7 +239,7 @@ Per-Host Resources: mem-estimate=138.19MB mem-reservation=48.81MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=112.62MB Threads=7 -Per-Host Resource Estimates: Memory=175MB +Per-Host Resource Estimates: Memory=178MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -288,7 +288,7 @@ Per-Instance Resources: mem-estimate=58.39MB mem-reservation=50.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=66.00MB mem-reservation=42.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=67.58MB mem-reservation=42.00MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(ws_ext_sales_price) | group by: i_item_id, i_item_desc, i_category, i_class, i_current_price @@ -318,7 +318,7 @@ Per-Instance Resources: mem-estimate=66.00MB mem-reservation=42.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '1999-03-24', CAST(d_date AS DATE) >= DATE '1999-02-22' @@ -354,7 +354,7 @@ Per-Instance Resources: mem-estimate=66.00MB mem-reservation=42.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.76MB mem-reservation=2.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category IN ('Sports', 'Books', 'Home') diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test index b50e2c28a..a4a7bcd8a 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test @@ -181,7 +181,7 @@ PLAN-ROOT SINK in pipelines: 02(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=36.08MB Threads=14 -Per-Host Resource Estimates: Memory=335MB +Per-Host Resource Estimates: Memory=337MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -221,7 +221,7 @@ Per-Host Resources: mem-estimate=19.61MB mem-reservation=12.44MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -247,7 +247,7 @@ Per-Host Resources: mem-estimate=19.61MB mem-reservation=12.44MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.50MB mem-reservation=8.00MB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'M' OR cd_marital_status = 'S' AND cd_education_status = 'College' OR cd_marital_status = 'W' AND cd_education_status = '2 yr Degree', cd_education_status = 'Advanced Degree' OR cd_marital_status = 'S' AND cd_education_status = 'College' OR cd_marital_status = 'W' AND cd_education_status = '2 yr Degree' @@ -265,7 +265,7 @@ Per-Host Resources: mem-estimate=19.61MB mem-reservation=12.44MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=139.26MB mem-reservation=14.81MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=140.16MB mem-reservation=14.81MB thread-reservation=2 runtime-filters-memory=5.00MB 08:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_hdemo_sk = hd_demo_sk | fk/pk conjuncts: ss_hdemo_sk = hd_demo_sk @@ -281,7 +281,7 @@ Per-Host Resources: mem-estimate=139.26MB mem-reservation=14.81MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=64.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: hd_dep_count IN (CAST(3 AS INT), CAST(1 AS INT), CAST(1 AS INT)) @@ -309,7 +309,7 @@ Per-Host Resources: mem-estimate=139.26MB mem-reservation=14.81MB thread-reserva | | in pipelines: 05(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT) @@ -338,7 +338,7 @@ Per-Host Resources: mem-estimate=139.26MB mem-reservation=14.81MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.18MB mem-reservation=256.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_state IN ('TX', 'OH', 'TX', 'OR', 'NM', 'KY', 'VA', 'TX', 'MS'), ca_country = 'United States' @@ -367,7 +367,7 @@ Per-Host Resources: mem-estimate=139.26MB mem-reservation=14.81MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=52.83MB Threads=16 -Per-Host Resource Estimates: Memory=164MB +Per-Host Resource Estimates: Memory=170MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -415,7 +415,7 @@ Per-Instance Resources: mem-estimate=802.87KB mem-reservation=0B thread-reservat | | in pipelines: 01(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -449,7 +449,7 @@ Per-Instance Resources: mem-estimate=802.87KB mem-reservation=0B thread-reservat | | in pipelines: 02(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.00MB mem-reservation=8.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'M' OR cd_marital_status = 'S' AND cd_education_status = 'College' OR cd_marital_status = 'W' AND cd_education_status = '2 yr Degree', cd_education_status = 'Advanced Degree' OR cd_marital_status = 'S' AND cd_education_status = 'College' OR cd_marital_status = 'W' AND cd_education_status = '2 yr Degree' @@ -468,7 +468,7 @@ Per-Instance Resources: mem-estimate=802.87KB mem-reservation=0B thread-reservat | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=18.70MB mem-reservation=4.00MB thread-reservation=1 08:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=02 | hash predicates: ss_hdemo_sk = hd_demo_sk @@ -492,7 +492,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=64.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: hd_dep_count IN (CAST(3 AS INT), CAST(1 AS INT), CAST(1 AS INT)) @@ -528,7 +528,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT) @@ -565,7 +565,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.18MB mem-reservation=256.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_state IN ('TX', 'OH', 'TX', 'OR', 'NM', 'KY', 'VA', 'TX', 'MS'), ca_country = 'United States' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test index 9af2145ab..6d4222ac8 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test @@ -1351,7 +1351,7 @@ Per-Host Resources: mem-estimate=137.09MB mem-reservation=93.62MB thread-reserva | in pipelines: 151(GETNEXT), 179(GETNEXT), 207(GETNEXT) | F78:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reservation=1 +Per-Host Resources: mem-estimate=112.88MB mem-reservation=81.75MB thread-reservation=1 127:AGGREGATE [STREAMING] | Class 0 | output: sum(sales), sum(number_sales) @@ -1389,7 +1389,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | in pipelines: 212(GETNEXT) | | | | | F77:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | 212:AGGREGATE [FINALIZE] | | | output: avg:merge(quantity * list_price) | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1428,7 +1428,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 123(GETNEXT) | | | | | | | | | F75:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 123:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -1467,7 +1467,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 120(GETNEXT) | | | | | | | | | F73:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 120:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -1505,7 +1505,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | in pipelines: 117(GETNEXT) | | | | | | | F71:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 117:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -1598,7 +1598,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | in pipelines: 88(GETNEXT) | | | | | | | F67:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=64.28MB mem-reservation=256.00KB thread-reservation=2 | | | 88:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -1655,7 +1655,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 102(GETNEXT) | | | | | | | | | F65:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 102:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT) @@ -1683,7 +1683,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 101(GETNEXT) | | | | | | | | | F64:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | | 101:SCAN HDFS [tpcds_parquet.item iws, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -1751,7 +1751,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 97(GETNEXT) | | | | | | | | | F61:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 97:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT) @@ -1779,7 +1779,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 96(GETNEXT) | | | | | | | | | F60:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | | 96:SCAN HDFS [tpcds_parquet.item ics, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -1833,7 +1833,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | in pipelines: 91(GETNEXT) | | | | | | | F57:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 91:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT) @@ -1861,7 +1861,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | in pipelines: 90(GETNEXT) | | | | | | | F56:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | 90:SCAN HDFS [tpcds_parquet.item iss, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | runtime filters: RF081[min_max] -> iss.i_brand_id, RF082[min_max] -> iss.i_category_id, RF083[min_max] -> iss.i_class_id @@ -1899,7 +1899,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | in pipelines: 86(GETNEXT) | | | | | F54:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | 86:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF073[min_max] -> tpcds_parquet.item.i_item_sk @@ -1925,7 +1925,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | in pipelines: 87(GETNEXT) | | | | | F53:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 87:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT) @@ -1962,7 +1962,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | in pipelines: 184(GETNEXT) | | | | | F51:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | 184:AGGREGATE [FINALIZE] | | | output: avg:merge(quantity * list_price) | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -2001,7 +2001,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 81(GETNEXT) | | | | | | | | | F49:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 81:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -2040,7 +2040,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 78(GETNEXT) | | | | | | | | | F47:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 78:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -2078,7 +2078,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | in pipelines: 75(GETNEXT) | | | | | | | F45:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 75:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -2171,7 +2171,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | in pipelines: 46(GETNEXT) | | | | | | | F41:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=64.28MB mem-reservation=256.00KB thread-reservation=2 | | | 46:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -2228,7 +2228,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 60(GETNEXT) | | | | | | | | | F39:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 60:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT) @@ -2256,7 +2256,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 59(GETNEXT) | | | | | | | | | F38:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | | 59:SCAN HDFS [tpcds_parquet.item iws, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -2324,7 +2324,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 55(GETNEXT) | | | | | | | | | F35:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 55:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT) @@ -2352,7 +2352,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | | in pipelines: 54(GETNEXT) | | | | | | | | | F34:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | | 54:SCAN HDFS [tpcds_parquet.item ics, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -2406,7 +2406,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | in pipelines: 49(GETNEXT) | | | | | | | F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 49:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT) @@ -2434,7 +2434,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | in pipelines: 48(GETNEXT) | | | | | | | F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | 48:SCAN HDFS [tpcds_parquet.item iss, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | runtime filters: RF045[min_max] -> iss.i_brand_id, RF046[min_max] -> iss.i_category_id, RF047[min_max] -> iss.i_class_id @@ -2472,7 +2472,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | in pipelines: 44(GETNEXT) | | | | | F28:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | 44:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF037[min_max] -> tpcds_parquet.item.i_item_sk @@ -2498,7 +2498,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | in pipelines: 45(GETNEXT) | | | | | F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 45:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT) @@ -2535,7 +2535,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | in pipelines: 156(GETNEXT) | | | F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 156:AGGREGATE [FINALIZE] | | output: avg:merge(quantity * list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -2574,7 +2574,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | in pipelines: 39(GETNEXT) | | | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 39:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -2613,7 +2613,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | | in pipelines: 36(GETNEXT) | | | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 36:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -2651,7 +2651,7 @@ Per-Host Resources: mem-estimate=109.84MB mem-reservation=81.75MB thread-reserva | | | in pipelines: 33(GETNEXT) | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 33:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -2744,7 +2744,7 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=12.81MB thread-reservat | | | in pipelines: 04(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.28MB mem-reservation=256.00KB thread-reservation=2 | | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -2802,7 +2802,7 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=12.81MB thread-reservat | | | | in pipelines: 18(GETNEXT) | | | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 18:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT) @@ -2830,7 +2830,7 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=12.81MB thread-reservat | | | | in pipelines: 17(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | 17:SCAN HDFS [tpcds_parquet.item iws, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -2898,7 +2898,7 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=12.81MB thread-reservat | | | | in pipelines: 13(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 13:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT) @@ -2926,7 +2926,7 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=12.81MB thread-reservat | | | | in pipelines: 12(GETNEXT) | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | 12:SCAN HDFS [tpcds_parquet.item ics, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -2980,7 +2980,7 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=12.81MB thread-reservat | | | in pipelines: 07(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 07:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT) @@ -3008,7 +3008,7 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=12.81MB thread-reservat | | | in pipelines: 06(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=69.00MB mem-reservation=5.25MB thread-reservation=2 runtime-filters-memory=5.00MB +| | Per-Host Resources: mem-estimate=69.09MB mem-reservation=5.25MB thread-reservation=2 runtime-filters-memory=5.00MB | | 06:SCAN HDFS [tpcds_parquet.item iss, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF009[min_max] -> iss.i_brand_id, RF010[min_max] -> iss.i_category_id, RF011[min_max] -> iss.i_class_id, RF006[bloom] -> iss.i_brand_id, RF007[bloom] -> iss.i_category_id, RF008[bloom] -> iss.i_class_id, RF012[bloom] -> iss.i_brand_id, RF013[bloom] -> iss.i_category_id @@ -3046,7 +3046,7 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=12.81MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=65.00MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=65.09MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk @@ -3072,7 +3072,7 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=12.81MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT) @@ -3099,7 +3099,7 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=12.81MB thread-reservat in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=662.12MB Threads=141 -Per-Host Resource Estimates: Memory=2.26GB +Per-Host Resource Estimates: Memory=2.28GB F80:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -3154,7 +3154,7 @@ Per-Instance Resources: mem-estimate=109.52MB mem-reservation=85.12MB thread-res | in pipelines: 151(GETNEXT), 179(GETNEXT), 207(GETNEXT) | F78:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-reservation=1 +Per-Instance Resources: mem-estimate=82.03MB mem-reservation=45.88MB thread-reservation=1 127:AGGREGATE [STREAMING] | Class 0 | output: sum(sales), sum(number_sales) @@ -3199,7 +3199,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | in pipelines: 212(GETNEXT) | | | | | F77:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | 212:AGGREGATE [FINALIZE] | | | output: avg:merge(quantity * list_price) | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -3246,7 +3246,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 123(GETNEXT) | | | | | | | | | F75:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 123:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -3293,7 +3293,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 120(GETNEXT) | | | | | | | | | F73:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 120:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -3339,7 +3339,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 117(GETNEXT) | | | | | | | F71:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 117:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -3378,7 +3378,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | F52:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=42.00MB mem-reservation=6.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=42.94MB mem-reservation=6.00MB thread-reservation=1 | 114:AGGREGATE [STREAMING] | | output: sum(CAST(ws_quantity AS DECIMAL(10,0)) * ws_list_price), count(*) | | group by: i_brand_id, i_class_id, i_category_id @@ -3449,7 +3449,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 88(GETNEXT) | | | | | | | F67:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.56MB mem-reservation=256.00KB thread-reservation=1 | | | 88:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -3493,7 +3493,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 100(GETNEXT) | | | | | | | F63:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | | Per-Instance Resources: mem-estimate=42.00MB mem-reservation=9.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1 | | | 107:AGGREGATE [STREAMING] | | | | group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id | | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -3522,7 +3522,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 102(GETNEXT) | | | | | | | | | F65:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 102:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT) @@ -3558,7 +3558,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 101(GETNEXT) | | | | | | | | | F64:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | | 101:SCAN HDFS [tpcds_parquet.item iws, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -3613,7 +3613,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 95(GETNEXT) | | | | | | | F59:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | | Per-Instance Resources: mem-estimate=58.00MB mem-reservation=9.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1 | | | 105:AGGREGATE [STREAMING] | | | | group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id | | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -3642,7 +3642,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 97(GETNEXT) | | | | | | | | | F61:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 97:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT) @@ -3678,7 +3678,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 96(GETNEXT) | | | | | | | | | F60:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | | 96:SCAN HDFS [tpcds_parquet.item ics, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -3712,7 +3712,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | in pipelines: 89(GETNEXT) | | | | | F55:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=5.50MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=26.38MB mem-reservation=5.50MB thread-reservation=1 | | 94:AGGREGATE [STREAMING] | | | group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -3740,7 +3740,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 91(GETNEXT) | | | | | | | F57:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 91:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT) @@ -3776,7 +3776,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 90(GETNEXT) | | | | | | | F56:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | 90:SCAN HDFS [tpcds_parquet.item iss, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | runtime filters: RF081[min_max] -> iss.i_brand_id, RF082[min_max] -> iss.i_category_id, RF083[min_max] -> iss.i_class_id @@ -3822,7 +3822,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | in pipelines: 86(GETNEXT) | | | | | F54:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | 86:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF073[min_max] -> tpcds_parquet.item.i_item_sk @@ -3856,7 +3856,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | in pipelines: 87(GETNEXT) | | | | | F53:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 87:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT) @@ -3900,7 +3900,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | in pipelines: 184(GETNEXT) | | | | | F51:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | 184:AGGREGATE [FINALIZE] | | | output: avg:merge(quantity * list_price) | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -3947,7 +3947,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 81(GETNEXT) | | | | | | | | | F49:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 81:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -3994,7 +3994,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 78(GETNEXT) | | | | | | | | | F47:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 78:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -4040,7 +4040,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 75(GETNEXT) | | | | | | | F45:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 75:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -4079,7 +4079,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=58.00MB mem-reservation=6.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=58.94MB mem-reservation=6.00MB thread-reservation=1 | 72:AGGREGATE [STREAMING] | | output: sum(CAST(cs_quantity AS DECIMAL(10,0)) * cs_list_price), count(*) | | group by: i_brand_id, i_class_id, i_category_id @@ -4150,7 +4150,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 46(GETNEXT) | | | | | | | F41:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.56MB mem-reservation=256.00KB thread-reservation=1 | | | 46:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -4194,7 +4194,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 58(GETNEXT) | | | | | | | F37:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | | Per-Instance Resources: mem-estimate=42.00MB mem-reservation=9.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1 | | | 65:AGGREGATE [STREAMING] | | | | group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id | | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -4223,7 +4223,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 60(GETNEXT) | | | | | | | | | F39:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 60:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT) @@ -4259,7 +4259,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 59(GETNEXT) | | | | | | | | | F38:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | | 59:SCAN HDFS [tpcds_parquet.item iws, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -4314,7 +4314,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 53(GETNEXT) | | | | | | | F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | | Per-Instance Resources: mem-estimate=58.00MB mem-reservation=9.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1 | | | 63:AGGREGATE [STREAMING] | | | | group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id | | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -4343,7 +4343,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 55(GETNEXT) | | | | | | | | | F35:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 55:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT) @@ -4379,7 +4379,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | | in pipelines: 54(GETNEXT) | | | | | | | | | F34:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | | 54:SCAN HDFS [tpcds_parquet.item ics, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -4413,7 +4413,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | in pipelines: 47(GETNEXT) | | | | | F29:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=5.50MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=26.38MB mem-reservation=5.50MB thread-reservation=1 | | 52:AGGREGATE [STREAMING] | | | group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -4441,7 +4441,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 49(GETNEXT) | | | | | | | F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 49:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT) @@ -4477,7 +4477,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 48(GETNEXT) | | | | | | | F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | 48:SCAN HDFS [tpcds_parquet.item iss, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | runtime filters: RF045[min_max] -> iss.i_brand_id, RF046[min_max] -> iss.i_category_id, RF047[min_max] -> iss.i_class_id @@ -4523,7 +4523,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | in pipelines: 44(GETNEXT) | | | | | F28:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | 44:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF037[min_max] -> tpcds_parquet.item.i_item_sk @@ -4557,7 +4557,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | in pipelines: 45(GETNEXT) | | | | | F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 45:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT) @@ -4601,7 +4601,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | in pipelines: 156(GETNEXT) | | | F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 156:AGGREGATE [FINALIZE] | | output: avg:merge(quantity * list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -4648,7 +4648,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 39(GETNEXT) | | | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 39:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -4695,7 +4695,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | | in pipelines: 36(GETNEXT) | | | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 36:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -4741,7 +4741,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | | | in pipelines: 33(GETNEXT) | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 33:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -4780,7 +4780,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=45.88MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.94MB mem-reservation=3.00MB thread-reservation=1 30:AGGREGATE [STREAMING] | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_list_price), count(*) | group by: i_brand_id, i_class_id, i_category_id @@ -4851,7 +4851,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | in pipelines: 04(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.56MB mem-reservation=256.00KB thread-reservation=1 | | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -4896,7 +4896,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | in pipelines: 16(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Instance Resources: mem-estimate=42.00MB mem-reservation=9.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1 | | 23:AGGREGATE [STREAMING] | | | group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -4925,7 +4925,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | in pipelines: 18(GETNEXT) | | | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 18:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT) @@ -4961,7 +4961,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | in pipelines: 17(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | 17:SCAN HDFS [tpcds_parquet.item iws, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -5016,7 +5016,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | in pipelines: 11(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | Per-Instance Resources: mem-estimate=58.00MB mem-reservation=9.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1 | | 21:AGGREGATE [STREAMING] | | | group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -5045,7 +5045,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | in pipelines: 13(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 13:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT) @@ -5081,7 +5081,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | in pipelines: 12(GETNEXT) | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | 12:SCAN HDFS [tpcds_parquet.item ics, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -5115,7 +5115,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=5.50MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.38MB mem-reservation=5.50MB thread-reservation=1 | 10:AGGREGATE [STREAMING] | | group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -5143,7 +5143,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | in pipelines: 07(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 07:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT) @@ -5180,7 +5180,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | 06:SCAN HDFS [tpcds_parquet.item iss, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF009[min_max] -> iss.i_brand_id, RF010[min_max] -> iss.i_category_id, RF011[min_max] -> iss.i_class_id, RF006[bloom] -> iss.i_brand_id, RF007[bloom] -> iss.i_category_id, RF008[bloom] -> iss.i_class_id, RF012[bloom] -> iss.i_brand_id, RF013[bloom] -> iss.i_category_id @@ -5227,7 +5227,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk @@ -5261,7 +5261,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14b.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14b.test index 12e663c66..fbdb08aba 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14b.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14b.test @@ -1009,7 +1009,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | in pipelines: 150(GETNEXT) | | | | | F55:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | 150:AGGREGATE [FINALIZE] | | | output: avg:merge(quantity * list_price) | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1048,7 +1048,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | | in pipelines: 86(GETNEXT) | | | | | | | | | F53:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 86:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -1087,7 +1087,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | | in pipelines: 83(GETNEXT) | | | | | | | | | F51:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 83:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -1125,7 +1125,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | in pipelines: 80(GETNEXT) | | | | | | | F49:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 80:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -1184,7 +1184,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | in pipelines: 71(GETNEXT) | | | | | F46:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| | Per-Host Resources: mem-estimate=80.00KB mem-reservation=0B thread-reservation=1 | | 71:CARDINALITY CHECK | | | limit: 1 | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -1198,7 +1198,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | in pipelines: 70(GETNEXT) | | | | | F45:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.08MB mem-reservation=512.00KB thread-reservation=2 | | 70:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1999 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT) @@ -1260,7 +1260,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | in pipelines: 48(GETNEXT) | | | | | | | F43:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=64.28MB mem-reservation=256.00KB thread-reservation=2 | | | 48:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -1317,7 +1317,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | | in pipelines: 62(GETNEXT) | | | | | | | | | F41:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 62:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT) @@ -1345,7 +1345,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | | in pipelines: 61(GETNEXT) | | | | | | | | | F40:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | | 61:SCAN HDFS [tpcds_parquet.item iws, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -1413,7 +1413,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | | in pipelines: 57(GETNEXT) | | | | | | | | | F37:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 57:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT) @@ -1441,7 +1441,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | | in pipelines: 56(GETNEXT) | | | | | | | | | F36:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | | 56:SCAN HDFS [tpcds_parquet.item ics, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -1495,7 +1495,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | in pipelines: 51(GETNEXT) | | | | | | | F33:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 51:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT) @@ -1523,7 +1523,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | in pipelines: 50(GETNEXT) | | | | | | | F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | 50:SCAN HDFS [tpcds_parquet.item iss, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | runtime filters: RF055[min_max] -> iss.i_brand_id, RF056[min_max] -> iss.i_category_id, RF057[min_max] -> iss.i_class_id @@ -1560,7 +1560,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | in pipelines: 47(GETNEXT) | | | | | F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.05MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | | 47:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF045[min_max] -> d_week_seq, RF044[bloom] -> d_week_seq @@ -1586,7 +1586,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | in pipelines: 46(GETNEXT) | | | | | F29:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | 46:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF047[min_max] -> tpcds_parquet.item.i_item_sk @@ -1622,7 +1622,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | in pipelines: 120(GETNEXT) | | | F27:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 120:AGGREGATE [FINALIZE] | | output: avg:merge(quantity * list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1661,7 +1661,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | in pipelines: 41(GETNEXT) | | | | | | | F25:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 41:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -1700,7 +1700,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | | in pipelines: 38(GETNEXT) | | | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 38:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -1738,7 +1738,7 @@ Per-Host Resources: mem-estimate=25.00MB mem-reservation=8.81MB thread-reservati | | | in pipelines: 35(GETNEXT) | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 35:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -1797,7 +1797,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat | | in pipelines: 26(GETNEXT) | | | F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=80.00KB mem-reservation=0B thread-reservation=1 | 26:CARDINALITY CHECK | | limit: 1 | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -1811,7 +1811,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat | | in pipelines: 25(GETNEXT) | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.08MB mem-reservation=512.00KB thread-reservation=2 | 25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2000 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT) @@ -1873,7 +1873,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat | | | in pipelines: 03(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.28MB mem-reservation=256.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -1930,7 +1930,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat | | | | in pipelines: 17(GETNEXT) | | | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 17:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT) @@ -1958,7 +1958,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat | | | | in pipelines: 16(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | 16:SCAN HDFS [tpcds_parquet.item iws, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -2026,7 +2026,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat | | | | in pipelines: 12(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 12:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT) @@ -2054,7 +2054,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat | | | | in pipelines: 11(GETNEXT) | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=64.09MB mem-reservation=256.00KB thread-reservation=2 | | | 11:SCAN HDFS [tpcds_parquet.item ics, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -2108,7 +2108,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat | | | in pipelines: 06(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 06:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT) @@ -2136,7 +2136,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat | | | in pipelines: 05(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=66.00MB mem-reservation=2.25MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=66.09MB mem-reservation=2.25MB thread-reservation=2 runtime-filters-memory=2.00MB | | 05:SCAN HDFS [tpcds_parquet.item iss, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF017[min_max] -> iss.i_brand_id, RF018[min_max] -> iss.i_category_id, RF019[min_max] -> iss.i_class_id, RF014[bloom] -> iss.i_brand_id, RF015[bloom] -> iss.i_category_id @@ -2174,7 +2174,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.05MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF007[min_max] -> d_week_seq, RF006[bloom] -> d_week_seq @@ -2200,7 +2200,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=68.00MB mem-reservation=4.25MB thread-reservation=2 runtime-filters-memory=4.00MB +| Per-Host Resources: mem-estimate=68.09MB mem-reservation=4.25MB thread-reservation=2 runtime-filters-memory=4.00MB | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF003[min_max] -> tpcds_parquet.item.i_brand_id, RF004[min_max] -> tpcds_parquet.item.i_category_id, RF005[min_max] -> tpcds_parquet.item.i_class_id, RF009[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_brand_id, RF001[bloom] -> tpcds_parquet.item.i_category_id, RF002[bloom] -> tpcds_parquet.item.i_class_id, RF008[bloom] -> tpcds_parquet.item.i_item_sk @@ -2225,7 +2225,7 @@ Per-Host Resources: mem-estimate=73.69MB mem-reservation=17.56MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=308.38MB Threads=103 -Per-Host Resource Estimates: Memory=1.37GB +Per-Host Resource Estimates: Memory=1.38GB F56:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -2282,7 +2282,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | in pipelines: 150(GETNEXT) | | | | | F55:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | 150:AGGREGATE [FINALIZE] | | | output: avg:merge(quantity * list_price) | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -2329,7 +2329,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | | in pipelines: 86(GETNEXT) | | | | | | | | | F53:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 86:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -2376,7 +2376,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | | in pipelines: 83(GETNEXT) | | | | | | | | | F51:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 83:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -2422,7 +2422,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 80(GETNEXT) | | | | | | | F49:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 80:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -2460,7 +2460,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | in pipelines: 45(GETNEXT) | | | F28:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.94MB mem-reservation=3.00MB thread-reservation=1 | 77:AGGREGATE [STREAMING] | | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_list_price), count(*) | | group by: i_brand_id, i_class_id, i_category_id @@ -2489,7 +2489,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | in pipelines: 71(GETNEXT) | | | | | F46:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| | Per-Instance Resources: mem-estimate=80.00KB mem-reservation=0B thread-reservation=1 | | 71:CARDINALITY CHECK | | | limit: 1 | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -2503,7 +2503,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | in pipelines: 70(GETNEXT) | | | | | F45:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.08MB mem-reservation=512.00KB thread-reservation=1 | | 70:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1999 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT) @@ -2581,7 +2581,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 48(GETNEXT) | | | | | | | F43:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.56MB mem-reservation=256.00KB thread-reservation=1 | | | 48:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -2625,7 +2625,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 60(GETNEXT) | | | | | | | F39:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | | Per-Instance Resources: mem-estimate=42.00MB mem-reservation=9.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1 | | | 67:AGGREGATE [STREAMING] | | | | group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id | | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -2654,7 +2654,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | | in pipelines: 62(GETNEXT) | | | | | | | | | F41:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 62:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT) @@ -2690,7 +2690,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | | in pipelines: 61(GETNEXT) | | | | | | | | | F40:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | | 61:SCAN HDFS [tpcds_parquet.item iws, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -2745,7 +2745,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 55(GETNEXT) | | | | | | | F35:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | | Per-Instance Resources: mem-estimate=58.00MB mem-reservation=9.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1 | | | 65:AGGREGATE [STREAMING] | | | | group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id | | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -2774,7 +2774,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | | in pipelines: 57(GETNEXT) | | | | | | | | | F37:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 57:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT) @@ -2810,7 +2810,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | | in pipelines: 56(GETNEXT) | | | | | | | | | F36:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | | 56:SCAN HDFS [tpcds_parquet.item ics, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -2844,7 +2844,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | in pipelines: 49(GETNEXT) | | | | | F31:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=5.50MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=26.38MB mem-reservation=5.50MB thread-reservation=1 | | 54:AGGREGATE [STREAMING] | | | group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -2872,7 +2872,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 51(GETNEXT) | | | | | | | F33:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 51:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT) @@ -2908,7 +2908,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 50(GETNEXT) | | | | | | | F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | 50:SCAN HDFS [tpcds_parquet.item iss, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | runtime filters: RF055[min_max] -> iss.i_brand_id, RF056[min_max] -> iss.i_category_id, RF057[min_max] -> iss.i_class_id @@ -2954,7 +2954,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | | F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 47:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF045[min_max] -> d_week_seq, RF044[bloom] -> d_week_seq @@ -2988,7 +2988,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | in pipelines: 46(GETNEXT) | | | | | F29:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | 46:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF047[min_max] -> tpcds_parquet.item.i_item_sk @@ -3031,7 +3031,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | in pipelines: 120(GETNEXT) | | | F27:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 120:AGGREGATE [FINALIZE] | | output: avg:merge(quantity * list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -3078,7 +3078,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 41(GETNEXT) | | | | | | | F25:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 41:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -3125,7 +3125,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 38(GETNEXT) | | | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 38:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -3171,7 +3171,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | | | in pipelines: 35(GETNEXT) | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 35:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT) @@ -3210,7 +3210,7 @@ Per-Instance Resources: mem-estimate=10.07MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.94MB mem-reservation=3.00MB thread-reservation=1 32:AGGREGATE [STREAMING] | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_list_price), count(*) | group by: i_brand_id, i_class_id, i_category_id @@ -3239,7 +3239,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 26(GETNEXT) | | | F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=80.00KB mem-reservation=0B thread-reservation=1 | 26:CARDINALITY CHECK | | limit: 1 | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -3253,7 +3253,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 25(GETNEXT) | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.08MB mem-reservation=512.00KB thread-reservation=1 | 25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2000 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT) @@ -3331,7 +3331,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | in pipelines: 03(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.56MB mem-reservation=256.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -3375,7 +3375,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | in pipelines: 15(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Instance Resources: mem-estimate=42.00MB mem-reservation=9.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1 | | 22:AGGREGATE [STREAMING] | | | group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -3404,7 +3404,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | in pipelines: 17(GETNEXT) | | | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 17:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT) @@ -3440,7 +3440,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | in pipelines: 16(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | 16:SCAN HDFS [tpcds_parquet.item iws, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -3495,7 +3495,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | in pipelines: 10(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | Per-Instance Resources: mem-estimate=58.00MB mem-reservation=9.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1 | | 20:AGGREGATE [STREAMING] | | | group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id | | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -3524,7 +3524,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | in pipelines: 12(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 12:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT) @@ -3560,7 +3560,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | in pipelines: 11(GETNEXT) | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | 11:SCAN HDFS [tpcds_parquet.item ics, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -3594,7 +3594,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=5.50MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.38MB mem-reservation=5.50MB thread-reservation=1 | 09:AGGREGATE [STREAMING] | | group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id | | mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0 @@ -3622,7 +3622,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | in pipelines: 06(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 06:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT) @@ -3659,7 +3659,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | 05:SCAN HDFS [tpcds_parquet.item iss, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF017[min_max] -> iss.i_brand_id, RF018[min_max] -> iss.i_category_id, RF019[min_max] -> iss.i_class_id, RF014[bloom] -> iss.i_brand_id, RF015[bloom] -> iss.i_category_id @@ -3706,7 +3706,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF007[min_max] -> d_week_seq, RF006[bloom] -> d_week_seq @@ -3741,7 +3741,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF003[min_max] -> tpcds_parquet.item.i_brand_id, RF004[min_max] -> tpcds_parquet.item.i_category_id, RF005[min_max] -> tpcds_parquet.item.i_class_id, RF009[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_brand_id, RF001[bloom] -> tpcds_parquet.item.i_category_id, RF002[bloom] -> tpcds_parquet.item.i_class_id, RF008[bloom] -> tpcds_parquet.item.i_item_sk diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test index 81c2d039a..c4550c7b1 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test @@ -119,7 +119,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=26.62MB Threads=12 -Per-Host Resource Estimates: Memory=314MB +Per-Host Resource Estimates: Memory=315MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -177,7 +177,7 @@ Per-Host Resources: mem-estimate=17.21MB mem-reservation=5.88MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.46MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -209,7 +209,7 @@ Per-Host Resources: mem-estimate=5.86MB mem-reservation=3.88MB thread-reservatio | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.14MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 01:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> c_current_addr_sk, RF000[bloom] -> c_current_addr_sk @@ -227,7 +227,7 @@ Per-Host Resources: mem-estimate=5.86MB mem-reservation=3.88MB thread-reservatio | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=147.95MB mem-reservation=7.94MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=148.31MB mem-reservation=7.94MB thread-reservation=2 runtime-filters-memory=2.00MB 04:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: cs_sold_date_sk = d_date_sk | fk/pk conjuncts: cs_sold_date_sk = d_date_sk @@ -242,7 +242,7 @@ Per-Host Resources: mem-estimate=147.95MB mem-reservation=7.94MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_qoy = CAST(2 AS INT) @@ -268,7 +268,7 @@ Per-Host Resources: mem-estimate=147.95MB mem-reservation=7.94MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=29.56MB Threads=11 -Per-Host Resource Estimates: Memory=141MB +Per-Host Resource Estimates: Memory=143MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -303,7 +303,7 @@ Per-Instance Resources: mem-estimate=10.15MB mem-reservation=1.94MB thread-reser | in pipelines: 00(GETNEXT) | F05:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=11.62MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=12.06MB mem-reservation=2.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | output: sum(cs_sales_price) | group by: ca_zip @@ -334,7 +334,7 @@ Per-Instance Resources: mem-estimate=11.62MB mem-reservation=2.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.46MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -351,7 +351,7 @@ Per-Instance Resources: mem-estimate=11.62MB mem-reservation=2.00MB thread-reser | in pipelines: 00(GETNEXT) | F03:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=1.21MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=1.73MB mem-reservation=0B thread-reservation=1 05:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=01 | hash predicates: cs_bill_customer_sk = c_customer_sk @@ -375,7 +375,7 @@ Per-Instance Resources: mem-estimate=1.21MB mem-reservation=0B thread-reservatio | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.14MB mem-reservation=1.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> c_current_addr_sk, RF000[bloom] -> c_current_addr_sk @@ -394,7 +394,7 @@ Per-Instance Resources: mem-estimate=1.21MB mem-reservation=0B thread-reservatio | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=48.38MB mem-reservation=4.00MB thread-reservation=1 04:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=02 | hash predicates: cs_sold_date_sk = d_date_sk @@ -417,7 +417,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_qoy = CAST(2 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test index 244687d19..7a86ca81b 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test @@ -161,7 +161,7 @@ PLAN-ROOT SINK in pipelines: 04(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=41.77MB Threads=14 -Per-Host Resource Estimates: Memory=584MB +Per-Host Resource Estimates: Memory=586MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -212,7 +212,7 @@ Per-Host Resources: mem-estimate=22.30MB mem-reservation=8.69MB thread-reservati | | in pipelines: 05(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=24.14MB mem-reservation=2.00MB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.catalog_returns cr1, RANDOM] | HDFS partitions=1/1 files=1 size=10.62MB | stored statistics: @@ -237,7 +237,7 @@ Per-Host Resources: mem-estimate=22.30MB mem-reservation=8.69MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=345.07MB mem-reservation=24.81MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=346.28MB mem-reservation=24.81MB thread-reservation=2 runtime-filters-memory=3.00MB | 08:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: cs1.cs_call_center_sk = cc_call_center_sk | | fk/pk conjuncts: cs1.cs_call_center_sk = cc_call_center_sk @@ -252,7 +252,7 @@ Per-Host Resources: mem-estimate=22.30MB mem-reservation=8.69MB thread-reservati | | | in pipelines: 03(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.14MB mem-reservation=16.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | | HDFS partitions=1/1 files=1 size=10.28KB | | predicates: cc_county = 'Williamson County' @@ -280,7 +280,7 @@ Per-Host Resources: mem-estimate=22.30MB mem-reservation=8.69MB thread-reservati | | | in pipelines: 01(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '2002-04-02', CAST(d_date AS DATE) >= DATE '2002-02-01' @@ -308,7 +308,7 @@ Per-Host Resources: mem-estimate=22.30MB mem-reservation=8.69MB thread-reservati | | | in pipelines: 02(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=256.00KB thread-reservation=2 | | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_state = 'GA' @@ -339,7 +339,7 @@ Per-Host Resources: mem-estimate=22.30MB mem-reservation=8.69MB thread-reservati | in pipelines: 04(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=97.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=97.19MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB 04:SCAN HDFS [tpcds_parquet.catalog_sales cs2, RANDOM] HDFS partitions=1/1 files=3 size=96.62MB runtime filters: RF001[min_max] -> cs2.cs_order_number, RF000[bloom] -> cs2.cs_order_number @@ -352,7 +352,7 @@ Per-Host Resources: mem-estimate=97.00MB mem-reservation=5.00MB thread-reservati in pipelines: 04(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=50.58MB Threads=13 -Per-Host Resource Estimates: Memory=212MB +Per-Host Resource Estimates: Memory=214MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -411,7 +411,7 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser | | in pipelines: 05(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.14MB mem-reservation=2.00MB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.catalog_returns cr1, RANDOM] | HDFS partitions=1/1 files=1 size=10.62MB | stored statistics: @@ -445,7 +445,7 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=49.46MB mem-reservation=16.00MB thread-reservation=1 | 08:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=02 | | hash predicates: cs1.cs_call_center_sk = cc_call_center_sk @@ -468,7 +468,7 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser | | | in pipelines: 03(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | | HDFS partitions=1/1 files=1 size=10.28KB | | predicates: cc_county = 'Williamson County' @@ -504,7 +504,7 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser | | | in pipelines: 01(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '2002-04-02', CAST(d_date AS DATE) >= DATE '2002-02-01' @@ -540,7 +540,7 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser | | | in pipelines: 02(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_state = 'GA' @@ -572,7 +572,7 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=48.19MB mem-reservation=4.00MB thread-reservation=1 04:SCAN HDFS [tpcds_parquet.catalog_sales cs2, RANDOM] HDFS partitions=1/1 files=3 size=96.62MB runtime filters: RF001[min_max] -> cs2.cs_order_number, RF000[bloom] -> cs2.cs_order_number diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test index 336f833ea..556055083 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test @@ -219,7 +219,7 @@ PLAN-ROOT SINK in pipelines: 07(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=70.52MB Threads=20 -Per-Host Resource Estimates: Memory=648MB +Per-Host Resource Estimates: Memory=651MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -254,7 +254,7 @@ Per-Host Resources: mem-estimate=10.82MB mem-reservation=1.94MB thread-reservati | in pipelines: 02(GETNEXT) | F09:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=16.30MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=16.31MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=1.00MB 15:AGGREGATE [STREAMING] | output: count(ss_quantity), avg(CAST(ss_quantity AS BIGINT)), stddev_samp(ss_quantity), count(sr_return_quantity), avg(CAST(sr_return_quantity AS BIGINT)), stddev_samp(sr_return_quantity), count(cs_quantity), avg(CAST(cs_quantity AS BIGINT)), stddev_samp(cs_quantity) | group by: i_item_id, i_item_desc, s_state @@ -276,7 +276,7 @@ Per-Host Resources: mem-estimate=16.30MB mem-reservation=4.94MB thread-reservati | | in pipelines: 07(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=49.79MB mem-reservation=2.00MB thread-reservation=2 | 07:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -293,7 +293,7 @@ Per-Host Resources: mem-estimate=16.30MB mem-reservation=4.94MB thread-reservati | in pipelines: 02(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=205.57MB mem-reservation=19.75MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=206.00MB mem-reservation=19.75MB thread-reservation=2 runtime-filters-memory=5.00MB 13:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_store_sk = s_store_sk | fk/pk conjuncts: ss_store_sk = s_store_sk @@ -308,7 +308,7 @@ Per-Host Resources: mem-estimate=205.57MB mem-reservation=19.75MB thread-reserva | | in pipelines: 06(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | 06:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -333,7 +333,7 @@ Per-Host Resources: mem-estimate=205.57MB mem-reservation=19.75MB thread-reserva | | in pipelines: 05(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.10MB mem-reservation=512.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d3.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') @@ -376,7 +376,7 @@ Per-Host Resources: mem-estimate=205.57MB mem-reservation=19.75MB thread-reserva | | | in pipelines: 04(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.10MB mem-reservation=512.00KB thread-reservation=2 | | 04:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d2.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') @@ -404,7 +404,7 @@ Per-Host Resources: mem-estimate=205.57MB mem-reservation=19.75MB thread-reserva | | | in pipelines: 01(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=122.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=122.38MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB | | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF011[min_max] -> sr_returned_date_sk, RF001[min_max] -> tpcds_parquet.store_returns.sr_item_sk, RF010[bloom] -> sr_returned_date_sk, RF000[bloom] -> tpcds_parquet.store_returns.sr_item_sk @@ -422,7 +422,7 @@ Per-Host Resources: mem-estimate=205.57MB mem-reservation=19.75MB thread-reserva | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=87.95MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=6.00MB +| Per-Host Resources: mem-estimate=88.66MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=6.00MB | 08:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_sold_date_sk = d1.d_date_sk | | fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk @@ -437,7 +437,7 @@ Per-Host Resources: mem-estimate=205.57MB mem-reservation=19.75MB thread-reserva | | | in pipelines: 03(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.10MB mem-reservation=512.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_quarter_name = '2001Q1' @@ -475,7 +475,7 @@ Per-Host Resources: mem-estimate=205.57MB mem-reservation=19.75MB thread-reserva in pipelines: 02(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=93.14MB Threads=22 -Per-Host Resource Estimates: Memory=293MB +Per-Host Resource Estimates: Memory=305MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -510,7 +510,7 @@ Per-Instance Resources: mem-estimate=10.82MB mem-reservation=1.94MB thread-reser | in pipelines: 02(GETNEXT) | F09:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=13.37MB mem-reservation=2.00MB thread-reservation=1 15:AGGREGATE [STREAMING] | output: count(ss_quantity), avg(CAST(ss_quantity AS BIGINT)), stddev_samp(ss_quantity), count(sr_return_quantity), avg(CAST(sr_return_quantity AS BIGINT)), stddev_samp(sr_return_quantity), count(cs_quantity), avg(CAST(cs_quantity AS BIGINT)), stddev_samp(cs_quantity) | group by: i_item_id, i_item_desc, s_state @@ -540,7 +540,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser | | in pipelines: 07(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.79MB mem-reservation=2.00MB thread-reservation=1 | 07:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -558,7 +558,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=50.25MB mem-reservation=8.00MB thread-reservation=1 13:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=01 | hash predicates: ss_store_sk = s_store_sk @@ -581,7 +581,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 06(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | 06:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -614,7 +614,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.10MB mem-reservation=512.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d3.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') @@ -650,7 +650,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F04:PLAN FRAGMENT [HASH(ss_customer_sk,ss_item_sk,ss_ticket_number)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=2.83MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=3.30MB mem-reservation=0B thread-reservation=1 | 10:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=04 | | hash predicates: sr_returned_date_sk = d2.d_date_sk @@ -673,7 +673,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | in pipelines: 04(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.10MB mem-reservation=512.00KB thread-reservation=1 | | 04:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d2.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') @@ -710,7 +710,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.75MB mem-reservation=8.00MB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF011[min_max] -> sr_returned_date_sk, RF001[min_max] -> tpcds_parquet.store_returns.sr_item_sk, RF010[bloom] -> sr_returned_date_sk, RF000[bloom] -> tpcds_parquet.store_returns.sr_item_sk @@ -729,7 +729,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=6.00MB mem-reservation=6.00MB thread-reservation=0 runtime-filters-memory=6.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.45MB mem-reservation=4.00MB thread-reservation=1 | 08:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=06 | | hash predicates: ss_sold_date_sk = d1.d_date_sk @@ -752,7 +752,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | in pipelines: 03(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.10MB mem-reservation=512.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_quarter_name = '2001Q1' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test index 61a65d10e..a2502d510 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test @@ -207,7 +207,7 @@ PLAN-ROOT SINK in pipelines: 02(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=194.25MB Threads=18 -Per-Host Resource Estimates: Memory=886MB +Per-Host Resource Estimates: Memory=888MB F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -297,7 +297,7 @@ Per-Host Resources: mem-estimate=98.17MB mem-reservation=75.94MB thread-reservat | | in pipelines: 06(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.16MB mem-reservation=512.00KB thread-reservation=2 | 06:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -322,7 +322,7 @@ Per-Host Resources: mem-estimate=98.17MB mem-reservation=75.94MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.09MB mem-reservation=4.00MB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.customer_demographics cd2, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | stored statistics: @@ -354,7 +354,7 @@ Per-Host Resources: mem-estimate=19.13MB mem-reservation=9.50MB thread-reservati | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.55MB mem-reservation=8.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.customer_demographics cd1, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd1.cd_gender = 'M', cd1.cd_education_status = 'College' @@ -374,7 +374,7 @@ Per-Host Resources: mem-estimate=19.13MB mem-reservation=9.50MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=399.14MB mem-reservation=29.62MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=399.42MB mem-reservation=29.62MB thread-reservation=2 runtime-filters-memory=5.00MB 09:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: c_current_addr_sk = ca_address_sk | fk/pk conjuncts: c_current_addr_sk = ca_address_sk @@ -389,7 +389,7 @@ Per-Host Resources: mem-estimate=399.14MB mem-reservation=29.62MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.29MB mem-reservation=512.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_state IN ('ND', 'WI', 'AL', 'NC', 'OK', 'MS', 'TN') @@ -417,7 +417,7 @@ Per-Host Resources: mem-estimate=399.14MB mem-reservation=29.62MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=82.00MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=82.09MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB | 03:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | predicates: c_birth_month IN (CAST(9 AS INT), CAST(5 AS INT), CAST(12 AS INT), CAST(4 AS INT), CAST(1 AS INT), CAST(10 AS INT)) @@ -446,7 +446,7 @@ Per-Host Resources: mem-estimate=399.14MB mem-reservation=29.62MB thread-reserva | | in pipelines: 05(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT) @@ -472,7 +472,7 @@ Per-Host Resources: mem-estimate=399.14MB mem-reservation=29.62MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=206.81MB Threads=17 -Per-Host Resource Estimates: Memory=371MB +Per-Host Resource Estimates: Memory=384MB F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -527,7 +527,7 @@ Per-Instance Resources: mem-estimate=75.94MB mem-reservation=57.69MB thread-rese | in pipelines: 00(GETNEXT) | F07:PLAN FRAGMENT [HASH(c_current_cdemo_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=52.24MB mem-reservation=38.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=59.98MB mem-reservation=38.00MB thread-reservation=1 13:AGGREGATE [STREAMING] | Class 0 | output: avg(CAST(cs_quantity AS DECIMAL(12,2))), avg(CAST(cs_list_price AS DECIMAL(12,2))), avg(CAST(cs_coupon_amt AS DECIMAL(12,2))), avg(CAST(cs_sales_price AS DECIMAL(12,2))), avg(CAST(cs_net_profit AS DECIMAL(12,2))), avg(CAST(c_birth_year AS DECIMAL(12,2))), avg(CAST(cd1.cd_dep_count AS DECIMAL(12,2))) @@ -570,7 +570,7 @@ Per-Instance Resources: mem-estimate=52.24MB mem-reservation=38.00MB thread-rese | | in pipelines: 06(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | 06:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -603,7 +603,7 @@ Per-Instance Resources: mem-estimate=52.24MB mem-reservation=38.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=4.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.customer_demographics cd2, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | stored statistics: @@ -620,7 +620,7 @@ Per-Instance Resources: mem-estimate=52.24MB mem-reservation=38.00MB thread-rese | in pipelines: 00(GETNEXT) | F05:PLAN FRAGMENT [HASH(cs_bill_cdemo_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=1.71MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=4.05MB mem-reservation=0B thread-reservation=1 10:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=02 | hash predicates: cs_bill_cdemo_sk = cd1.cd_demo_sk @@ -643,7 +643,7 @@ Per-Instance Resources: mem-estimate=1.71MB mem-reservation=0B thread-reservatio | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.55MB mem-reservation=8.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.customer_demographics cd1, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd1.cd_gender = 'M', cd1.cd_education_status = 'College' @@ -664,7 +664,7 @@ Per-Instance Resources: mem-estimate=1.71MB mem-reservation=0B thread-reservatio | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=49.79MB mem-reservation=16.00MB thread-reservation=1 09:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=03 | hash predicates: c_current_addr_sk = ca_address_sk @@ -687,7 +687,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-rese | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.29MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_state IN ('ND', 'WI', 'AL', 'NC', 'OK', 'MS', 'TN') @@ -724,7 +724,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-rese | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=2.00MB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | predicates: c_birth_month IN (CAST(9 AS INT), CAST(5 AS INT), CAST(12 AS INT), CAST(4 AS INT), CAST(1 AS INT), CAST(10 AS INT)) @@ -761,7 +761,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-rese | | in pipelines: 05(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test index 6015548ba..294ef1339 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test @@ -168,7 +168,7 @@ PLAN-ROOT SINK in pipelines: 04(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=31.08MB Threads=16 -Per-Host Resource Estimates: Memory=350MB +Per-Host Resource Estimates: Memory=353MB F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -226,7 +226,7 @@ Per-Host Resources: mem-estimate=17.36MB mem-reservation=7.88MB thread-reservati | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.10MB mem-reservation=16.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -251,7 +251,7 @@ Per-Host Resources: mem-estimate=17.36MB mem-reservation=7.88MB thread-reservati | | in pipelines: 04(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.29MB mem-reservation=512.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -268,7 +268,7 @@ Per-Host Resources: mem-estimate=17.36MB mem-reservation=7.88MB thread-reservati | in pipelines: 01(GETNEXT) | F04:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=5.06MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=5.83MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB 08:HASH JOIN [INNER JOIN, PARTITIONED] | hash predicates: ss_customer_sk = c_customer_sk | fk/pk conjuncts: ss_customer_sk = c_customer_sk @@ -283,7 +283,7 @@ Per-Host Resources: mem-estimate=5.06MB mem-reservation=3.88MB thread-reservatio | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.14MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 03:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF003[min_max] -> c_current_addr_sk, RF002[bloom] -> c_current_addr_sk @@ -301,7 +301,7 @@ Per-Host Resources: mem-estimate=5.06MB mem-reservation=3.88MB thread-reservatio | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=71.92MB mem-reservation=9.88MB thread-reservation=2 runtime-filters-memory=4.00MB +Per-Host Resources: mem-estimate=73.28MB mem-reservation=9.88MB thread-reservation=2 runtime-filters-memory=4.00MB 07:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_sold_date_sk = d_date_sk | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -316,7 +316,7 @@ Per-Host Resources: mem-estimate=71.92MB mem-reservation=9.88MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT) @@ -344,7 +344,7 @@ Per-Host Resources: mem-estimate=71.92MB mem-reservation=9.88MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=96.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=96.30MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_manager_id = CAST(7 AS INT) @@ -371,7 +371,7 @@ Per-Host Resources: mem-estimate=71.92MB mem-reservation=9.88MB thread-reservati in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=49.77MB Threads=21 -Per-Host Resource Estimates: Memory=195MB +Per-Host Resource Estimates: Memory=212MB F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -406,7 +406,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | in pipelines: 01(GETNEXT) | F06:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=10.83MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=12.70MB mem-reservation=2.00MB thread-reservation=1 11:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: i_brand, i_brand_id, i_manufact_id, i_manufact @@ -437,7 +437,7 @@ Per-Instance Resources: mem-estimate=10.83MB mem-reservation=2.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.10MB mem-reservation=16.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -470,7 +470,7 @@ Per-Instance Resources: mem-estimate=10.83MB mem-reservation=2.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.59MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -487,7 +487,7 @@ Per-Instance Resources: mem-estimate=10.83MB mem-reservation=2.00MB thread-reser | in pipelines: 01(GETNEXT) | F04:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=777.31KB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=3.84MB mem-reservation=0B thread-reservation=1 08:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=02 | hash predicates: ss_customer_sk = c_customer_sk @@ -511,7 +511,7 @@ Per-Instance Resources: mem-estimate=777.31KB mem-reservation=0B thread-reservat | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.28MB mem-reservation=1.00MB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF003[min_max] -> c_current_addr_sk, RF002[bloom] -> c_current_addr_sk @@ -530,7 +530,7 @@ Per-Instance Resources: mem-estimate=777.31KB mem-reservation=0B thread-reservat | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=18.80MB mem-reservation=2.00MB thread-reservation=1 07:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=03 | hash predicates: ss_sold_date_sk = d_date_sk @@ -553,7 +553,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT) @@ -589,7 +589,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.30MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_manager_id = CAST(7 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test index 703e9963a..8d82b56ff 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test @@ -117,7 +117,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=105.81MB Threads=8 -Per-Host Resource Estimates: Memory=383MB +Per-Host Resource Estimates: Memory=385MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -165,7 +165,7 @@ Per-Host Resources: mem-estimate=65.24MB mem-reservation=50.00MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=186.19MB mem-reservation=48.81MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=187.18MB mem-reservation=48.81MB thread-reservation=2 runtime-filters-memory=2.00MB 05:AGGREGATE [STREAMING] | output: sum(cs_ext_sales_price) | group by: i_item_id, i_item_desc, i_category, i_class, i_current_price @@ -187,7 +187,7 @@ Per-Host Resources: mem-estimate=186.19MB mem-reservation=48.81MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '1999-03-24', CAST(d_date AS DATE) >= DATE '1999-02-22' @@ -215,7 +215,7 @@ Per-Host Resources: mem-estimate=186.19MB mem-reservation=48.81MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=96.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=96.76MB mem-reservation=2.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category IN ('Sports', 'Books', 'Home') @@ -241,7 +241,7 @@ Per-Host Resources: mem-estimate=186.19MB mem-reservation=48.81MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=112.62MB Threads=7 -Per-Host Resource Estimates: Memory=198MB +Per-Host Resource Estimates: Memory=202MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -290,7 +290,7 @@ Per-Instance Resources: mem-estimate=65.24MB mem-reservation=50.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=82.00MB mem-reservation=42.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=84.37MB mem-reservation=42.00MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(cs_ext_sales_price) | group by: i_item_id, i_item_desc, i_category, i_class, i_current_price @@ -320,7 +320,7 @@ Per-Instance Resources: mem-estimate=82.00MB mem-reservation=42.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '1999-03-24', CAST(d_date AS DATE) >= DATE '1999-02-22' @@ -356,7 +356,7 @@ Per-Instance Resources: mem-estimate=82.00MB mem-reservation=42.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.76MB mem-reservation=2.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category IN ('Sports', 'Books', 'Home') diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test index 32ecb020c..4e71a3d6c 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test @@ -161,7 +161,7 @@ Per-Host Resources: mem-estimate=11.39MB mem-reservation=2.88MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -Per-Host Resources: mem-estimate=147.15MB mem-reservation=29.81MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=147.42MB mem-reservation=29.81MB thread-reservation=2 runtime-filters-memory=3.00MB 07:AGGREGATE [STREAMING] | output: sum(CAST(CASE WHEN (CAST(d_date AS DATE) < DATE '2000-03-11') THEN inv_quantity_on_hand ELSE CAST(0 AS INT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(d_date AS DATE) >= DATE '2000-03-11') THEN inv_quantity_on_hand ELSE CAST(0 AS INT) END AS BIGINT)) | group by: w_warehouse_name, i_item_id @@ -183,7 +183,7 @@ Per-Host Resources: mem-estimate=147.15MB mem-reservation=29.81MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '2000-04-10', CAST(d_date AS DATE) >= DATE '2000-02-10' @@ -211,7 +211,7 @@ Per-Host Resources: mem-estimate=147.15MB mem-reservation=29.81MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.15MB mem-reservation=16.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | stored statistics: @@ -236,7 +236,7 @@ Per-Host Resources: mem-estimate=147.15MB mem-reservation=29.81MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.17MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_current_price <= CAST(1.49 AS DECIMAL(3,2)), i_current_price >= CAST(0.99 AS DECIMAL(2,2)) @@ -262,7 +262,7 @@ Per-Host Resources: mem-estimate=147.15MB mem-reservation=29.81MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=47.02MB Threads=9 -Per-Host Resource Estimates: Memory=123MB +Per-Host Resource Estimates: Memory=124MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -299,7 +299,7 @@ Per-Instance Resources: mem-estimate=11.39MB mem-reservation=2.88MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=42.00MB mem-reservation=21.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=42.61MB mem-reservation=21.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | output: sum(CAST(CASE WHEN (CAST(d_date AS DATE) < DATE '2000-03-11') THEN inv_quantity_on_hand ELSE CAST(0 AS INT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(d_date AS DATE) >= DATE '2000-03-11') THEN inv_quantity_on_hand ELSE CAST(0 AS INT) END AS BIGINT)) | group by: w_warehouse_name, i_item_id @@ -329,7 +329,7 @@ Per-Instance Resources: mem-estimate=42.00MB mem-reservation=21.00MB thread-rese | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '2000-04-10', CAST(d_date AS DATE) >= DATE '2000-02-10' @@ -365,7 +365,7 @@ Per-Instance Resources: mem-estimate=42.00MB mem-reservation=21.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.15MB mem-reservation=16.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | stored statistics: @@ -398,7 +398,7 @@ Per-Instance Resources: mem-estimate=42.00MB mem-reservation=21.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.17MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_current_price <= CAST(1.49 AS DECIMAL(3,2)), i_current_price >= CAST(0.99 AS DECIMAL(2,2)) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q22.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q22.test index 966b45067..cd555434a 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q22.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q22.test @@ -219,7 +219,7 @@ Per-Host Resources: mem-estimate=5.50GB mem-reservation=140.62MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | stored statistics: @@ -244,7 +244,7 @@ Per-Host Resources: mem-estimate=5.50GB mem-reservation=140.62MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=80.42MB mem-reservation=1.00MB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -269,7 +269,7 @@ Per-Host Resources: mem-estimate=5.50GB mem-reservation=140.62MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1223 AS INT), d_month_seq >= CAST(1212 AS INT) @@ -351,7 +351,7 @@ Per-Instance Resources: mem-estimate=20.49GB mem-reservation=146.44MB thread-res | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=5.39GB mem-reservation=129.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=5.40GB mem-reservation=129.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | Class 0 | output: avg(CAST(inv_quantity_on_hand AS BIGINT)) @@ -394,7 +394,7 @@ Per-Instance Resources: mem-estimate=5.39GB mem-reservation=129.00MB thread-rese | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | stored statistics: @@ -427,7 +427,7 @@ Per-Instance Resources: mem-estimate=5.39GB mem-reservation=129.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.42MB mem-reservation=1.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -460,7 +460,7 @@ Per-Instance Resources: mem-estimate=5.39GB mem-reservation=129.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1223 AS INT), d_month_seq >= CAST(1212 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test index a501f7c83..44f67cb3a 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test @@ -553,7 +553,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | in pipelines: 25(GETNEXT) | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=165.06MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=165.39MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=1.00MB | | 47:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | | hash predicates: ws_item_sk = i_item_sk | | | runtime filters: RF025[min_max] <- i_item_sk @@ -623,7 +623,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | | | in pipelines: 29(GETNEXT) | | | | | | | | | F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=2.00MB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.49MB mem-reservation=2.00MB thread-reservation=2 | | | | 29:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -648,7 +648,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | | | in pipelines: 28(GETNEXT) | | | | | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=48.13MB mem-reservation=2.00MB thread-reservation=2 | | | | 28:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -688,7 +688,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | | in pipelines: 26(GETNEXT) | | | | | | | F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | | 26:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT) @@ -725,7 +725,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | in pipelines: 77(GETNEXT) | | | | | F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | 77:AGGREGATE [FINALIZE] | | | output: max:merge(csales) | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -780,7 +780,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | | in pipelines: 38(GETNEXT) | | | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=2 | | | 38:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | stored statistics: @@ -805,7 +805,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | | in pipelines: 39(GETNEXT) | | | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 39:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -866,7 +866,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | in pipelines: 34(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=17.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=17.03MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | | 34:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF017[min_max] -> tpcds_parquet.customer.c_customer_sk, RF016[bloom] -> tpcds_parquet.customer.c_customer_sk @@ -903,7 +903,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | in pipelines: 01(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=245.06MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=245.39MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=1.00MB | 23:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | hash predicates: cs_item_sk = i_item_sk | | runtime filters: RF009[min_max] <- i_item_sk @@ -973,7 +973,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | | in pipelines: 05(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=2.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.49MB mem-reservation=2.00MB thread-reservation=2 | | | 05:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -998,7 +998,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | | in pipelines: 04(GETNEXT) | | | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=48.13MB mem-reservation=2.00MB thread-reservation=2 | | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -1038,7 +1038,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | in pipelines: 02(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT) @@ -1075,7 +1075,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | in pipelines: 58(GETNEXT) | | | F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 58:AGGREGATE [FINALIZE] | | output: max:merge(csales) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1130,7 +1130,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | in pipelines: 14(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=2 | | 14:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -1155,7 +1155,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati | | | in pipelines: 15(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -1216,7 +1216,7 @@ Per-Host Resources: mem-estimate=65.14MB mem-reservation=12.75MB thread-reservat | | in pipelines: 10(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=17.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=17.03MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | 10:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_sk @@ -1241,7 +1241,7 @@ Per-Host Resources: mem-estimate=65.14MB mem-reservation=12.75MB thread-reservat in pipelines: 09(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=497.12MB Threads=63 -Per-Host Resource Estimates: Memory=1.11GB +Per-Host Resource Estimates: Memory=1.12GB F31:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -1295,7 +1295,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=33.03MB mem-reservation=8.00MB thread-reservation=1 | | 47:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | | hash-table-id=09 | | | hash predicates: ws_item_sk = i_item_sk @@ -1352,7 +1352,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | | | | F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | | Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=51.27MB mem-reservation=34.50MB thread-reservation=1 | | | 32:AGGREGATE [STREAMING] | | | | output: count(*) | | | | group by: substr(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), i_item_sk, d_date @@ -1382,7 +1382,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | | in pipelines: 29(GETNEXT) | | | | | | | | | F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1 | | | | 29:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | | HDFS partitions=1/1 files=1 size=1.73MB | | | | stored statistics: @@ -1415,7 +1415,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | | in pipelines: 28(GETNEXT) | | | | | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | | | | 28:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -1463,7 +1463,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 26(GETNEXT) | | | | | | | F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | | 26:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT) @@ -1507,7 +1507,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | in pipelines: 77(GETNEXT) | | | | | F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | 77:AGGREGATE [FINALIZE] | | | output: max:merge(csales) | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1541,7 +1541,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=26.56MB mem-reservation=6.00MB thread-reservation=1 | | 42:AGGREGATE [STREAMING] | | | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price) | | | group by: c_customer_sk @@ -1571,7 +1571,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 38(GETNEXT) | | | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1 | | | 38:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | stored statistics: @@ -1604,7 +1604,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 39(GETNEXT) | | | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 39:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -1644,7 +1644,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.56MB mem-reservation=6.00MB thread-reservation=1 | 36:AGGREGATE [STREAMING] | | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price) | | group by: c_customer_sk @@ -1675,7 +1675,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1 | | 34:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF017[min_max] -> tpcds_parquet.customer.c_customer_sk, RF016[bloom] -> tpcds_parquet.customer.c_customer_sk @@ -1721,7 +1721,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=49.03MB mem-reservation=8.00MB thread-reservation=1 | 23:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | hash-table-id=01 | | hash predicates: cs_item_sk = i_item_sk @@ -1778,7 +1778,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=51.27MB mem-reservation=34.50MB thread-reservation=1 | | 08:AGGREGATE [STREAMING] | | | output: count(*) | | | group by: substr(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), i_item_sk, d_date @@ -1808,7 +1808,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 05(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1 | | | 05:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | stored statistics: @@ -1841,7 +1841,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 04(GETNEXT) | | | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -1889,7 +1889,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | in pipelines: 02(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT) @@ -1933,7 +1933,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | in pipelines: 58(GETNEXT) | | | F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 58:AGGREGATE [FINALIZE] | | output: max:merge(csales) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1967,7 +1967,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.56MB mem-reservation=6.00MB thread-reservation=1 | 18:AGGREGATE [STREAMING] | | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price) | | group by: c_customer_sk @@ -1997,7 +1997,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | in pipelines: 14(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -2030,7 +2030,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | | | in pipelines: 15(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -2070,7 +2070,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.56MB mem-reservation=6.00MB thread-reservation=1 12:AGGREGATE [STREAMING] | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price) | group by: c_customer_sk @@ -2101,7 +2101,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1 | 10:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_sk diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test index cd9e784f6..3f74ceb9e 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test @@ -585,7 +585,7 @@ PLAN-ROOT SINK in pipelines: 04(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=293.69MB Threads=56 -Per-Host Resource Estimates: Memory=1.46GB +Per-Host Resource Estimates: Memory=1.47GB F35:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -663,7 +663,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati | | | | in pipelines: 28(GETNEXT) | | | | | | | F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=48.34MB mem-reservation=1.00MB thread-reservation=2 | | | 28:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | stored statistics: @@ -680,7 +680,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati | | | in pipelines: 27(GETNEXT) | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=166.35MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=166.53MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB | | 49:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: ws_bill_customer_sk = c_customer_sk | | | fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk @@ -710,7 +710,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati | | | | in pipelines: 36(GETNEXT) | | | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | | Per-Host Resources: mem-estimate=63.15MB mem-reservation=14.75MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=63.23MB mem-reservation=14.75MB thread-reservation=2 | | | 47:AGGREGATE [STREAMING] | | | | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales)) | | | | group by: c_customer_sk @@ -729,7 +729,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati | | | | | in pipelines: 86(GETNEXT) | | | | | | | | | F29:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | | | Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | | | 86:AGGREGATE [FINALIZE] | | | | | output: max:merge(csales) | | | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -784,7 +784,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati | | | | | | in pipelines: 39(GETNEXT) | | | | | | | | | | | F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | | Per-Host Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | | Per-Host Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=2 | | | | | 39:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | | | HDFS partitions=1/1 files=1 size=5.49MB | | | | | stored statistics: @@ -809,7 +809,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati | | | | | | in pipelines: 40(GETNEXT) | | | | | | | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | | 40:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -849,7 +849,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati | | | | | in pipelines: 37(GETNEXT) | | | | | | | | | F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=2 | | | | 37:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | | HDFS partitions=1/1 files=1 size=5.49MB | | | | runtime filters: RF025[min_max] -> tpcds_parquet.customer.c_customer_sk @@ -887,7 +887,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati | | | | in pipelines: 29(GETNEXT) | | | | | | | F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | | 29:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT) @@ -948,7 +948,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati | | | in pipelines: 32(GETNEXT) | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.49MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 32:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF019[min_max] -> tpcds_parquet.item.i_item_sk, RF018[bloom] -> tpcds_parquet.item.i_item_sk @@ -974,7 +974,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati | | | in pipelines: 31(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.13MB mem-reservation=2.00MB thread-reservation=2 | | 31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -1050,7 +1050,7 @@ Per-Host Resources: mem-estimate=71.94MB mem-reservation=39.88MB thread-reservat | | | in pipelines: 02(GETNEXT) | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.52MB mem-reservation=1.00MB thread-reservation=2 | | 02:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -1067,7 +1067,7 @@ Per-Host Resources: mem-estimate=71.94MB mem-reservation=39.88MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=246.35MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=246.86MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 23:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: cs_bill_customer_sk = c_customer_sk | | fk/pk conjuncts: cs_bill_customer_sk = c_customer_sk @@ -1097,7 +1097,7 @@ Per-Host Resources: mem-estimate=71.94MB mem-reservation=39.88MB thread-reservat | | | in pipelines: 10(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | Per-Host Resources: mem-estimate=63.15MB mem-reservation=14.75MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=63.23MB mem-reservation=14.75MB thread-reservation=2 | | 21:AGGREGATE [STREAMING] | | | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales)) | | | group by: c_customer_sk @@ -1116,7 +1116,7 @@ Per-Host Resources: mem-estimate=71.94MB mem-reservation=39.88MB thread-reservat | | | | in pipelines: 65(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | | Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | | 65:AGGREGATE [FINALIZE] | | | | output: max:merge(csales) | | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1171,7 +1171,7 @@ Per-Host Resources: mem-estimate=71.94MB mem-reservation=39.88MB thread-reservat | | | | | in pipelines: 13(GETNEXT) | | | | | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=2 | | | | 13:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | | HDFS partitions=1/1 files=1 size=5.49MB | | | | stored statistics: @@ -1196,7 +1196,7 @@ Per-Host Resources: mem-estimate=71.94MB mem-reservation=39.88MB thread-reservat | | | | | in pipelines: 14(GETNEXT) | | | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | | 14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -1236,7 +1236,7 @@ Per-Host Resources: mem-estimate=71.94MB mem-reservation=39.88MB thread-reservat | | | | in pipelines: 11(GETNEXT) | | | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=2 | | | 11:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF007[min_max] -> tpcds_parquet.customer.c_customer_sk @@ -1274,7 +1274,7 @@ Per-Host Resources: mem-estimate=71.94MB mem-reservation=39.88MB thread-reservat | | | in pipelines: 03(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT) @@ -1335,7 +1335,7 @@ Per-Host Resources: mem-estimate=61.76MB mem-reservation=43.19MB thread-reservat | | in pipelines: 06(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.49MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 06:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk @@ -1361,7 +1361,7 @@ Per-Host Resources: mem-estimate=61.76MB mem-reservation=43.19MB thread-reservat | | in pipelines: 05(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.13MB mem-reservation=2.00MB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -1388,7 +1388,7 @@ Per-Host Resources: mem-estimate=61.76MB mem-reservation=43.19MB thread-reservat in pipelines: 04(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=530.38MB Threads=67 -Per-Host Resource Estimates: Memory=1.16GB +Per-Host Resource Estimates: Memory=1.18GB F35:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -1459,7 +1459,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | in pipelines: 27(GETNEXT) | | | | | F32:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2 -| | Per-Instance Resources: mem-estimate=506.41KB mem-reservation=0B thread-reservation=1 +| | Per-Instance Resources: mem-estimate=1018.27KB mem-reservation=0B thread-reservation=1 | | 50:HASH JOIN [INNER JOIN, PARTITIONED] | | | hash-table-id=11 | | | hash predicates: ws_bill_customer_sk = customer.c_customer_sk @@ -1482,7 +1482,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 28(GETNEXT) | | | | | | | F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.34MB mem-reservation=1.00MB thread-reservation=1 | | | 28:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | stored statistics: @@ -1500,7 +1500,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=32.66MB mem-reservation=8.00MB thread-reservation=1 | | 49:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=12 | | | hash predicates: ws_bill_customer_sk = c_customer_sk @@ -1538,7 +1538,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 36(GETNEXT) | | | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| | | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=26.94MB mem-reservation=10.00MB thread-reservation=1 | | | 47:AGGREGATE [STREAMING] | | | | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales)) | | | | group by: c_customer_sk @@ -1564,7 +1564,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | | | in pipelines: 86(GETNEXT) | | | | | | | | | F29:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | | | 86:AGGREGATE [FINALIZE] | | | | | output: max:merge(csales) | | | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1598,7 +1598,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | | | | | | | F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | | | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=26.56MB mem-reservation=6.00MB thread-reservation=1 | | | | 43:AGGREGATE [STREAMING] | | | | | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price) | | | | | group by: c_customer_sk @@ -1628,7 +1628,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | | | | in pipelines: 39(GETNEXT) | | | | | | | | | | | F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1 | | | | | 39:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | | | HDFS partitions=1/1 files=1 size=5.49MB | | | | | stored statistics: @@ -1661,7 +1661,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | | | | in pipelines: 40(GETNEXT) | | | | | | | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | | 40:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -1709,7 +1709,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | | | in pipelines: 37(GETNEXT) | | | | | | | | | F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1 | | | | 37:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | | HDFS partitions=1/1 files=1 size=5.49MB | | | | runtime filters: RF025[min_max] -> tpcds_parquet.customer.c_customer_sk @@ -1755,7 +1755,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 29(GETNEXT) | | | | | | | F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | | 29:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT) @@ -1795,7 +1795,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=51.27MB mem-reservation=34.50MB thread-reservation=1 | 35:AGGREGATE [STREAMING] | | output: count(*) | | group by: substring(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), tpcds_parquet.item.i_item_sk, d_date @@ -1826,7 +1826,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1 | | 32:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF019[min_max] -> tpcds_parquet.item.i_item_sk, RF018[bloom] -> tpcds_parquet.item.i_item_sk @@ -1860,7 +1860,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser | | | in pipelines: 31(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | | 31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -1929,7 +1929,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F15:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3 -| Per-Instance Resources: mem-estimate=498.70KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=1010.56KB mem-reservation=0B thread-reservation=1 | 24:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=01 | | hash predicates: cs_bill_customer_sk = customer.c_customer_sk @@ -1952,7 +1952,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese | | | in pipelines: 02(GETNEXT) | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.52MB mem-reservation=1.00MB thread-reservation=1 | | 02:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -1970,7 +1970,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese | | | F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=48.98MB mem-reservation=8.00MB thread-reservation=1 | 23:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=02 | | hash predicates: cs_bill_customer_sk = c_customer_sk @@ -2008,7 +2008,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese | | | in pipelines: 10(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=26.94MB mem-reservation=10.00MB thread-reservation=1 | | 21:AGGREGATE [STREAMING] | | | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales)) | | | group by: c_customer_sk @@ -2034,7 +2034,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese | | | | in pipelines: 65(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | | | 65:AGGREGATE [FINALIZE] | | | | output: max:merge(csales) | | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -2068,7 +2068,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=26.56MB mem-reservation=6.00MB thread-reservation=1 | | | 17:AGGREGATE [STREAMING] | | | | output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price) | | | | group by: c_customer_sk @@ -2098,7 +2098,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese | | | | | in pipelines: 13(GETNEXT) | | | | | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1 | | | | 13:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | | HDFS partitions=1/1 files=1 size=5.49MB | | | | stored statistics: @@ -2131,7 +2131,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese | | | | | in pipelines: 14(GETNEXT) | | | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | | 14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | | HDFS partitions=1/1 files=1 size=2.15MB | | | | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) @@ -2179,7 +2179,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese | | | | in pipelines: 11(GETNEXT) | | | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1 | | | 11:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF007[min_max] -> tpcds_parquet.customer.c_customer_sk @@ -2225,7 +2225,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese | | | in pipelines: 03(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT) @@ -2265,7 +2265,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-reservation=1 +Per-Instance Resources: mem-estimate=51.27MB mem-reservation=34.50MB thread-reservation=1 09:AGGREGATE [STREAMING] | output: count(*) | group by: substring(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), tpcds_parquet.item.i_item_sk, d_date @@ -2296,7 +2296,7 @@ Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-rese | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1 | 06:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk @@ -2330,7 +2330,7 @@ Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-rese | | in pipelines: 05(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT)) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test index c146642f8..a4857b8be 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test @@ -354,7 +354,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | in pipelines: 48(GETNEXT) | | | F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 48:AGGREGATE [FINALIZE] | | output: avg:merge(netpaid) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -410,7 +410,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 18(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.25MB mem-reservation=512.00KB thread-reservation=2 | | 18:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -435,7 +435,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 17(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=80.27MB mem-reservation=2.00MB thread-reservation=2 | | 17:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF016[min_max] -> c_current_addr_sk @@ -461,7 +461,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 16(GETNEXT) | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=96.27MB mem-reservation=256.00KB thread-reservation=2 | | 16:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -486,7 +486,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 14(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.08MB mem-reservation=4.00MB thread-reservation=2 | | 14:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF021[min_max] -> tpcds_parquet.store_returns.sr_item_sk @@ -512,7 +512,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=80.00MB mem-reservation=40.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=80.23MB mem-reservation=40.00KB thread-reservation=2 | | 15:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: s_market_id = CAST(8 AS INT) @@ -552,7 +552,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | in pipelines: 37(GETNEXT) | F09:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=20.84MB mem-reservation=3.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=20.85MB mem-reservation=3.94MB thread-reservation=1 12:AGGREGATE [STREAMING] | output: sum(sum(ss_net_paid)) | group by: c_last_name, c_first_name, s_store_name @@ -596,7 +596,7 @@ Per-Host Resources: mem-estimate=19.09MB mem-reservation=6.88MB thread-reservati | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.75MB mem-reservation=512.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -628,7 +628,7 @@ Per-Host Resources: mem-estimate=12.92MB mem-reservation=5.75MB thread-reservati | | in pipelines: 04(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=81.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=81.80MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 04:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF002[min_max] -> c_current_addr_sk, RF000[bloom] -> c_current_addr_sk @@ -661,7 +661,7 @@ Per-Host Resources: mem-estimate=12.12MB mem-reservation=6.75MB thread-reservati | | in pipelines: 01(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.23MB mem-reservation=4.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | stored statistics: @@ -678,7 +678,7 @@ Per-Host Resources: mem-estimate=12.12MB mem-reservation=6.75MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=90.76MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=5.00MB 07:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_store_sk = s_store_sk | fk/pk conjuncts: ss_store_sk = s_store_sk @@ -693,7 +693,7 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=81.00MB mem-reservation=1.04MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=81.23MB mem-reservation=1.04MB thread-reservation=2 runtime-filters-memory=1.00MB | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_market_id = CAST(8 AS INT) @@ -722,7 +722,7 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=97.00MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=97.27MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: tpcds_parquet.item.i_color = 'peach' @@ -750,7 +750,7 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=319.08MB Threads=42 -Per-Host Resource Estimates: Memory=671MB +Per-Host Resource Estimates: Memory=719MB F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.59MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -790,7 +790,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | in pipelines: 48(GETNEXT) | | | F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 48:AGGREGATE [FINALIZE] | | output: avg:merge(netpaid) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -824,7 +824,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=33.00MB mem-reservation=21.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=36.71MB mem-reservation=21.00MB thread-reservation=1 | 24:AGGREGATE [STREAMING] | | output: sum(ss_net_paid) | | group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size @@ -855,7 +855,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | in pipelines: 18(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.25MB mem-reservation=512.00KB thread-reservation=1 | | 18:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -888,7 +888,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | in pipelines: 17(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.27MB mem-reservation=2.00MB thread-reservation=1 | | 17:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF016[min_max] -> c_current_addr_sk @@ -922,7 +922,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | in pipelines: 16(GETNEXT) | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.27MB mem-reservation=256.00KB thread-reservation=1 | | 16:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -955,7 +955,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | in pipelines: 14(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.08MB mem-reservation=4.00MB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF021[min_max] -> tpcds_parquet.store_returns.sr_item_sk @@ -989,7 +989,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=40.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.23MB mem-reservation=40.00KB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: s_market_id = CAST(8 AS INT) @@ -1029,7 +1029,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | in pipelines: 37(GETNEXT) | F09:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=21.30MB mem-reservation=3.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=21.69MB mem-reservation=3.94MB thread-reservation=1 12:AGGREGATE [STREAMING] | output: sum(sum(ss_net_paid)) | group by: c_last_name, c_first_name, s_store_name @@ -1050,7 +1050,7 @@ Per-Instance Resources: mem-estimate=21.30MB mem-reservation=3.94MB thread-reser | in pipelines: 00(GETNEXT) | F08:PLAN FRAGMENT [HASH(c_current_addr_sk,s_zip)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=12.02MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=15.73MB mem-reservation=2.00MB thread-reservation=1 11:AGGREGATE [STREAMING] | output: sum(ss_net_paid) | group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size @@ -1081,7 +1081,7 @@ Per-Instance Resources: mem-estimate=12.02MB mem-reservation=2.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.50MB mem-reservation=512.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -1098,7 +1098,7 @@ Per-Instance Resources: mem-estimate=12.02MB mem-reservation=2.00MB thread-reser | in pipelines: 00(GETNEXT) | F06:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=1.46MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=7.32MB mem-reservation=0B thread-reservation=1 09:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=07 | hash predicates: ss_customer_sk = c_customer_sk @@ -1122,7 +1122,7 @@ Per-Instance Resources: mem-estimate=1.46MB mem-reservation=0B thread-reservatio | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.61MB mem-reservation=2.00MB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF002[min_max] -> c_current_addr_sk, RF000[bloom] -> c_current_addr_sk @@ -1140,7 +1140,7 @@ Per-Instance Resources: mem-estimate=1.46MB mem-reservation=0B thread-reservatio | in pipelines: 00(GETNEXT) | F04:PLAN FRAGMENT [HASH(ss_item_sk,ss_ticket_number)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=1.44MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=5.68MB mem-reservation=0B thread-reservation=1 08:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=08 | hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number @@ -1163,7 +1163,7 @@ Per-Instance Resources: mem-estimate=1.44MB mem-reservation=0B thread-reservatio | | in pipelines: 01(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.47MB mem-reservation=4.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | stored statistics: @@ -1181,7 +1181,7 @@ Per-Instance Resources: mem-estimate=1.44MB mem-reservation=0B thread-reservatio | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=19.78MB mem-reservation=4.00MB thread-reservation=1 07:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=09 | hash predicates: ss_store_sk = s_store_sk @@ -1205,7 +1205,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=40.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.23MB mem-reservation=40.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_market_id = CAST(8 AS INT) @@ -1243,7 +1243,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.27MB mem-reservation=256.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: tpcds_parquet.item.i_color = 'peach' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test index 8e19e37c4..92095cfb0 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test @@ -354,7 +354,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | in pipelines: 48(GETNEXT) | | | F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 48:AGGREGATE [FINALIZE] | | output: avg:merge(netpaid) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -410,7 +410,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 18(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.25MB mem-reservation=512.00KB thread-reservation=2 | | 18:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -435,7 +435,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 17(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=80.27MB mem-reservation=2.00MB thread-reservation=2 | | 17:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF016[min_max] -> c_current_addr_sk @@ -461,7 +461,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 16(GETNEXT) | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=96.27MB mem-reservation=256.00KB thread-reservation=2 | | 16:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -486,7 +486,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 14(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.08MB mem-reservation=4.00MB thread-reservation=2 | | 14:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF021[min_max] -> tpcds_parquet.store_returns.sr_item_sk @@ -512,7 +512,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=80.00MB mem-reservation=40.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=80.23MB mem-reservation=40.00KB thread-reservation=2 | | 15:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: s_market_id = CAST(8 AS INT) @@ -552,7 +552,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat | in pipelines: 37(GETNEXT) | F09:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=20.84MB mem-reservation=3.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=20.85MB mem-reservation=3.94MB thread-reservation=1 12:AGGREGATE [STREAMING] | output: sum(sum(ss_net_paid)) | group by: c_last_name, c_first_name, s_store_name @@ -596,7 +596,7 @@ Per-Host Resources: mem-estimate=19.09MB mem-reservation=6.88MB thread-reservati | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.75MB mem-reservation=512.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -628,7 +628,7 @@ Per-Host Resources: mem-estimate=12.92MB mem-reservation=5.75MB thread-reservati | | in pipelines: 04(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=81.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=81.80MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 04:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF002[min_max] -> c_current_addr_sk, RF000[bloom] -> c_current_addr_sk @@ -661,7 +661,7 @@ Per-Host Resources: mem-estimate=12.12MB mem-reservation=6.75MB thread-reservati | | in pipelines: 01(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.23MB mem-reservation=4.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | stored statistics: @@ -678,7 +678,7 @@ Per-Host Resources: mem-estimate=12.12MB mem-reservation=6.75MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=90.76MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=5.00MB 07:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_store_sk = s_store_sk | fk/pk conjuncts: ss_store_sk = s_store_sk @@ -693,7 +693,7 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=81.00MB mem-reservation=1.04MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=81.23MB mem-reservation=1.04MB thread-reservation=2 runtime-filters-memory=1.00MB | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_market_id = CAST(8 AS INT) @@ -722,7 +722,7 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=97.00MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=97.27MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: tpcds_parquet.item.i_color = 'saddle' @@ -750,7 +750,7 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=319.08MB Threads=42 -Per-Host Resource Estimates: Memory=671MB +Per-Host Resource Estimates: Memory=719MB F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.59MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -790,7 +790,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | in pipelines: 48(GETNEXT) | | | F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 48:AGGREGATE [FINALIZE] | | output: avg:merge(netpaid) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -824,7 +824,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=33.00MB mem-reservation=21.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=36.71MB mem-reservation=21.00MB thread-reservation=1 | 24:AGGREGATE [STREAMING] | | output: sum(ss_net_paid) | | group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size @@ -855,7 +855,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | in pipelines: 18(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.25MB mem-reservation=512.00KB thread-reservation=1 | | 18:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -888,7 +888,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | in pipelines: 17(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.27MB mem-reservation=2.00MB thread-reservation=1 | | 17:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF016[min_max] -> c_current_addr_sk @@ -922,7 +922,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | in pipelines: 16(GETNEXT) | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.27MB mem-reservation=256.00KB thread-reservation=1 | | 16:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -955,7 +955,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | in pipelines: 14(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.08MB mem-reservation=4.00MB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF021[min_max] -> tpcds_parquet.store_returns.sr_item_sk @@ -989,7 +989,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=40.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.23MB mem-reservation=40.00KB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: s_market_id = CAST(8 AS INT) @@ -1029,7 +1029,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | in pipelines: 37(GETNEXT) | F09:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=21.30MB mem-reservation=3.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=21.69MB mem-reservation=3.94MB thread-reservation=1 12:AGGREGATE [STREAMING] | output: sum(sum(ss_net_paid)) | group by: c_last_name, c_first_name, s_store_name @@ -1050,7 +1050,7 @@ Per-Instance Resources: mem-estimate=21.30MB mem-reservation=3.94MB thread-reser | in pipelines: 00(GETNEXT) | F08:PLAN FRAGMENT [HASH(c_current_addr_sk,s_zip)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=12.02MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=15.73MB mem-reservation=2.00MB thread-reservation=1 11:AGGREGATE [STREAMING] | output: sum(ss_net_paid) | group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size @@ -1081,7 +1081,7 @@ Per-Instance Resources: mem-estimate=12.02MB mem-reservation=2.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.50MB mem-reservation=512.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -1098,7 +1098,7 @@ Per-Instance Resources: mem-estimate=12.02MB mem-reservation=2.00MB thread-reser | in pipelines: 00(GETNEXT) | F06:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=1.46MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=7.32MB mem-reservation=0B thread-reservation=1 09:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=07 | hash predicates: ss_customer_sk = c_customer_sk @@ -1122,7 +1122,7 @@ Per-Instance Resources: mem-estimate=1.46MB mem-reservation=0B thread-reservatio | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.61MB mem-reservation=2.00MB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF002[min_max] -> c_current_addr_sk, RF000[bloom] -> c_current_addr_sk @@ -1140,7 +1140,7 @@ Per-Instance Resources: mem-estimate=1.46MB mem-reservation=0B thread-reservatio | in pipelines: 00(GETNEXT) | F04:PLAN FRAGMENT [HASH(ss_item_sk,ss_ticket_number)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=1.44MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=5.68MB mem-reservation=0B thread-reservation=1 08:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=08 | hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number @@ -1163,7 +1163,7 @@ Per-Instance Resources: mem-estimate=1.44MB mem-reservation=0B thread-reservatio | | in pipelines: 01(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.47MB mem-reservation=4.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | stored statistics: @@ -1181,7 +1181,7 @@ Per-Instance Resources: mem-estimate=1.44MB mem-reservation=0B thread-reservatio | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=19.78MB mem-reservation=4.00MB thread-reservation=1 07:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=09 | hash predicates: ss_store_sk = s_store_sk @@ -1205,7 +1205,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=40.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.23MB mem-reservation=40.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_market_id = CAST(8 AS INT) @@ -1243,7 +1243,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.27MB mem-reservation=256.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: tpcds_parquet.item.i_color = 'saddle' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test index ed97fe816..7e439f2c0 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test @@ -213,7 +213,7 @@ PLAN-ROOT SINK in pipelines: 07(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=69.59MB Threads=20 -Per-Host Resource Estimates: Memory=709MB +Per-Host Resource Estimates: Memory=713MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.08MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -248,7 +248,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati | in pipelines: 02(GETNEXT) | F09:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=15.90MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=15.98MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=1.00MB 15:AGGREGATE [STREAMING] | output: sum(ss_net_profit), sum(sr_net_loss), sum(cs_net_profit) | group by: i_item_id, i_item_desc, s_store_id, s_store_name @@ -270,7 +270,7 @@ Per-Host Resources: mem-estimate=15.90MB mem-reservation=4.94MB thread-reservati | | in pipelines: 07(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=49.79MB mem-reservation=2.00MB thread-reservation=2 | 07:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -287,7 +287,7 @@ Per-Host Resources: mem-estimate=15.90MB mem-reservation=4.94MB thread-reservati | in pipelines: 02(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=203.73MB mem-reservation=18.81MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=205.07MB mem-reservation=18.81MB thread-reservation=2 runtime-filters-memory=5.00MB 13:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_store_sk = s_store_sk | fk/pk conjuncts: ss_store_sk = s_store_sk @@ -302,7 +302,7 @@ Per-Host Resources: mem-estimate=203.73MB mem-reservation=18.81MB thread-reserva | | in pipelines: 06(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.20MB mem-reservation=24.00KB thread-reservation=2 | 06:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -327,7 +327,7 @@ Per-Host Resources: mem-estimate=203.73MB mem-reservation=18.81MB thread-reserva | | in pipelines: 05(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d3.d_year = CAST(2001 AS INT), d3.d_moy <= CAST(10 AS INT), d3.d_moy >= CAST(4 AS INT) @@ -370,7 +370,7 @@ Per-Host Resources: mem-estimate=203.73MB mem-reservation=18.81MB thread-reserva | | | in pipelines: 04(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 04:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d2.d_year = CAST(2001 AS INT), d2.d_moy <= CAST(10 AS INT), d2.d_moy >= CAST(4 AS INT) @@ -398,7 +398,7 @@ Per-Host Resources: mem-estimate=203.73MB mem-reservation=18.81MB thread-reserva | | | in pipelines: 01(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=122.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=122.38MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB | | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF011[min_max] -> sr_returned_date_sk, RF001[min_max] -> tpcds_parquet.store_returns.sr_item_sk, RF010[bloom] -> sr_returned_date_sk, RF000[bloom] -> tpcds_parquet.store_returns.sr_item_sk @@ -416,7 +416,7 @@ Per-Host Resources: mem-estimate=203.73MB mem-reservation=18.81MB thread-reserva | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=87.95MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=6.00MB +| Per-Host Resources: mem-estimate=88.55MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=6.00MB | 08:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_sold_date_sk = d1.d_date_sk | | fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk @@ -431,7 +431,7 @@ Per-Host Resources: mem-estimate=203.73MB mem-reservation=18.81MB thread-reserva | | | in pipelines: 03(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year = CAST(2001 AS INT), d1.d_moy = CAST(4 AS INT) @@ -469,7 +469,7 @@ Per-Host Resources: mem-estimate=203.73MB mem-reservation=18.81MB thread-reserva in pipelines: 02(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=91.27MB Threads=22 -Per-Host Resource Estimates: Memory=289MB +Per-Host Resource Estimates: Memory=300MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.08MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -504,7 +504,7 @@ Per-Instance Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reser | in pipelines: 02(GETNEXT) | F09:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=10.27MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=13.04MB mem-reservation=2.00MB thread-reservation=1 15:AGGREGATE [STREAMING] | output: sum(ss_net_profit), sum(sr_net_loss), sum(cs_net_profit) | group by: i_item_id, i_item_desc, s_store_id, s_store_name @@ -534,7 +534,7 @@ Per-Instance Resources: mem-estimate=10.27MB mem-reservation=2.00MB thread-reser | | in pipelines: 07(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.79MB mem-reservation=2.00MB thread-reservation=1 | 07:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -552,7 +552,7 @@ Per-Instance Resources: mem-estimate=10.27MB mem-reservation=2.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=50.25MB mem-reservation=8.00MB thread-reservation=1 13:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=01 | hash predicates: ss_store_sk = s_store_sk @@ -575,7 +575,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 06(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.20MB mem-reservation=24.00KB thread-reservation=1 | 06:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -608,7 +608,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d3.d_year = CAST(2001 AS INT), d3.d_moy <= CAST(10 AS INT), d3.d_moy >= CAST(4 AS INT) @@ -644,7 +644,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F04:PLAN FRAGMENT [HASH(ss_customer_sk,ss_item_sk,ss_ticket_number)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=2.69MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=3.08MB mem-reservation=0B thread-reservation=1 | 10:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=04 | | hash predicates: sr_returned_date_sk = d2.d_date_sk @@ -667,7 +667,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | in pipelines: 04(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 04:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d2.d_year = CAST(2001 AS INT), d2.d_moy <= CAST(10 AS INT), d2.d_moy >= CAST(4 AS INT) @@ -704,7 +704,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.75MB mem-reservation=8.00MB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF011[min_max] -> sr_returned_date_sk, RF001[min_max] -> tpcds_parquet.store_returns.sr_item_sk, RF010[bloom] -> sr_returned_date_sk, RF000[bloom] -> tpcds_parquet.store_returns.sr_item_sk @@ -723,7 +723,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=6.00MB mem-reservation=6.00MB thread-reservation=0 runtime-filters-memory=6.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.22MB mem-reservation=4.00MB thread-reservation=1 | 08:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=06 | | hash predicates: ss_sold_date_sk = d1.d_date_sk @@ -746,7 +746,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | in pipelines: 03(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year = CAST(2001 AS INT), d1.d_moy = CAST(4 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test index 622a246e0..2b5f184ca 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test @@ -135,7 +135,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=50.53MB Threads=13 -Per-Host Resource Estimates: Memory=612MB +Per-Host Resource Estimates: Memory=613MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -192,7 +192,7 @@ Per-Host Resources: mem-estimate=31.42MB mem-reservation=13.62MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.16MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -217,7 +217,7 @@ Per-Host Resources: mem-estimate=31.42MB mem-reservation=13.62MB thread-reservat | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=32.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.13MB mem-reservation=32.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | predicates: (p_channel_email = 'N' OR p_channel_event = 'N') @@ -243,7 +243,7 @@ Per-Host Resources: mem-estimate=31.42MB mem-reservation=13.62MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.65MB mem-reservation=8.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'S', cd_gender = 'M', cd_education_status = 'College' @@ -263,7 +263,7 @@ Per-Host Resources: mem-estimate=31.42MB mem-reservation=13.62MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=389.95MB mem-reservation=21.94MB thread-reservation=2 runtime-filters-memory=4.00MB +Per-Host Resources: mem-estimate=390.55MB mem-reservation=21.94MB thread-reservation=2 runtime-filters-memory=4.00MB 05:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: cs_sold_date_sk = d_date_sk | fk/pk conjuncts: cs_sold_date_sk = d_date_sk @@ -278,7 +278,7 @@ Per-Host Resources: mem-estimate=389.95MB mem-reservation=21.94MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2000 AS INT) @@ -304,7 +304,7 @@ Per-Host Resources: mem-estimate=389.95MB mem-reservation=21.94MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=57.34MB Threads=12 -Per-Host Resource Estimates: Memory=171MB +Per-Host Resource Estimates: Memory=173MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -339,7 +339,7 @@ Per-Instance Resources: mem-estimate=10.36MB mem-reservation=1.94MB thread-reser | in pipelines: 00(GETNEXT) | F03:PLAN FRAGMENT [HASH(cs_bill_cdemo_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=14.27MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=15.02MB mem-reservation=2.00MB thread-reservation=1 09:AGGREGATE [STREAMING] | output: avg(CAST(cs_quantity AS BIGINT)), avg(cs_list_price), avg(cs_coupon_amt), avg(cs_sales_price) | group by: i_item_id @@ -369,7 +369,7 @@ Per-Instance Resources: mem-estimate=14.27MB mem-reservation=2.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -402,7 +402,7 @@ Per-Instance Resources: mem-estimate=14.27MB mem-reservation=2.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.13MB mem-reservation=32.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | predicates: (p_channel_email = 'N' OR p_channel_event = 'N') @@ -436,7 +436,7 @@ Per-Instance Resources: mem-estimate=14.27MB mem-reservation=2.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.65MB mem-reservation=8.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'S', cd_gender = 'M', cd_education_status = 'College' @@ -457,7 +457,7 @@ Per-Instance Resources: mem-estimate=14.27MB mem-reservation=2.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=48.61MB mem-reservation=16.00MB thread-reservation=1 05:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=03 | hash predicates: cs_sold_date_sk = d_date_sk @@ -480,7 +480,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2000 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test index c20d31451..bcde921c2 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test @@ -227,7 +227,7 @@ Per-Host Resources: mem-estimate=165.85MB mem-reservation=30.31MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.16MB mem-reservation=512.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -252,7 +252,7 @@ Per-Host Resources: mem-estimate=165.85MB mem-reservation=30.31MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_state IN ('TN') @@ -280,7 +280,7 @@ Per-Host Resources: mem-estimate=165.85MB mem-reservation=30.31MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.22MB mem-reservation=8.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'S', cd_gender = 'M', cd_education_status = 'College' @@ -308,7 +308,7 @@ Per-Host Resources: mem-estimate=165.85MB mem-reservation=30.31MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2002 AS INT) @@ -335,7 +335,7 @@ Per-Host Resources: mem-estimate=165.85MB mem-reservation=30.31MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=89.14MB Threads=13 -Per-Host Resource Estimates: Memory=282MB +Per-Host Resource Estimates: Memory=293MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -385,7 +385,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=46.00MB mem-reservation=12.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=51.02MB mem-reservation=12.00MB thread-reservation=1 09:AGGREGATE [STREAMING] | Class 0 | output: avg(CAST(ss_quantity AS BIGINT)), avg(ss_list_price), avg(ss_coupon_amt), avg(ss_sales_price) @@ -422,7 +422,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=12.00MB thread-rese | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -455,7 +455,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=12.00MB thread-rese | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_state IN ('TN') @@ -491,7 +491,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=12.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.22MB mem-reservation=8.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'S', cd_gender = 'M', cd_education_status = 'College' @@ -527,7 +527,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=12.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2002 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q28.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q28.test index 291245fca..96bceb897 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q28.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q28.test @@ -248,7 +248,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=39.62MB Threads=24 -Per-Host Resource Estimates: Memory=509MB +Per-Host Resource Estimates: Memory=512MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.09MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -267,7 +267,7 @@ PLAN-ROOT SINK | | in pipelines: 50(GETNEXT) | | | F17:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=128.00KB mem-reservation=0B thread-reservation=1 | 50:AGGREGATE [FINALIZE] | | output: count:merge(ss_list_price), avg:merge(ss_list_price), count:merge(ss_list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -300,7 +300,7 @@ PLAN-ROOT SINK | | in pipelines: 15(GETNEXT) | | | F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=74.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=74.28MB mem-reservation=4.00MB thread-reservation=2 | 16:AGGREGATE [STREAMING] | | output: avg(ss_list_price), count(ss_list_price) | | group by: ss_list_price @@ -333,7 +333,7 @@ PLAN-ROOT SINK | | in pipelines: 45(GETNEXT) | | | F14:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=128.00KB mem-reservation=0B thread-reservation=1 | 45:AGGREGATE [FINALIZE] | | output: count:merge(ss_list_price), avg:merge(ss_list_price), count:merge(ss_list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -366,7 +366,7 @@ PLAN-ROOT SINK | | in pipelines: 12(GETNEXT) | | | F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=74.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=74.28MB mem-reservation=4.00MB thread-reservation=2 | 13:AGGREGATE [STREAMING] | | output: avg(ss_list_price), count(ss_list_price) | | group by: ss_list_price @@ -399,7 +399,7 @@ PLAN-ROOT SINK | | in pipelines: 40(GETNEXT) | | | F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=128.00KB mem-reservation=0B thread-reservation=1 | 40:AGGREGATE [FINALIZE] | | output: count:merge(ss_list_price), avg:merge(ss_list_price), count:merge(ss_list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -432,7 +432,7 @@ PLAN-ROOT SINK | | in pipelines: 09(GETNEXT) | | | F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=74.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=74.28MB mem-reservation=4.00MB thread-reservation=2 | 10:AGGREGATE [STREAMING] | | output: avg(ss_list_price), count(ss_list_price) | | group by: ss_list_price @@ -465,7 +465,7 @@ PLAN-ROOT SINK | | in pipelines: 35(GETNEXT) | | | F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=128.00KB mem-reservation=0B thread-reservation=1 | 35:AGGREGATE [FINALIZE] | | output: count:merge(ss_list_price), avg:merge(ss_list_price), count:merge(ss_list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -498,7 +498,7 @@ PLAN-ROOT SINK | | in pipelines: 06(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=74.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=74.28MB mem-reservation=4.00MB thread-reservation=2 | 07:AGGREGATE [STREAMING] | | output: avg(ss_list_price), count(ss_list_price) | | group by: ss_list_price @@ -531,7 +531,7 @@ PLAN-ROOT SINK | | in pipelines: 30(GETNEXT) | | | F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=128.00KB mem-reservation=0B thread-reservation=1 | 30:AGGREGATE [FINALIZE] | | output: count:merge(ss_list_price), avg:merge(ss_list_price), count:merge(ss_list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -564,7 +564,7 @@ PLAN-ROOT SINK | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=74.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=74.28MB mem-reservation=4.00MB thread-reservation=2 | 04:AGGREGATE [STREAMING] | | output: avg(ss_list_price), count(ss_list_price) | | group by: ss_list_price @@ -618,7 +618,7 @@ Per-Host Resources: mem-estimate=10.20MB mem-reservation=1.94MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=74.00MB mem-reservation=4.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=74.28MB mem-reservation=4.00MB thread-reservation=2 01:AGGREGATE [STREAMING] | output: avg(ss_list_price), count(ss_list_price) | group by: ss_list_price @@ -641,7 +641,7 @@ Per-Host Resources: mem-estimate=74.00MB mem-reservation=4.00MB thread-reservati in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=75.25MB Threads=35 -Per-Host Resource Estimates: Memory=439MB +Per-Host Resource Estimates: Memory=447MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -667,7 +667,7 @@ PLAN-ROOT SINK | | in pipelines: 50(GETNEXT) | | | F17:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=128.00KB mem-reservation=0B thread-reservation=1 | 50:AGGREGATE [FINALIZE] | | output: count:merge(ss_list_price), avg:merge(ss_list_price), count:merge(ss_list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -700,7 +700,7 @@ PLAN-ROOT SINK | | in pipelines: 15(GETNEXT) | | | F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reservation=1 | 16:AGGREGATE [STREAMING] | | output: avg(ss_list_price), count(ss_list_price) | | group by: ss_list_price @@ -740,7 +740,7 @@ PLAN-ROOT SINK | | in pipelines: 45(GETNEXT) | | | F14:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=128.00KB mem-reservation=0B thread-reservation=1 | 45:AGGREGATE [FINALIZE] | | output: count:merge(ss_list_price), avg:merge(ss_list_price), count:merge(ss_list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -773,7 +773,7 @@ PLAN-ROOT SINK | | in pipelines: 12(GETNEXT) | | | F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reservation=1 | 13:AGGREGATE [STREAMING] | | output: avg(ss_list_price), count(ss_list_price) | | group by: ss_list_price @@ -813,7 +813,7 @@ PLAN-ROOT SINK | | in pipelines: 40(GETNEXT) | | | F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=128.00KB mem-reservation=0B thread-reservation=1 | 40:AGGREGATE [FINALIZE] | | output: count:merge(ss_list_price), avg:merge(ss_list_price), count:merge(ss_list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -846,7 +846,7 @@ PLAN-ROOT SINK | | in pipelines: 09(GETNEXT) | | | F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reservation=1 | 10:AGGREGATE [STREAMING] | | output: avg(ss_list_price), count(ss_list_price) | | group by: ss_list_price @@ -886,7 +886,7 @@ PLAN-ROOT SINK | | in pipelines: 35(GETNEXT) | | | F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=128.00KB mem-reservation=0B thread-reservation=1 | 35:AGGREGATE [FINALIZE] | | output: count:merge(ss_list_price), avg:merge(ss_list_price), count:merge(ss_list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -919,7 +919,7 @@ PLAN-ROOT SINK | | in pipelines: 06(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reservation=1 | 07:AGGREGATE [STREAMING] | | output: avg(ss_list_price), count(ss_list_price) | | group by: ss_list_price @@ -959,7 +959,7 @@ PLAN-ROOT SINK | | in pipelines: 30(GETNEXT) | | | F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=128.00KB mem-reservation=0B thread-reservation=1 | 30:AGGREGATE [FINALIZE] | | output: count:merge(ss_list_price), avg:merge(ss_list_price), count:merge(ss_list_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -992,7 +992,7 @@ PLAN-ROOT SINK | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reservation=1 | 04:AGGREGATE [STREAMING] | | output: avg(ss_list_price), count(ss_list_price) | | group by: ss_list_price @@ -1046,7 +1046,7 @@ Per-Instance Resources: mem-estimate=10.27MB mem-reservation=1.94MB thread-reser | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reservation=1 01:AGGREGATE [STREAMING] | output: avg(ss_list_price), count(ss_list_price) | group by: ss_list_price diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test index c10d0db1d..cc1ef30f1 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test @@ -213,7 +213,7 @@ PLAN-ROOT SINK in pipelines: 07(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=69.59MB Threads=20 -Per-Host Resource Estimates: Memory=694MB +Per-Host Resource Estimates: Memory=698MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -270,7 +270,7 @@ Per-Host Resources: mem-estimate=16.40MB mem-reservation=4.94MB thread-reservati | | in pipelines: 07(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=49.79MB mem-reservation=2.00MB thread-reservation=2 | 07:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -287,7 +287,7 @@ Per-Host Resources: mem-estimate=16.40MB mem-reservation=4.94MB thread-reservati | in pipelines: 02(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=203.74MB mem-reservation=18.81MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=205.02MB mem-reservation=18.81MB thread-reservation=2 runtime-filters-memory=5.00MB 13:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_store_sk = s_store_sk | fk/pk conjuncts: ss_store_sk = s_store_sk @@ -302,7 +302,7 @@ Per-Host Resources: mem-estimate=203.74MB mem-reservation=18.81MB thread-reserva | | in pipelines: 06(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.20MB mem-reservation=24.00KB thread-reservation=2 | 06:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -327,7 +327,7 @@ Per-Host Resources: mem-estimate=203.74MB mem-reservation=18.81MB thread-reserva | | in pipelines: 05(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d3.d_year IN (CAST(1999 AS INT), CAST(2000 AS INT), CAST(2001 AS INT)) @@ -370,7 +370,7 @@ Per-Host Resources: mem-estimate=203.74MB mem-reservation=18.81MB thread-reserva | | | in pipelines: 04(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 04:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d2.d_year = CAST(1999 AS INT), d2.d_moy <= CAST(12 AS INT), d2.d_moy >= CAST(9 AS INT) @@ -398,7 +398,7 @@ Per-Host Resources: mem-estimate=203.74MB mem-reservation=18.81MB thread-reserva | | | in pipelines: 01(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=122.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=122.38MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB | | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF011[min_max] -> sr_returned_date_sk, RF001[min_max] -> tpcds_parquet.store_returns.sr_item_sk, RF010[bloom] -> sr_returned_date_sk, RF000[bloom] -> tpcds_parquet.store_returns.sr_item_sk @@ -416,7 +416,7 @@ Per-Host Resources: mem-estimate=203.74MB mem-reservation=18.81MB thread-reserva | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=87.95MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=6.00MB +| Per-Host Resources: mem-estimate=88.55MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=6.00MB | 08:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_sold_date_sk = d1.d_date_sk | | fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk @@ -431,7 +431,7 @@ Per-Host Resources: mem-estimate=203.74MB mem-reservation=18.81MB thread-reserva | | | in pipelines: 03(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year = CAST(1999 AS INT), d1.d_moy = CAST(9 AS INT) @@ -469,7 +469,7 @@ Per-Host Resources: mem-estimate=203.74MB mem-reservation=18.81MB thread-reserva in pipelines: 02(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=91.27MB Threads=22 -Per-Host Resource Estimates: Memory=290MB +Per-Host Resource Estimates: Memory=301MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -504,7 +504,7 @@ Per-Instance Resources: mem-estimate=10.91MB mem-reservation=1.94MB thread-reser | in pipelines: 02(GETNEXT) | F09:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=10.77MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=13.26MB mem-reservation=2.00MB thread-reservation=1 15:AGGREGATE [STREAMING] | output: sum(CAST(ss_quantity AS BIGINT)), sum(CAST(sr_return_quantity AS BIGINT)), sum(CAST(cs_quantity AS BIGINT)) | group by: i_item_id, i_item_desc, s_store_id, s_store_name @@ -534,7 +534,7 @@ Per-Instance Resources: mem-estimate=10.77MB mem-reservation=2.00MB thread-reser | | in pipelines: 07(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.79MB mem-reservation=2.00MB thread-reservation=1 | 07:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -552,7 +552,7 @@ Per-Instance Resources: mem-estimate=10.77MB mem-reservation=2.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=50.21MB mem-reservation=8.00MB thread-reservation=1 13:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=01 | hash predicates: ss_store_sk = s_store_sk @@ -575,7 +575,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 06(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.20MB mem-reservation=24.00KB thread-reservation=1 | 06:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -608,7 +608,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d3.d_year IN (CAST(1999 AS INT), CAST(2000 AS INT), CAST(2001 AS INT)) @@ -644,7 +644,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F04:PLAN FRAGMENT [HASH(ss_customer_sk,ss_item_sk,ss_ticket_number)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=2.69MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=3.08MB mem-reservation=0B thread-reservation=1 | 10:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=04 | | hash predicates: sr_returned_date_sk = d2.d_date_sk @@ -667,7 +667,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | in pipelines: 04(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 04:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d2.d_year = CAST(1999 AS INT), d2.d_moy <= CAST(12 AS INT), d2.d_moy >= CAST(9 AS INT) @@ -704,7 +704,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.75MB mem-reservation=8.00MB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF011[min_max] -> sr_returned_date_sk, RF001[min_max] -> tpcds_parquet.store_returns.sr_item_sk, RF010[bloom] -> sr_returned_date_sk, RF000[bloom] -> tpcds_parquet.store_returns.sr_item_sk @@ -723,7 +723,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=6.00MB mem-reservation=6.00MB thread-reservation=0 runtime-filters-memory=6.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.22MB mem-reservation=4.00MB thread-reservation=1 | 08:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=06 | | hash predicates: ss_sold_date_sk = d1.d_date_sk @@ -746,7 +746,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | in pipelines: 03(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year = CAST(1999 AS INT), d1.d_moy = CAST(9 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test index 4f9731e19..1397e384f 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test @@ -271,7 +271,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva | | in pipelines: 30(GETNEXT) | | | F10:PLAN FRAGMENT [HASH(ctr2.ctr_state)] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.13MB mem-reservation=1.94MB thread-reservation=1 | 30:AGGREGATE [FINALIZE] | | output: avg:merge(ctr_total_return) | | group by: ctr2.ctr_state @@ -328,7 +328,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva | | | in pipelines: 08(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=66.95MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=67.06MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 11:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: wr_returned_date_sk = d_date_sk | | | fk/pk conjuncts: wr_returned_date_sk = d_date_sk @@ -343,7 +343,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva | | | | in pipelines: 09(GETNEXT) | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 09:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2002 AS INT) @@ -429,7 +429,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva | | | in pipelines: 00(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=66.95MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=67.06MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 03:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: wr_returned_date_sk = d_date_sk | | | fk/pk conjuncts: wr_returned_date_sk = d_date_sk @@ -444,7 +444,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva | | | | in pipelines: 01(GETNEXT) | | | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2002 AS INT) @@ -494,7 +494,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva | | in pipelines: 06(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.09MB mem-reservation=256.00KB thread-reservation=2 | 06:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_state = 'GA' @@ -520,7 +520,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva in pipelines: 07(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=68.69MB Threads=19 -Per-Host Resource Estimates: Memory=235MB +Per-Host Resource Estimates: Memory=236MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -565,7 +565,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser | | in pipelines: 30(GETNEXT) | | | F10:PLAN FRAGMENT [HASH(ctr2.ctr_state)] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.13MB mem-reservation=1.94MB thread-reservation=1 | 30:AGGREGATE [FINALIZE] | | output: avg:merge(ctr_total_return) | | group by: ctr2.ctr_state @@ -601,7 +601,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.25MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.15MB mem-reservation=2.25MB thread-reservation=1 | 13:AGGREGATE [STREAMING] | | output: sum(wr_return_amt) | | group by: wr_returning_customer_sk, ca_state @@ -632,7 +632,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 11:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=02 | | | hash predicates: wr_returned_date_sk = d_date_sk @@ -655,7 +655,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser | | | | in pipelines: 09(GETNEXT) | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 09:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2002 AS INT) @@ -728,7 +728,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.25MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.15MB mem-reservation=2.25MB thread-reservation=1 | 05:AGGREGATE [STREAMING] | | output: sum(wr_return_amt) | | group by: wr_returning_customer_sk, ca_state @@ -759,7 +759,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 03:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=05 | | | hash predicates: wr_returned_date_sk = d_date_sk @@ -782,7 +782,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser | | | | in pipelines: 01(GETNEXT) | | | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2002 AS INT) @@ -840,7 +840,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser | | in pipelines: 06(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | 06:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_state = 'GA' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test index 6d9c84b8c..a62690431 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test @@ -549,7 +549,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | | in pipelines: 14(GETNEXT) | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.13MB mem-reservation=512.00KB thread-reservation=2 | | 14:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -574,7 +574,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | | in pipelines: 13(GETNEXT) | | | | | F25:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(3 AS INT) @@ -651,7 +651,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | | in pipelines: 08(GETNEXT) | | | | | F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.13MB mem-reservation=512.00KB thread-reservation=2 | | 08:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer_address.ca_county @@ -677,7 +677,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | | in pipelines: 07(GETNEXT) | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(2 AS INT) @@ -753,7 +753,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | | in pipelines: 32(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.27MB mem-reservation=512.00KB thread-reservation=2 | | 32:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer_address.ca_county, RF003[min_max] -> tpcds_parquet.customer_address.ca_county @@ -771,7 +771,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | in pipelines: 30(GETNEXT) | | | F15:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=99.95MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=100.19MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 33:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_sold_date_sk = d_date_sk | | fk/pk conjuncts: ws_sold_date_sk = d_date_sk @@ -786,7 +786,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | | in pipelines: 31(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(3 AS INT) @@ -861,7 +861,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | | in pipelines: 26(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.27MB mem-reservation=512.00KB thread-reservation=2 | | 26:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | runtime filters: RF005[min_max] -> tpcds_parquet.customer_address.ca_county, RF001[min_max] -> tpcds_parquet.customer_address.ca_county, RF003[min_max] -> tpcds_parquet.customer_address.ca_county @@ -879,7 +879,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | in pipelines: 24(GETNEXT) | | | F10:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=99.95MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=100.19MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 27:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_sold_date_sk = d_date_sk | | fk/pk conjuncts: ws_sold_date_sk = d_date_sk @@ -894,7 +894,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | | in pipelines: 25(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(2 AS INT) @@ -969,7 +969,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | | in pipelines: 20(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.27MB mem-reservation=512.00KB thread-reservation=2 | | 20:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | runtime filters: RF007[min_max] -> tpcds_parquet.customer_address.ca_county, RF005[min_max] -> tpcds_parquet.customer_address.ca_county, RF001[min_max] -> tpcds_parquet.customer_address.ca_county, RF003[min_max] -> tpcds_parquet.customer_address.ca_county @@ -987,7 +987,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | in pipelines: 18(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=99.95MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=100.19MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 21:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_sold_date_sk = d_date_sk | | fk/pk conjuncts: ws_sold_date_sk = d_date_sk @@ -1002,7 +1002,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat | | | in pipelines: 19(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 19:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(1 AS INT) @@ -1069,7 +1069,7 @@ Per-Host Resources: mem-estimate=52.16MB mem-reservation=18.69MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.13MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | runtime filters: RF009[min_max] -> tpcds_parquet.customer_address.ca_county, RF007[min_max] -> tpcds_parquet.customer_address.ca_county, RF005[min_max] -> tpcds_parquet.customer_address.ca_county, RF001[min_max] -> tpcds_parquet.customer_address.ca_county, RF003[min_max] -> tpcds_parquet.customer_address.ca_county @@ -1095,7 +1095,7 @@ Per-Host Resources: mem-estimate=52.16MB mem-reservation=18.69MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(1 AS INT) @@ -1181,7 +1181,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | F24:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=27.26MB mem-reservation=6.00MB thread-reservation=1 | 17:AGGREGATE [STREAMING] | | output: sum(ss_ext_sales_price) | | group by: ca_county, d_qoy, d_year @@ -1211,7 +1211,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | in pipelines: 14(GETNEXT) | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.13MB mem-reservation=512.00KB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -1244,7 +1244,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | in pipelines: 13(GETNEXT) | | | | | F25:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(3 AS INT) @@ -1308,7 +1308,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=27.26MB mem-reservation=6.00MB thread-reservation=1 | 11:AGGREGATE [STREAMING] | | output: sum(ss_ext_sales_price) | | group by: ca_county, d_qoy, d_year @@ -1338,7 +1338,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | in pipelines: 08(GETNEXT) | | | | | F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.13MB mem-reservation=512.00KB thread-reservation=1 | | 08:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer_address.ca_county @@ -1372,7 +1372,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | in pipelines: 07(GETNEXT) | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(2 AS INT) @@ -1434,7 +1434,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | in pipelines: 30(GETNEXT) | | | F18:PLAN FRAGMENT [HASH(ws_bill_addr_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=10.91MB mem-reservation=3.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=11.33MB mem-reservation=3.00MB thread-reservation=1 | 35:AGGREGATE [STREAMING] | | output: sum(ws_ext_sales_price) | | group by: ca_county, d_qoy, d_year @@ -1464,7 +1464,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | in pipelines: 32(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.27MB mem-reservation=512.00KB thread-reservation=1 | | 32:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer_address.ca_county, RF003[min_max] -> tpcds_parquet.customer_address.ca_county @@ -1483,7 +1483,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | F15:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.25MB mem-reservation=8.00MB thread-reservation=1 | 33:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=08 | | hash predicates: ws_sold_date_sk = d_date_sk @@ -1506,7 +1506,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | in pipelines: 31(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(3 AS INT) @@ -1567,7 +1567,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | in pipelines: 24(GETNEXT) | | | F13:PLAN FRAGMENT [HASH(ws_bill_addr_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=10.91MB mem-reservation=3.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=11.33MB mem-reservation=3.00MB thread-reservation=1 | 29:AGGREGATE [STREAMING] | | output: sum(ws_ext_sales_price) | | group by: ca_county, d_qoy, d_year @@ -1597,7 +1597,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | in pipelines: 26(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.27MB mem-reservation=512.00KB thread-reservation=1 | | 26:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | runtime filters: RF005[min_max] -> tpcds_parquet.customer_address.ca_county, RF001[min_max] -> tpcds_parquet.customer_address.ca_county, RF003[min_max] -> tpcds_parquet.customer_address.ca_county @@ -1616,7 +1616,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | F10:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.25MB mem-reservation=8.00MB thread-reservation=1 | 27:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=11 | | hash predicates: ws_sold_date_sk = d_date_sk @@ -1639,7 +1639,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | in pipelines: 25(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(2 AS INT) @@ -1700,7 +1700,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | in pipelines: 18(GETNEXT) | | | F07:PLAN FRAGMENT [HASH(ws_bill_addr_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=10.91MB mem-reservation=3.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=11.33MB mem-reservation=3.00MB thread-reservation=1 | 23:AGGREGATE [STREAMING] | | output: sum(ws_ext_sales_price) | | group by: ca_county, d_qoy, d_year @@ -1730,7 +1730,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | in pipelines: 20(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.27MB mem-reservation=512.00KB thread-reservation=1 | | 20:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | runtime filters: RF007[min_max] -> tpcds_parquet.customer_address.ca_county, RF005[min_max] -> tpcds_parquet.customer_address.ca_county, RF001[min_max] -> tpcds_parquet.customer_address.ca_county, RF003[min_max] -> tpcds_parquet.customer_address.ca_county @@ -1749,7 +1749,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | F04:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.25MB mem-reservation=8.00MB thread-reservation=1 | 21:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=14 | | hash predicates: ws_sold_date_sk = d_date_sk @@ -1772,7 +1772,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser | | | in pipelines: 19(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 19:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(1 AS INT) @@ -1818,7 +1818,7 @@ Per-Instance Resources: mem-estimate=14.98MB mem-reservation=4.75MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=27.26MB mem-reservation=6.00MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: ca_county, d_qoy, d_year @@ -1848,7 +1848,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.13MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | runtime filters: RF009[min_max] -> tpcds_parquet.customer_address.ca_county, RF007[min_max] -> tpcds_parquet.customer_address.ca_county, RF005[min_max] -> tpcds_parquet.customer_address.ca_county, RF001[min_max] -> tpcds_parquet.customer_address.ca_county, RF003[min_max] -> tpcds_parquet.customer_address.ca_county @@ -1882,7 +1882,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT), tpcds_parquet.date_dim.d_qoy = CAST(1 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test index 78a4a1e28..29ac5b143 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test @@ -133,7 +133,7 @@ PLAN-ROOT SINK in pipelines: 03(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=35.94MB Threads=13 -Per-Host Resource Estimates: Memory=419MB +Per-Host Resource Estimates: Memory=420MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -174,7 +174,7 @@ Per-Host Resources: mem-estimate=13.31MB mem-reservation=4.88MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F06:PLAN FRAGMENT [HASH(cs_sold_date_sk)] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=3.27MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=3.83MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB | 08:HASH JOIN [INNER JOIN, PARTITIONED] | | hash predicates: cs_sold_date_sk = d_date_sk | | fk/pk conjuncts: cs_sold_date_sk = d_date_sk @@ -189,7 +189,7 @@ Per-Host Resources: mem-estimate=13.31MB mem-reservation=4.88MB thread-reservati | | | in pipelines: 02(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.35MB mem-reservation=1.00MB thread-reservation=2 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '2000-04-26', CAST(d_date AS DATE) >= DATE '2000-01-27' @@ -209,7 +209,7 @@ Per-Host Resources: mem-estimate=13.31MB mem-reservation=4.88MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=147.95MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=148.36MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 07:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: cs_item_sk = i_item_sk | | fk/pk conjuncts: cs_item_sk = i_item_sk @@ -224,7 +224,7 @@ Per-Host Resources: mem-estimate=13.31MB mem-reservation=4.88MB thread-reservati | | | in pipelines: 01(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.06MB mem-reservation=256.00KB thread-reservation=2 | | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_manufact_id = CAST(977 AS INT) @@ -262,7 +262,7 @@ Per-Host Resources: mem-estimate=13.31MB mem-reservation=4.88MB thread-reservati | in pipelines: 03(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=158.15MB mem-reservation=13.94MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=158.17MB mem-reservation=13.94MB thread-reservation=2 runtime-filters-memory=2.00MB 06:AGGREGATE [STREAMING] | output: avg(cs_ext_discount_amt) | group by: cs_item_sk @@ -284,7 +284,7 @@ Per-Host Resources: mem-estimate=158.15MB mem-reservation=13.94MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '2000-04-26', CAST(d_date AS DATE) >= DATE '2000-01-27' @@ -310,7 +310,7 @@ Per-Host Resources: mem-estimate=158.15MB mem-reservation=13.94MB thread-reserva in pipelines: 03(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=41.81MB Threads=12 -Per-Host Resource Estimates: Memory=185MB +Per-Host Resource Estimates: Memory=187MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -359,7 +359,7 @@ Per-Instance Resources: mem-estimate=10.15MB mem-reservation=1.94MB thread-reser | | in pipelines: 00(GETNEXT) | | | F06:PLAN FRAGMENT [HASH(cs_sold_date_sk)] hosts=3 instances=3 -| Per-Instance Resources: mem-estimate=121.87KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=913.87KB mem-reservation=0B thread-reservation=1 | 08:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=01 | | hash predicates: cs_sold_date_sk = d_date_sk @@ -382,7 +382,7 @@ Per-Instance Resources: mem-estimate=10.15MB mem-reservation=1.94MB thread-reser | | | in pipelines: 02(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.35MB mem-reservation=1.00MB thread-reservation=1 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '2000-04-26', CAST(d_date AS DATE) >= DATE '2000-01-27' @@ -403,7 +403,7 @@ Per-Instance Resources: mem-estimate=10.15MB mem-reservation=1.94MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=48.42MB mem-reservation=8.00MB thread-reservation=1 | 07:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=02 | | hash predicates: cs_item_sk = i_item_sk @@ -426,7 +426,7 @@ Per-Instance Resources: mem-estimate=10.15MB mem-reservation=1.94MB thread-reser | | | in pipelines: 01(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=256.00KB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_manufact_id = CAST(977 AS INT) @@ -465,7 +465,7 @@ Per-Instance Resources: mem-estimate=10.15MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=58.00MB mem-reservation=10.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=58.23MB mem-reservation=10.00MB thread-reservation=1 06:AGGREGATE [STREAMING] | output: avg(cs_ext_discount_amt) | group by: cs_item_sk @@ -495,7 +495,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=10.00MB thread-rese | | in pipelines: 04(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '2000-04-26', CAST(d_date AS DATE) >= DATE '2000-01-27' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test index c7ec5879a..2195b5c3e 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test @@ -397,7 +397,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=73.50MB Threads=35 -Per-Host Resource Estimates: Memory=902MB +Per-Host Resource Estimates: Memory=904MB F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -432,7 +432,7 @@ Per-Host Resources: mem-estimate=10.09MB mem-reservation=1.94MB thread-reservati | in pipelines: 38(GETNEXT), 45(GETNEXT), 52(GETNEXT) | F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=20.35MB mem-reservation=3.94MB thread-reservation=1 31:AGGREGATE [STREAMING] | output: sum(total_sales) | group by: i_manufact_id @@ -479,7 +479,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 25(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=128.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.10MB mem-reservation=128.00KB thread-reservation=2 | | 25:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category IN ('Electronics') @@ -507,7 +507,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 24(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=256.00KB thread-reservation=2 | | 24:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF017[min_max] -> i_manufact_id @@ -525,7 +525,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati | | in pipelines: 21(GETNEXT) | | | F13:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=133.96MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=134.28MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 27:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_bill_addr_sk = ca_address_sk | | fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk @@ -540,7 +540,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 23(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | | 23:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -568,7 +568,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 22(GETNEXT) | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 22:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(5 AS INT) @@ -627,7 +627,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 15(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=128.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.10MB mem-reservation=128.00KB thread-reservation=2 | | 15:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category IN ('Electronics') @@ -655,7 +655,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 14(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.19MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB | | 14:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF009[min_max] -> i_manufact_id, RF008[bloom] -> i_manufact_id @@ -673,7 +673,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati | | in pipelines: 11(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=198.96MB mem-reservation=14.88MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=199.48MB mem-reservation=14.88MB thread-reservation=2 runtime-filters-memory=3.00MB | 17:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: cs_bill_addr_sk = ca_address_sk | | fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk @@ -688,7 +688,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 13(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | | 13:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -716,7 +716,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 12(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(5 AS INT) @@ -775,7 +775,7 @@ Per-Host Resources: mem-estimate=70.12MB mem-reservation=15.75MB thread-reservat | | in pipelines: 05(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=128.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.10MB mem-reservation=128.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category IN ('Electronics') @@ -803,7 +803,7 @@ Per-Host Resources: mem-estimate=70.12MB mem-reservation=15.75MB thread-reservat | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.06MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> i_manufact_id, RF000[bloom] -> i_manufact_id @@ -829,7 +829,7 @@ Per-Host Resources: mem-estimate=70.12MB mem-reservation=15.75MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -857,7 +857,7 @@ Per-Host Resources: mem-estimate=70.12MB mem-reservation=15.75MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(5 AS INT) @@ -884,7 +884,7 @@ Per-Host Resources: mem-estimate=70.12MB mem-reservation=15.75MB thread-reservat in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=109.75MB Threads=35 -Per-Host Resource Estimates: Memory=473MB +Per-Host Resource Estimates: Memory=478MB F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -919,7 +919,7 @@ Per-Instance Resources: mem-estimate=10.16MB mem-reservation=1.94MB thread-reser | in pipelines: 38(GETNEXT), 45(GETNEXT), 52(GETNEXT) | F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=20.70MB mem-reservation=3.94MB thread-reservation=1 31:AGGREGATE [STREAMING] | output: sum(total_sales) | group by: i_manufact_id @@ -945,7 +945,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | in pipelines: 21(GETNEXT) | | | F17:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=10.28MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.85MB mem-reservation=2.00MB thread-reservation=1 | 30:AGGREGATE [STREAMING] | | output: sum(ws_ext_sales_price) | | group by: i_manufact_id @@ -974,7 +974,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | | in pipelines: 25(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.10MB mem-reservation=128.00KB thread-reservation=1 | | 25:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category IN ('Electronics') @@ -1010,7 +1010,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | | in pipelines: 24(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=256.00KB thread-reservation=1 | | 24:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF017[min_max] -> i_manufact_id @@ -1029,7 +1029,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | | F13:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.41MB mem-reservation=8.00MB thread-reservation=1 | 27:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=10 | | hash predicates: ws_bill_addr_sk = ca_address_sk @@ -1052,7 +1052,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | | in pipelines: 23(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | | 23:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1088,7 +1088,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | | in pipelines: 22(GETNEXT) | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 22:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(5 AS INT) @@ -1126,7 +1126,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | in pipelines: 11(GETNEXT) | | | F10:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3 -| Per-Instance Resources: mem-estimate=10.33MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.90MB mem-reservation=2.00MB thread-reservation=1 | 20:AGGREGATE [STREAMING] | | output: sum(cs_ext_sales_price) | | group by: i_manufact_id @@ -1155,7 +1155,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | | in pipelines: 15(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.10MB mem-reservation=128.00KB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category IN ('Electronics') @@ -1192,7 +1192,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF009[min_max] -> i_manufact_id, RF008[bloom] -> i_manufact_id @@ -1211,7 +1211,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=48.61MB mem-reservation=8.00MB thread-reservation=1 | 17:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=06 | | hash predicates: cs_bill_addr_sk = ca_address_sk @@ -1234,7 +1234,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | | in pipelines: 13(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | | 13:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1270,7 +1270,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | | | in pipelines: 12(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(5 AS INT) @@ -1309,7 +1309,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reservation=1 10:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: i_manufact_id @@ -1338,7 +1338,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.10MB mem-reservation=128.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category IN ('Electronics') @@ -1375,7 +1375,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=256.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> i_manufact_id, RF000[bloom] -> i_manufact_id @@ -1409,7 +1409,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1445,7 +1445,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(5 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test index f93f1dac3..b85075e84 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test @@ -160,7 +160,7 @@ PLAN-ROOT SINK in pipelines: 08(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=27.08MB Threads=13 -Per-Host Resource Estimates: Memory=335MB +Per-Host Resource Estimates: Memory=336MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.33MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -196,7 +196,7 @@ Per-Host Resources: mem-estimate=12.42MB mem-reservation=5.75MB thread-reservati | | in pipelines: 08(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=80.85MB mem-reservation=1.00MB thread-reservation=2 | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | stored statistics: @@ -213,7 +213,7 @@ Per-Host Resources: mem-estimate=12.42MB mem-reservation=5.75MB thread-reservati | in pipelines: 15(GETNEXT) | F04:PLAN FRAGMENT [HASH(ss_ticket_number,ss_customer_sk)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=10.27MB mem-reservation=1.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=10.28MB mem-reservation=1.94MB thread-reservation=1 15:AGGREGATE [FINALIZE] | output: count:merge(*) | group by: ss_ticket_number, ss_customer_sk @@ -228,7 +228,7 @@ Per-Host Resources: mem-estimate=10.27MB mem-reservation=1.94MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=83.87MB mem-reservation=13.81MB thread-reservation=2 runtime-filters-memory=4.00MB +Per-Host Resources: mem-estimate=84.09MB mem-reservation=13.81MB thread-reservation=2 runtime-filters-memory=4.00MB 07:AGGREGATE [STREAMING] | output: count(*) | group by: ss_ticket_number, ss_customer_sk @@ -250,7 +250,7 @@ Per-Host Resources: mem-estimate=83.87MB mem-reservation=13.81MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.14MB mem-reservation=16.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_county IN ('Saginaw County', 'Sumner County', 'Appanoose County', 'Daviess County', 'Fairfield County', 'Raleigh County', 'Ziebach County', 'Williamson County') @@ -278,7 +278,7 @@ Per-Host Resources: mem-estimate=83.87MB mem-reservation=13.81MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: date_dim.d_year IN (CAST(1998 AS INT), CAST(1999 AS INT), CAST(2000 AS INT)), (date_dim.d_dom >= CAST(1 AS INT) AND date_dim.d_dom <= CAST(3 AS INT) OR date_dim.d_dom >= CAST(25 AS INT) AND date_dim.d_dom <= CAST(28 AS INT)) @@ -306,7 +306,7 @@ Per-Host Resources: mem-estimate=83.87MB mem-reservation=13.81MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.14MB mem-reservation=64.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: household_demographics.hd_vehicle_count > CAST(0 AS INT), household_demographics.hd_buy_potential IN ('>10000', 'unknown'), (CASE WHEN household_demographics.hd_vehicle_count > CAST(0 AS INT) THEN CAST(household_demographics.hd_dep_count AS DOUBLE) / CAST(household_demographics.hd_vehicle_count AS DOUBLE) ELSE NULL END) > CAST(1.2 AS DOUBLE) @@ -333,7 +333,7 @@ Per-Host Resources: mem-estimate=83.87MB mem-reservation=13.81MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=43.83MB Threads=16 -Per-Host Resource Estimates: Memory=182MB +Per-Host Resource Estimates: Memory=186MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.57MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -348,7 +348,7 @@ PLAN-ROOT SINK | in pipelines: 10(GETNEXT) | F06:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=411.49KB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=583.79KB mem-reservation=0B thread-reservation=1 10:TOP-N [LIMIT=100000] | order by: c_last_name ASC, c_first_name ASC, c_salutation ASC, c_preferred_cust_flag DESC | mem-estimate=246.96KB mem-reservation=0B thread-reservation=0 @@ -377,7 +377,7 @@ Per-Instance Resources: mem-estimate=411.49KB mem-reservation=0B thread-reservat | | in pipelines: 08(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.69MB mem-reservation=1.00MB thread-reservation=1 | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | stored statistics: @@ -394,7 +394,7 @@ Per-Instance Resources: mem-estimate=411.49KB mem-reservation=0B thread-reservat | in pipelines: 15(GETNEXT) | F04:PLAN FRAGMENT [HASH(ss_ticket_number,ss_customer_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=10.56MB mem-reservation=1.94MB thread-reservation=1 15:AGGREGATE [FINALIZE] | output: count:merge(*) | group by: ss_ticket_number, ss_customer_sk @@ -410,7 +410,7 @@ Per-Instance Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | output: count(*) | group by: ss_ticket_number, ss_customer_sk @@ -440,7 +440,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_county IN ('Saginaw County', 'Sumner County', 'Appanoose County', 'Daviess County', 'Fairfield County', 'Raleigh County', 'Ziebach County', 'Williamson County') @@ -476,7 +476,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: date_dim.d_year IN (CAST(1998 AS INT), CAST(1999 AS INT), CAST(2000 AS INT)), (date_dim.d_dom >= CAST(1 AS INT) AND date_dim.d_dom <= CAST(3 AS INT) OR date_dim.d_dom >= CAST(25 AS INT) AND date_dim.d_dom <= CAST(28 AS INT)) @@ -512,7 +512,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.14MB mem-reservation=64.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: household_demographics.hd_vehicle_count > CAST(0 AS INT), household_demographics.hd_buy_potential IN ('>10000', 'unknown'), (CASE WHEN household_demographics.hd_vehicle_count > CAST(0 AS INT) THEN CAST(household_demographics.hd_dep_count AS DOUBLE) / CAST(household_demographics.hd_vehicle_count AS DOUBLE) ELSE NULL END) > CAST(1.2 AS DOUBLE) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test index aba88cf1c..3f2793c95 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test @@ -300,7 +300,7 @@ Per-Host Resources: mem-estimate=28.04MB mem-reservation=14.62MB thread-reservat | | in pipelines: 07(GETNEXT), 10(GETNEXT) | | | F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=99.95MB mem-reservation=7.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=100.05MB mem-reservation=7.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 06:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=10 row-size=4B cardinality=140.03K @@ -320,7 +320,7 @@ Per-Host Resources: mem-estimate=28.04MB mem-reservation=14.62MB thread-reservat | | | | in pipelines: 11(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | | 11:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2002 AS INT), d_qoy < CAST(4 AS INT) @@ -359,7 +359,7 @@ Per-Host Resources: mem-estimate=28.04MB mem-reservation=14.62MB thread-reservat | | | in pipelines: 08(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2002 AS INT), d_qoy < CAST(4 AS INT) @@ -412,7 +412,7 @@ Per-Host Resources: mem-estimate=28.04MB mem-reservation=14.62MB thread-reservat | | | in pipelines: 01(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=256.00KB thread-reservation=2 | | 01:SCAN HDFS [tpcds_parquet.customer_address ca, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -437,7 +437,7 @@ Per-Host Resources: mem-estimate=28.04MB mem-reservation=14.62MB thread-reservat | | | in pipelines: 00(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=50.00MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=50.06MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB | | 00:SCAN HDFS [tpcds_parquet.customer c, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF001[min_max] -> c.c_customer_sk, RF007[min_max] -> c.c_current_addr_sk, RF000[bloom] -> c.c_customer_sk, RF006[bloom] -> c.c_current_addr_sk @@ -466,7 +466,7 @@ Per-Host Resources: mem-estimate=28.04MB mem-reservation=14.62MB thread-reservat | in pipelines: 03(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=20.95MB mem-reservation=5.44MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=21.27MB mem-reservation=5.44MB thread-reservation=2 runtime-filters-memory=3.00MB 05:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_sold_date_sk = d_date_sk | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -481,7 +481,7 @@ Per-Host Resources: mem-estimate=20.95MB mem-reservation=5.44MB thread-reservati | | in pipelines: 04(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2002 AS INT), d_qoy < CAST(4 AS INT) @@ -508,7 +508,7 @@ Per-Host Resources: mem-estimate=20.95MB mem-reservation=5.44MB thread-reservati in pipelines: 03(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=97.25MB Threads=23 -Per-Host Resource Estimates: Memory=311MB +Per-Host Resource Estimates: Memory=320MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -543,7 +543,7 @@ Per-Instance Resources: mem-estimate=13.77MB mem-reservation=2.88MB thread-reser | in pipelines: 03(GETNEXT) | F05:PLAN FRAGMENT [HASH(c.c_customer_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=11.35MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=13.97MB mem-reservation=3.00MB thread-reservation=1 17:AGGREGATE [STREAMING] | output: count(*), min(cd_dep_count), max(cd_dep_count), avg(CAST(cd_dep_count AS BIGINT)), min(cd_dep_employed_count), max(cd_dep_employed_count), avg(CAST(cd_dep_employed_count AS BIGINT)), min(cd_dep_college_count), max(cd_dep_college_count), avg(CAST(cd_dep_college_count AS BIGINT)) | group by: ca_state, cd_gender, cd_marital_status, cd_dep_count, cd_dep_employed_count, cd_dep_college_count @@ -573,7 +573,7 @@ Per-Instance Resources: mem-estimate=11.35MB mem-reservation=3.00MB thread-reser | | | F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=48.19MB mem-reservation=4.00MB thread-reservation=1 | 06:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=10 row-size=4B cardinality=140.03K @@ -601,7 +601,7 @@ Per-Instance Resources: mem-estimate=11.35MB mem-reservation=3.00MB thread-reser | | | | in pipelines: 11(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | | 11:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2002 AS INT), d_qoy < CAST(4 AS INT) @@ -648,7 +648,7 @@ Per-Instance Resources: mem-estimate=11.35MB mem-reservation=3.00MB thread-reser | | | in pipelines: 08(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2002 AS INT), d_qoy < CAST(4 AS INT) @@ -695,7 +695,7 @@ Per-Instance Resources: mem-estimate=11.35MB mem-reservation=3.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.97MB mem-reservation=8.00MB thread-reservation=1 | 14:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=04 | | hash predicates: c.c_current_addr_sk = ca.ca_address_sk @@ -718,7 +718,7 @@ Per-Instance Resources: mem-estimate=11.35MB mem-reservation=3.00MB thread-reser | | | in pipelines: 01(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.customer_address ca, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -752,7 +752,7 @@ Per-Instance Resources: mem-estimate=11.35MB mem-reservation=3.00MB thread-reser | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=2.00MB thread-reservation=1 | | 00:SCAN HDFS [tpcds_parquet.customer c, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF001[min_max] -> c.c_customer_sk, RF007[min_max] -> c.c_current_addr_sk, RF000[bloom] -> c.c_customer_sk, RF006[bloom] -> c.c_current_addr_sk @@ -782,7 +782,7 @@ Per-Instance Resources: mem-estimate=11.35MB mem-reservation=3.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +Per-Instance Resources: mem-estimate=16.66MB mem-reservation=512.00KB thread-reservation=1 05:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=06 | hash predicates: ss_sold_date_sk = d_date_sk @@ -805,7 +805,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-res | | in pipelines: 04(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2002 AS INT), d_qoy < CAST(4 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test index 4d0fd3e9e..b28e754b9 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test @@ -151,7 +151,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=46.27MB Threads=11 -Per-Host Resource Estimates: Memory=277MB +Per-Host Resource Estimates: Memory=279MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -222,7 +222,7 @@ Per-Host Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=104.61MB mem-reservation=17.75MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=106.09MB mem-reservation=17.75MB thread-reservation=2 runtime-filters-memory=3.00MB 07:AGGREGATE [STREAMING] | Class 0 | output: sum(ss_net_profit), sum(ss_ext_sales_price) @@ -251,7 +251,7 @@ Per-Host Resources: mem-estimate=104.61MB mem-reservation=17.75MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_state IN ('TN') @@ -279,7 +279,7 @@ Per-Host Resources: mem-estimate=104.61MB mem-reservation=17.75MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.19MB mem-reservation=256.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -304,7 +304,7 @@ Per-Host Resources: mem-estimate=104.61MB mem-reservation=17.75MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d1.d_year = CAST(2001 AS INT) @@ -331,7 +331,7 @@ Per-Host Resources: mem-estimate=104.61MB mem-reservation=17.75MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=87.77MB Threads=13 -Per-Host Resource Estimates: Memory=277MB +Per-Host Resource Estimates: Memory=287MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -403,7 +403,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=46.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=50.68MB mem-reservation=8.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | Class 0 | output: sum(ss_net_profit), sum(ss_ext_sales_price) @@ -440,7 +440,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_state IN ('TN') @@ -476,7 +476,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -509,7 +509,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d1.d_year = CAST(2001 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test index b7ac3b093..7aaa78f19 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test @@ -118,7 +118,7 @@ PLAN-ROOT SINK in pipelines: 03(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=52.31MB Threads=11 -Per-Host Resource Estimates: Memory=297MB +Per-Host Resource Estimates: Memory=301MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -152,7 +152,7 @@ Per-Host Resources: mem-estimate=16.20MB mem-reservation=8.50MB thread-reservati | in pipelines: 03(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=61.64MB mem-reservation=13.94MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=62.68MB mem-reservation=13.94MB thread-reservation=2 runtime-filters-memory=1.00MB 07:AGGREGATE [STREAMING] | group by: i_item_id, i_item_desc, i_current_price | mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0 @@ -173,7 +173,7 @@ Per-Host Resources: mem-estimate=61.64MB mem-reservation=13.94MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [HASH(inv_date_sk)] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=3.63MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=4.24MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB | 05:HASH JOIN [INNER JOIN, PARTITIONED] | | hash predicates: inv_date_sk = d_date_sk | | fk/pk conjuncts: inv_date_sk = d_date_sk @@ -188,7 +188,7 @@ Per-Host Resources: mem-estimate=61.64MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 02(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.23MB mem-reservation=1.00MB thread-reservation=2 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '2000-04-01', CAST(d_date AS DATE) >= DATE '2000-02-01' @@ -208,7 +208,7 @@ Per-Host Resources: mem-estimate=61.64MB mem-reservation=13.94MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=99.95MB mem-reservation=19.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=101.35MB mem-reservation=19.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 04:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: inv_item_sk = i_item_sk | | fk/pk conjuncts: inv_item_sk = i_item_sk @@ -223,7 +223,7 @@ Per-Host Resources: mem-estimate=61.64MB mem-reservation=13.94MB thread-reservat | | | in pipelines: 00(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=80.63MB mem-reservation=2.00MB thread-reservation=2 | | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_current_price <= CAST(98 AS DECIMAL(3,0)), i_current_price >= CAST(68 AS DECIMAL(3,0)), i_manufact_id IN (CAST(677 AS INT), CAST(940 AS INT), CAST(694 AS INT), CAST(808 AS INT)) @@ -263,7 +263,7 @@ Per-Host Resources: mem-estimate=61.64MB mem-reservation=13.94MB thread-reservat in pipelines: 03(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=58.19MB Threads=10 -Per-Host Resource Estimates: Memory=159MB +Per-Host Resource Estimates: Memory=164MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -298,7 +298,7 @@ Per-Instance Resources: mem-estimate=16.20MB mem-reservation=8.50MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=58.00MB mem-reservation=11.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=59.74MB mem-reservation=11.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | group by: i_item_id, i_item_desc, i_current_price | mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0 @@ -327,7 +327,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=11.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [HASH(inv_date_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=492.72KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=1.30MB mem-reservation=0B thread-reservation=1 | 05:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=01 | | hash predicates: inv_date_sk = d_date_sk @@ -350,7 +350,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=11.00MB thread-rese | | | in pipelines: 02(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.23MB mem-reservation=1.00MB thread-reservation=1 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '2000-04-01', CAST(d_date AS DATE) >= DATE '2000-02-01' @@ -371,7 +371,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=11.00MB thread-rese | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=33.41MB mem-reservation=16.00MB thread-reservation=1 | 04:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=02 | | hash predicates: inv_item_sk = i_item_sk @@ -394,7 +394,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=11.00MB thread-rese | | | in pipelines: 00(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.63MB mem-reservation=2.00MB thread-reservation=1 | | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_current_price <= CAST(98 AS DECIMAL(3,0)), i_current_price >= CAST(68 AS DECIMAL(3,0)), i_manufact_id IN (CAST(677 AS INT), CAST(940 AS INT), CAST(694 AS INT), CAST(808 AS INT)) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q38.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q38.test index 6932c4722..9c03dd8de 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q38.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q38.test @@ -231,7 +231,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=336.81MB Threads=22 -Per-Host Resource Estimates: Memory=899MB +Per-Host Resource Estimates: Memory=900MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -306,7 +306,7 @@ Per-Host Resources: mem-estimate=177.11MB mem-reservation=108.00MB thread-reserv | | | in pipelines: 14(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.17MB mem-reservation=1.00MB thread-reservation=2 | | 14:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -331,7 +331,7 @@ Per-Host Resources: mem-estimate=177.11MB mem-reservation=108.00MB thread-reserv | | | in pipelines: 13(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.13MB mem-reservation=2.00MB thread-reservation=2 | | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -404,7 +404,7 @@ Per-Host Resources: mem-estimate=177.11MB mem-reservation=108.00MB thread-reserv | | | in pipelines: 08(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.17MB mem-reservation=1.00MB thread-reservation=2 | | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -429,7 +429,7 @@ Per-Host Resources: mem-estimate=177.11MB mem-reservation=108.00MB thread-reserv | | | in pipelines: 07(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.13MB mem-reservation=2.00MB thread-reservation=2 | | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -487,7 +487,7 @@ Per-Host Resources: mem-estimate=80.58MB mem-reservation=46.94MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=52.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=4.00MB +| Per-Host Resources: mem-estimate=52.17MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=4.00MB | 02:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF000[bloom] -> tpcds_parquet.customer.c_first_name, RF001[bloom] -> tpcds_parquet.customer.c_last_name, RF003[bloom] -> tpcds_parquet.customer.c_first_name, RF004[bloom] -> tpcds_parquet.customer.c_last_name @@ -513,7 +513,7 @@ Per-Host Resources: mem-estimate=80.58MB mem-reservation=46.94MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=50.00MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=50.13MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -541,7 +541,7 @@ Per-Host Resources: mem-estimate=80.58MB mem-reservation=46.94MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=480.62MB Threads=25 -Per-Host Resource Estimates: Memory=765MB +Per-Host Resource Estimates: Memory=770MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -603,7 +603,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | in pipelines: 12(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=66.00MB mem-reservation=38.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=66.48MB mem-reservation=38.00MB thread-reservation=1 | 17:AGGREGATE [STREAMING] | | group by: c_last_name, c_first_name, d_date | | mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -632,7 +632,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | | in pipelines: 14(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.17MB mem-reservation=1.00MB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -665,7 +665,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | | in pipelines: 13(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -726,7 +726,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | | F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=82.00MB mem-reservation=38.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=82.73MB mem-reservation=38.00MB thread-reservation=1 | 11:AGGREGATE [STREAMING] | | group by: c_last_name, c_first_name, d_date | | mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -755,7 +755,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | | in pipelines: 08(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.17MB mem-reservation=1.00MB thread-reservation=1 | | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -788,7 +788,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | | in pipelines: 07(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -826,7 +826,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-reservation=1 +Per-Instance Resources: mem-estimate=51.45MB mem-reservation=34.50MB thread-reservation=1 05:AGGREGATE [STREAMING] | group by: c_last_name, c_first_name, d_date | mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -856,7 +856,7 @@ Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-rese | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.17MB mem-reservation=1.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF000[bloom] -> tpcds_parquet.customer.c_first_name, RF001[bloom] -> tpcds_parquet.customer.c_last_name, RF003[bloom] -> tpcds_parquet.customer.c_first_name, RF004[bloom] -> tpcds_parquet.customer.c_last_name @@ -891,7 +891,7 @@ Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-rese | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test index 9778892a4..8658fb34f 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test @@ -285,7 +285,7 @@ Per-Host Resources: mem-estimate=59.29MB mem-reservation=42.00MB thread-reservat | | in pipelines: 08(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=263.77MB mem-reservation=58.81MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=264.10MB mem-reservation=58.81MB thread-reservation=2 runtime-filters-memory=3.00MB | 15:AGGREGATE [STREAMING] | | output: stddev_samp(inv_quantity_on_hand), avg(CAST(inv_quantity_on_hand AS BIGINT)) | | group by: w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy @@ -307,7 +307,7 @@ Per-Host Resources: mem-estimate=59.29MB mem-reservation=42.00MB thread-reservat | | | in pipelines: 10(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.15MB mem-reservation=16.00KB thread-reservation=2 | | 10:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | | HDFS partitions=1/1 files=1 size=4.38KB | | stored statistics: @@ -332,7 +332,7 @@ Per-Host Resources: mem-estimate=59.29MB mem-reservation=42.00MB thread-reservat | | | in pipelines: 09(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=2 | | 09:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -357,7 +357,7 @@ Per-Host Resources: mem-estimate=59.29MB mem-reservation=42.00MB thread-reservat | | | in pipelines: 11(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 11:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), tpcds_parquet.date_dim.d_moy = CAST(2 AS INT) @@ -403,7 +403,7 @@ Per-Host Resources: mem-estimate=161.32MB mem-reservation=34.00MB thread-reserva | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -Per-Host Resources: mem-estimate=265.77MB mem-reservation=60.81MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=266.10MB mem-reservation=60.81MB thread-reservation=2 runtime-filters-memory=5.00MB 07:AGGREGATE [STREAMING] | output: stddev_samp(inv_quantity_on_hand), avg(CAST(inv_quantity_on_hand AS BIGINT)) | group by: w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy @@ -425,7 +425,7 @@ Per-Host Resources: mem-estimate=265.77MB mem-reservation=60.81MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.15MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB | 02:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | runtime filters: RF003[min_max] -> tpcds_parquet.warehouse.w_warehouse_sk, RF001[bloom] -> tpcds_parquet.warehouse.w_warehouse_sk @@ -451,7 +451,7 @@ Per-Host Resources: mem-estimate=265.77MB mem-reservation=60.81MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=17.00MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=17.05MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF002[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk @@ -477,7 +477,7 @@ Per-Host Resources: mem-estimate=265.77MB mem-reservation=60.81MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), tpcds_parquet.date_dim.d_moy = CAST(1 AS INT) @@ -503,7 +503,7 @@ Per-Host Resources: mem-estimate=265.77MB mem-reservation=60.81MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=254.78MB Threads=19 -Per-Host Resource Estimates: Memory=885MB +Per-Host Resource Estimates: Memory=886MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=39.63MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -562,7 +562,7 @@ Per-Instance Resources: mem-estimate=23.29MB mem-reservation=6.00MB thread-reser | | | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=158.78MB mem-reservation=50.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=159.29MB mem-reservation=50.00MB thread-reservation=1 | 15:AGGREGATE [STREAMING] | | output: stddev_samp(inv_quantity_on_hand), avg(CAST(inv_quantity_on_hand AS BIGINT)) | | group by: w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy @@ -592,7 +592,7 @@ Per-Instance Resources: mem-estimate=23.29MB mem-reservation=6.00MB thread-reser | | | in pipelines: 10(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.15MB mem-reservation=16.00KB thread-reservation=1 | | 10:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | | HDFS partitions=1/1 files=1 size=4.38KB | | stored statistics: @@ -625,7 +625,7 @@ Per-Instance Resources: mem-estimate=23.29MB mem-reservation=6.00MB thread-reser | | | in pipelines: 09(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | | 09:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -658,7 +658,7 @@ Per-Instance Resources: mem-estimate=23.29MB mem-reservation=6.00MB thread-reser | | | in pipelines: 11(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 11:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), tpcds_parquet.date_dim.d_moy = CAST(2 AS INT) @@ -705,7 +705,7 @@ Per-Instance Resources: mem-estimate=161.32MB mem-reservation=34.00MB thread-res | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=158.78MB mem-reservation=50.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=159.29MB mem-reservation=50.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | output: stddev_samp(inv_quantity_on_hand), avg(CAST(inv_quantity_on_hand AS BIGINT)) | group by: w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy @@ -736,7 +736,7 @@ Per-Instance Resources: mem-estimate=158.78MB mem-reservation=50.00MB thread-res | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.15MB mem-reservation=16.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | runtime filters: RF003[min_max] -> tpcds_parquet.warehouse.w_warehouse_sk, RF001[bloom] -> tpcds_parquet.warehouse.w_warehouse_sk @@ -771,7 +771,7 @@ Per-Instance Resources: mem-estimate=158.78MB mem-reservation=50.00MB thread-res | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF002[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk @@ -805,7 +805,7 @@ Per-Instance Resources: mem-estimate=158.78MB mem-reservation=50.00MB thread-res | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), tpcds_parquet.date_dim.d_moy = CAST(1 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test index da8d7f839..85ed07e3c 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test @@ -286,7 +286,7 @@ Per-Host Resources: mem-estimate=59.29MB mem-reservation=42.00MB thread-reservat | | in pipelines: 08(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=263.77MB mem-reservation=58.81MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=264.10MB mem-reservation=58.81MB thread-reservation=2 runtime-filters-memory=3.00MB | 15:AGGREGATE [STREAMING] | | output: stddev_samp(inv_quantity_on_hand), avg(CAST(inv_quantity_on_hand AS BIGINT)) | | group by: w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy @@ -308,7 +308,7 @@ Per-Host Resources: mem-estimate=59.29MB mem-reservation=42.00MB thread-reservat | | | in pipelines: 10(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.15MB mem-reservation=16.00KB thread-reservation=2 | | 10:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | | HDFS partitions=1/1 files=1 size=4.38KB | | stored statistics: @@ -333,7 +333,7 @@ Per-Host Resources: mem-estimate=59.29MB mem-reservation=42.00MB thread-reservat | | | in pipelines: 09(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=2 | | 09:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -358,7 +358,7 @@ Per-Host Resources: mem-estimate=59.29MB mem-reservation=42.00MB thread-reservat | | | in pipelines: 11(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 11:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), tpcds_parquet.date_dim.d_moy = CAST(2 AS INT) @@ -404,7 +404,7 @@ Per-Host Resources: mem-estimate=161.32MB mem-reservation=34.00MB thread-reserva | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -Per-Host Resources: mem-estimate=265.77MB mem-reservation=60.81MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=266.10MB mem-reservation=60.81MB thread-reservation=2 runtime-filters-memory=5.00MB 07:AGGREGATE [STREAMING] | output: stddev_samp(inv_quantity_on_hand), avg(CAST(inv_quantity_on_hand AS BIGINT)) | group by: w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy @@ -426,7 +426,7 @@ Per-Host Resources: mem-estimate=265.77MB mem-reservation=60.81MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.15MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB | 02:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | runtime filters: RF003[min_max] -> tpcds_parquet.warehouse.w_warehouse_sk, RF001[bloom] -> tpcds_parquet.warehouse.w_warehouse_sk @@ -452,7 +452,7 @@ Per-Host Resources: mem-estimate=265.77MB mem-reservation=60.81MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=17.00MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=17.05MB mem-reservation=1.25MB thread-reservation=2 runtime-filters-memory=1.00MB | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF002[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk @@ -478,7 +478,7 @@ Per-Host Resources: mem-estimate=265.77MB mem-reservation=60.81MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), tpcds_parquet.date_dim.d_moy = CAST(1 AS INT) @@ -504,7 +504,7 @@ Per-Host Resources: mem-estimate=265.77MB mem-reservation=60.81MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=254.78MB Threads=19 -Per-Host Resource Estimates: Memory=885MB +Per-Host Resource Estimates: Memory=886MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=39.63MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -563,7 +563,7 @@ Per-Instance Resources: mem-estimate=23.29MB mem-reservation=6.00MB thread-reser | | | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=158.78MB mem-reservation=50.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=159.29MB mem-reservation=50.00MB thread-reservation=1 | 15:AGGREGATE [STREAMING] | | output: stddev_samp(inv_quantity_on_hand), avg(CAST(inv_quantity_on_hand AS BIGINT)) | | group by: w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy @@ -593,7 +593,7 @@ Per-Instance Resources: mem-estimate=23.29MB mem-reservation=6.00MB thread-reser | | | in pipelines: 10(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.15MB mem-reservation=16.00KB thread-reservation=1 | | 10:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | | HDFS partitions=1/1 files=1 size=4.38KB | | stored statistics: @@ -626,7 +626,7 @@ Per-Instance Resources: mem-estimate=23.29MB mem-reservation=6.00MB thread-reser | | | in pipelines: 09(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | | 09:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -659,7 +659,7 @@ Per-Instance Resources: mem-estimate=23.29MB mem-reservation=6.00MB thread-reser | | | in pipelines: 11(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 11:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), tpcds_parquet.date_dim.d_moy = CAST(2 AS INT) @@ -706,7 +706,7 @@ Per-Instance Resources: mem-estimate=161.32MB mem-reservation=34.00MB thread-res | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=158.78MB mem-reservation=50.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=159.29MB mem-reservation=50.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | output: stddev_samp(inv_quantity_on_hand), avg(CAST(inv_quantity_on_hand AS BIGINT)) | group by: w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy @@ -737,7 +737,7 @@ Per-Instance Resources: mem-estimate=158.78MB mem-reservation=50.00MB thread-res | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.15MB mem-reservation=16.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | runtime filters: RF003[min_max] -> tpcds_parquet.warehouse.w_warehouse_sk, RF001[bloom] -> tpcds_parquet.warehouse.w_warehouse_sk @@ -772,7 +772,7 @@ Per-Instance Resources: mem-estimate=158.78MB mem-reservation=50.00MB thread-res | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF002[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk @@ -806,7 +806,7 @@ Per-Instance Resources: mem-estimate=158.78MB mem-reservation=50.00MB thread-res | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), tpcds_parquet.date_dim.d_moy = CAST(1 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test index b7091b9b0..72ccbd481 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test @@ -193,7 +193,7 @@ Per-Host Resources: mem-estimate=270.42MB mem-reservation=27.31MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '2000-04-10', CAST(d_date AS DATE) >= DATE '2000-02-10' @@ -221,7 +221,7 @@ Per-Host Resources: mem-estimate=270.42MB mem-reservation=27.31MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | stored statistics: @@ -246,7 +246,7 @@ Per-Host Resources: mem-estimate=270.42MB mem-reservation=27.31MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.17MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_current_price <= CAST(1.49 AS DECIMAL(3,2)), i_current_price >= CAST(0.99 AS DECIMAL(2,2)) @@ -273,7 +273,7 @@ Per-Host Resources: mem-estimate=270.42MB mem-reservation=27.31MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=73.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=73.09MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 01:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | HDFS partitions=1/1 files=1 size=10.62MB | runtime filters: RF005[min_max] -> tpcds_parquet.catalog_returns.cr_item_sk, RF004[bloom] -> tpcds_parquet.catalog_returns.cr_item_sk @@ -297,7 +297,7 @@ Per-Host Resources: mem-estimate=270.42MB mem-reservation=27.31MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=55.08MB Threads=11 -Per-Host Resource Estimates: Memory=183MB +Per-Host Resource Estimates: Memory=185MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -333,7 +333,7 @@ Per-Instance Resources: mem-estimate=10.44MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=58.00MB mem-reservation=10.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=58.91MB mem-reservation=10.00MB thread-reservation=1 09:AGGREGATE [STREAMING] | output: sum(CASE WHEN (CAST(d_date AS DATE) < DATE '2000-03-11') THEN cs_sales_price - coalesce(cr_refunded_cash, CAST(0 AS DECIMAL(7,2))) ELSE CAST(0 AS DECIMAL(8,2)) END), sum(CASE WHEN (CAST(d_date AS DATE) >= DATE '2000-03-11') THEN cs_sales_price - coalesce(cr_refunded_cash, CAST(0 AS DECIMAL(7,2))) ELSE CAST(0 AS DECIMAL(8,2)) END) | group by: w_state, i_item_id @@ -363,7 +363,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=10.00MB thread-rese | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '2000-04-10', CAST(d_date AS DATE) >= DATE '2000-02-10' @@ -399,7 +399,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=10.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | stored statistics: @@ -432,7 +432,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=10.00MB thread-rese | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.17MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_current_price <= CAST(1.49 AS DECIMAL(3,2)), i_current_price >= CAST(0.99 AS DECIMAL(2,2)) @@ -468,7 +468,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=10.00MB thread-rese | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.09MB mem-reservation=2.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | HDFS partitions=1/1 files=1 size=10.62MB | runtime filters: RF005[min_max] -> tpcds_parquet.catalog_returns.cr_item_sk, RF004[bloom] -> tpcds_parquet.catalog_returns.cr_item_sk diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q41.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q41.test index 0b7f4c411..5bac801b3 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q41.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q41.test @@ -161,7 +161,7 @@ Per-Host Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Host Resources: mem-estimate=59.95MB mem-reservation=4.44MB thread-reservation=2 +Per-Host Resources: mem-estimate=60.07MB mem-reservation=4.44MB thread-reservation=2 04:AGGREGATE [STREAMING] | group by: (i_product_name) | mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0 @@ -182,7 +182,7 @@ Per-Host Resources: mem-estimate=59.95MB mem-reservation=4.44MB thread-reservati | | in pipelines: 07(GETNEXT) | | | F02:PLAN FRAGMENT [HASH(i_manufact)] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.14MB mem-reservation=1.94MB thread-reservation=1 | 07:AGGREGATE [FINALIZE] | | output: count:merge(*) | | group by: i_manufact @@ -197,7 +197,7 @@ Per-Host Resources: mem-estimate=59.95MB mem-reservation=4.44MB thread-reservati | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=90.00MB mem-reservation=2.12MB thread-reservation=2 +| Per-Host Resources: mem-estimate=90.14MB mem-reservation=2.12MB thread-reservation=2 | 02:AGGREGATE [STREAMING] | | output: count(*) | | group by: i_manufact @@ -264,7 +264,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.50MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.13MB mem-reservation=2.50MB thread-reservation=1 04:AGGREGATE [STREAMING] | group by: (i_product_name) | mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0 @@ -293,7 +293,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.50MB thread-reser | | in pipelines: 07(GETNEXT) | | | F02:PLAN FRAGMENT [HASH(i_manufact)] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.14MB mem-reservation=1.94MB thread-reservation=1 | 07:AGGREGATE [FINALIZE] | | output: count:merge(*) | | group by: i_manufact @@ -308,7 +308,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.50MB thread-reser | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.12MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.14MB mem-reservation=2.12MB thread-reservation=1 | 02:AGGREGATE [STREAMING] | | output: count(*) | | group by: i_manufact diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test index 2bf4074b6..69247f590 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test @@ -101,7 +101,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=15.56MB Threads=8 -Per-Host Resource Estimates: Memory=174MB +Per-Host Resource Estimates: Memory=175MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -136,7 +136,7 @@ Per-Host Resources: mem-estimate=10.16MB mem-reservation=1.94MB thread-reservati | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=48.41MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=2.00MB 05:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: dt.d_year, item.i_category_id, item.i_category @@ -158,7 +158,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.date_dim dt, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: dt.d_year = CAST(1998 AS INT), dt.d_moy = CAST(12 AS INT) @@ -186,7 +186,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.15MB mem-reservation=256.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: item.i_manager_id = CAST(1 AS INT) @@ -213,7 +213,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=26.38MB Threads=9 -Per-Host Resource Estimates: Memory=120MB +Per-Host Resource Estimates: Memory=123MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -249,7 +249,7 @@ Per-Instance Resources: mem-estimate=10.29MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=27.08MB mem-reservation=3.00MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: dt.d_year, item.i_category_id, item.i_category @@ -279,7 +279,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.date_dim dt, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: dt.d_year = CAST(1998 AS INT), dt.d_moy = CAST(12 AS INT) @@ -315,7 +315,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.15MB mem-reservation=256.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: item.i_manager_id = CAST(1 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test old mode 100755 new mode 100644 index b2478af7f..c448add93 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test @@ -109,7 +109,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=14.84MB Threads=8 -Per-Host Resource Estimates: Memory=174MB +Per-Host Resource Estimates: Memory=176MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -144,7 +144,7 @@ Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservati | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.38MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=49.75MB mem-reservation=8.38MB thread-reservation=2 runtime-filters-memory=2.00MB 05:AGGREGATE [STREAMING] | output: sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) | group by: s_store_name, s_store_id @@ -166,7 +166,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.38MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=32.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.22MB mem-reservation=32.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -194,7 +194,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.38MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.12MB mem-reservation=512.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT) @@ -221,7 +221,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.38MB thread-reservati in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=25.16MB Threads=9 -Per-Host Resource Estimates: Memory=120MB +Per-Host Resource Estimates: Memory=128MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -257,7 +257,7 @@ Per-Instance Resources: mem-estimate=10.05MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.50MB thread-reservation=1 +Per-Instance Resources: mem-estimate=29.76MB mem-reservation=2.50MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) | group by: s_store_name, s_store_id @@ -287,7 +287,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.50MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.22MB mem-reservation=32.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -323,7 +323,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.50MB thread-reser | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=512.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test old mode 100755 new mode 100644 index 53f360fb5..86f241995 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test @@ -261,7 +261,7 @@ PLAN-ROOT SINK in pipelines: 19(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=36.44MB Threads=20 -Per-Host Resource Estimates: Memory=354MB +Per-Host Resource Estimates: Memory=355MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=13.11MB mem-reservation=11.69MB thread-reservation=1 PLAN-ROOT SINK @@ -287,7 +287,7 @@ PLAN-ROOT SINK | | in pipelines: 19(GETNEXT) | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.16MB mem-reservation=512.00KB thread-reservation=2 | 19:SCAN HDFS [tpcds_parquet.item i2, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -311,7 +311,7 @@ PLAN-ROOT SINK | | in pipelines: 40(GETNEXT) | | | F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 +| Per-Host Resources: mem-estimate=4.17MB mem-reservation=4.00MB thread-reservation=1 | 17:SELECT | | predicates: rank() < CAST(11 AS BIGINT) | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -360,7 +360,7 @@ PLAN-ROOT SINK | | | in pipelines: 13(GETNEXT) | | | | | F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| | Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | | 13:CARDINALITY CHECK | | | limit: 1 | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -374,7 +374,7 @@ PLAN-ROOT SINK | | | in pipelines: 36(GETNEXT) | | | | | F10:PLAN FRAGMENT [HASH(ss_store_sk)] hosts=3 instances=3 -| | Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| | Per-Host Resources: mem-estimate=10.06MB mem-reservation=1.94MB thread-reservation=1 | | 36:AGGREGATE [FINALIZE] | | | output: avg:merge(ss_net_profit) | | | group by: ss_store_sk @@ -389,7 +389,7 @@ PLAN-ROOT SINK | | | in pipelines: 11(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | Per-Host Resources: mem-estimate=58.00MB mem-reservation=3.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=58.19MB mem-reservation=3.00MB thread-reservation=2 | | 12:AGGREGATE [STREAMING] | | | output: avg(ss_net_profit) | | | group by: ss_store_sk @@ -424,7 +424,7 @@ PLAN-ROOT SINK | | in pipelines: 09(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=58.00MB mem-reservation=3.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=58.23MB mem-reservation=3.00MB thread-reservation=2 | 10:AGGREGATE [STREAMING] | | output: avg(ss_net_profit) | | group by: ss_item_sk @@ -459,7 +459,7 @@ PLAN-ROOT SINK | | in pipelines: 18(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.16MB mem-reservation=512.00KB thread-reservation=2 | 18:SCAN HDFS [tpcds_parquet.item i1, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -518,7 +518,7 @@ Per-Host Resources: mem-estimate=10.17MB mem-reservation=1.94MB thread-reservati | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 04:CARDINALITY CHECK | | limit: 1 | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -532,7 +532,7 @@ Per-Host Resources: mem-estimate=10.17MB mem-reservation=1.94MB thread-reservati | | in pipelines: 27(GETNEXT) | | | F03:PLAN FRAGMENT [HASH(ss_store_sk)] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.06MB mem-reservation=1.94MB thread-reservation=1 | 27:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | group by: ss_store_sk @@ -547,7 +547,7 @@ Per-Host Resources: mem-estimate=10.17MB mem-reservation=1.94MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=58.00MB mem-reservation=3.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=58.19MB mem-reservation=3.00MB thread-reservation=2 | 03:AGGREGATE [STREAMING] | | output: avg(ss_net_profit) | | group by: ss_store_sk @@ -582,7 +582,7 @@ Per-Host Resources: mem-estimate=10.17MB mem-reservation=1.94MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=58.00MB mem-reservation=3.00MB thread-reservation=2 +Per-Host Resources: mem-estimate=58.23MB mem-reservation=3.00MB thread-reservation=2 01:AGGREGATE [STREAMING] | output: avg(ss_net_profit) | group by: ss_item_sk @@ -605,7 +605,7 @@ Per-Host Resources: mem-estimate=58.00MB mem-reservation=3.00MB thread-reservati in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=63.88MB Threads=27 -Per-Host Resource Estimates: Memory=346MB +Per-Host Resource Estimates: Memory=350MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -639,7 +639,7 @@ PLAN-ROOT SINK | | in pipelines: 19(GETNEXT) | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | 19:SCAN HDFS [tpcds_parquet.item i2, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -671,7 +671,7 @@ PLAN-ROOT SINK | | in pipelines: 40(GETNEXT) | | | F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=4.17MB mem-reservation=4.00MB thread-reservation=1 | 17:SELECT | | predicates: rank() < CAST(11 AS BIGINT) | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -727,7 +727,7 @@ PLAN-ROOT SINK | | | in pipelines: 13(GETNEXT) | | | | | F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| | Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | | 13:CARDINALITY CHECK | | | limit: 1 | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -741,7 +741,7 @@ PLAN-ROOT SINK | | | in pipelines: 36(GETNEXT) | | | | | F10:PLAN FRAGMENT [HASH(ss_store_sk)] hosts=3 instances=6 -| | Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=10.06MB mem-reservation=1.94MB thread-reservation=1 | | 36:AGGREGATE [FINALIZE] | | | output: avg:merge(ss_net_profit) | | | group by: ss_store_sk @@ -756,7 +756,7 @@ PLAN-ROOT SINK | | | in pipelines: 11(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=26.38MB mem-reservation=3.00MB thread-reservation=1 | | 12:AGGREGATE [STREAMING] | | | output: avg(ss_net_profit) | | | group by: ss_store_sk @@ -791,7 +791,7 @@ PLAN-ROOT SINK | | in pipelines: 09(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.47MB mem-reservation=3.00MB thread-reservation=1 | 10:AGGREGATE [STREAMING] | | output: avg(ss_net_profit) | | group by: ss_item_sk @@ -834,7 +834,7 @@ PLAN-ROOT SINK | | in pipelines: 18(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | 18:SCAN HDFS [tpcds_parquet.item i1, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -900,7 +900,7 @@ Per-Instance Resources: mem-estimate=10.21MB mem-reservation=1.94MB thread-reser | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 04:CARDINALITY CHECK | | limit: 1 | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -914,7 +914,7 @@ Per-Instance Resources: mem-estimate=10.21MB mem-reservation=1.94MB thread-reser | | in pipelines: 27(GETNEXT) | | | F03:PLAN FRAGMENT [HASH(ss_store_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.06MB mem-reservation=1.94MB thread-reservation=1 | 27:AGGREGATE [FINALIZE] | | output: avg:merge(ss_net_profit) | | group by: ss_store_sk @@ -929,7 +929,7 @@ Per-Instance Resources: mem-estimate=10.21MB mem-reservation=1.94MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.38MB mem-reservation=3.00MB thread-reservation=1 | 03:AGGREGATE [STREAMING] | | output: avg(ss_net_profit) | | group by: ss_store_sk @@ -964,7 +964,7 @@ Per-Instance Resources: mem-estimate=10.21MB mem-reservation=1.94MB thread-reser | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.47MB mem-reservation=3.00MB thread-reservation=1 01:AGGREGATE [STREAMING] | output: avg(ss_net_profit) | group by: ss_item_sk diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test index 9e8add1f2..318eeb488 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test @@ -177,7 +177,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=46.12MB Threads=16 -Per-Host Resource Estimates: Memory=390MB +Per-Host Resource Estimates: Memory=391MB F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -234,7 +234,7 @@ Per-Host Resources: mem-estimate=22.98MB mem-reservation=11.75MB thread-reservat | | in pipelines: 20(GETNEXT) | | | F07:PLAN FRAGMENT [HASH(i_item_id)] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.12MB mem-reservation=1.94MB thread-reservation=1 | 20:AGGREGATE [FINALIZE] | | group by: i_item_id | | mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 @@ -247,7 +247,7 @@ Per-Host Resources: mem-estimate=22.98MB mem-reservation=11.75MB thread-reservat | | in pipelines: 05(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=42.00MB mem-reservation=2.50MB thread-reservation=2 +| Per-Host Resources: mem-estimate=42.12MB mem-reservation=2.50MB thread-reservation=2 | 06:AGGREGATE [STREAMING] | | group by: i_item_id | | mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0 @@ -281,7 +281,7 @@ Per-Host Resources: mem-estimate=22.98MB mem-reservation=11.75MB thread-reservat | | in pipelines: 04(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.16MB mem-reservation=512.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -306,7 +306,7 @@ Per-Host Resources: mem-estimate=22.98MB mem-reservation=11.75MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.36MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -338,7 +338,7 @@ Per-Host Resources: mem-estimate=138.48MB mem-reservation=17.69MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.05MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 01:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF003[min_max] -> c_current_addr_sk, RF002[bloom] -> c_current_addr_sk @@ -364,7 +364,7 @@ Per-Host Resources: mem-estimate=138.48MB mem-reservation=17.69MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_qoy = CAST(2 AS INT) @@ -390,7 +390,7 @@ Per-Host Resources: mem-estimate=138.48MB mem-reservation=17.69MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=58.69MB Threads=15 -Per-Host Resource Estimates: Memory=195MB +Per-Host Resource Estimates: Memory=197MB F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -425,7 +425,7 @@ Per-Instance Resources: mem-estimate=12.01MB mem-reservation=4.75MB thread-reser | in pipelines: 00(GETNEXT) | F04:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=2 instances=2 -Per-Instance Resources: mem-estimate=11.51MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=11.96MB mem-reservation=3.00MB thread-reservation=1 12:AGGREGATE [STREAMING] | output: sum(ws_sales_price) | group by: ca_zip, ca_city @@ -455,7 +455,7 @@ Per-Instance Resources: mem-estimate=11.51MB mem-reservation=3.00MB thread-reser | | in pipelines: 20(GETNEXT) | | | F07:PLAN FRAGMENT [HASH(i_item_id)] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.12MB mem-reservation=1.94MB thread-reservation=1 | 20:AGGREGATE [FINALIZE] | | group by: i_item_id | | mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 @@ -468,7 +468,7 @@ Per-Instance Resources: mem-estimate=11.51MB mem-reservation=3.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.50MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.12MB mem-reservation=2.50MB thread-reservation=1 | 06:AGGREGATE [STREAMING] | | group by: i_item_id | | mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0 @@ -510,7 +510,7 @@ Per-Instance Resources: mem-estimate=11.51MB mem-reservation=3.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -543,7 +543,7 @@ Per-Instance Resources: mem-estimate=11.51MB mem-reservation=3.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.36MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -561,7 +561,7 @@ Per-Instance Resources: mem-estimate=11.51MB mem-reservation=3.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=32.41MB mem-reservation=8.00MB thread-reservation=1 08:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=03 | hash predicates: ws_bill_customer_sk = c_customer_sk @@ -585,7 +585,7 @@ Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=1.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF003[min_max] -> c_current_addr_sk, RF002[bloom] -> c_current_addr_sk @@ -619,7 +619,7 @@ Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_qoy = CAST(2 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test index 4c08d3d29..f36da6a73 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test @@ -201,7 +201,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=57.14MB Threads=17 -Per-Host Resource Estimates: Memory=437MB +Per-Host Resource Estimates: Memory=438MB F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -238,7 +238,7 @@ Per-Host Resources: mem-estimate=22.22MB mem-reservation=11.50MB thread-reservat | | in pipelines: 11(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.11MB mem-reservation=512.00KB thread-reservation=2 | 11:SCAN HDFS [tpcds_parquet.customer_address current_addr, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -263,7 +263,7 @@ Per-Host Resources: mem-estimate=22.22MB mem-reservation=11.50MB thread-reservat | | in pipelines: 10(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=65.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=65.56MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 10:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> customer.c_current_addr_sk, RF000[bloom] -> customer.c_current_addr_sk @@ -317,7 +317,7 @@ Per-Host Resources: mem-estimate=138.85MB mem-reservation=28.56MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.11MB mem-reservation=512.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -342,7 +342,7 @@ Per-Host Resources: mem-estimate=138.85MB mem-reservation=28.56MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.10MB mem-reservation=16.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_city IN ('Midway', 'Concord', 'Spring Hill', 'Brownsville', 'Greenville') @@ -370,7 +370,7 @@ Per-Host Resources: mem-estimate=138.85MB mem-reservation=28.56MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: date_dim.d_dow IN (CAST(6 AS INT), CAST(0 AS INT)), date_dim.d_year IN (CAST(1999 AS INT), CAST(2000 AS INT), CAST(2001 AS INT)) @@ -398,7 +398,7 @@ Per-Host Resources: mem-estimate=138.85MB mem-reservation=28.56MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=64.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: (household_demographics.hd_dep_count = CAST(5 AS INT) OR household_demographics.hd_vehicle_count = CAST(3 AS INT)) @@ -423,7 +423,7 @@ Per-Host Resources: mem-estimate=138.85MB mem-reservation=28.56MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=84.45MB Threads=20 -Per-Host Resource Estimates: Memory=254MB +Per-Host Resource Estimates: Memory=259MB F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -468,7 +468,7 @@ Per-Instance Resources: mem-estimate=5.49MB mem-reservation=0B thread-reservatio | | in pipelines: 11(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.11MB mem-reservation=512.00KB thread-reservation=1 | 11:SCAN HDFS [tpcds_parquet.customer_address current_addr, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -502,7 +502,7 @@ Per-Instance Resources: mem-estimate=5.49MB mem-reservation=0B thread-reservatio | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.12MB mem-reservation=2.00MB thread-reservation=1 | 10:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> customer.c_current_addr_sk, RF000[bloom] -> customer.c_current_addr_sk @@ -535,7 +535,7 @@ Per-Instance Resources: mem-estimate=15.48MB mem-reservation=4.75MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=9.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=27.71MB mem-reservation=9.00MB thread-reservation=1 09:AGGREGATE [STREAMING] | output: sum(ss_coupon_amt), sum(ss_net_profit) | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city @@ -565,7 +565,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=9.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.11MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -598,7 +598,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=9.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.10MB mem-reservation=16.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_city IN ('Midway', 'Concord', 'Spring Hill', 'Brownsville', 'Greenville') @@ -634,7 +634,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=9.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: date_dim.d_dow IN (CAST(6 AS INT), CAST(0 AS INT)), date_dim.d_year IN (CAST(1999 AS INT), CAST(2000 AS INT), CAST(2001 AS INT)) @@ -670,7 +670,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=9.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=64.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: (household_demographics.hd_dep_count = CAST(5 AS INT) OR household_demographics.hd_vehicle_count = CAST(3 AS INT)) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test index a3dc044f4..e155a2dce 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test @@ -364,7 +364,7 @@ PLAN-ROOT SINK in pipelines: 14(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=330.51MB Threads=29 -Per-Host Resource Estimates: Memory=959MB +Per-Host Resource Estimates: Memory=961MB F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -428,7 +428,7 @@ Per-Host Resources: mem-estimate=55.11MB mem-reservation=34.00MB thread-reservat | | in pipelines: 24(GETNEXT) | | | F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=92.77MB mem-reservation=44.75MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=93.03MB mem-reservation=44.75MB thread-reservation=2 runtime-filters-memory=3.00MB | 30:AGGREGATE [STREAMING] | | output: sum(ss_sales_price) | | group by: i_category, i_brand, s_store_name, s_company_name, d_year, d_moy @@ -450,7 +450,7 @@ Per-Host Resources: mem-estimate=55.11MB mem-reservation=34.00MB thread-reservat | | | in pipelines: 26(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.17MB mem-reservation=24.00KB thread-reservation=2 | | 26:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -475,7 +475,7 @@ Per-Host Resources: mem-estimate=55.11MB mem-reservation=34.00MB thread-reservat | | | in pipelines: 23(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.23MB mem-reservation=256.00KB thread-reservation=2 | | 23:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -500,7 +500,7 @@ Per-Host Resources: mem-estimate=55.11MB mem-reservation=34.00MB thread-reservat | | | in pipelines: 25(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: (d_year = CAST(2000 AS INT) OR (d_year = CAST(1999 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2001 AS INT) AND d_moy = CAST(1 AS INT))) @@ -592,7 +592,7 @@ Per-Host Resources: mem-estimate=63.43MB mem-reservation=51.94MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=92.77MB mem-reservation=44.75MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=93.03MB mem-reservation=44.75MB thread-reservation=2 runtime-filters-memory=3.00MB | 07:AGGREGATE [STREAMING] | | output: sum(ss_sales_price) | | group by: i_category, i_brand, s_store_name, s_company_name, d_year, d_moy @@ -614,7 +614,7 @@ Per-Host Resources: mem-estimate=63.43MB mem-reservation=51.94MB thread-reservat | | | in pipelines: 03(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.17MB mem-reservation=24.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -639,7 +639,7 @@ Per-Host Resources: mem-estimate=63.43MB mem-reservation=51.94MB thread-reservat | | | in pipelines: 00(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.23MB mem-reservation=256.00KB thread-reservation=2 | | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -664,7 +664,7 @@ Per-Host Resources: mem-estimate=63.43MB mem-reservation=51.94MB thread-reservat | | | in pipelines: 02(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: (d_year = CAST(2000 AS INT) OR (d_year = CAST(1999 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2001 AS INT) AND d_moy = CAST(1 AS INT))) @@ -716,7 +716,7 @@ Per-Host Resources: mem-estimate=63.43MB mem-reservation=51.94MB thread-reservat | in pipelines: 14(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=92.77MB mem-reservation=44.75MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=93.03MB mem-reservation=44.75MB thread-reservation=2 runtime-filters-memory=3.00MB 20:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: i_category, i_brand, s_store_name, s_company_name, d_year, d_moy @@ -738,7 +738,7 @@ Per-Host Resources: mem-estimate=92.77MB mem-reservation=44.75MB thread-reservat | | in pipelines: 16(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.17MB mem-reservation=24.00KB thread-reservation=2 | 16:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -763,7 +763,7 @@ Per-Host Resources: mem-estimate=92.77MB mem-reservation=44.75MB thread-reservat | | in pipelines: 13(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.23MB mem-reservation=256.00KB thread-reservation=2 | 13:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -788,7 +788,7 @@ Per-Host Resources: mem-estimate=92.77MB mem-reservation=44.75MB thread-reservat | | in pipelines: 15(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: (d_year = CAST(2000 AS INT) OR (d_year = CAST(1999 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2001 AS INT) AND d_moy = CAST(1 AS INT))) @@ -813,7 +813,7 @@ Per-Host Resources: mem-estimate=92.77MB mem-reservation=44.75MB thread-reservat in pipelines: 14(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=416.70MB Threads=36 -Per-Host Resource Estimates: Memory=697MB +Per-Host Resource Estimates: Memory=714MB F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -886,7 +886,7 @@ Per-Instance Resources: mem-estimate=11.53MB mem-reservation=0B thread-reservati | | | F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=33.00MB mem-reservation=18.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=35.56MB mem-reservation=18.00MB thread-reservation=1 | 30:AGGREGATE [STREAMING] | | output: sum(ss_sales_price) | | group by: i_category, i_brand, s_store_name, s_company_name, d_year, d_moy @@ -916,7 +916,7 @@ Per-Instance Resources: mem-estimate=11.53MB mem-reservation=0B thread-reservati | | | in pipelines: 26(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.17MB mem-reservation=24.00KB thread-reservation=1 | | 26:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -949,7 +949,7 @@ Per-Instance Resources: mem-estimate=11.53MB mem-reservation=0B thread-reservati | | | in pipelines: 23(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.23MB mem-reservation=256.00KB thread-reservation=1 | | 23:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -982,7 +982,7 @@ Per-Instance Resources: mem-estimate=11.53MB mem-reservation=0B thread-reservati | | | in pipelines: 25(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: (d_year = CAST(2000 AS INT) OR (d_year = CAST(1999 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2001 AS INT) AND d_moy = CAST(1 AS INT))) @@ -1083,7 +1083,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese | | | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=33.00MB mem-reservation=18.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=35.56MB mem-reservation=18.00MB thread-reservation=1 | 07:AGGREGATE [STREAMING] | | output: sum(ss_sales_price) | | group by: i_category, i_brand, s_store_name, s_company_name, d_year, d_moy @@ -1113,7 +1113,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese | | | in pipelines: 03(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.17MB mem-reservation=24.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -1146,7 +1146,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese | | | in pipelines: 00(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.23MB mem-reservation=256.00KB thread-reservation=1 | | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -1179,7 +1179,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese | | | in pipelines: 02(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: (d_year = CAST(2000 AS INT) OR (d_year = CAST(1999 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2001 AS INT) AND d_moy = CAST(1 AS INT))) @@ -1232,7 +1232,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=33.00MB mem-reservation=18.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=35.56MB mem-reservation=18.00MB thread-reservation=1 20:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: i_category, i_brand, s_store_name, s_company_name, d_year, d_moy @@ -1262,7 +1262,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=18.00MB thread-rese | | in pipelines: 16(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.17MB mem-reservation=24.00KB thread-reservation=1 | 16:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -1295,7 +1295,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=18.00MB thread-rese | | in pipelines: 13(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.23MB mem-reservation=256.00KB thread-reservation=1 | 13:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -1328,7 +1328,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=18.00MB thread-rese | | in pipelines: 15(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: (d_year = CAST(2000 AS INT) OR (d_year = CAST(1999 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2001 AS INT) AND d_moy = CAST(1 AS INT))) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test index 37542b51d..8cd18ca58 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test @@ -151,7 +151,7 @@ PLAN-ROOT SINK in pipelines: 02(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=31.08MB Threads=12 -Per-Host Resource Estimates: Memory=268MB +Per-Host Resource Estimates: Memory=269MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -191,7 +191,7 @@ Per-Host Resources: mem-estimate=19.70MB mem-reservation=12.44MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -217,7 +217,7 @@ Per-Host Resources: mem-estimate=19.70MB mem-reservation=12.44MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.50MB mem-reservation=8.00MB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'M' OR cd_marital_status = 'D' AND cd_education_status = '2 yr Degree' OR cd_marital_status = 'S' AND cd_education_status = 'College', cd_education_status = '4 yr Degree' OR cd_marital_status = 'D' AND cd_education_status = '2 yr Degree' OR cd_marital_status = 'S' AND cd_education_status = 'College' @@ -235,7 +235,7 @@ Per-Host Resources: mem-estimate=19.70MB mem-reservation=12.44MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=104.29MB mem-reservation=9.88MB thread-reservation=2 runtime-filters-memory=4.00MB +Per-Host Resources: mem-estimate=104.94MB mem-reservation=9.88MB thread-reservation=2 runtime-filters-memory=4.00MB 06:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_sold_date_sk = d_date_sk | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -250,7 +250,7 @@ Per-Host Resources: mem-estimate=104.29MB mem-reservation=9.88MB thread-reservat | | in pipelines: 04(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2000 AS INT) @@ -279,7 +279,7 @@ Per-Host Resources: mem-estimate=104.29MB mem-reservation=9.88MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.18MB mem-reservation=256.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_state IN ('CO', 'OH', 'TX', 'OR', 'MN', 'KY', 'VA', 'CA', 'MS'), ca_country = 'United States' @@ -308,7 +308,7 @@ Per-Host Resources: mem-estimate=104.29MB mem-reservation=9.88MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=42.89MB Threads=14 -Per-Host Resource Estimates: Memory=142MB +Per-Host Resource Estimates: Memory=147MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -356,7 +356,7 @@ Per-Instance Resources: mem-estimate=825.95KB mem-reservation=0B thread-reservat | | in pipelines: 01(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -390,7 +390,7 @@ Per-Instance Resources: mem-estimate=825.95KB mem-reservation=0B thread-reservat | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.00MB mem-reservation=8.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'M' OR cd_marital_status = 'D' AND cd_education_status = '2 yr Degree' OR cd_marital_status = 'S' AND cd_education_status = 'College', cd_education_status = '4 yr Degree' OR cd_marital_status = 'D' AND cd_education_status = '2 yr Degree' OR cd_marital_status = 'S' AND cd_education_status = 'College' @@ -409,7 +409,7 @@ Per-Instance Resources: mem-estimate=825.95KB mem-reservation=0B thread-reservat | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=18.13MB mem-reservation=2.00MB thread-reservation=1 06:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=02 | hash predicates: ss_sold_date_sk = d_date_sk @@ -432,7 +432,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2000 AS INT) @@ -469,7 +469,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.18MB mem-reservation=256.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_state IN ('CO', 'OH', 'TX', 'OR', 'MN', 'KY', 'VA', 'CA', 'MS'), ca_country = 'United States' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test index 99738c37c..71a75c8a6 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test @@ -427,7 +427,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=108.81MB Threads=22 -Per-Host Resource Estimates: Memory=1.05GB +Per-Host Resource Estimates: Memory=1.06GB F15:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=24.61MB mem-reservation=15.94MB thread-reservation=1 PLAN-ROOT SINK @@ -508,7 +508,7 @@ PLAN-ROOT SINK | | in pipelines: 23(GETNEXT) | | | F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=96.51MB mem-reservation=11.81MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=96.52MB mem-reservation=11.81MB thread-reservation=2 runtime-filters-memory=1.00MB | 28:AGGREGATE [STREAMING] | | output: sum(CAST(coalesce(sr.sr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(sts.ss_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(sr.sr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(sts.ss_net_paid, CAST(0 AS DECIMAL(7,2)))) | | group by: sts.ss_item_sk @@ -530,7 +530,7 @@ PLAN-ROOT SINK | | | in pipelines: 25(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT) @@ -558,7 +558,7 @@ PLAN-ROOT SINK | | | in pipelines: 24(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=96.11MB mem-reservation=4.00MB thread-reservation=2 | | 24:SCAN HDFS [tpcds_parquet.store_returns sr, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0)) @@ -644,7 +644,7 @@ PLAN-ROOT SINK | | in pipelines: 12(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=303.25MB mem-reservation=22.88MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=303.58MB mem-reservation=22.88MB thread-reservation=2 runtime-filters-memory=1.00MB | 17:AGGREGATE [STREAMING] | | output: sum(CAST(coalesce(cr.cr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(cs.cs_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(cr.cr_return_amount, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(cs.cs_net_paid, CAST(0 AS DECIMAL(7,2)))) | | group by: cs.cs_item_sk @@ -666,7 +666,7 @@ PLAN-ROOT SINK | | | in pipelines: 14(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT) @@ -694,7 +694,7 @@ PLAN-ROOT SINK | | | in pipelines: 13(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=96.11MB mem-reservation=4.00MB thread-reservation=2 | | 13:SCAN HDFS [tpcds_parquet.catalog_returns cr, RANDOM] | | HDFS partitions=1/1 files=1 size=10.62MB | | predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0)) @@ -779,7 +779,7 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reservati | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -Per-Host Resources: mem-estimate=207.08MB mem-reservation=22.88MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=207.34MB mem-reservation=22.88MB thread-reservation=2 runtime-filters-memory=1.00MB 06:AGGREGATE [STREAMING] | output: sum(CAST(coalesce(wr.wr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(ws.ws_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(wr.wr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(ws.ws_net_paid, CAST(0 AS DECIMAL(7,2)))) | group by: ws.ws_item_sk @@ -801,7 +801,7 @@ Per-Host Resources: mem-estimate=207.08MB mem-reservation=22.88MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT) @@ -829,7 +829,7 @@ Per-Host Resources: mem-estimate=207.08MB mem-reservation=22.88MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.11MB mem-reservation=2.00MB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.web_returns wr, RANDOM] | HDFS partitions=1/1 files=1 size=5.66MB | predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0)) @@ -858,7 +858,7 @@ Per-Host Resources: mem-estimate=207.08MB mem-reservation=22.88MB thread-reserva in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=138.31MB Threads=21 -Per-Host Resource Estimates: Memory=385MB +Per-Host Resource Estimates: Memory=390MB F15:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=24.84MB mem-reservation=15.94MB thread-reservation=1 PLAN-ROOT SINK @@ -940,7 +940,7 @@ PLAN-ROOT SINK | | | F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=27.41MB mem-reservation=6.00MB thread-reservation=1 | 28:AGGREGATE [STREAMING] | | output: sum(CAST(coalesce(sr.sr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(sts.ss_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(sr.sr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(sts.ss_net_paid, CAST(0 AS DECIMAL(7,2)))) | | group by: sts.ss_item_sk @@ -970,7 +970,7 @@ PLAN-ROOT SINK | | | in pipelines: 25(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT) @@ -1006,7 +1006,7 @@ PLAN-ROOT SINK | | | in pipelines: 24(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.11MB mem-reservation=4.00MB thread-reservation=1 | | 24:SCAN HDFS [tpcds_parquet.store_returns sr, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0)) @@ -1093,7 +1093,7 @@ PLAN-ROOT SINK | | | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=58.00MB mem-reservation=18.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=58.70MB mem-reservation=18.00MB thread-reservation=1 | 17:AGGREGATE [STREAMING] | | output: sum(CAST(coalesce(cr.cr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(cs.cs_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(cr.cr_return_amount, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(cs.cs_net_paid, CAST(0 AS DECIMAL(7,2)))) | | group by: cs.cs_item_sk @@ -1123,7 +1123,7 @@ PLAN-ROOT SINK | | | in pipelines: 14(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT) @@ -1159,7 +1159,7 @@ PLAN-ROOT SINK | | | in pipelines: 13(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.11MB mem-reservation=4.00MB thread-reservation=1 | | 13:SCAN HDFS [tpcds_parquet.catalog_returns cr, RANDOM] | | HDFS partitions=1/1 files=1 size=10.62MB | | predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0)) @@ -1245,7 +1245,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=42.00MB mem-reservation=18.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=42.47MB mem-reservation=18.00MB thread-reservation=1 06:AGGREGATE [STREAMING] | output: sum(CAST(coalesce(wr.wr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(ws.ws_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(wr.wr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(ws.ws_net_paid, CAST(0 AS DECIMAL(7,2)))) | group by: ws.ws_item_sk @@ -1275,7 +1275,7 @@ Per-Instance Resources: mem-estimate=42.00MB mem-reservation=18.00MB thread-rese | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT) @@ -1311,7 +1311,7 @@ Per-Instance Resources: mem-estimate=42.00MB mem-reservation=18.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.11MB mem-reservation=2.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.web_returns wr, RANDOM] | HDFS partitions=1/1 files=1 size=5.66MB | predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0)) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q50.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q50.test index c99e38655..1d578d81a 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q50.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q50.test @@ -178,7 +178,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=67.84MB Threads=12 -Per-Host Resource Estimates: Memory=429MB +Per-Host Resource Estimates: Memory=430MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -235,7 +235,7 @@ Per-Host Resources: mem-estimate=110.68MB mem-reservation=40.75MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -260,7 +260,7 @@ Per-Host Resources: mem-estimate=110.68MB mem-reservation=40.75MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=128.00MB mem-reservation=88.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=128.69MB mem-reservation=88.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -285,7 +285,7 @@ Per-Host Resources: mem-estimate=110.68MB mem-reservation=40.75MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d2.d_year = CAST(2001 AS INT), d2.d_moy = CAST(8 AS INT) @@ -313,7 +313,7 @@ Per-Host Resources: mem-estimate=110.68MB mem-reservation=40.75MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=97.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=97.11MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | runtime filters: RF005[min_max] -> sr_returned_date_sk, RF004[bloom] -> sr_returned_date_sk @@ -338,7 +338,7 @@ Per-Host Resources: mem-estimate=110.68MB mem-reservation=40.75MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=99.59MB Threads=13 -Per-Host Resource Estimates: Memory=232MB +Per-Host Resource Estimates: Memory=243MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.13MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -374,7 +374,7 @@ Per-Instance Resources: mem-estimate=18.89MB mem-reservation=8.50MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=7.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=31.00MB mem-reservation=7.00MB thread-reservation=1 09:AGGREGATE [STREAMING] | output: sum(CAST(CASE WHEN (CAST(sr_returned_date_sk AS BIGINT) - CAST(ss_sold_date_sk AS BIGINT) <= CAST(30 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(sr_returned_date_sk AS BIGINT) - CAST(ss_sold_date_sk AS BIGINT) > CAST(30 AS BIGINT)) AND (CAST(sr_returned_date_sk AS BIGINT) - CAST(ss_sold_date_sk AS BIGINT) <= CAST(60 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(sr_returned_date_sk AS BIGINT) - CAST(ss_sold_date_sk AS BIGINT) > CAST(60 AS BIGINT)) AND (CAST(sr_returned_date_sk AS BIGINT) - CAST(ss_sold_date_sk AS BIGINT) <= CAST(90 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(sr_returned_date_sk AS BIGINT) - CAST(ss_sold_date_sk AS BIGINT) > CAST(90 AS BIGINT)) AND (CAST(sr_returned_date_sk AS BIGINT) - CAST(ss_sold_date_sk AS BIGINT) <= CAST(120 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(sr_returned_date_sk AS BIGINT) - CAST(ss_sold_date_sk AS BIGINT) > CAST(120 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)) | group by: s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, s_suite_number, s_city, s_county, s_state, s_zip @@ -404,7 +404,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=7.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -437,7 +437,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=7.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.69MB mem-reservation=88.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -470,7 +470,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=7.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d2.d_year = CAST(2001 AS INT), d2.d_moy = CAST(8 AS INT) @@ -507,7 +507,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=7.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.11MB mem-reservation=4.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | runtime filters: RF005[min_max] -> sr_returned_date_sk, RF004[bloom] -> sr_returned_date_sk diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q51.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q51.test index d0aab8d1a..0ff678efd 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q51.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q51.test @@ -209,7 +209,7 @@ PLAN-ROOT SINK in pipelines: 06(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=219.88MB Threads=13 -Per-Host Resource Estimates: Memory=726MB +Per-Host Resource Estimates: Memory=727MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -301,7 +301,7 @@ Per-Host Resources: mem-estimate=54.34MB mem-reservation=34.00MB thread-reservat | | in pipelines: 00(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=116.18MB mem-reservation=23.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=116.33MB mem-reservation=23.94MB thread-reservation=2 runtime-filters-memory=1.00MB | 03:AGGREGATE [STREAMING] | | output: sum(ws_sales_price) | | group by: ws_item_sk, d_date @@ -323,7 +323,7 @@ Per-Host Resources: mem-estimate=54.34MB mem-reservation=34.00MB thread-reservat | | | in pipelines: 01(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.13MB mem-reservation=2.00MB thread-reservation=2 | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -385,7 +385,7 @@ Per-Host Resources: mem-estimate=99.23MB mem-reservation=50.00MB thread-reservat | in pipelines: 06(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=75.01MB mem-reservation=37.94MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=75.35MB mem-reservation=37.94MB thread-reservation=2 runtime-filters-memory=1.00MB 09:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: ss_item_sk, d_date @@ -407,7 +407,7 @@ Per-Host Resources: mem-estimate=75.01MB mem-reservation=37.94MB thread-reservat | | in pipelines: 07(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.13MB mem-reservation=2.00MB thread-reservation=2 | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -436,7 +436,7 @@ Per-Host Resources: mem-estimate=75.01MB mem-reservation=37.94MB thread-reservat in pipelines: 06(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=310.75MB Threads=16 -Per-Host Resource Estimates: Memory=672MB +Per-Host Resource Estimates: Memory=677MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -485,7 +485,7 @@ Per-Instance Resources: mem-estimate=227.67MB mem-reservation=16.00MB thread-res | in pipelines: 10(GETNEXT) | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=10.41MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=11.50MB mem-reservation=0B thread-reservation=1 12:HASH JOIN [FULL OUTER JOIN, PARTITIONED] | hash-table-id=00 | hash predicates: ss_item_sk = ws_item_sk, d_date = d_date @@ -537,7 +537,7 @@ Per-Instance Resources: mem-estimate=10.41MB mem-reservation=0B thread-reservati | | | F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=49.00MB mem-reservation=21.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=49.39MB mem-reservation=21.00MB thread-reservation=1 | 03:AGGREGATE [STREAMING] | | output: sum(ws_sales_price) | | group by: ws_item_sk, d_date @@ -567,7 +567,7 @@ Per-Instance Resources: mem-estimate=10.41MB mem-reservation=0B thread-reservati | | | in pipelines: 01(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -630,7 +630,7 @@ Per-Instance Resources: mem-estimate=59.06MB mem-reservation=50.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=50.00MB mem-reservation=35.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=51.17MB mem-reservation=35.00MB thread-reservation=1 09:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: ss_item_sk, d_date @@ -660,7 +660,7 @@ Per-Instance Resources: mem-estimate=50.00MB mem-reservation=35.00MB thread-rese | | in pipelines: 07(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test index 45ba9ba48..a50d4cd14 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test @@ -100,7 +100,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=15.56MB Threads=8 -Per-Host Resource Estimates: Memory=174MB +Per-Host Resource Estimates: Memory=175MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -135,7 +135,7 @@ Per-Host Resources: mem-estimate=10.19MB mem-reservation=1.94MB thread-reservati | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=48.53MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=2.00MB 05:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: dt.d_year, item.i_brand, item.i_brand_id @@ -157,7 +157,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.date_dim dt, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: dt.d_year = CAST(1998 AS INT), dt.d_moy = CAST(12 AS INT) @@ -185,7 +185,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.19MB mem-reservation=256.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: item.i_manager_id = CAST(1 AS INT) @@ -212,7 +212,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=26.38MB Threads=9 -Per-Host Resource Estimates: Memory=121MB +Per-Host Resource Estimates: Memory=123MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -248,7 +248,7 @@ Per-Instance Resources: mem-estimate=10.36MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=27.32MB mem-reservation=3.00MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: dt.d_year, item.i_brand, item.i_brand_id @@ -278,7 +278,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.date_dim dt, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: dt.d_year = CAST(1998 AS INT), dt.d_moy = CAST(12 AS INT) @@ -314,7 +314,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: item.i_manager_id = CAST(1 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test index 7fd67f387..9785a1ef6 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test @@ -122,7 +122,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=18.52MB Threads=10 -Per-Host Resource Estimates: Memory=225MB +Per-Host Resource Estimates: Memory=226MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -157,7 +157,7 @@ Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservati | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=67.14MB mem-reservation=11.81MB thread-reservation=2 runtime-filters-memory=3.00MB 07:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: i_manufact_id, d_qoy @@ -179,7 +179,7 @@ Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -204,7 +204,7 @@ Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq IN (CAST(1212 AS INT), CAST(1213 AS INT), CAST(1214 AS INT), CAST(1215 AS INT), CAST(1216 AS INT), CAST(1217 AS INT), CAST(1218 AS INT), CAST(1219 AS INT), CAST(1220 AS INT), CAST(1221 AS INT), CAST(1222 AS INT), CAST(1223 AS INT)) @@ -232,7 +232,7 @@ Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservat | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=80.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=80.32MB mem-reservation=256.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: ((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help') AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR (i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants') AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) @@ -257,7 +257,7 @@ Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservat in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=32.27MB Threads=11 -Per-Host Resource Estimates: Memory=142MB +Per-Host Resource Estimates: Memory=143MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -293,7 +293,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.66MB mem-reservation=3.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: i_manufact_id, d_qoy @@ -323,7 +323,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -356,7 +356,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq IN (CAST(1212 AS INT), CAST(1213 AS INT), CAST(1214 AS INT), CAST(1215 AS INT), CAST(1216 AS INT), CAST(1217 AS INT), CAST(1218 AS INT), CAST(1219 AS INT), CAST(1220 AS INT), CAST(1221 AS INT), CAST(1222 AS INT), CAST(1223 AS INT)) @@ -392,7 +392,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.32MB mem-reservation=256.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: ((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help') AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR (i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants') AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test index ff936a0e6..3ae861e98 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test @@ -321,7 +321,7 @@ PLAN-ROOT SINK in pipelines: 13(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=60.39MB Threads=31 -Per-Host Resource Estimates: Memory=648MB +Per-Host Resource Estimates: Memory=651MB F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -356,7 +356,7 @@ Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservati | in pipelines: 50(GETNEXT) | F20:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=20.02MB mem-reservation=3.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=20.19MB mem-reservation=3.94MB thread-reservation=1 27:AGGREGATE [STREAMING] | output: count(*) | group by: CAST((sum(ss_ext_sales_price) / CAST(50 AS DECIMAL(3,0))) AS INT) @@ -397,7 +397,7 @@ Per-Host Resources: mem-estimate=13.66MB mem-reservation=4.94MB thread-reservati | | in pipelines: 19(GETNEXT) | | | F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=48.00KB mem-reservation=0B thread-reservation=1 | 19:CARDINALITY CHECK | | limit: 1 | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -411,7 +411,7 @@ Per-Host Resources: mem-estimate=13.66MB mem-reservation=4.94MB thread-reservati | | in pipelines: 46(GETNEXT) | | | F18:PLAN FRAGMENT [HASH(d_month_seq + 3)] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.05MB mem-reservation=1.94MB thread-reservation=1 | 46:AGGREGATE [FINALIZE] | | group by: d_month_seq + 3 | | limit: 2 @@ -425,7 +425,7 @@ Per-Host Resources: mem-estimate=13.66MB mem-reservation=4.94MB thread-reservati | | in pipelines: 17(GETNEXT) | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=58.00MB mem-reservation=2.25MB thread-reservation=2 +| Per-Host Resources: mem-estimate=58.05MB mem-reservation=2.25MB thread-reservation=2 | 18:AGGREGATE [STREAMING] | | group by: CAST(d_month_seq AS BIGINT) + CAST(3 AS BIGINT) | | limit: 2 @@ -458,7 +458,7 @@ Per-Host Resources: mem-estimate=13.66MB mem-reservation=4.94MB thread-reservati | | in pipelines: 16(GETNEXT) | | | F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=48.00KB mem-reservation=0B thread-reservation=1 | 16:CARDINALITY CHECK | | limit: 1 | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -472,7 +472,7 @@ Per-Host Resources: mem-estimate=13.66MB mem-reservation=4.94MB thread-reservati | | in pipelines: 42(GETNEXT) | | | F15:PLAN FRAGMENT [HASH(d_month_seq + 1)] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.05MB mem-reservation=1.94MB thread-reservation=1 | 42:AGGREGATE [FINALIZE] | | group by: d_month_seq + 1 | | limit: 2 @@ -486,7 +486,7 @@ Per-Host Resources: mem-estimate=13.66MB mem-reservation=4.94MB thread-reservati | | in pipelines: 14(GETNEXT) | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=58.00MB mem-reservation=2.25MB thread-reservation=2 +| Per-Host Resources: mem-estimate=58.05MB mem-reservation=2.25MB thread-reservation=2 | 15:AGGREGATE [STREAMING] | | group by: CAST(d_month_seq AS BIGINT) + CAST(1 AS BIGINT) | | limit: 2 @@ -521,7 +521,7 @@ Per-Host Resources: mem-estimate=13.66MB mem-reservation=4.94MB thread-reservati | | in pipelines: 13(GETNEXT) | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.14MB mem-reservation=512.00KB thread-reservation=2 | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -553,7 +553,7 @@ Per-Host Resources: mem-estimate=10.08MB mem-reservation=7.81MB thread-reservati | | in pipelines: 12(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.18MB mem-reservation=16.00KB thread-reservation=2 | 12:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -578,7 +578,7 @@ Per-Host Resources: mem-estimate=10.08MB mem-reservation=7.81MB thread-reservati | | in pipelines: 11(GETNEXT) | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=50.00MB mem-reservation=2.50MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=50.56MB mem-reservation=2.50MB thread-reservation=2 runtime-filters-memory=2.00MB | 11:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | runtime filters: RF004[min_max] -> ca_state, RF005[min_max] -> ca_county, RF002[bloom] -> ca_state, RF003[bloom] -> ca_county @@ -596,7 +596,7 @@ Per-Host Resources: mem-estimate=10.08MB mem-reservation=7.81MB thread-reservati | in pipelines: 10(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=36.27MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=2.00MB 20:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_customer_sk = c_customer_sk | fk/pk conjuncts: ss_customer_sk = c_customer_sk @@ -611,7 +611,7 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati | | in pipelines: 34(GETNEXT) | | | F08:PLAN FRAGMENT [HASH(c_customer_sk,c_current_addr_sk)] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.05MB mem-reservation=1.94MB thread-reservation=1 | 34:AGGREGATE [FINALIZE] | | group by: c_customer_sk, c_current_addr_sk | | mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 @@ -645,7 +645,7 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati | | | in pipelines: 05(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.14MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 05:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF007[min_max] -> tpcds_parquet.customer.c_current_addr_sk, RF006[bloom] -> tpcds_parquet.customer.c_current_addr_sk @@ -663,7 +663,7 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati | | in pipelines: 01(GETNEXT), 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=150.91MB mem-reservation=10.88MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=151.88MB mem-reservation=10.88MB thread-reservation=2 runtime-filters-memory=3.00MB | 07:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: sold_date_sk = d_date_sk | | fk/pk conjuncts: assumed fk/pk @@ -678,7 +678,7 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati | | | in pipelines: 04(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(12 AS INT) @@ -706,7 +706,7 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati | | | in pipelines: 03(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.19MB mem-reservation=256.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Women', i_class = 'maternity' @@ -761,7 +761,7 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati in pipelines: 10(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=84.95MB Threads=37 -Per-Host Resource Estimates: Memory=404MB +Per-Host Resource Estimates: Memory=416MB F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -796,7 +796,7 @@ Per-Instance Resources: mem-estimate=10.03MB mem-reservation=1.94MB thread-reser | in pipelines: 50(GETNEXT) | F20:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=20.05MB mem-reservation=3.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=20.38MB mem-reservation=3.94MB thread-reservation=1 27:AGGREGATE [STREAMING] | output: count(*) | group by: CAST((sum(ss_ext_sales_price) / CAST(50 AS DECIMAL(3,0))) AS INT) @@ -817,7 +817,7 @@ Per-Instance Resources: mem-estimate=20.05MB mem-reservation=3.94MB thread-reser | in pipelines: 10(GETNEXT) | F13:PLAN FRAGMENT [HASH(ss_sold_date_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=10.24MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=10.81MB mem-reservation=2.00MB thread-reservation=1 26:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: c_customer_sk @@ -844,7 +844,7 @@ Per-Instance Resources: mem-estimate=10.24MB mem-reservation=2.00MB thread-reser | | in pipelines: 19(GETNEXT) | | | F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=48.00KB mem-reservation=0B thread-reservation=1 | 19:CARDINALITY CHECK | | limit: 1 | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -858,7 +858,7 @@ Per-Instance Resources: mem-estimate=10.24MB mem-reservation=2.00MB thread-reser | | in pipelines: 46(GETNEXT) | | | F18:PLAN FRAGMENT [HASH(d_month_seq + 3)] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.05MB mem-reservation=1.94MB thread-reservation=1 | 46:AGGREGATE [FINALIZE] | | group by: d_month_seq + 3 | | limit: 2 @@ -872,7 +872,7 @@ Per-Instance Resources: mem-estimate=10.24MB mem-reservation=2.00MB thread-reser | | in pipelines: 17(GETNEXT) | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.25MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.05MB mem-reservation=2.25MB thread-reservation=1 | 18:AGGREGATE [STREAMING] | | group by: CAST(d_month_seq AS BIGINT) + CAST(3 AS BIGINT) | | limit: 2 @@ -912,7 +912,7 @@ Per-Instance Resources: mem-estimate=10.24MB mem-reservation=2.00MB thread-reser | | in pipelines: 16(GETNEXT) | | | F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=48.00KB mem-reservation=0B thread-reservation=1 | 16:CARDINALITY CHECK | | limit: 1 | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -926,7 +926,7 @@ Per-Instance Resources: mem-estimate=10.24MB mem-reservation=2.00MB thread-reser | | in pipelines: 42(GETNEXT) | | | F15:PLAN FRAGMENT [HASH(d_month_seq + 1)] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.05MB mem-reservation=1.94MB thread-reservation=1 | 42:AGGREGATE [FINALIZE] | | group by: d_month_seq + 1 | | limit: 2 @@ -940,7 +940,7 @@ Per-Instance Resources: mem-estimate=10.24MB mem-reservation=2.00MB thread-reser | | in pipelines: 14(GETNEXT) | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.25MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.05MB mem-reservation=2.25MB thread-reservation=1 | 15:AGGREGATE [STREAMING] | | group by: CAST(d_month_seq AS BIGINT) + CAST(1 AS BIGINT) | | limit: 2 @@ -983,7 +983,7 @@ Per-Instance Resources: mem-estimate=10.24MB mem-reservation=2.00MB thread-reser | | in pipelines: 13(GETNEXT) | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.28MB mem-reservation=512.00KB thread-reservation=1 | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -1000,7 +1000,7 @@ Per-Instance Resources: mem-estimate=10.24MB mem-reservation=2.00MB thread-reser | in pipelines: 10(GETNEXT) | F10:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=190.35KB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=3.07MB mem-reservation=0B thread-reservation=1 22:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=03 | hash predicates: ca_state = s_state, ca_county = s_county @@ -1023,7 +1023,7 @@ Per-Instance Resources: mem-estimate=190.35KB mem-reservation=0B thread-reservat | | in pipelines: 12(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.18MB mem-reservation=16.00KB thread-reservation=1 | 12:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -1057,7 +1057,7 @@ Per-Instance Resources: mem-estimate=190.35KB mem-reservation=0B thread-reservat | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.12MB mem-reservation=512.00KB thread-reservation=1 | 11:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | runtime filters: RF004[min_max] -> ca_state, RF005[min_max] -> ca_county, RF002[bloom] -> ca_state, RF003[bloom] -> ca_county @@ -1076,7 +1076,7 @@ Per-Instance Resources: mem-estimate=190.35KB mem-reservation=0B thread-reservat | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=16.66MB mem-reservation=1.00MB thread-reservation=1 20:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=05 | hash predicates: ss_customer_sk = c_customer_sk @@ -1099,7 +1099,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reser | | in pipelines: 34(GETNEXT) | | | F08:PLAN FRAGMENT [HASH(c_customer_sk,c_current_addr_sk)] hosts=3 instances=3 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.05MB mem-reservation=1.94MB thread-reservation=1 | 34:AGGREGATE [FINALIZE] | | group by: c_customer_sk, c_current_addr_sk | | mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 @@ -1112,7 +1112,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reser | | in pipelines: 01(GETNEXT), 02(GETNEXT) | | | F07:PLAN FRAGMENT [HASH(customer_sk)] hosts=3 instances=3 -| Per-Instance Resources: mem-estimate=10.03MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.17MB mem-reservation=2.00MB thread-reservation=1 | 09:AGGREGATE [STREAMING] | | group by: c_customer_sk, c_current_addr_sk | | mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0 @@ -1142,7 +1142,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reser | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.14MB mem-reservation=1.00MB thread-reservation=1 | | 05:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF007[min_max] -> tpcds_parquet.customer.c_current_addr_sk, RF006[bloom] -> tpcds_parquet.customer.c_current_addr_sk @@ -1161,7 +1161,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=49.00MB mem-reservation=4.00MB thread-reservation=1 | 07:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=07 | | hash predicates: sold_date_sk = d_date_sk @@ -1184,7 +1184,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reser | | | in pipelines: 04(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(12 AS INT) @@ -1220,7 +1220,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reser | | | in pipelines: 03(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Women', i_class = 'maternity' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test index 1155263c3..bba1f32dd 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test @@ -97,7 +97,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=15.56MB Threads=8 -Per-Host Resource Estimates: Memory=174MB +Per-Host Resource Estimates: Memory=175MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -132,7 +132,7 @@ Per-Host Resources: mem-estimate=10.18MB mem-reservation=1.94MB thread-reservati | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=48.49MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=2.00MB 05:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: i_brand, i_brand_id @@ -154,7 +154,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT) @@ -182,7 +182,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.19MB mem-reservation=256.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_manager_id = CAST(36 AS INT) @@ -209,7 +209,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=26.38MB Threads=9 -Per-Host Resource Estimates: Memory=120MB +Per-Host Resource Estimates: Memory=123MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -245,7 +245,7 @@ Per-Instance Resources: mem-estimate=10.33MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=27.22MB mem-reservation=3.00MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: i_brand, i_brand_id @@ -275,7 +275,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT) @@ -311,7 +311,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_manager_id = CAST(36 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test old mode 100755 new mode 100644 index 0a43b7748..fc560e67c --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test @@ -399,7 +399,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=75.62MB Threads=36 -Per-Host Resource Estimates: Memory=905MB +Per-Host Resource Estimates: Memory=909MB F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -434,7 +434,7 @@ Per-Host Resources: mem-estimate=10.19MB mem-reservation=1.94MB thread-reservati | in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT) | F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=20.73MB mem-reservation=3.94MB thread-reservation=1 31:AGGREGATE [STREAMING] | output: sum(total_sales) | group by: i_item_id @@ -481,7 +481,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 25(GETNEXT) | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.19MB mem-reservation=256.00KB thread-reservation=2 | | 25:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_color IN ('slate', 'blanched', 'burnished') @@ -509,7 +509,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 24(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.31MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | | 24:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF017[min_max] -> i_item_id, RF016[bloom] -> i_item_id @@ -527,7 +527,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati | | in pipelines: 21(GETNEXT) | | | F14:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=133.96MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=134.28MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 27:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_bill_addr_sk = ca_address_sk | | fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk @@ -542,7 +542,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 23(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | | 23:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -570,7 +570,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 22(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 22:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(2 AS INT) @@ -629,7 +629,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.19MB mem-reservation=256.00KB thread-reservation=2 | | 15:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_color IN ('slate', 'blanched', 'burnished') @@ -657,7 +657,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 14(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.47MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | | 14:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF009[min_max] -> i_item_id, RF008[bloom] -> i_item_id @@ -675,7 +675,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati | | in pipelines: 11(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=197.96MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=198.48MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 17:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: cs_bill_addr_sk = ca_address_sk | | fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk @@ -690,7 +690,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 13(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | | 13:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -718,7 +718,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 12(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(2 AS INT) @@ -777,7 +777,7 @@ Per-Host Resources: mem-estimate=17.09MB mem-reservation=7.88MB thread-reservati | | in pipelines: 05(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.19MB mem-reservation=256.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_color IN ('slate', 'blanched', 'burnished') @@ -805,7 +805,7 @@ Per-Host Resources: mem-estimate=17.09MB mem-reservation=7.88MB thread-reservati | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.47MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> i_item_id, RF000[bloom] -> i_item_id @@ -823,7 +823,7 @@ Per-Host Resources: mem-estimate=17.09MB mem-reservation=7.88MB thread-reservati | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=54.96MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=55.48MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=3.00MB 07:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_addr_sk = ca_address_sk | fk/pk conjuncts: ss_addr_sk = ca_address_sk @@ -838,7 +838,7 @@ Per-Host Resources: mem-estimate=54.96MB mem-reservation=8.88MB thread-reservati | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -866,7 +866,7 @@ Per-Host Resources: mem-estimate=54.96MB mem-reservation=8.88MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(2 AS INT) @@ -893,7 +893,7 @@ Per-Host Resources: mem-estimate=54.96MB mem-reservation=8.88MB thread-reservati in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=111.88MB Threads=38 -Per-Host Resource Estimates: Memory=478MB +Per-Host Resource Estimates: Memory=491MB F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -928,7 +928,7 @@ Per-Instance Resources: mem-estimate=10.33MB mem-reservation=1.94MB thread-reser | in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT) | F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=21.43MB mem-reservation=3.94MB thread-reservation=1 31:AGGREGATE [STREAMING] | output: sum(total_sales) | group by: i_item_id @@ -954,7 +954,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | in pipelines: 21(GETNEXT) | | | F18:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=10.28MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=11.41MB mem-reservation=2.00MB thread-reservation=1 | 30:AGGREGATE [STREAMING] | | output: sum(ws_ext_sales_price) | | group by: i_item_id @@ -983,7 +983,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | | in pipelines: 25(GETNEXT) | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | | 25:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_color IN ('slate', 'blanched', 'burnished') @@ -1020,7 +1020,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.31MB mem-reservation=512.00KB thread-reservation=1 | | 24:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF017[min_max] -> i_item_id, RF016[bloom] -> i_item_id @@ -1039,7 +1039,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | | F14:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.41MB mem-reservation=8.00MB thread-reservation=1 | 27:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=10 | | hash predicates: ws_bill_addr_sk = ca_address_sk @@ -1062,7 +1062,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | | in pipelines: 23(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | | 23:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1098,7 +1098,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | | in pipelines: 22(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 22:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(2 AS INT) @@ -1136,7 +1136,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | in pipelines: 11(GETNEXT) | | | F11:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3 -| Per-Instance Resources: mem-estimate=10.33MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=11.46MB mem-reservation=2.00MB thread-reservation=1 | 20:AGGREGATE [STREAMING] | | output: sum(cs_ext_sales_price) | | group by: i_item_id @@ -1165,7 +1165,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_color IN ('slate', 'blanched', 'burnished') @@ -1202,7 +1202,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.47MB mem-reservation=512.00KB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF009[min_max] -> i_item_id, RF008[bloom] -> i_item_id @@ -1221,7 +1221,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | | F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=48.61MB mem-reservation=8.00MB thread-reservation=1 | 17:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=06 | | hash predicates: cs_bill_addr_sk = ca_address_sk @@ -1244,7 +1244,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | | in pipelines: 13(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | | 13:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1280,7 +1280,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | | | in pipelines: 12(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(2 AS INT) @@ -1318,7 +1318,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser | in pipelines: 01(GETNEXT) | F04:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=10.66MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=11.78MB mem-reservation=2.00MB thread-reservation=1 10:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: i_item_id @@ -1347,7 +1347,7 @@ Per-Instance Resources: mem-estimate=10.66MB mem-reservation=2.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_color IN ('slate', 'blanched', 'burnished') @@ -1384,7 +1384,7 @@ Per-Instance Resources: mem-estimate=10.66MB mem-reservation=2.00MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.94MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> i_item_id, RF000[bloom] -> i_item_id @@ -1403,7 +1403,7 @@ Per-Instance Resources: mem-estimate=10.66MB mem-reservation=2.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=17.22MB mem-reservation=2.00MB thread-reservation=1 07:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=02 | hash predicates: ss_addr_sk = ca_address_sk @@ -1426,7 +1426,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1462,7 +1462,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_moy = CAST(2 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test index 8a4859037..8b4f82524 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test @@ -441,7 +441,7 @@ Per-Host Resources: mem-estimate=38.01MB mem-reservation=17.00MB thread-reservat | | in pipelines: 24(GETNEXT) | | | F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=219.77MB mem-reservation=30.75MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=219.90MB mem-reservation=30.75MB thread-reservation=2 runtime-filters-memory=3.00MB | 30:AGGREGATE [STREAMING] | | output: sum(cs_sales_price) | | group by: i_category, i_brand, cc_name, d_year, d_moy @@ -463,7 +463,7 @@ Per-Host Resources: mem-estimate=38.01MB mem-reservation=17.00MB thread-reservat | | | in pipelines: 26(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=16.00KB thread-reservation=2 | | 26:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | | HDFS partitions=1/1 files=1 size=10.28KB | | stored statistics: @@ -488,7 +488,7 @@ Per-Host Resources: mem-estimate=38.01MB mem-reservation=17.00MB thread-reservat | | | in pipelines: 23(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.23MB mem-reservation=256.00KB thread-reservation=2 | | 23:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -513,7 +513,7 @@ Per-Host Resources: mem-estimate=38.01MB mem-reservation=17.00MB thread-reservat | | | in pipelines: 25(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: (d_year = CAST(1999 AS INT) OR (d_year = CAST(1998 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2000 AS INT) AND d_moy = CAST(1 AS INT))) @@ -604,7 +604,7 @@ Per-Host Resources: mem-estimate=35.20MB mem-reservation=34.94MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=219.77MB mem-reservation=30.75MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=219.90MB mem-reservation=30.75MB thread-reservation=2 runtime-filters-memory=3.00MB | 07:AGGREGATE [STREAMING] | | output: sum(cs_sales_price) | | group by: i_category, i_brand, cc_name, d_year, d_moy @@ -626,7 +626,7 @@ Per-Host Resources: mem-estimate=35.20MB mem-reservation=34.94MB thread-reservat | | | in pipelines: 03(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=16.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | | HDFS partitions=1/1 files=1 size=10.28KB | | stored statistics: @@ -651,7 +651,7 @@ Per-Host Resources: mem-estimate=35.20MB mem-reservation=34.94MB thread-reservat | | | in pipelines: 00(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.23MB mem-reservation=256.00KB thread-reservation=2 | | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -676,7 +676,7 @@ Per-Host Resources: mem-estimate=35.20MB mem-reservation=34.94MB thread-reservat | | | in pipelines: 02(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: (d_year = CAST(1999 AS INT) OR (d_year = CAST(1998 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2000 AS INT) AND d_moy = CAST(1 AS INT))) @@ -727,7 +727,7 @@ Per-Host Resources: mem-estimate=35.20MB mem-reservation=34.94MB thread-reservat | in pipelines: 14(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=219.77MB mem-reservation=30.75MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=219.90MB mem-reservation=30.75MB thread-reservation=2 runtime-filters-memory=3.00MB 20:AGGREGATE [STREAMING] | output: sum(cs_sales_price) | group by: i_category, i_brand, cc_name, d_year, d_moy @@ -749,7 +749,7 @@ Per-Host Resources: mem-estimate=219.77MB mem-reservation=30.75MB thread-reserva | | in pipelines: 16(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=16.00KB thread-reservation=2 | 16:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | HDFS partitions=1/1 files=1 size=10.28KB | stored statistics: @@ -774,7 +774,7 @@ Per-Host Resources: mem-estimate=219.77MB mem-reservation=30.75MB thread-reserva | | in pipelines: 13(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.23MB mem-reservation=256.00KB thread-reservation=2 | 13:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -799,7 +799,7 @@ Per-Host Resources: mem-estimate=219.77MB mem-reservation=30.75MB thread-reserva | | in pipelines: 15(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: (d_year = CAST(1999 AS INT) OR (d_year = CAST(1998 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2000 AS INT) AND d_moy = CAST(1 AS INT))) @@ -823,7 +823,7 @@ Per-Host Resources: mem-estimate=219.77MB mem-reservation=30.75MB thread-reserva in pipelines: 14(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=251.67MB Threads=28 -Per-Host Resource Estimates: Memory=550MB +Per-Host Resource Estimates: Memory=554MB F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -896,7 +896,7 @@ Per-Instance Resources: mem-estimate=10.70MB mem-reservation=0B thread-reservati | | | F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=65.00MB mem-reservation=21.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=66.15MB mem-reservation=21.00MB thread-reservation=1 | 30:AGGREGATE [STREAMING] | | output: sum(cs_sales_price) | | group by: i_category, i_brand, cc_name, d_year, d_moy @@ -926,7 +926,7 @@ Per-Instance Resources: mem-estimate=10.70MB mem-reservation=0B thread-reservati | | | in pipelines: 26(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=16.00KB thread-reservation=1 | | 26:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | | HDFS partitions=1/1 files=1 size=10.28KB | | stored statistics: @@ -959,7 +959,7 @@ Per-Instance Resources: mem-estimate=10.70MB mem-reservation=0B thread-reservati | | | in pipelines: 23(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.23MB mem-reservation=256.00KB thread-reservation=1 | | 23:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -992,7 +992,7 @@ Per-Instance Resources: mem-estimate=10.70MB mem-reservation=0B thread-reservati | | | in pipelines: 25(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: (d_year = CAST(1999 AS INT) OR (d_year = CAST(1998 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2000 AS INT) AND d_moy = CAST(1 AS INT))) @@ -1092,7 +1092,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese | | | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=65.00MB mem-reservation=21.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=66.15MB mem-reservation=21.00MB thread-reservation=1 | 07:AGGREGATE [STREAMING] | | output: sum(cs_sales_price) | | group by: i_category, i_brand, cc_name, d_year, d_moy @@ -1122,7 +1122,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese | | | in pipelines: 03(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=16.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | | HDFS partitions=1/1 files=1 size=10.28KB | | stored statistics: @@ -1155,7 +1155,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese | | | in pipelines: 00(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.23MB mem-reservation=256.00KB thread-reservation=1 | | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -1188,7 +1188,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese | | | in pipelines: 02(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: (d_year = CAST(1999 AS INT) OR (d_year = CAST(1998 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2000 AS INT) AND d_moy = CAST(1 AS INT))) @@ -1240,7 +1240,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=65.00MB mem-reservation=21.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=66.15MB mem-reservation=21.00MB thread-reservation=1 20:AGGREGATE [STREAMING] | output: sum(cs_sales_price) | group by: i_category, i_brand, cc_name, d_year, d_moy @@ -1270,7 +1270,7 @@ Per-Instance Resources: mem-estimate=65.00MB mem-reservation=21.00MB thread-rese | | in pipelines: 16(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=16.00KB thread-reservation=1 | 16:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | HDFS partitions=1/1 files=1 size=10.28KB | stored statistics: @@ -1303,7 +1303,7 @@ Per-Instance Resources: mem-estimate=65.00MB mem-reservation=21.00MB thread-rese | | in pipelines: 13(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.23MB mem-reservation=256.00KB thread-reservation=1 | 13:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -1336,7 +1336,7 @@ Per-Instance Resources: mem-estimate=65.00MB mem-reservation=21.00MB thread-rese | | in pipelines: 15(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: (d_year = CAST(1999 AS INT) OR (d_year = CAST(1998 AS INT) AND d_moy = CAST(12 AS INT)) OR (d_year = CAST(2000 AS INT) AND d_moy = CAST(1 AS INT))) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test index eeace4237..8e2c6b51d 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test @@ -406,7 +406,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=92.88MB Threads=35 -Per-Host Resource Estimates: Memory=777MB +Per-Host Resource Estimates: Memory=779MB F21:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -471,7 +471,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 14(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=34.95MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=35.05MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 17:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | | hash predicates: d_week_seq = d_week_seq | | | runtime filters: RF026[bloom] <- d_week_seq, RF027[min_max] <- d_week_seq @@ -485,7 +485,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat | | | | in pipelines: 16(GETNEXT) | | | | | | | F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| | | Per-Host Resources: mem-estimate=120.00KB mem-reservation=0B thread-reservation=1 | | | 16:CARDINALITY CHECK | | | | limit: 1 | | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -499,7 +499,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat | | | | in pipelines: 15(GETNEXT) | | | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_date = '2000-01-03' @@ -539,7 +539,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 13(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.12MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF021[min_max] -> d_date, RF020[bloom] -> d_date @@ -565,7 +565,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 12(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.16MB mem-reservation=512.00KB thread-reservation=2 | | 12:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -630,7 +630,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 25(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=34.95MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=35.05MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 28:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | | hash predicates: d_week_seq = d_week_seq | | | runtime filters: RF018[bloom] <- d_week_seq, RF019[min_max] <- d_week_seq @@ -644,7 +644,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat | | | | in pipelines: 27(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| | | Per-Host Resources: mem-estimate=120.00KB mem-reservation=0B thread-reservation=1 | | | 27:CARDINALITY CHECK | | | | limit: 1 | | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -658,7 +658,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat | | | | in pipelines: 26(GETNEXT) | | | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | | 26:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_date = '2000-01-03' @@ -698,7 +698,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 24(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.12MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF013[min_max] -> d_date, RF012[bloom] -> d_date @@ -724,7 +724,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 23(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.16MB mem-reservation=512.00KB thread-reservation=2 | | 23:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_id @@ -781,7 +781,7 @@ Per-Host Resources: mem-estimate=56.14MB mem-reservation=14.62MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=34.95MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=35.05MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB | 06:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | hash predicates: d_week_seq = d_week_seq | | runtime filters: RF010[bloom] <- d_week_seq, RF011[min_max] <- d_week_seq @@ -795,7 +795,7 @@ Per-Host Resources: mem-estimate=56.14MB mem-reservation=14.62MB thread-reservat | | | in pipelines: 05(GETNEXT) | | | | | F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| | Per-Host Resources: mem-estimate=120.00KB mem-reservation=0B thread-reservation=1 | | 05:CARDINALITY CHECK | | | limit: 1 | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -809,7 +809,7 @@ Per-Host Resources: mem-estimate=56.14MB mem-reservation=14.62MB thread-reservat | | | in pipelines: 04(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_date = '2000-01-03' @@ -849,7 +849,7 @@ Per-Host Resources: mem-estimate=56.14MB mem-reservation=14.62MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.12MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF005[min_max] -> d_date, RF004[bloom] -> d_date @@ -875,7 +875,7 @@ Per-Host Resources: mem-estimate=56.14MB mem-reservation=14.62MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.16MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF003[min_max] -> tpcds_parquet.item.i_item_id, RF001[min_max] -> tpcds_parquet.item.i_item_id, RF002[bloom] -> tpcds_parquet.item.i_item_id @@ -900,7 +900,7 @@ Per-Host Resources: mem-estimate=56.14MB mem-reservation=14.62MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=144.25MB Threads=38 -Per-Host Resource Estimates: Memory=508MB +Per-Host Resource Estimates: Memory=514MB F21:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -952,7 +952,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | in pipelines: 11(GETNEXT) | | | F14:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Instance Resources: mem-estimate=58.00MB mem-reservation=10.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=59.12MB mem-reservation=10.00MB thread-reservation=1 | 21:AGGREGATE [STREAMING] | | output: sum(cs_ext_sales_price) | | group by: i_item_id @@ -982,7 +982,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 17:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | | hash-table-id=02 | | | hash predicates: d_week_seq = d_week_seq @@ -1004,7 +1004,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 16(GETNEXT) | | | | | | | F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=120.00KB mem-reservation=0B thread-reservation=1 | | | 16:CARDINALITY CHECK | | | | limit: 1 | | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -1018,7 +1018,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 15(GETNEXT) | | | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_date = '2000-01-03' @@ -1067,7 +1067,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF021[min_max] -> d_date, RF020[bloom] -> d_date @@ -1101,7 +1101,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | | in pipelines: 12(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | | 12:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -1154,7 +1154,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | | F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=42.00MB mem-reservation=10.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=43.12MB mem-reservation=10.00MB thread-reservation=1 | 32:AGGREGATE [STREAMING] | | output: sum(ws_ext_sales_price) | | group by: i_item_id @@ -1184,7 +1184,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 28:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | | hash-table-id=07 | | | hash predicates: d_week_seq = d_week_seq @@ -1206,7 +1206,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 27(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=120.00KB mem-reservation=0B thread-reservation=1 | | | 27:CARDINALITY CHECK | | | | limit: 1 | | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -1220,7 +1220,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 26(GETNEXT) | | | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | | 26:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_date = '2000-01-03' @@ -1269,7 +1269,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF013[min_max] -> d_date, RF012[bloom] -> d_date @@ -1303,7 +1303,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | | | in pipelines: 23(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | | 23:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_id @@ -1340,7 +1340,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=27.12MB mem-reservation=3.00MB thread-reservation=1 10:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: i_item_id @@ -1370,7 +1370,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 06:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | hash-table-id=11 | | hash predicates: d_week_seq = d_week_seq @@ -1392,7 +1392,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | in pipelines: 05(GETNEXT) | | | | | F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1 +| | Per-Instance Resources: mem-estimate=120.00KB mem-reservation=0B thread-reservation=1 | | 05:CARDINALITY CHECK | | | limit: 1 | | | mem-estimate=0B mem-reservation=0B thread-reservation=0 @@ -1406,7 +1406,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | in pipelines: 04(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_date = '2000-01-03' @@ -1455,7 +1455,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF005[min_max] -> d_date, RF004[bloom] -> d_date @@ -1490,7 +1490,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF003[min_max] -> tpcds_parquet.item.i_item_id, RF001[min_max] -> tpcds_parquet.item.i_item_id, RF002[bloom] -> tpcds_parquet.item.i_item_id diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q59.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q59.test index cf803479d..afc2b0acb 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q59.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q59.test @@ -260,7 +260,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=87.54MB Threads=20 -Per-Host Resource Estimates: Memory=415MB +Per-Host Resource Estimates: Memory=416MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.09MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -311,7 +311,7 @@ Per-Host Resources: mem-estimate=13.17MB mem-reservation=6.75MB thread-reservati | | | in pipelines: 12(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.14MB mem-reservation=16.00KB thread-reservation=2 | | 12:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -336,7 +336,7 @@ Per-Host Resources: mem-estimate=13.17MB mem-reservation=6.75MB thread-reservati | | | in pipelines: 13(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 13:SCAN HDFS [tpcds_parquet.date_dim d, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1208 AS INT), d_month_seq >= CAST(1197 AS INT) @@ -385,7 +385,7 @@ Per-Host Resources: mem-estimate=13.17MB mem-reservation=6.75MB thread-reservati | | | in pipelines: 09(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=49.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=49.12MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | | 09:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF013[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF012[bloom] -> tpcds_parquet.date_dim.d_week_seq @@ -430,7 +430,7 @@ Per-Host Resources: mem-estimate=18.76MB mem-reservation=10.62MB thread-reservat | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=49.00MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=49.20MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB | 04:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | runtime filters: RF003[min_max] -> s_store_id, RF001[bloom] -> s_store_id @@ -456,7 +456,7 @@ Per-Host Resources: mem-estimate=18.76MB mem-reservation=10.62MB thread-reservat | | in pipelines: 05(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.05MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | 05:SCAN HDFS [tpcds_parquet.date_dim d, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1196 AS INT), d_month_seq >= CAST(1185 AS INT) @@ -506,7 +506,7 @@ Per-Host Resources: mem-estimate=57.67MB mem-reservation=24.25MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=50.00MB mem-reservation=2.50MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=50.12MB mem-reservation=2.50MB thread-reservation=2 runtime-filters-memory=2.00MB | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF007[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF002[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF006[bloom] -> tpcds_parquet.date_dim.d_week_seq, RF000[bloom] -> tpcds_parquet.date_dim.d_week_seq @@ -531,7 +531,7 @@ Per-Host Resources: mem-estimate=57.67MB mem-reservation=24.25MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=146.79MB Threads=25 -Per-Host Resource Estimates: Memory=363MB +Per-Host Resource Estimates: Memory=380MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.17MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -575,7 +575,7 @@ Per-Instance Resources: mem-estimate=3.94MB mem-reservation=0B thread-reservatio | | in pipelines: 25(GETNEXT) | | | F07:PLAN FRAGMENT [HASH(d_week_seq,ss_store_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=13.16MB mem-reservation=2.88MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=14.03MB mem-reservation=2.88MB thread-reservation=1 | 15:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=01 | | hash predicates: ss_store_sk = s_store_sk @@ -598,7 +598,7 @@ Per-Instance Resources: mem-estimate=3.94MB mem-reservation=0B thread-reservatio | | | in pipelines: 12(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1 | | 12:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -631,7 +631,7 @@ Per-Instance Resources: mem-estimate=3.94MB mem-reservation=0B thread-reservatio | | | in pipelines: 13(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 13:SCAN HDFS [tpcds_parquet.date_dim d, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1208 AS INT), d_month_seq >= CAST(1197 AS INT) @@ -659,7 +659,7 @@ Per-Instance Resources: mem-estimate=3.94MB mem-reservation=0B thread-reservatio | | | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=33.00MB mem-reservation=17.50MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=35.91MB mem-reservation=17.50MB thread-reservation=1 | 11:AGGREGATE [STREAMING] | | output: sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) | | group by: d_week_seq, ss_store_sk @@ -690,7 +690,7 @@ Per-Instance Resources: mem-estimate=3.94MB mem-reservation=0B thread-reservatio | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=512.00KB thread-reservation=1 | | 09:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF013[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF012[bloom] -> tpcds_parquet.date_dim.d_week_seq @@ -720,7 +720,7 @@ Per-Instance Resources: mem-estimate=3.94MB mem-reservation=0B thread-reservatio | in pipelines: 20(GETNEXT) | F02:PLAN FRAGMENT [HASH(d_week_seq,ss_store_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=13.16MB mem-reservation=2.88MB thread-reservation=1 +Per-Instance Resources: mem-estimate=14.41MB mem-reservation=2.88MB thread-reservation=1 07:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=04 | hash predicates: ss_store_sk = s_store_sk @@ -744,7 +744,7 @@ Per-Instance Resources: mem-estimate=13.16MB mem-reservation=2.88MB thread-reser | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.20MB mem-reservation=24.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | runtime filters: RF003[min_max] -> s_store_id, RF001[bloom] -> s_store_id @@ -779,7 +779,7 @@ Per-Instance Resources: mem-estimate=13.16MB mem-reservation=2.88MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.date_dim d, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1196 AS INT), d_month_seq >= CAST(1185 AS INT) @@ -808,7 +808,7 @@ Per-Instance Resources: mem-estimate=13.16MB mem-reservation=2.88MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=33.00MB mem-reservation=17.50MB thread-reservation=1 +Per-Instance Resources: mem-estimate=35.91MB mem-reservation=17.50MB thread-reservation=1 03:AGGREGATE [STREAMING] | output: sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) | group by: d_week_seq, ss_store_sk @@ -839,7 +839,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=17.50MB thread-rese | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF007[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF002[min_max] -> tpcds_parquet.date_dim.d_week_seq, RF006[bloom] -> tpcds_parquet.date_dim.d_week_seq, RF000[bloom] -> tpcds_parquet.date_dim.d_week_seq diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test old mode 100755 new mode 100644 index 61417df3a..5be636d4c --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test @@ -400,7 +400,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=75.62MB Threads=36 -Per-Host Resource Estimates: Memory=906MB +Per-Host Resource Estimates: Memory=910MB F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -435,7 +435,7 @@ Per-Host Resources: mem-estimate=10.29MB mem-reservation=1.94MB thread-reservati | in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT) | F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=20.78MB mem-reservation=3.94MB thread-reservation=1 31:AGGREGATE [STREAMING] | output: sum(total_sales) | group by: i_item_id @@ -482,7 +482,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 25(GETNEXT) | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.19MB mem-reservation=256.00KB thread-reservation=2 | | 25:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category IN ('Music') @@ -510,7 +510,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 24(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.31MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | | 24:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF017[min_max] -> i_item_id, RF016[bloom] -> i_item_id @@ -528,7 +528,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati | | in pipelines: 21(GETNEXT) | | | F14:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=133.96MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=134.28MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 27:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_bill_addr_sk = ca_address_sk | | fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk @@ -543,7 +543,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 23(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | | 23:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -571,7 +571,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 22(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 22:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(9 AS INT) @@ -630,7 +630,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.19MB mem-reservation=256.00KB thread-reservation=2 | | 15:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category IN ('Music') @@ -658,7 +658,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 14(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.47MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | | 14:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF009[min_max] -> i_item_id, RF008[bloom] -> i_item_id @@ -676,7 +676,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati | | in pipelines: 11(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=197.96MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=198.48MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 17:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: cs_bill_addr_sk = ca_address_sk | | fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk @@ -691,7 +691,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 13(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | | 13:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -719,7 +719,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati | | | in pipelines: 12(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(9 AS INT) @@ -778,7 +778,7 @@ Per-Host Resources: mem-estimate=17.16MB mem-reservation=7.88MB thread-reservati | | in pipelines: 05(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.19MB mem-reservation=256.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category IN ('Music') @@ -806,7 +806,7 @@ Per-Host Resources: mem-estimate=17.16MB mem-reservation=7.88MB thread-reservati | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.47MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> i_item_id, RF000[bloom] -> i_item_id @@ -824,7 +824,7 @@ Per-Host Resources: mem-estimate=17.16MB mem-reservation=7.88MB thread-reservati | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=54.96MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=55.48MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=3.00MB 07:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_addr_sk = ca_address_sk | fk/pk conjuncts: ss_addr_sk = ca_address_sk @@ -839,7 +839,7 @@ Per-Host Resources: mem-estimate=54.96MB mem-reservation=8.88MB thread-reservati | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -867,7 +867,7 @@ Per-Host Resources: mem-estimate=54.96MB mem-reservation=8.88MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(9 AS INT) @@ -894,7 +894,7 @@ Per-Host Resources: mem-estimate=54.96MB mem-reservation=8.88MB thread-reservati in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=111.88MB Threads=38 -Per-Host Resource Estimates: Memory=478MB +Per-Host Resource Estimates: Memory=491MB F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -929,7 +929,7 @@ Per-Instance Resources: mem-estimate=10.43MB mem-reservation=1.94MB thread-reser | in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT) | F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=21.48MB mem-reservation=3.94MB thread-reservation=1 31:AGGREGATE [STREAMING] | output: sum(total_sales) | group by: i_item_id @@ -955,7 +955,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | in pipelines: 21(GETNEXT) | | | F18:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=10.28MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=11.41MB mem-reservation=2.00MB thread-reservation=1 | 30:AGGREGATE [STREAMING] | | output: sum(ws_ext_sales_price) | | group by: i_item_id @@ -984,7 +984,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | | in pipelines: 25(GETNEXT) | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | | 25:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category IN ('Music') @@ -1021,7 +1021,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.31MB mem-reservation=512.00KB thread-reservation=1 | | 24:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF017[min_max] -> i_item_id, RF016[bloom] -> i_item_id @@ -1040,7 +1040,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | | F14:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.41MB mem-reservation=8.00MB thread-reservation=1 | 27:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=10 | | hash predicates: ws_bill_addr_sk = ca_address_sk @@ -1063,7 +1063,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | | in pipelines: 23(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | | 23:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1099,7 +1099,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | | in pipelines: 22(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 22:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(9 AS INT) @@ -1137,7 +1137,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | in pipelines: 11(GETNEXT) | | | F11:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3 -| Per-Instance Resources: mem-estimate=10.33MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=11.46MB mem-reservation=2.00MB thread-reservation=1 | 20:AGGREGATE [STREAMING] | | output: sum(cs_ext_sales_price) | | group by: i_item_id @@ -1166,7 +1166,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category IN ('Music') @@ -1203,7 +1203,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.47MB mem-reservation=512.00KB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF009[min_max] -> i_item_id, RF008[bloom] -> i_item_id @@ -1222,7 +1222,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | | F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=48.61MB mem-reservation=8.00MB thread-reservation=1 | 17:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=06 | | hash predicates: cs_bill_addr_sk = ca_address_sk @@ -1245,7 +1245,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | | in pipelines: 13(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | | 13:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1281,7 +1281,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | | | in pipelines: 12(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(9 AS INT) @@ -1319,7 +1319,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser | in pipelines: 01(GETNEXT) | F04:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=10.66MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=11.78MB mem-reservation=2.00MB thread-reservation=1 10:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: i_item_id @@ -1348,7 +1348,7 @@ Per-Instance Resources: mem-estimate=10.66MB mem-reservation=2.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category IN ('Music') @@ -1385,7 +1385,7 @@ Per-Instance Resources: mem-estimate=10.66MB mem-reservation=2.00MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.94MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF001[min_max] -> i_item_id, RF000[bloom] -> i_item_id @@ -1404,7 +1404,7 @@ Per-Instance Resources: mem-estimate=10.66MB mem-reservation=2.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=17.22MB mem-reservation=2.00MB thread-reservation=1 07:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=02 | hash predicates: ss_addr_sk = ca_address_sk @@ -1427,7 +1427,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1463,7 +1463,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(9 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test index e0ab2ad10..eb62c82c0 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test @@ -328,7 +328,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=49.25MB Threads=30 -Per-Host Resource Estimates: Memory=605MB +Per-Host Resource Estimates: Memory=608MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -352,7 +352,7 @@ PLAN-ROOT SINK | | in pipelines: 44(GETNEXT) | | | F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 44:AGGREGATE [FINALIZE] | | output: sum:merge(ss_ext_sales_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -386,7 +386,7 @@ PLAN-ROOT SINK | | | in pipelines: 18(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | | 18:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -414,7 +414,7 @@ PLAN-ROOT SINK | | | in pipelines: 17(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.14MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 17:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF013[min_max] -> c_current_addr_sk, RF012[bloom] -> c_current_addr_sk @@ -432,7 +432,7 @@ PLAN-ROOT SINK | | in pipelines: 14(GETNEXT) | | | F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=72.92MB mem-reservation=10.81MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=73.82MB mem-reservation=10.81MB thread-reservation=2 runtime-filters-memory=3.00MB | 22:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_store_sk = s_store_sk | | fk/pk conjuncts: ss_store_sk = s_store_sk @@ -447,7 +447,7 @@ PLAN-ROOT SINK | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=16.00KB thread-reservation=2 | | 15:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: s_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -475,7 +475,7 @@ PLAN-ROOT SINK | | | in pipelines: 19(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=256.00KB thread-reservation=2 | | 19:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Jewelry' @@ -503,7 +503,7 @@ PLAN-ROOT SINK | | | in pipelines: 16(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 16:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(11 AS INT) @@ -562,7 +562,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -590,7 +590,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio | | in pipelines: 04(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.14MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 04:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> c_current_addr_sk, RF000[bloom] -> c_current_addr_sk @@ -608,7 +608,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=94.35MB mem-reservation=14.75MB thread-reservation=2 runtime-filters-memory=5.00MB 10:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_store_sk = s_store_sk | fk/pk conjuncts: ss_store_sk = s_store_sk @@ -623,7 +623,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=16.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -651,7 +651,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=32.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.18MB mem-reservation=32.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | predicates: (p_channel_dmail = 'Y' OR p_channel_email = 'Y' OR p_channel_tv = 'Y') @@ -677,7 +677,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat | | in pipelines: 06(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=256.00KB thread-reservation=2 | 06:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category = 'Jewelry' @@ -705,7 +705,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(11 AS INT) @@ -732,7 +732,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=80.69MB Threads=35 -Per-Host Resource Estimates: Memory=317MB +Per-Host Resource Estimates: Memory=329MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -763,7 +763,7 @@ PLAN-ROOT SINK | | in pipelines: 44(GETNEXT) | | | F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 44:AGGREGATE [FINALIZE] | | output: sum:merge(ss_ext_sales_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -805,7 +805,7 @@ PLAN-ROOT SINK | | | in pipelines: 18(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | | 18:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -842,7 +842,7 @@ PLAN-ROOT SINK | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.28MB mem-reservation=1.00MB thread-reservation=1 | | 17:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF013[min_max] -> c_current_addr_sk, RF012[bloom] -> c_current_addr_sk @@ -861,7 +861,7 @@ PLAN-ROOT SINK | | | F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=18.01MB mem-reservation=2.00MB thread-reservation=1 | 22:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=03 | | hash predicates: ss_store_sk = s_store_sk @@ -884,7 +884,7 @@ PLAN-ROOT SINK | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=16.00KB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: s_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -920,7 +920,7 @@ PLAN-ROOT SINK | | | in pipelines: 19(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=256.00KB thread-reservation=1 | | 19:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Jewelry' @@ -956,7 +956,7 @@ PLAN-ROOT SINK | | | in pipelines: 16(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 16:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(11 AS INT) @@ -1023,7 +1023,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1060,7 +1060,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.28MB mem-reservation=1.00MB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> c_current_addr_sk, RF000[bloom] -> c_current_addr_sk @@ -1079,7 +1079,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=19.21MB mem-reservation=2.00MB thread-reservation=1 10:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=08 | hash predicates: ss_store_sk = s_store_sk @@ -1102,7 +1102,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=16.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1138,7 +1138,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.18MB mem-reservation=32.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | predicates: (p_channel_dmail = 'Y' OR p_channel_email = 'Y' OR p_channel_tv = 'Y') @@ -1172,7 +1172,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 06(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=256.00KB thread-reservation=1 | 06:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category = 'Jewelry' @@ -1208,7 +1208,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(11 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q62.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q62.test index b5b763b5c..3fa48775d 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q62.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q62.test @@ -154,7 +154,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=24.23MB Threads=12 -Per-Host Resource Estimates: Memory=324MB +Per-Host Resource Estimates: Memory=325MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -189,7 +189,7 @@ Per-Host Resources: mem-estimate=10.03MB mem-reservation=1.94MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -Per-Host Resources: mem-estimate=181.86MB mem-reservation=17.75MB thread-reservation=2 runtime-filters-memory=4.00MB +Per-Host Resources: mem-estimate=182.48MB mem-reservation=17.75MB thread-reservation=2 runtime-filters-memory=4.00MB 09:AGGREGATE [STREAMING] | output: sum(CAST(CASE WHEN (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) <= CAST(30 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) > CAST(30 AS BIGINT)) AND (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) <= CAST(60 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) > CAST(60 AS BIGINT)) AND (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) <= CAST(90 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) > CAST(90 AS BIGINT)) AND (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) <= CAST(120 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) > CAST(120 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)) | group by: substring(w_warehouse_name, CAST(1 AS BIGINT), CAST(20 AS BIGINT)), sm_type, web_name @@ -211,7 +211,7 @@ Per-Host Resources: mem-estimate=181.86MB mem-reservation=17.75MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -239,7 +239,7 @@ Per-Host Resources: mem-estimate=181.86MB mem-reservation=17.75MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.10MB mem-reservation=16.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.web_site, RANDOM] | HDFS partitions=1/1 files=1 size=11.91KB | stored statistics: @@ -264,7 +264,7 @@ Per-Host Resources: mem-estimate=181.86MB mem-reservation=17.75MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.11MB mem-reservation=16.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.ship_mode, RANDOM] | HDFS partitions=1/1 files=1 size=2.68KB | stored statistics: @@ -289,7 +289,7 @@ Per-Host Resources: mem-estimate=181.86MB mem-reservation=17.75MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.15MB mem-reservation=16.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | stored statistics: @@ -312,7 +312,7 @@ Per-Host Resources: mem-estimate=181.86MB mem-reservation=17.75MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=35.98MB Threads=11 -Per-Host Resource Estimates: Memory=144MB +Per-Host Resource Estimates: Memory=145MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -348,7 +348,7 @@ Per-Instance Resources: mem-estimate=10.03MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=42.00MB mem-reservation=6.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=42.73MB mem-reservation=6.00MB thread-reservation=1 09:AGGREGATE [STREAMING] | output: sum(CAST(CASE WHEN (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) <= CAST(30 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) > CAST(30 AS BIGINT)) AND (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) <= CAST(60 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) > CAST(60 AS BIGINT)) AND (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) <= CAST(90 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) > CAST(90 AS BIGINT)) AND (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) <= CAST(120 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(ws_ship_date_sk AS BIGINT) - CAST(ws_sold_date_sk AS BIGINT) > CAST(120 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)) | group by: substring(w_warehouse_name, CAST(1 AS BIGINT), CAST(20 AS BIGINT)), sm_type, web_name @@ -378,7 +378,7 @@ Per-Instance Resources: mem-estimate=42.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -414,7 +414,7 @@ Per-Instance Resources: mem-estimate=42.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.10MB mem-reservation=16.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.web_site, RANDOM] | HDFS partitions=1/1 files=1 size=11.91KB | stored statistics: @@ -447,7 +447,7 @@ Per-Instance Resources: mem-estimate=42.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.11MB mem-reservation=16.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.ship_mode, RANDOM] | HDFS partitions=1/1 files=1 size=2.68KB | stored statistics: @@ -480,7 +480,7 @@ Per-Instance Resources: mem-estimate=42.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.15MB mem-reservation=16.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | stored statistics: diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test index a6a9e98b9..3a954b33c 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test @@ -145,7 +145,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=28.52MB Threads=10 -Per-Host Resource Estimates: Memory=235MB +Per-Host Resource Estimates: Memory=236MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -199,7 +199,7 @@ Per-Host Resources: mem-estimate=20.00MB mem-reservation=11.94MB thread-reservat | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=67.14MB mem-reservation=11.81MB thread-reservation=2 runtime-filters-memory=3.00MB 07:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: i_manager_id, d_moy @@ -221,7 +221,7 @@ Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -246,7 +246,7 @@ Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq IN (CAST(1212 AS INT), CAST(1213 AS INT), CAST(1214 AS INT), CAST(1215 AS INT), CAST(1216 AS INT), CAST(1217 AS INT), CAST(1218 AS INT), CAST(1219 AS INT), CAST(1220 AS INT), CAST(1221 AS INT), CAST(1222 AS INT), CAST(1223 AS INT)) @@ -274,7 +274,7 @@ Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservat | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=80.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=80.32MB mem-reservation=256.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: ((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'refernece', 'self-help') AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR (i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants') AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) @@ -299,7 +299,7 @@ Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservat in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=52.27MB Threads=11 -Per-Host Resource Estimates: Memory=162MB +Per-Host Resource Estimates: Memory=163MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -354,7 +354,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=11.94MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.66MB mem-reservation=3.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: i_manager_id, d_moy @@ -384,7 +384,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -417,7 +417,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq IN (CAST(1212 AS INT), CAST(1213 AS INT), CAST(1214 AS INT), CAST(1215 AS INT), CAST(1216 AS INT), CAST(1217 AS INT), CAST(1218 AS INT), CAST(1219 AS INT), CAST(1220 AS INT), CAST(1221 AS INT), CAST(1222 AS INT), CAST(1223 AS INT)) @@ -453,7 +453,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.32MB mem-reservation=256.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: ((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'refernece', 'self-help') AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR (i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants') AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test index 3aa37d1af..aad0be77f 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test @@ -885,7 +885,7 @@ PLAN-ROOT SINK in pipelines: 17(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=521.64MB Threads=98 -Per-Host Resource Estimates: Memory=2.83GB +Per-Host Resource Estimates: Memory=2.85GB F59:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=121.76MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -994,7 +994,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | | in pipelines: 42(GETNEXT) | | | | | | | F55:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=120.00MB mem-reservation=4.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=120.12MB mem-reservation=4.00MB thread-reservation=2 | | | 42:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=10.62MB | | | stored statistics: @@ -1030,7 +1030,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 58(GETNEXT) | | | | | F53:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=2 | | 58:SCAN HDFS [tpcds_parquet.income_band ib2, RANDOM] | | HDFS partitions=1/1 files=1 size=1.21KB | | stored statistics: @@ -1055,7 +1055,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 57(GETNEXT) | | | | | F52:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=2 | | 57:SCAN HDFS [tpcds_parquet.income_band ib1, RANDOM] | | HDFS partitions=1/1 files=1 size=1.21KB | | stored statistics: @@ -1080,7 +1080,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 56(GETNEXT) | | | | | F50:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=80.96MB mem-reservation=1.00MB thread-reservation=2 | | 56:SCAN HDFS [tpcds_parquet.customer_address ad2, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -1097,7 +1097,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | in pipelines: 39(GETNEXT) | | | F49:PLAN FRAGMENT [HASH(ss_addr_sk)] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=8.33MB mem-reservation=2.88MB thread-reservation=1 +| Per-Host Resources: mem-estimate=9.28MB mem-reservation=2.88MB thread-reservation=1 | 72:HASH JOIN [INNER JOIN, PARTITIONED] | | hash predicates: ss_addr_sk = ad1.ca_address_sk | | fk/pk conjuncts: ss_addr_sk = ad1.ca_address_sk @@ -1112,7 +1112,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 55(GETNEXT) | | | | | F48:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=80.96MB mem-reservation=1.00MB thread-reservation=2 | | 55:SCAN HDFS [tpcds_parquet.customer_address ad1, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -1145,7 +1145,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 51(GETNEXT) | | | | | F46:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.25MB mem-reservation=8.00MB thread-reservation=2 | | 51:SCAN HDFS [tpcds_parquet.customer_demographics cd2, RANDOM] | | HDFS partitions=1/1 files=1 size=7.49MB | | stored statistics: @@ -1177,7 +1177,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 50(GETNEXT) | | | | | F44:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.25MB mem-reservation=8.00MB thread-reservation=2 | | 50:SCAN HDFS [tpcds_parquet.customer_demographics cd1, RANDOM] | | HDFS partitions=1/1 files=1 size=7.49MB | | stored statistics: @@ -1194,7 +1194,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | in pipelines: 39(GETNEXT) | | | F43:PLAN FRAGMENT [HASH(c_first_shipto_date_sk)] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=3.91MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=6.63MB mem-reservation=1.94MB thread-reservation=1 | 69:HASH JOIN [INNER JOIN, PARTITIONED] | | hash predicates: c_first_shipto_date_sk = d3.d_date_sk | | fk/pk conjuncts: c_first_shipto_date_sk = d3.d_date_sk @@ -1209,7 +1209,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 47(GETNEXT) | | | | | F42:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.14MB mem-reservation=512.00KB thread-reservation=2 | | 47:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | stored statistics: @@ -1226,7 +1226,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | in pipelines: 39(GETNEXT) | | | F41:PLAN FRAGMENT [HASH(c_first_sales_date_sk)] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=3.85MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=6.43MB mem-reservation=1.94MB thread-reservation=1 | 68:HASH JOIN [INNER JOIN, PARTITIONED] | | hash predicates: c_first_sales_date_sk = d2.d_date_sk | | fk/pk conjuncts: c_first_sales_date_sk = d2.d_date_sk @@ -1241,7 +1241,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 46(GETNEXT) | | | | | F40:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.14MB mem-reservation=512.00KB thread-reservation=2 | | 46:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | stored statistics: @@ -1258,7 +1258,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | in pipelines: 39(GETNEXT) | | | F38:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=8.35MB mem-reservation=4.81MB thread-reservation=1 +| Per-Host Resources: mem-estimate=8.97MB mem-reservation=4.81MB thread-reservation=1 | 67:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: c_current_hdemo_sk = hd2.hd_demo_sk | | fk/pk conjuncts: c_current_hdemo_sk = hd2.hd_demo_sk @@ -1273,7 +1273,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 54(GETNEXT) | | | | | F39:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=64.00KB thread-reservation=2 | | 54:SCAN HDFS [tpcds_parquet.household_demographics hd2, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | runtime filters: RF049[min_max] -> hd2.hd_income_band_sk @@ -1299,7 +1299,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 49(GETNEXT) | | | | | F37:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=96.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=96.33MB mem-reservation=2.00MB thread-reservation=2 | | 49:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF065[min_max] -> c_current_hdemo_sk, RF053[min_max] -> c_current_addr_sk, RF057[min_max] -> c_current_cdemo_sk, RF061[min_max] -> c_first_shipto_date_sk, RF063[min_max] -> c_first_sales_date_sk @@ -1332,7 +1332,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 48(GETNEXT) | | | | | F36:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.16MB mem-reservation=24.00KB thread-reservation=2 | | 48:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -1357,7 +1357,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 52(GETNEXT) | | | | | F35:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=16.03MB mem-reservation=32.00KB thread-reservation=2 | | 52:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | | HDFS partitions=1/1 files=1 size=23.30KB | | stored statistics: @@ -1382,7 +1382,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 53(GETNEXT) | | | | | F34:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=64.00KB thread-reservation=2 | | 53:SCAN HDFS [tpcds_parquet.household_demographics hd1, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | runtime filters: RF051[min_max] -> hd1.hd_income_band_sk @@ -1408,7 +1408,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 40(GETNEXT) | | | | | F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.23MB mem-reservation=4.00MB thread-reservation=2 | | 40:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF047[min_max] -> tpcds_parquet.store_returns.sr_item_sk @@ -1426,7 +1426,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | in pipelines: 39(GETNEXT) | | | F29:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=133.93MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=135.46MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 61:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_sold_date_sk = d1.d_date_sk | | fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk @@ -1441,7 +1441,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 45(GETNEXT) | | | | | F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 45:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year = CAST(2000 AS INT) @@ -1469,7 +1469,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat | | | in pipelines: 59(GETNEXT) | | | | | F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.25MB mem-reservation=512.00KB thread-reservation=2 | | 59:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_color IN ('purple', 'burlywood', 'indian', 'spring', 'floral', 'medium'), i_current_price <= CAST(74 AS DECIMAL(3,0)), i_current_price <= CAST(79 AS DECIMAL(3,0)), i_current_price >= CAST(64 AS DECIMAL(3,0)), i_current_price >= CAST(65 AS DECIMAL(3,0)) @@ -1575,7 +1575,7 @@ Per-Host Resources: mem-estimate=48.81MB mem-reservation=42.69MB thread-reservat | | | in pipelines: 03(GETNEXT) | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=120.00MB mem-reservation=4.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=120.12MB mem-reservation=4.00MB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=10.62MB | | runtime filters: RF003[min_max] -> tpcds_parquet.catalog_returns.cr_item_sk @@ -1612,7 +1612,7 @@ Per-Host Resources: mem-estimate=48.81MB mem-reservation=42.69MB thread-reservat | | in pipelines: 19(GETNEXT) | | | F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=2 | 19:SCAN HDFS [tpcds_parquet.income_band ib2, RANDOM] | HDFS partitions=1/1 files=1 size=1.21KB | stored statistics: @@ -1637,7 +1637,7 @@ Per-Host Resources: mem-estimate=48.81MB mem-reservation=42.69MB thread-reservat | | in pipelines: 18(GETNEXT) | | | F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=2 | 18:SCAN HDFS [tpcds_parquet.income_band ib1, RANDOM] | HDFS partitions=1/1 files=1 size=1.21KB | stored statistics: @@ -1662,7 +1662,7 @@ Per-Host Resources: mem-estimate=48.81MB mem-reservation=42.69MB thread-reservat | | in pipelines: 17(GETNEXT) | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=80.96MB mem-reservation=1.00MB thread-reservation=2 | 17:SCAN HDFS [tpcds_parquet.customer_address ad2, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -1679,7 +1679,7 @@ Per-Host Resources: mem-estimate=48.81MB mem-reservation=42.69MB thread-reservat | in pipelines: 00(GETNEXT) | F20:PLAN FRAGMENT [HASH(ss_addr_sk)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=8.33MB mem-reservation=2.88MB thread-reservation=1 +Per-Host Resources: mem-estimate=9.28MB mem-reservation=2.88MB thread-reservation=1 33:HASH JOIN [INNER JOIN, PARTITIONED] | hash predicates: ss_addr_sk = ad1.ca_address_sk | fk/pk conjuncts: ss_addr_sk = ad1.ca_address_sk @@ -1694,7 +1694,7 @@ Per-Host Resources: mem-estimate=8.33MB mem-reservation=2.88MB thread-reservatio | | in pipelines: 16(GETNEXT) | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=80.96MB mem-reservation=1.00MB thread-reservation=2 | 16:SCAN HDFS [tpcds_parquet.customer_address ad1, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -1727,7 +1727,7 @@ Per-Host Resources: mem-estimate=45.59MB mem-reservation=34.00MB thread-reservat | | in pipelines: 12(GETNEXT) | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.25MB mem-reservation=8.00MB thread-reservation=2 | 12:SCAN HDFS [tpcds_parquet.customer_demographics cd2, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | stored statistics: @@ -1759,7 +1759,7 @@ Per-Host Resources: mem-estimate=45.48MB mem-reservation=34.00MB thread-reservat | | in pipelines: 11(GETNEXT) | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.25MB mem-reservation=8.00MB thread-reservation=2 | 11:SCAN HDFS [tpcds_parquet.customer_demographics cd1, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | stored statistics: @@ -1776,7 +1776,7 @@ Per-Host Resources: mem-estimate=45.48MB mem-reservation=34.00MB thread-reservat | in pipelines: 00(GETNEXT) | F14:PLAN FRAGMENT [HASH(c_first_shipto_date_sk)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=3.91MB mem-reservation=1.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=6.63MB mem-reservation=1.94MB thread-reservation=1 30:HASH JOIN [INNER JOIN, PARTITIONED] | hash predicates: c_first_shipto_date_sk = d3.d_date_sk | fk/pk conjuncts: c_first_shipto_date_sk = d3.d_date_sk @@ -1791,7 +1791,7 @@ Per-Host Resources: mem-estimate=3.91MB mem-reservation=1.94MB thread-reservatio | | in pipelines: 08(GETNEXT) | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.14MB mem-reservation=512.00KB thread-reservation=2 | 08:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -1808,7 +1808,7 @@ Per-Host Resources: mem-estimate=3.91MB mem-reservation=1.94MB thread-reservatio | in pipelines: 00(GETNEXT) | F12:PLAN FRAGMENT [HASH(c_first_sales_date_sk)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=3.85MB mem-reservation=1.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=6.43MB mem-reservation=1.94MB thread-reservation=1 29:HASH JOIN [INNER JOIN, PARTITIONED] | hash predicates: c_first_sales_date_sk = d2.d_date_sk | fk/pk conjuncts: c_first_sales_date_sk = d2.d_date_sk @@ -1823,7 +1823,7 @@ Per-Host Resources: mem-estimate=3.85MB mem-reservation=1.94MB thread-reservatio | | in pipelines: 07(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.14MB mem-reservation=512.00KB thread-reservation=2 | 07:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -1840,7 +1840,7 @@ Per-Host Resources: mem-estimate=3.85MB mem-reservation=1.94MB thread-reservatio | in pipelines: 00(GETNEXT) | F09:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3 -Per-Host Resources: mem-estimate=8.35MB mem-reservation=4.81MB thread-reservation=1 +Per-Host Resources: mem-estimate=8.97MB mem-reservation=4.81MB thread-reservation=1 28:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: c_current_hdemo_sk = hd2.hd_demo_sk | fk/pk conjuncts: c_current_hdemo_sk = hd2.hd_demo_sk @@ -1855,7 +1855,7 @@ Per-Host Resources: mem-estimate=8.35MB mem-reservation=4.81MB thread-reservatio | | in pipelines: 15(GETNEXT) | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=64.00KB thread-reservation=2 | 15:SCAN HDFS [tpcds_parquet.household_demographics hd2, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | runtime filters: RF009[min_max] -> hd2.hd_income_band_sk @@ -1881,7 +1881,7 @@ Per-Host Resources: mem-estimate=8.35MB mem-reservation=4.81MB thread-reservatio | | in pipelines: 10(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=96.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=96.33MB mem-reservation=2.00MB thread-reservation=2 | 10:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF025[min_max] -> c_current_hdemo_sk, RF013[min_max] -> c_current_addr_sk, RF017[min_max] -> c_current_cdemo_sk, RF021[min_max] -> c_first_shipto_date_sk, RF023[min_max] -> c_first_sales_date_sk @@ -1914,7 +1914,7 @@ Per-Host Resources: mem-estimate=17.94MB mem-reservation=12.56MB thread-reservat | | in pipelines: 09(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.16MB mem-reservation=24.00KB thread-reservation=2 | 09:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | runtime filters: RF004[min_max] -> tpcds_parquet.store.s_store_name, RF005[min_max] -> tpcds_parquet.store.s_zip @@ -1940,7 +1940,7 @@ Per-Host Resources: mem-estimate=17.94MB mem-reservation=12.56MB thread-reservat | | in pipelines: 13(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=32.00KB thread-reservation=2 | 13:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | stored statistics: @@ -1965,7 +1965,7 @@ Per-Host Resources: mem-estimate=17.94MB mem-reservation=12.56MB thread-reservat | | in pipelines: 14(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=64.00KB thread-reservation=2 | 14:SCAN HDFS [tpcds_parquet.household_demographics hd1, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | runtime filters: RF011[min_max] -> hd1.hd_income_band_sk @@ -1991,7 +1991,7 @@ Per-Host Resources: mem-estimate=17.94MB mem-reservation=12.56MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.23MB mem-reservation=4.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | runtime filters: RF003[min_max] -> tpcds_parquet.store_returns.sr_item_sk, RF007[min_max] -> tpcds_parquet.store_returns.sr_item_sk @@ -2009,7 +2009,7 @@ Per-Host Resources: mem-estimate=17.94MB mem-reservation=12.56MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=135.93MB mem-reservation=15.88MB thread-reservation=2 runtime-filters-memory=4.00MB +Per-Host Resources: mem-estimate=137.46MB mem-reservation=15.88MB thread-reservation=2 runtime-filters-memory=4.00MB 22:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_sold_date_sk = d1.d_date_sk | fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk @@ -2024,7 +2024,7 @@ Per-Host Resources: mem-estimate=135.93MB mem-reservation=15.88MB thread-reserva | | in pipelines: 06(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 06:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d1.d_year = CAST(1999 AS INT) @@ -2052,7 +2052,7 @@ Per-Host Resources: mem-estimate=135.93MB mem-reservation=15.88MB thread-reserva | | in pipelines: 20(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=65.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=65.25MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | 20:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_color IN ('purple', 'burlywood', 'indian', 'spring', 'floral', 'medium'), i_current_price <= CAST(74 AS DECIMAL(3,0)), i_current_price <= CAST(79 AS DECIMAL(3,0)), i_current_price >= CAST(64 AS DECIMAL(3,0)), i_current_price >= CAST(65 AS DECIMAL(3,0)) @@ -2080,7 +2080,7 @@ Per-Host Resources: mem-estimate=135.93MB mem-reservation=15.88MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=627.27MB Threads=135 -Per-Host Resource Estimates: Memory=1.69GB +Per-Host Resource Estimates: Memory=1.93GB F59:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=123.00MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -2138,7 +2138,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | in pipelines: 39(GETNEXT) | | | F51:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=20.30MB mem-reservation=17.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=27.11MB mem-reservation=17.00MB thread-reservation=1 | 77:AGGREGATE [STREAMING] | | output: count(*), sum(ss_wholesale_cost), sum(ss_list_price), sum(ss_coupon_amt) | | group by: i_product_name, i_item_sk, s_store_name, s_zip, ad1.ca_street_number, ad1.ca_street_name, ad1.ca_city, ad1.ca_zip, ad2.ca_street_number, ad2.ca_street_name, ad2.ca_city, ad2.ca_zip, d1.d_year, d2.d_year, d3.d_year @@ -2184,7 +2184,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | | | F54:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=58.00MB mem-reservation=10.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=58.52MB mem-reservation=10.00MB thread-reservation=1 | | 44:AGGREGATE [STREAMING] | | | output: sum(cs_ext_list_price), sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit) | | | group by: cs_item_sk @@ -2214,7 +2214,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | | in pipelines: 42(GETNEXT) | | | | | | | F55:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=24.12MB mem-reservation=4.00MB thread-reservation=1 | | | 42:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=10.62MB | | | stored statistics: @@ -2258,7 +2258,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 58(GETNEXT) | | | | | F53:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=1 | | 58:SCAN HDFS [tpcds_parquet.income_band ib2, RANDOM] | | HDFS partitions=1/1 files=1 size=1.21KB | | stored statistics: @@ -2291,7 +2291,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 57(GETNEXT) | | | | | F52:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=1 | | 57:SCAN HDFS [tpcds_parquet.income_band ib1, RANDOM] | | HDFS partitions=1/1 files=1 size=1.21KB | | stored statistics: @@ -2324,7 +2324,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 56(GETNEXT) | | | | | F50:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=17.91MB mem-reservation=1.00MB thread-reservation=1 | | 56:SCAN HDFS [tpcds_parquet.customer_address ad2, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -2341,7 +2341,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | in pipelines: 39(GETNEXT) | | | F49:PLAN FRAGMENT [HASH(ss_addr_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=2.62MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=12.09MB mem-reservation=0B thread-reservation=1 | 72:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=06 | | hash predicates: ss_addr_sk = ad1.ca_address_sk @@ -2364,7 +2364,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 55(GETNEXT) | | | | | F48:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=17.91MB mem-reservation=1.00MB thread-reservation=1 | | 55:SCAN HDFS [tpcds_parquet.customer_address ad1, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | stored statistics: @@ -2381,7 +2381,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | in pipelines: 39(GETNEXT) | | | F47:PLAN FRAGMENT [HASH(c_current_cdemo_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=2.45MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.01MB mem-reservation=0B thread-reservation=1 | 71:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=07 | | hash predicates: c_current_cdemo_sk = cd2.cd_demo_sk @@ -2405,7 +2405,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 51(GETNEXT) | | | | | F46:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.49MB mem-reservation=8.00MB thread-reservation=1 | | 51:SCAN HDFS [tpcds_parquet.customer_demographics cd2, RANDOM] | | HDFS partitions=1/1 files=1 size=7.49MB | | stored statistics: @@ -2422,7 +2422,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | in pipelines: 39(GETNEXT) | | | F45:PLAN FRAGMENT [HASH(ss_cdemo_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=2.28MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=9.35MB mem-reservation=0B thread-reservation=1 | 70:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=08 | | hash predicates: ss_cdemo_sk = cd1.cd_demo_sk @@ -2445,7 +2445,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 50(GETNEXT) | | | | | F44:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.49MB mem-reservation=8.00MB thread-reservation=1 | | 50:SCAN HDFS [tpcds_parquet.customer_demographics cd1, RANDOM] | | HDFS partitions=1/1 files=1 size=7.49MB | | stored statistics: @@ -2462,7 +2462,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | in pipelines: 39(GETNEXT) | | | F43:PLAN FRAGMENT [HASH(c_first_shipto_date_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=2.19MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=8.77MB mem-reservation=0B thread-reservation=1 | 69:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=09 | | hash predicates: c_first_shipto_date_sk = d3.d_date_sk @@ -2485,7 +2485,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 47(GETNEXT) | | | | | F42:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.28MB mem-reservation=512.00KB thread-reservation=1 | | 47:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | stored statistics: @@ -2502,7 +2502,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | in pipelines: 39(GETNEXT) | | | F41:PLAN FRAGMENT [HASH(c_first_sales_date_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=2.10MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=8.39MB mem-reservation=0B thread-reservation=1 | 68:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=10 | | hash predicates: c_first_sales_date_sk = d2.d_date_sk @@ -2525,7 +2525,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 46(GETNEXT) | | | | | F40:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.28MB mem-reservation=512.00KB thread-reservation=1 | | 46:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | stored statistics: @@ -2542,7 +2542,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | in pipelines: 39(GETNEXT) | | | F38:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=1.79MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=7.80MB mem-reservation=0B thread-reservation=1 | 67:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=11 | | hash predicates: c_current_hdemo_sk = hd2.hd_demo_sk @@ -2565,7 +2565,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 54(GETNEXT) | | | | | F39:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=64.00KB thread-reservation=1 | | 54:SCAN HDFS [tpcds_parquet.household_demographics hd2, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | runtime filters: RF049[min_max] -> hd2.hd_income_band_sk @@ -2599,7 +2599,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 49(GETNEXT) | | | | | F37:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.66MB mem-reservation=2.00MB thread-reservation=1 | | 49:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF065[min_max] -> c_current_hdemo_sk, RF053[min_max] -> c_current_addr_sk, RF057[min_max] -> c_current_cdemo_sk, RF061[min_max] -> c_first_shipto_date_sk, RF063[min_max] -> c_first_sales_date_sk @@ -2617,7 +2617,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | in pipelines: 39(GETNEXT) | | | F33:PLAN FRAGMENT [HASH(ss_item_sk,ss_ticket_number)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=1.27MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=6.35MB mem-reservation=0B thread-reservation=1 | 65:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=13 | | hash predicates: ss_store_sk = s_store_sk @@ -2640,7 +2640,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 48(GETNEXT) | | | | | F36:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.16MB mem-reservation=24.00KB thread-reservation=1 | | 48:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -2673,7 +2673,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 52(GETNEXT) | | | | | F35:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=32.00KB thread-reservation=1 | | 52:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | | HDFS partitions=1/1 files=1 size=23.30KB | | stored statistics: @@ -2706,7 +2706,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 53(GETNEXT) | | | | | F34:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=64.00KB thread-reservation=1 | | 53:SCAN HDFS [tpcds_parquet.household_demographics hd1, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | runtime filters: RF051[min_max] -> hd1.hd_income_band_sk @@ -2740,7 +2740,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 40(GETNEXT) | | | | | F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.47MB mem-reservation=4.00MB thread-reservation=1 | | 40:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | runtime filters: RF047[min_max] -> tpcds_parquet.store_returns.sr_item_sk @@ -2759,7 +2759,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | F29:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=19.17MB mem-reservation=8.00MB thread-reservation=1 | 61:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=17 | | hash predicates: ss_sold_date_sk = d1.d_date_sk @@ -2782,7 +2782,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 45(GETNEXT) | | | | | F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 45:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year = CAST(2000 AS INT) @@ -2818,7 +2818,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese | | | in pipelines: 59(GETNEXT) | | | | | F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.25MB mem-reservation=512.00KB thread-reservation=1 | | 59:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_color IN ('purple', 'burlywood', 'indian', 'spring', 'floral', 'medium'), i_current_price <= CAST(74 AS DECIMAL(3,0)), i_current_price <= CAST(79 AS DECIMAL(3,0)), i_current_price >= CAST(64 AS DECIMAL(3,0)), i_current_price >= CAST(65 AS DECIMAL(3,0)) @@ -2865,7 +2865,7 @@ Per-Instance Resources: mem-estimate=28.70MB mem-reservation=17.00MB thread-rese | in pipelines: 00(GETNEXT) | F22:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=20.30MB mem-reservation=17.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=27.11MB mem-reservation=17.00MB thread-reservation=1 38:AGGREGATE [STREAMING] | output: count(*), sum(ss_wholesale_cost), sum(ss_list_price), sum(ss_coupon_amt) | group by: i_product_name, i_item_sk, s_store_name, s_zip, ad1.ca_street_number, ad1.ca_street_name, ad1.ca_city, ad1.ca_zip, ad2.ca_street_number, ad2.ca_street_name, ad2.ca_city, ad2.ca_zip, d1.d_year, d2.d_year, d3.d_year @@ -2911,7 +2911,7 @@ Per-Instance Resources: mem-estimate=20.30MB mem-reservation=17.00MB thread-rese | | | F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=58.00MB mem-reservation=10.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=58.52MB mem-reservation=10.00MB thread-reservation=1 | 05:AGGREGATE [STREAMING] | | output: sum(cs_ext_list_price), sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit) | | group by: cs_item_sk @@ -2941,7 +2941,7 @@ Per-Instance Resources: mem-estimate=20.30MB mem-reservation=17.00MB thread-rese | | | in pipelines: 03(GETNEXT) | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.12MB mem-reservation=4.00MB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=10.62MB | | runtime filters: RF003[min_max] -> tpcds_parquet.catalog_returns.cr_item_sk @@ -2986,7 +2986,7 @@ Per-Instance Resources: mem-estimate=20.30MB mem-reservation=17.00MB thread-rese | | in pipelines: 19(GETNEXT) | | | F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=1 | 19:SCAN HDFS [tpcds_parquet.income_band ib2, RANDOM] | HDFS partitions=1/1 files=1 size=1.21KB | stored statistics: @@ -3019,7 +3019,7 @@ Per-Instance Resources: mem-estimate=20.30MB mem-reservation=17.00MB thread-rese | | in pipelines: 18(GETNEXT) | | | F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=1 | 18:SCAN HDFS [tpcds_parquet.income_band ib1, RANDOM] | HDFS partitions=1/1 files=1 size=1.21KB | stored statistics: @@ -3052,7 +3052,7 @@ Per-Instance Resources: mem-estimate=20.30MB mem-reservation=17.00MB thread-rese | | in pipelines: 17(GETNEXT) | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.91MB mem-reservation=1.00MB thread-reservation=1 | 17:SCAN HDFS [tpcds_parquet.customer_address ad2, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -3069,7 +3069,7 @@ Per-Instance Resources: mem-estimate=20.30MB mem-reservation=17.00MB thread-rese | in pipelines: 00(GETNEXT) | F20:PLAN FRAGMENT [HASH(ss_addr_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=2.62MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=12.09MB mem-reservation=0B thread-reservation=1 33:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=24 | hash predicates: ss_addr_sk = ad1.ca_address_sk @@ -3092,7 +3092,7 @@ Per-Instance Resources: mem-estimate=2.62MB mem-reservation=0B thread-reservatio | | in pipelines: 16(GETNEXT) | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.91MB mem-reservation=1.00MB thread-reservation=1 | 16:SCAN HDFS [tpcds_parquet.customer_address ad1, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -3109,7 +3109,7 @@ Per-Instance Resources: mem-estimate=2.62MB mem-reservation=0B thread-reservatio | in pipelines: 00(GETNEXT) | F18:PLAN FRAGMENT [HASH(c_current_cdemo_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=2.45MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=10.01MB mem-reservation=0B thread-reservation=1 32:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=25 | hash predicates: c_current_cdemo_sk = cd2.cd_demo_sk @@ -3133,7 +3133,7 @@ Per-Instance Resources: mem-estimate=2.45MB mem-reservation=0B thread-reservatio | | in pipelines: 12(GETNEXT) | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.49MB mem-reservation=8.00MB thread-reservation=1 | 12:SCAN HDFS [tpcds_parquet.customer_demographics cd2, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | stored statistics: @@ -3150,7 +3150,7 @@ Per-Instance Resources: mem-estimate=2.45MB mem-reservation=0B thread-reservatio | in pipelines: 00(GETNEXT) | F16:PLAN FRAGMENT [HASH(ss_cdemo_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=2.28MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=9.35MB mem-reservation=0B thread-reservation=1 31:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=26 | hash predicates: ss_cdemo_sk = cd1.cd_demo_sk @@ -3173,7 +3173,7 @@ Per-Instance Resources: mem-estimate=2.28MB mem-reservation=0B thread-reservatio | | in pipelines: 11(GETNEXT) | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.49MB mem-reservation=8.00MB thread-reservation=1 | 11:SCAN HDFS [tpcds_parquet.customer_demographics cd1, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | stored statistics: @@ -3190,7 +3190,7 @@ Per-Instance Resources: mem-estimate=2.28MB mem-reservation=0B thread-reservatio | in pipelines: 00(GETNEXT) | F14:PLAN FRAGMENT [HASH(c_first_shipto_date_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=2.19MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=8.77MB mem-reservation=0B thread-reservation=1 30:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=27 | hash predicates: c_first_shipto_date_sk = d3.d_date_sk @@ -3213,7 +3213,7 @@ Per-Instance Resources: mem-estimate=2.19MB mem-reservation=0B thread-reservatio | | in pipelines: 08(GETNEXT) | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.28MB mem-reservation=512.00KB thread-reservation=1 | 08:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -3230,7 +3230,7 @@ Per-Instance Resources: mem-estimate=2.19MB mem-reservation=0B thread-reservatio | in pipelines: 00(GETNEXT) | F12:PLAN FRAGMENT [HASH(c_first_sales_date_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=2.10MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=8.39MB mem-reservation=0B thread-reservation=1 29:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=28 | hash predicates: c_first_sales_date_sk = d2.d_date_sk @@ -3253,7 +3253,7 @@ Per-Instance Resources: mem-estimate=2.10MB mem-reservation=0B thread-reservatio | | in pipelines: 07(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.28MB mem-reservation=512.00KB thread-reservation=1 | 07:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -3270,7 +3270,7 @@ Per-Instance Resources: mem-estimate=2.10MB mem-reservation=0B thread-reservatio | in pipelines: 00(GETNEXT) | F09:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=1.79MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=7.80MB mem-reservation=0B thread-reservation=1 28:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=29 | hash predicates: c_current_hdemo_sk = hd2.hd_demo_sk @@ -3293,7 +3293,7 @@ Per-Instance Resources: mem-estimate=1.79MB mem-reservation=0B thread-reservatio | | in pipelines: 15(GETNEXT) | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=64.00KB thread-reservation=1 | 15:SCAN HDFS [tpcds_parquet.household_demographics hd2, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | runtime filters: RF009[min_max] -> hd2.hd_income_band_sk @@ -3327,7 +3327,7 @@ Per-Instance Resources: mem-estimate=1.79MB mem-reservation=0B thread-reservatio | | in pipelines: 10(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.66MB mem-reservation=2.00MB thread-reservation=1 | 10:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF025[min_max] -> c_current_hdemo_sk, RF013[min_max] -> c_current_addr_sk, RF017[min_max] -> c_current_cdemo_sk, RF021[min_max] -> c_first_shipto_date_sk, RF023[min_max] -> c_first_sales_date_sk @@ -3345,7 +3345,7 @@ Per-Instance Resources: mem-estimate=1.79MB mem-reservation=0B thread-reservatio | in pipelines: 00(GETNEXT) | F04:PLAN FRAGMENT [HASH(ss_item_sk,ss_ticket_number)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=1.27MB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=6.35MB mem-reservation=0B thread-reservation=1 26:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=31 | hash predicates: ss_store_sk = s_store_sk @@ -3368,7 +3368,7 @@ Per-Instance Resources: mem-estimate=1.27MB mem-reservation=0B thread-reservatio | | in pipelines: 09(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.16MB mem-reservation=24.00KB thread-reservation=1 | 09:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | runtime filters: RF004[min_max] -> tpcds_parquet.store.s_store_name, RF005[min_max] -> tpcds_parquet.store.s_zip @@ -3402,7 +3402,7 @@ Per-Instance Resources: mem-estimate=1.27MB mem-reservation=0B thread-reservatio | | in pipelines: 13(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=32.00KB thread-reservation=1 | 13:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | stored statistics: @@ -3435,7 +3435,7 @@ Per-Instance Resources: mem-estimate=1.27MB mem-reservation=0B thread-reservatio | | in pipelines: 14(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=64.00KB thread-reservation=1 | 14:SCAN HDFS [tpcds_parquet.household_demographics hd1, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | runtime filters: RF011[min_max] -> hd1.hd_income_band_sk @@ -3469,7 +3469,7 @@ Per-Instance Resources: mem-estimate=1.27MB mem-reservation=0B thread-reservatio | | in pipelines: 01(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.47MB mem-reservation=4.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | runtime filters: RF003[min_max] -> tpcds_parquet.store_returns.sr_item_sk, RF007[min_max] -> tpcds_parquet.store_returns.sr_item_sk @@ -3488,7 +3488,7 @@ Per-Instance Resources: mem-estimate=1.27MB mem-reservation=0B thread-reservatio | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=19.17MB mem-reservation=8.00MB thread-reservation=1 22:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=35 | hash predicates: ss_sold_date_sk = d1.d_date_sk @@ -3511,7 +3511,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 06(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 06:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d1.d_year = CAST(1999 AS INT) @@ -3548,7 +3548,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.25MB mem-reservation=512.00KB thread-reservation=1 | 20:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_color IN ('purple', 'burlywood', 'indian', 'spring', 'floral', 'medium'), i_current_price <= CAST(74 AS DECIMAL(3,0)), i_current_price <= CAST(79 AS DECIMAL(3,0)), i_current_price >= CAST(64 AS DECIMAL(3,0)), i_current_price >= CAST(65 AS DECIMAL(3,0)) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q65.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q65.test index 7b2d8a0e7..906e9a568 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q65.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q65.test @@ -202,7 +202,7 @@ PLAN-ROOT SINK in pipelines: 07(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=56.33MB Threads=17 -Per-Host Resource Estimates: Memory=363MB +Per-Host Resource Estimates: Memory=365MB F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -239,7 +239,7 @@ Per-Host Resources: mem-estimate=11.49MB mem-reservation=6.81MB thread-reservati | | in pipelines: 25(GETNEXT) | | | F09:PLAN FRAGMENT [HASH(ss_store_sk)] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.09MB mem-reservation=1.94MB thread-reservation=1 | 25:AGGREGATE [FINALIZE] | | output: avg:merge(revenue) | | group by: ss_store_sk @@ -274,7 +274,7 @@ Per-Host Resources: mem-estimate=11.49MB mem-reservation=6.81MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=61.00MB mem-reservation=12.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=61.31MB mem-reservation=12.94MB thread-reservation=2 runtime-filters-memory=1.00MB | 05:AGGREGATE [STREAMING] | | output: sum(ss_sales_price) | | group by: ss_store_sk, ss_item_sk @@ -296,7 +296,7 @@ Per-Host Resources: mem-estimate=11.49MB mem-reservation=6.81MB thread-reservati | | | in pipelines: 03(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1223 AS INT), d_month_seq >= CAST(1212 AS INT) @@ -336,7 +336,7 @@ Per-Host Resources: mem-estimate=11.49MB mem-reservation=6.81MB thread-reservati | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=81.88MB mem-reservation=2.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -368,7 +368,7 @@ Per-Host Resources: mem-estimate=14.01MB mem-reservation=5.81MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.09MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB | 00:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | runtime filters: RF001[min_max] -> tpcds_parquet.store.s_store_sk, RF000[bloom] -> tpcds_parquet.store.s_store_sk @@ -393,7 +393,7 @@ Per-Host Resources: mem-estimate=14.01MB mem-reservation=5.81MB thread-reservati | in pipelines: 07(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=64.00MB mem-reservation=15.94MB thread-reservation=2 runtime-filters-memory=4.00MB +Per-Host Resources: mem-estimate=64.31MB mem-reservation=15.94MB thread-reservation=2 runtime-filters-memory=4.00MB 10:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: ss_store_sk, ss_item_sk @@ -415,7 +415,7 @@ Per-Host Resources: mem-estimate=64.00MB mem-reservation=15.94MB thread-reservat | | in pipelines: 08(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1223 AS INT), d_month_seq >= CAST(1212 AS INT) @@ -442,7 +442,7 @@ Per-Host Resources: mem-estimate=64.00MB mem-reservation=15.94MB thread-reservat in pipelines: 07(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=94.02MB Threads=23 -Per-Host Resource Estimates: Memory=298MB +Per-Host Resource Estimates: Memory=305MB F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.11MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -487,7 +487,7 @@ Per-Instance Resources: mem-estimate=2.00MB mem-reservation=0B thread-reservatio | | in pipelines: 25(GETNEXT) | | | F09:PLAN FRAGMENT [HASH(ss_store_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.09MB mem-reservation=1.94MB thread-reservation=1 | 25:AGGREGATE [FINALIZE] | | output: avg:merge(revenue) | | group by: ss_store_sk @@ -523,7 +523,7 @@ Per-Instance Resources: mem-estimate=2.00MB mem-reservation=0B thread-reservatio | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.75MB mem-reservation=10.00MB thread-reservation=1 | 05:AGGREGATE [STREAMING] | | output: sum(ss_sales_price) | | group by: ss_store_sk, ss_item_sk @@ -553,7 +553,7 @@ Per-Instance Resources: mem-estimate=2.00MB mem-reservation=0B thread-reservatio | | | in pipelines: 03(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1223 AS INT), d_month_seq >= CAST(1212 AS INT) @@ -601,7 +601,7 @@ Per-Instance Resources: mem-estimate=2.00MB mem-reservation=0B thread-reservatio | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=19.76MB mem-reservation=2.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -618,7 +618,7 @@ Per-Instance Resources: mem-estimate=2.00MB mem-reservation=0B thread-reservatio | in pipelines: 17(GETNEXT) | F02:PLAN FRAGMENT [HASH(ss_store_sk,ss_item_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=11.15MB mem-reservation=1.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=11.32MB mem-reservation=1.94MB thread-reservation=1 11:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=03 | hash predicates: ss_store_sk = s_store_sk @@ -642,7 +642,7 @@ Per-Instance Resources: mem-estimate=11.15MB mem-reservation=1.94MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | runtime filters: RF001[min_max] -> tpcds_parquet.store.s_store_sk, RF000[bloom] -> tpcds_parquet.store.s_store_sk @@ -668,7 +668,7 @@ Per-Instance Resources: mem-estimate=11.15MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.75MB mem-reservation=10.00MB thread-reservation=1 10:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: ss_store_sk, ss_item_sk @@ -698,7 +698,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese | | in pipelines: 08(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1223 AS INT), d_month_seq >= CAST(1212 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test index d9dd1372e..aa33a4c02 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test @@ -321,7 +321,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=55.25MB Threads=30 -Per-Host Resource Estimates: Memory=611MB +Per-Host Resource Estimates: Memory=614MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=10.00MB mem-reservation=10.00MB thread-reservation=1 PLAN-ROOT SINK @@ -345,7 +345,7 @@ PLAN-ROOT SINK | | in pipelines: 44(GETNEXT) | | | F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 44:AGGREGATE [FINALIZE] | | output: sum:merge(ss_ext_sales_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -379,7 +379,7 @@ PLAN-ROOT SINK | | | in pipelines: 18(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | | 18:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -407,7 +407,7 @@ PLAN-ROOT SINK | | | in pipelines: 17(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.14MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 17:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF013[min_max] -> c_current_addr_sk, RF012[bloom] -> c_current_addr_sk @@ -425,7 +425,7 @@ PLAN-ROOT SINK | | in pipelines: 14(GETNEXT) | | | F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=72.92MB mem-reservation=10.81MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=73.82MB mem-reservation=10.81MB thread-reservation=2 runtime-filters-memory=3.00MB | 22:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_store_sk = s_store_sk | | fk/pk conjuncts: ss_store_sk = s_store_sk @@ -440,7 +440,7 @@ PLAN-ROOT SINK | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=16.00KB thread-reservation=2 | | 15:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: s_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -468,7 +468,7 @@ PLAN-ROOT SINK | | | in pipelines: 19(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=256.00KB thread-reservation=2 | | 19:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Books' @@ -496,7 +496,7 @@ PLAN-ROOT SINK | | | in pipelines: 16(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 16:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(11 AS INT) @@ -555,7 +555,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -583,7 +583,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio | | in pipelines: 04(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.14MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 04:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> c_current_addr_sk, RF000[bloom] -> c_current_addr_sk @@ -601,7 +601,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=94.35MB mem-reservation=14.75MB thread-reservation=2 runtime-filters-memory=5.00MB 10:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_store_sk = s_store_sk | fk/pk conjuncts: ss_store_sk = s_store_sk @@ -616,7 +616,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=16.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -644,7 +644,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=32.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.18MB mem-reservation=32.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | predicates: (p_channel_dmail = 'Y' OR p_channel_email = 'Y' OR p_channel_tv = 'Y') @@ -670,7 +670,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat | | in pipelines: 06(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=256.00KB thread-reservation=2 | 06:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category = 'Books' @@ -698,7 +698,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(11 AS INT) @@ -725,7 +725,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=86.69MB Threads=35 -Per-Host Resource Estimates: Memory=323MB +Per-Host Resource Estimates: Memory=335MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=10.00MB mem-reservation=10.00MB thread-reservation=1 PLAN-ROOT SINK @@ -756,7 +756,7 @@ PLAN-ROOT SINK | | in pipelines: 44(GETNEXT) | | | F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1 | 44:AGGREGATE [FINALIZE] | | output: sum:merge(ss_ext_sales_price) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -798,7 +798,7 @@ PLAN-ROOT SINK | | | in pipelines: 18(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | | 18:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -835,7 +835,7 @@ PLAN-ROOT SINK | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.28MB mem-reservation=1.00MB thread-reservation=1 | | 17:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF013[min_max] -> c_current_addr_sk, RF012[bloom] -> c_current_addr_sk @@ -854,7 +854,7 @@ PLAN-ROOT SINK | | | F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=18.01MB mem-reservation=2.00MB thread-reservation=1 | 22:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=03 | | hash predicates: ss_store_sk = s_store_sk @@ -877,7 +877,7 @@ PLAN-ROOT SINK | | | in pipelines: 15(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=16.00KB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: s_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -913,7 +913,7 @@ PLAN-ROOT SINK | | | in pipelines: 19(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=256.00KB thread-reservation=1 | | 19:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Books' @@ -949,7 +949,7 @@ PLAN-ROOT SINK | | | in pipelines: 16(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 16:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(11 AS INT) @@ -1016,7 +1016,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1053,7 +1053,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.28MB mem-reservation=1.00MB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> c_current_addr_sk, RF000[bloom] -> c_current_addr_sk @@ -1072,7 +1072,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=19.21MB mem-reservation=2.00MB thread-reservation=1 10:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=08 | hash predicates: ss_store_sk = s_store_sk @@ -1095,7 +1095,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=16.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s_gmt_offset = CAST(-5 AS DECIMAL(3,0)) @@ -1131,7 +1131,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.18MB mem-reservation=32.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | predicates: (p_channel_dmail = 'Y' OR p_channel_email = 'Y' OR p_channel_tv = 'Y') @@ -1165,7 +1165,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 06(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=256.00KB thread-reservation=1 | 06:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category = 'Books' @@ -1201,7 +1201,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2000 AS INT), d_moy = CAST(11 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test index fbb83d403..4697fdd99 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test @@ -187,7 +187,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=478.39MB Threads=11 -Per-Host Resource Estimates: Memory=31.92GB +Per-Host Resource Estimates: Memory=31.93GB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -291,7 +291,7 @@ Per-Host Resources: mem-estimate=27.19GB mem-reservation=244.75MB thread-reserva | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=4.53GB mem-reservation=223.62MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=4.54GB mem-reservation=223.62MB thread-reservation=2 runtime-filters-memory=3.00MB 07:AGGREGATE [STREAMING] | Class 0 | output: sum(coalesce(ss_sales_price * CAST(ss_quantity AS DECIMAL(10,0)), CAST(0 AS DECIMAL(18,2)))) @@ -338,7 +338,7 @@ Per-Host Resources: mem-estimate=4.53GB mem-reservation=223.62MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=80.42MB mem-reservation=1.00MB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -363,7 +363,7 @@ Per-Host Resources: mem-estimate=4.53GB mem-reservation=223.62MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.14MB mem-reservation=16.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -388,7 +388,7 @@ Per-Host Resources: mem-estimate=4.53GB mem-reservation=223.62MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=80.09MB mem-reservation=1.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1223 AS INT), d_month_seq >= CAST(1212 AS INT) @@ -415,7 +415,7 @@ Per-Host Resources: mem-estimate=4.53GB mem-reservation=223.62MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=950.77MB Threads=13 -Per-Host Resource Estimates: Memory=32.56GB +Per-Host Resource Estimates: Memory=32.62GB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -520,7 +520,7 @@ Per-Instance Resources: mem-estimate=13.96GB mem-reservation=244.75MB thread-res | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=2.28GB mem-reservation=212.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=2.30GB mem-reservation=212.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | Class 0 | output: sum(coalesce(ss_sales_price * CAST(ss_quantity AS DECIMAL(10,0)), CAST(0 AS DECIMAL(18,2)))) @@ -575,7 +575,7 @@ Per-Instance Resources: mem-estimate=2.28GB mem-reservation=212.00MB thread-rese | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.42MB mem-reservation=1.00MB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -608,7 +608,7 @@ Per-Instance Resources: mem-estimate=2.28GB mem-reservation=212.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -641,7 +641,7 @@ Per-Instance Resources: mem-estimate=2.28GB mem-reservation=212.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=1.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1223 AS INT), d_month_seq >= CAST(1212 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test index e3662e3ec..3adaaa878 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test @@ -199,7 +199,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=111.89MB Threads=16 -Per-Host Resource Estimates: Memory=489MB +Per-Host Resource Estimates: Memory=490MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -236,7 +236,7 @@ Per-Host Resources: mem-estimate=64.97MB mem-reservation=49.25MB thread-reservat | | in pipelines: 11(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.11MB mem-reservation=512.00KB thread-reservation=2 | 11:SCAN HDFS [tpcds_parquet.customer_address current_addr, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -261,7 +261,7 @@ Per-Host Resources: mem-estimate=64.97MB mem-reservation=49.25MB thread-reservat | | in pipelines: 10(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=65.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=65.19MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 10:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> customer.c_current_addr_sk, RF000[bloom] -> customer.c_current_addr_sk @@ -308,7 +308,7 @@ Per-Host Resources: mem-estimate=179.04MB mem-reservation=53.56MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.11MB mem-reservation=512.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -333,7 +333,7 @@ Per-Host Resources: mem-estimate=179.04MB mem-reservation=53.56MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.10MB mem-reservation=16.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_city IN ('Midway', 'Fairview') @@ -361,7 +361,7 @@ Per-Host Resources: mem-estimate=179.04MB mem-reservation=53.56MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_date <= '1999-03-31', d_date >= '1999-01-01' @@ -389,7 +389,7 @@ Per-Host Resources: mem-estimate=179.04MB mem-reservation=53.56MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=64.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: (household_demographics.hd_dep_count = CAST(5 AS INT) OR household_demographics.hd_vehicle_count = CAST(3 AS INT)) @@ -414,7 +414,7 @@ Per-Host Resources: mem-estimate=179.04MB mem-reservation=53.56MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=143.70MB Threads=17 -Per-Host Resource Estimates: Memory=288MB +Per-Host Resource Estimates: Memory=293MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.08MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -459,7 +459,7 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese | | in pipelines: 11(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.11MB mem-reservation=512.00KB thread-reservation=1 | 11:SCAN HDFS [tpcds_parquet.customer_address current_addr, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -493,7 +493,7 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.19MB mem-reservation=2.00MB thread-reservation=1 | 10:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF001[min_max] -> customer.c_current_addr_sk, RF000[bloom] -> customer.c_current_addr_sk @@ -519,7 +519,7 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=33.00MB mem-reservation=21.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=35.08MB mem-reservation=21.00MB thread-reservation=1 09:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price), sum(ss_ext_list_price), sum(ss_ext_tax) | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city @@ -549,7 +549,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=21.00MB thread-rese | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.11MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | HDFS partitions=1/1 files=1 size=1.16MB | stored statistics: @@ -582,7 +582,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=21.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.10MB mem-reservation=16.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_city IN ('Midway', 'Fairview') @@ -618,7 +618,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=21.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_date <= '1999-03-31', d_date >= '1999-01-01' @@ -654,7 +654,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=21.00MB thread-rese | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=64.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: (household_demographics.hd_dep_count = CAST(5 AS INT) OR household_demographics.hd_vehicle_count = CAST(3 AS INT)) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test index 78a773d46..f38e661d2 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test @@ -241,7 +241,7 @@ PLAN-ROOT SINK in pipelines: 09(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=58.25MB Threads=21 -Per-Host Resource Estimates: Memory=554MB +Per-Host Resource Estimates: Memory=555MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -302,7 +302,7 @@ Per-Host Resources: mem-estimate=18.75MB mem-reservation=8.81MB thread-reservati | | | in pipelines: 06(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=66.95MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=67.27MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 08:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: ws_sold_date_sk = d_date_sk | | | fk/pk conjuncts: ws_sold_date_sk = d_date_sk @@ -317,7 +317,7 @@ Per-Host Resources: mem-estimate=18.75MB mem-reservation=8.81MB thread-reservati | | | | in pipelines: 07(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2001 AS INT), d_moy <= CAST(6 AS INT), d_moy >= CAST(4 AS INT) @@ -355,7 +355,7 @@ Per-Host Resources: mem-estimate=18.75MB mem-reservation=8.81MB thread-reservati | | | in pipelines: 02(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=109.67MB mem-reservation=20.44MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=109.80MB mem-reservation=20.44MB thread-reservation=2 runtime-filters-memory=2.00MB | | 13:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: c.c_current_addr_sk = ca.ca_address_sk | | | fk/pk conjuncts: c.c_current_addr_sk = ca.ca_address_sk @@ -370,7 +370,7 @@ Per-Host Resources: mem-estimate=18.75MB mem-reservation=8.81MB thread-reservati | | | | in pipelines: 01(GETNEXT) | | | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.09MB mem-reservation=256.00KB thread-reservation=2 | | | 01:SCAN HDFS [tpcds_parquet.customer_address ca, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.16MB | | | predicates: ca_state IN ('KY', 'GA', 'NM') @@ -398,7 +398,7 @@ Per-Host Resources: mem-estimate=18.75MB mem-reservation=8.81MB thread-reservati | | | | in pipelines: 00(GETNEXT) | | | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=49.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | | Per-Host Resources: mem-estimate=49.06MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | | 00:SCAN HDFS [tpcds_parquet.customer c, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF007[min_max] -> c.c_current_addr_sk, RF006[bloom] -> c.c_current_addr_sk @@ -427,7 +427,7 @@ Per-Host Resources: mem-estimate=18.75MB mem-reservation=8.81MB thread-reservati | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=19.95MB mem-reservation=4.44MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=20.27MB mem-reservation=4.44MB thread-reservation=2 runtime-filters-memory=2.00MB | 05:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_sold_date_sk = d_date_sk | | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -442,7 +442,7 @@ Per-Host Resources: mem-estimate=18.75MB mem-reservation=8.81MB thread-reservati | | | in pipelines: 04(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy <= CAST(6 AS INT), d_moy >= CAST(4 AS INT) @@ -474,7 +474,7 @@ Per-Host Resources: mem-estimate=18.75MB mem-reservation=8.81MB thread-reservati | in pipelines: 09(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=98.95MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=99.27MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB 11:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: cs_sold_date_sk = d_date_sk | fk/pk conjuncts: cs_sold_date_sk = d_date_sk @@ -489,7 +489,7 @@ Per-Host Resources: mem-estimate=98.95MB mem-reservation=6.94MB thread-reservati | | in pipelines: 10(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_moy <= CAST(6 AS INT), d_moy >= CAST(4 AS INT) @@ -515,7 +515,7 @@ Per-Host Resources: mem-estimate=98.95MB mem-reservation=6.94MB thread-reservati in pipelines: 09(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=79.00MB Threads=21 -Per-Host Resource Estimates: Memory=287MB +Per-Host Resource Estimates: Memory=291MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -550,7 +550,7 @@ Per-Instance Resources: mem-estimate=10.38MB mem-reservation=1.94MB thread-reser | in pipelines: 09(GETNEXT) | F07:PLAN FRAGMENT [HASH(c.c_customer_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=11.64MB mem-reservation=2.00MB thread-reservation=1 17:AGGREGATE [STREAMING] | output: count(*) | group by: cd_gender, cd_marital_status, cd_education_status, cd_purchase_estimate, cd_credit_rating @@ -593,7 +593,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=32.33MB mem-reservation=4.00MB thread-reservation=1 | | 08:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=02 | | | hash predicates: ws_sold_date_sk = d_date_sk @@ -616,7 +616,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser | | | | in pipelines: 07(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2001 AS INT), d_moy <= CAST(6 AS INT), d_moy >= CAST(4 AS INT) @@ -663,7 +663,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=17.37MB mem-reservation=8.00MB thread-reservation=1 | | 13:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=04 | | | hash predicates: c.c_current_addr_sk = ca.ca_address_sk @@ -686,7 +686,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser | | | | in pipelines: 01(GETNEXT) | | | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | 01:SCAN HDFS [tpcds_parquet.customer_address ca, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.16MB | | | predicates: ca_state IN ('KY', 'GA', 'NM') @@ -723,7 +723,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser | | | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=2.00MB thread-reservation=1 | | | 00:SCAN HDFS [tpcds_parquet.customer c, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.49MB | | | runtime filters: RF007[min_max] -> c.c_current_addr_sk, RF006[bloom] -> c.c_current_addr_sk @@ -753,7 +753,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.33MB mem-reservation=512.00KB thread-reservation=1 | 05:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=06 | | hash predicates: ss_sold_date_sk = d_date_sk @@ -776,7 +776,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser | | | in pipelines: 04(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2001 AS INT), d_moy <= CAST(6 AS INT), d_moy >= CAST(4 AS INT) @@ -809,7 +809,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=48.33MB mem-reservation=4.00MB thread-reservation=1 11:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=07 | hash predicates: cs_sold_date_sk = d_date_sk @@ -832,7 +832,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 10(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(2001 AS INT), d_moy <= CAST(6 AS INT), d_moy >= CAST(4 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test index 8ee4acfa1..08f6a6b3e 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test @@ -229,7 +229,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=72.41MB Threads=16 -Per-Host Resource Estimates: Memory=350MB +Per-Host Resource Estimates: Memory=352MB F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -300,7 +300,7 @@ Per-Host Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=70.91MB mem-reservation=15.81MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=72.61MB mem-reservation=15.81MB thread-reservation=2 runtime-filters-memory=3.00MB 15:AGGREGATE [STREAMING] | Class 0 | output: sum(ss_net_profit) @@ -366,7 +366,7 @@ Per-Host Resources: mem-estimate=70.91MB mem-reservation=15.81MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=47.96MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=48.27MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 08:AGGREGATE [STREAMING] | | output: sum(ss_net_profit) | | group by: s_state @@ -388,7 +388,7 @@ Per-Host Resources: mem-estimate=70.91MB mem-reservation=15.81MB thread-reservat | | | in pipelines: 05(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1223 AS INT), d_month_seq >= CAST(1212 AS INT) @@ -416,7 +416,7 @@ Per-Host Resources: mem-estimate=70.91MB mem-reservation=15.81MB thread-reservat | | | in pipelines: 04(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | | 04:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -453,7 +453,7 @@ Per-Host Resources: mem-estimate=70.91MB mem-reservation=15.81MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=49.00MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=49.20MB mem-reservation=1.02MB thread-reservation=2 runtime-filters-memory=1.00MB | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | runtime filters: RF001[min_max] -> s_state, RF000[bloom] -> s_state @@ -479,7 +479,7 @@ Per-Host Resources: mem-estimate=70.91MB mem-reservation=15.81MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d1.d_month_seq <= CAST(1223 AS INT), d1.d_month_seq >= CAST(1212 AS INT) @@ -506,7 +506,7 @@ Per-Host Resources: mem-estimate=70.91MB mem-reservation=15.81MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=137.79MB Threads=20 -Per-Host Resource Estimates: Memory=406MB +Per-Host Resource Estimates: Memory=415MB F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -578,7 +578,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=46.00MB mem-reservation=7.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=49.59MB mem-reservation=7.00MB thread-reservation=1 15:AGGREGATE [STREAMING] | Class 0 | output: sum(ss_net_profit) @@ -653,7 +653,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=7.00MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.80MB mem-reservation=3.00MB thread-reservation=1 | 08:AGGREGATE [STREAMING] | | output: sum(ss_net_profit) | | group by: s_state @@ -683,7 +683,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=7.00MB thread-reser | | | in pipelines: 05(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1223 AS INT), d_month_seq >= CAST(1212 AS INT) @@ -719,7 +719,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=7.00MB thread-reser | | | in pipelines: 04(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | | 04:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -765,7 +765,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=7.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.20MB mem-reservation=24.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | runtime filters: RF001[min_max] -> s_state, RF000[bloom] -> s_state @@ -799,7 +799,7 @@ Per-Instance Resources: mem-estimate=46.00MB mem-reservation=7.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d1.d_month_seq <= CAST(1223 AS INT), d1.d_month_seq >= CAST(1212 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test index cbad5b3b6..e5f9648aa 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test @@ -213,7 +213,7 @@ PLAN-ROOT SINK in pipelines: 11(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=42.94MB Threads=15 -Per-Host Resource Estimates: Memory=515MB +Per-Host Resource Estimates: Memory=516MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.19MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -269,7 +269,7 @@ Per-Host Resources: mem-estimate=15.82MB mem-reservation=5.88MB thread-reservati | | in pipelines: 11(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.41MB mem-reservation=512.00KB thread-reservation=2 | 11:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | HDFS partitions=1/1 files=1 size=1.31MB | predicates: t_meal_time IN ('breakfast', 'dinner') @@ -289,7 +289,7 @@ Per-Host Resources: mem-estimate=15.82MB mem-reservation=5.88MB thread-reservati | in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT) | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=200.91MB mem-reservation=16.88MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=201.69MB mem-reservation=16.88MB thread-reservation=2 runtime-filters-memory=5.00MB 12:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: sold_item_sk = i_item_sk | fk/pk conjuncts: assumed fk/pk @@ -304,7 +304,7 @@ Per-Host Resources: mem-estimate=200.91MB mem-reservation=16.88MB thread-reserva | | in pipelines: 00(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.19MB mem-reservation=256.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_manager_id = CAST(1 AS INT) @@ -337,7 +337,7 @@ Per-Host Resources: mem-estimate=200.91MB mem-reservation=16.88MB thread-reserva | | | in pipelines: 09(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 09:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT) @@ -377,7 +377,7 @@ Per-Host Resources: mem-estimate=200.91MB mem-reservation=16.88MB thread-reserva | | | in pipelines: 06(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 06:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT) @@ -416,7 +416,7 @@ Per-Host Resources: mem-estimate=200.91MB mem-reservation=16.88MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT) @@ -442,7 +442,7 @@ Per-Host Resources: mem-estimate=200.91MB mem-reservation=16.88MB thread-reserva in pipelines: 02(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=84.50MB Threads=18 -Per-Host Resource Estimates: Memory=279MB +Per-Host Resource Estimates: Memory=286MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.36MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -476,7 +476,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT) | F09:PLAN FRAGMENT [HASH(time_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=10.41MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=11.82MB mem-reservation=2.00MB thread-reservation=1 14:AGGREGATE [STREAMING] | output: sum(ext_price) | group by: i_brand, i_brand_id, t_hour, t_minute @@ -506,7 +506,7 @@ Per-Instance Resources: mem-estimate=10.41MB mem-reservation=2.00MB thread-reser | | in pipelines: 11(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.82MB mem-reservation=512.00KB thread-reservation=1 | 11:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | HDFS partitions=1/1 files=1 size=1.31MB | predicates: t_meal_time IN ('breakfast', 'dinner') @@ -527,7 +527,7 @@ Per-Instance Resources: mem-estimate=10.41MB mem-reservation=2.00MB thread-reser | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=49.60MB mem-reservation=8.00MB thread-reservation=1 12:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=01 | hash predicates: sold_item_sk = i_item_sk @@ -550,7 +550,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_manager_id = CAST(1 AS INT) @@ -591,7 +591,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | in pipelines: 09(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 09:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT) @@ -639,7 +639,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | | in pipelines: 06(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 06:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT) @@ -686,7 +686,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q72.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q72.test index c1a4a11df..4c0e995fa 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q72.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q72.test @@ -320,7 +320,7 @@ Per-Host Resources: mem-estimate=178.42MB mem-reservation=124.12MB thread-reserv | | in pipelines: 10(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.08MB mem-reservation=2.00MB thread-reservation=2 | 10:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | HDFS partitions=1/1 files=1 size=10.62MB | stored statistics: @@ -344,7 +344,7 @@ Per-Host Resources: mem-estimate=178.42MB mem-reservation=124.12MB thread-reserv | | in pipelines: 09(GETNEXT) | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=32.00KB thread-reservation=2 | 09:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | stored statistics: @@ -370,7 +370,7 @@ Per-Host Resources: mem-estimate=178.42MB mem-reservation=124.12MB thread-reserv | | in pipelines: 08(GETNEXT) | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 08:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -395,7 +395,7 @@ Per-Host Resources: mem-estimate=178.42MB mem-reservation=124.12MB thread-reserv | | in pipelines: 04(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.08MB mem-reservation=8.00MB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'D' @@ -423,7 +423,7 @@ Per-Host Resources: mem-estimate=178.42MB mem-reservation=124.12MB thread-reserv | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.11MB mem-reservation=64.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: hd_buy_potential = '>10000' @@ -467,7 +467,7 @@ Per-Host Resources: mem-estimate=178.42MB mem-reservation=124.12MB thread-reserv | | | in pipelines: 06(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.15MB mem-reservation=2.00MB thread-reservation=2 | | 06:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year = CAST(1999 AS INT) @@ -495,7 +495,7 @@ Per-Host Resources: mem-estimate=178.42MB mem-reservation=124.12MB thread-reserv | | | in pipelines: 07(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.05MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | | 07:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF011[min_max] -> d2.d_week_seq, RF010[bloom] -> d2.d_week_seq @@ -521,7 +521,7 @@ Per-Host Resources: mem-estimate=178.42MB mem-reservation=124.12MB thread-reserv | | | in pipelines: 03(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.49MB mem-reservation=2.00MB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -546,7 +546,7 @@ Per-Host Resources: mem-estimate=178.42MB mem-reservation=124.12MB thread-reserv | | | in pipelines: 02(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.15MB mem-reservation=16.00KB thread-reservation=2 | | 02:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | | HDFS partitions=1/1 files=1 size=4.38KB | | stored statistics: @@ -574,7 +574,7 @@ Per-Host Resources: mem-estimate=178.42MB mem-reservation=124.12MB thread-reserv | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=389.00MB mem-reservation=21.00MB thread-reservation=2 runtime-filters-memory=5.00MB +Per-Host Resources: mem-estimate=389.52MB mem-reservation=21.00MB thread-reservation=2 runtime-filters-memory=5.00MB 00:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM] HDFS partitions=1/1 files=3 size=96.62MB runtime filters: RF005[min_max] -> cs_bill_hdemo_sk, RF001[min_max] -> cs_ship_date_sk, RF003[min_max] -> cs_bill_cdemo_sk, RF008[min_max] -> cs_sold_date_sk, RF009[min_max] -> cs_item_sk, RF004[bloom] -> cs_bill_hdemo_sk, RF000[bloom] -> cs_ship_date_sk, RF002[bloom] -> cs_bill_cdemo_sk, RF006[bloom] -> cs_sold_date_sk, RF007[bloom] -> cs_item_sk @@ -587,7 +587,7 @@ Per-Host Resources: mem-estimate=389.00MB mem-reservation=21.00MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=300.61MB Threads=24 -Per-Host Resource Estimates: Memory=595MB +Per-Host Resource Estimates: Memory=602MB F13:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -622,7 +622,7 @@ Per-Instance Resources: mem-estimate=86.95MB mem-reservation=34.00MB thread-rese | in pipelines: 00(GETNEXT) | F06:PLAN FRAGMENT [HASH(cs_sold_date_sk,cs_item_sk)] hosts=3 instances=3 -Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=69.50MB mem-reservation=34.00MB thread-reservation=1 21:AGGREGATE [STREAMING] | output: sum(CAST(CASE WHEN p_promo_sk IS NULL THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN p_promo_sk IS NOT NULL THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), count(*) | group by: i_item_desc, w_warehouse_name, d1.d_week_seq @@ -651,7 +651,7 @@ Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-rese | | in pipelines: 10(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.08MB mem-reservation=2.00MB thread-reservation=1 | 10:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | HDFS partitions=1/1 files=1 size=10.62MB | stored statistics: @@ -683,7 +683,7 @@ Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-rese | | in pipelines: 09(GETNEXT) | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=32.00KB thread-reservation=1 | 09:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | stored statistics: @@ -717,7 +717,7 @@ Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-rese | | in pipelines: 08(GETNEXT) | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 08:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -750,7 +750,7 @@ Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-rese | | in pipelines: 04(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.08MB mem-reservation=8.00MB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.customer_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=7.49MB | predicates: cd_marital_status = 'D' @@ -786,7 +786,7 @@ Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-rese | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.11MB mem-reservation=64.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: hd_buy_potential = '>10000' @@ -824,7 +824,7 @@ Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-rese | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=34.76MB mem-reservation=16.00MB thread-reservation=1 | 14:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=06 | | hash predicates: d2.d_week_seq = d1.d_week_seq @@ -847,7 +847,7 @@ Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-rese | | | in pipelines: 06(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.15MB mem-reservation=2.00MB thread-reservation=1 | | 06:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d1.d_year = CAST(1999 AS INT) @@ -884,7 +884,7 @@ Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-rese | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 07:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF011[min_max] -> d2.d_week_seq, RF010[bloom] -> d2.d_week_seq @@ -918,7 +918,7 @@ Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-rese | | | in pipelines: 03(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -951,7 +951,7 @@ Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-rese | | | in pipelines: 02(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.15MB mem-reservation=16.00KB thread-reservation=1 | | 02:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | | HDFS partitions=1/1 files=1 size=4.38KB | | stored statistics: @@ -980,7 +980,7 @@ Per-Instance Resources: mem-estimate=67.46MB mem-reservation=34.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=48.52MB mem-reservation=16.00MB thread-reservation=1 00:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM] HDFS partitions=1/1 files=3 size=96.62MB runtime filters: RF005[min_max] -> cs_bill_hdemo_sk, RF001[min_max] -> cs_ship_date_sk, RF003[min_max] -> cs_bill_cdemo_sk, RF008[min_max] -> cs_sold_date_sk, RF009[min_max] -> cs_item_sk, RF004[bloom] -> cs_bill_hdemo_sk, RF000[bloom] -> cs_ship_date_sk, RF002[bloom] -> cs_bill_cdemo_sk, RF006[bloom] -> cs_sold_date_sk, RF007[bloom] -> cs_item_sk diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test index d4ceb8a26..4e48cb75c 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test @@ -151,7 +151,7 @@ PLAN-ROOT SINK in pipelines: 08(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=29.95MB Threads=13 -Per-Host Resource Estimates: Memory=305MB +Per-Host Resource Estimates: Memory=306MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.27MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -187,7 +187,7 @@ Per-Host Resources: mem-estimate=12.50MB mem-reservation=5.75MB thread-reservati | | in pipelines: 08(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=80.85MB mem-reservation=1.00MB thread-reservation=2 | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | stored statistics: @@ -241,7 +241,7 @@ Per-Host Resources: mem-estimate=85.08MB mem-reservation=15.75MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.14MB mem-reservation=16.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_county IN ('Saginaw County', 'Sumner County', 'Appanoose County', 'Daviess County') @@ -269,7 +269,7 @@ Per-Host Resources: mem-estimate=85.08MB mem-reservation=15.75MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -294,7 +294,7 @@ Per-Host Resources: mem-estimate=85.08MB mem-reservation=15.75MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.14MB mem-reservation=64.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: household_demographics.hd_vehicle_count > CAST(0 AS INT), household_demographics.hd_buy_potential IN ('>10000', 'unknown'), CASE WHEN household_demographics.hd_vehicle_count > CAST(0 AS INT) THEN CAST(household_demographics.hd_dep_count AS DOUBLE) / CAST(household_demographics.hd_vehicle_count AS DOUBLE) ELSE NULL END > CAST(1 AS DOUBLE) @@ -321,7 +321,7 @@ Per-Host Resources: mem-estimate=85.08MB mem-reservation=15.75MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=45.70MB Threads=16 -Per-Host Resource Estimates: Memory=185MB +Per-Host Resource Estimates: Memory=189MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.51MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -336,7 +336,7 @@ PLAN-ROOT SINK | in pipelines: 10(GETNEXT) | F06:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=328.09KB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=415.15KB mem-reservation=0B thread-reservation=1 10:TOP-N [LIMIT=1000] | order by: cnt DESC | mem-estimate=78.33KB mem-reservation=0B thread-reservation=0 @@ -365,7 +365,7 @@ Per-Instance Resources: mem-estimate=328.09KB mem-reservation=0B thread-reservat | | in pipelines: 08(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.69MB mem-reservation=1.00MB thread-reservation=1 | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | stored statistics: @@ -398,7 +398,7 @@ Per-Instance Resources: mem-estimate=11.17MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | output: count(*) | group by: ss_ticket_number, ss_customer_sk @@ -428,7 +428,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_county IN ('Saginaw County', 'Sumner County', 'Appanoose County', 'Daviess County') @@ -464,7 +464,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -497,7 +497,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.14MB mem-reservation=64.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: household_demographics.hd_vehicle_count > CAST(0 AS INT), household_demographics.hd_buy_potential IN ('>10000', 'unknown'), CASE WHEN household_demographics.hd_vehicle_count > CAST(0 AS INT) THEN CAST(household_demographics.hd_dep_count AS DOUBLE) / CAST(household_demographics.hd_vehicle_count AS DOUBLE) ELSE NULL END > CAST(1 AS DOUBLE) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test index 1c8001dac..ce52286a4 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test @@ -361,7 +361,7 @@ PLAN-ROOT SINK in pipelines: 09(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=239.81MB Threads=34 -Per-Host Resource Estimates: Memory=925MB +Per-Host Resource Estimates: Memory=932MB F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -398,7 +398,7 @@ Per-Host Resources: mem-estimate=15.89MB mem-reservation=9.62MB thread-reservati | | in pipelines: 54(GETNEXT) | | | F24:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=16.10MB mem-reservation=8.50MB thread-reservation=1 +| Per-Host Resources: mem-estimate=16.66MB mem-reservation=8.50MB thread-reservation=1 | 21:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=38 row-size=44B cardinality=148.00K @@ -439,7 +439,7 @@ Per-Host Resources: mem-estimate=15.89MB mem-reservation=9.62MB thread-reservati | | | in pipelines: 22(GETNEXT) | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.56MB mem-reservation=4.00MB thread-reservation=2 | | 22:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -456,7 +456,7 @@ Per-Host Resources: mem-estimate=15.89MB mem-reservation=9.62MB thread-reservati | | in pipelines: 23(GETNEXT) | | | F19:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=98.95MB mem-reservation=10.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=99.16MB mem-reservation=10.94MB thread-reservation=2 runtime-filters-memory=1.00MB | 25:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_sold_date_sk = d_date_sk | | fk/pk conjuncts: ws_sold_date_sk = d_date_sk @@ -471,7 +471,7 @@ Per-Host Resources: mem-estimate=15.89MB mem-reservation=9.62MB thread-reservati | | | in pipelines: 24(GETNEXT) | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -510,7 +510,7 @@ Per-Host Resources: mem-estimate=15.89MB mem-reservation=9.62MB thread-reservati | | in pipelines: 47(GETNEXT) | | | F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=44.26MB mem-reservation=34.00MB thread-reservation=1 +| Per-Host Resources: mem-estimate=44.82MB mem-reservation=34.00MB thread-reservation=1 | 00:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=8 row-size=44B cardinality=58.90K @@ -552,7 +552,7 @@ Per-Host Resources: mem-estimate=15.89MB mem-reservation=9.62MB thread-reservati | | | in pipelines: 01(GETNEXT) | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=65.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=65.84MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 01:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -570,7 +570,7 @@ Per-Host Resources: mem-estimate=15.89MB mem-reservation=9.62MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=36.27MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 04:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_sold_date_sk = d_date_sk | | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -585,7 +585,7 @@ Per-Host Resources: mem-estimate=15.89MB mem-reservation=9.62MB thread-reservati | | | in pipelines: 03(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -617,7 +617,7 @@ Per-Host Resources: mem-estimate=15.89MB mem-reservation=9.62MB thread-reservati | in pipelines: 36(GETNEXT) | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=47.91MB mem-reservation=36.94MB thread-reservation=1 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=48.74MB mem-reservation=36.94MB thread-reservation=1 runtime-filters-memory=1.00MB 28:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: customer_id = customer_id | fk/pk conjuncts: assumed fk/pk @@ -632,7 +632,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=36.94MB thread-reservat | | in pipelines: 41(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=16.10MB mem-reservation=8.50MB thread-reservation=1 +| Per-Host Resources: mem-estimate=16.28MB mem-reservation=8.50MB thread-reservation=1 | 14:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=28 row-size=44B cardinality=14.80K @@ -674,7 +674,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=36.94MB thread-reservat | | | in pipelines: 15(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=66.00MB mem-reservation=6.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=66.56MB mem-reservation=6.00MB thread-reservation=2 runtime-filters-memory=2.00MB | | 15:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id @@ -692,7 +692,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=36.94MB thread-reservat | | in pipelines: 16(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=99.95MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=100.16MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 18:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_sold_date_sk = d_date_sk | | fk/pk conjuncts: ws_sold_date_sk = d_date_sk @@ -707,7 +707,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=36.94MB thread-reservat | | | in pipelines: 17(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -772,7 +772,7 @@ Per-Host Resources: mem-estimate=33.13MB mem-reservation=22.75MB thread-reservat | | in pipelines: 08(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=67.00MB mem-reservation=7.00MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=67.84MB mem-reservation=7.00MB thread-reservation=2 runtime-filters-memory=3.00MB | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id @@ -790,7 +790,7 @@ Per-Host Resources: mem-estimate=33.13MB mem-reservation=22.75MB thread-reservat | in pipelines: 09(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=36.27MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=2.00MB 11:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_sold_date_sk = d_date_sk | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -805,7 +805,7 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati | | in pipelines: 10(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -832,7 +832,7 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati in pipelines: 09(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=265.50MB Threads=44 -Per-Host Resource Estimates: Memory=619MB +Per-Host Resource Estimates: Memory=646MB F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -877,7 +877,7 @@ Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservatio | | in pipelines: 54(GETNEXT) | | | F24:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=16.10MB mem-reservation=8.50MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.22MB mem-reservation=8.50MB thread-reservation=1 | 21:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=38 row-size=44B cardinality=148.00K @@ -896,7 +896,7 @@ Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservatio | | in pipelines: 23(GETNEXT) | | | F22:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=11.47MB mem-reservation=9.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=12.15MB mem-reservation=9.00MB thread-reservation=1 | 27:AGGREGATE [STREAMING] | | output: sum(ws_net_paid) | | group by: c_customer_id, c_first_name, c_last_name, d_year @@ -926,7 +926,7 @@ Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservatio | | | in pipelines: 22(GETNEXT) | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.56MB mem-reservation=4.00MB thread-reservation=1 | | 22:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -944,7 +944,7 @@ Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservatio | | | F19:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.22MB mem-reservation=8.00MB thread-reservation=1 | 25:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=02 | | hash predicates: ws_sold_date_sk = d_date_sk @@ -967,7 +967,7 @@ Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservatio | | | in pipelines: 24(GETNEXT) | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -1014,7 +1014,7 @@ Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservatio | | in pipelines: 47(GETNEXT) | | | F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=28.64MB mem-reservation=17.00MB thread-reservation=1 | 00:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=8 row-size=44B cardinality=58.90K @@ -1034,7 +1034,7 @@ Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservatio | | in pipelines: 02(GETNEXT) | | | F15:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=13.91MB mem-reservation=9.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=15.97MB mem-reservation=9.00MB thread-reservation=1 | 06:AGGREGATE [STREAMING] | | output: sum(ss_net_paid) | | group by: c_customer_id, c_first_name, c_last_name, d_year @@ -1065,7 +1065,7 @@ Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservatio | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=17.69MB mem-reservation=4.00MB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1084,7 +1084,7 @@ Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservatio | | | F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.66MB mem-reservation=1.00MB thread-reservation=1 | 04:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=05 | | hash predicates: ss_sold_date_sk = d_date_sk @@ -1107,7 +1107,7 @@ Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservatio | | | in pipelines: 03(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -1139,7 +1139,7 @@ Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservatio | in pipelines: 36(GETNEXT) | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=30.61MB mem-reservation=17.00MB thread-reservation=1 28:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=06 | hash predicates: customer_id = customer_id @@ -1162,7 +1162,7 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese | | in pipelines: 41(GETNEXT) | | | F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=16.10MB mem-reservation=8.50MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.28MB mem-reservation=8.50MB thread-reservation=1 | 14:UNION | | mem-estimate=0B mem-reservation=0B thread-reservation=0 | | tuple-ids=28 row-size=44B cardinality=14.80K @@ -1182,7 +1182,7 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese | | in pipelines: 16(GETNEXT) | | | F09:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=11.47MB mem-reservation=9.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=12.15MB mem-reservation=9.00MB thread-reservation=1 | 20:AGGREGATE [STREAMING] | | output: sum(ws_net_paid) | | group by: c_customer_id, c_first_name, c_last_name, d_year @@ -1213,7 +1213,7 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.56MB mem-reservation=4.00MB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1232,7 +1232,7 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese | | | F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.22MB mem-reservation=8.00MB thread-reservation=1 | 18:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=08 | | hash predicates: ws_sold_date_sk = d_date_sk @@ -1255,7 +1255,7 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese | | | in pipelines: 17(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -1298,7 +1298,7 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese | in pipelines: 09(GETNEXT) | F03:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=13.91MB mem-reservation=9.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=15.97MB mem-reservation=9.00MB thread-reservation=1 13:AGGREGATE [STREAMING] | output: sum(ss_net_paid) | group by: c_customer_id, c_first_name, c_last_name, d_year @@ -1329,7 +1329,7 @@ Per-Instance Resources: mem-estimate=13.91MB mem-reservation=9.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.69MB mem-reservation=4.00MB thread-reservation=1 | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | runtime filters: RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id @@ -1348,7 +1348,7 @@ Per-Instance Resources: mem-estimate=13.91MB mem-reservation=9.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=16.66MB mem-reservation=1.00MB thread-reservation=1 11:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=10 | hash predicates: ss_sold_date_sk = d_date_sk @@ -1371,7 +1371,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reser | | in pipelines: 10(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test index faf29c5c2..0882c96e0 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test @@ -573,7 +573,7 @@ PLAN-ROOT SINK in pipelines: 04(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=140.19MB Threads=56 -Per-Host Resource Estimates: Memory=2.31GB +Per-Host Resource Estimates: Memory=2.32GB F37:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -645,7 +645,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | in pipelines: 25(GETNEXT), 32(GETNEXT), 39(GETNEXT) | | | F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservation=1 +| Per-Host Resources: mem-estimate=27.25MB mem-reservation=11.50MB thread-reservation=1 | 46:AGGREGATE [STREAMING] | | group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt | | mem-estimate=10.00MB mem-reservation=3.00MB spill-buffer=128.00KB thread-reservation=0 @@ -670,7 +670,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | | | in pipelines: 42(GETNEXT) | | | | | | | F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=64.33MB mem-reservation=2.00MB thread-reservation=2 | | | 42:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.66MB | | | stored statistics: @@ -687,7 +687,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | | in pipelines: 39(GETNEXT) | | | | | F28:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=165.01MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=165.93MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=1.00MB | | 44:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: ws_sold_date_sk = d_date_sk | | | fk/pk conjuncts: ws_sold_date_sk = d_date_sk @@ -702,7 +702,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | | | in pipelines: 41(GETNEXT) | | | | | | | F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 41:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -730,7 +730,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | | | in pipelines: 40(GETNEXT) | | | | | | | F29:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=96.18MB mem-reservation=256.00KB thread-reservation=2 | | | 40:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | predicates: i_category = 'Books' @@ -768,7 +768,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | | | in pipelines: 35(GETNEXT) | | | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=96.33MB mem-reservation=4.00MB thread-reservation=2 | | | 35:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=15.43MB | | | stored statistics: @@ -785,7 +785,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | | in pipelines: 32(GETNEXT) | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| | Per-Host Resources: mem-estimate=70.01MB mem-reservation=7.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=70.93MB mem-reservation=7.88MB thread-reservation=2 runtime-filters-memory=2.00MB | | 37:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: ss_sold_date_sk = d_date_sk | | | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -800,7 +800,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | | | in pipelines: 34(GETNEXT) | | | | | | | F25:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 34:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -828,7 +828,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | | | in pipelines: 33(GETNEXT) | | | | | | | F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=96.18MB mem-reservation=256.00KB thread-reservation=2 | | | 33:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | predicates: i_category = 'Books' @@ -867,7 +867,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | | in pipelines: 28(GETNEXT) | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=96.33MB mem-reservation=4.00MB thread-reservation=2 | | 28:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=10.62MB | | stored statistics: @@ -884,7 +884,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | in pipelines: 25(GETNEXT) | | | F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=246.01MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=246.93MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 30:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: cs_sold_date_sk = d_date_sk | | fk/pk conjuncts: cs_sold_date_sk = d_date_sk @@ -899,7 +899,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | | in pipelines: 27(GETNEXT) | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 27:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -927,7 +927,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio | | | in pipelines: 26(GETNEXT) | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=96.18MB mem-reservation=256.00KB thread-reservation=2 | | 26:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Books' @@ -993,7 +993,7 @@ Per-Host Resources: mem-estimate=21.15MB mem-reservation=5.88MB thread-reservati | in pipelines: 01(GETNEXT), 08(GETNEXT), 15(GETNEXT) | F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservation=1 +Per-Host Resources: mem-estimate=27.25MB mem-reservation=11.50MB thread-reservation=1 22:AGGREGATE [STREAMING] | group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt | mem-estimate=10.00MB mem-reservation=3.00MB spill-buffer=128.00KB thread-reservation=0 @@ -1018,7 +1018,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat | | | in pipelines: 18(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=64.33MB mem-reservation=2.00MB thread-reservation=2 | | 18:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=5.66MB | | stored statistics: @@ -1035,7 +1035,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat | | in pipelines: 15(GETNEXT) | | | F10:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=165.01MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=165.93MB mem-reservation=12.88MB thread-reservation=2 runtime-filters-memory=1.00MB | 20:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_sold_date_sk = d_date_sk | | fk/pk conjuncts: ws_sold_date_sk = d_date_sk @@ -1050,7 +1050,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat | | | in pipelines: 17(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -1078,7 +1078,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat | | | in pipelines: 16(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=96.18MB mem-reservation=256.00KB thread-reservation=2 | | 16:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Books' @@ -1117,7 +1117,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat | | | in pipelines: 11(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=96.33MB mem-reservation=4.00MB thread-reservation=2 | | 11:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | stored statistics: @@ -1134,7 +1134,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat | | in pipelines: 08(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=70.01MB mem-reservation=7.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=70.93MB mem-reservation=7.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 13:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_sold_date_sk = d_date_sk | | fk/pk conjuncts: ss_sold_date_sk = d_date_sk @@ -1149,7 +1149,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat | | | in pipelines: 10(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -1177,7 +1177,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat | | | in pipelines: 09(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=96.18MB mem-reservation=256.00KB thread-reservation=2 | | 09:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Books' @@ -1217,7 +1217,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=96.33MB mem-reservation=4.00MB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | HDFS partitions=1/1 files=1 size=10.62MB | stored statistics: @@ -1234,7 +1234,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=246.01MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=246.93MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB 06:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: cs_sold_date_sk = d_date_sk | fk/pk conjuncts: cs_sold_date_sk = d_date_sk @@ -1249,7 +1249,7 @@ Per-Host Resources: mem-estimate=246.01MB mem-reservation=13.88MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -1277,7 +1277,7 @@ Per-Host Resources: mem-estimate=246.01MB mem-reservation=13.88MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=96.18MB mem-reservation=256.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category = 'Books' @@ -1304,7 +1304,7 @@ Per-Host Resources: mem-estimate=246.01MB mem-reservation=13.88MB thread-reserva in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=208.62MB Threads=67 -Per-Host Resource Estimates: Memory=884MB +Per-Host Resource Estimates: Memory=910MB F37:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -1319,7 +1319,7 @@ PLAN-ROOT SINK | in pipelines: 49(GETNEXT) | F36:PLAN FRAGMENT [HASH(i_brand_id,i_category_id,i_class_id,i_manufact_id)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservation=1 +Per-Instance Resources: mem-estimate=311.03KB mem-reservation=0B thread-reservation=1 49:TOP-N [LIMIT=100] | order by: sales_cnt - sales_cnt ASC, sales_amt - sales_amt ASC | mem-estimate=7.03KB mem-reservation=0B thread-reservation=0 @@ -1384,7 +1384,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | in pipelines: 25(GETNEXT), 32(GETNEXT), 39(GETNEXT) | | | F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -| Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=12.83MB mem-reservation=2.00MB thread-reservation=1 | 46:AGGREGATE [STREAMING] | | group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt | | mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0 @@ -1417,7 +1417,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | | in pipelines: 42(GETNEXT) | | | | | | | F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.66MB mem-reservation=2.00MB thread-reservation=1 | | | 42:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.66MB | | | stored statistics: @@ -1435,7 +1435,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | | | F28:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=34.11MB mem-reservation=8.00MB thread-reservation=1 | | 44:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=08 | | | hash predicates: ws_sold_date_sk = d_date_sk @@ -1458,7 +1458,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | | in pipelines: 41(GETNEXT) | | | | | | | F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 41:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -1494,7 +1494,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | | in pipelines: 40(GETNEXT) | | | | | | | F29:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.18MB mem-reservation=256.00KB thread-reservation=1 | | | 40:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | predicates: i_category = 'Books' @@ -1540,7 +1540,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | | in pipelines: 35(GETNEXT) | | | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=24.66MB mem-reservation=4.00MB thread-reservation=1 | | | 35:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=15.43MB | | | stored statistics: @@ -1558,7 +1558,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=18.11MB mem-reservation=2.00MB thread-reservation=1 | | 37:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=06 | | | hash predicates: ss_sold_date_sk = d_date_sk @@ -1581,7 +1581,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | | in pipelines: 34(GETNEXT) | | | | | | | F25:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 34:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -1617,7 +1617,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | | in pipelines: 33(GETNEXT) | | | | | | | F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.18MB mem-reservation=256.00KB thread-reservation=1 | | | 33:SCAN HDFS [tpcds_parquet.item, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.73MB | | | predicates: i_category = 'Books' @@ -1664,7 +1664,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | in pipelines: 28(GETNEXT) | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.66MB mem-reservation=4.00MB thread-reservation=1 | | 28:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=10.62MB | | stored statistics: @@ -1682,7 +1682,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=50.11MB mem-reservation=8.00MB thread-reservation=1 | 30:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=04 | | hash predicates: cs_sold_date_sk = d_date_sk @@ -1705,7 +1705,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | in pipelines: 27(GETNEXT) | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 27:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT) @@ -1741,7 +1741,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati | | | in pipelines: 26(GETNEXT) | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.18MB mem-reservation=256.00KB thread-reservation=1 | | 26:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Books' @@ -1807,7 +1807,7 @@ Per-Instance Resources: mem-estimate=21.26MB mem-reservation=3.94MB thread-reser | in pipelines: 01(GETNEXT), 08(GETNEXT), 15(GETNEXT) | F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=12.83MB mem-reservation=2.00MB thread-reservation=1 22:AGGREGATE [STREAMING] | group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt | mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0 @@ -1840,7 +1840,7 @@ Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reser | | | in pipelines: 18(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.66MB mem-reservation=2.00MB thread-reservation=1 | | 18:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=5.66MB | | stored statistics: @@ -1858,7 +1858,7 @@ Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reser | | | F10:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=34.11MB mem-reservation=8.00MB thread-reservation=1 | 20:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=17 | | hash predicates: ws_sold_date_sk = d_date_sk @@ -1881,7 +1881,7 @@ Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reser | | | in pipelines: 17(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -1917,7 +1917,7 @@ Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reser | | | in pipelines: 16(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.18MB mem-reservation=256.00KB thread-reservation=1 | | 16:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Books' @@ -1964,7 +1964,7 @@ Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reser | | | in pipelines: 11(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.66MB mem-reservation=4.00MB thread-reservation=1 | | 11:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=15.43MB | | stored statistics: @@ -1982,7 +1982,7 @@ Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reser | | | F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=18.11MB mem-reservation=2.00MB thread-reservation=1 | 13:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=15 | | hash predicates: ss_sold_date_sk = d_date_sk @@ -2005,7 +2005,7 @@ Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reser | | | in pipelines: 10(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -2041,7 +2041,7 @@ Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reser | | | in pipelines: 09(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.18MB mem-reservation=256.00KB thread-reservation=1 | | 09:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_category = 'Books' @@ -2089,7 +2089,7 @@ Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.66MB mem-reservation=4.00MB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | HDFS partitions=1/1 files=1 size=10.62MB | stored statistics: @@ -2107,7 +2107,7 @@ Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=50.11MB mem-reservation=8.00MB thread-reservation=1 06:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=13 | hash predicates: cs_sold_date_sk = d_date_sk @@ -2130,7 +2130,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT) @@ -2166,7 +2166,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.18MB mem-reservation=256.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category = 'Books' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q76.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q76.test index 7456974d0..ec412b220 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q76.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q76.test @@ -230,7 +230,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=51.31MB Threads=19 -Per-Host Resource Estimates: Memory=624MB +Per-Host Resource Estimates: Memory=626MB F13:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -265,7 +265,7 @@ Per-Host Resources: mem-estimate=13.46MB mem-reservation=4.75MB thread-reservati | in pipelines: 01(GETNEXT), 07(GETNEXT), 11(GETNEXT) | F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=70.01MB mem-reservation=16.69MB thread-reservation=2 runtime-filters-memory=4.00MB +Per-Host Resources: mem-estimate=70.93MB mem-reservation=16.69MB thread-reservation=2 runtime-filters-memory=4.00MB 16:AGGREGATE [STREAMING] | output: count(*), sum(ext_sales_price) | group by: channel, col_name, d_year, d_qoy, i_category @@ -292,7 +292,7 @@ Per-Host Resources: mem-estimate=70.01MB mem-reservation=16.69MB thread-reservat | | | in pipelines: 12(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.35MB mem-reservation=256.00KB thread-reservation=2 | | 12:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -324,7 +324,7 @@ Per-Host Resources: mem-estimate=70.01MB mem-reservation=16.69MB thread-reservat | | | in pipelines: 13(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.19MB mem-reservation=512.00KB thread-reservation=2 | | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | stored statistics: @@ -341,7 +341,7 @@ Per-Host Resources: mem-estimate=70.01MB mem-reservation=16.69MB thread-reservat | | in pipelines: 11(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=194.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=194.28MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB | 11:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM] | HDFS partitions=1/1 files=3 size=96.62MB | predicates: cs_ship_addr_sk IS NULL @@ -368,7 +368,7 @@ Per-Host Resources: mem-estimate=70.01MB mem-reservation=16.69MB thread-reservat | | | in pipelines: 08(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=50.95MB mem-reservation=3.44MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=51.09MB mem-reservation=3.44MB thread-reservation=2 runtime-filters-memory=1.00MB | | 09:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: d_date_sk = ws_sold_date_sk | | | fk/pk conjuncts: none @@ -383,7 +383,7 @@ Per-Host Resources: mem-estimate=70.01MB mem-reservation=16.69MB thread-reservat | | | | in pipelines: 06(GETNEXT) | | | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | | Per-Host Resources: mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=128.09MB mem-reservation=8.00MB thread-reservation=2 | | | 06:SCAN HDFS [tpcds_parquet.web_sales, RANDOM] | | | HDFS partitions=1/1 files=2 size=45.09MB | | | predicates: ws_ship_customer_sk IS NULL @@ -431,7 +431,7 @@ Per-Host Resources: mem-estimate=70.01MB mem-reservation=16.69MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -456,7 +456,7 @@ Per-Host Resources: mem-estimate=70.01MB mem-reservation=16.69MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=256.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -481,7 +481,7 @@ Per-Host Resources: mem-estimate=70.01MB mem-reservation=16.69MB thread-reservat in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=75.69MB Threads=20 -Per-Host Resource Estimates: Memory=287MB +Per-Host Resource Estimates: Memory=293MB F13:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -517,7 +517,7 @@ Per-Instance Resources: mem-estimate=13.68MB mem-reservation=2.88MB thread-reser | F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=27.83MB mem-reservation=4.00MB thread-reservation=1 16:AGGREGATE [STREAMING] | output: count(*), sum(ext_sales_price) | group by: channel, col_name, d_year, d_qoy, i_category @@ -552,7 +552,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | | in pipelines: 12(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.70MB mem-reservation=256.00KB thread-reservation=1 | | 12:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -569,7 +569,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 11(GETNEXT) | | | F08:PLAN FRAGMENT [HASH(cs_sold_date_sk)] hosts=3 instances=3 -| Per-Instance Resources: mem-estimate=118.65KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=1.05MB mem-reservation=0B thread-reservation=1 | 14:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=05 | | hash predicates: cs_sold_date_sk = d_date_sk @@ -592,7 +592,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | | in pipelines: 13(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.19MB mem-reservation=512.00KB thread-reservation=1 | | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | stored statistics: @@ -610,7 +610,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | | F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=48.28MB mem-reservation=8.00MB thread-reservation=1 | 11:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM] | HDFS partitions=1/1 files=3 size=96.62MB | predicates: cs_ship_addr_sk IS NULL @@ -646,7 +646,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | | 09:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=03 | | | hash predicates: d_date_sk = ws_sold_date_sk @@ -669,7 +669,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | | | in pipelines: 06(GETNEXT) | | | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | | Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=32.09MB mem-reservation=8.00MB thread-reservation=1 | | | 06:SCAN HDFS [tpcds_parquet.web_sales, RANDOM] | | | HDFS partitions=1/1 files=2 size=45.09MB | | | predicates: ws_ship_customer_sk IS NULL @@ -725,7 +725,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | stored statistics: @@ -758,7 +758,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=256.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q77.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q77.test index 9bb262a6d..ab8454ad9 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q77.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q77.test @@ -520,7 +520,7 @@ Per-Host Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reservati | in pipelines: 42(GETNEXT), 49(GETNEXT), 57(GETNEXT) | F22:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservation=1 +Per-Host Resources: mem-estimate=54.34MB mem-reservation=11.81MB thread-reservation=1 36:AGGREGATE [STREAMING] | Class 0 | output: sum(sales), sum(returns), sum(profit) @@ -560,7 +560,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | | in pipelines: 29(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=80.10MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=80.34MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=2.00MB | | 34:AGGREGATE [STREAMING] | | | output: sum(wr_return_amt), sum(wr_net_loss) | | | group by: wp_web_page_sk @@ -582,7 +582,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | | | in pipelines: 31(GETNEXT) | | | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=2 | | | 31:SCAN HDFS [tpcds_parquet.web_page, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.56KB | | | stored statistics: @@ -607,7 +607,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | | | in pipelines: 30(GETNEXT) | | | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | | 30:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -645,7 +645,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | in pipelines: 23(GETNEXT) | | | F14:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=144.10MB mem-reservation=15.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=144.34MB mem-reservation=15.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 28:AGGREGATE [STREAMING] | | output: sum(ws_ext_sales_price), sum(ws_net_profit) | | group by: wp_web_page_sk @@ -667,7 +667,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | | in pipelines: 25(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=2 | | 25:SCAN HDFS [tpcds_parquet.web_page, RANDOM] | | HDFS partitions=1/1 files=1 size=5.56KB | | stored statistics: @@ -692,7 +692,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | | in pipelines: 24(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -728,7 +728,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | | in pipelines: 52(GETNEXT) | | | | | F13:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | Per-Host Resources: mem-estimate=160.00KB mem-reservation=0B thread-reservation=1 | | 52:AGGREGATE [FINALIZE] | | | output: sum:merge(cr_return_amount), sum:merge(cr_net_loss) | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -762,7 +762,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | | | in pipelines: 19(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | | 19:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -800,7 +800,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | in pipelines: 14(GETNEXT) | | | F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=205.15MB mem-reservation=12.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=205.41MB mem-reservation=12.94MB thread-reservation=2 runtime-filters-memory=1.00MB | 17:AGGREGATE [STREAMING] | | output: sum(cs_ext_sales_price), sum(cs_net_profit) | | group by: cs_call_center_sk @@ -822,7 +822,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | | in pipelines: 15(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -867,7 +867,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | in pipelines: 07(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=112.10MB mem-reservation=11.88MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=112.34MB mem-reservation=11.88MB thread-reservation=2 runtime-filters-memory=2.00MB | 12:AGGREGATE [STREAMING] | | output: sum(sr_return_amt), sum(sr_net_loss) | | group by: s_store_sk @@ -889,7 +889,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | | in pipelines: 09(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=2 | | 09:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -914,7 +914,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | | | in pipelines: 08(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | 08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -952,7 +952,7 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=11.81MB thread-reservat | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=64.10MB mem-reservation=9.88MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=64.34MB mem-reservation=9.88MB thread-reservation=2 runtime-filters-memory=2.00MB 06:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price), sum(ss_net_profit) | group by: s_store_sk @@ -974,7 +974,7 @@ Per-Host Resources: mem-estimate=64.10MB mem-reservation=9.88MB thread-reservati | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -999,7 +999,7 @@ Per-Host Resources: mem-estimate=64.10MB mem-reservation=9.88MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -1026,7 +1026,7 @@ Per-Host Resources: mem-estimate=64.10MB mem-reservation=9.88MB thread-reservati in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=154.67MB Threads=38 -Per-Host Resource Estimates: Memory=668MB +Per-Host Resource Estimates: Memory=684MB F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -1075,7 +1075,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reser | in pipelines: 42(GETNEXT), 49(GETNEXT), 57(GETNEXT) | F22:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=44.80MB mem-reservation=7.94MB thread-reservation=1 36:AGGREGATE [STREAMING] | Class 0 | output: sum(sales), sum(returns), sum(profit) @@ -1124,7 +1124,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=26.94MB mem-reservation=3.00MB thread-reservation=1 | | 34:AGGREGATE [STREAMING] | | | output: sum(wr_return_amt), sum(wr_net_loss) | | | group by: wp_web_page_sk @@ -1154,7 +1154,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | | in pipelines: 31(GETNEXT) | | | | | | | F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=1 | | | 31:SCAN HDFS [tpcds_parquet.web_page, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.56KB | | | stored statistics: @@ -1187,7 +1187,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | | in pipelines: 30(GETNEXT) | | | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | | 30:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -1226,7 +1226,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | F14:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=42.00MB mem-reservation=10.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=42.94MB mem-reservation=10.00MB thread-reservation=1 | 28:AGGREGATE [STREAMING] | | output: sum(ws_ext_sales_price), sum(ws_net_profit) | | group by: wp_web_page_sk @@ -1256,7 +1256,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | in pipelines: 25(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=1 | | 25:SCAN HDFS [tpcds_parquet.web_page, RANDOM] | | HDFS partitions=1/1 files=1 size=5.56KB | | stored statistics: @@ -1289,7 +1289,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | in pipelines: 24(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -1332,7 +1332,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | in pipelines: 52(GETNEXT) | | | | | F13:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| | Per-Instance Resources: mem-estimate=160.00KB mem-reservation=0B thread-reservation=1 | | 52:AGGREGATE [FINALIZE] | | | output: sum:merge(cr_return_amount), sum:merge(cr_net_loss) | | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1375,7 +1375,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | | in pipelines: 19(GETNEXT) | | | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | | 19:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -1414,7 +1414,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=58.00MB mem-reservation=10.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=58.94MB mem-reservation=10.00MB thread-reservation=1 | 17:AGGREGATE [STREAMING] | | output: sum(cs_ext_sales_price), sum(cs_net_profit) | | group by: cs_call_center_sk @@ -1444,7 +1444,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | in pipelines: 15(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -1498,7 +1498,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=34.00MB mem-reservation=6.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=34.94MB mem-reservation=6.00MB thread-reservation=1 | 12:AGGREGATE [STREAMING] | | output: sum(sr_return_amt), sum(sr_net_loss) | | group by: s_store_sk @@ -1528,7 +1528,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | in pipelines: 09(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=1 | | 09:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | stored statistics: @@ -1561,7 +1561,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | | | in pipelines: 08(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -1600,7 +1600,7 @@ Per-Instance Resources: mem-estimate=40.02MB mem-reservation=7.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.94MB mem-reservation=4.00MB thread-reservation=1 06:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price), sum(ss_net_profit) | group by: s_store_sk @@ -1630,7 +1630,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.03MB mem-reservation=16.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -1663,7 +1663,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q78.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q78.test index 85d356cb2..3fc915656 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q78.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q78.test @@ -376,7 +376,7 @@ Per-Host Resources: mem-estimate=20.75MB mem-reservation=13.44MB thread-reservat | | | | in pipelines: 02(GETNEXT) | | | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT) @@ -404,7 +404,7 @@ Per-Host Resources: mem-estimate=20.75MB mem-reservation=13.44MB thread-reservat | | | | in pipelines: 01(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=48.08MB mem-reservation=4.00MB thread-reservation=2 | | | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=15.43MB | | | stored statistics: @@ -462,7 +462,7 @@ Per-Host Resources: mem-estimate=20.75MB mem-reservation=13.44MB thread-reservat | | | in pipelines: 08(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.05MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | | 08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT) @@ -491,7 +491,7 @@ Per-Host Resources: mem-estimate=20.75MB mem-reservation=13.44MB thread-reservat | | | in pipelines: 07(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.08MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 07:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=5.66MB | | runtime filters: RF013[min_max] -> tpcds_parquet.web_returns.wr_item_sk, RF010[bloom] -> tpcds_parquet.web_returns.wr_item_sk @@ -549,7 +549,7 @@ Per-Host Resources: mem-estimate=361.67MB mem-reservation=38.44MB thread-reserva | | in pipelines: 14(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.05MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | 14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT) @@ -578,7 +578,7 @@ Per-Host Resources: mem-estimate=361.67MB mem-reservation=38.44MB thread-reserva | | in pipelines: 13(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=49.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=49.08MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 13:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | HDFS partitions=1/1 files=1 size=10.62MB | runtime filters: RF005[min_max] -> tpcds_parquet.catalog_returns.cr_item_sk, RF002[bloom] -> tpcds_parquet.catalog_returns.cr_item_sk @@ -602,7 +602,7 @@ Per-Host Resources: mem-estimate=361.67MB mem-reservation=38.44MB thread-reserva in pipelines: 12(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=214.50MB Threads=23 -Per-Host Resource Estimates: Memory=443MB +Per-Host Resource Estimates: Memory=447MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -687,7 +687,7 @@ Per-Instance Resources: mem-estimate=15.42MB mem-reservation=8.50MB thread-reser | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=26.00MB mem-reservation=13.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=27.41MB mem-reservation=13.00MB thread-reservation=1 | | 05:AGGREGATE [STREAMING] | | | output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price) | | | group by: d_year, ss_item_sk, ss_customer_sk @@ -717,7 +717,7 @@ Per-Instance Resources: mem-estimate=15.42MB mem-reservation=8.50MB thread-reser | | | | in pipelines: 02(GETNEXT) | | | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT) @@ -753,7 +753,7 @@ Per-Instance Resources: mem-estimate=15.42MB mem-reservation=8.50MB thread-reser | | | | in pipelines: 01(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=24.08MB mem-reservation=4.00MB thread-reservation=1 | | | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=15.43MB | | | stored statistics: @@ -790,7 +790,7 @@ Per-Instance Resources: mem-estimate=15.42MB mem-reservation=8.50MB thread-reser | | | F04:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=42.00MB mem-reservation=21.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=42.47MB mem-reservation=21.00MB thread-reservation=1 | 11:AGGREGATE [STREAMING] | | output: sum(CAST(ws_quantity AS BIGINT)), sum(ws_wholesale_cost), sum(ws_sales_price) | | group by: d_year, ws_item_sk, ws_bill_customer_sk @@ -821,7 +821,7 @@ Per-Instance Resources: mem-estimate=15.42MB mem-reservation=8.50MB thread-reser | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT) @@ -859,7 +859,7 @@ Per-Instance Resources: mem-estimate=15.42MB mem-reservation=8.50MB thread-reser | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.08MB mem-reservation=1.00MB thread-reservation=1 | | 07:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=5.66MB | | runtime filters: RF013[min_max] -> tpcds_parquet.web_returns.wr_item_sk, RF010[bloom] -> tpcds_parquet.web_returns.wr_item_sk @@ -896,7 +896,7 @@ Per-Instance Resources: mem-estimate=15.42MB mem-reservation=8.50MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=58.00MB mem-reservation=25.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=58.70MB mem-reservation=25.00MB thread-reservation=1 17:AGGREGATE [STREAMING] | output: sum(CAST(cs_quantity AS BIGINT)), sum(cs_wholesale_cost), sum(cs_sales_price) | group by: d_year, cs_item_sk, cs_bill_customer_sk @@ -927,7 +927,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=25.00MB thread-rese | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT) @@ -965,7 +965,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=25.00MB thread-rese | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.08MB mem-reservation=2.00MB thread-reservation=1 | 13:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | HDFS partitions=1/1 files=1 size=10.62MB | runtime filters: RF005[min_max] -> tpcds_parquet.catalog_returns.cr_item_sk, RF002[bloom] -> tpcds_parquet.catalog_returns.cr_item_sk diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q79.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q79.test index 7cb818403..999dd7c75 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q79.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q79.test @@ -157,7 +157,7 @@ PLAN-ROOT SINK in pipelines: 08(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=30.52MB Threads=13 -Per-Host Resource Estimates: Memory=334MB +Per-Host Resource Estimates: Memory=335MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -194,7 +194,7 @@ Per-Host Resources: mem-estimate=10.80MB mem-reservation=5.75MB thread-reservati | | in pipelines: 08(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.52MB mem-reservation=1.00MB thread-reservation=2 | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | stored statistics: @@ -225,7 +225,7 @@ Per-Host Resources: mem-estimate=11.20MB mem-reservation=2.88MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=132.06MB mem-reservation=15.81MB thread-reservation=2 runtime-filters-memory=4.00MB +Per-Host Resources: mem-estimate=132.64MB mem-reservation=15.81MB thread-reservation=2 runtime-filters-memory=4.00MB 07:AGGREGATE [STREAMING] | output: sum(ss_coupon_amt), sum(ss_net_profit) | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, store.s_city @@ -247,7 +247,7 @@ Per-Host Resources: mem-estimate=132.06MB mem-reservation=15.81MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_date <= '1999-03-31', d_date >= '1999-01-01' @@ -275,7 +275,7 @@ Per-Host Resources: mem-estimate=132.06MB mem-reservation=15.81MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.12MB mem-reservation=24.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_number_employees <= CAST(295 AS INT), store.s_number_employees >= CAST(200 AS INT) @@ -303,7 +303,7 @@ Per-Host Resources: mem-estimate=132.06MB mem-reservation=15.81MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=64.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: (household_demographics.hd_dep_count = CAST(8 AS INT) OR household_demographics.hd_vehicle_count > CAST(0 AS INT)) @@ -328,7 +328,7 @@ Per-Host Resources: mem-estimate=132.06MB mem-reservation=15.81MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=48.34MB Threads=16 -Per-Host Resource Estimates: Memory=180MB +Per-Host Resource Estimates: Memory=185MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -373,7 +373,7 @@ Per-Instance Resources: mem-estimate=1.41MB mem-reservation=0B thread-reservatio | | in pipelines: 08(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.03MB mem-reservation=1.00MB thread-reservation=1 | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | stored statistics: @@ -390,7 +390,7 @@ Per-Instance Resources: mem-estimate=1.41MB mem-reservation=0B thread-reservatio | in pipelines: 15(GETNEXT) | F04:PLAN FRAGMENT [HASH(ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city)] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=11.40MB mem-reservation=1.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=11.65MB mem-reservation=1.94MB thread-reservation=1 15:AGGREGATE [FINALIZE] | output: sum:merge(ss_coupon_amt), sum:merge(ss_net_profit) | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, store.s_city @@ -405,7 +405,7 @@ Per-Instance Resources: mem-estimate=11.40MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=27.65MB mem-reservation=6.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | output: sum(ss_coupon_amt), sum(ss_net_profit) | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, store.s_city @@ -435,7 +435,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_date <= '1999-03-31', d_date >= '1999-01-01' @@ -471,7 +471,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=24.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_number_employees <= CAST(295 AS INT), store.s_number_employees >= CAST(200 AS INT) @@ -507,7 +507,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=64.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: (household_demographics.hd_dep_count = CAST(8 AS INT) OR household_demographics.hd_vehicle_count > CAST(0 AS INT)) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test index e2931ffc2..636215d55 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test @@ -486,7 +486,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=146.31MB Threads=39 -Per-Host Resource Estimates: Memory=1.45GB +Per-Host Resource Estimates: Memory=1.46GB F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -535,7 +535,7 @@ Per-Host Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reservati | in pipelines: 46(GETNEXT), 53(GETNEXT), 60(GETNEXT) | F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservation=1 +Per-Host Resources: mem-estimate=43.19MB mem-reservation=7.94MB thread-reservation=1 37:AGGREGATE [STREAMING] | Class 0 | output: sum(sales), sum(returns), sum(profit) @@ -590,7 +590,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 28(GETNEXT) | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.14MB mem-reservation=16.00KB thread-reservation=2 | | 28:SCAN HDFS [tpcds_parquet.web_site, RANDOM] | | HDFS partitions=1/1 files=1 size=11.91KB | | stored statistics: @@ -615,7 +615,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 27(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | 27:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -643,7 +643,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 30(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.08MB mem-reservation=32.00KB thread-reservation=2 | | 30:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | | HDFS partitions=1/1 files=1 size=23.30KB | | predicates: p_channel_tv = 'N' @@ -671,7 +671,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 29(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.06MB mem-reservation=256.00KB thread-reservation=2 | | 29:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_current_price > CAST(50 AS DECIMAL(3,0)) @@ -698,7 +698,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 26(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=65.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=65.11MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 26:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=5.66MB | | runtime filters: RF023[min_max] -> tpcds_parquet.web_returns.wr_item_sk, RF022[bloom] -> tpcds_parquet.web_returns.wr_item_sk @@ -756,7 +756,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 16(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.14MB mem-reservation=256.00KB thread-reservation=2 | | 16:SCAN HDFS [tpcds_parquet.catalog_page, RANDOM] | | HDFS partitions=1/1 files=1 size=739.17KB | | stored statistics: @@ -781,7 +781,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 15(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -809,7 +809,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 18(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.08MB mem-reservation=32.00KB thread-reservation=2 | | 18:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | | HDFS partitions=1/1 files=1 size=23.30KB | | predicates: p_channel_tv = 'N' @@ -837,7 +837,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 17(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.06MB mem-reservation=256.00KB thread-reservation=2 | | 17:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_current_price > CAST(50 AS DECIMAL(3,0)) @@ -864,7 +864,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati | | | in pipelines: 14(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=97.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=97.11MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 14:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=10.62MB | | runtime filters: RF015[min_max] -> tpcds_parquet.catalog_returns.cr_item_sk, RF014[bloom] -> tpcds_parquet.catalog_returns.cr_item_sk @@ -922,7 +922,7 @@ Per-Host Resources: mem-estimate=141.64MB mem-reservation=34.75MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.14MB mem-reservation=16.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -947,7 +947,7 @@ Per-Host Resources: mem-estimate=141.64MB mem-reservation=34.75MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -975,7 +975,7 @@ Per-Host Resources: mem-estimate=141.64MB mem-reservation=34.75MB thread-reserva | | in pipelines: 06(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.08MB mem-reservation=32.00KB thread-reservation=2 | 06:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | predicates: p_channel_tv = 'N' @@ -1003,7 +1003,7 @@ Per-Host Resources: mem-estimate=141.64MB mem-reservation=34.75MB thread-reserva | | in pipelines: 05(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.06MB mem-reservation=256.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_current_price > CAST(50 AS DECIMAL(3,0)) @@ -1030,7 +1030,7 @@ Per-Host Resources: mem-estimate=141.64MB mem-reservation=34.75MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=97.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=97.11MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 02:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | runtime filters: RF007[min_max] -> tpcds_parquet.store_returns.sr_item_sk, RF006[bloom] -> tpcds_parquet.store_returns.sr_item_sk @@ -1055,7 +1055,7 @@ Per-Host Resources: mem-estimate=141.64MB mem-reservation=34.75MB thread-reserva in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=231.50MB Threads=39 -Per-Host Resource Estimates: Memory=716MB +Per-Host Resource Estimates: Memory=736MB F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -1104,7 +1104,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reser | in pipelines: 46(GETNEXT), 53(GETNEXT), 60(GETNEXT) | F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservation=1 +Per-Instance Resources: mem-estimate=45.86MB mem-reservation=7.94MB thread-reservation=1 37:AGGREGATE [STREAMING] | Class 0 | output: sum(sales), sum(returns), sum(profit) @@ -1138,7 +1138,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | F14:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=42.00MB mem-reservation=18.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=43.88MB mem-reservation=18.00MB thread-reservation=1 | 36:AGGREGATE [STREAMING] | | output: sum(ws_ext_sales_price), sum(coalesce(wr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(ws_net_profit - coalesce(wr_net_loss, CAST(0 AS DECIMAL(7,2)))) | | group by: web_site_id @@ -1168,7 +1168,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | in pipelines: 28(GETNEXT) | | | | | F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1 | | 28:SCAN HDFS [tpcds_parquet.web_site, RANDOM] | | HDFS partitions=1/1 files=1 size=11.91KB | | stored statistics: @@ -1201,7 +1201,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | in pipelines: 27(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 27:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -1237,7 +1237,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | in pipelines: 30(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.08MB mem-reservation=32.00KB thread-reservation=1 | | 30:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | | HDFS partitions=1/1 files=1 size=23.30KB | | predicates: p_channel_tv = 'N' @@ -1273,7 +1273,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | in pipelines: 29(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=256.00KB thread-reservation=1 | | 29:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_current_price > CAST(50 AS DECIMAL(3,0)) @@ -1309,7 +1309,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.11MB mem-reservation=2.00MB thread-reservation=1 | | 26:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=5.66MB | | runtime filters: RF023[min_max] -> tpcds_parquet.web_returns.wr_item_sk, RF022[bloom] -> tpcds_parquet.web_returns.wr_item_sk @@ -1346,7 +1346,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -| Per-Instance Resources: mem-estimate=58.00MB mem-reservation=18.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=59.88MB mem-reservation=18.00MB thread-reservation=1 | 24:AGGREGATE [STREAMING] | | output: sum(cs_ext_sales_price), sum(coalesce(cr_return_amount, CAST(0 AS DECIMAL(7,2)))), sum(cs_net_profit - coalesce(cr_net_loss, CAST(0 AS DECIMAL(7,2)))) | | group by: cp_catalog_page_id @@ -1376,7 +1376,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | in pipelines: 16(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.14MB mem-reservation=256.00KB thread-reservation=1 | | 16:SCAN HDFS [tpcds_parquet.catalog_page, RANDOM] | | HDFS partitions=1/1 files=1 size=739.17KB | | stored statistics: @@ -1409,7 +1409,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | in pipelines: 15(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -1445,7 +1445,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | in pipelines: 18(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.08MB mem-reservation=32.00KB thread-reservation=1 | | 18:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | | HDFS partitions=1/1 files=1 size=23.30KB | | predicates: p_channel_tv = 'N' @@ -1481,7 +1481,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | in pipelines: 17(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=256.00KB thread-reservation=1 | | 17:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_current_price > CAST(50 AS DECIMAL(3,0)) @@ -1517,7 +1517,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.11MB mem-reservation=4.00MB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=10.62MB | | runtime filters: RF015[min_max] -> tpcds_parquet.catalog_returns.cr_item_sk, RF014[bloom] -> tpcds_parquet.catalog_returns.cr_item_sk @@ -1554,7 +1554,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=27.88MB mem-reservation=6.00MB thread-reservation=1 12:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price), sum(coalesce(sr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(ss_net_profit - coalesce(sr_net_loss, CAST(0 AS DECIMAL(7,2)))) | group by: s_store_id @@ -1584,7 +1584,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -1617,7 +1617,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04' @@ -1653,7 +1653,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 06(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.08MB mem-reservation=32.00KB thread-reservation=1 | 06:SCAN HDFS [tpcds_parquet.promotion, RANDOM] | HDFS partitions=1/1 files=1 size=23.30KB | predicates: p_channel_tv = 'N' @@ -1689,7 +1689,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 05(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=256.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_current_price > CAST(50 AS DECIMAL(3,0)) @@ -1725,7 +1725,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.11MB mem-reservation=8.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | runtime filters: RF007[min_max] -> tpcds_parquet.store_returns.sr_item_sk, RF006[bloom] -> tpcds_parquet.store_returns.sr_item_sk diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test index 8a487b929..224e533fb 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test @@ -242,7 +242,7 @@ PLAN-ROOT SINK in pipelines: 02(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=52.62MB Threads=20 -Per-Host Resource Estimates: Memory=637MB +Per-Host Resource Estimates: Memory=639MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -278,7 +278,7 @@ Per-Host Resources: mem-estimate=17.68MB mem-reservation=8.75MB thread-reservati | | in pipelines: 30(GETNEXT) | | | F10:PLAN FRAGMENT [HASH(ctr2.ctr_state)] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Host Resources: mem-estimate=10.13MB mem-reservation=1.94MB thread-reservation=1 | 30:AGGREGATE [FINALIZE] | | output: avg:merge(ctr_total_return) | | group by: ctr2.ctr_state @@ -335,7 +335,7 @@ Per-Host Resources: mem-estimate=17.68MB mem-reservation=8.75MB thread-reservati | | | in pipelines: 08(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=98.95MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=99.06MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 11:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: cr_returned_date_sk = d_date_sk | | | fk/pk conjuncts: cr_returned_date_sk = d_date_sk @@ -350,7 +350,7 @@ Per-Host Resources: mem-estimate=17.68MB mem-reservation=8.75MB thread-reservati | | | | in pipelines: 09(GETNEXT) | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | | 09:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2000 AS INT) @@ -400,7 +400,7 @@ Per-Host Resources: mem-estimate=17.68MB mem-reservation=8.75MB thread-reservati | | in pipelines: 07(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=99.31MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=100.10MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB | 15:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: c_current_addr_sk = ca_address_sk | | fk/pk conjuncts: c_current_addr_sk = ca_address_sk @@ -415,7 +415,7 @@ Per-Host Resources: mem-estimate=17.68MB mem-reservation=8.75MB thread-reservati | | | in pipelines: 06(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=128.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=128.81MB mem-reservation=1.00MB thread-reservation=2 | | 06:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_state = 'GA' @@ -475,7 +475,7 @@ Per-Host Resources: mem-estimate=47.51MB mem-reservation=7.12MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=99.95MB mem-reservation=5.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=100.06MB mem-reservation=5.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 03:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: cr_returned_date_sk = d_date_sk | | fk/pk conjuncts: cr_returned_date_sk = d_date_sk @@ -490,7 +490,7 @@ Per-Host Resources: mem-estimate=47.51MB mem-reservation=7.12MB thread-reservati | | | in pipelines: 01(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT) @@ -527,7 +527,7 @@ Per-Host Resources: mem-estimate=47.51MB mem-reservation=7.12MB thread-reservati in pipelines: 02(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=73.06MB Threads=19 -Per-Host Resource Estimates: Memory=257MB +Per-Host Resource Estimates: Memory=260MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -571,7 +571,7 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser | | in pipelines: 30(GETNEXT) | | | F10:PLAN FRAGMENT [HASH(ctr2.ctr_state)] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=10.13MB mem-reservation=1.94MB thread-reservation=1 | 30:AGGREGATE [FINALIZE] | | output: avg:merge(ctr_total_return) | | group by: ctr2.ctr_state @@ -607,7 +607,7 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.25MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.15MB mem-reservation=2.25MB thread-reservation=1 | 13:AGGREGATE [STREAMING] | | output: sum(cr_return_amt_inc_tax) | | group by: cr_returning_customer_sk, ca_state @@ -638,7 +638,7 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=24.12MB mem-reservation=2.00MB thread-reservation=1 | | 11:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=02 | | | hash predicates: cr_returned_date_sk = d_date_sk @@ -661,7 +661,7 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser | | | | in pipelines: 09(GETNEXT) | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | | 09:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_year = CAST(2000 AS INT) @@ -720,7 +720,7 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=17.17MB mem-reservation=4.00MB thread-reservation=1 | 15:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=04 | | hash predicates: c_current_addr_sk = ca_address_sk @@ -743,7 +743,7 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser | | | in pipelines: 06(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.81MB mem-reservation=1.00MB thread-reservation=1 | | 06:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_state = 'GA' @@ -782,7 +782,7 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.25MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.15MB mem-reservation=2.25MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(cr_return_amt_inc_tax) | group by: cr_returning_customer_sk, ca_state @@ -813,7 +813,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.25MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.12MB mem-reservation=2.00MB thread-reservation=1 | 03:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=06 | | hash predicates: cr_returned_date_sk = d_date_sk @@ -836,7 +836,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.25MB thread-reser | | | in pipelines: 01(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q82.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q82.test index 5a286d68e..40a97a41a 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q82.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q82.test @@ -121,7 +121,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=100.31MB Threads=10 -Per-Host Resource Estimates: Memory=402MB +Per-Host Resource Estimates: Memory=403MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -176,7 +176,7 @@ Per-Host Resources: mem-estimate=168.99MB mem-reservation=55.88MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '2000-07-24', CAST(d_date AS DATE) >= DATE '2000-05-25' @@ -204,7 +204,7 @@ Per-Host Resources: mem-estimate=168.99MB mem-reservation=55.88MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=18.95MB mem-reservation=3.44MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=19.61MB mem-reservation=3.44MB thread-reservation=2 runtime-filters-memory=1.00MB | 04:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ss_item_sk = i_item_sk | | fk/pk conjuncts: ss_item_sk = i_item_sk @@ -219,7 +219,7 @@ Per-Host Resources: mem-estimate=168.99MB mem-reservation=55.88MB thread-reserva | | | in pipelines: 00(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=80.63MB mem-reservation=2.00MB thread-reservation=2 | | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_current_price <= CAST(92 AS DECIMAL(3,0)), i_current_price >= CAST(62 AS DECIMAL(3,0)), i_manufact_id IN (CAST(129 AS INT), CAST(270 AS INT), CAST(821 AS INT), CAST(423 AS INT)) @@ -260,7 +260,7 @@ Per-Host Resources: mem-estimate=168.99MB mem-reservation=55.88MB thread-reserva in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=109.62MB Threads=10 -Per-Host Resource Estimates: Memory=283MB +Per-Host Resource Estimates: Memory=287MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -295,7 +295,7 @@ Per-Instance Resources: mem-estimate=98.02MB mem-reservation=34.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=97.80MB mem-reservation=50.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=98.96MB mem-reservation=50.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | group by: i_item_id, i_item_desc, i_current_price | mem-estimate=65.80MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -324,7 +324,7 @@ Per-Instance Resources: mem-estimate=97.80MB mem-reservation=50.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '2000-07-24', CAST(d_date AS DATE) >= DATE '2000-05-25' @@ -361,7 +361,7 @@ Per-Instance Resources: mem-estimate=97.80MB mem-reservation=50.00MB thread-rese | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.67MB mem-reservation=512.00KB thread-reservation=1 | 04:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=02 | | hash predicates: ss_item_sk = i_item_sk @@ -384,7 +384,7 @@ Per-Instance Resources: mem-estimate=97.80MB mem-reservation=50.00MB thread-rese | | | in pipelines: 00(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.63MB mem-reservation=2.00MB thread-reservation=1 | | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_current_price <= CAST(92 AS DECIMAL(3,0)), i_current_price >= CAST(62 AS DECIMAL(3,0)), i_manufact_id IN (CAST(129 AS INT), CAST(270 AS INT), CAST(821 AS INT), CAST(423 AS INT)) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q83.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q83.test index b84ca5b63..d63d5631b 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q83.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q83.test @@ -383,7 +383,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=78.38MB Threads=32 -Per-Host Resource Estimates: Memory=695MB +Per-Host Resource Estimates: Memory=697MB F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -447,7 +447,7 @@ Per-Host Resources: mem-estimate=34.89MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 13(GETNEXT) | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=34.95MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=35.05MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 15:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | | hash predicates: d_week_seq = d_week_seq | | | runtime filters: RF026[bloom] <- d_week_seq, RF027[min_max] <- d_week_seq @@ -461,7 +461,7 @@ Per-Host Resources: mem-estimate=34.89MB mem-reservation=10.69MB thread-reservat | | | | in pipelines: 14(GETNEXT) | | | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | | 14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_date IN ('2000-06-30', '2000-09-27', '2000-11-17') @@ -500,7 +500,7 @@ Per-Host Resources: mem-estimate=34.89MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 12(GETNEXT) | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=33.12MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF021[min_max] -> d_date, RF020[bloom] -> d_date @@ -526,7 +526,7 @@ Per-Host Resources: mem-estimate=34.89MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 11(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.16MB mem-reservation=512.00KB thread-reservation=2 | | 11:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -590,7 +590,7 @@ Per-Host Resources: mem-estimate=34.89MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 23(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=34.95MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=35.05MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 25:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | | hash predicates: d_week_seq = d_week_seq | | | runtime filters: RF018[bloom] <- d_week_seq, RF019[min_max] <- d_week_seq @@ -604,7 +604,7 @@ Per-Host Resources: mem-estimate=34.89MB mem-reservation=10.69MB thread-reservat | | | | in pipelines: 24(GETNEXT) | | | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_date IN ('2000-06-30', '2000-09-27', '2000-11-17') @@ -643,7 +643,7 @@ Per-Host Resources: mem-estimate=34.89MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 21(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.16MB mem-reservation=512.00KB thread-reservation=2 | | 21:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_id @@ -669,7 +669,7 @@ Per-Host Resources: mem-estimate=34.89MB mem-reservation=10.69MB thread-reservat | | | in pipelines: 20(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.08MB mem-reservation=512.00KB thread-reservation=2 | | 20:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=5.66MB | | runtime filters: RF015[min_max] -> wr_item_sk @@ -726,7 +726,7 @@ Per-Host Resources: mem-estimate=96.14MB mem-reservation=15.62MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=34.95MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=35.05MB mem-reservation=3.94MB thread-reservation=2 runtime-filters-memory=1.00MB | 05:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | hash predicates: d_week_seq = d_week_seq | | runtime filters: RF010[bloom] <- d_week_seq, RF011[min_max] <- d_week_seq @@ -740,7 +740,7 @@ Per-Host Resources: mem-estimate=96.14MB mem-reservation=15.62MB thread-reservat | | | in pipelines: 04(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_date IN ('2000-06-30', '2000-09-27', '2000-11-17') @@ -779,7 +779,7 @@ Per-Host Resources: mem-estimate=96.14MB mem-reservation=15.62MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.12MB mem-reservation=2.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF005[min_max] -> d_date, RF004[bloom] -> d_date @@ -805,7 +805,7 @@ Per-Host Resources: mem-estimate=96.14MB mem-reservation=15.62MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=33.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=33.16MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF003[min_max] -> tpcds_parquet.item.i_item_id, RF001[min_max] -> tpcds_parquet.item.i_item_id, RF002[bloom] -> tpcds_parquet.item.i_item_id @@ -829,7 +829,7 @@ Per-Host Resources: mem-estimate=96.14MB mem-reservation=15.62MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=117.06MB Threads=31 -Per-Host Resource Estimates: Memory=414MB +Per-Host Resource Estimates: Memory=416MB F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -880,7 +880,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | | in pipelines: 10(GETNEXT) | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=34.00MB mem-reservation=3.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=34.16MB mem-reservation=3.00MB thread-reservation=1 | 19:AGGREGATE [STREAMING] | | output: sum(CAST(cr_return_quantity AS BIGINT)) | | group by: i_item_id @@ -910,7 +910,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | | | | | F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 15:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | | hash-table-id=02 | | | hash predicates: d_week_seq = d_week_seq @@ -932,7 +932,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 14(GETNEXT) | | | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | | 14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_date IN ('2000-06-30', '2000-09-27', '2000-11-17') @@ -980,7 +980,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | | | | | F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | runtime filters: RF021[min_max] -> d_date, RF020[bloom] -> d_date @@ -1014,7 +1014,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | | | in pipelines: 11(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | | 11:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | stored statistics: @@ -1066,7 +1066,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=26.16MB mem-reservation=3.00MB thread-reservation=1 | 29:AGGREGATE [STREAMING] | | output: sum(CAST(wr_return_quantity AS BIGINT)) | | group by: i_item_id @@ -1096,7 +1096,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 25:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | | hash-table-id=07 | | | hash predicates: d_week_seq = d_week_seq @@ -1118,7 +1118,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | | | | in pipelines: 24(GETNEXT) | | | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | | 24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: d_date IN ('2000-06-30', '2000-09-27', '2000-11-17') @@ -1165,7 +1165,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | | | in pipelines: 21(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | | 21:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_id @@ -1199,7 +1199,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | | | in pipelines: 20(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.08MB mem-reservation=512.00KB thread-reservation=1 | | 20:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | HDFS partitions=1/1 files=1 size=5.66MB | | runtime filters: RF015[min_max] -> wr_item_sk @@ -1236,7 +1236,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=34.16MB mem-reservation=4.00MB thread-reservation=1 09:AGGREGATE [STREAMING] | output: sum(CAST(sr_return_quantity AS BIGINT)) | group by: i_item_id @@ -1266,7 +1266,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 05:HASH JOIN [LEFT SEMI JOIN, BROADCAST] | | hash-table-id=11 | | hash predicates: d_week_seq = d_week_seq @@ -1288,7 +1288,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser | | | in pipelines: 04(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_date IN ('2000-06-30', '2000-09-27', '2000-11-17') @@ -1336,7 +1336,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | runtime filters: RF005[min_max] -> d_date, RF004[bloom] -> d_date @@ -1371,7 +1371,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | runtime filters: RF003[min_max] -> tpcds_parquet.item.i_item_id, RF001[min_max] -> tpcds_parquet.item.i_item_id, RF002[bloom] -> tpcds_parquet.item.i_item_id diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test index 7b7ddda38..f35c80a06 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test @@ -122,7 +122,7 @@ PLAN-ROOT SINK in pipelines: 04(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=22.34MB Threads=11 -Per-Host Resource Estimates: Memory=248MB +Per-Host Resource Estimates: Memory=249MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -158,7 +158,7 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=4.94MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=37.92MB mem-reservation=5.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=38.41MB mem-reservation=5.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 07:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: hd_income_band_sk = ib_income_band_sk | | fk/pk conjuncts: hd_income_band_sk = ib_income_band_sk @@ -173,7 +173,7 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=4.94MB thread-reservati | | | in pipelines: 03(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=24.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.income_band, RANDOM] | | HDFS partitions=1/1 files=1 size=1.21KB | | predicates: ib_lower_bound >= CAST(38128 AS INT), ib_upper_bound <= CAST(88128 AS INT) @@ -201,7 +201,7 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=4.94MB thread-reservati | | | in pipelines: 00(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=98.95MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=99.36MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB | | 05:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: c_current_addr_sk = ca_address_sk | | | fk/pk conjuncts: c_current_addr_sk = ca_address_sk @@ -216,7 +216,7 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=4.94MB thread-reservati | | | | in pipelines: 01(GETNEXT) | | | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.11MB mem-reservation=512.00KB thread-reservation=2 | | | 01:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.16MB | | | predicates: ca_city = 'Edgewood' @@ -264,7 +264,7 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=4.94MB thread-reservati in pipelines: 04(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=34.09MB Threads=10 -Per-Host Resource Estimates: Memory=116MB +Per-Host Resource Estimates: Memory=117MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -310,7 +310,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.53MB mem-reservation=64.00KB thread-reservation=1 | 07:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=01 | | hash predicates: hd_income_band_sk = ib_income_band_sk @@ -333,7 +333,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser | | | in pipelines: 03(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=24.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.income_band, RANDOM] | | HDFS partitions=1/1 files=1 size=1.21KB | | predicates: ib_lower_bound >= CAST(38128 AS INT), ib_upper_bound <= CAST(88128 AS INT) @@ -370,7 +370,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.43MB mem-reservation=4.00MB thread-reservation=1 | | 05:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=03 | | | hash predicates: c_current_addr_sk = ca_address_sk @@ -393,7 +393,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser | | | | in pipelines: 01(GETNEXT) | | | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.11MB mem-reservation=512.00KB thread-reservation=1 | | | 01:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.16MB | | | predicates: ca_city = 'Edgewood' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test index c38cd94cf..303b1e3ff 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test @@ -234,7 +234,7 @@ PLAN-ROOT SINK in pipelines: 04(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=83.65MB Threads=20 -Per-Host Resource Estimates: Memory=650MB +Per-Host Resource Estimates: Memory=651MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -270,7 +270,7 @@ Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservati | in pipelines: 04(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservation=2 runtime-filters-memory=4.00MB +Per-Host Resources: mem-estimate=66.10MB mem-reservation=17.88MB thread-reservation=2 runtime-filters-memory=4.00MB 15:AGGREGATE [STREAMING] | output: avg(CAST(ws_quantity AS BIGINT)), avg(wr_refunded_cash), avg(wr_fee) | group by: r_reason_desc @@ -292,7 +292,7 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat | | in pipelines: 07(GETNEXT) | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.14MB mem-reservation=16.00KB thread-reservation=2 | 07:SCAN HDFS [tpcds_parquet.reason, RANDOM] | HDFS partitions=1/1 files=1 size=1.92KB | stored statistics: @@ -317,7 +317,7 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat | | in pipelines: 00(GETNEXT) | | | F07:PLAN FRAGMENT [HASH(wr_refunded_addr_sk)] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=6.66MB mem-reservation=5.88MB thread-reservation=1 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=6.99MB mem-reservation=5.88MB thread-reservation=1 runtime-filters-memory=2.00MB | 12:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_web_page_sk = wp_web_page_sk | | fk/pk conjuncts: ws_web_page_sk = wp_web_page_sk @@ -332,7 +332,7 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat | | | in pipelines: 02(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=2 | | 02:SCAN HDFS [tpcds_parquet.web_page, RANDOM] | | HDFS partitions=1/1 files=1 size=5.56KB | | stored statistics: @@ -358,7 +358,7 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat | | | in pipelines: 05(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.37MB mem-reservation=256.00KB thread-reservation=2 | | 05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_state IN ('IN', 'OH', 'NJ', 'WI', 'CT', 'KY', 'LA', 'IA', 'AR'), ca_country = 'United States' @@ -393,7 +393,7 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat | | | in pipelines: 06(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 06:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT) @@ -437,7 +437,7 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat | | | | in pipelines: 01(GETNEXT) | | | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=130.00MB mem-reservation=6.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| | | Per-Host Resources: mem-estimate=130.17MB mem-reservation=6.00MB thread-reservation=2 runtime-filters-memory=2.00MB | | | 01:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.66MB | | | runtime filters: RF011[min_max] -> wr_refunded_addr_sk, RF001[min_max] -> wr_reason_sk, RF010[bloom] -> wr_refunded_addr_sk, RF000[bloom] -> wr_reason_sk @@ -467,7 +467,7 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=228.00MB mem-reservation=20.00MB thread-reservation=2 runtime-filters-memory=4.00MB +| Per-Host Resources: mem-estimate=228.31MB mem-reservation=20.00MB thread-reservation=2 runtime-filters-memory=4.00MB | 00:SCAN HDFS [tpcds_parquet.web_sales, RANDOM] | HDFS partitions=1/1 files=2 size=45.09MB | predicates: ws_net_profit <= CAST(200 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ws_net_profit <= CAST(200 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(50 AS DECIMAL(3,0)), ws_net_profit <= CAST(200 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ws_net_profit <= CAST(200 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(50 AS DECIMAL(3,0)), ws_net_profit >= CAST(100 AS DECIMAL(3,0)) OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ws_net_profit >= CAST(100 AS DECIMAL(3,0)) OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(50 AS DECIMAL(3,0)), ws_net_profit >= CAST(100 AS DECIMAL(3,0)) OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ws_net_profit >= CAST(100 AS DECIMAL(3,0)) OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(50 AS DECIMAL(3,0)), ws_sales_price <= CAST(150.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(200.00 AS DECIMAL(5,2)), ws_sales_price <= CAST(150.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(150.00 AS DECIMAL(5,2)), ws_sales_price <= CAST(150.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ws_sales_price <= CAST(200.00 AS DECIMAL(5,2)), ws_sales_price <= CAST(150.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ws_sales_price >= CAST(150.00 AS DECIMAL(5,2)), ws_sales_price >= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(200.00 AS DECIMAL(5,2)), ws_sales_price >= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(150.00 AS DECIMAL(5,2)), ws_sales_price >= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ws_sales_price <= CAST(200.00 AS DECIMAL(5,2)), ws_sales_price >= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ws_sales_price >= CAST(150.00 AS DECIMAL(5,2)) @@ -494,7 +494,7 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat in pipelines: 04(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=103.90MB Threads=19 -Per-Host Resource Estimates: Memory=238MB +Per-Host Resource Estimates: Memory=242MB F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -531,7 +531,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.22MB mem-reservation=10.00MB thread-reservation=1 15:AGGREGATE [STREAMING] | output: avg(CAST(ws_quantity AS BIGINT)), avg(wr_refunded_cash), avg(wr_fee) | group by: r_reason_desc @@ -561,7 +561,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese | | in pipelines: 07(GETNEXT) | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1 | 07:SCAN HDFS [tpcds_parquet.reason, RANDOM] | HDFS partitions=1/1 files=1 size=1.92KB | stored statistics: @@ -594,7 +594,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese | | in pipelines: 00(GETNEXT) | | | F07:PLAN FRAGMENT [HASH(wr_refunded_addr_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=372.19KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=1.12MB mem-reservation=0B thread-reservation=1 | 12:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=02 | | hash predicates: ws_web_page_sk = wp_web_page_sk @@ -617,7 +617,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese | | | in pipelines: 02(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.03MB mem-reservation=8.00KB thread-reservation=1 | | 02:SCAN HDFS [tpcds_parquet.web_page, RANDOM] | | HDFS partitions=1/1 files=1 size=5.56KB | | stored statistics: @@ -651,7 +651,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese | | | in pipelines: 05(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.37MB mem-reservation=256.00KB thread-reservation=1 | | 05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_state IN ('IN', 'OH', 'NJ', 'WI', 'CT', 'KY', 'LA', 'IA', 'AR'), ca_country = 'United States' @@ -671,7 +671,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese | | in pipelines: 00(GETNEXT) | | | F04:PLAN FRAGMENT [HASH(ws_item_sk,ws_order_number)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=1.31MB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=2.40MB mem-reservation=0B thread-reservation=1 | 10:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=04 | | hash predicates: ws_sold_date_sk = d_date_sk @@ -694,7 +694,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese | | | in pipelines: 06(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 06:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_year = CAST(2000 AS INT) @@ -732,7 +732,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.68MB mem-reservation=8.00MB thread-reservation=1 | | 08:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=06 | | | hash predicates: cd1.cd_demo_sk = wr_refunded_cdemo_sk @@ -756,7 +756,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese | | | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.17MB mem-reservation=4.00MB thread-reservation=1 | | | 01:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | | | HDFS partitions=1/1 files=1 size=5.66MB | | | runtime filters: RF011[min_max] -> wr_refunded_addr_sk, RF001[min_max] -> wr_reason_sk, RF010[bloom] -> wr_refunded_addr_sk, RF000[bloom] -> wr_reason_sk @@ -787,7 +787,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.31MB mem-reservation=16.00MB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.web_sales, RANDOM] | HDFS partitions=1/1 files=2 size=45.09MB | predicates: ws_net_profit <= CAST(200 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ws_net_profit <= CAST(200 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(50 AS DECIMAL(3,0)), ws_net_profit <= CAST(200 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ws_net_profit <= CAST(200 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(50 AS DECIMAL(3,0)), ws_net_profit >= CAST(100 AS DECIMAL(3,0)) OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ws_net_profit >= CAST(100 AS DECIMAL(3,0)) OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(50 AS DECIMAL(3,0)), ws_net_profit >= CAST(100 AS DECIMAL(3,0)) OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ws_net_profit >= CAST(100 AS DECIMAL(3,0)) OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(50 AS DECIMAL(3,0)), ws_sales_price <= CAST(150.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(200.00 AS DECIMAL(5,2)), ws_sales_price <= CAST(150.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(150.00 AS DECIMAL(5,2)), ws_sales_price <= CAST(150.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ws_sales_price <= CAST(200.00 AS DECIMAL(5,2)), ws_sales_price <= CAST(150.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ws_sales_price >= CAST(150.00 AS DECIMAL(5,2)), ws_sales_price >= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(200.00 AS DECIMAL(5,2)), ws_sales_price >= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(150.00 AS DECIMAL(5,2)), ws_sales_price >= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ws_sales_price <= CAST(200.00 AS DECIMAL(5,2)), ws_sales_price >= CAST(100.00 AS DECIMAL(5,2)) OR ws_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ws_sales_price >= CAST(150.00 AS DECIMAL(5,2)) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q86.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q86.test index 7358232fe..37e43731d 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q86.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q86.test @@ -197,7 +197,7 @@ Per-Host Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -Per-Host Resources: mem-estimate=133.71MB mem-reservation=20.81MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=134.00MB mem-reservation=20.81MB thread-reservation=2 runtime-filters-memory=2.00MB 05:AGGREGATE [STREAMING] | Class 0 | output: sum(ws_net_paid) @@ -226,7 +226,7 @@ Per-Host Resources: mem-estimate=133.71MB mem-reservation=20.81MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.19MB mem-reservation=256.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -251,7 +251,7 @@ Per-Host Resources: mem-estimate=133.71MB mem-reservation=20.81MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d1.d_month_seq <= CAST(1223 AS INT), d1.d_month_seq >= CAST(1212 AS INT) @@ -277,7 +277,7 @@ Per-Host Resources: mem-estimate=133.71MB mem-reservation=20.81MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=56.12MB Threads=8 -Per-Host Resource Estimates: Memory=169MB +Per-Host Resource Estimates: Memory=170MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -349,7 +349,7 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=62.00MB mem-reservation=14.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=63.18MB mem-reservation=14.00MB thread-reservation=1 05:AGGREGATE [STREAMING] | Class 0 | output: sum(ws_net_paid) @@ -386,7 +386,7 @@ Per-Instance Resources: mem-estimate=62.00MB mem-reservation=14.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | stored statistics: @@ -419,7 +419,7 @@ Per-Instance Resources: mem-estimate=62.00MB mem-reservation=14.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d1.d_month_seq <= CAST(1223 AS INT), d1.d_month_seq >= CAST(1212 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q87.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q87.test index c45aa0be2..cc1055b10 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q87.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q87.test @@ -226,7 +226,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=326.81MB Threads=22 -Per-Host Resource Estimates: Memory=889MB +Per-Host Resource Estimates: Memory=890MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -298,7 +298,7 @@ Per-Host Resources: mem-estimate=171.11MB mem-reservation=102.00MB thread-reserv | | | in pipelines: 14(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.17MB mem-reservation=1.00MB thread-reservation=2 | | 14:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -323,7 +323,7 @@ Per-Host Resources: mem-estimate=171.11MB mem-reservation=102.00MB thread-reserv | | | in pipelines: 13(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.13MB mem-reservation=2.00MB thread-reservation=2 | | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -394,7 +394,7 @@ Per-Host Resources: mem-estimate=171.11MB mem-reservation=102.00MB thread-reserv | | | in pipelines: 08(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.17MB mem-reservation=1.00MB thread-reservation=2 | | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -419,7 +419,7 @@ Per-Host Resources: mem-estimate=171.11MB mem-reservation=102.00MB thread-reserv | | | in pipelines: 07(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.13MB mem-reservation=2.00MB thread-reservation=2 | | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -477,7 +477,7 @@ Per-Host Resources: mem-estimate=80.58MB mem-reservation=46.94MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.17MB mem-reservation=1.00MB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | stored statistics: @@ -502,7 +502,7 @@ Per-Host Resources: mem-estimate=80.58MB mem-reservation=46.94MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.13MB mem-reservation=2.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -529,7 +529,7 @@ Per-Host Resources: mem-estimate=80.58MB mem-reservation=46.94MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=466.62MB Threads=25 -Per-Host Resource Estimates: Memory=751MB +Per-Host Resource Estimates: Memory=756MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -589,7 +589,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | | F08:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=66.00MB mem-reservation=38.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=66.48MB mem-reservation=38.00MB thread-reservation=1 | 17:AGGREGATE [STREAMING] | | group by: c_last_name, c_first_name, d_date | | mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -618,7 +618,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | | in pipelines: 14(GETNEXT) | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.17MB mem-reservation=1.00MB thread-reservation=1 | | 14:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -651,7 +651,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | | in pipelines: 13(GETNEXT) | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | | 13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -710,7 +710,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | | F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=82.00MB mem-reservation=38.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=82.73MB mem-reservation=38.00MB thread-reservation=1 | 11:AGGREGATE [STREAMING] | | group by: c_last_name, c_first_name, d_date | | mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -739,7 +739,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | | in pipelines: 08(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.17MB mem-reservation=1.00MB thread-reservation=1 | | 08:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | stored statistics: @@ -772,7 +772,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | | | in pipelines: 07(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | | 07:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -810,7 +810,7 @@ Per-Instance Resources: mem-estimate=44.36MB mem-reservation=34.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-reservation=1 +Per-Instance Resources: mem-estimate=51.45MB mem-reservation=34.50MB thread-reservation=1 05:AGGREGATE [STREAMING] | group by: c_last_name, c_first_name, d_date | mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -839,7 +839,7 @@ Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-rese | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.17MB mem-reservation=1.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.customer, RANDOM] | HDFS partitions=1/1 files=1 size=5.49MB | stored statistics: @@ -872,7 +872,7 @@ Per-Instance Resources: mem-estimate=50.00MB mem-reservation=34.50MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q88.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q88.test index 7243d195b..5eb4813b7 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q88.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q88.test @@ -766,7 +766,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=73.12MB Threads=72 -Per-Host Resource Estimates: Memory=1.43GB +Per-Host Resource Estimates: Memory=1.44GB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.13MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -784,7 +784,7 @@ PLAN-ROOT SINK | | in pipelines: 116(GETNEXT) | | | F39:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 116:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -818,7 +818,7 @@ PLAN-ROOT SINK | | | in pipelines: 59(GETNEXT) | | | | | F38:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | | 59:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -846,7 +846,7 @@ PLAN-ROOT SINK | | | in pipelines: 57(GETNEXT) | | | | | F37:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=64.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=64.00KB thread-reservation=2 | | 57:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -872,7 +872,7 @@ PLAN-ROOT SINK | | | in pipelines: 58(GETNEXT) | | | | | F36:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 58:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(12 AS INT), time_dim.t_minute < CAST(30 AS INT) @@ -909,7 +909,7 @@ PLAN-ROOT SINK | | in pipelines: 110(GETNEXT) | | | F34:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 110:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -943,7 +943,7 @@ PLAN-ROOT SINK | | | in pipelines: 51(GETNEXT) | | | | | F33:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | | 51:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -971,7 +971,7 @@ PLAN-ROOT SINK | | | in pipelines: 49(GETNEXT) | | | | | F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=64.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=64.00KB thread-reservation=2 | | 49:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -997,7 +997,7 @@ PLAN-ROOT SINK | | | in pipelines: 50(GETNEXT) | | | | | F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 50:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(11 AS INT), time_dim.t_minute >= CAST(30 AS INT) @@ -1034,7 +1034,7 @@ PLAN-ROOT SINK | | in pipelines: 104(GETNEXT) | | | F29:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 104:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1068,7 +1068,7 @@ PLAN-ROOT SINK | | | in pipelines: 43(GETNEXT) | | | | | F28:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | | 43:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -1096,7 +1096,7 @@ PLAN-ROOT SINK | | | in pipelines: 41(GETNEXT) | | | | | F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=64.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=64.00KB thread-reservation=2 | | 41:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -1122,7 +1122,7 @@ PLAN-ROOT SINK | | | in pipelines: 42(GETNEXT) | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 42:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(11 AS INT), time_dim.t_minute < CAST(30 AS INT) @@ -1159,7 +1159,7 @@ PLAN-ROOT SINK | | in pipelines: 98(GETNEXT) | | | F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 98:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1193,7 +1193,7 @@ PLAN-ROOT SINK | | | in pipelines: 35(GETNEXT) | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | | 35:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -1221,7 +1221,7 @@ PLAN-ROOT SINK | | | in pipelines: 33(GETNEXT) | | | | | F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=64.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=64.00KB thread-reservation=2 | | 33:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -1247,7 +1247,7 @@ PLAN-ROOT SINK | | | in pipelines: 34(GETNEXT) | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 34:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(10 AS INT), time_dim.t_minute >= CAST(30 AS INT) @@ -1284,7 +1284,7 @@ PLAN-ROOT SINK | | in pipelines: 92(GETNEXT) | | | F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 92:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1318,7 +1318,7 @@ PLAN-ROOT SINK | | | in pipelines: 27(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | | 27:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -1346,7 +1346,7 @@ PLAN-ROOT SINK | | | in pipelines: 25(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=64.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=64.00KB thread-reservation=2 | | 25:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -1372,7 +1372,7 @@ PLAN-ROOT SINK | | | in pipelines: 26(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 26:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(10 AS INT), time_dim.t_minute < CAST(30 AS INT) @@ -1409,7 +1409,7 @@ PLAN-ROOT SINK | | in pipelines: 86(GETNEXT) | | | F14:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 86:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1443,7 +1443,7 @@ PLAN-ROOT SINK | | | in pipelines: 19(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | | 19:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -1471,7 +1471,7 @@ PLAN-ROOT SINK | | | in pipelines: 17(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=64.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=64.00KB thread-reservation=2 | | 17:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -1497,7 +1497,7 @@ PLAN-ROOT SINK | | | in pipelines: 18(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 18:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(9 AS INT), time_dim.t_minute >= CAST(30 AS INT) @@ -1534,7 +1534,7 @@ PLAN-ROOT SINK | | in pipelines: 80(GETNEXT) | | | F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 80:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1568,7 +1568,7 @@ PLAN-ROOT SINK | | | in pipelines: 11(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | | 11:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -1596,7 +1596,7 @@ PLAN-ROOT SINK | | | in pipelines: 09(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=64.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=64.00KB thread-reservation=2 | | 09:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -1622,7 +1622,7 @@ PLAN-ROOT SINK | | | in pipelines: 10(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | | 10:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(9 AS INT), time_dim.t_minute < CAST(30 AS INT) @@ -1681,7 +1681,7 @@ Per-Host Resources: mem-estimate=55.88MB mem-reservation=8.81MB thread-reservati | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_store_name = 'ese' @@ -1709,7 +1709,7 @@ Per-Host Resources: mem-estimate=55.88MB mem-reservation=8.81MB thread-reservati | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=64.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -1735,7 +1735,7 @@ Per-Host Resources: mem-estimate=55.88MB mem-reservation=8.81MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | HDFS partitions=1/1 files=1 size=1.31MB | predicates: time_dim.t_hour = CAST(8 AS INT), time_dim.t_minute >= CAST(30 AS INT) @@ -1762,7 +1762,7 @@ Per-Host Resources: mem-estimate=55.88MB mem-reservation=8.81MB thread-reservati in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=137.62MB Threads=79 -Per-Host Resource Estimates: Memory=758MB +Per-Host Resource Estimates: Memory=760MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -1787,7 +1787,7 @@ PLAN-ROOT SINK | | in pipelines: 116(GETNEXT) | | | F39:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 116:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1830,7 +1830,7 @@ PLAN-ROOT SINK | | | in pipelines: 59(GETNEXT) | | | | | F38:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | | 59:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -1866,7 +1866,7 @@ PLAN-ROOT SINK | | | in pipelines: 57(GETNEXT) | | | | | F37:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=64.00KB thread-reservation=1 | | 57:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -1900,7 +1900,7 @@ PLAN-ROOT SINK | | | in pipelines: 58(GETNEXT) | | | | | F36:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 58:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(12 AS INT), time_dim.t_minute < CAST(30 AS INT) @@ -1944,7 +1944,7 @@ PLAN-ROOT SINK | | in pipelines: 110(GETNEXT) | | | F34:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 110:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -1987,7 +1987,7 @@ PLAN-ROOT SINK | | | in pipelines: 51(GETNEXT) | | | | | F33:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | | 51:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -2023,7 +2023,7 @@ PLAN-ROOT SINK | | | in pipelines: 49(GETNEXT) | | | | | F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=64.00KB thread-reservation=1 | | 49:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -2057,7 +2057,7 @@ PLAN-ROOT SINK | | | in pipelines: 50(GETNEXT) | | | | | F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 50:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(11 AS INT), time_dim.t_minute >= CAST(30 AS INT) @@ -2101,7 +2101,7 @@ PLAN-ROOT SINK | | in pipelines: 104(GETNEXT) | | | F29:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 104:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -2144,7 +2144,7 @@ PLAN-ROOT SINK | | | in pipelines: 43(GETNEXT) | | | | | F28:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | | 43:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -2180,7 +2180,7 @@ PLAN-ROOT SINK | | | in pipelines: 41(GETNEXT) | | | | | F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=64.00KB thread-reservation=1 | | 41:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -2214,7 +2214,7 @@ PLAN-ROOT SINK | | | in pipelines: 42(GETNEXT) | | | | | F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 42:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(11 AS INT), time_dim.t_minute < CAST(30 AS INT) @@ -2258,7 +2258,7 @@ PLAN-ROOT SINK | | in pipelines: 98(GETNEXT) | | | F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 98:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -2301,7 +2301,7 @@ PLAN-ROOT SINK | | | in pipelines: 35(GETNEXT) | | | | | F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | | 35:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -2337,7 +2337,7 @@ PLAN-ROOT SINK | | | in pipelines: 33(GETNEXT) | | | | | F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=64.00KB thread-reservation=1 | | 33:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -2371,7 +2371,7 @@ PLAN-ROOT SINK | | | in pipelines: 34(GETNEXT) | | | | | F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 34:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(10 AS INT), time_dim.t_minute >= CAST(30 AS INT) @@ -2415,7 +2415,7 @@ PLAN-ROOT SINK | | in pipelines: 92(GETNEXT) | | | F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 92:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -2458,7 +2458,7 @@ PLAN-ROOT SINK | | | in pipelines: 27(GETNEXT) | | | | | F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | | 27:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -2494,7 +2494,7 @@ PLAN-ROOT SINK | | | in pipelines: 25(GETNEXT) | | | | | F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=64.00KB thread-reservation=1 | | 25:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -2528,7 +2528,7 @@ PLAN-ROOT SINK | | | in pipelines: 26(GETNEXT) | | | | | F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 26:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(10 AS INT), time_dim.t_minute < CAST(30 AS INT) @@ -2572,7 +2572,7 @@ PLAN-ROOT SINK | | in pipelines: 86(GETNEXT) | | | F14:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 86:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -2615,7 +2615,7 @@ PLAN-ROOT SINK | | | in pipelines: 19(GETNEXT) | | | | | F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | | 19:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -2651,7 +2651,7 @@ PLAN-ROOT SINK | | | in pipelines: 17(GETNEXT) | | | | | F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=64.00KB thread-reservation=1 | | 17:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -2685,7 +2685,7 @@ PLAN-ROOT SINK | | | in pipelines: 18(GETNEXT) | | | | | F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 18:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(9 AS INT), time_dim.t_minute >= CAST(30 AS INT) @@ -2729,7 +2729,7 @@ PLAN-ROOT SINK | | in pipelines: 80(GETNEXT) | | | F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 80:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -2772,7 +2772,7 @@ PLAN-ROOT SINK | | | in pipelines: 11(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | | 11:SCAN HDFS [tpcds_parquet.store, RANDOM] | | HDFS partitions=1/1 files=1 size=9.93KB | | predicates: store.s_store_name = 'ese' @@ -2808,7 +2808,7 @@ PLAN-ROOT SINK | | | in pipelines: 09(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=64.00KB thread-reservation=1 | | 09:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -2842,7 +2842,7 @@ PLAN-ROOT SINK | | | in pipelines: 10(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | | 10:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour = CAST(9 AS INT), time_dim.t_minute < CAST(30 AS INT) @@ -2910,7 +2910,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=1.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: store.s_store_name = 'ese' @@ -2946,7 +2946,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=1.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=64.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: ((household_demographics.hd_dep_count = CAST(1 AS INT) AND household_demographics.hd_vehicle_count <= CAST(3 AS INT)) OR (household_demographics.hd_dep_count = CAST(0 AS INT) AND household_demographics.hd_vehicle_count <= CAST(2 AS INT)) OR (household_demographics.hd_dep_count = CAST(2 AS INT) AND household_demographics.hd_vehicle_count <= CAST(4 AS INT))) @@ -2980,7 +2980,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=1.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | HDFS partitions=1/1 files=1 size=1.31MB | predicates: time_dim.t_hour = CAST(8 AS INT), time_dim.t_minute >= CAST(30 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q89.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q89.test index 73a18cf59..21129247a 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q89.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q89.test @@ -136,7 +136,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=34.52MB Threads=10 -Per-Host Resource Estimates: Memory=257MB +Per-Host Resource Estimates: Memory=259MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -190,7 +190,7 @@ Per-Host Resources: mem-estimate=26.00MB mem-reservation=17.94MB thread-reservat | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=66.89MB mem-reservation=11.81MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=68.28MB mem-reservation=11.81MB thread-reservation=2 runtime-filters-memory=3.00MB 07:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: i_category, i_class, i_brand, s_store_name, s_company_name, d_moy @@ -212,7 +212,7 @@ Per-Host Resources: mem-estimate=66.89MB mem-reservation=11.81MB thread-reservat | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.17MB mem-reservation=24.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -237,7 +237,7 @@ Per-Host Resources: mem-estimate=66.89MB mem-reservation=11.81MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year IN (CAST(2000 AS INT)) @@ -265,7 +265,7 @@ Per-Host Resources: mem-estimate=66.89MB mem-reservation=11.81MB thread-reservat | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.30MB mem-reservation=256.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: ((i_category IN ('Children', 'Music', 'Home') AND i_class IN ('toddlers', 'pop', 'lighting')) OR (i_category IN ('Jewelry', 'Books', 'Sports') AND i_class IN ('costume', 'travel', 'football'))) @@ -290,7 +290,7 @@ Per-Host Resources: mem-estimate=66.89MB mem-reservation=11.81MB thread-reservat in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=64.27MB Threads=11 -Per-Host Resource Estimates: Memory=174MB +Per-Host Resource Estimates: Memory=180MB F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.09MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -345,7 +345,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=17.94MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=28.93MB mem-reservation=3.00MB thread-reservation=1 07:AGGREGATE [STREAMING] | output: sum(ss_sales_price) | group by: i_category, i_class, i_brand, s_store_name, s_company_name, d_moy @@ -375,7 +375,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.17MB mem-reservation=24.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.store, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | stored statistics: @@ -408,7 +408,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year IN (CAST(2000 AS INT)) @@ -444,7 +444,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.30MB mem-reservation=256.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: ((i_category IN ('Children', 'Music', 'Home') AND i_class IN ('toddlers', 'pop', 'lighting')) OR (i_category IN ('Jewelry', 'Books', 'Sports') AND i_class IN ('costume', 'travel', 'football'))) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q90.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q90.test index 7d7b537a2..b15227c42 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q90.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q90.test @@ -233,7 +233,7 @@ PLAN-ROOT SINK | | in pipelines: 27(GETNEXT) | | | F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Host Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 27:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -267,7 +267,7 @@ PLAN-ROOT SINK | | | in pipelines: 10(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 10:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour <= CAST(20 AS INT), time_dim.t_hour >= CAST(19 AS INT) @@ -295,7 +295,7 @@ PLAN-ROOT SINK | | | in pipelines: 11(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=16.00KB thread-reservation=2 | | 11:SCAN HDFS [tpcds_parquet.web_page, RANDOM] | | HDFS partitions=1/1 files=1 size=5.56KB | | predicates: web_page.wp_char_count <= CAST(5200 AS INT), web_page.wp_char_count >= CAST(5000 AS INT) @@ -323,7 +323,7 @@ PLAN-ROOT SINK | | | in pipelines: 09(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=64.00KB thread-reservation=2 | | 09:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: household_demographics.hd_dep_count = CAST(6 AS INT) @@ -381,7 +381,7 @@ Per-Host Resources: mem-estimate=104.92MB mem-reservation=12.81MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | HDFS partitions=1/1 files=1 size=1.31MB | predicates: time_dim.t_hour <= CAST(9 AS INT), time_dim.t_hour >= CAST(8 AS INT) @@ -409,7 +409,7 @@ Per-Host Resources: mem-estimate=104.92MB mem-reservation=12.81MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=16.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.web_page, RANDOM] | HDFS partitions=1/1 files=1 size=5.56KB | predicates: web_page.wp_char_count <= CAST(5200 AS INT), web_page.wp_char_count >= CAST(5000 AS INT) @@ -437,7 +437,7 @@ Per-Host Resources: mem-estimate=104.92MB mem-reservation=12.81MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=64.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: household_demographics.hd_dep_count = CAST(6 AS INT) @@ -495,7 +495,7 @@ PLAN-ROOT SINK | | in pipelines: 27(GETNEXT) | | | F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=64.00KB mem-reservation=0B thread-reservation=1 | 27:AGGREGATE [FINALIZE] | | output: count:merge(*) | | mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0 @@ -538,7 +538,7 @@ PLAN-ROOT SINK | | | in pipelines: 10(GETNEXT) | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 10:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=1.31MB | | predicates: time_dim.t_hour <= CAST(20 AS INT), time_dim.t_hour >= CAST(19 AS INT) @@ -574,7 +574,7 @@ PLAN-ROOT SINK | | | in pipelines: 11(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=16.00KB thread-reservation=1 | | 11:SCAN HDFS [tpcds_parquet.web_page, RANDOM] | | HDFS partitions=1/1 files=1 size=5.56KB | | predicates: web_page.wp_char_count <= CAST(5200 AS INT), web_page.wp_char_count >= CAST(5000 AS INT) @@ -610,7 +610,7 @@ PLAN-ROOT SINK | | | in pipelines: 09(GETNEXT) | | | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=64.00KB thread-reservation=1 | | 09:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: household_demographics.hd_dep_count = CAST(6 AS INT) @@ -677,7 +677,7 @@ Per-Instance Resources: mem-estimate=32.02MB mem-reservation=4.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.time_dim, RANDOM] | HDFS partitions=1/1 files=1 size=1.31MB | predicates: time_dim.t_hour <= CAST(9 AS INT), time_dim.t_hour >= CAST(8 AS INT) @@ -713,7 +713,7 @@ Per-Instance Resources: mem-estimate=32.02MB mem-reservation=4.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=16.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.web_page, RANDOM] | HDFS partitions=1/1 files=1 size=5.56KB | predicates: web_page.wp_char_count <= CAST(5200 AS INT), web_page.wp_char_count >= CAST(5000 AS INT) @@ -749,7 +749,7 @@ Per-Instance Resources: mem-estimate=32.02MB mem-reservation=4.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=64.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: household_demographics.hd_dep_count = CAST(6 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test index 3e6c2859e..0aecefa29 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test @@ -183,7 +183,7 @@ PLAN-ROOT SINK in pipelines: 01(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=58.97MB Threads=16 -Per-Host Resource Estimates: Memory=448MB +Per-Host Resource Estimates: Memory=449MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -217,7 +217,7 @@ Per-Host Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-reservat | in pipelines: 01(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -Per-Host Resources: mem-estimate=115.09MB mem-reservation=12.81MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=115.32MB mem-reservation=12.81MB thread-reservation=2 runtime-filters-memory=3.00MB 13:AGGREGATE [STREAMING] | output: sum(cr_net_loss) | group by: cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status @@ -239,7 +239,7 @@ Per-Host Resources: mem-estimate=115.09MB mem-reservation=12.81MB thread-reserva | | in pipelines: 00(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=64.00MB mem-reservation=32.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=64.33MB mem-reservation=32.00KB thread-reservation=2 | 00:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | HDFS partitions=1/1 files=1 size=10.28KB | stored statistics: @@ -264,7 +264,7 @@ Per-Host Resources: mem-estimate=115.09MB mem-reservation=12.81MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(11 AS INT) @@ -307,7 +307,7 @@ Per-Host Resources: mem-estimate=115.09MB mem-reservation=12.81MB thread-reserva | | | in pipelines: 04(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=256.00KB thread-reservation=2 | | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-7 AS DECIMAL(3,0)) @@ -335,7 +335,7 @@ Per-Host Resources: mem-estimate=115.09MB mem-reservation=12.81MB thread-reserva | | | in pipelines: 06(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.11MB mem-reservation=64.00KB thread-reservation=2 | | 06:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: hd_buy_potential LIKE 'Unknown%' @@ -362,7 +362,7 @@ Per-Host Resources: mem-estimate=115.09MB mem-reservation=12.81MB thread-reserva | | | in pipelines: 03(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=66.00MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| | Per-Host Resources: mem-estimate=66.08MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB | | 03:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF009[min_max] -> c_current_hdemo_sk, RF007[min_max] -> c_current_addr_sk, RF008[bloom] -> c_current_hdemo_sk, RF006[bloom] -> c_current_addr_sk @@ -398,7 +398,7 @@ Per-Host Resources: mem-estimate=115.09MB mem-reservation=12.81MB thread-reserva in pipelines: 01(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=81.16MB Threads=15 -Per-Host Resource Estimates: Memory=206MB +Per-Host Resource Estimates: Memory=208MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -433,7 +433,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=34.51MB mem-reservation=4.00MB thread-reservation=1 13:AGGREGATE [STREAMING] | output: sum(cr_net_loss) | group by: cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status @@ -463,7 +463,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 00(GETNEXT) | | | F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.33MB mem-reservation=32.00KB thread-reservation=1 | 00:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | HDFS partitions=1/1 files=1 size=10.28KB | stored statistics: @@ -496,7 +496,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_year = CAST(1998 AS INT), d_moy = CAST(11 AS INT) @@ -533,7 +533,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.40MB mem-reservation=8.00MB thread-reservation=1 | 09:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=03 | | hash predicates: c_current_addr_sk = ca_address_sk @@ -556,7 +556,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser | | | in pipelines: 04(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=256.00KB thread-reservation=1 | | 04:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_gmt_offset = CAST(-7 AS DECIMAL(3,0)) @@ -592,7 +592,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser | | | in pipelines: 06(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.11MB mem-reservation=64.00KB thread-reservation=1 | | 06:SCAN HDFS [tpcds_parquet.household_demographics, RANDOM] | | HDFS partitions=1/1 files=1 size=41.69KB | | predicates: hd_buy_potential LIKE 'Unknown%' @@ -628,7 +628,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.08MB mem-reservation=2.00MB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.customer, RANDOM] | | HDFS partitions=1/1 files=1 size=5.49MB | | runtime filters: RF009[min_max] -> c_current_hdemo_sk, RF007[min_max] -> c_current_addr_sk, RF008[bloom] -> c_current_hdemo_sk, RF006[bloom] -> c_current_addr_sk diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q92.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q92.test index 3cd762f93..690d482db 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q92.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q92.test @@ -140,7 +140,7 @@ PLAN-ROOT SINK in pipelines: 03(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=35.94MB Threads=13 -Per-Host Resource Estimates: Memory=323MB +Per-Host Resource Estimates: Memory=324MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -186,7 +186,7 @@ Per-Host Resources: mem-estimate=13.23MB mem-reservation=4.88MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F06:PLAN FRAGMENT [HASH(ws_sold_date_sk)] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=3.21MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=3.52MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB | 08:HASH JOIN [INNER JOIN, PARTITIONED] | | hash predicates: ws_sold_date_sk = d_date_sk | | fk/pk conjuncts: ws_sold_date_sk = d_date_sk @@ -201,7 +201,7 @@ Per-Host Resources: mem-estimate=13.23MB mem-reservation=4.88MB thread-reservati | | | in pipelines: 02(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.23MB mem-reservation=1.00MB thread-reservation=2 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '2000-04-26', CAST(d_date AS DATE) >= DATE '2000-01-27' @@ -221,7 +221,7 @@ Per-Host Resources: mem-estimate=13.23MB mem-reservation=4.88MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=99.95MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=100.22MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB | 07:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws_item_sk = i_item_sk | | fk/pk conjuncts: ws_item_sk = i_item_sk @@ -236,7 +236,7 @@ Per-Host Resources: mem-estimate=13.23MB mem-reservation=4.88MB thread-reservati | | | in pipelines: 01(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.06MB mem-reservation=256.00KB thread-reservation=2 | | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_manufact_id = CAST(350 AS INT) @@ -296,7 +296,7 @@ Per-Host Resources: mem-estimate=110.15MB mem-reservation=13.94MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '2000-04-26', CAST(d_date AS DATE) >= DATE '2000-01-27' @@ -322,7 +322,7 @@ Per-Host Resources: mem-estimate=110.15MB mem-reservation=13.94MB thread-reserva in pipelines: 03(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=41.81MB Threads=12 -Per-Host Resource Estimates: Memory=152MB +Per-Host Resource Estimates: Memory=154MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -376,7 +376,7 @@ Per-Instance Resources: mem-estimate=10.18MB mem-reservation=1.94MB thread-reser | | in pipelines: 00(GETNEXT) | | | F06:PLAN FRAGMENT [HASH(ws_sold_date_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=63.83KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=591.83KB mem-reservation=0B thread-reservation=1 | 08:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=01 | | hash predicates: ws_sold_date_sk = d_date_sk @@ -399,7 +399,7 @@ Per-Instance Resources: mem-estimate=10.18MB mem-reservation=1.94MB thread-reser | | | in pipelines: 02(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.23MB mem-reservation=1.00MB thread-reservation=1 | | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '2000-04-26', CAST(d_date AS DATE) >= DATE '2000-01-27' @@ -420,7 +420,7 @@ Per-Instance Resources: mem-estimate=10.18MB mem-reservation=1.94MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.28MB mem-reservation=8.00MB thread-reservation=1 | 07:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=02 | | hash predicates: ws_item_sk = i_item_sk @@ -443,7 +443,7 @@ Per-Instance Resources: mem-estimate=10.18MB mem-reservation=1.94MB thread-reser | | | in pipelines: 01(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.06MB mem-reservation=256.00KB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | | HDFS partitions=1/1 files=1 size=1.73MB | | predicates: i_manufact_id = CAST(350 AS INT) @@ -482,7 +482,7 @@ Per-Instance Resources: mem-estimate=10.18MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=42.00MB mem-reservation=10.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=42.16MB mem-reservation=10.00MB thread-reservation=1 06:AGGREGATE [STREAMING] | output: avg(ws_ext_discount_amt) | group by: ws_item_sk @@ -512,7 +512,7 @@ Per-Instance Resources: mem-estimate=42.00MB mem-reservation=10.00MB thread-rese | | in pipelines: 04(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS DATE) <= DATE '2000-04-26', CAST(d_date AS DATE) >= DATE '2000-01-27' diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q93.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q93.test index 0cf5cdec1..dbb4db0c0 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q93.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q93.test @@ -149,7 +149,7 @@ Per-Host Resources: mem-estimate=116.56MB mem-reservation=25.94MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.14MB mem-reservation=16.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.reason, RANDOM] | HDFS partitions=1/1 files=1 size=1.92KB | predicates: r_reason_desc = 'reason 28' @@ -176,7 +176,7 @@ Per-Host Resources: mem-estimate=116.56MB mem-reservation=25.94MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=97.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=97.11MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | runtime filters: RF001[min_max] -> sr_reason_sk, RF000[bloom] -> sr_reason_sk @@ -200,7 +200,7 @@ Per-Host Resources: mem-estimate=116.56MB mem-reservation=25.94MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=63.77MB Threads=9 -Per-Host Resource Estimates: Memory=164MB +Per-Host Resource Estimates: Memory=165MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -235,7 +235,7 @@ Per-Instance Resources: mem-estimate=10.66MB mem-reservation=1.94MB thread-reser | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 -Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=26.56MB mem-reservation=6.00MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(CASE WHEN sr_return_quantity IS NOT NULL THEN CAST((CAST(ss_quantity AS BIGINT) - CAST(sr_return_quantity AS BIGINT)) AS DECIMAL(19,0)) * ss_sales_price ELSE CAST((CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price) AS DECIMAL(27,2)) END) | group by: ss_customer_sk @@ -265,7 +265,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.reason, RANDOM] | HDFS partitions=1/1 files=1 size=1.92KB | predicates: r_reason_desc = 'reason 28' @@ -301,7 +301,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=24.11MB mem-reservation=4.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM] | HDFS partitions=1/1 files=1 size=15.43MB | runtime filters: RF001[min_max] -> sr_reason_sk, RF000[bloom] -> sr_reason_sk diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test index 10f2409b9..beca61a5a 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test @@ -163,7 +163,7 @@ PLAN-ROOT SINK in pipelines: 05(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=38.33MB Threads=15 -Per-Host Resource Estimates: Memory=431MB +Per-Host Resource Estimates: Memory=433MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -214,7 +214,7 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=16.09MB mem-reservation=512.00KB thread-reservation=2 | 05:SCAN HDFS [tpcds_parquet.web_returns wr1, RANDOM] | HDFS partitions=1/1 files=1 size=5.66MB | stored statistics: @@ -239,7 +239,7 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F05:PLAN FRAGMENT [HASH(ws1.ws_ship_date_sk)] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=3.41MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=4.08MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB | 08:HASH JOIN [INNER JOIN, PARTITIONED] | | hash predicates: ws1.ws_ship_date_sk = d_date_sk | | fk/pk conjuncts: ws1.ws_ship_date_sk = d_date_sk @@ -254,7 +254,7 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati | | | in pipelines: 01(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.23MB mem-reservation=1.00MB thread-reservation=2 | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1999-04-02', CAST(d_date AS DATE) >= DATE '1999-02-01' @@ -274,7 +274,7 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati | | in pipelines: 00(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=230.93MB mem-reservation=22.88MB thread-reservation=2 runtime-filters-memory=3.00MB +| Per-Host Resources: mem-estimate=231.52MB mem-reservation=22.88MB thread-reservation=2 runtime-filters-memory=3.00MB | 07:HASH JOIN [INNER JOIN, BROADCAST] | | hash predicates: ws1.ws_web_site_sk = web_site_sk | | fk/pk conjuncts: ws1.ws_web_site_sk = web_site_sk @@ -289,7 +289,7 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati | | | in pipelines: 03(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | | 03:SCAN HDFS [tpcds_parquet.web_site, RANDOM] | | HDFS partitions=1/1 files=1 size=11.91KB | | predicates: web_company_name = 'pri' @@ -317,7 +317,7 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati | | | in pipelines: 02(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=256.00KB thread-reservation=2 | | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_state = 'IL' @@ -348,7 +348,7 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati | in pipelines: 04(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -Per-Host Resources: mem-estimate=65.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=65.12MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB 04:SCAN HDFS [tpcds_parquet.web_sales ws2, RANDOM] HDFS partitions=1/1 files=2 size=45.09MB runtime filters: RF001[min_max] -> ws2.ws_order_number, RF000[bloom] -> ws2.ws_order_number @@ -361,7 +361,7 @@ Per-Host Resources: mem-estimate=65.00MB mem-reservation=3.00MB thread-reservati in pipelines: 04(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=44.20MB Threads=14 -Per-Host Resource Estimates: Memory=166MB +Per-Host Resource Estimates: Memory=168MB F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -420,7 +420,7 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser | | in pipelines: 05(GETNEXT) | | | F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=512.00KB thread-reservation=1 | 05:SCAN HDFS [tpcds_parquet.web_returns wr1, RANDOM] | HDFS partitions=1/1 files=1 size=5.66MB | stored statistics: @@ -453,7 +453,7 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser | | in pipelines: 00(GETNEXT) | | | F05:PLAN FRAGMENT [HASH(ws1.ws_ship_date_sk)] hosts=2 instances=2 -| Per-Instance Resources: mem-estimate=271.68KB mem-reservation=0B thread-reservation=1 +| Per-Instance Resources: mem-estimate=1.14MB mem-reservation=0B thread-reservation=1 | 08:HASH JOIN [INNER JOIN, PARTITIONED] | | hash-table-id=02 | | hash predicates: ws1.ws_ship_date_sk = d_date_sk @@ -476,7 +476,7 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser | | | in pipelines: 01(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.23MB mem-reservation=1.00MB thread-reservation=1 | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: CAST(d_date AS DATE) <= DATE '1999-04-02', CAST(d_date AS DATE) >= DATE '1999-02-01' @@ -497,7 +497,7 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.64MB mem-reservation=16.00MB thread-reservation=1 | 07:HASH JOIN [INNER JOIN, BROADCAST] | | hash-table-id=03 | | hash predicates: ws1.ws_web_site_sk = web_site_sk @@ -520,7 +520,7 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser | | | in pipelines: 03(GETNEXT) | | | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | | 03:SCAN HDFS [tpcds_parquet.web_site, RANDOM] | | HDFS partitions=1/1 files=1 size=11.91KB | | predicates: web_company_name = 'pri' @@ -556,7 +556,7 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser | | | in pipelines: 02(GETNEXT) | | | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | HDFS partitions=1/1 files=1 size=1.16MB | | predicates: ca_state = 'IL' @@ -588,7 +588,7 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=32.00MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=32.12MB mem-reservation=2.00MB thread-reservation=1 04:SCAN HDFS [tpcds_parquet.web_sales ws2, RANDOM] HDFS partitions=1/1 files=2 size=45.09MB runtime filters: RF001[min_max] -> ws2.ws_order_number, RF000[bloom] -> ws2.ws_order_number diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test index fe88fa396..4d9e95789 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test @@ -237,7 +237,7 @@ PLAN-ROOT SINK in pipelines: 08(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=127.20MB Threads=21 -Per-Host Resource Estimates: Memory=686MB +Per-Host Resource Estimates: Memory=688MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -296,7 +296,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva | | | in pipelines: 00(GETNEXT) | | | | | F11:PLAN FRAGMENT [HASH(ws1.ws_ship_date_sk)] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=3.40MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=4.03MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB | | 14:HASH JOIN [INNER JOIN, PARTITIONED] | | | hash predicates: ws1.ws_ship_date_sk = d_date_sk | | | fk/pk conjuncts: ws1.ws_ship_date_sk = d_date_sk @@ -311,7 +311,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva | | | | in pipelines: 01(GETNEXT) | | | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.23MB mem-reservation=1.00MB thread-reservation=2 | | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: CAST(d_date AS TIMESTAMP) <= TIMESTAMP '1999-04-02 00:00:00', CAST(d_date AS TIMESTAMP) >= TIMESTAMP '1999-02-01 00:00:00' @@ -330,7 +330,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva | | | in pipelines: 00(GETNEXT) | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=198.93MB mem-reservation=22.88MB thread-reservation=2 runtime-filters-memory=3.00MB +| | Per-Host Resources: mem-estimate=199.48MB mem-reservation=22.88MB thread-reservation=2 runtime-filters-memory=3.00MB | | 13:HASH JOIN [INNER JOIN, BROADCAST] | | | hash predicates: ws1.ws_web_site_sk = web_site_sk | | | fk/pk conjuncts: ws1.ws_web_site_sk = web_site_sk @@ -345,7 +345,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva | | | | in pipelines: 03(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | | | 03:SCAN HDFS [tpcds_parquet.web_site, RANDOM] | | | HDFS partitions=1/1 files=1 size=11.91KB | | | predicates: web_company_name = 'pri' @@ -373,7 +373,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva | | | | in pipelines: 02(GETNEXT) | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2 +| | | Per-Host Resources: mem-estimate=32.09MB mem-reservation=256.00KB thread-reservation=2 | | | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.16MB | | | predicates: ca_state = 'IL' @@ -419,7 +419,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva | | | in pipelines: 05(GETNEXT) | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| | Per-Host Resources: mem-estimate=65.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB +| | Per-Host Resources: mem-estimate=65.12MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB | | 05:SCAN HDFS [tpcds_parquet.web_sales ws2, RANDOM] | | HDFS partitions=1/1 files=2 size=45.09MB | | runtime filters: RF007[min_max] -> ws2.ws_order_number, RF006[bloom] -> ws2.ws_order_number @@ -437,7 +437,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=66.00MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=66.12MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB | 04:SCAN HDFS [tpcds_parquet.web_sales ws1, RANDOM] | HDFS partitions=1/1 files=2 size=45.09MB | runtime filters: RF007[min_max] -> ws1.ws_order_number, RF009[min_max] -> ws1.ws_order_number, RF006[bloom] -> ws1.ws_order_number, RF008[bloom] -> ws1.ws_order_number @@ -469,7 +469,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva | | in pipelines: 07(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=17.00MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=17.09MB mem-reservation=1.50MB thread-reservation=2 runtime-filters-memory=1.00MB | 07:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | HDFS partitions=1/1 files=1 size=5.66MB | runtime filters: RF001[min_max] -> tpcds_parquet.web_returns.wr_order_number, RF000[bloom] -> tpcds_parquet.web_returns.wr_order_number @@ -496,7 +496,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva | | in pipelines: 09(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -| Per-Host Resources: mem-estimate=66.00MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB +| Per-Host Resources: mem-estimate=66.12MB mem-reservation=4.00MB thread-reservation=2 runtime-filters-memory=2.00MB | 09:SCAN HDFS [tpcds_parquet.web_sales ws2, RANDOM] | HDFS partitions=1/1 files=2 size=45.09MB | runtime filters: RF001[min_max] -> ws2.ws_order_number, RF003[min_max] -> ws2.ws_order_number, RF000[bloom] -> ws2.ws_order_number, RF002[bloom] -> ws2.ws_order_number @@ -514,7 +514,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva | in pipelines: 08(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 -Per-Host Resources: mem-estimate=67.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=3.00MB +Per-Host Resources: mem-estimate=67.12MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=3.00MB 08:SCAN HDFS [tpcds_parquet.web_sales ws1, RANDOM] HDFS partitions=1/1 files=2 size=45.09MB runtime filters: RF001[min_max] -> ws1.ws_order_number, RF003[min_max] -> ws1.ws_order_number, RF005[min_max] -> ws1.ws_order_number, RF000[bloom] -> ws1.ws_order_number, RF002[bloom] -> ws1.ws_order_number, RF004[bloom] -> ws1.ws_order_number @@ -527,7 +527,7 @@ Per-Host Resources: mem-estimate=67.00MB mem-reservation=5.00MB thread-reservati in pipelines: 08(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=136.95MB Threads=20 -Per-Host Resource Estimates: Memory=370MB +Per-Host Resource Estimates: Memory=373MB F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -602,7 +602,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser | | | in pipelines: 00(GETNEXT) | | | | | F11:PLAN FRAGMENT [HASH(ws1.ws_ship_date_sk)] hosts=2 instances=2 -| | Per-Instance Resources: mem-estimate=257.52KB mem-reservation=0B thread-reservation=1 +| | Per-Instance Resources: mem-estimate=1.09MB mem-reservation=0B thread-reservation=1 | | 14:HASH JOIN [INNER JOIN, PARTITIONED] | | | hash-table-id=02 | | | hash predicates: ws1.ws_ship_date_sk = d_date_sk @@ -625,7 +625,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser | | | | in pipelines: 01(GETNEXT) | | | | | | | F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.23MB mem-reservation=1.00MB thread-reservation=1 | | | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | | HDFS partitions=1/1 files=1 size=2.15MB | | | predicates: CAST(d_date AS TIMESTAMP) <= TIMESTAMP '1999-04-02 00:00:00', CAST(d_date AS TIMESTAMP) >= TIMESTAMP '1999-02-01 00:00:00' @@ -645,7 +645,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser | | | | | F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | | Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -| | Per-Instance Resources: mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=32.61MB mem-reservation=16.00MB thread-reservation=1 | | 13:HASH JOIN [INNER JOIN, BROADCAST] | | | hash-table-id=03 | | | hash predicates: ws1.ws_web_site_sk = web_site_sk @@ -668,7 +668,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser | | | | in pipelines: 03(GETNEXT) | | | | | | | F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | | | 03:SCAN HDFS [tpcds_parquet.web_site, RANDOM] | | | HDFS partitions=1/1 files=1 size=11.91KB | | | predicates: web_company_name = 'pri' @@ -704,7 +704,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser | | | | in pipelines: 02(GETNEXT) | | | | | | | F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1 +| | | Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1 | | | 02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM] | | | HDFS partitions=1/1 files=1 size=1.16MB | | | predicates: ca_state = 'IL' @@ -759,7 +759,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser | | | | | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| | Per-Instance Resources: mem-estimate=32.00MB mem-reservation=2.00MB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=32.12MB mem-reservation=2.00MB thread-reservation=1 | | 05:SCAN HDFS [tpcds_parquet.web_sales ws2, RANDOM] | | HDFS partitions=1/1 files=2 size=45.09MB | | runtime filters: RF007[min_max] -> ws2.ws_order_number, RF006[bloom] -> ws2.ws_order_number @@ -778,7 +778,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser | | | F04:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.12MB mem-reservation=2.00MB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.web_sales ws1, RANDOM] | HDFS partitions=1/1 files=2 size=45.09MB | runtime filters: RF007[min_max] -> ws1.ws_order_number, RF009[min_max] -> ws1.ws_order_number, RF006[bloom] -> ws1.ws_order_number, RF008[bloom] -> ws1.ws_order_number @@ -819,7 +819,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=512.00KB thread-reservation=1 | 07:SCAN HDFS [tpcds_parquet.web_returns, RANDOM] | HDFS partitions=1/1 files=1 size=5.66MB | runtime filters: RF001[min_max] -> tpcds_parquet.web_returns.wr_order_number, RF000[bloom] -> tpcds_parquet.web_returns.wr_order_number @@ -855,7 +855,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -| Per-Instance Resources: mem-estimate=32.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=32.12MB mem-reservation=2.00MB thread-reservation=1 | 09:SCAN HDFS [tpcds_parquet.web_sales ws2, RANDOM] | HDFS partitions=1/1 files=2 size=45.09MB | runtime filters: RF001[min_max] -> ws2.ws_order_number, RF003[min_max] -> ws2.ws_order_number, RF000[bloom] -> ws2.ws_order_number, RF002[bloom] -> ws2.ws_order_number @@ -874,7 +874,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB -Per-Instance Resources: mem-estimate=32.00MB mem-reservation=2.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=32.12MB mem-reservation=2.00MB thread-reservation=1 08:SCAN HDFS [tpcds_parquet.web_sales ws1, RANDOM] HDFS partitions=1/1 files=2 size=45.09MB runtime filters: RF001[min_max] -> ws1.ws_order_number, RF003[min_max] -> ws1.ws_order_number, RF005[min_max] -> ws1.ws_order_number, RF000[bloom] -> ws1.ws_order_number, RF002[bloom] -> ws1.ws_order_number, RF004[bloom] -> ws1.ws_order_number diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q96.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q96.test index 4f1067e59..54142ec8c 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q96.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q96.test @@ -144,7 +144,7 @@ Per-Host Resources: mem-estimate=56.87MB mem-reservation=9.81MB thread-reservati | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.store s, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s.s_store_name = 'ese' @@ -172,7 +172,7 @@ Per-Host Resources: mem-estimate=56.87MB mem-reservation=9.81MB thread-reservati | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=64.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.household_demographics hd, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: hd.hd_dep_count = CAST(5 AS INT) @@ -200,7 +200,7 @@ Per-Host Resources: mem-estimate=56.87MB mem-reservation=9.81MB thread-reservati | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=48.06MB mem-reservation=512.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.time_dim td, RANDOM] | HDFS partitions=1/1 files=1 size=1.31MB | predicates: td.t_hour = CAST(8 AS INT), td.t_minute >= CAST(30 AS INT) @@ -276,7 +276,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=1.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.09MB mem-reservation=16.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.store s, RANDOM] | HDFS partitions=1/1 files=1 size=9.93KB | predicates: s.s_store_name = 'ese' @@ -312,7 +312,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=1.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=64.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=64.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.household_demographics hd, RANDOM] | HDFS partitions=1/1 files=1 size=41.69KB | predicates: hd.hd_dep_count = CAST(5 AS INT) @@ -348,7 +348,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=1.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.time_dim td, RANDOM] | HDFS partitions=1/1 files=1 size=1.31MB | predicates: td.t_hour = CAST(8 AS INT), td.t_minute >= CAST(30 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q97.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q97.test index f22454563..1e17706a0 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q97.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q97.test @@ -181,7 +181,7 @@ Per-Host Resources: mem-estimate=95.05MB mem-reservation=85.00MB thread-reservat | | in pipelines: 04(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -| Per-Host Resources: mem-estimate=164.00MB mem-reservation=23.94MB thread-reservation=2 runtime-filters-memory=1.00MB +| Per-Host Resources: mem-estimate=164.12MB mem-reservation=23.94MB thread-reservation=2 runtime-filters-memory=1.00MB | 07:AGGREGATE [STREAMING] | | group by: cs_bill_customer_sk, cs_item_sk | | mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0 @@ -202,7 +202,7 @@ Per-Host Resources: mem-estimate=95.05MB mem-reservation=85.00MB thread-reservat | | | in pipelines: 05(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| | Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | | 05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -239,7 +239,7 @@ Per-Host Resources: mem-estimate=95.05MB mem-reservation=85.00MB thread-reservat | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=69.00MB mem-reservation=37.94MB thread-reservation=2 runtime-filters-memory=1.00MB +Per-Host Resources: mem-estimate=69.12MB mem-reservation=37.94MB thread-reservation=2 runtime-filters-memory=1.00MB 03:AGGREGATE [STREAMING] | group by: ss_customer_sk, ss_item_sk | mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0 @@ -260,7 +260,7 @@ Per-Host Resources: mem-estimate=69.00MB mem-reservation=37.94MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -287,7 +287,7 @@ Per-Host Resources: mem-estimate=69.00MB mem-reservation=37.94MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=154.75MB Threads=12 -Per-Host Resource Estimates: Memory=283MB +Per-Host Resource Estimates: Memory=284MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1 PLAN-ROOT SINK @@ -342,7 +342,7 @@ Per-Instance Resources: mem-estimate=27.09MB mem-reservation=17.00MB thread-rese | | | F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -| Per-Instance Resources: mem-estimate=65.00MB mem-reservation=21.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=65.38MB mem-reservation=21.00MB thread-reservation=1 | 07:AGGREGATE [STREAMING] | | group by: cs_bill_customer_sk, cs_item_sk | | mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0 @@ -371,7 +371,7 @@ Per-Instance Resources: mem-estimate=27.09MB mem-reservation=17.00MB thread-rese | | | in pipelines: 05(GETNEXT) | | | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| | Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| | Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | | 05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | | HDFS partitions=1/1 files=1 size=2.15MB | | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -409,7 +409,7 @@ Per-Instance Resources: mem-estimate=27.09MB mem-reservation=17.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB -Per-Instance Resources: mem-estimate=33.00MB mem-reservation=18.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=33.38MB mem-reservation=18.00MB thread-reservation=1 03:AGGREGATE [STREAMING] | group by: ss_customer_sk, ss_item_sk | mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0 @@ -438,7 +438,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=18.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q98.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q98.test index 6f578da90..0d232e71e 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q98.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q98.test @@ -118,7 +118,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=98.81MB Threads=8 -Per-Host Resource Estimates: Memory=332MB +Per-Host Resource Estimates: Memory=334MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.69MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -166,7 +166,7 @@ Per-Host Resources: mem-estimate=116.13MB mem-reservation=50.00MB thread-reserva | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=83.48MB mem-reservation=41.81MB thread-reservation=2 runtime-filters-memory=2.00MB +Per-Host Resources: mem-estimate=84.47MB mem-reservation=41.81MB thread-reservation=2 runtime-filters-memory=2.00MB 05:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: i_item_id, i_item_desc, i_category, i_class, i_current_price @@ -188,7 +188,7 @@ Per-Host Resources: mem-estimate=83.48MB mem-reservation=41.81MB thread-reservat | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=1.00MB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS TIMESTAMP) <= TIMESTAMP '1999-03-24 00:00:00', CAST(d_date AS TIMESTAMP) >= TIMESTAMP '1999-02-22 00:00:00' @@ -215,7 +215,7 @@ Per-Host Resources: mem-estimate=83.48MB mem-reservation=41.81MB thread-reservat | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=96.00MB mem-reservation=2.00MB thread-reservation=2 +| Per-Host Resources: mem-estimate=96.76MB mem-reservation=2.00MB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category IN ('Jewelry', 'Sports', 'Books') @@ -242,7 +242,7 @@ Per-Host Resources: mem-estimate=83.48MB mem-reservation=41.81MB thread-reservat in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=190.62MB Threads=9 -Per-Host Resource Estimates: Memory=283MB +Per-Host Resource Estimates: Memory=293MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=5.32MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -291,7 +291,7 @@ Per-Instance Resources: mem-estimate=65.21MB mem-reservation=50.00MB thread-rese | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB -Per-Instance Resources: mem-estimate=50.00MB mem-reservation=35.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=54.74MB mem-reservation=35.00MB thread-reservation=1 05:AGGREGATE [STREAMING] | output: sum(ss_ext_sales_price) | group by: i_item_id, i_item_desc, i_category, i_class, i_current_price @@ -321,7 +321,7 @@ Per-Instance Resources: mem-estimate=50.00MB mem-reservation=35.00MB thread-rese | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: CAST(d_date AS TIMESTAMP) <= TIMESTAMP '1999-03-24 00:00:00', CAST(d_date AS TIMESTAMP) >= TIMESTAMP '1999-02-22 00:00:00' @@ -356,7 +356,7 @@ Per-Instance Resources: mem-estimate=50.00MB mem-reservation=35.00MB thread-rese | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.76MB mem-reservation=2.00MB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.item, RANDOM] | HDFS partitions=1/1 files=1 size=1.73MB | predicates: i_category IN ('Jewelry', 'Sports', 'Books') diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q99.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q99.test index 4773e65fb..c1f796523 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q99.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q99.test @@ -154,7 +154,7 @@ PLAN-ROOT SINK in pipelines: 00(GETNEXT) ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=24.23MB Threads=12 -Per-Host Resource Estimates: Memory=404MB +Per-Host Resource Estimates: Memory=405MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -190,7 +190,7 @@ Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservati | in pipelines: 00(GETNEXT) | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 -Per-Host Resources: mem-estimate=261.86MB mem-reservation=17.75MB thread-reservation=2 runtime-filters-memory=4.00MB +Per-Host Resources: mem-estimate=262.91MB mem-reservation=17.75MB thread-reservation=2 runtime-filters-memory=4.00MB 09:AGGREGATE [STREAMING] | output: sum(CAST(CASE WHEN (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) <= CAST(30 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) > CAST(30 AS BIGINT)) AND (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) <= CAST(60 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) > CAST(60 AS BIGINT)) AND (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) <= CAST(90 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) > CAST(90 AS BIGINT)) AND (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) <= CAST(120 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) > CAST(120 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)) | group by: substring(w_warehouse_name, CAST(1 AS BIGINT), CAST(20 AS BIGINT)), sm_type, cc_name @@ -212,7 +212,7 @@ Per-Host Resources: mem-estimate=261.86MB mem-reservation=17.75MB thread-reserva | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.05MB mem-reservation=512.00KB thread-reservation=2 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -240,7 +240,7 @@ Per-Host Resources: mem-estimate=261.86MB mem-reservation=17.75MB thread-reserva | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.12MB mem-reservation=16.00KB thread-reservation=2 | 03:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | HDFS partitions=1/1 files=1 size=10.28KB | stored statistics: @@ -265,7 +265,7 @@ Per-Host Resources: mem-estimate=261.86MB mem-reservation=17.75MB thread-reserva | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.11MB mem-reservation=16.00KB thread-reservation=2 | 02:SCAN HDFS [tpcds_parquet.ship_mode, RANDOM] | HDFS partitions=1/1 files=1 size=2.68KB | stored statistics: @@ -290,7 +290,7 @@ Per-Host Resources: mem-estimate=261.86MB mem-reservation=17.75MB thread-reserva | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2 +| Per-Host Resources: mem-estimate=32.15MB mem-reservation=16.00KB thread-reservation=2 | 01:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | stored statistics: @@ -313,7 +313,7 @@ Per-Host Resources: mem-estimate=261.86MB mem-reservation=17.75MB thread-reserva in pipelines: 00(GETNEXT) ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=35.98MB Threads=11 -Per-Host Resource Estimates: Memory=160MB +Per-Host Resource Estimates: Memory=161MB F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 | Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1 PLAN-ROOT SINK @@ -350,7 +350,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB -Per-Instance Resources: mem-estimate=58.00MB mem-reservation=6.00MB thread-reservation=1 +Per-Instance Resources: mem-estimate=59.16MB mem-reservation=6.00MB thread-reservation=1 09:AGGREGATE [STREAMING] | output: sum(CAST(CASE WHEN (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) <= CAST(30 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) > CAST(30 AS BIGINT)) AND (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) <= CAST(60 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) > CAST(60 AS BIGINT)) AND (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) <= CAST(90 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) > CAST(90 AS BIGINT)) AND (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) <= CAST(120 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN (CAST(cs_ship_date_sk AS BIGINT) - CAST(cs_sold_date_sk AS BIGINT) > CAST(120 AS BIGINT)) THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)) | group by: substring(w_warehouse_name, CAST(1 AS BIGINT), CAST(20 AS BIGINT)), sm_type, cc_name @@ -380,7 +380,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 04(GETNEXT) | | | F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1 | 04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM] | HDFS partitions=1/1 files=1 size=2.15MB | predicates: d_month_seq <= CAST(1211 AS INT), d_month_seq >= CAST(1200 AS INT) @@ -416,7 +416,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 03(GETNEXT) | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.12MB mem-reservation=16.00KB thread-reservation=1 | 03:SCAN HDFS [tpcds_parquet.call_center, RANDOM] | HDFS partitions=1/1 files=1 size=10.28KB | stored statistics: @@ -449,7 +449,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 02(GETNEXT) | | | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.11MB mem-reservation=16.00KB thread-reservation=1 | 02:SCAN HDFS [tpcds_parquet.ship_mode, RANDOM] | HDFS partitions=1/1 files=1 size=2.68KB | stored statistics: @@ -482,7 +482,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=6.00MB thread-reser | | in pipelines: 01(GETNEXT) | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 -| Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1 +| Per-Instance Resources: mem-estimate=16.15MB mem-reservation=16.00KB thread-reservation=1 | 01:SCAN HDFS [tpcds_parquet.warehouse, RANDOM] | HDFS partitions=1/1 files=1 size=4.38KB | stored statistics: diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test index b7a7e15eb..6961cfe16 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test @@ -41,7 +41,7 @@ PLAN-ROOT SINK row-size=80B cardinality=600.12K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=27.94MB Threads=4 -Per-Host Resource Estimates: Memory=300MB +Per-Host Resource Estimates: Memory=301MB PLAN-ROOT SINK | 05:MERGING-EXCHANGE [UNPARTITIONED] @@ -69,7 +69,7 @@ PLAN-ROOT SINK row-size=80B cardinality=600.12K ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=51.88MB Threads=5 -Per-Host Resource Estimates: Memory=244MB +Per-Host Resource Estimates: Memory=250MB PLAN-ROOT SINK | 05:MERGING-EXCHANGE [UNPARTITIONED] @@ -242,7 +242,7 @@ PLAN-ROOT SINK row-size=24B cardinality=800.00K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=73.03MB Threads=21 -Per-Host Resource Estimates: Memory=666MB +Per-Host Resource Estimates: Memory=668MB PLAN-ROOT SINK | 30:MERGING-EXCHANGE [UNPARTITIONED] @@ -369,7 +369,7 @@ PLAN-ROOT SINK row-size=24B cardinality=800.00K ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=90.59MB Threads=20 -Per-Host Resource Estimates: Memory=380MB +Per-Host Resource Estimates: Memory=382MB PLAN-ROOT SINK | 30:MERGING-EXCHANGE [UNPARTITIONED] @@ -594,7 +594,7 @@ PLAN-ROOT SINK row-size=46B cardinality=600.12K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=54.81MB Threads=8 -Per-Host Resource Estimates: Memory=558MB +Per-Host Resource Estimates: Memory=559MB PLAN-ROOT SINK | 11:MERGING-EXCHANGE [UNPARTITIONED] @@ -649,7 +649,7 @@ PLAN-ROOT SINK row-size=46B cardinality=600.12K ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=87.62MB Threads=9 -Per-Host Resource Estimates: Memory=392MB +Per-Host Resource Estimates: Memory=395MB PLAN-ROOT SINK | 11:MERGING-EXCHANGE [UNPARTITIONED] @@ -765,7 +765,7 @@ PLAN-ROOT SINK row-size=52B cardinality=600.12K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=42.69MB Threads=7 -Per-Host Resource Estimates: Memory=497MB +Per-Host Resource Estimates: Memory=498MB PLAN-ROOT SINK | 09:MERGING-EXCHANGE [UNPARTITIONED] @@ -808,7 +808,7 @@ PLAN-ROOT SINK row-size=52B cardinality=600.12K ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=68.62MB Threads=10 -Per-Host Resource Estimates: Memory=369MB +Per-Host Resource Estimates: Memory=374MB PLAN-ROOT SINK | 09:MERGING-EXCHANGE [UNPARTITIONED] @@ -951,7 +951,7 @@ PLAN-ROOT SINK row-size=32B cardinality=6.00M ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=88.27MB Threads=14 -Per-Host Resource Estimates: Memory=685MB +Per-Host Resource Estimates: Memory=686MB PLAN-ROOT SINK | 20:MERGING-EXCHANGE [UNPARTITIONED] @@ -1040,7 +1040,7 @@ PLAN-ROOT SINK row-size=32B cardinality=6.00M ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=145.52MB Threads=15 -Per-Host Resource Estimates: Memory=495MB +Per-Host Resource Estimates: Memory=497MB PLAN-ROOT SINK | 20:MERGING-EXCHANGE [UNPARTITIONED] @@ -1323,7 +1323,7 @@ PLAN-ROOT SINK row-size=54B cardinality=600.12K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=98.64MB Threads=16 -Per-Host Resource Estimates: Memory=713MB +Per-Host Resource Estimates: Memory=714MB PLAN-ROOT SINK | 22:MERGING-EXCHANGE [UNPARTITIONED] @@ -1417,7 +1417,7 @@ PLAN-ROOT SINK row-size=54B cardinality=600.12K ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=131.39MB Threads=21 -Per-Host Resource Estimates: Memory=522MB +Per-Host Resource Estimates: Memory=534MB PLAN-ROOT SINK | 22:MERGING-EXCHANGE [UNPARTITIONED] @@ -1659,7 +1659,7 @@ PLAN-ROOT SINK row-size=10B cardinality=150.00K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=78.27MB Threads=21 -Per-Host Resource Estimates: Memory=763MB +Per-Host Resource Estimates: Memory=767MB PLAN-ROOT SINK | 29:MERGING-EXCHANGE [UNPARTITIONED] @@ -1777,7 +1777,7 @@ PLAN-ROOT SINK row-size=40B cardinality=6.00M ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=111.90MB Threads=28 -Per-Host Resource Estimates: Memory=510MB +Per-Host Resource Estimates: Memory=531MB PLAN-ROOT SINK | 29:MERGING-EXCHANGE [UNPARTITIONED] @@ -2023,7 +2023,7 @@ PLAN-ROOT SINK row-size=48B cardinality=6.00M ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=145.63MB Threads=15 -Per-Host Resource Estimates: Memory=909MB +Per-Host Resource Estimates: Memory=910MB PLAN-ROOT SINK | 21:MERGING-EXCHANGE [UNPARTITIONED] @@ -2111,7 +2111,7 @@ PLAN-ROOT SINK row-size=48B cardinality=6.00M ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=214.38MB Threads=18 -Per-Host Resource Estimates: Memory=668MB +Per-Host Resource Estimates: Memory=677MB PLAN-ROOT SINK | 21:MERGING-EXCHANGE [UNPARTITIONED] @@ -2303,7 +2303,7 @@ PLAN-ROOT SINK row-size=197B cardinality=150.00K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=102.95MB Threads=11 -Per-Host Resource Estimates: Memory=646MB +Per-Host Resource Estimates: Memory=649MB PLAN-ROOT SINK | 15:MERGING-EXCHANGE [UNPARTITIONED] @@ -2371,7 +2371,7 @@ PLAN-ROOT SINK row-size=37B cardinality=2.00M ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=132.88MB Threads=14 -Per-Host Resource Estimates: Memory=477MB +Per-Host Resource Estimates: Memory=496MB PLAN-ROOT SINK | 15:MERGING-EXCHANGE [UNPARTITIONED] @@ -2831,7 +2831,7 @@ PLAN-ROOT SINK row-size=28B cardinality=1.50M ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=71.94MB Threads=7 -Per-Host Resource Estimates: Memory=532MB +Per-Host Resource Estimates: Memory=533MB PLAN-ROOT SINK | 09:MERGING-EXCHANGE [UNPARTITIONED] @@ -2873,7 +2873,7 @@ PLAN-ROOT SINK row-size=90B cardinality=320.78K ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=96.88MB Threads=10 -Per-Host Resource Estimates: Memory=409MB +Per-Host Resource Estimates: Memory=416MB PLAN-ROOT SINK | 09:MERGING-EXCHANGE [UNPARTITIONED] @@ -2975,7 +2975,7 @@ PLAN-ROOT SINK row-size=77B cardinality=150.00K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=58.50MB Threads=7 -Per-Host Resource Estimates: Memory=298MB +Per-Host Resource Estimates: Memory=299MB PLAN-ROOT SINK | 10:MERGING-EXCHANGE [UNPARTITIONED] @@ -3112,7 +3112,7 @@ PLAN-ROOT SINK row-size=46B cardinality=600.12K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=26.50MB Threads=6 -Per-Host Resource Estimates: Memory=355MB +Per-Host Resource Estimates: Memory=356MB PLAN-ROOT SINK | 07:AGGREGATE [FINALIZE] @@ -3145,7 +3145,7 @@ PLAN-ROOT SINK row-size=46B cardinality=600.12K ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=36.50MB Threads=8 -Per-Host Resource Estimates: Memory=254MB +Per-Host Resource Estimates: Memory=258MB PLAN-ROOT SINK | 07:AGGREGATE [FINALIZE] @@ -3261,7 +3261,7 @@ PLAN-ROOT SINK row-size=102B cardinality=10.00K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=47.75MB Threads=10 -Per-Host Resource Estimates: Memory=622MB +Per-Host Resource Estimates: Memory=624MB PLAN-ROOT SINK | 17:MERGING-EXCHANGE [UNPARTITIONED] @@ -3334,7 +3334,7 @@ PLAN-ROOT SINK row-size=46B cardinality=600.12K ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=88.50MB Threads=14 -Per-Host Resource Estimates: Memory=489MB +Per-Host Resource Estimates: Memory=495MB PLAN-ROOT SINK | 17:MERGING-EXCHANGE [UNPARTITIONED] @@ -3550,7 +3550,7 @@ PLAN-ROOT SINK row-size=16B cardinality=800.00K ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=59.94MB Threads=8 -Per-Host Resource Estimates: Memory=207MB +Per-Host Resource Estimates: Memory=208MB PLAN-ROOT SINK | 14:MERGING-EXCHANGE [UNPARTITIONED] @@ -3678,7 +3678,7 @@ PLAN-ROOT SINK row-size=16B cardinality=6.00M ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=42.75MB Threads=8 -Per-Host Resource Estimates: Memory=621MB +Per-Host Resource Estimates: Memory=622MB PLAN-ROOT SINK | 12:AGGREGATE [FINALIZE] @@ -3734,7 +3734,7 @@ PLAN-ROOT SINK row-size=16B cardinality=6.00M ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=74.62MB Threads=11 -Per-Host Resource Estimates: Memory=441MB +Per-Host Resource Estimates: Memory=446MB PLAN-ROOT SINK | 12:AGGREGATE [FINALIZE] @@ -3887,7 +3887,7 @@ PLAN-ROOT SINK row-size=16B cardinality=6.00M ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=217.75MB Threads=11 -Per-Host Resource Estimates: Memory=1016MB +Per-Host Resource Estimates: Memory=1018MB PLAN-ROOT SINK | 17:MERGING-EXCHANGE [UNPARTITIONED] @@ -3963,7 +3963,7 @@ PLAN-ROOT SINK row-size=16B cardinality=6.00M ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=322.75MB Threads=16 -Per-Host Resource Estimates: Memory=830MB +Per-Host Resource Estimates: Memory=839MB PLAN-ROOT SINK | 17:MERGING-EXCHANGE [UNPARTITIONED] @@ -4288,7 +4288,7 @@ PLAN-ROOT SINK row-size=46B cardinality=600.12K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=90.57MB Threads=13 -Per-Host Resource Estimates: Memory=659MB +Per-Host Resource Estimates: Memory=660MB PLAN-ROOT SINK | 20:MERGING-EXCHANGE [UNPARTITIONED] @@ -4376,7 +4376,7 @@ PLAN-ROOT SINK row-size=46B cardinality=600.12K ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=111.38MB Threads=15 -Per-Host Resource Estimates: Memory=459MB +Per-Host Resource Estimates: Memory=462MB PLAN-ROOT SINK | 20:MERGING-EXCHANGE [UNPARTITIONED] @@ -4686,7 +4686,7 @@ PLAN-ROOT SINK row-size=60B cardinality=600.12K ---- PARALLELPLANS Max Per-Host Resource Reservation: Memory=104.88MB Threads=22 -Per-Host Resource Estimates: Memory=800MB +Per-Host Resource Estimates: Memory=810MB PLAN-ROOT SINK | 21:MERGING-EXCHANGE [UNPARTITIONED] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test index 352480215..8548ccdc6 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test @@ -41,7 +41,7 @@ PLAN-ROOT SINK row-size=68B cardinality=1.50M ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=116.00MB Threads=4 -Per-Host Resource Estimates: Memory=1.03GB +Per-Host Resource Estimates: Memory=1.04GB PLAN-ROOT SINK | 05:MERGING-EXCHANGE [UNPARTITIONED] @@ -415,7 +415,7 @@ PLAN-ROOT SINK row-size=33B cardinality=30.00K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=104.00MB Threads=4 -Per-Host Resource Estimates: Memory=886MB +Per-Host Resource Estimates: Memory=887MB PLAN-ROOT SINK | 13:MERGING-EXCHANGE [UNPARTITIONED] @@ -1548,7 +1548,7 @@ PLAN-ROOT SINK row-size=209B cardinality=150.00K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=66.89MB Threads=6 -Per-Host Resource Estimates: Memory=892MB +Per-Host Resource Estimates: Memory=894MB PLAN-ROOT SINK | 16:MERGING-EXCHANGE [UNPARTITIONED] @@ -1726,7 +1726,7 @@ PLAN-ROOT SINK row-size=14B cardinality=10.00K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=115.91MB Threads=11 -Per-Host Resource Estimates: Memory=730MB +Per-Host Resource Estimates: Memory=731MB PLAN-ROOT SINK | 25:MERGING-EXCHANGE [UNPARTITIONED] @@ -2185,7 +2185,7 @@ PLAN-ROOT SINK row-size=36B cardinality=1.50M ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=190.88MB Threads=10 -Per-Host Resource Estimates: Memory=1.60GB +Per-Host Resource Estimates: Memory=1.61GB PLAN-ROOT SINK | 17:MERGING-EXCHANGE [UNPARTITIONED] @@ -2448,7 +2448,7 @@ PLAN-ROOT SINK row-size=24B cardinality=15.00M ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=154.94MB Threads=8 -Per-Host Resource Estimates: Memory=986MB +Per-Host Resource Estimates: Memory=987MB PLAN-ROOT SINK | 12:AGGREGATE [FINALIZE] @@ -3200,7 +3200,7 @@ PLAN-ROOT SINK row-size=47B cardinality=15.00K ---- DISTRIBUTEDPLAN Max Per-Host Resource Reservation: Memory=31.94MB Threads=8 -Per-Host Resource Estimates: Memory=652MB +Per-Host Resource Estimates: Memory=653MB PLAN-ROOT SINK | 15:MERGING-EXCHANGE [UNPARTITIONED] diff --git a/testdata/workloads/functional-query/queries/QueryTest/dedicated-coord-mem-estimates.test b/testdata/workloads/functional-query/queries/QueryTest/dedicated-coord-mem-estimates.test index 5a3d65bcf..29731ae91 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/dedicated-coord-mem-estimates.test +++ b/testdata/workloads/functional-query/queries/QueryTest/dedicated-coord-mem-estimates.test @@ -39,9 +39,9 @@ row_regex: .*Cluster Memory Admitted: 169.47 MB.* # SELECT with non-merging exchange. select * from functional.alltypes; ---- RUNTIME_PROFILE -row_regex: .*Per-Host Resource Estimates: Memory=20MB.* +row_regex: .*Per-Host Resource Estimates: Memory=21MB.* row_regex: .*Dedicated Coordinator Resource Estimate: Memory=104MB.* -row_regex: .*Cluster Memory Admitted: 145.47 MB.* +row_regex: .*Cluster Memory Admitted: 146.20 MB.* ==== ---- QUERY # SELECT with a non-grouping aggregate in the coordinator fragment. diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test index 5ffcd130e..75444c03e 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test +++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test @@ -67,7 +67,7 @@ from functional_avro.alltypes t1 left outer join functional_avro.alltypes t3 on (t2.id = t3.id) where t1.month = 1 and t2.year = 2009 and t3.bool_col = false ---- RESULTS: VERIFY_IS_SUBSET -'Per-Host Resource Estimates: Memory=54MB' +'Per-Host Resource Estimates: Memory=55MB' 'WARNING: The following tables are missing relevant table and/or column statistics.' 'functional_avro.alltypes, functional_parquet.alltypessmall' ==== @@ -91,6 +91,6 @@ explain select year, month from functional.alltypes intersect select year, month from functional.alltypes where year=2009; ---- RESULTS: VERIFY_IS_SUBSET -'Per-Host Resources: mem-estimate=13.97MB mem-reservation=5.88MB thread-reservation=1 runtime-filters-memory=2.00MB' +'Per-Host Resources: mem-estimate=13.98MB mem-reservation=5.88MB thread-reservation=1 runtime-filters-memory=2.00MB' '04:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]' -==== \ No newline at end of file +==== diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling-large-rows.test b/testdata/workloads/functional-query/queries/QueryTest/spilling-large-rows.test index 24a243aa6..ac08b4a2b 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/spilling-large-rows.test +++ b/testdata/workloads/functional-query/queries/QueryTest/spilling-large-rows.test @@ -21,6 +21,7 @@ Row of size 9.54 MB could not be materialized by AGGREGATION_NODE ==== ---- QUERY # Agg should be able to process the large strings if we increase the row size. +set mem_limit="1gb"; set max_row_size=10m; select id, count(distinct bigstr) from bigstrs @@ -152,7 +153,7 @@ Row of size 9.54 MB could not be materialized by SORT_NODE (id=1). Increase the ---- QUERY # Sort and analytic should be able to process the large strings if we increase the row # size. -set mem_limit="1gb"; +set mem_limit="1.4gb"; set max_row_size=10m; SELECT id, int_col, substring(bigstr, 1, 10), substring(bigstr, 9999999, 1), rank FROM ( diff --git a/testdata/workloads/tpch/queries/datastream-sender.test b/testdata/workloads/tpch/queries/datastream-sender.test new file mode 100644 index 000000000..01b03dca7 --- /dev/null +++ b/testdata/workloads/tpch/queries/datastream-sender.test @@ -0,0 +1,48 @@ +==== +---- QUERY: SHUFFLES +# Shuffling around 1 GB of rows +SET mem_limit="1gb"; +WITH cte1 + AS (SELECT c1, + c2, + orderkey + FROM (SELECT Repeat(Group_concat(o_comment, o_comment), 20) AS c1, + Repeat(Group_concat(o_comment, o_comment), 20) AS c2, + o_orderkey % 700 AS + orderkey + FROM orders + WHERE o_orderkey < 800000000 + GROUP BY orderkey + UNION ALL + SELECT Repeat(Group_concat(o_comment, o_comment), 20) AS c1, + Repeat(Group_concat(o_comment, o_comment), 20) AS c2, + o_orderkey % 700 AS + orderkey + FROM orders + WHERE o_orderkey = 1 + GROUP BY orderkey) a), + cte2 + AS (SELECT o_orderkey % 700 AS orderkey + FROM orders + WHERE o_orderkey < 800000000 + GROUP BY orderkey + UNION ALL + SELECT o_orderkey % 700 AS orderkey + FROM orders + WHERE o_orderkey = 1 + GROUP BY orderkey) +SELECT /* +straight_join */ Count(*), + Max(Length(t1.c1)), + Max(Length(t1.c2)), + Max(Length(Substr(t1.c1, 1, 30))), + Max(Length(Substr(t1.c2, 1, 30))) +FROM cte1 t1 + JOIN /* +shuffle */ cte2 t2 + ON t1.orderkey = t2.orderkey +---- RESULTS +703,4251520,4251520,30,30 +---- TYPES +BIGINT, INT, INT, INT, INT +---- RUNTIME_PROFILE +row_regex:.*:EXCHANGE SENDER.*[0-9\.]+ MB.* +===== diff --git a/tests/query_test/test_datastream_sender.py b/tests/query_test/test_datastream_sender.py new file mode 100644 index 000000000..c48798884 --- /dev/null +++ b/tests/query_test/test_datastream_sender.py @@ -0,0 +1,35 @@ +# 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. + +from tests.common.impala_test_suite import ImpalaTestSuite + + +class TestDataStreamSenderTpch(ImpalaTestSuite): + """"Test suite to verify the counters in runtime profile update + using queries that send row batch between hosts""" + @classmethod + def get_workload(cls): + return 'tpch' + + @classmethod + def add_test_dimensions(cls): + super(TestDataStreamSenderTpch, cls).add_test_dimensions() + cls.ImpalaTestMatrix.add_constraint(lambda v: + v.get_value('table_format').file_format in ['parquet']) + + def test_krpc_datastream_sender_shuffle(self, vector): + self.run_test_case('datastream-sender', vector)