diff --git a/be/src/exec/blocking-join-node.cc b/be/src/exec/blocking-join-node.cc index cd3dd4665..eeaccd6a0 100644 --- a/be/src/exec/blocking-join-node.cc +++ b/be/src/exec/blocking-join-node.cc @@ -144,21 +144,28 @@ void BlockingJoinNode::Close(RuntimeState* state) { ExecNode::Close(state); } -void BlockingJoinNode::ProcessBuildInputAsync(RuntimeState* state, DataSink* build_sink, - Status* status) { +void BlockingJoinNode::ProcessBuildInputAsync( + RuntimeState* state, DataSink* build_sink, Status* status) { DCHECK(status != nullptr); SCOPED_THREAD_COUNTER_MEASUREMENT(state->total_thread_statistics()); - if (build_sink == nullptr){ - *status = ProcessBuildInput(state); - } else { - *status = SendBuildInputToSink(state, build_sink); + { + SCOPED_STOP_WATCH(&built_probe_overlap_stop_watch_); + *status = child(1)->Open(state); + } + if (status->ok()) *status = AcquireResourcesForBuild(state); + if (status->ok()) { + if (build_sink == nullptr){ + *status = ProcessBuildInput(state); + } else { + *status = SendBuildInputToSink(state, build_sink); + } } // IMPALA-1863: If the build-side thread failed, then we need to close the right // (build-side) child to avoid a potential deadlock between fragment instances. This // is safe to do because while the build may have partially completed, it will not be - // probed. BlockJoinNode::Open() will return failure as soon as child(0)->Open() + // probed. BlockingJoinNode::Open() will return failure as soon as child(0)->Open() // completes. - if (!status->ok()) child(1)->Close(state); + if (CanCloseBuildEarly() || !status->ok()) child(1)->Close(state); // Release the thread token as soon as possible (before the main thread joins // on it). This way, if we had a chain of 10 joins using 1 additional thread, // we'd keep the additional thread busy the whole time. @@ -180,9 +187,9 @@ Status BlockingJoinNode::ProcessBuildInputAndOpenProbe( // Inside a subplan we expect Open() to be called a number of times proportional to the // input data of the SubplanNode, so we prefer doing processing the build input in the // main thread, assuming that thread creation is expensive relative to a single subplan - // iteration. + // iteration. TODO-MT: disable async build thread when mt_dop >= 1. // - // In this block, we also compute the 'overlap' time for the left and right child. This + // In this block, we also compute the 'overlap' time for the left and right child. This // is the time (i.e. clock reads) when the right child stops overlapping with the left // child. For the single threaded case, the left and right child never overlap. For the // build side in a different thread, the overlap stops when the left child Open() @@ -217,6 +224,8 @@ Status BlockingJoinNode::ProcessBuildInputAndOpenProbe( // TODO: Remove this special-case behavior for subplans once we have proper // projection. See UnnestNode for details on the current projection implementation. RETURN_IF_ERROR(child(0)->Open(state)); + RETURN_IF_ERROR(child(1)->Open(state)); + RETURN_IF_ERROR(AcquireResourcesForBuild(state)); if (build_sink == NULL) { RETURN_IF_ERROR(ProcessBuildInput(state)); } else { @@ -225,11 +234,16 @@ Status BlockingJoinNode::ProcessBuildInputAndOpenProbe( } else { // The left/right child never overlap. The overlap stops here. built_probe_overlap_stop_watch_.SetTimeCeiling(); + // Open the build side before acquiring our own resources so that the build side + // can release any resources only used during its Open(). + RETURN_IF_ERROR(child(1)->Open(state)); + RETURN_IF_ERROR(AcquireResourcesForBuild(state)); if (build_sink == NULL) { RETURN_IF_ERROR(ProcessBuildInput(state)); } else { RETURN_IF_ERROR(SendBuildInputToSink(state, build_sink)); } + if (CanCloseBuildEarly()) child(1)->Close(state); RETURN_IF_ERROR(child(0)->Open(state)); } return Status::OK(); @@ -258,11 +272,6 @@ Status BlockingJoinNode::GetFirstProbeRow(RuntimeState* state) { template Status BlockingJoinNode::SendBuildInputToSink(RuntimeState* state, DataSink* build_sink) { - { - CONDITIONAL_SCOPED_STOP_WATCH(&built_probe_overlap_stop_watch_, ASYNC_BUILD); - RETURN_IF_ERROR(child(1)->Open(state)); - } - { SCOPED_TIMER(build_timer_); RETURN_IF_ERROR(build_sink->Open(state)); diff --git a/be/src/exec/blocking-join-node.h b/be/src/exec/blocking-join-node.h index a14d97915..1972b34ed 100644 --- a/be/src/exec/blocking-join-node.h +++ b/be/src/exec/blocking-join-node.h @@ -101,27 +101,35 @@ class BlockingJoinNode : public ExecNode { /// with the probe child Open(). MonotonicStopWatch built_probe_overlap_stop_watch_; + // True for a join node subclass if the build side can be closed before the probe + // side is opened. Should be true wherever possible to reduce resource consumption. + // E.g. this is true or PartitionedHashJoinNode because it rematerializes the build rows + // and false for NestedLoopJoinNode because it accumulates RowBatches that may reference + // memory still owned by the build-side ExecNode tree. + // Changes here must be kept in sync with the planner's resource profile computation. + // TODO: IMPALA-4179: this should always be true once resource transfer has been fixed. + virtual bool CanCloseBuildEarly() const { return false; } + + /// Called by BlockingJoinNode after opening child(1) succeeds and before + /// SendBuildInputToSink is called to allocate resources for this ExecNode. + virtual Status AcquireResourcesForBuild(RuntimeState* state) { return Status::OK(); } + /// Processes the build-side input. /// Called from ProcessBuildInputAndOpenProbe() if the subclass does not provide a - /// DataSink to consume the build input. + /// DataSink to consume the build input. The build-side input is already open when + /// this is called. /// Note that this can be called concurrently with Open'ing the left child to /// increase parallelism. If, for example, the left child is another join node, /// it can start its own build at the same time. /// TODO: move all subclasses to use the DataSink interface and remove this method. virtual Status ProcessBuildInput(RuntimeState* state) = 0; - /// Processes the build-side input and opens the probe side. Will do both concurrently - /// if the plan shape and thread token availability permit it. - /// If 'build_sink' is non-NULL, sends the build-side input to 'build_sink'. Otherwise - /// calls ProcessBuildInput on the subclass. + /// Processes the build-side input, which should be already open, and opens the probe + /// side. Will do both concurrently if not in a subplan and an extra thread token is + /// available. If 'build_sink' is non-NULL, sends the build-side input to 'build_sink'. + /// Otherwise calls ProcessBuildInput on the subclass. Status ProcessBuildInputAndOpenProbe(RuntimeState* state, DataSink* build_sink); - /// Helper function to process the build input by sending it to a DataSink. - /// ASYNC_BUILD enables timers that impose some overhead but are required if the build - /// is processed concurrently with the Open() of the left child. - template - Status SendBuildInputToSink(RuntimeState* state, DataSink* build_sink); - /// Set up 'current_probe_row_' to point to the first input row from the left child /// (probe side). Fills 'probe_batch_' with rows from the left child and updates /// 'probe_batch_pos_' to the index of the row in 'probe_batch_' after @@ -199,12 +207,19 @@ class BlockingJoinNode : public ExecNode { const MonotonicStopWatch* child_overlap_timer); private: - /// The main function for the thread that processes the build input asynchronously. - /// Its status is returned in the 'status' promise. If 'build_sink' is non-NULL, it - /// is used for the build. Otherwise, ProcessBuildInput() is called on the subclass. + /// Helper function to process the build input by sending it to a DataSink. The build + /// input must already be open before calling this. ASYNC_BUILD enables timers that + /// impose some overhead but are required if the build is processed concurrently with + /// the Open() of the left child. + template + Status SendBuildInputToSink(RuntimeState* state, DataSink* build_sink); + + /// The main function for the thread that opens the build side and processes the build + /// input asynchronously. Its status is returned in the 'status' promise. If + /// 'build_sink' is non-NULL, it is used for the build. Otherwise, ProcessBuildInput() + /// is called on the subclass. void ProcessBuildInputAsync(RuntimeState* state, DataSink* build_sink, Status* status); }; - } #endif diff --git a/be/src/exec/exec-node.h b/be/src/exec/exec-node.h index b6dcd6e86..a107f6214 100644 --- a/be/src/exec/exec-node.h +++ b/be/src/exec/exec-node.h @@ -83,6 +83,14 @@ class ExecNode { /// If overridden in subclass, must first call superclass's Open(). /// Open() is called after Prepare() or Reset(), i.e., possibly multiple times /// throughout the lifetime of this node. + /// + /// Memory resources must be acquired by an ExecNode only during or after the first + /// call to Open(). Blocking ExecNodes outside of a subplan must call Open() on their + /// child before acquiring their own resources to reduce the peak resource requirement. + /// This is particularly important if there are multiple blocking ExecNodes in a + /// pipeline because the lower nodes will release resources in Close() before the + /// Open() of their parent retuns. The resource profile calculation in the frontend + /// relies on this when computing the peak resources required for a query. virtual Status Open(RuntimeState* state) WARN_UNUSED_RESULT; /// Retrieves rows and returns them via row_batch. Sets eos to true diff --git a/be/src/exec/hash-join-node.cc b/be/src/exec/hash-join-node.cc index deee7f27e..c35d05d28 100644 --- a/be/src/exec/hash-join-node.cc +++ b/be/src/exec/hash-join-node.cc @@ -223,10 +223,6 @@ Status HashJoinNode::ProcessBuildInput(RuntimeState* state) { // row ptrs. The row ptrs are copied into the hash table's internal structure so they // don't need to be stored in the build_pool_. RowBatch build_batch(child(1)->row_desc(), state->batch_size(), mem_tracker()); - { - SCOPED_STOP_WATCH(&built_probe_overlap_stop_watch_); - RETURN_IF_ERROR(child(1)->Open(state)); - } while (true) { RETURN_IF_CANCELLED(state); RETURN_IF_ERROR(QueryMaintenance(state)); diff --git a/be/src/exec/partitioned-aggregation-node.cc b/be/src/exec/partitioned-aggregation-node.cc index 83ebbc2d3..e5ab0f39e 100644 --- a/be/src/exec/partitioned-aggregation-node.cc +++ b/be/src/exec/partitioned-aggregation-node.cc @@ -263,6 +263,9 @@ void PartitionedAggregationNode::Codegen(RuntimeState* state) { Status PartitionedAggregationNode::Open(RuntimeState* state) { SCOPED_TIMER(runtime_profile_->total_time_counter()); + // Open the child before consuming resources in this node. + RETURN_IF_ERROR(child(0)->Open(state)); + RETURN_IF_ERROR(ExecNode::Open(state)); if (ht_ctx_.get() != nullptr) RETURN_IF_ERROR(ht_ctx_->Open(state)); RETURN_IF_ERROR(AggFnEvaluator::Open(agg_fn_evals_, state)); @@ -278,8 +281,6 @@ Status PartitionedAggregationNode::Open(RuntimeState* state) { RETURN_IF_ERROR(CreateHashPartitions(0)); } - RETURN_IF_ERROR(children_[0]->Open(state)); - // Streaming preaggregations do all processing in GetNext(). if (is_streaming_preagg_) return Status::OK(); diff --git a/be/src/exec/partitioned-hash-join-node.cc b/be/src/exec/partitioned-hash-join-node.cc index f93174390..927bf1ad6 100644 --- a/be/src/exec/partitioned-hash-join-node.cc +++ b/be/src/exec/partitioned-hash-join-node.cc @@ -156,11 +156,6 @@ Status PartitionedHashJoinNode::Open(RuntimeState* state) { RETURN_IF_ERROR(ht_ctx_->Open(state)); RETURN_IF_ERROR(ScalarExprEvaluator::Open(other_join_conjunct_evals_, state)); - if (join_op_ == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) { - RETURN_IF_ERROR(InitNullAwareProbePartition()); - RETURN_IF_ERROR(InitNullProbeRows()); - } - // Check for errors and free local allocations before opening children. RETURN_IF_CANCELLED(state); RETURN_IF_ERROR(QueryMaintenance(state)); @@ -181,6 +176,16 @@ Status PartitionedHashJoinNode::Open(RuntimeState* state) { return Status::OK(); } +Status PartitionedHashJoinNode::AcquireResourcesForBuild(RuntimeState* state) { + if (join_op_ == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) { + // Initialize these partitions before doing the build so that the build does not + // use the reservation intended for them. + RETURN_IF_ERROR(InitNullAwareProbePartition()); + RETURN_IF_ERROR(InitNullProbeRows()); + } + return Status::OK(); +} + Status PartitionedHashJoinNode::QueryMaintenance(RuntimeState* state) { // Build expressions may be evaluated during probing, so must be freed. // Probe side expr is not included in QueryMaintenance(). We cache the probe expression diff --git a/be/src/exec/partitioned-hash-join-node.h b/be/src/exec/partitioned-hash-join-node.h index 68177ad00..bf90ae466 100644 --- a/be/src/exec/partitioned-hash-join-node.h +++ b/be/src/exec/partitioned-hash-join-node.h @@ -108,18 +108,23 @@ class PartitionedHashJoinNode : public BlockingJoinNode { const DescriptorTbl& descs); virtual ~PartitionedHashJoinNode(); - virtual Status Init(const TPlanNode& tnode, RuntimeState* state); - virtual Status Prepare(RuntimeState* state); - virtual void Codegen(RuntimeState* state); - virtual Status Open(RuntimeState* state); - virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos); - virtual Status Reset(RuntimeState* state); - virtual void Close(RuntimeState* state); + virtual Status Init(const TPlanNode& tnode, RuntimeState* state) override; + virtual Status Prepare(RuntimeState* state) override; + virtual void Codegen(RuntimeState* state) override; + virtual Status Open(RuntimeState* state) override; + virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override; + virtual Status Reset(RuntimeState* state) override; + virtual void Close(RuntimeState* state) override; protected: - virtual Status QueryMaintenance(RuntimeState* state); - virtual void AddToDebugString(int indentation_level, std::stringstream* out) const; - virtual Status ProcessBuildInput(RuntimeState* state); + virtual Status QueryMaintenance(RuntimeState* state) override; + virtual void AddToDebugString( + int indentation_level, std::stringstream* out) const override; + virtual Status ProcessBuildInput(RuntimeState* state) override; + + // Safe to close the build side early because we rematerialize the build rows always. + virtual bool CanCloseBuildEarly() const override { return true; } + virtual Status AcquireResourcesForBuild(RuntimeState* state) override; private: class ProbePartition; diff --git a/be/src/exec/sort-node.cc b/be/src/exec/sort-node.cc index 552ee8937..fd4212493 100644 --- a/be/src/exec/sort-node.cc +++ b/be/src/exec/sort-node.cc @@ -69,20 +69,17 @@ void SortNode::Codegen(RuntimeState* state) { Status SortNode::Open(RuntimeState* state) { SCOPED_TIMER(runtime_profile_->total_time_counter()); + // Open the child before consuming resources in this node. + RETURN_IF_ERROR(child(0)->Open(state)); RETURN_IF_ERROR(ExecNode::Open(state)); RETURN_IF_ERROR(less_than_->Open(pool_, state, expr_mem_pool())); RETURN_IF_ERROR(sorter_->Open()); RETURN_IF_CANCELLED(state); RETURN_IF_ERROR(QueryMaintenance(state)); - RETURN_IF_ERROR(child(0)->Open(state)); // The child has been opened and the sorter created. Sort the input. // The final merge is done on-demand as rows are requested in GetNext(). RETURN_IF_ERROR(SortInput(state)); - - // Unless we are inside a subplan expecting to call Open()/GetNext() on the child - // again, the child can be closed at this point. - if (!IsInSubplan()) child(0)->Close(state); return Status::OK(); } @@ -163,12 +160,17 @@ Status SortNode::SortInput(RuntimeState* state) { RowBatch batch(child(0)->row_desc(), state->batch_size(), mem_tracker()); bool eos; do { - batch.Reset(); RETURN_IF_ERROR(child(0)->GetNext(state, &batch, &eos)); RETURN_IF_ERROR(sorter_->AddBatch(&batch)); + batch.Reset(); RETURN_IF_CANCELLED(state); RETURN_IF_ERROR(QueryMaintenance(state)); } while(!eos); + + // Unless we are inside a subplan expecting to call Open()/GetNext() on the child + // again, the child can be closed at this point to release resources. + if (!IsInSubplan()) child(0)->Close(state); + RETURN_IF_ERROR(sorter_->InputDone()); return Status::OK(); } diff --git a/be/src/util/backend-gflag-util.cc b/be/src/util/backend-gflag-util.cc index 96c8f8794..ab66a73f5 100644 --- a/be/src/util/backend-gflag-util.cc +++ b/be/src/util/backend-gflag-util.cc @@ -27,6 +27,7 @@ DECLARE_bool(load_catalog_in_background); DECLARE_bool(load_auth_to_local_rules); DECLARE_bool(enable_stats_extrapolation); +DECLARE_bool(enable_partitioned_hash_join); DECLARE_int32(non_impala_java_vlog); DECLARE_int32(read_size); DECLARE_int32(num_metadata_loading_threads); @@ -71,6 +72,7 @@ Status GetThriftBackendGflags(JNIEnv* jni_env, jbyteArray* cfg_bytes) { cfg.__set_lineage_event_log_dir(FLAGS_lineage_event_log_dir); cfg.__set_local_library_path(FLAGS_local_library_dir); cfg.__set_kudu_operation_timeout_ms(FLAGS_kudu_operation_timeout_ms); + cfg.__set_enable_partitioned_hash_join(FLAGS_enable_partitioned_hash_join); RETURN_IF_ERROR(SerializeThriftMsg(jni_env, &cfg, cfg_bytes)); return Status::OK(); } diff --git a/common/thrift/BackendGflags.thrift b/common/thrift/BackendGflags.thrift index c76d8e086..3a410c282 100644 --- a/common/thrift/BackendGflags.thrift +++ b/common/thrift/BackendGflags.thrift @@ -56,4 +56,6 @@ struct TBackendGflags { 16: required i32 kudu_operation_timeout_ms 17: required i32 initial_hms_cnxn_timeout_s + + 18: required bool enable_partitioned_hash_join } diff --git a/common/thrift/Frontend.thrift b/common/thrift/Frontend.thrift index e07fe84a4..79da0d6ed 100644 --- a/common/thrift/Frontend.thrift +++ b/common/thrift/Frontend.thrift @@ -389,14 +389,23 @@ struct TQueryExecRequest { // Estimated per-host peak memory consumption in bytes. Used for resource management. 8: optional i64 per_host_mem_estimate - // Minimum buffer reservation required per host in bytes. + // Minimum query-wide buffer reservation required per host in bytes. This is the peak + // minimum reservation that may be required by the concurrently-executing operators at + // any point in query execution. It may be less than the initial reservation total + // claims (below) if execution of some operators never overlaps, which allows reuse of + // reservations. 9: optional i64 per_host_min_reservation; + // Total of the initial buffer reservations that we expect to be claimed per host. + // I.e. the sum over all operators in all fragment instances that execute on that host. + // Measured in bytes. + 10: optional i64 per_host_initial_reservation_total_claims; + // List of replica hosts. Used by the host_idx field of TScanRangeLocation. - 10: required list host_list + 11: required list host_list // Column lineage graph - 11: optional LineageGraph.TLineageGraph lineage_graph + 12: optional LineageGraph.TLineageGraph lineage_graph } enum TCatalogOpType { diff --git a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java index 33fbb8b3d..365091daf 100644 --- a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java +++ b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java @@ -19,7 +19,6 @@ package org.apache.impala.analysis; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; diff --git a/fe/src/main/java/org/apache/impala/catalog/HBaseTable.java b/fe/src/main/java/org/apache/impala/catalog/HBaseTable.java index 4f60c96e5..43cf2b946 100644 --- a/fe/src/main/java/org/apache/impala/catalog/HBaseTable.java +++ b/fe/src/main/java/org/apache/impala/catalog/HBaseTable.java @@ -19,7 +19,6 @@ package org.apache.impala.catalog; import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -33,10 +32,8 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.ServerLoad; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.RegionLoad; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; @@ -670,11 +667,6 @@ public class HBaseTable extends Table { return hbaseTableName_; } - public int getNumNodes() { - // TODO: implement - return 100; - } - @Override public TCatalogObjectType getCatalogObjectType() { return TCatalogObjectType.TABLE; diff --git a/fe/src/main/java/org/apache/impala/common/TreeNode.java b/fe/src/main/java/org/apache/impala/common/TreeNode.java index 730fd4411..b89f13352 100644 --- a/fe/src/main/java/org/apache/impala/common/TreeNode.java +++ b/fe/src/main/java/org/apache/impala/common/TreeNode.java @@ -66,6 +66,22 @@ public abstract class TreeNode> { for (NodeType child: children_) child.getNodesPreOrderAux(result); } + /** + * Return list of all nodes of the tree rooted at 'this', obtained + * through post-order traversal. + */ + public > ArrayList getNodesPostOrder() { + ArrayList result = new ArrayList(); + getNodesPostOrderAux(result); + return result; + } + + protected > void getNodesPostOrderAux( + ArrayList result) { + for (NodeType child: children_) child.getNodesPostOrderAux(result); + result.add((C) this); + } + /** * Count the total number of nodes in this tree. Leaf node will return 1. * Non-leaf node will include all its children. diff --git a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java index d1b7419ba..004c84efe 100644 --- a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java +++ b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java @@ -281,7 +281,7 @@ public class AggregationNode extends PlanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { Preconditions.checkNotNull( fragment_, "PlanNode must be placed into a fragment before calling this method."); long perInstanceCardinality = fragment_.getPerInstanceNdv( @@ -319,7 +319,7 @@ public class AggregationNode extends PlanNode { perInstanceMinBuffers = bufferSize * minBuffers; } - resourceProfile_ = + nodeResourceProfile_ = new ResourceProfile(perInstanceMemEstimate, perInstanceMinBuffers); } } diff --git a/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java b/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java index f17226b06..d4bafcf91 100644 --- a/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java +++ b/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java @@ -100,7 +100,7 @@ public class AnalyticEvalNode extends PlanNode { } @Override - public boolean isBlockingNode() { return true; } + public boolean isBlockingNode() { return false; } public List getPartitionExprs() { return partitionExprs_; } public List getOrderByElements() { return orderByElements_; } @@ -242,7 +242,7 @@ public class AnalyticEvalNode extends PlanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { Preconditions.checkNotNull( fragment_, "PlanNode must be placed into a fragment before calling this method."); // TODO: come up with estimate based on window @@ -250,6 +250,6 @@ public class AnalyticEvalNode extends PlanNode { // Must be kept in sync with MIN_REQUIRED_BUFFERS in AnalyticEvalNode in be. long perInstanceMinBufferBytes = 2 * getDefaultSpillableBufferBytes(); - resourceProfile_ = new ResourceProfile(perInstanceMemEstimate, perInstanceMinBufferBytes); + nodeResourceProfile_ = new ResourceProfile(perInstanceMemEstimate, perInstanceMinBufferBytes); } } diff --git a/fe/src/main/java/org/apache/impala/planner/DataSink.java b/fe/src/main/java/org/apache/impala/planner/DataSink.java index 7fc0c83ae..6acb4b9dd 100644 --- a/fe/src/main/java/org/apache/impala/planner/DataSink.java +++ b/fe/src/main/java/org/apache/impala/planner/DataSink.java @@ -34,7 +34,7 @@ public abstract class DataSink { // resource requirements and estimates for this plan node. // set in computeResourceProfile() - protected ResourceProfile resourceProfile_ = null; + protected ResourceProfile resourceProfile_ = ResourceProfile.invalid(); /** * Return an explain string for the DataSink. Each line of the explain will be prefixed diff --git a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java index 6dc89678c..879d9d8d5 100644 --- a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java +++ b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java @@ -330,9 +330,9 @@ public class DataSourceScanNode extends ScanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { // TODO: What's a good estimate of memory consumption? - resourceProfile_ = new ResourceProfile(1024L * 1024L * 1024L, 0); + nodeResourceProfile_ = new ResourceProfile(1024L * 1024L * 1024L, 0); } @Override diff --git a/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java b/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java index 174051d30..0d0acc9b1 100644 --- a/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java +++ b/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java @@ -60,9 +60,9 @@ public class EmptySetNode extends PlanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { // TODO: add an estimate - resourceProfile_ = new ResourceProfile(0, 0); + nodeResourceProfile_ = new ResourceProfile(0, 0); } @Override diff --git a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java index 3f4ea1e3a..478a05488 100644 --- a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java +++ b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java @@ -17,9 +17,6 @@ package org.apache.impala.planner; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import org.apache.impala.analysis.Analyzer; import org.apache.impala.analysis.Expr; import org.apache.impala.analysis.SortInfo; @@ -185,9 +182,16 @@ public class ExchangeNode extends PlanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { // TODO: add an estimate - resourceProfile_ = new ResourceProfile(0, 0); + nodeResourceProfile_ = new ResourceProfile(0, 0); + } + + @Override + public ExecPhaseResourceProfiles computeTreeResourceProfiles( + TQueryOptions queryOptions) { + // Don't include resources of child in different plan fragment. + return new ExecPhaseResourceProfiles(nodeResourceProfile_, nodeResourceProfile_); } @Override diff --git a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java index 48b772a1d..bbecbf1a7 100644 --- a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java +++ b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java @@ -24,8 +24,6 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.filter.CompareFilter; @@ -54,6 +52,7 @@ import org.apache.impala.thrift.TQueryOptions; import org.apache.impala.thrift.TScanRange; import org.apache.impala.thrift.TScanRangeLocation; import org.apache.impala.thrift.TScanRangeLocationList; +import org.apache.impala.util.MembershipSnapshot; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -217,8 +216,10 @@ public class HBaseScanNode extends ScanNode { LOG.trace("computeStats HbaseScan: cardinality=" + Long.toString(cardinality_)); } - // TODO: take actual regions into account - numNodes_ = Math.max(1, tbl.getNumNodes()); + // Assume that each node in the cluster gets a scan range, unless there are fewer + // scan ranges than nodes. + numNodes_ = Math.max(1, + Math.min(scanRanges_.size(), MembershipSnapshot.getCluster().numNodes())); if (LOG.isTraceEnabled()) { LOG.trace("computeStats HbaseScan: #nodes=" + Integer.toString(numNodes_)); } @@ -494,9 +495,9 @@ public class HBaseScanNode extends ScanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { // TODO: What's a good estimate of memory consumption? - resourceProfile_ = new ResourceProfile(1024L * 1024L * 1024L, 0); + nodeResourceProfile_ = new ResourceProfile(1024L * 1024L * 1024L, 0); } /** diff --git a/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java b/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java index 48492b15d..e8281258a 100644 --- a/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java +++ b/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java @@ -197,7 +197,7 @@ public class HashJoinNode extends JoinNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { long perInstanceMemEstimate; long perInstanceDataBytes; int numInstances = fragment_.getNumInstances(queryOptions.getMt_dop()); @@ -234,6 +234,6 @@ public class HashJoinNode extends JoinNode { } long perInstanceMinBufferBytes = bufferSize * minBuffers; - resourceProfile_ = new ResourceProfile(perInstanceMemEstimate, perInstanceMinBufferBytes); + nodeResourceProfile_ = new ResourceProfile(perInstanceMemEstimate, perInstanceMinBufferBytes); } } diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java index fa9038a67..0ba5bc65e 100644 --- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java +++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java @@ -1018,10 +1018,10 @@ public class HdfsScanNode extends ScanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { Preconditions.checkNotNull(scanRanges_, "Cost estimation requires scan ranges."); if (scanRanges_.isEmpty()) { - resourceProfile_ = new ResourceProfile(0, 0); + nodeResourceProfile_ = new ResourceProfile(0, 0); return; } Preconditions.checkState(0 < numNodes_ && numNodes_ <= scanRanges_.size()); @@ -1075,7 +1075,7 @@ public class HdfsScanNode extends ScanNode { PrintUtils.printBytes(perHostUpperBound))); perInstanceMemEstimate = perHostUpperBound; } - resourceProfile_ = new ResourceProfile(perInstanceMemEstimate, 0); + nodeResourceProfile_ = new ResourceProfile(perInstanceMemEstimate, 0); } /** diff --git a/fe/src/main/java/org/apache/impala/planner/JoinNode.java b/fe/src/main/java/org/apache/impala/planner/JoinNode.java index 47fa3e506..5e0beb48a 100644 --- a/fe/src/main/java/org/apache/impala/planner/JoinNode.java +++ b/fe/src/main/java/org/apache/impala/planner/JoinNode.java @@ -33,6 +33,8 @@ import org.apache.impala.catalog.Table; import org.apache.impala.common.ImpalaException; import org.apache.impala.common.Pair; import org.apache.impala.thrift.TJoinDistributionMode; +import org.apache.impala.service.BackendConfig; +import org.apache.impala.thrift.TQueryOptions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -630,4 +632,49 @@ public abstract class JoinNode extends PlanNode { eqJoinConjuncts_ = orderConjunctsByCost(eqJoinConjuncts_); otherJoinConjuncts_ = orderConjunctsByCost(otherJoinConjuncts_); } + + @Override + public ExecPhaseResourceProfiles computeTreeResourceProfiles( + TQueryOptions queryOptions) { + Preconditions.checkState(isBlockingJoinNode(), "Only blocking join nodes supported"); + + ExecPhaseResourceProfiles buildSideProfile = + getChild(1).computeTreeResourceProfiles(queryOptions); + ExecPhaseResourceProfiles probeSideProfile = + getChild(0).computeTreeResourceProfiles(queryOptions); + + // The peak resource consumption of the build phase is either during the Open() of + // the build side or while we're doing the join build and calling GetNext() on the + // build side. + ResourceProfile buildPhaseProfile = buildSideProfile.duringOpenProfile.max( + buildSideProfile.postOpenProfile.sum(nodeResourceProfile_)); + + ResourceProfile finishedBuildProfile = nodeResourceProfile_; + if (this instanceof NestedLoopJoinNode + || !BackendConfig.INSTANCE.isPartitionedHashJoinEnabled()) { + // These exec node implementations may hold references into the build side, which + // prevents closing of the build side in a timely manner. This means we have to + // count the post-open resource consumption of the build side in the same way as + // the other in-memory data structures. + // TODO: IMPALA-4179: remove this workaround + finishedBuildProfile = buildSideProfile.postOpenProfile.sum(nodeResourceProfile_); + } + + // Peak resource consumption of this subtree during Open(). + ResourceProfile duringOpenProfile; + if (queryOptions.getMt_dop() == 0) { + // The build and probe side can be open and therefore consume resources + // simultaneously when mt_dop = 0 because of the async build thread. + duringOpenProfile = buildPhaseProfile.sum(probeSideProfile.duringOpenProfile); + } else { + // Open() of the probe side happens after the build completes. + duringOpenProfile = buildPhaseProfile.max( + finishedBuildProfile.sum(probeSideProfile.duringOpenProfile)); + } + + // After Open(), the probe side remains open and the join build remain in memory. + ResourceProfile probePhaseProfile = + finishedBuildProfile.sum(probeSideProfile.postOpenProfile); + return new ExecPhaseResourceProfiles(duringOpenProfile, probePhaseProfile); + } } diff --git a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java index b1aa5ba00..46871297c 100644 --- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java +++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java @@ -268,8 +268,8 @@ public class KuduScanNode extends ScanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { - resourceProfile_ = new ResourceProfile(0, 0); + public void computeNodeResourceProfile(TQueryOptions queryOptions) { + nodeResourceProfile_ = new ResourceProfile(0, 0); } @Override diff --git a/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java b/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java index e69f97b47..0ec8e4f5e 100644 --- a/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java +++ b/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java @@ -77,7 +77,7 @@ public class NestedLoopJoinNode extends JoinNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { long perInstanceMemEstimate; if (getChild(1).getCardinality() == -1 || getChild(1).getAvgRowSize() == -1 || numNodes_ == 0) { @@ -86,7 +86,7 @@ public class NestedLoopJoinNode extends JoinNode { perInstanceMemEstimate = (long) Math.ceil(getChild(1).cardinality_ * getChild(1).avgRowSize_); } - resourceProfile_ = new ResourceProfile(perInstanceMemEstimate, 0); + nodeResourceProfile_ = new ResourceProfile(perInstanceMemEstimate, 0); } @Override diff --git a/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java b/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java index f5cc5d862..979302bdf 100644 --- a/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java +++ b/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java @@ -81,18 +81,14 @@ public class ParallelPlanner { List joins = Lists.newArrayList(); collectJoins(fragment.getPlanRoot(), joins); if (!joins.isEmpty()) { - List joinIds = Lists.newArrayList(); - for (JoinNode join: joins) joinIds.add(join.getId().toString()); - if (buildCohortId == null) buildCohortId = cohortIdGenerator_.getNextId(); for (JoinNode join: joins) createBuildPlan(join, buildCohortId); } - if (!fragment.getChildren().isEmpty()) { - List ids = Lists.newArrayList(); - for (PlanFragment c: fragment.getChildren()) ids.add(c.getId().toString()); - } for (PlanFragment child: fragment.getChildren()) { + // We already recursed on the join build fragment in createBuildPlan(). + if (child.getSink() instanceof JoinBuildSink) continue; + // Propagate the plan and cohort IDs to children that are part of the same plan. child.setPlanId(fragment.getPlanId()); child.setCohortId(fragment.getCohortId()); createBuildPlans(child, buildCohortId); @@ -171,19 +167,16 @@ public class ParallelPlanner { join.getChild(1), join.getFragment().getDataPartition()); buildFragment.setSink(buildSink); - // move input fragments + // Fix up the child/parent relationships in the PlanFragment tree. for (int i = 0; i < exchNodes.size(); ++i) { Preconditions.checkState(exchNodes.get(i).getFragment() == buildFragment); join.getFragment().removeChild(inputFragments.get(i)); buildFragment.getChildren().add(inputFragments.get(i)); } - - // compute the resource profile for the newly-added build sink. - buildSink.computeResourceProfile(ctx_.getQueryOptions()); + join.getFragment().addChild(buildFragment); // assign plan and cohort id buildFragment.setPlanId(planIdGenerator_.getNextId()); - PlanId parentPlanId = join.getFragment().getPlanId(); buildFragment.setCohortId(cohortId); planRoots_.add(buildFragment); diff --git a/fe/src/main/java/org/apache/impala/planner/PipelinedPlanNodeSet.java b/fe/src/main/java/org/apache/impala/planner/PipelinedPlanNodeSet.java deleted file mode 100644 index c2ae0fd1b..000000000 --- a/fe/src/main/java/org/apache/impala/planner/PipelinedPlanNodeSet.java +++ /dev/null @@ -1,204 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.impala.planner; - -import java.util.ArrayList; -import java.util.List; -import java.util.Set; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.impala.thrift.TQueryOptions; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - -/** - * Represents a set of PlanNodes and DataSinks that execute and consume resources - * concurrently. PlanNodes and DataSinks in such a pipelined plan node set may belong - * to different plan fragments because data is streamed across fragments. - * - * For example, a series of left-deep joins consists of two plan node sets. The first - * set contains all build-side nodes. The second set contains the leftmost - * scan. Both sets contain all join nodes because they execute and consume - * resources during the build and probe phases. Similarly, all nodes below a 'blocking' - * node (e.g, an AggregationNode) are placed into a different plan node set than the - * nodes above it, but the blocking node itself belongs to both sets. - */ -public class PipelinedPlanNodeSet { - private final static Logger LOG = LoggerFactory.getLogger(PipelinedPlanNodeSet.class); - - // Minimum per-host resource requirements to ensure that no plan node set can have - // estimates of zero, even if the contained PlanNodes have estimates of zero. - public static final long MIN_PER_HOST_MEM_ESTIMATE_BYTES = 10 * 1024 * 1024; - - // List of plan nodes that execute and consume resources concurrently. - private final ArrayList planNodes_ = Lists.newArrayList(); - - // DataSinks that execute and consume resources concurrently. - // Primarily used for estimating the cost of insert queries. - private final List dataSinks_ = Lists.newArrayList(); - - private void addNode(PlanNode node) { - Preconditions.checkNotNull(node.getFragment()); - planNodes_.add(node); - } - - private void addSink(DataSink sink) { - Preconditions.checkNotNull(sink); - dataSinks_.add(sink); - } - - /** - * Computes the per-host resource profile of this plan node set. - * - * If there are no nodes included in the estimate, the returned estimate will not be - * valid. - */ - public ResourceProfile computePerHostResources(TQueryOptions queryOptions) { - Set uniqueFragments = Sets.newHashSet(); - - // Distinguish the per-host memory estimates for scan nodes and non-scan nodes to - // get a tighter estimate on the amount of memory required by multiple concurrent - // scans. The memory required by all concurrent scans of the same type (Hdfs/Hbase) - // cannot exceed the per-host upper memory bound for that scan type. Intuitively, - // the amount of I/O buffers is limited by the disk bandwidth. - long hbaseScanMemEstimate = 0L; - long hdfsScanMemEstimate = 0L; - long nonScanMemEstimate = 0L; - long minReservationBytes = 0L; - int numNodesIncluded = 0; - - for (PlanNode node : planNodes_) { - PlanFragment fragment = node.getFragment(); - // Multiple instances of a partitioned fragment may execute per host - int instancesPerHost = fragment.getNumInstancesPerHost(queryOptions.getMt_dop()); - - ResourceProfile nodeProfile = node.getResourceProfile(); - Preconditions.checkState(nodeProfile.getMemEstimateBytes() >= 0); - long memEstimate = instancesPerHost * nodeProfile.getMemEstimateBytes(); - ++numNodesIncluded; - uniqueFragments.add(fragment); - if (node instanceof HBaseScanNode) { - hbaseScanMemEstimate += memEstimate; - } else if (node instanceof HdfsScanNode) { - hdfsScanMemEstimate += memEstimate; - } else { - nonScanMemEstimate += memEstimate; - } - Preconditions.checkState(nodeProfile.getMinReservationBytes() >= 0); - minReservationBytes += instancesPerHost * nodeProfile.getMinReservationBytes(); - } - - if (queryOptions.getMt_dop() == 0) { - // The thread tokens for the non-MT path impose a limit on the memory that can - // be consumed by concurrent scans. - hbaseScanMemEstimate = - Math.min(hbaseScanMemEstimate, HBaseScanNode.getPerHostMemUpperBound()); - hdfsScanMemEstimate = - Math.min(hdfsScanMemEstimate, HdfsScanNode.getPerHostMemUpperBound()); - } - - long dataSinkMemEstimate = 0L; - for (DataSink sink: dataSinks_) { - PlanFragment fragment = sink.getFragment(); - // Sanity check that this plan-node set has at least one PlanNode of fragment. - Preconditions.checkState(uniqueFragments.contains(fragment)); - int instancesPerHost = fragment.getNumInstancesPerHost(queryOptions.getMt_dop()); - - ResourceProfile sinkProfile = sink.getResourceProfile(); - Preconditions.checkState(sinkProfile.getMemEstimateBytes() >= 0); - dataSinkMemEstimate += instancesPerHost * sinkProfile.getMemEstimateBytes(); - Preconditions.checkState(sinkProfile.getMinReservationBytes() >= 0); - minReservationBytes += instancesPerHost * sinkProfile.getMinReservationBytes(); - } - - // Combine the memory estimates of all sinks, scans nodes and non-scan nodes. - long perHostMemEstimate = - Math.max(MIN_PER_HOST_MEM_ESTIMATE_BYTES, hdfsScanMemEstimate - + hbaseScanMemEstimate + nonScanMemEstimate + dataSinkMemEstimate); - // This plan node set might only have unpartitioned fragments and be invalid. - return numNodesIncluded > 0 ? - new ResourceProfile(perHostMemEstimate, minReservationBytes) : - ResourceProfile.invalid(); - } - - /** - * Computes and returns the pipelined plan node sets of the given plan. - */ - public static ArrayList computePlanNodeSets(PlanNode root) { - ArrayList planNodeSets = - Lists.newArrayList(new PipelinedPlanNodeSet()); - computePlanNodeSets(root, planNodeSets.get(0), null, planNodeSets); - return planNodeSets; - } - - /** - * Populates 'planNodeSets' by recursively traversing the plan tree rooted at 'node' - * The plan node sets are computed top-down. As a result, the plan node sets are added - * in reverse order of their runtime execution. - * - * Nodes are generally added to lhsSet. Joins are treated specially in that their - * left child is added to lhsSet and their right child to rhsSet to make sure - * that concurrent join builds end up in the same plan node set. - */ - private static void computePlanNodeSets(PlanNode node, PipelinedPlanNodeSet lhsSet, - PipelinedPlanNodeSet rhsSet, ArrayList planNodeSets) { - lhsSet.addNode(node); - if (node == node.getFragment().getPlanRoot() && node.getFragment().hasSink()) { - lhsSet.addSink(node.getFragment().getSink()); - } - - if (node instanceof JoinNode && ((JoinNode)node).isBlockingJoinNode()) { - // Create a new set for the right-hand sides of joins if necessary. - if (rhsSet == null) { - rhsSet = new PipelinedPlanNodeSet(); - planNodeSets.add(rhsSet); - } - // The join node itself is added to the lhsSet (above) and the rhsSet. - rhsSet.addNode(node); - computePlanNodeSets(node.getChild(1), rhsSet, null, planNodeSets); - computePlanNodeSets(node.getChild(0), lhsSet, rhsSet, planNodeSets); - return; - } - - if (node.isBlockingNode()) { - // We add blocking nodes to two plan node sets because they require resources while - // consuming their input (execution of the preceding set) and while they - // emit their output (execution of the following set). - // TODO: IMPALA-4862: this logic does not accurately reflect the behaviour of - // concurrent join builds in the backend - lhsSet = new PipelinedPlanNodeSet(); - lhsSet.addNode(node); - planNodeSets.add(lhsSet); - // Join builds under this blocking node belong in a new rhsSet. - rhsSet = null; - } - - // Assume that non-join, non-blocking nodes with multiple children - // (e.g., ExchangeNodes) consume their inputs in an arbitrary order, - // i.e., all child subtrees execute concurrently. - // TODO: IMPALA-4862: can overestimate resource consumption of UnionNodes - the - // execution of union branches is serialised within a fragment (but not across - // fragment boundaries). - for (PlanNode child: node.getChildren()) { - computePlanNodeSets(child, lhsSet, rhsSet, planNodeSets); - } - } -} diff --git a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java index 79c953a62..05bcf25c0 100644 --- a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java +++ b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java @@ -28,6 +28,7 @@ import org.apache.impala.common.InternalException; import org.apache.impala.common.NotImplementedException; import org.apache.impala.common.PrintUtils; import org.apache.impala.common.TreeNode; +import org.apache.impala.planner.PlanNode.ExecPhaseResourceProfiles; import org.apache.impala.thrift.TExplainLevel; import org.apache.impala.thrift.TPartitionType; import org.apache.impala.thrift.TPlanFragment; @@ -104,6 +105,11 @@ public class PlanFragment extends TreeNode { // if the output is UNPARTITIONED, it is being broadcast private DataPartition outputPartition_; + // Resource requirements and estimates for all instances of this plan fragment running + // on a host. Initialized with a dummy value. Gets set correctly in + // computeResourceProfile(). + private ResourceProfile perHostResourceProfile_ = ResourceProfile.invalid(); + /** * C'tor for fragment with specific partition; the output is by default broadcast. */ @@ -128,11 +134,12 @@ public class PlanFragment extends TreeNode { } /** - * Collect all PlanNodes that belong to the exec tree of this fragment. + * Collect and return all PlanNodes that belong to the exec tree of this fragment. */ - public void collectPlanNodes(List nodes) { - Preconditions.checkNotNull(nodes); + public List collectPlanNodes() { + List nodes = Lists.newArrayList(); collectPlanNodesHelper(planRoot_, nodes); + return nodes; } private void collectPlanNodesHelper(PlanNode root, List nodes) { @@ -148,14 +155,13 @@ public class PlanFragment extends TreeNode { public List getOutputExprs() { return outputExprs_; } /** - * Finalize plan tree and create stream sink, if needed. - * Computes resource profiles for all nodes and sinks in this fragment. + * Do any final work to set up the ExchangeNodes and DataStreamSinks for this fragment. * If this fragment is hash partitioned, ensures that the corresponding partition * exprs of all hash-partitioning senders are cast to identical types. * Otherwise, the hashes generated for identical partition values may differ * among senders if the partition-expr types are not identical. */ - public void finalize(Analyzer analyzer) + public void finalizeExchanges(Analyzer analyzer) throws InternalException, NotImplementedException { if (destNode_ != null) { Preconditions.checkState(sink_ == null); @@ -164,7 +170,6 @@ public class PlanFragment extends TreeNode { streamSink.setFragment(this); sink_ = streamSink; } - computeResourceProfile(analyzer); if (!dataPartition_.isHashPartitioned()) return; @@ -202,16 +207,38 @@ public class PlanFragment extends TreeNode { } /** - * Compute the resource profile of the fragment. Must be called after all the - * plan nodes and sinks are added to the fragment. + * Compute the peak resource profile for all instances of this fragment per host. Must + * be called after all the plan nodes and sinks are added to the fragment and resource + * profiles of all children fragments are computed. */ - private void computeResourceProfile(Analyzer analyzer) { + public void computeResourceProfile(Analyzer analyzer) { + // Compute resource profiles for all plan nodes and sinks in the fragment. sink_.computeResourceProfile(analyzer.getQueryOptions()); - List nodes = Lists.newArrayList(); - collectPlanNodes(nodes); - for (PlanNode node: nodes) { - node.computeResourceProfile(analyzer.getQueryOptions()); + for (PlanNode node: collectPlanNodes()) { + node.computeNodeResourceProfile(analyzer.getQueryOptions()); } + + if (sink_ instanceof JoinBuildSink) { + // Resource consumption of fragments with join build sinks is included in the + // parent fragment because the join node blocks waiting for the join build to + // finish - see JoinNode.computeTreeResourceProfiles(). + perHostResourceProfile_ = ResourceProfile.invalid(); + return; + } + + ExecPhaseResourceProfiles planTreeProfile = + planRoot_.computeTreeResourceProfiles(analyzer.getQueryOptions()); + // The sink is opened after the plan tree. + ResourceProfile fInstancePostOpenProfile = + planTreeProfile.postOpenProfile.sum(sink_.getResourceProfile()); + ResourceProfile fInstanceProfile = + planTreeProfile.duringOpenProfile.max(fInstancePostOpenProfile); + int numInstances = getNumInstancesPerHost(analyzer.getQueryOptions().getMt_dop()); + perHostResourceProfile_ = fInstanceProfile.multiply(numInstances); + } + + public ResourceProfile getPerHostResourceProfile() { + return perHostResourceProfile_; } /** @@ -313,17 +340,15 @@ public class PlanFragment extends TreeNode { prefix = " "; rootPrefix = " "; detailPrefix = prefix + "| "; - str.append(getFragmentHeaderString(queryOptions.getMt_dop())); - str.append("\n"); + str.append(getFragmentHeaderString("", "", queryOptions.getMt_dop())); if (sink_ != null && sink_ instanceof DataStreamSink) { str.append( sink_.getExplainString(rootPrefix, detailPrefix, queryOptions, detailLevel)); } } else if (detailLevel == TExplainLevel.EXTENDED) { // Print a fragment prefix displaying the # nodes and # instances - str.append(rootPrefix); - str.append(getFragmentHeaderString(queryOptions.getMt_dop())); - str.append("\n"); + str.append( + getFragmentHeaderString(rootPrefix, detailPrefix, queryOptions.getMt_dop())); rootPrefix = prefix; } @@ -348,12 +373,22 @@ public class PlanFragment extends TreeNode { /** * Get a header string for a fragment in an explain plan. */ - public String getFragmentHeaderString(int mt_dop) { + public String getFragmentHeaderString(String firstLinePrefix, String detailPrefix, + int mt_dop) { StringBuilder builder = new StringBuilder(); - builder.append(String.format("%s:PLAN FRAGMENT [%s]", fragmentId_.toString(), - dataPartition_.getExplainString())); + builder.append(String.format("%s%s:PLAN FRAGMENT [%s]", firstLinePrefix, + fragmentId_.toString(), dataPartition_.getExplainString())); builder.append(PrintUtils.printNumHosts(" ", getNumNodes())); builder.append(PrintUtils.printNumInstances(" ", getNumInstances(mt_dop))); + builder.append("\n"); + builder.append(detailPrefix); + builder.append("Per-Host Resources: "); + if (sink_ instanceof JoinBuildSink) { + builder.append("included in parent fragment"); + } else { + builder.append(perHostResourceProfile_.getExplainString()); + } + builder.append("\n"); return builder.toString(); } @@ -419,8 +454,7 @@ public class PlanFragment extends TreeNode { */ public void verifyTree() { // PlanNode.fragment_ is set correctly - List nodes = Lists.newArrayList(); - collectPlanNodes(nodes); + List nodes = collectPlanNodes(); List exchNodes = Lists.newArrayList(); for (PlanNode node: nodes) { if (node instanceof ExchangeNode) exchNodes.add(node); diff --git a/fe/src/main/java/org/apache/impala/planner/PlanNode.java b/fe/src/main/java/org/apache/impala/planner/PlanNode.java index 8448da59d..9723c4a60 100644 --- a/fe/src/main/java/org/apache/impala/planner/PlanNode.java +++ b/fe/src/main/java/org/apache/impala/planner/PlanNode.java @@ -115,7 +115,7 @@ abstract public class PlanNode extends TreeNode { // resource requirements and estimates for this plan node. // Initialized with a dummy value. Gets set correctly in // computeResourceProfile(). - protected ResourceProfile resourceProfile_ = ResourceProfile.invalid(); + protected ResourceProfile nodeResourceProfile_ = ResourceProfile.invalid(); // sum of tupleIds_' avgSerializedSizes; set in computeStats() protected float avgRowSize_; @@ -192,7 +192,7 @@ abstract public class PlanNode extends TreeNode { public boolean hasLimit() { return limit_ > -1; } public long getCardinality() { return cardinality_; } public int getNumNodes() { return numNodes_; } - public ResourceProfile getResourceProfile() { return resourceProfile_; } + public ResourceProfile getNodeResourceProfile() { return nodeResourceProfile_; } public float getAvgRowSize() { return avgRowSize_; } public void setFragment(PlanFragment fragment) { fragment_ = fragment; } public PlanFragment getFragment() { return fragment_; } @@ -306,7 +306,7 @@ abstract public class PlanNode extends TreeNode { if (detailLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) { // Print resource profile. expBuilder.append(detailPrefix); - expBuilder.append(resourceProfile_.getExplainString()); + expBuilder.append(nodeResourceProfile_.getExplainString()); expBuilder.append("\n"); // Print tuple ids, row size and cardinality. @@ -344,10 +344,8 @@ abstract public class PlanNode extends TreeNode { PlanFragment childFragment = children_.get(0).fragment_; if (fragment_ != childFragment && detailLevel == TExplainLevel.EXTENDED) { // we're crossing a fragment boundary - print the fragment header. - expBuilder.append(prefix); - expBuilder.append( - childFragment.getFragmentHeaderString(queryOptions.getMt_dop())); - expBuilder.append("\n"); + expBuilder.append(childFragment.getFragmentHeaderString(prefix, prefix, + queryOptions.getMt_dop())); } expBuilder.append( children_.get(0).getExplainString(prefix, prefix, queryOptions, detailLevel)); @@ -390,7 +388,7 @@ abstract public class PlanNode extends TreeNode { TExecStats estimatedStats = new TExecStats(); estimatedStats.setCardinality(cardinality_); - estimatedStats.setMemory_used(resourceProfile_.getMemEstimateBytes()); + estimatedStats.setMemory_used(nodeResourceProfile_.getMemEstimateBytes()); msg.setLabel(getDisplayLabel()); msg.setLabel_detail(getDisplayLabelDetail()); msg.setEstimated_stats(estimatedStats); @@ -616,15 +614,71 @@ abstract public class PlanNode extends TreeNode { public boolean isBlockingNode() { return false; } /** - * Compute resources consumed when executing this PlanNode, initializing - * 'resource_profile_'. May only be called after this PlanNode has been placed in a - * PlanFragment because the cost computation is dependent on the enclosing fragment's + * Compute peak resources consumed when executing this PlanNode, initializing + * 'nodeResourceProfile_'. May only be called after this PlanNode has been placed in + * a PlanFragment because the cost computation is dependent on the enclosing fragment's * data partition. */ - public abstract void computeResourceProfile(TQueryOptions queryOptions); + public abstract void computeNodeResourceProfile(TQueryOptions queryOptions); /** - * The default size of buffer used in spilling nodes. Used in computeResourceProfile(). + * Wrapper class to represent resource profiles during different phases of execution. + */ + public static class ExecPhaseResourceProfiles { + public ExecPhaseResourceProfiles( + ResourceProfile duringOpenProfile, ResourceProfile postOpenProfile) { + this.duringOpenProfile = duringOpenProfile; + this.postOpenProfile = postOpenProfile; + } + + /** Peak resources consumed while Open() is executing for this subtree */ + public final ResourceProfile duringOpenProfile; + + /** + * Peak resources consumed for this subtree from the time when ExecNode::Open() + * returns until the time when ExecNode::Close() returns. + */ + public final ResourceProfile postOpenProfile; + } + + /** + * Recursive function used to compute the peak resources consumed by this subtree of + * the plan within a fragment instance. The default implementation of this function + * is correct for streaming and blocking PlanNodes with a single child. PlanNodes + * that don't meet this description must override this function. + * + * Not called for PlanNodes inside a subplan: the root SubplanNode is responsible for + * computing the peak resources for the entire subplan. + * + * computeNodeResourceProfile() must be called on all plan nodes in this subtree before + * calling this function. + */ + public ExecPhaseResourceProfiles computeTreeResourceProfiles( + TQueryOptions queryOptions) { + Preconditions.checkState( + children_.size() <= 1, "Plan nodes with > 1 child must override"); + if (children_.isEmpty()) { + return new ExecPhaseResourceProfiles(nodeResourceProfile_, nodeResourceProfile_); + } + ExecPhaseResourceProfiles childResources = + getChild(0).computeTreeResourceProfiles(queryOptions); + if (isBlockingNode()) { + // This does not consume resources until after child's Open() returns. The child is + // then closed before Open() of this node returns. + ResourceProfile duringOpenProfile = childResources.duringOpenProfile.max( + childResources.postOpenProfile.sum(nodeResourceProfile_)); + return new ExecPhaseResourceProfiles(duringOpenProfile, nodeResourceProfile_); + } else { + // Streaming node: this node, child and ancestor execute concurrently. + return new ExecPhaseResourceProfiles( + childResources.duringOpenProfile.sum(nodeResourceProfile_), + childResources.postOpenProfile.sum(nodeResourceProfile_)); + } + } + + /** + * The default size of buffer used in spilling nodes. Used in + * computeNodeResourceProfile(). */ protected final static long getDefaultSpillableBufferBytes() { // BufferedBlockMgr uses --read_size to determine buffer size. diff --git a/fe/src/main/java/org/apache/impala/planner/Planner.java b/fe/src/main/java/org/apache/impala/planner/Planner.java index 936847a1c..c65c66812 100644 --- a/fe/src/main/java/org/apache/impala/planner/Planner.java +++ b/fe/src/main/java/org/apache/impala/planner/Planner.java @@ -58,6 +58,13 @@ import com.google.common.collect.Lists; public class Planner { private final static Logger LOG = LoggerFactory.getLogger(Planner.class); + // Minimum per-host resource requirements to ensure that no plan node set can have + // estimates of zero, even if the contained PlanNodes have estimates of zero. + public static final long MIN_PER_HOST_MEM_ESTIMATE_BYTES = 10 * 1024 * 1024; + + public static final ResourceProfile MIN_PER_HOST_RESOURCES = + new ResourceProfile(MIN_PER_HOST_MEM_ESTIMATE_BYTES, 0); + private final PlannerContext ctx_; public Planner(AnalysisContext.AnalysisResult analysisResult, TQueryCtx queryCtx) { @@ -159,7 +166,7 @@ public class Planner { LOG.trace("finalize plan fragments"); } for (PlanFragment fragment: fragments) { - fragment.finalize(ctx_.getRootAnalyzer()); + fragment.finalizeExchanges(ctx_.getRootAnalyzer()); } Collections.reverse(fragments); @@ -332,49 +339,67 @@ public class Planner { } /** - * Estimates the per-host resource requirements for the given plans, and sets the - * results in request. - * TODO: The LOG.warn() messages should eventually become Preconditions checks - * once resource estimation is more robust. + * Computes the per-host resource profile for the given plans, i.e. the peak resources + * consumed by all fragment instances belonging to the query per host. Sets the + * per-host resource values in 'request'. */ public void computeResourceReqs(List planRoots, TQueryExecRequest request) { Preconditions.checkState(!planRoots.isEmpty()); Preconditions.checkNotNull(request); + TQueryOptions queryOptions = ctx_.getRootAnalyzer().getQueryOptions(); + int mtDop = queryOptions.getMt_dop(); - // Compute the sum over all plans. - // TODO: Revisit during MT work - scheduling of fragments will change and computing - // the sum may not be correct or optimal. - ResourceProfile totalResources = ResourceProfile.invalid(); - for (PlanFragment planRoot: planRoots) { - ResourceProfile planMaxResources = ResourceProfile.invalid(); - ArrayList fragments = planRoot.getNodesPreOrder(); - // Compute pipelined plan node sets. - ArrayList planNodeSets = - PipelinedPlanNodeSet.computePlanNodeSets(fragments.get(0).getPlanRoot()); + // Peak per-host peak resources for all plan fragments. + ResourceProfile perHostPeakResources = ResourceProfile.invalid(); + // Total of initial reservation claims in bytes by all operators in all fragment + // instances per host. Computed by summing the per-host minimum reservations of + // all plan nodes and sinks. + long perHostInitialReservationTotal = 0; - // Compute the max of the per-host resources requirement. - // Note that the different maxes may come from different plan node sets. - for (PipelinedPlanNodeSet planNodeSet : planNodeSets) { - TQueryOptions queryOptions = ctx_.getQueryOptions(); - ResourceProfile perHostResources = - planNodeSet.computePerHostResources(queryOptions); - if (!perHostResources.isValid()) continue; - planMaxResources = ResourceProfile.max(planMaxResources, perHostResources); + // Do a pass over all the fragments to compute resource profiles. Compute the + // profiles bottom-up since a fragment's profile may depend on its descendants. + List allFragments = planRoots.get(0).getNodesPostOrder(); + for (PlanFragment fragment: allFragments) { + // Compute the per-node, per-sink and aggregate profiles for the fragment. + fragment.computeResourceProfile(ctx_.getRootAnalyzer()); + + // Different fragments do not synchronize their Open() and Close(), so the backend + // does not provide strong guarantees about whether one fragment instance releases + // resources before another acquires them. Conservatively assume that all fragment + // instances can consume their peak resources at the same time, i.e. that the + // query-wide peak resources is the sum of the per-fragment-instance peak + // resources. + perHostPeakResources = + perHostPeakResources.sum(fragment.getPerHostResourceProfile()); + perHostInitialReservationTotal += fragment.getNumInstancesPerHost(mtDop) + * fragment.getSink().getResourceProfile().getMinReservationBytes(); + + for (PlanNode node: fragment.collectPlanNodes()) { + perHostInitialReservationTotal += fragment.getNumInstances(mtDop) + * node.getNodeResourceProfile().getMinReservationBytes(); } - totalResources = ResourceProfile.sum(totalResources, planMaxResources); } - Preconditions.checkState(totalResources.getMemEstimateBytes() >= 0); - Preconditions.checkState(totalResources.getMinReservationBytes() >= 0); - request.setPer_host_mem_estimate(totalResources.getMemEstimateBytes()); - request.setPer_host_min_reservation(totalResources.getMinReservationBytes()); + Preconditions.checkState(perHostPeakResources.getMemEstimateBytes() >= 0, + perHostPeakResources.getMemEstimateBytes()); + Preconditions.checkState(perHostPeakResources.getMinReservationBytes() >= 0, + perHostPeakResources.getMinReservationBytes()); + + perHostPeakResources = MIN_PER_HOST_RESOURCES.max(perHostPeakResources); + + request.setPer_host_mem_estimate(perHostPeakResources.getMemEstimateBytes()); + request.setPer_host_min_reservation(perHostPeakResources.getMinReservationBytes()); + request.setPer_host_initial_reservation_total_claims(perHostInitialReservationTotal); if (LOG.isTraceEnabled()) { - LOG.trace("Per-host min buffer : " + totalResources.getMinReservationBytes()); - LOG.trace("Estimated per-host memory: " + totalResources.getMemEstimateBytes()); + LOG.trace("Per-host min buffer : " + perHostPeakResources.getMinReservationBytes()); + LOG.trace( + "Estimated per-host memory: " + perHostPeakResources.getMemEstimateBytes()); + LOG.trace("Per-host initial reservation total: " + perHostInitialReservationTotal); } } + /** * Traverses the plan tree rooted at 'root' and inverts outer and semi joins * in the following situations: diff --git a/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java b/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java index c0dc60773..18cde7e06 100644 --- a/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java +++ b/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java @@ -18,9 +18,11 @@ package org.apache.impala.planner; import org.apache.impala.common.PrintUtils; +import org.apache.impala.util.MathUtil; /** - * The resources that will be consumed by a set of plan nodes. + * The resources that will be consumed by some part of a plan, e.g. a plan node or + * plan fragment. */ public class ResourceProfile { // If the computed values are valid. @@ -64,20 +66,29 @@ public class ResourceProfile { return output.toString(); } - // Returns a profile with the max of each value in 'p1' and 'p2'. - public static ResourceProfile max(ResourceProfile p1, ResourceProfile p2) { - if (!p1.isValid()) return p2; - if (!p2.isValid()) return p1; + // Returns a profile with the max of each value in 'this' and 'other'. + public ResourceProfile max(ResourceProfile other) { + if (!isValid()) return other; + if (!other.isValid()) return this; return new ResourceProfile( - Math.max(p1.getMemEstimateBytes(), p2.getMemEstimateBytes()), - Math.max(p1.getMinReservationBytes(), p2.getMinReservationBytes())); + Math.max(getMemEstimateBytes(), other.getMemEstimateBytes()), + Math.max(getMinReservationBytes(), other.getMinReservationBytes())); } - // Returns a profile with the sum of each value in 'p1' and 'p2'. - public static ResourceProfile sum(ResourceProfile p1, ResourceProfile p2) { - if (!p1.isValid()) return p2; - if (!p2.isValid()) return p1; - return new ResourceProfile(p1.getMemEstimateBytes() + p2.getMemEstimateBytes(), - p1.getMinReservationBytes() + p2.getMinReservationBytes()); + // Returns a profile with the sum of each value in 'this' and 'other'. + public ResourceProfile sum(ResourceProfile other) { + if (!isValid()) return other; + if (!other.isValid()) return this; + return new ResourceProfile( + MathUtil.saturatingAdd(getMemEstimateBytes(), other.getMemEstimateBytes()), + MathUtil.saturatingAdd(getMinReservationBytes(), other.getMinReservationBytes())); } -} \ No newline at end of file + + // Returns a profile with all values multiplied by 'factor'. + public ResourceProfile multiply(int factor) { + if (!isValid()) return this; + return new ResourceProfile( + MathUtil.saturatingMultiply(memEstimateBytes_, factor), + MathUtil.saturatingMultiply(minReservationBytes_, factor)); + } +} diff --git a/fe/src/main/java/org/apache/impala/planner/SelectNode.java b/fe/src/main/java/org/apache/impala/planner/SelectNode.java index c346df9b3..97dfa5bd1 100644 --- a/fe/src/main/java/org/apache/impala/planner/SelectNode.java +++ b/fe/src/main/java/org/apache/impala/planner/SelectNode.java @@ -82,9 +82,9 @@ public class SelectNode extends PlanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { // TODO: add an estimate - resourceProfile_ = new ResourceProfile(0, 0); + nodeResourceProfile_ = new ResourceProfile(0, 0); } @Override diff --git a/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java b/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java index 82a1c412f..bed1c9a81 100644 --- a/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java +++ b/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java @@ -66,9 +66,9 @@ public class SingularRowSrcNode extends PlanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { // TODO: add an estimate - resourceProfile_ = new ResourceProfile(0, 0); + nodeResourceProfile_ = new ResourceProfile(0, 0); } @Override diff --git a/fe/src/main/java/org/apache/impala/planner/SortNode.java b/fe/src/main/java/org/apache/impala/planner/SortNode.java index 3517bee5d..f6288855f 100644 --- a/fe/src/main/java/org/apache/impala/planner/SortNode.java +++ b/fe/src/main/java/org/apache/impala/planner/SortNode.java @@ -216,12 +216,12 @@ public class SortNode extends PlanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { Preconditions.checkState(hasValidStats()); if (useTopN_) { long perInstanceMemEstimate = (long) Math.ceil((cardinality_ + offset_) * avgRowSize_); - resourceProfile_ = new ResourceProfile(perInstanceMemEstimate, 0); + nodeResourceProfile_ = new ResourceProfile(perInstanceMemEstimate, 0); return; } @@ -253,7 +253,7 @@ public class SortNode extends PlanNode { if (info_.getSortTupleDescriptor().hasVarLenSlots()) { perInstanceMinReservation *= 2; } - resourceProfile_ = + nodeResourceProfile_ = new ResourceProfile(perInstanceMemEstimate, perInstanceMinReservation); } diff --git a/fe/src/main/java/org/apache/impala/planner/SubplanNode.java b/fe/src/main/java/org/apache/impala/planner/SubplanNode.java index 169abc107..c09efe5d4 100644 --- a/fe/src/main/java/org/apache/impala/planner/SubplanNode.java +++ b/fe/src/main/java/org/apache/impala/planner/SubplanNode.java @@ -93,9 +93,26 @@ public class SubplanNode extends PlanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { // TODO: add an estimate - resourceProfile_ = new ResourceProfile(0, 0); + nodeResourceProfile_ = new ResourceProfile(0, 0); + } + + @Override + public ExecPhaseResourceProfiles computeTreeResourceProfiles( + TQueryOptions queryOptions) { + // All nodes in a subplan remain open at the same time across iterations of a subplan, + // therefore the peak resource consumption is simply the sum of all node resources. + ResourceProfile subplanProfile = subplanComputePeakResources(this); + return new ExecPhaseResourceProfiles(subplanProfile, subplanProfile); + } + + private static ResourceProfile subplanComputePeakResources(PlanNode node) { + ResourceProfile result = node.nodeResourceProfile_; + for (PlanNode child: node.getChildren()) { + result = result.sum(subplanComputePeakResources(child)); + } + return result; } @Override diff --git a/fe/src/main/java/org/apache/impala/planner/UnionNode.java b/fe/src/main/java/org/apache/impala/planner/UnionNode.java index 2c72c633a..44e296792 100644 --- a/fe/src/main/java/org/apache/impala/planner/UnionNode.java +++ b/fe/src/main/java/org/apache/impala/planner/UnionNode.java @@ -129,9 +129,26 @@ public class UnionNode extends PlanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { // TODO: add an estimate - resourceProfile_ = new ResourceProfile(0, 0); + nodeResourceProfile_ = new ResourceProfile(0, 0); + } + + @Override + public ExecPhaseResourceProfiles computeTreeResourceProfiles( + TQueryOptions queryOptions) { + // The union executes concurrently with Open() and GetNext() on each of it's + // children. + ResourceProfile maxProfile = ResourceProfile.invalid(); + for (PlanNode child : children_) { + // Children are opened either during Open() or GetNext() of the union. + ExecPhaseResourceProfiles childResources = + child.computeTreeResourceProfiles(queryOptions); + maxProfile = maxProfile.max(childResources.duringOpenProfile); + maxProfile = maxProfile.max(childResources.postOpenProfile); + } + ResourceProfile peakResources = nodeResourceProfile_.sum(maxProfile); + return new ExecPhaseResourceProfiles(peakResources, peakResources); } /** diff --git a/fe/src/main/java/org/apache/impala/planner/UnnestNode.java b/fe/src/main/java/org/apache/impala/planner/UnnestNode.java index 5847e6226..695ec240c 100644 --- a/fe/src/main/java/org/apache/impala/planner/UnnestNode.java +++ b/fe/src/main/java/org/apache/impala/planner/UnnestNode.java @@ -74,9 +74,9 @@ public class UnnestNode extends PlanNode { } @Override - public void computeResourceProfile(TQueryOptions queryOptions) { + public void computeNodeResourceProfile(TQueryOptions queryOptions) { // TODO: add an estimate - resourceProfile_ = new ResourceProfile(0, 0); + nodeResourceProfile_ = new ResourceProfile(0, 0); } @Override diff --git a/fe/src/main/java/org/apache/impala/service/BackendConfig.java b/fe/src/main/java/org/apache/impala/service/BackendConfig.java index 80f8aceab..a1566c753 100644 --- a/fe/src/main/java/org/apache/impala/service/BackendConfig.java +++ b/fe/src/main/java/org/apache/impala/service/BackendConfig.java @@ -61,6 +61,10 @@ public class BackendConfig { public int getImpalaLogLevel() { return backendCfg_.impala_log_lvl; } public int getNonImpalaJavaVlogLevel() { return backendCfg_.non_impala_java_vlog; } + public boolean isPartitionedHashJoinEnabled() { + return backendCfg_.enable_partitioned_hash_join; + } + // Inits the auth_to_local configuration in the static KerberosName class. private static void initAuthToLocal() { // If auth_to_local is enabled, we read the configuration hadoop.security.auth_to_local diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java index d0a3b4dc7..855a07670 100644 --- a/fe/src/main/java/org/apache/impala/service/Frontend.java +++ b/fe/src/main/java/org/apache/impala/service/Frontend.java @@ -35,10 +35,6 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hive.service.rpc.thrift.TGetColumnsReq; -import org.apache.hive.service.rpc.thrift.TGetFunctionsReq; -import org.apache.hive.service.rpc.thrift.TGetSchemasReq; -import org.apache.hive.service.rpc.thrift.TGetTablesReq; import org.apache.impala.analysis.AnalysisContext; import org.apache.impala.analysis.CreateDataSrcStmt; import org.apache.impala.analysis.CreateDropRoleStmt; @@ -1008,6 +1004,9 @@ public class Frontend { planRoots.add(planner.createPlan().get(0)); } + // Compute resource requirements of the final plans. + planner.computeResourceReqs(planRoots, result); + // create per-plan exec info; // also assemble list of names of tables with missing or corrupt stats for // assembling a warning message @@ -1016,10 +1015,6 @@ public class Frontend { createPlanExecInfo(planRoot, planner, queryCtx, result)); } - // Compute resource requirements after scan range locations because the cost - // estimates of scan nodes rely on them. - planner.computeResourceReqs(planRoots, result); - // Optionally disable spilling in the backend. Allow spilling if there are plan hints // or if all tables have stats. boolean disableSpilling = diff --git a/fe/src/main/java/org/apache/impala/util/BitUtil.java b/fe/src/main/java/org/apache/impala/util/BitUtil.java index 51b778a4c..839dd6eaf 100644 --- a/fe/src/main/java/org/apache/impala/util/BitUtil.java +++ b/fe/src/main/java/org/apache/impala/util/BitUtil.java @@ -26,7 +26,7 @@ public class BitUtil { } // Round up 'val' to the nearest power of two. 'val' must be > 0. - public static int roundUpToPowerOf2(long val) { - return 1 << log2Ceiling(val); + public static long roundUpToPowerOf2(long val) { + return 1L << log2Ceiling(val); } } diff --git a/fe/src/main/java/org/apache/impala/util/MathUtil.java b/fe/src/main/java/org/apache/impala/util/MathUtil.java new file mode 100644 index 000000000..c4029fa59 --- /dev/null +++ b/fe/src/main/java/org/apache/impala/util/MathUtil.java @@ -0,0 +1,45 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.impala.util; + +import com.google.common.math.LongMath; + +public class MathUtil { + + // Multiply two numbers. If the multiply would overflow, return either Long.MIN_VALUE + // (if a xor b is negative) or Long.MAX_VALUE otherwise. The overflow path is not + // optimised at all and may be somewhat slow. + public static long saturatingMultiply(long a, long b) { + try { + return LongMath.checkedMultiply(a, b); + } catch (ArithmeticException e) { + return a < 0 != b < 0 ? Long.MIN_VALUE : Long.MAX_VALUE; + } + } + + // Add two numbers. If the add would overflow, return either Long.MAX_VALUE if both are + // positive or Long.MIN_VALUE if both are negative. The overflow path is not optimised + // at all and may be somewhat slow. + public static long saturatingAdd(long a, long b) { + try { + return LongMath.checkedAdd(a, b); + } catch (ArithmeticException e) { + return a < 0 ? Long.MIN_VALUE : Long.MAX_VALUE; + } + } +} diff --git a/fe/src/test/java/org/apache/impala/util/BitUtilTest.java b/fe/src/test/java/org/apache/impala/util/BitUtilTest.java new file mode 100644 index 000000000..a134b6a57 --- /dev/null +++ b/fe/src/test/java/org/apache/impala/util/BitUtilTest.java @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.impala.util; + +import static org.junit.Assert.*; + +import org.junit.Test; + +/** + * Unit tests for BitUtil functions. + */ +public class BitUtilTest { + + // Test the log2/PowerOf2 functions + @Test + public void testPowersOf2() { + assertEquals(8, BitUtil.log2Ceiling(256)); // Power-of-two + assertEquals(256, BitUtil.roundUpToPowerOf2(256)); // Power-of-two + assertEquals(4, BitUtil.log2Ceiling(10)); // Rounds up + assertEquals(16, BitUtil.roundUpToPowerOf2(10)); // Rounds up + assertEquals(33, BitUtil.log2Ceiling(8L * 1000L * 1000L * 1000L)); // > 32bit number + assertEquals(8L * 1024L * 1024L * 1024L, + BitUtil.roundUpToPowerOf2(8L * 1000L * 1000L * 1000L)); // > 32bit number + + assertEquals(0, BitUtil.log2Ceiling(1)); // Minimum valid input + assertEquals(1, BitUtil.roundUpToPowerOf2(1)); // Minimum valid input + assertEquals(63, BitUtil.log2Ceiling(Long.MAX_VALUE)); // Maximum valid input + // Overflow to -2^62. + assertEquals(-0x8000000000000000L, BitUtil.roundUpToPowerOf2(Long.MAX_VALUE)); + // Maximum non-overflow output: 2^62. + assertEquals(0x8000000000000000L, BitUtil.roundUpToPowerOf2(0x8000000000000000L - 1)); + } + +} diff --git a/fe/src/test/java/org/apache/impala/util/MathUtilTest.java b/fe/src/test/java/org/apache/impala/util/MathUtilTest.java new file mode 100644 index 000000000..612006793 --- /dev/null +++ b/fe/src/test/java/org/apache/impala/util/MathUtilTest.java @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.impala.util; + +import static org.junit.Assert.*; + +import org.junit.Test; + +/** + * Unit tests for MathUtil functions. + */ +public class MathUtilTest { + + @Test + public void testSaturatingMultiply() { + // Positive * positive + assertEquals(10, MathUtil.saturatingMultiply(2, 5)); + assertEquals(Long.MAX_VALUE, MathUtil.saturatingMultiply(2, Long.MAX_VALUE)); + assertEquals(Long.MAX_VALUE, MathUtil.saturatingMultiply(3, Long.MAX_VALUE / 2)); + + // Positive * negative + assertEquals(-10, MathUtil.saturatingMultiply(2, -5)); + assertEquals(Long.MIN_VALUE, MathUtil.saturatingMultiply(2, Long.MIN_VALUE)); + assertEquals(Long.MIN_VALUE, MathUtil.saturatingMultiply(-3, Long.MAX_VALUE / 2)); + + // Negative * negative + assertEquals(10, MathUtil.saturatingMultiply(-2, -5)); + assertEquals(Long.MAX_VALUE, MathUtil.saturatingMultiply(-1, Long.MIN_VALUE)); + assertEquals(Long.MAX_VALUE, MathUtil.saturatingMultiply(Long.MIN_VALUE / 10, -100)); + } + + @Test + public void testSaturatingAdd() { + // No overflow + assertEquals(1234, MathUtil.saturatingAdd(1200, 34)); + assertEquals(-1, MathUtil.saturatingAdd(Long.MAX_VALUE, Long.MIN_VALUE)); + + // Underflow + assertEquals(Long.MIN_VALUE, MathUtil.saturatingAdd(Long.MIN_VALUE, -1)); + assertEquals(Long.MIN_VALUE, MathUtil.saturatingAdd(Long.MIN_VALUE, Long.MIN_VALUE / 2)); + + // Overflow + assertEquals(Long.MAX_VALUE, MathUtil.saturatingAdd(Long.MAX_VALUE - 10, 11)); + assertEquals(Long.MAX_VALUE, MathUtil.saturatingAdd(Long.MAX_VALUE, Long.MAX_VALUE / 2)); + } +} diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test b/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test index 82f5c3ecc..ed4f684fc 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test @@ -5,6 +5,7 @@ where 5 + 5 < c_custkey and o_orderkey = (2 + 2) and (coalesce(2, 3, 4) * 10) + l_linenumber < (0 * 1) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -45,12 +46,12 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=0B cardinality=10 | 00:SCAN HDFS [tpch_nested_parquet.customer c] - partitions=1/1 files=4 size=292.36MB + partitions=1/1 files=4 size=292.35MB predicates: c_custkey > 10, !empty(c.c_orders) predicates on o: !empty(o.o_lineitems), o_orderkey = 4 predicates on o_lineitems: 20 + l_linenumber < 0 stats-rows=150000 extrapolated-rows=disabled - table stats: rows=150000 size=292.36MB + table stats: rows=150000 size=292.35MB columns missing stats: c_orders parquet statistics predicates: c_custkey > 10 parquet dictionary predicates: c_custkey > 10 @@ -63,6 +64,7 @@ where string_col = cast(4 as string) and 2 + 3 = tinyint_col and id between concat('1', '0') and upper('20') ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -81,6 +83,7 @@ select * from functional.alltypes_datasource where tinyint_col < (pow(2, 8)) and float_col != 0 and 1 + 1 > int_col ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -99,6 +102,7 @@ having 1024 * 1024 * count(*) % 2 = 0 and (sm between 5 and 10) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=138.00MB mem-reservation=264.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -125,6 +129,7 @@ left outer join functional.alltypes b where round(1.11 + 2.22 + 3.33 + 4.44, 1) < cast(b.double_col as decimal(3, 2)) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=256.02MB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -163,6 +168,7 @@ left outer join functional.alltypes b where cast(b.double_col as decimal(3, 2)) > round(1.11 + 2.22 + 3.33 + 4.44, 1) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=256.01MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -197,6 +203,7 @@ group by timestamp_col = cast('2015-11-15' as timestamp) + interval 1 year having 1024 * 1024 * count(*) % 2 = 0 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=138.00MB mem-reservation=528.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -227,6 +234,7 @@ from functional.alltypes having 1024 * 1024 * count(*) % 2 = 0 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=138.00MB mem-reservation=264.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -257,6 +265,7 @@ select first_value(1 + 1 + int_col - (1 - 1)) over from functional.alltypes ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=144.00MB mem-reservation=64.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -287,6 +296,7 @@ select int_col from functional.alltypes order by id * abs((factorial(5) / power(2, 4))) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=136.00MB mem-reservation=24.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -309,6 +319,7 @@ select id, int_col, cast(1 + 1 + 1 + year as int), cast(month - (1 - 1 - 1) as i from functional.alltypessmall ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(CAST(3 + year AS INT),CAST(month - -1 AS INT))] | partitions=4 | mem-estimate=1.56KB mem-reservation=0B @@ -330,6 +341,7 @@ select sum(id + c3) from ) v3 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=138.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test b/testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test index a707e44ac..e64691af5 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test @@ -2,7 +2,7 @@ select count(*) from functional.alltypes ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B -Per-Host Resource Estimates: Memory=138.00MB +Per-Host Resource Estimates: Memory=148.00MB Codegen disabled by planner PLAN-ROOT SINK @@ -22,7 +22,7 @@ PLAN-ROOT SINK select count(*) from functional.alltypesagg ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B -Per-Host Resource Estimates: Memory=90.00MB +Per-Host Resource Estimates: Memory=100.00MB PLAN-ROOT SINK | @@ -41,7 +41,7 @@ PLAN-ROOT SINK select count(*) from functional_parquet.alltypes ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B -Per-Host Resource Estimates: Memory=26.00MB +Per-Host Resource Estimates: Memory=36.00MB WARNING: The following tables are missing relevant table and/or column statistics. functional_parquet.alltypes @@ -56,7 +56,7 @@ PLAN-ROOT SINK | output: sum_init_zero(functional_parquet.alltypes.parquet-stats: num_rows) | 00:SCAN HDFS [functional_parquet.alltypes] - partitions=24/24 files=24 size=179.68KB + partitions=24/24 files=24 size=178.13KB ==== # > 3000 rows returned to coordinator: codegen should be enabled select * from functional_parquet.alltypes @@ -71,7 +71,7 @@ PLAN-ROOT SINK 01:EXCHANGE [UNPARTITIONED] | 00:SCAN HDFS [functional_parquet.alltypes] - partitions=24/24 files=24 size=179.68KB + partitions=24/24 files=24 size=178.13KB ==== # Optimisation is enabled for join producing < 3000 rows select count(*) @@ -79,7 +79,7 @@ from functional.alltypes t1 join functional.alltypestiny t2 on t1.id = t2.id ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=136.00MB -Per-Host Resource Estimates: Memory=138.00MB +Per-Host Resource Estimates: Memory=180.00MB Codegen disabled by planner PLAN-ROOT SINK @@ -109,7 +109,7 @@ PLAN-ROOT SINK select count(*) from functional.alltypes t1, functional.alltypes t2 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B -Per-Host Resource Estimates: Memory=138.00MB +Per-Host Resource Estimates: Memory=276.00MB PLAN-ROOT SINK | @@ -138,7 +138,7 @@ select count(*) from ( select * from functional.alltypestiny) v ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B -Per-Host Resource Estimates: Memory=170.00MB +Per-Host Resource Estimates: Memory=148.00MB Codegen disabled by planner PLAN-ROOT SINK @@ -167,7 +167,7 @@ select count(*) from ( select * from functional.alltypes) v ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B -Per-Host Resource Estimates: Memory=266.00MB +Per-Host Resource Estimates: Memory=148.00MB PLAN-ROOT SINK | diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test index c2065ed85..8e8ddc0da 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test @@ -5,6 +5,7 @@ on ss_customer_sk = c_customer_sk where c_salutation = 'Mrs.' ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=180.46MB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -42,6 +43,7 @@ on ss_customer_sk = c_customer_sk where c_salutation = 'Mrs.' ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=180.46MB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -78,6 +80,7 @@ on ss_customer_sk = c_customer_sk where c_salutation = 'Mrs.' ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=180.46MB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -114,6 +117,7 @@ on ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number where sr_return_quantity < 10 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=210.65MB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -149,6 +153,7 @@ tpcds.store_sales inner join tpcds.web_sales on ss_sold_time_sk = ws_sold_time_sk ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=396.67MB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -183,6 +188,7 @@ on a.d_date_sk = b.d_date_sk where a.d_holiday = "Y" ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=107.62MB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -223,6 +229,7 @@ where ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number and d1.d_fy_week_seq = 1000 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=352.73MB mem-reservation=544.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -304,6 +311,7 @@ tpcds.store_sales inner join tpcds.customer on ss_customer_sk % 10 = c_customer_sk / 100 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=202.79MB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -338,6 +346,7 @@ tpcds.store_sales inner join tpcds_seq_snap.customer on ss_customer_sk = c_customer_sk ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=2.17GB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -349,7 +358,7 @@ PLAN-ROOT SINK | tuple-ids=0,1 row-size=8B cardinality=2880404 | |--01:SCAN HDFS [tpcds_seq_snap.customer] -| partitions=1/1 files=1 size=8.59MB +| partitions=1/1 files=1 size=8.58MB | stats-rows=unavailable extrapolated-rows=disabled | table stats: rows=unavailable size=unavailable | column stats: unavailable @@ -371,6 +380,7 @@ tpcds_seq_snap.store_sales inner join tpcds.customer on ss_customer_sk = c_customer_sk ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=176.42MB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -390,7 +400,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=4B cardinality=100000 | 00:SCAN HDFS [tpcds_seq_snap.store_sales] - partitions=1824/1824 files=1824 size=207.90MB + partitions=1824/1824 files=1824 size=207.85MB runtime filters: RF000 -> ss_customer_sk stats-rows=unavailable extrapolated-rows=disabled table stats: rows=unavailable size=unavailable @@ -406,6 +416,7 @@ tpcds.store_sales inner join on ss_sold_time_sk = ws_sold_time_sk ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=298.00MB mem-reservation=400.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test index 48e98af06..02843c001 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test @@ -1,6 +1,7 @@ select * from functional_kudu.zipcode_incomes where id = '8600000US00601' ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -10,6 +11,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 tuple-ids=0 row-size=124B cardinality=1 ---- DISTRIBUTEDPLAN F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -18,6 +20,7 @@ F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 tuple-ids=0 row-size=124B cardinality=1 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=0B mem-reservation=0B DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED] | mem-estimate=0B mem-reservation=0B 00:SCAN KUDU [functional_kudu.zipcode_incomes] @@ -29,6 +32,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 select * from functional_kudu.zipcode_incomes where id != '1' and zip = '2' ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -39,6 +43,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 tuple-ids=0 row-size=124B cardinality=1 ---- DISTRIBUTEDPLAN F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -47,6 +52,7 @@ F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 tuple-ids=0 row-size=124B cardinality=1 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=0B mem-reservation=0B DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED] | mem-estimate=0B mem-reservation=0B 00:SCAN KUDU [functional_kudu.zipcode_incomes] @@ -58,6 +64,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 select * from functional_kudu.zipcode_incomes where id > '1' and zip > '2' ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -67,6 +74,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 tuple-ids=0 row-size=124B cardinality=3317 ---- DISTRIBUTEDPLAN F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -75,6 +83,7 @@ F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 tuple-ids=0 row-size=124B cardinality=3317 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=0B mem-reservation=0B DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED] | mem-estimate=0B mem-reservation=0B 00:SCAN KUDU [functional_kudu.zipcode_incomes] @@ -85,6 +94,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 select * from functional_kudu.zipcode_incomes where id = '1' or id = '2' or zip = '3' ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -94,6 +104,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 tuple-ids=0 row-size=124B cardinality=3 ---- DISTRIBUTEDPLAN F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -102,6 +113,7 @@ F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 tuple-ids=0 row-size=124B cardinality=3 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=0B mem-reservation=0B DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED] | mem-estimate=0B mem-reservation=0B 00:SCAN KUDU [functional_kudu.zipcode_incomes] @@ -134,6 +146,7 @@ string_col not in ("bar") and id in (int_col) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -151,6 +164,7 @@ timestamp_col > (nanoseconds_add(cast('1987-05-19 00:00:00' as timestamp), 10)) timestamp_col < (seconds_add(cast('9999-12-31 24:59:59' as timestamp), 10)) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -158,13 +172,14 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 predicates: CAST(date_string_col AS TINYINT) IS NULL, timestamp_col < NULL kudu predicates: smallint_col IS NULL, tinyint_col IS NOT NULL, timestamp_col > TIMESTAMP '1987-05-19 00:00:00.000000010' mem-estimate=0B mem-reservation=0B - tuple-ids=0 row-size=126B cardinality=730 + tuple-ids=0 row-size=97B cardinality=730 ==== select * from functional_kudu.alltypes where timestamp_col in (cast('2010-03-01 00:00:00' as timestamp), cast('2010-03-01 00:01:00' as timestamp)) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -179,6 +194,7 @@ timestamp_col in (cast('2010-03-01 00:00:00' as timestamp), cast('2010-03-01 00:01:00' as timestamp)) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test index 41b76bc37..f22e359e6 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test @@ -40,6 +40,7 @@ order by cnt, bigint_col limit 10 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=144.00MB mem-reservation=264.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -66,6 +67,7 @@ PLAN-ROOT SINK tuple-ids=0 row-size=16B cardinality=unavailable ---- PARALLELPLANS F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -76,6 +78,7 @@ PLAN-ROOT SINK | tuple-ids=2 row-size=16B cardinality=10 | F01:PLAN FRAGMENT [HASH(bigint_col)] hosts=3 instances=9 +Per-Host Resources: mem-estimate=384.00MB mem-reservation=792.00MB 02:TOP-N [LIMIT=10] | order by: count(int_col) ASC, bigint_col ASC | mem-estimate=160B mem-reservation=0B @@ -92,6 +95,7 @@ F01:PLAN FRAGMENT [HASH(bigint_col)] hosts=3 instances=9 | tuple-ids=1 row-size=16B cardinality=unavailable | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 +Per-Host Resources: mem-estimate=432.00MB mem-reservation=0B 01:AGGREGATE [STREAMING] | output: count(int_col) | group by: bigint_col @@ -115,6 +119,7 @@ from functional_parquet.alltypes where id < 10 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=16.00MB mem-reservation=40.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -143,6 +148,7 @@ PLAN-ROOT SINK tuple-ids=0 row-size=8B cardinality=unavailable ---- PARALLELPLANS F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -151,6 +157,7 @@ PLAN-ROOT SINK | tuple-ids=4,3 row-size=16B cardinality=unavailable | F01:PLAN FRAGMENT [HASH(int_col)] hosts=3 instances=9 +Per-Host Resources: mem-estimate=0B mem-reservation=120.00MB 02:ANALYTIC | functions: row_number() | partition by: int_col @@ -169,6 +176,7 @@ F01:PLAN FRAGMENT [HASH(int_col)] hosts=3 instances=9 | tuple-ids=0 row-size=8B cardinality=unavailable | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 +Per-Host Resources: mem-estimate=48.00MB mem-reservation=0B 00:SCAN HDFS [functional_parquet.alltypes, RANDOM] partitions=24/24 files=24 size=178.13KB predicates: id < 10 @@ -186,6 +194,7 @@ from tpch_nested_parquet.customer c, c.c_orders o, o.o_lineitems where c_custkey < 10 and o_orderkey < 5 and l_linenumber < 3 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -226,12 +235,12 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=0B cardinality=10 | 00:SCAN HDFS [tpch_nested_parquet.customer c] - partitions=1/1 files=4 size=292.36MB + partitions=1/1 files=4 size=292.35MB predicates: c_custkey < 10, !empty(c.c_orders) predicates on o: !empty(o.o_lineitems), o_orderkey < 5 predicates on o_lineitems: l_linenumber < 3 stats-rows=150000 extrapolated-rows=disabled - table stats: rows=150000 size=292.36MB + table stats: rows=150000 size=292.35MB columns missing stats: c_orders parquet statistics predicates: c_custkey < 10 parquet dictionary predicates: c_custkey < 10 @@ -239,6 +248,7 @@ PLAN-ROOT SINK tuple-ids=0 row-size=254B cardinality=15000 ---- PARALLELPLANS F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -247,6 +257,7 @@ PLAN-ROOT SINK | tuple-ids=2,1,0 row-size=562B cardinality=1500000 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 +Per-Host Resources: mem-estimate=264.00MB mem-reservation=0B 01:SUBPLAN | mem-estimate=0B mem-reservation=0B | tuple-ids=2,1,0 row-size=562B cardinality=1500000 @@ -284,12 +295,12 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 | tuple-ids=1 row-size=0B cardinality=10 | 00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM] - partitions=1/1 files=4 size=292.36MB + partitions=1/1 files=4 size=292.35MB predicates: c_custkey < 10, !empty(c.c_orders) predicates on o: !empty(o.o_lineitems), o_orderkey < 5 predicates on o_lineitems: l_linenumber < 3 stats-rows=150000 extrapolated-rows=disabled - table stats: rows=150000 size=292.36MB + table stats: rows=150000 size=292.35MB columns missing stats: c_orders parquet statistics predicates: c_custkey < 10 parquet dictionary predicates: c_custkey < 10 @@ -302,6 +313,7 @@ from tpch_nested_parquet.customer c, c.c_orders o1, c.c_orders o2 where o1.o_orderkey = o2.o_orderkey + 2 and o1.o_orderkey < 5 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=88.00MB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -335,16 +347,17 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=0B cardinality=10 | 00:SCAN HDFS [tpch_nested_parquet.customer c] - partitions=1/1 files=4 size=292.36MB + partitions=1/1 files=4 size=292.35MB predicates: !empty(c.c_orders), !empty(c.c_orders) predicates on o1: o1.o_orderkey < 5 stats-rows=150000 extrapolated-rows=disabled - table stats: rows=150000 size=292.36MB + table stats: rows=150000 size=292.35MB columns missing stats: c_orders, c_orders mem-estimate=88.00MB mem-reservation=0B tuple-ids=0 row-size=270B cardinality=150000 ---- PARALLELPLANS F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -353,6 +366,7 @@ PLAN-ROOT SINK | tuple-ids=1,0,2 row-size=286B cardinality=1500000 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 +Per-Host Resources: mem-estimate=264.00MB mem-reservation=408.00MB 01:SUBPLAN | mem-estimate=0B mem-reservation=0B | tuple-ids=1,0,2 row-size=286B cardinality=1500000 @@ -383,11 +397,11 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 | tuple-ids=1 row-size=0B cardinality=10 | 00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM] - partitions=1/1 files=4 size=292.36MB + partitions=1/1 files=4 size=292.35MB predicates: !empty(c.c_orders), !empty(c.c_orders) predicates on o1: o1.o_orderkey < 5 stats-rows=150000 extrapolated-rows=disabled - table stats: rows=150000 size=292.36MB + table stats: rows=150000 size=292.35MB columns missing stats: c_orders, c_orders mem-estimate=88.00MB mem-reservation=0B tuple-ids=0 row-size=270B cardinality=150000 diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test index f5ae46d7c..0de7109a3 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test @@ -8,6 +8,7 @@ where int_col > 1 and int_col * rand() > 50 and int_col is null and int_col > tinyint_col; ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=42.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -37,6 +38,7 @@ and timestamp_cmp(timestamp_col, '2016-11-20 00:00:00') = 1 and year > 2000 and month < 12; ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=138.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -67,6 +69,7 @@ and mod(int_col,50) IN (0,1) and id IN (int_col); ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=58.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test index bb97c2699..f3dd19af4 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test @@ -1,10 +1,27 @@ # Parquet scan select * from tpch_parquet.lineitem +---- PLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=80.00MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +00:SCAN HDFS [tpch_parquet.lineitem] + partitions=1/1 files=3 size=193.92MB + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=193.92MB + column stats: all + mem-estimate=80.00MB mem-reservation=0B + tuple-ids=0 row-size=263B cardinality=6001215 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=80.00MB F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -13,6 +30,7 @@ PLAN-ROOT SINK | tuple-ids=0 row-size=263B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM] partitions=1/1 files=3 size=193.92MB stats-rows=6001215 extrapolated-rows=disabled @@ -25,6 +43,7 @@ Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=160.00MB F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -33,6 +52,7 @@ PLAN-ROOT SINK | tuple-ids=0 row-size=263B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=160.00MB mem-reservation=0B 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM] partitions=1/1 files=3 size=193.92MB stats-rows=6001215 extrapolated-rows=disabled @@ -43,11 +63,28 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 ==== # Text scan select * from tpch.lineitem; +---- PLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=88.00MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +00:SCAN HDFS [tpch.lineitem] + partitions=1/1 files=1 size=718.94MB + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=718.94MB + column stats: all + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=263B cardinality=6001215 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=88.00MB F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -56,6 +93,7 @@ PLAN-ROOT SINK | tuple-ids=0 row-size=263B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B 00:SCAN HDFS [tpch.lineitem, RANDOM] partitions=1/1 files=1 size=718.94MB stats-rows=6001215 extrapolated-rows=disabled @@ -68,6 +106,7 @@ Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=176.00MB F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -76,6 +115,7 @@ PLAN-ROOT SINK | tuple-ids=0 row-size=263B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B 00:SCAN HDFS [tpch.lineitem, RANDOM] partitions=1/1 files=1 size=718.94MB stats-rows=6001215 extrapolated-rows=disabled @@ -86,6 +126,22 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 ==== # HBase scan select * from functional_hbase.alltypes +---- PLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=1.00GB +WARNING: The following tables are missing relevant table and/or column statistics. +functional_hbase.alltypes + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +00:SCAN HBASE [functional_hbase.alltypes] + table stats: rows=unavailable + column stats: unavailable + mem-estimate=1.00GB mem-reservation=0B + tuple-ids=0 row-size=88B cardinality=14298 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=1.00GB @@ -93,6 +149,7 @@ WARNING: The following tables are missing relevant table and/or column statistic functional_hbase.alltypes F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -100,7 +157,8 @@ PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | tuple-ids=0 row-size=88B cardinality=14298 | -F00:PLAN FRAGMENT [RANDOM] hosts=100 instances=100 +F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B 00:SCAN HBASE [functional_hbase.alltypes] table stats: rows=unavailable column stats: unavailable @@ -113,6 +171,7 @@ WARNING: The following tables are missing relevant table and/or column statistic functional_hbase.alltypes F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -120,7 +179,8 @@ PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | tuple-ids=0 row-size=88B cardinality=14298 | -F00:PLAN FRAGMENT [RANDOM] hosts=100 instances=200 +F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +Per-Host Resources: mem-estimate=2.00GB mem-reservation=0B 00:SCAN HBASE [functional_hbase.alltypes] table stats: rows=unavailable column stats: unavailable @@ -129,6 +189,20 @@ F00:PLAN FRAGMENT [RANDOM] hosts=100 instances=200 ==== # Data source scan select * from functional.alltypes_datasource +---- PLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=1.00GB +WARNING: The following tables are missing relevant table and/or column statistics. +functional.alltypes_datasource + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +00:SCAN DATA SOURCE [functional.alltypes_datasource] + mem-estimate=1.00GB mem-reservation=0B + tuple-ids=0 row-size=116B cardinality=5000 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=1.00GB @@ -136,6 +210,7 @@ WARNING: The following tables are missing relevant table and/or column statistic functional.alltypes_datasource F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -144,6 +219,7 @@ PLAN-ROOT SINK | tuple-ids=0 row-size=116B cardinality=5000 | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B 00:SCAN DATA SOURCE [functional.alltypes_datasource] mem-estimate=1.00GB mem-reservation=0B tuple-ids=0 row-size=116B cardinality=5000 @@ -154,6 +230,7 @@ WARNING: The following tables are missing relevant table and/or column statistic functional.alltypes_datasource F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -162,6 +239,7 @@ PLAN-ROOT SINK | tuple-ids=0 row-size=116B cardinality=5000 | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +Per-Host Resources: mem-estimate=2.00GB mem-reservation=0B 00:SCAN DATA SOURCE [functional.alltypes_datasource] mem-estimate=1.00GB mem-reservation=0B tuple-ids=0 row-size=116B cardinality=5000 @@ -170,11 +248,41 @@ F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 select * from tpch.lineitem union all select * from tpch.lineitem +---- PLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=88.00MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +00:UNION +| pass-through-operands: all +| mem-estimate=0B mem-reservation=0B +| tuple-ids=2 row-size=263B cardinality=12002430 +| +|--02:SCAN HDFS [tpch.lineitem] +| partitions=1/1 files=1 size=718.94MB +| stats-rows=6001215 extrapolated-rows=disabled +| table stats: rows=6001215 size=718.94MB +| column stats: all +| mem-estimate=88.00MB mem-reservation=0B +| tuple-ids=1 row-size=263B cardinality=6001215 +| +01:SCAN HDFS [tpch.lineitem] + partitions=1/1 files=1 size=718.94MB + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=718.94MB + column stats: all + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=263B cardinality=6001215 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B -Per-Host Resource Estimates: Memory=176.00MB +Per-Host Resource Estimates: Memory=88.00MB F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -183,6 +291,7 @@ PLAN-ROOT SINK | tuple-ids=2 row-size=263B cardinality=12002430 | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B 00:UNION | pass-through-operands: all | mem-estimate=0B mem-reservation=0B @@ -205,9 +314,10 @@ F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 tuple-ids=0 row-size=263B cardinality=6001215 ---- PARALLELPLANS Per-Host Resource Reservation: Memory=0B -Per-Host Resource Estimates: Memory=352.00MB +Per-Host Resource Estimates: Memory=176.00MB F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -216,6 +326,7 @@ PLAN-ROOT SINK | tuple-ids=2 row-size=263B cardinality=12002430 | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B 00:UNION | pass-through-operands: all | mem-estimate=0B mem-reservation=0B @@ -241,11 +352,34 @@ F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 select l_orderkey, count(*) from tpch_parquet.lineitem group by l_orderkey +---- PLAN +Per-Host Resource Reservation: Memory=264.00MB +Per-Host Resource Estimates: Memory=106.24MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=106.24MB mem-reservation=264.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +01:AGGREGATE [FINALIZE] +| output: count(*) +| group by: l_orderkey +| mem-estimate=26.24MB mem-reservation=264.00MB +| tuple-ids=1 row-size=16B cardinality=1563438 +| +00:SCAN HDFS [tpch_parquet.lineitem] + partitions=1/1 files=3 size=193.92MB + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=193.92MB + column stats: all + mem-estimate=80.00MB mem-reservation=0B + tuple-ids=0 row-size=8B cardinality=6001215 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=264.00MB Per-Host Resource Estimates: Memory=116.24MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -254,6 +388,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=16B cardinality=1563438 | F01:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 +Per-Host Resources: mem-estimate=10.00MB mem-reservation=264.00MB 03:AGGREGATE [FINALIZE] | output: count:merge(*) | group by: l_orderkey @@ -265,6 +400,7 @@ F01:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 | tuple-ids=1 row-size=16B cardinality=1563438 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=106.24MB mem-reservation=0B 01:AGGREGATE [STREAMING] | output: count(*) | group by: l_orderkey @@ -283,6 +419,7 @@ Per-Host Resource Reservation: Memory=528.00MB Per-Host Resource Estimates: Memory=232.48MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -291,6 +428,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=16B cardinality=1563438 | F01:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 +Per-Host Resources: mem-estimate=20.00MB mem-reservation=528.00MB 03:AGGREGATE [FINALIZE] | output: count:merge(*) | group by: l_orderkey @@ -302,6 +440,7 @@ F01:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 | tuple-ids=1 row-size=16B cardinality=1563438 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=212.48MB mem-reservation=0B 01:AGGREGATE [STREAMING] | output: count(*) | group by: l_orderkey @@ -318,11 +457,33 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 ==== # Non-grouping aggregation with zero-slot parquet scan select count(*) from tpch_parquet.lineitem ----- DISTRIBUTEDPLAN +---- PLAN Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=90.00MB +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=90.00MB mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +01:AGGREGATE [FINALIZE] +| output: sum_init_zero(tpch_parquet.lineitem.parquet-stats: num_rows) +| mem-estimate=10.00MB mem-reservation=0B +| tuple-ids=1 row-size=8B cardinality=1 +| +00:SCAN HDFS [tpch_parquet.lineitem] + partitions=1/1 files=3 size=193.92MB + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=193.92MB + column stats: all + mem-estimate=80.00MB mem-reservation=0B + tuple-ids=0 row-size=8B cardinality=6001215 +---- DISTRIBUTEDPLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=100.00MB + F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=10.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -336,6 +497,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=8B cardinality=1 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=90.00MB mem-reservation=0B 01:AGGREGATE | output: sum_init_zero(tpch_parquet.lineitem.parquet-stats: num_rows) | mem-estimate=10.00MB mem-reservation=0B @@ -350,9 +512,10 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 tuple-ids=0 row-size=8B cardinality=6001215 ---- PARALLELPLANS Per-Host Resource Reservation: Memory=0B -Per-Host Resource Estimates: Memory=180.00MB +Per-Host Resource Estimates: Memory=190.00MB F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=10.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -366,6 +529,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=8B cardinality=1 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=180.00MB mem-reservation=0B 01:AGGREGATE | output: sum_init_zero(tpch_parquet.lineitem.parquet-stats: num_rows) | mem-estimate=10.00MB mem-reservation=0B @@ -383,11 +547,33 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 select * from tpch_parquet.lineitem order by l_comment +---- PLAN +Per-Host Resource Reservation: Memory=48.00MB +Per-Host Resource Estimates: Memory=240.00MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=240.00MB mem-reservation=48.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +01:SORT +| order by: l_comment ASC +| mem-estimate=160.00MB mem-reservation=48.00MB +| tuple-ids=1 row-size=263B cardinality=6001215 +| +00:SCAN HDFS [tpch_parquet.lineitem] + partitions=1/1 files=3 size=193.92MB + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=193.92MB + column stats: all + mem-estimate=80.00MB mem-reservation=0B + tuple-ids=0 row-size=263B cardinality=6001215 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=48.00MB Per-Host Resource Estimates: Memory=240.00MB F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -397,6 +583,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=263B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=240.00MB mem-reservation=48.00MB 01:SORT | order by: l_comment ASC | mem-estimate=160.00MB mem-reservation=48.00MB @@ -414,6 +601,7 @@ Per-Host Resource Reservation: Memory=96.00MB Per-Host Resource Estimates: Memory=480.00MB F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -423,6 +611,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=263B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=480.00MB mem-reservation=96.00MB 01:SORT | order by: l_comment ASC | mem-estimate=160.00MB mem-reservation=48.00MB @@ -441,11 +630,33 @@ select * from tpch_parquet.lineitem order by l_comment limit 100 +---- PLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=80.03MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=80.03MB mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +01:TOP-N [LIMIT=100] +| order by: l_comment ASC +| mem-estimate=25.66KB mem-reservation=0B +| tuple-ids=1 row-size=263B cardinality=100 +| +00:SCAN HDFS [tpch_parquet.lineitem] + partitions=1/1 files=3 size=193.92MB + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=193.92MB + column stats: all + mem-estimate=80.00MB mem-reservation=0B + tuple-ids=0 row-size=263B cardinality=6001215 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=80.03MB F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -456,6 +667,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=263B cardinality=100 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=80.03MB mem-reservation=0B 01:TOP-N [LIMIT=100] | order by: l_comment ASC | mem-estimate=25.66KB mem-reservation=0B @@ -473,6 +685,7 @@ Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=160.05MB F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -483,6 +696,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=263B cardinality=100 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=160.05MB mem-reservation=0B 01:TOP-N [LIMIT=100] | order by: l_comment ASC | mem-estimate=25.66KB mem-reservation=0B @@ -496,14 +710,47 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 mem-estimate=80.00MB mem-reservation=0B tuple-ids=0 row-size=263B cardinality=6001215 ==== -# Hash Join +# Broadcast Hash Join select * from tpch.lineitem inner join tpch.orders on l_orderkey = o_orderkey +---- PLAN +Per-Host Resource Reservation: Memory=136.00MB +Per-Host Resource Estimates: Memory=476.41MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=476.41MB mem-reservation=136.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +02:HASH JOIN [INNER JOIN] +| hash predicates: l_orderkey = o_orderkey +| fk/pk conjuncts: l_orderkey = o_orderkey +| runtime filters: RF000 <- o_orderkey +| mem-estimate=300.41MB mem-reservation=136.00MB +| tuple-ids=0,1 row-size=454B cardinality=5757710 +| +|--01:SCAN HDFS [tpch.orders] +| partitions=1/1 files=1 size=162.56MB +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=162.56MB +| column stats: all +| mem-estimate=88.00MB mem-reservation=0B +| tuple-ids=1 row-size=191B cardinality=1500000 +| +00:SCAN HDFS [tpch.lineitem] + partitions=1/1 files=1 size=718.94MB + runtime filters: RF000 -> l_orderkey + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=718.94MB + column stats: all + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=263B cardinality=6001215 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=136.00MB -Per-Host Resource Estimates: Memory=388.41MB +Per-Host Resource Estimates: Memory=476.41MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -512,6 +759,7 @@ PLAN-ROOT SINK | tuple-ids=0,1 row-size=454B cardinality=5757710 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=388.41MB mem-reservation=136.00MB 02:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: l_orderkey = o_orderkey | fk/pk conjuncts: l_orderkey = o_orderkey @@ -524,6 +772,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | | tuple-ids=1 row-size=191B cardinality=1500000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B | 01:SCAN HDFS [tpch.orders, RANDOM] | partitions=1/1 files=1 size=162.56MB | stats-rows=1500000 extrapolated-rows=disabled @@ -545,6 +794,7 @@ Per-Host Resource Reservation: Memory=272.00MB Per-Host Resource Estimates: Memory=952.83MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -553,6 +803,7 @@ PLAN-ROOT SINK | tuple-ids=0,1 row-size=454B cardinality=5757710 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=776.83MB mem-reservation=272.00MB 02:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=00 | hash predicates: l_orderkey = o_orderkey @@ -562,6 +813,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | tuple-ids=0,1 row-size=454B cardinality=5757710 | |--F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| | Per-Host Resources: included in parent fragment | JOIN BUILD | | join-table-id=00 plan-id=01 cohort-id=01 | | build expressions: o_orderkey @@ -572,6 +824,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | tuple-ids=1 row-size=191B cardinality=1500000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B | 01:SCAN HDFS [tpch.orders, RANDOM] | partitions=1/1 files=1 size=162.56MB | stats-rows=1500000 extrapolated-rows=disabled @@ -589,14 +842,187 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 mem-estimate=88.00MB mem-reservation=0B tuple-ids=0 row-size=263B cardinality=6001215 ==== +# Shuffle Hash Join +select * +from tpch.lineitem inner join /* +shuffle */ tpch.orders on l_orderkey = o_orderkey +---- PLAN +Per-Host Resource Reservation: Memory=136.00MB +Per-Host Resource Estimates: Memory=476.41MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=476.41MB mem-reservation=136.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +02:HASH JOIN [INNER JOIN] +| hash predicates: l_orderkey = o_orderkey +| fk/pk conjuncts: l_orderkey = o_orderkey +| runtime filters: RF000 <- o_orderkey +| mem-estimate=300.41MB mem-reservation=136.00MB +| tuple-ids=0,1 row-size=454B cardinality=5757710 +| +|--01:SCAN HDFS [tpch.orders] +| partitions=1/1 files=1 size=162.56MB +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=162.56MB +| column stats: all +| mem-estimate=88.00MB mem-reservation=0B +| tuple-ids=1 row-size=191B cardinality=1500000 +| +00:SCAN HDFS [tpch.lineitem] + partitions=1/1 files=1 size=718.94MB + runtime filters: RF000 -> l_orderkey + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=718.94MB + column stats: all + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=263B cardinality=6001215 +---- DISTRIBUTEDPLAN +Per-Host Resource Reservation: Memory=136.00MB +Per-Host Resource Estimates: Memory=276.14MB + +F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +05:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=0,1 row-size=454B cardinality=5757710 +| +F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 +Per-Host Resources: mem-estimate=100.14MB mem-reservation=136.00MB +02:HASH JOIN [INNER JOIN, PARTITIONED] +| hash predicates: l_orderkey = o_orderkey +| fk/pk conjuncts: l_orderkey = o_orderkey +| runtime filters: RF000 <- o_orderkey +| mem-estimate=100.14MB mem-reservation=136.00MB +| tuple-ids=0,1 row-size=454B cardinality=5757710 +| +|--04:EXCHANGE [HASH(o_orderkey)] +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=1 row-size=191B cardinality=1500000 +| | +| F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B +| 01:SCAN HDFS [tpch.orders, RANDOM] +| partitions=1/1 files=1 size=162.56MB +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=162.56MB +| column stats: all +| mem-estimate=88.00MB mem-reservation=0B +| tuple-ids=1 row-size=191B cardinality=1500000 +| +03:EXCHANGE [HASH(l_orderkey)] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=0 row-size=263B cardinality=6001215 +| +F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B +00:SCAN HDFS [tpch.lineitem, RANDOM] + partitions=1/1 files=1 size=718.94MB + runtime filters: RF000 -> l_orderkey + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=718.94MB + column stats: all + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=263B cardinality=6001215 +---- PARALLELPLANS +Per-Host Resource Reservation: Memory=272.00MB +Per-Host Resource Estimates: Memory=452.14MB + +F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +05:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=0,1 row-size=454B cardinality=5757710 +| +F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 +Per-Host Resources: mem-estimate=100.14MB mem-reservation=272.00MB +02:HASH JOIN [INNER JOIN, PARTITIONED] +| hash-table-id=00 +| hash predicates: l_orderkey = o_orderkey +| fk/pk conjuncts: l_orderkey = o_orderkey +| runtime filters: RF000 <- o_orderkey +| mem-estimate=50.07MB mem-reservation=136.00MB +| tuple-ids=0,1 row-size=454B cardinality=5757710 +| +|--F04:PLAN FRAGMENT [HASH(l_orderkey)] hosts=2 instances=4 +| | Per-Host Resources: included in parent fragment +| JOIN BUILD +| | join-table-id=00 plan-id=01 cohort-id=01 +| | build expressions: o_orderkey +| | mem-estimate=0B mem-reservation=0B +| | +| 04:EXCHANGE [HASH(o_orderkey)] +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=1 row-size=191B cardinality=1500000 +| | +| F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B +| 01:SCAN HDFS [tpch.orders, RANDOM] +| partitions=1/1 files=1 size=162.56MB +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=162.56MB +| column stats: all +| mem-estimate=88.00MB mem-reservation=0B +| tuple-ids=1 row-size=191B cardinality=1500000 +| +03:EXCHANGE [HASH(l_orderkey)] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=0 row-size=263B cardinality=6001215 +| +F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B +00:SCAN HDFS [tpch.lineitem, RANDOM] + partitions=1/1 files=1 size=718.94MB + runtime filters: RF000 -> l_orderkey + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=718.94MB + column stats: all + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=263B cardinality=6001215 +==== # Nested loop join select * from tpch.lineitem, tpch.orders +---- PLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=449.10MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=449.10MB mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +02:NESTED LOOP JOIN [CROSS JOIN] +| mem-estimate=273.10MB mem-reservation=0B +| tuple-ids=0,1 row-size=454B cardinality=9001822500000 +| +|--01:SCAN HDFS [tpch.orders] +| partitions=1/1 files=1 size=162.56MB +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=162.56MB +| column stats: all +| mem-estimate=88.00MB mem-reservation=0B +| tuple-ids=1 row-size=191B cardinality=1500000 +| +00:SCAN HDFS [tpch.lineitem] + partitions=1/1 files=1 size=718.94MB + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=718.94MB + column stats: all + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=263B cardinality=6001215 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B -Per-Host Resource Estimates: Memory=361.10MB +Per-Host Resource Estimates: Memory=449.10MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -605,6 +1031,7 @@ PLAN-ROOT SINK | tuple-ids=0,1 row-size=454B cardinality=9001822500000 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=361.10MB mem-reservation=0B 02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] | mem-estimate=273.10MB mem-reservation=0B | tuple-ids=0,1 row-size=454B cardinality=9001822500000 @@ -614,6 +1041,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | | tuple-ids=1 row-size=191B cardinality=1500000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B | 01:SCAN HDFS [tpch.orders, RANDOM] | partitions=1/1 files=1 size=162.56MB | stats-rows=1500000 extrapolated-rows=disabled @@ -634,6 +1062,7 @@ Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=898.21MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -642,12 +1071,14 @@ PLAN-ROOT SINK | tuple-ids=0,1 row-size=454B cardinality=9001822500000 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=722.21MB mem-reservation=0B 02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] | join table id: 00 | mem-estimate=273.10MB mem-reservation=0B | tuple-ids=0,1 row-size=454B cardinality=9001822500000 | |--F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| | Per-Host Resources: included in parent fragment | JOIN BUILD | | join-table-id=00 plan-id=01 cohort-id=01 | | mem-estimate=0B mem-reservation=0B @@ -657,6 +1088,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | tuple-ids=1 row-size=191B cardinality=1500000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B | 01:SCAN HDFS [tpch.orders, RANDOM] | partitions=1/1 files=1 size=162.56MB | stats-rows=1500000 extrapolated-rows=disabled @@ -675,12 +1107,26 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 ==== # Empty set node select * from functional.alltypes where 1 = 2 +---- PLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=10.00MB +Codegen disabled by planner + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +00:EMPTYSET + mem-estimate=0B mem-reservation=0B + tuple-ids=0 row-size=0B cardinality=0 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=10.00MB Codegen disabled by planner F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -693,6 +1139,7 @@ Per-Host Resource Estimates: Memory=10.00MB Codegen disabled by planner F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -703,12 +1150,41 @@ PLAN-ROOT SINK # Analytic function select max(tinyint_col) over(partition by int_col) from functional.alltypes +---- PLAN +Per-Host Resource Reservation: Memory=40.00MB +Per-Host Resource Estimates: Memory=24.00MB +Codegen disabled by planner + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=24.00MB mem-reservation=40.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +02:ANALYTIC +| functions: max(tinyint_col) +| partition by: int_col +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=3,2 row-size=6B cardinality=7300 +| +01:SORT +| order by: int_col ASC NULLS FIRST +| mem-estimate=8.00MB mem-reservation=24.00MB +| tuple-ids=3 row-size=5B cardinality=7300 +| +00:SCAN HDFS [functional.alltypes] + partitions=24/24 files=24 size=478.45KB + stats-rows=7300 extrapolated-rows=disabled + table stats: rows=7300 size=478.45KB + column stats: all + mem-estimate=16.00MB mem-reservation=0B + tuple-ids=0 row-size=5B cardinality=7300 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=40.00MB Per-Host Resource Estimates: Memory=24.00MB Codegen disabled by planner F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -717,6 +1193,7 @@ PLAN-ROOT SINK | tuple-ids=3,2 row-size=6B cardinality=7300 | F01:PLAN FRAGMENT [HASH(int_col)] hosts=3 instances=3 +Per-Host Resources: mem-estimate=8.00MB mem-reservation=40.00MB 02:ANALYTIC | functions: max(tinyint_col) | partition by: int_col @@ -733,6 +1210,7 @@ F01:PLAN FRAGMENT [HASH(int_col)] hosts=3 instances=3 | tuple-ids=0 row-size=5B cardinality=7300 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B 00:SCAN HDFS [functional.alltypes, RANDOM] partitions=24/24 files=24 size=478.45KB stats-rows=7300 extrapolated-rows=disabled @@ -746,6 +1224,7 @@ Per-Host Resource Estimates: Memory=48.00MB Codegen disabled by planner F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -754,6 +1233,7 @@ PLAN-ROOT SINK | tuple-ids=3,2 row-size=6B cardinality=7300 | F01:PLAN FRAGMENT [HASH(int_col)] hosts=3 instances=6 +Per-Host Resources: mem-estimate=16.00MB mem-reservation=80.00MB 02:ANALYTIC | functions: max(tinyint_col) | partition by: int_col @@ -770,6 +1250,7 @@ F01:PLAN FRAGMENT [HASH(int_col)] hosts=3 instances=6 | tuple-ids=0 row-size=5B cardinality=7300 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B 00:SCAN HDFS [functional.alltypes, RANDOM] partitions=24/24 files=24 size=478.45KB stats-rows=7300 extrapolated-rows=disabled @@ -778,6 +1259,599 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 mem-estimate=16.00MB mem-reservation=0B tuple-ids=0 row-size=5B cardinality=7300 ==== +# Pipeline of blocking operators from analytic fns. Blocking operators break +# the pipeline so they do not all consume resources concurrently. +select *, row_number() over (order by o_totalprice) rnum_price, + row_number() over (order by o_orderdate) rnum_date, + row_number() over (order by o_orderpriority) rnum_priority +from tpch_parquet.orders +---- PLAN +Per-Host Resource Reservation: Memory=144.00MB +Per-Host Resource Estimates: Memory=160.00MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=160.00MB mem-reservation=144.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +06:ANALYTIC +| functions: row_number() +| order by: o_orderpriority ASC +| window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=10,5 row-size=215B cardinality=1500000 +| +05:SORT +| order by: o_orderpriority ASC +| mem-estimate=80.00MB mem-reservation=48.00MB +| tuple-ids=10 row-size=207B cardinality=1500000 +| +04:ANALYTIC +| functions: row_number() +| order by: o_orderdate ASC +| window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=8,4 row-size=207B cardinality=1500000 +| +03:SORT +| order by: o_orderdate ASC +| mem-estimate=80.00MB mem-reservation=48.00MB +| tuple-ids=8 row-size=199B cardinality=1500000 +| +02:ANALYTIC +| functions: row_number() +| order by: o_totalprice ASC +| window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=6,3 row-size=199B cardinality=1500000 +| +01:SORT +| order by: o_totalprice ASC +| mem-estimate=80.00MB mem-reservation=48.00MB +| tuple-ids=6 row-size=191B cardinality=1500000 +| +00:SCAN HDFS [tpch_parquet.orders] + partitions=1/1 files=2 size=54.20MB + stats-rows=1500000 extrapolated-rows=disabled + table stats: rows=1500000 size=54.20MB + column stats: all + mem-estimate=40.00MB mem-reservation=0B + tuple-ids=0 row-size=191B cardinality=1500000 +---- DISTRIBUTEDPLAN +Per-Host Resource Reservation: Memory=176.00MB +Per-Host Resource Estimates: Memory=280.00MB + +F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=160.00MB mem-reservation=128.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +06:ANALYTIC +| functions: row_number() +| order by: o_orderpriority ASC +| window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=10,5 row-size=215B cardinality=1500000 +| +05:SORT +| order by: o_orderpriority ASC +| mem-estimate=80.00MB mem-reservation=48.00MB +| tuple-ids=10 row-size=207B cardinality=1500000 +| +04:ANALYTIC +| functions: row_number() +| order by: o_orderdate ASC +| window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=8,4 row-size=207B cardinality=1500000 +| +03:SORT +| order by: o_orderdate ASC +| mem-estimate=80.00MB mem-reservation=48.00MB +| tuple-ids=8 row-size=199B cardinality=1500000 +| +02:ANALYTIC +| functions: row_number() +| order by: o_totalprice ASC +| window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=6,3 row-size=199B cardinality=1500000 +| +07:MERGING-EXCHANGE [UNPARTITIONED] +| order by: o_totalprice ASC +| mem-estimate=0B mem-reservation=0B +| tuple-ids=6 row-size=191B cardinality=1500000 +| +F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +Per-Host Resources: mem-estimate=120.00MB mem-reservation=48.00MB +01:SORT +| order by: o_totalprice ASC +| mem-estimate=80.00MB mem-reservation=48.00MB +| tuple-ids=6 row-size=191B cardinality=1500000 +| +00:SCAN HDFS [tpch_parquet.orders, RANDOM] + partitions=1/1 files=2 size=54.20MB + stats-rows=1500000 extrapolated-rows=disabled + table stats: rows=1500000 size=54.20MB + column stats: all + mem-estimate=40.00MB mem-reservation=0B + tuple-ids=0 row-size=191B cardinality=1500000 +---- PARALLELPLANS +Per-Host Resource Reservation: Memory=224.00MB +Per-Host Resource Estimates: Memory=400.00MB + +F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=160.00MB mem-reservation=128.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +06:ANALYTIC +| functions: row_number() +| order by: o_orderpriority ASC +| window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=10,5 row-size=215B cardinality=1500000 +| +05:SORT +| order by: o_orderpriority ASC +| mem-estimate=80.00MB mem-reservation=48.00MB +| tuple-ids=10 row-size=207B cardinality=1500000 +| +04:ANALYTIC +| functions: row_number() +| order by: o_orderdate ASC +| window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=8,4 row-size=207B cardinality=1500000 +| +03:SORT +| order by: o_orderdate ASC +| mem-estimate=80.00MB mem-reservation=48.00MB +| tuple-ids=8 row-size=199B cardinality=1500000 +| +02:ANALYTIC +| functions: row_number() +| order by: o_totalprice ASC +| window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=6,3 row-size=199B cardinality=1500000 +| +07:MERGING-EXCHANGE [UNPARTITIONED] +| order by: o_totalprice ASC +| mem-estimate=0B mem-reservation=0B +| tuple-ids=6 row-size=191B cardinality=1500000 +| +F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +Per-Host Resources: mem-estimate=240.00MB mem-reservation=96.00MB +01:SORT +| order by: o_totalprice ASC +| mem-estimate=80.00MB mem-reservation=48.00MB +| tuple-ids=6 row-size=191B cardinality=1500000 +| +00:SCAN HDFS [tpch_parquet.orders, RANDOM] + partitions=1/1 files=2 size=54.20MB + stats-rows=1500000 extrapolated-rows=disabled + table stats: rows=1500000 size=54.20MB + column stats: all + mem-estimate=40.00MB mem-reservation=0B + tuple-ids=0 row-size=191B cardinality=1500000 +==== +# Union with non-trivial branches: each branch executes sequentially within fragment. +select distinct l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment +from tpch_parquet.lineitem join tpch_parquet.orders on l_orderkey = o_orderkey +where l_tax > 10 +union all +select l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment +from tpch_parquet.lineitem join tpch_parquet.orders on l_orderkey = o_orderkey +where o_orderpriority = '2-HIGH' +union all +select l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment +from tpch_parquet.lineitem join tpch_parquet.orders on l_orderkey = o_orderkey +where l_shipmode = 'F' +---- PLAN +Per-Host Resource Reservation: Memory=400.00MB +Per-Host Resource Estimates: Memory=135.17MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=135.17MB mem-reservation=400.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +00:UNION +| pass-through-operands: 04 +| mem-estimate=0B mem-reservation=0B +| tuple-ids=7 row-size=70B cardinality=2549844 +| +|--10:HASH JOIN [INNER JOIN] +| | hash predicates: l_orderkey = o_orderkey +| | fk/pk conjuncts: l_orderkey = o_orderkey +| | runtime filters: RF002 <- o_orderkey +| | mem-estimate=12.59MB mem-reservation=136.00MB +| | tuple-ids=5,6 row-size=99B cardinality=822530 +| | +| |--09:SCAN HDFS [tpch_parquet.orders] +| | partitions=1/1 files=2 size=54.20MB +| | stats-rows=1500000 extrapolated-rows=disabled +| | table stats: rows=1500000 size=54.20MB +| | column stats: all +| | mem-estimate=40.00MB mem-reservation=0B +| | tuple-ids=6 row-size=8B cardinality=1500000 +| | +| 08:SCAN HDFS [tpch_parquet.lineitem] +| partitions=1/1 files=3 size=193.92MB +| predicates: l_shipmode = 'F' +| runtime filters: RF002 -> l_orderkey +| stats-rows=6001215 extrapolated-rows=disabled +| table stats: rows=6001215 size=193.92MB +| column stats: all +| parquet statistics predicates: l_shipmode = 'F' +| parquet dictionary predicates: l_shipmode = 'F' +| mem-estimate=80.00MB mem-reservation=0B +| tuple-ids=5 row-size=91B cardinality=857316 +| +|--07:HASH JOIN [INNER JOIN] +| | hash predicates: l_orderkey = o_orderkey +| | fk/pk conjuncts: l_orderkey = o_orderkey +| | runtime filters: RF001 <- o_orderkey +| | mem-estimate=10.20MB mem-reservation=136.00MB +| | tuple-ids=3,4 row-size=103B cardinality=1151542 +| | +| |--06:SCAN HDFS [tpch_parquet.orders] +| | partitions=1/1 files=2 size=54.20MB +| | predicates: o_orderpriority = '2-HIGH' +| | stats-rows=1500000 extrapolated-rows=disabled +| | table stats: rows=1500000 size=54.20MB +| | column stats: all +| | parquet statistics predicates: o_orderpriority = '2-HIGH' +| | parquet dictionary predicates: o_orderpriority = '2-HIGH' +| | mem-estimate=40.00MB mem-reservation=0B +| | tuple-ids=4 row-size=32B cardinality=300000 +| | +| 05:SCAN HDFS [tpch_parquet.lineitem] +| partitions=1/1 files=3 size=193.92MB +| runtime filters: RF001 -> l_orderkey +| stats-rows=6001215 extrapolated-rows=disabled +| table stats: rows=6001215 size=193.92MB +| column stats: all +| mem-estimate=80.00MB mem-reservation=0B +| tuple-ids=3 row-size=70B cardinality=6001215 +| +04:AGGREGATE [FINALIZE] +| group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment +| mem-estimate=42.58MB mem-reservation=264.00MB +| tuple-ids=2 row-size=70B cardinality=575772 +| +03:HASH JOIN [INNER JOIN] +| hash predicates: l_orderkey = o_orderkey +| fk/pk conjuncts: l_orderkey = o_orderkey +| runtime filters: RF000 <- o_orderkey +| mem-estimate=12.59MB mem-reservation=136.00MB +| tuple-ids=0,1 row-size=86B cardinality=575772 +| +|--02:SCAN HDFS [tpch_parquet.orders] +| partitions=1/1 files=2 size=54.20MB +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=54.20MB +| column stats: all +| mem-estimate=40.00MB mem-reservation=0B +| tuple-ids=1 row-size=8B cardinality=1500000 +| +01:SCAN HDFS [tpch_parquet.lineitem] + partitions=1/1 files=3 size=193.92MB + predicates: l_tax > 10 + runtime filters: RF000 -> l_orderkey + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=193.92MB + column stats: all + parquet statistics predicates: l_tax > 10 + parquet dictionary predicates: l_tax > 10 + mem-estimate=80.00MB mem-reservation=0B + tuple-ids=0 row-size=78B cardinality=600122 +---- DISTRIBUTEDPLAN +Per-Host Resource Reservation: Memory=400.00MB +Per-Host Resource Estimates: Memory=339.36MB + +F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +17:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=7 row-size=70B cardinality=2549844 +| +F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=92.59MB mem-reservation=264.00MB +00:UNION +| pass-through-operands: 14 +| mem-estimate=0B mem-reservation=0B +| tuple-ids=7 row-size=70B cardinality=2549844 +| +|--10:HASH JOIN [INNER JOIN, BROADCAST] +| | hash predicates: l_orderkey = o_orderkey +| | fk/pk conjuncts: l_orderkey = o_orderkey +| | runtime filters: RF002 <- o_orderkey +| | mem-estimate=12.59MB mem-reservation=136.00MB +| | tuple-ids=5,6 row-size=99B cardinality=822530 +| | +| |--16:EXCHANGE [BROADCAST] +| | | mem-estimate=0B mem-reservation=0B +| | | tuple-ids=6 row-size=8B cardinality=1500000 +| | | +| | F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| | Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B +| | 09:SCAN HDFS [tpch_parquet.orders, RANDOM] +| | partitions=1/1 files=2 size=54.20MB +| | stats-rows=1500000 extrapolated-rows=disabled +| | table stats: rows=1500000 size=54.20MB +| | column stats: all +| | mem-estimate=40.00MB mem-reservation=0B +| | tuple-ids=6 row-size=8B cardinality=1500000 +| | +| 08:SCAN HDFS [tpch_parquet.lineitem, RANDOM] +| partitions=1/1 files=3 size=193.92MB +| predicates: l_shipmode = 'F' +| runtime filters: RF002 -> l_orderkey +| stats-rows=6001215 extrapolated-rows=disabled +| table stats: rows=6001215 size=193.92MB +| column stats: all +| parquet statistics predicates: l_shipmode = 'F' +| parquet dictionary predicates: l_shipmode = 'F' +| mem-estimate=80.00MB mem-reservation=0B +| tuple-ids=5 row-size=91B cardinality=857316 +| +|--07:HASH JOIN [INNER JOIN, BROADCAST] +| | hash predicates: l_orderkey = o_orderkey +| | fk/pk conjuncts: l_orderkey = o_orderkey +| | runtime filters: RF001 <- o_orderkey +| | mem-estimate=10.20MB mem-reservation=136.00MB +| | tuple-ids=3,4 row-size=103B cardinality=1151542 +| | +| |--15:EXCHANGE [BROADCAST] +| | | mem-estimate=0B mem-reservation=0B +| | | tuple-ids=4 row-size=32B cardinality=300000 +| | | +| | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| | Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B +| | 06:SCAN HDFS [tpch_parquet.orders, RANDOM] +| | partitions=1/1 files=2 size=54.20MB +| | predicates: o_orderpriority = '2-HIGH' +| | stats-rows=1500000 extrapolated-rows=disabled +| | table stats: rows=1500000 size=54.20MB +| | column stats: all +| | parquet statistics predicates: o_orderpriority = '2-HIGH' +| | parquet dictionary predicates: o_orderpriority = '2-HIGH' +| | mem-estimate=40.00MB mem-reservation=0B +| | tuple-ids=4 row-size=32B cardinality=300000 +| | +| 05:SCAN HDFS [tpch_parquet.lineitem, RANDOM] +| partitions=1/1 files=3 size=193.92MB +| runtime filters: RF001 -> l_orderkey +| stats-rows=6001215 extrapolated-rows=disabled +| table stats: rows=6001215 size=193.92MB +| column stats: all +| mem-estimate=80.00MB mem-reservation=0B +| tuple-ids=3 row-size=70B cardinality=6001215 +| +14:AGGREGATE [FINALIZE] +| group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment +| mem-estimate=42.58MB mem-reservation=264.00MB +| tuple-ids=2 row-size=70B cardinality=575772 +| +13:EXCHANGE [HASH(l_orderkey,l_partkey,l_suppkey,l_linenumber,l_comment)] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=2 row-size=70B cardinality=575772 +| +F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 +Per-Host Resources: mem-estimate=46.78MB mem-reservation=136.00MB +04:AGGREGATE [STREAMING] +| group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment +| mem-estimate=42.58MB mem-reservation=0B +| tuple-ids=2 row-size=70B cardinality=575772 +| +03:HASH JOIN [INNER JOIN, PARTITIONED] +| hash predicates: l_orderkey = o_orderkey +| fk/pk conjuncts: l_orderkey = o_orderkey +| runtime filters: RF000 <- o_orderkey +| mem-estimate=4.20MB mem-reservation=136.00MB +| tuple-ids=0,1 row-size=86B cardinality=575772 +| +|--12:EXCHANGE [HASH(o_orderkey)] +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=1 row-size=8B cardinality=1500000 +| | +| F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B +| 02:SCAN HDFS [tpch_parquet.orders, RANDOM] +| partitions=1/1 files=2 size=54.20MB +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=54.20MB +| column stats: all +| mem-estimate=40.00MB mem-reservation=0B +| tuple-ids=1 row-size=8B cardinality=1500000 +| +11:EXCHANGE [HASH(l_orderkey)] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=0 row-size=78B cardinality=600122 +| +F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B +01:SCAN HDFS [tpch_parquet.lineitem, RANDOM] + partitions=1/1 files=3 size=193.92MB + predicates: l_tax > 10 + runtime filters: RF000 -> l_orderkey + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=193.92MB + column stats: all + parquet statistics predicates: l_tax > 10 + parquet dictionary predicates: l_tax > 10 + mem-estimate=80.00MB mem-reservation=0B + tuple-ids=0 row-size=78B cardinality=600122 +---- PARALLELPLANS +Per-Host Resource Reservation: Memory=800.00MB +Per-Host Resource Estimates: Memory=674.53MB + +F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +17:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=7 row-size=70B cardinality=2549844 +| +F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=185.18MB mem-reservation=528.00MB +00:UNION +| pass-through-operands: 14 +| mem-estimate=0B mem-reservation=0B +| tuple-ids=7 row-size=70B cardinality=2549844 +| +|--10:HASH JOIN [INNER JOIN, BROADCAST] +| | hash-table-id=01 +| | hash predicates: l_orderkey = o_orderkey +| | fk/pk conjuncts: l_orderkey = o_orderkey +| | runtime filters: RF002 <- o_orderkey +| | mem-estimate=12.59MB mem-reservation=136.00MB +| | tuple-ids=5,6 row-size=99B cardinality=822530 +| | +| |--F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| | | Per-Host Resources: included in parent fragment +| | JOIN BUILD +| | | join-table-id=01 plan-id=02 cohort-id=01 +| | | build expressions: o_orderkey +| | | mem-estimate=0B mem-reservation=0B +| | | +| | 16:EXCHANGE [BROADCAST] +| | | mem-estimate=0B mem-reservation=0B +| | | tuple-ids=6 row-size=8B cardinality=1500000 +| | | +| | F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| | Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B +| | 09:SCAN HDFS [tpch_parquet.orders, RANDOM] +| | partitions=1/1 files=2 size=54.20MB +| | stats-rows=1500000 extrapolated-rows=disabled +| | table stats: rows=1500000 size=54.20MB +| | column stats: all +| | mem-estimate=40.00MB mem-reservation=0B +| | tuple-ids=6 row-size=8B cardinality=1500000 +| | +| 08:SCAN HDFS [tpch_parquet.lineitem, RANDOM] +| partitions=1/1 files=3 size=193.92MB +| predicates: l_shipmode = 'F' +| runtime filters: RF002 -> l_orderkey +| stats-rows=6001215 extrapolated-rows=disabled +| table stats: rows=6001215 size=193.92MB +| column stats: all +| parquet statistics predicates: l_shipmode = 'F' +| parquet dictionary predicates: l_shipmode = 'F' +| mem-estimate=80.00MB mem-reservation=0B +| tuple-ids=5 row-size=91B cardinality=857316 +| +|--07:HASH JOIN [INNER JOIN, BROADCAST] +| | hash-table-id=00 +| | hash predicates: l_orderkey = o_orderkey +| | fk/pk conjuncts: l_orderkey = o_orderkey +| | runtime filters: RF001 <- o_orderkey +| | mem-estimate=10.20MB mem-reservation=136.00MB +| | tuple-ids=3,4 row-size=103B cardinality=1151542 +| | +| |--F10:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| | | Per-Host Resources: included in parent fragment +| | JOIN BUILD +| | | join-table-id=00 plan-id=01 cohort-id=01 +| | | build expressions: o_orderkey +| | | mem-estimate=0B mem-reservation=0B +| | | +| | 15:EXCHANGE [BROADCAST] +| | | mem-estimate=0B mem-reservation=0B +| | | tuple-ids=4 row-size=32B cardinality=300000 +| | | +| | F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| | Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B +| | 06:SCAN HDFS [tpch_parquet.orders, RANDOM] +| | partitions=1/1 files=2 size=54.20MB +| | predicates: o_orderpriority = '2-HIGH' +| | stats-rows=1500000 extrapolated-rows=disabled +| | table stats: rows=1500000 size=54.20MB +| | column stats: all +| | parquet statistics predicates: o_orderpriority = '2-HIGH' +| | parquet dictionary predicates: o_orderpriority = '2-HIGH' +| | mem-estimate=40.00MB mem-reservation=0B +| | tuple-ids=4 row-size=32B cardinality=300000 +| | +| 05:SCAN HDFS [tpch_parquet.lineitem, RANDOM] +| partitions=1/1 files=3 size=193.92MB +| runtime filters: RF001 -> l_orderkey +| stats-rows=6001215 extrapolated-rows=disabled +| table stats: rows=6001215 size=193.92MB +| column stats: all +| mem-estimate=80.00MB mem-reservation=0B +| tuple-ids=3 row-size=70B cardinality=6001215 +| +14:AGGREGATE [FINALIZE] +| group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment +| mem-estimate=42.58MB mem-reservation=264.00MB +| tuple-ids=2 row-size=70B cardinality=575772 +| +13:EXCHANGE [HASH(l_orderkey,l_partkey,l_suppkey,l_linenumber,l_comment)] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=2 row-size=70B cardinality=575772 +| +F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 +Per-Host Resources: mem-estimate=89.35MB mem-reservation=272.00MB +04:AGGREGATE [STREAMING] +| group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment +| mem-estimate=42.58MB mem-reservation=0B +| tuple-ids=2 row-size=70B cardinality=575772 +| +03:HASH JOIN [INNER JOIN, PARTITIONED] +| hash-table-id=02 +| hash predicates: l_orderkey = o_orderkey +| fk/pk conjuncts: l_orderkey = o_orderkey +| runtime filters: RF000 <- o_orderkey +| mem-estimate=2.10MB mem-reservation=136.00MB +| tuple-ids=0,1 row-size=86B cardinality=575772 +| +|--F12:PLAN FRAGMENT [HASH(l_orderkey)] hosts=2 instances=4 +| | Per-Host Resources: included in parent fragment +| JOIN BUILD +| | join-table-id=02 plan-id=03 cohort-id=01 +| | build expressions: o_orderkey +| | mem-estimate=0B mem-reservation=0B +| | +| 12:EXCHANGE [HASH(o_orderkey)] +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=1 row-size=8B cardinality=1500000 +| | +| F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B +| 02:SCAN HDFS [tpch_parquet.orders, RANDOM] +| partitions=1/1 files=2 size=54.20MB +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=54.20MB +| column stats: all +| mem-estimate=40.00MB mem-reservation=0B +| tuple-ids=1 row-size=8B cardinality=1500000 +| +11:EXCHANGE [HASH(l_orderkey)] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=0 row-size=78B cardinality=600122 +| +F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=160.00MB mem-reservation=0B +01:SCAN HDFS [tpch_parquet.lineitem, RANDOM] + partitions=1/1 files=3 size=193.92MB + predicates: l_tax > 10 + runtime filters: RF000 -> l_orderkey + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=193.92MB + column stats: all + parquet statistics predicates: l_tax > 10 + parquet dictionary predicates: l_tax > 10 + mem-estimate=80.00MB mem-reservation=0B + tuple-ids=0 row-size=78B cardinality=600122 +==== # TPC-H Q18: Pipeline of joins with some non-trivial subtrees on right side of joins select c_name, @@ -813,11 +1887,92 @@ order by o_totalprice desc, o_orderdate limit 100 ----- DISTRIBUTEDPLAN +---- PLAN Per-Host Resource Reservation: Memory=672.00MB -Per-Host Resource Estimates: Memory=242.95MB +Per-Host Resource Estimates: Memory=391.29MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=391.29MB mem-reservation=672.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +09:TOP-N [LIMIT=100] +| order by: o_totalprice DESC, o_orderdate ASC +| mem-estimate=9.77KB mem-reservation=0B +| tuple-ids=7 row-size=100B cardinality=100 +| +08:AGGREGATE [FINALIZE] +| output: sum(l_quantity) +| group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice +| mem-estimate=60.40MB mem-reservation=264.00MB +| tuple-ids=6 row-size=100B cardinality=575772 +| +07:HASH JOIN [LEFT SEMI JOIN] +| hash predicates: o_orderkey = l_orderkey +| runtime filters: RF000 <- l_orderkey +| mem-estimate=3.94MB mem-reservation=136.00MB +| tuple-ids=2,1,0 row-size=108B cardinality=575772 +| +|--04:AGGREGATE [FINALIZE] +| | output: sum(l_quantity) +| | group by: l_orderkey +| | having: sum(l_quantity) > 300 +| | mem-estimate=10.00MB mem-reservation=264.00MB +| | tuple-ids=4 row-size=24B cardinality=156344 +| | +| 03:SCAN HDFS [tpch.lineitem] +| partitions=1/1 files=1 size=718.94MB +| stats-rows=6001215 extrapolated-rows=disabled +| table stats: rows=6001215 size=718.94MB +| column stats: all +| mem-estimate=88.00MB mem-reservation=0B +| tuple-ids=3 row-size=16B cardinality=6001215 +| +06:HASH JOIN [INNER JOIN] +| hash predicates: o_custkey = c_custkey +| fk/pk conjuncts: o_custkey = c_custkey +| runtime filters: RF001 <- c_custkey +| mem-estimate=6.61MB mem-reservation=136.00MB +| tuple-ids=2,1,0 row-size=108B cardinality=5757710 +| +|--00:SCAN HDFS [tpch.customer] +| partitions=1/1 files=1 size=23.08MB +| stats-rows=150000 extrapolated-rows=disabled +| table stats: rows=150000 size=23.08MB +| column stats: all +| mem-estimate=32.00MB mem-reservation=0B +| tuple-ids=0 row-size=42B cardinality=150000 +| +05:HASH JOIN [INNER JOIN] +| hash predicates: l_orderkey = o_orderkey +| fk/pk conjuncts: l_orderkey = o_orderkey +| runtime filters: RF002 <- o_orderkey +| mem-estimate=78.68MB mem-reservation=136.00MB +| tuple-ids=2,1 row-size=66B cardinality=5757710 +| +|--01:SCAN HDFS [tpch.orders] +| partitions=1/1 files=1 size=162.56MB +| runtime filters: RF000 -> o_orderkey, RF001 -> o_custkey +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=162.56MB +| column stats: all +| mem-estimate=88.00MB mem-reservation=0B +| tuple-ids=1 row-size=50B cardinality=1500000 +| +02:SCAN HDFS [tpch.lineitem] + partitions=1/1 files=1 size=718.94MB + runtime filters: RF000 -> tpch.lineitem.l_orderkey, RF002 -> l_orderkey + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=718.94MB + column stats: all + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=2 row-size=16B cardinality=6001215 +---- DISTRIBUTEDPLAN +Per-Host Resource Reservation: Memory=936.00MB +Per-Host Resource Estimates: Memory=500.32MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -828,6 +1983,7 @@ PLAN-ROOT SINK | tuple-ids=7 row-size=100B cardinality=100 | F06:PLAN FRAGMENT [HASH(c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice)] hosts=3 instances=3 +Per-Host Resources: mem-estimate=60.41MB mem-reservation=264.00MB 09:TOP-N [LIMIT=100] | order by: o_totalprice DESC, o_orderdate ASC | mem-estimate=9.77KB mem-reservation=0B @@ -844,6 +2000,7 @@ F06:PLAN FRAGMENT [HASH(c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice)] h | tuple-ids=6 row-size=100B cardinality=575772 | F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 +Per-Host Resources: mem-estimate=104.55MB mem-reservation=672.00MB 08:AGGREGATE [STREAMING] | output: sum(l_quantity) | group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice @@ -868,6 +2025,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 | | tuple-ids=4 row-size=24B cardinality=1563438 | | | F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +| Per-Host Resources: mem-estimate=127.36MB mem-reservation=0B | 04:AGGREGATE [STREAMING] | | output: sum(l_quantity) | | group by: l_orderkey @@ -894,6 +2052,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 | | tuple-ids=0 row-size=42B cardinality=150000 | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B | 00:SCAN HDFS [tpch.customer, RANDOM] | partitions=1/1 files=1 size=23.08MB | stats-rows=150000 extrapolated-rows=disabled @@ -914,6 +2073,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 | | tuple-ids=1 row-size=50B cardinality=1500000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B | 01:SCAN HDFS [tpch.orders, RANDOM] | partitions=1/1 files=1 size=162.56MB | runtime filters: RF000 -> o_orderkey, RF001 -> o_custkey @@ -928,6 +2088,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 | tuple-ids=2 row-size=16B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B 02:SCAN HDFS [tpch.lineitem, RANDOM] partitions=1/1 files=1 size=718.94MB runtime filters: RF000 -> tpch.lineitem.l_orderkey, RF002 -> l_orderkey @@ -937,10 +2098,11 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 mem-estimate=88.00MB mem-reservation=0B tuple-ids=2 row-size=16B cardinality=6001215 ---- PARALLELPLANS -Per-Host Resource Reservation: Memory=1.83GB -Per-Host Resource Estimates: Memory=973.08MB +Per-Host Resource Reservation: Memory=1.31GB +Per-Host Resource Estimates: Memory=953.10MB F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -951,6 +2113,7 @@ PLAN-ROOT SINK | tuple-ids=7 row-size=100B cardinality=100 | F06:PLAN FRAGMENT [HASH(c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice)] hosts=3 instances=6 +Per-Host Resources: mem-estimate=120.82MB mem-reservation=528.00MB 09:TOP-N [LIMIT=100] | order by: o_totalprice DESC, o_orderdate ASC | mem-estimate=9.77KB mem-reservation=0B @@ -967,6 +2130,7 @@ F06:PLAN FRAGMENT [HASH(c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice)] h | tuple-ids=6 row-size=100B cardinality=575772 | F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 +Per-Host Resources: mem-estimate=161.56MB mem-reservation=816.00MB 08:AGGREGATE [STREAMING] | output: sum(l_quantity) | group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice @@ -981,6 +2145,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 | tuple-ids=2,1,0 row-size=108B cardinality=575772 | |--F08:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 +| | Per-Host Resources: included in parent fragment | JOIN BUILD | | join-table-id=00 plan-id=01 cohort-id=01 | | build expressions: l_orderkey @@ -998,6 +2163,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 | | tuple-ids=4 row-size=24B cardinality=1563438 | | | F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +| Per-Host Resources: mem-estimate=254.73MB mem-reservation=0B | 04:AGGREGATE [STREAMING] | | output: sum(l_quantity) | | group by: l_orderkey @@ -1021,6 +2187,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 | tuple-ids=2,1,0 row-size=108B cardinality=5757710 | |--F09:PLAN FRAGMENT [HASH(l_orderkey)] hosts=1 instances=2 +| | Per-Host Resources: included in parent fragment | JOIN BUILD | | join-table-id=01 plan-id=02 cohort-id=01 | | build expressions: c_custkey @@ -1031,6 +2198,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 | | tuple-ids=0 row-size=42B cardinality=150000 | | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| Per-Host Resources: mem-estimate=64.00MB mem-reservation=0B | 00:SCAN HDFS [tpch.customer, RANDOM] | partitions=1/1 files=1 size=23.08MB | stats-rows=150000 extrapolated-rows=disabled @@ -1048,6 +2216,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 | tuple-ids=2,1 row-size=66B cardinality=5757710 | |--F10:PLAN FRAGMENT [HASH(l_orderkey)] hosts=2 instances=4 +| | Per-Host Resources: included in parent fragment | JOIN BUILD | | join-table-id=02 plan-id=03 cohort-id=01 | | build expressions: o_orderkey @@ -1058,6 +2227,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 | | tuple-ids=1 row-size=50B cardinality=1500000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B | 01:SCAN HDFS [tpch.orders, RANDOM] | partitions=1/1 files=1 size=162.56MB | runtime filters: RF000 -> o_orderkey, RF001 -> o_custkey @@ -1072,6 +2242,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 | tuple-ids=2 row-size=16B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B 02:SCAN HDFS [tpch.lineitem, RANDOM] partitions=1/1 files=1 size=718.94MB runtime filters: RF000 -> tpch.lineitem.l_orderkey, RF002 -> l_orderkey @@ -1087,12 +2258,31 @@ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col from functional.alltypes where year=2009 and month=05 +---- PLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=16.03MB +Codegen disabled by planner + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=0B +WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false] +| partitions=1 +| mem-estimate=26.94KB mem-reservation=0B +| +00:SCAN HDFS [functional.alltypes] + partitions=1/24 files=1 size=20.36KB + stats-rows=310 extrapolated-rows=disabled + table stats: rows=7300 size=478.45KB + column stats: all + mem-estimate=16.00MB mem-reservation=0B + tuple-ids=0 row-size=89B cardinality=310 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=16.03MB Codegen disabled by planner F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=16.03MB mem-reservation=0B WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false] | partitions=1 | mem-estimate=26.94KB mem-reservation=0B @@ -1110,6 +2300,7 @@ Per-Host Resource Estimates: Memory=32.03MB Codegen disabled by planner F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| Per-Host Resources: mem-estimate=32.03MB mem-reservation=0B WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false] | partitions=1 | mem-estimate=13.47KB mem-reservation=0B @@ -1126,11 +2317,29 @@ WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false] create table dummy_insert partitioned by (l_partkey) as select l_comment, l_partkey from tpch.lineitem +---- PLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=376.99MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=376.99MB mem-reservation=0B +WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey)] +| partitions=200516 +| mem-estimate=288.99MB mem-reservation=0B +| +00:SCAN HDFS [tpch.lineitem] + partitions=1/1 files=1 size=718.94MB + stats-rows=6001215 extrapolated-rows=disabled + table stats: rows=6001215 size=718.94MB + column stats: all + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=50B cardinality=6001215 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=184.33MB F01:PLAN FRAGMENT [HASH(l_partkey)] hosts=3 instances=3 +| Per-Host Resources: mem-estimate=96.33MB mem-reservation=0B WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey)] | partitions=200516 | mem-estimate=96.33MB mem-reservation=0B @@ -1140,6 +2349,7 @@ WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey) | tuple-ids=0 row-size=50B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B 00:SCAN HDFS [tpch.lineitem, RANDOM] partitions=1/1 files=1 size=718.94MB stats-rows=6001215 extrapolated-rows=disabled @@ -1152,6 +2362,7 @@ Per-Host Resource Reservation: Memory=0B Per-Host Resource Estimates: Memory=272.33MB F01:PLAN FRAGMENT [HASH(l_partkey)] hosts=3 instances=6 +| Per-Host Resources: mem-estimate=96.33MB mem-reservation=0B WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey)] | partitions=200516 | mem-estimate=48.16MB mem-reservation=0B @@ -1161,6 +2372,7 @@ WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey) | tuple-ids=0 row-size=50B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B 00:SCAN HDFS [tpch.lineitem, RANDOM] partitions=1/1 files=1 size=718.94MB stats-rows=6001215 extrapolated-rows=disabled @@ -1169,3 +2381,1055 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 mem-estimate=88.00MB mem-reservation=0B tuple-ids=0 row-size=50B cardinality=6001215 ==== +# Variety of operators in and outside of subplan. All operators in subplan consume +# resources while the subplan is open. +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 + join c.c_orders o2 on o1.o_orderkey = o2.o_orderkey + order by o1.o_orderkey limit 100) v +---- PLAN +Per-Host Resource Reservation: Memory=664.00MB +Per-Host Resource Estimates: Memory=344.00MB +WARNING: The following tables are missing relevant table and/or column statistics. +tpch_nested_parquet.customer + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=344.00MB mem-reservation=664.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +09:AGGREGATE [FINALIZE] +| group by: c_name, o1.o_orderkey, o2.o_orderstatus +| mem-estimate=128.00MB mem-reservation=264.00MB +| tuple-ids=6 row-size=58B cardinality=1500000 +| +01:SUBPLAN +| mem-estimate=0B mem-reservation=0B +| tuple-ids=4,0 row-size=90B cardinality=1500000 +| +|--08:NESTED LOOP JOIN [CROSS JOIN] +| | mem-estimate=66B mem-reservation=0B +| | tuple-ids=4,0 row-size=90B cardinality=10 +| | +| |--02:SINGULAR ROW SRC +| | parent-subplan=01 +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=0 row-size=66B cardinality=1 +| | +| 07:TOP-N [LIMIT=100] +| | order by: o1.o_orderkey ASC +| | mem-estimate=240B mem-reservation=0B +| | tuple-ids=4 row-size=24B cardinality=10 +| | +| 06:AGGREGATE [FINALIZE] +| | group by: o1.o_orderkey, o2.o_orderstatus +| | mem-estimate=128.00MB mem-reservation=264.00MB +| | tuple-ids=3 row-size=24B cardinality=10 +| | +| 05:HASH JOIN [INNER JOIN] +| | hash predicates: o1.o_orderkey = o2.o_orderkey +| | fk/pk conjuncts: assumed fk/pk +| | mem-estimate=0B mem-reservation=136.00MB +| | tuple-ids=1,2 row-size=32B cardinality=10 +| | +| |--04:UNNEST [c.c_orders o2] +| | parent-subplan=01 +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=2 row-size=0B cardinality=10 +| | +| 03:UNNEST [c.c_orders o1] +| parent-subplan=01 +| mem-estimate=0B mem-reservation=0B +| tuple-ids=1 row-size=0B cardinality=10 +| +00:SCAN HDFS [tpch_nested_parquet.customer c] + partitions=1/1 files=4 size=292.35MB + stats-rows=150000 extrapolated-rows=disabled + table stats: rows=150000 size=292.35MB + columns missing stats: c_orders, c_orders + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=66B cardinality=150000 +---- DISTRIBUTEDPLAN +Per-Host Resource Reservation: Memory=664.00MB +Per-Host Resource Estimates: Memory=472.00MB +WARNING: The following tables are missing relevant table and/or column statistics. +tpch_nested_parquet.customer + +F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +12:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=6 row-size=58B cardinality=1500000 +| +F01:PLAN FRAGMENT [HASH(c_name,v.o_orderkey,v.o_orderstatus)] hosts=3 instances=3 +Per-Host Resources: mem-estimate=128.00MB mem-reservation=264.00MB +11:AGGREGATE [FINALIZE] +| group by: c_name, v.o_orderkey, v.o_orderstatus +| mem-estimate=128.00MB mem-reservation=264.00MB +| tuple-ids=6 row-size=58B cardinality=1500000 +| +10:EXCHANGE [HASH(c_name,v.o_orderkey,v.o_orderstatus)] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=6 row-size=58B cardinality=1500000 +| +F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=344.00MB mem-reservation=400.00MB +09:AGGREGATE [STREAMING] +| group by: c_name, o1.o_orderkey, o2.o_orderstatus +| mem-estimate=128.00MB mem-reservation=0B +| tuple-ids=6 row-size=58B cardinality=1500000 +| +01:SUBPLAN +| mem-estimate=0B mem-reservation=0B +| tuple-ids=4,0 row-size=90B cardinality=1500000 +| +|--08:NESTED LOOP JOIN [CROSS JOIN] +| | mem-estimate=66B mem-reservation=0B +| | tuple-ids=4,0 row-size=90B cardinality=10 +| | +| |--02:SINGULAR ROW SRC +| | parent-subplan=01 +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=0 row-size=66B cardinality=1 +| | +| 07:TOP-N [LIMIT=100] +| | order by: o1.o_orderkey ASC +| | mem-estimate=240B mem-reservation=0B +| | tuple-ids=4 row-size=24B cardinality=10 +| | +| 06:AGGREGATE [FINALIZE] +| | group by: o1.o_orderkey, o2.o_orderstatus +| | mem-estimate=128.00MB mem-reservation=264.00MB +| | tuple-ids=3 row-size=24B cardinality=10 +| | +| 05:HASH JOIN [INNER JOIN] +| | hash predicates: o1.o_orderkey = o2.o_orderkey +| | fk/pk conjuncts: assumed fk/pk +| | mem-estimate=0B mem-reservation=136.00MB +| | tuple-ids=1,2 row-size=32B cardinality=10 +| | +| |--04:UNNEST [c.c_orders o2] +| | parent-subplan=01 +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=2 row-size=0B cardinality=10 +| | +| 03:UNNEST [c.c_orders o1] +| parent-subplan=01 +| mem-estimate=0B mem-reservation=0B +| tuple-ids=1 row-size=0B cardinality=10 +| +00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM] + partitions=1/1 files=4 size=292.35MB + stats-rows=150000 extrapolated-rows=disabled + table stats: rows=150000 size=292.35MB + columns missing stats: c_orders, c_orders + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=66B cardinality=150000 +---- PARALLELPLANS +Per-Host Resource Reservation: Memory=1.30GB +Per-Host Resource Estimates: Memory=944.00MB +WARNING: The following tables are missing relevant table and/or column statistics. +tpch_nested_parquet.customer + +F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +12:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=6 row-size=58B cardinality=1500000 +| +F01:PLAN FRAGMENT [HASH(c_name,v.o_orderkey,v.o_orderstatus)] hosts=3 instances=6 +Per-Host Resources: mem-estimate=256.00MB mem-reservation=528.00MB +11:AGGREGATE [FINALIZE] +| group by: c_name, v.o_orderkey, v.o_orderstatus +| mem-estimate=128.00MB mem-reservation=264.00MB +| tuple-ids=6 row-size=58B cardinality=1500000 +| +10:EXCHANGE [HASH(c_name,v.o_orderkey,v.o_orderstatus)] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=6 row-size=58B cardinality=1500000 +| +F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=688.00MB mem-reservation=800.00MB +09:AGGREGATE [STREAMING] +| group by: c_name, o1.o_orderkey, o2.o_orderstatus +| mem-estimate=128.00MB mem-reservation=0B +| tuple-ids=6 row-size=58B cardinality=1500000 +| +01:SUBPLAN +| mem-estimate=0B mem-reservation=0B +| tuple-ids=4,0 row-size=90B cardinality=1500000 +| +|--08:NESTED LOOP JOIN [CROSS JOIN] +| | mem-estimate=66B mem-reservation=0B +| | tuple-ids=4,0 row-size=90B cardinality=10 +| | +| |--02:SINGULAR ROW SRC +| | parent-subplan=01 +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=0 row-size=66B cardinality=1 +| | +| 07:TOP-N [LIMIT=100] +| | order by: o1.o_orderkey ASC +| | mem-estimate=240B mem-reservation=0B +| | tuple-ids=4 row-size=24B cardinality=10 +| | +| 06:AGGREGATE [FINALIZE] +| | group by: o1.o_orderkey, o2.o_orderstatus +| | mem-estimate=128.00MB mem-reservation=264.00MB +| | tuple-ids=3 row-size=24B cardinality=10 +| | +| 05:HASH JOIN [INNER JOIN] +| | hash predicates: o1.o_orderkey = o2.o_orderkey +| | fk/pk conjuncts: assumed fk/pk +| | mem-estimate=0B mem-reservation=136.00MB +| | tuple-ids=1,2 row-size=32B cardinality=10 +| | +| |--04:UNNEST [c.c_orders o2] +| | parent-subplan=01 +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=2 row-size=0B cardinality=10 +| | +| 03:UNNEST [c.c_orders o1] +| parent-subplan=01 +| mem-estimate=0B mem-reservation=0B +| tuple-ids=1 row-size=0B cardinality=10 +| +00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM] + partitions=1/1 files=4 size=292.35MB + stats-rows=150000 extrapolated-rows=disabled + table stats: rows=150000 size=292.35MB + columns missing stats: c_orders, c_orders + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=66B cardinality=150000 +==== +# Pipeline of blocking operators in a subplan. All operators consume resources at the +# same time (unlike blocking operators outside of a subplan). +select * +from tpch_nested_parquet.customer c, + (select *, row_number() over (order by o_totalprice) rnum_price, + row_number() over (order by o_orderdate) rnum_date, + row_number() over (order by o_orderpriority) rnum_priority + from c.c_orders) v; +---- PLAN +Per-Host Resource Reservation: Memory=192.00MB +Per-Host Resource Estimates: Memory=136.00MB +WARNING: The following tables are missing relevant table and/or column statistics. +tpch_nested_parquet.customer + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=136.00MB mem-reservation=192.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +01:SUBPLAN +| mem-estimate=0B mem-reservation=0B +| tuple-ids=12,7,0 row-size=418B cardinality=1500000 +| +|--10:NESTED LOOP JOIN [CROSS JOIN] +| | mem-estimate=254B mem-reservation=0B +| | tuple-ids=12,7,0 row-size=418B cardinality=10 +| | +| |--02:SINGULAR ROW SRC +| | parent-subplan=01 +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=0 row-size=254B cardinality=1 +| | +| 09:ANALYTIC +| | functions: row_number() +| | order by: o_orderpriority ASC +| | window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| | mem-estimate=0B mem-reservation=16.00MB +| | tuple-ids=12,7 row-size=164B cardinality=10 +| | +| 08:SORT +| | order by: o_orderpriority ASC +| | mem-estimate=16.00MB mem-reservation=48.00MB +| | tuple-ids=12 row-size=156B cardinality=10 +| | +| 07:ANALYTIC +| | functions: row_number() +| | order by: o_orderdate ASC +| | window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| | mem-estimate=0B mem-reservation=16.00MB +| | tuple-ids=10,6 row-size=156B cardinality=10 +| | +| 06:SORT +| | order by: o_orderdate ASC +| | mem-estimate=16.00MB mem-reservation=48.00MB +| | tuple-ids=10 row-size=148B cardinality=10 +| | +| 05:ANALYTIC +| | functions: row_number() +| | order by: o_totalprice ASC +| | window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| | mem-estimate=0B mem-reservation=16.00MB +| | tuple-ids=8,5 row-size=148B cardinality=10 +| | +| 04:SORT +| | order by: o_totalprice ASC +| | mem-estimate=16.00MB mem-reservation=48.00MB +| | tuple-ids=8 row-size=140B cardinality=10 +| | +| 03:UNNEST [c.c_orders] +| parent-subplan=01 +| mem-estimate=0B mem-reservation=0B +| tuple-ids=1 row-size=0B cardinality=10 +| +00:SCAN HDFS [tpch_nested_parquet.customer c] + partitions=1/1 files=4 size=292.35MB + stats-rows=150000 extrapolated-rows=disabled + table stats: rows=150000 size=292.35MB + columns missing stats: c_orders + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=254B cardinality=150000 +---- DISTRIBUTEDPLAN +Per-Host Resource Reservation: Memory=192.00MB +Per-Host Resource Estimates: Memory=136.00MB +WARNING: The following tables are missing relevant table and/or column statistics. +tpch_nested_parquet.customer + +F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +11:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=12,7,0 row-size=418B cardinality=1500000 +| +F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=136.00MB mem-reservation=192.00MB +01:SUBPLAN +| mem-estimate=0B mem-reservation=0B +| tuple-ids=12,7,0 row-size=418B cardinality=1500000 +| +|--10:NESTED LOOP JOIN [CROSS JOIN] +| | mem-estimate=254B mem-reservation=0B +| | tuple-ids=12,7,0 row-size=418B cardinality=10 +| | +| |--02:SINGULAR ROW SRC +| | parent-subplan=01 +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=0 row-size=254B cardinality=1 +| | +| 09:ANALYTIC +| | functions: row_number() +| | order by: o_orderpriority ASC +| | window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| | mem-estimate=0B mem-reservation=16.00MB +| | tuple-ids=12,7 row-size=164B cardinality=10 +| | +| 08:SORT +| | order by: o_orderpriority ASC +| | mem-estimate=16.00MB mem-reservation=48.00MB +| | tuple-ids=12 row-size=156B cardinality=10 +| | +| 07:ANALYTIC +| | functions: row_number() +| | order by: o_orderdate ASC +| | window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| | mem-estimate=0B mem-reservation=16.00MB +| | tuple-ids=10,6 row-size=156B cardinality=10 +| | +| 06:SORT +| | order by: o_orderdate ASC +| | mem-estimate=16.00MB mem-reservation=48.00MB +| | tuple-ids=10 row-size=148B cardinality=10 +| | +| 05:ANALYTIC +| | functions: row_number() +| | order by: o_totalprice ASC +| | window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| | mem-estimate=0B mem-reservation=16.00MB +| | tuple-ids=8,5 row-size=148B cardinality=10 +| | +| 04:SORT +| | order by: o_totalprice ASC +| | mem-estimate=16.00MB mem-reservation=48.00MB +| | tuple-ids=8 row-size=140B cardinality=10 +| | +| 03:UNNEST [c.c_orders] +| parent-subplan=01 +| mem-estimate=0B mem-reservation=0B +| tuple-ids=1 row-size=0B cardinality=10 +| +00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM] + partitions=1/1 files=4 size=292.35MB + stats-rows=150000 extrapolated-rows=disabled + table stats: rows=150000 size=292.35MB + columns missing stats: c_orders + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=254B cardinality=150000 +---- PARALLELPLANS +Per-Host Resource Reservation: Memory=384.00MB +Per-Host Resource Estimates: Memory=272.00MB +WARNING: The following tables are missing relevant table and/or column statistics. +tpch_nested_parquet.customer + +F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +11:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=12,7,0 row-size=418B cardinality=1500000 +| +F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=272.00MB mem-reservation=384.00MB +01:SUBPLAN +| mem-estimate=0B mem-reservation=0B +| tuple-ids=12,7,0 row-size=418B cardinality=1500000 +| +|--10:NESTED LOOP JOIN [CROSS JOIN] +| | mem-estimate=254B mem-reservation=0B +| | tuple-ids=12,7,0 row-size=418B cardinality=10 +| | +| |--02:SINGULAR ROW SRC +| | parent-subplan=01 +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=0 row-size=254B cardinality=1 +| | +| 09:ANALYTIC +| | functions: row_number() +| | order by: o_orderpriority ASC +| | window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| | mem-estimate=0B mem-reservation=16.00MB +| | tuple-ids=12,7 row-size=164B cardinality=10 +| | +| 08:SORT +| | order by: o_orderpriority ASC +| | mem-estimate=16.00MB mem-reservation=48.00MB +| | tuple-ids=12 row-size=156B cardinality=10 +| | +| 07:ANALYTIC +| | functions: row_number() +| | order by: o_orderdate ASC +| | window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| | mem-estimate=0B mem-reservation=16.00MB +| | tuple-ids=10,6 row-size=156B cardinality=10 +| | +| 06:SORT +| | order by: o_orderdate ASC +| | mem-estimate=16.00MB mem-reservation=48.00MB +| | tuple-ids=10 row-size=148B cardinality=10 +| | +| 05:ANALYTIC +| | functions: row_number() +| | order by: o_totalprice ASC +| | window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| | mem-estimate=0B mem-reservation=16.00MB +| | tuple-ids=8,5 row-size=148B cardinality=10 +| | +| 04:SORT +| | order by: o_totalprice ASC +| | mem-estimate=16.00MB mem-reservation=48.00MB +| | tuple-ids=8 row-size=140B cardinality=10 +| | +| 03:UNNEST [c.c_orders] +| parent-subplan=01 +| mem-estimate=0B mem-reservation=0B +| tuple-ids=1 row-size=0B cardinality=10 +| +00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM] + partitions=1/1 files=4 size=292.35MB + stats-rows=150000 extrapolated-rows=disabled + table stats: rows=150000 size=292.35MB + columns missing stats: c_orders + mem-estimate=88.00MB mem-reservation=0B + tuple-ids=0 row-size=254B cardinality=150000 +==== +# Bushy hash join plan. Only two of the hash joins can execute concurrently, so we only +# need enough reservation for 2 hash joins. +select straight_join * +from tpch_parquet.orders t1 +join ( + select straight_join t2.o_orderkey k2, k3, k4 + from tpch_parquet.orders t2 + join ( + select straight_join t3.o_orderkey k3, t4.o_orderkey k4 + from tpch_parquet.orders t3 + join tpch_parquet.orders t4 on t3.o_orderkey = t4.o_orderkey + ) v2 on v2.k3 = t2.o_orderkey +) v1 on v1.k3 = t1.o_orderkey +---- PLAN +Per-Host Resource Reservation: Memory=272.00MB +Per-Host Resource Estimates: Memory=172.59MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=172.59MB mem-reservation=272.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +06:HASH JOIN [INNER JOIN] +| hash predicates: t1.o_orderkey = t3.o_orderkey +| fk/pk conjuncts: t1.o_orderkey = t3.o_orderkey +| runtime filters: RF000 <- t3.o_orderkey +| mem-estimate=37.77MB mem-reservation=136.00MB +| tuple-ids=0,1,2,3 row-size=215B cardinality=1500000 +| +|--05:HASH JOIN [INNER JOIN] +| | hash predicates: t2.o_orderkey = t3.o_orderkey +| | fk/pk conjuncts: t2.o_orderkey = t3.o_orderkey +| | runtime filters: RF001 <- t3.o_orderkey +| | mem-estimate=25.18MB mem-reservation=136.00MB +| | tuple-ids=1,2,3 row-size=24B cardinality=1500000 +| | +| |--04:HASH JOIN [INNER JOIN] +| | | hash predicates: t3.o_orderkey = t4.o_orderkey +| | | fk/pk conjuncts: t3.o_orderkey = t4.o_orderkey +| | | runtime filters: RF002 <- t4.o_orderkey +| | | mem-estimate=12.59MB mem-reservation=136.00MB +| | | tuple-ids=2,3 row-size=16B cardinality=1500000 +| | | +| | |--03:SCAN HDFS [tpch_parquet.orders t4] +| | | partitions=1/1 files=2 size=54.20MB +| | | stats-rows=1500000 extrapolated-rows=disabled +| | | table stats: rows=1500000 size=54.20MB +| | | column stats: all +| | | mem-estimate=40.00MB mem-reservation=0B +| | | tuple-ids=3 row-size=8B cardinality=1500000 +| | | +| | 02:SCAN HDFS [tpch_parquet.orders t3] +| | partitions=1/1 files=2 size=54.20MB +| | runtime filters: RF002 -> t3.o_orderkey +| | stats-rows=1500000 extrapolated-rows=disabled +| | table stats: rows=1500000 size=54.20MB +| | column stats: all +| | mem-estimate=40.00MB mem-reservation=0B +| | tuple-ids=2 row-size=8B cardinality=1500000 +| | +| 01:SCAN HDFS [tpch_parquet.orders t2] +| partitions=1/1 files=2 size=54.20MB +| runtime filters: RF001 -> t2.o_orderkey +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=54.20MB +| column stats: all +| mem-estimate=40.00MB mem-reservation=0B +| tuple-ids=1 row-size=8B cardinality=1500000 +| +00:SCAN HDFS [tpch_parquet.orders t1] + partitions=1/1 files=2 size=54.20MB + runtime filters: RF000 -> t1.o_orderkey + stats-rows=1500000 extrapolated-rows=disabled + table stats: rows=1500000 size=54.20MB + column stats: all + mem-estimate=40.00MB mem-reservation=0B + tuple-ids=0 row-size=191B cardinality=1500000 +---- DISTRIBUTEDPLAN +Per-Host Resource Reservation: Memory=408.00MB +Per-Host Resource Estimates: Memory=216.65MB + +F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +11:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=0,1,2,3 row-size=215B cardinality=1500000 +| +F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +Per-Host Resources: mem-estimate=77.77MB mem-reservation=136.00MB +06:HASH JOIN [INNER JOIN, BROADCAST] +| hash predicates: t1.o_orderkey = t3.o_orderkey +| fk/pk conjuncts: t1.o_orderkey = t3.o_orderkey +| runtime filters: RF000 <- t3.o_orderkey +| mem-estimate=37.77MB mem-reservation=136.00MB +| tuple-ids=0,1,2,3 row-size=215B cardinality=1500000 +| +|--10:EXCHANGE [BROADCAST] +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=1,2,3 row-size=24B cardinality=1500000 +| | +| F04:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=2 +| Per-Host Resources: mem-estimate=18.88MB mem-reservation=272.00MB +| 05:HASH JOIN [INNER JOIN, PARTITIONED] +| | hash predicates: t2.o_orderkey = t3.o_orderkey +| | fk/pk conjuncts: t2.o_orderkey = t3.o_orderkey +| | runtime filters: RF001 <- t3.o_orderkey +| | mem-estimate=12.59MB mem-reservation=136.00MB +| | tuple-ids=1,2,3 row-size=24B cardinality=1500000 +| | +| |--04:HASH JOIN [INNER JOIN, PARTITIONED] +| | | hash predicates: t3.o_orderkey = t4.o_orderkey +| | | fk/pk conjuncts: t3.o_orderkey = t4.o_orderkey +| | | runtime filters: RF002 <- t4.o_orderkey +| | | mem-estimate=6.29MB mem-reservation=136.00MB +| | | tuple-ids=2,3 row-size=16B cardinality=1500000 +| | | +| | |--08:EXCHANGE [HASH(t4.o_orderkey)] +| | | | mem-estimate=0B mem-reservation=0B +| | | | tuple-ids=3 row-size=8B cardinality=1500000 +| | | | +| | | F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| | | Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B +| | | 03:SCAN HDFS [tpch_parquet.orders t4, RANDOM] +| | | partitions=1/1 files=2 size=54.20MB +| | | stats-rows=1500000 extrapolated-rows=disabled +| | | table stats: rows=1500000 size=54.20MB +| | | column stats: all +| | | mem-estimate=40.00MB mem-reservation=0B +| | | tuple-ids=3 row-size=8B cardinality=1500000 +| | | +| | 07:EXCHANGE [HASH(t3.o_orderkey)] +| | | mem-estimate=0B mem-reservation=0B +| | | tuple-ids=2 row-size=8B cardinality=1500000 +| | | +| | F02:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| | Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B +| | 02:SCAN HDFS [tpch_parquet.orders t3, RANDOM] +| | partitions=1/1 files=2 size=54.20MB +| | runtime filters: RF002 -> t3.o_orderkey +| | stats-rows=1500000 extrapolated-rows=disabled +| | table stats: rows=1500000 size=54.20MB +| | column stats: all +| | mem-estimate=40.00MB mem-reservation=0B +| | tuple-ids=2 row-size=8B cardinality=1500000 +| | +| 09:EXCHANGE [HASH(t2.o_orderkey)] +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=1 row-size=8B cardinality=1500000 +| | +| F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B +| 01:SCAN HDFS [tpch_parquet.orders t2, RANDOM] +| partitions=1/1 files=2 size=54.20MB +| runtime filters: RF001 -> t2.o_orderkey +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=54.20MB +| column stats: all +| mem-estimate=40.00MB mem-reservation=0B +| tuple-ids=1 row-size=8B cardinality=1500000 +| +00:SCAN HDFS [tpch_parquet.orders t1, RANDOM] + partitions=1/1 files=2 size=54.20MB + runtime filters: RF000 -> t1.o_orderkey + stats-rows=1500000 extrapolated-rows=disabled + table stats: rows=1500000 size=54.20MB + column stats: all + mem-estimate=40.00MB mem-reservation=0B + tuple-ids=0 row-size=191B cardinality=1500000 +---- PARALLELPLANS +Per-Host Resource Reservation: Memory=816.00MB +Per-Host Resource Estimates: Memory=414.41MB + +F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +11:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=0,1,2,3 row-size=215B cardinality=1500000 +| +F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +Per-Host Resources: mem-estimate=155.53MB mem-reservation=272.00MB +06:HASH JOIN [INNER JOIN, BROADCAST] +| hash-table-id=00 +| hash predicates: t1.o_orderkey = t3.o_orderkey +| fk/pk conjuncts: t1.o_orderkey = t3.o_orderkey +| runtime filters: RF000 <- t3.o_orderkey +| mem-estimate=37.77MB mem-reservation=136.00MB +| tuple-ids=0,1,2,3 row-size=215B cardinality=1500000 +| +|--F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| | Per-Host Resources: included in parent fragment +| JOIN BUILD +| | join-table-id=00 plan-id=01 cohort-id=01 +| | build expressions: t3.o_orderkey +| | mem-estimate=0B mem-reservation=0B +| | +| 10:EXCHANGE [BROADCAST] +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=1,2,3 row-size=24B cardinality=1500000 +| | +| F04:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=4 +| Per-Host Resources: mem-estimate=18.88MB mem-reservation=544.00MB +| 05:HASH JOIN [INNER JOIN, PARTITIONED] +| | hash-table-id=01 +| | hash predicates: t2.o_orderkey = t3.o_orderkey +| | fk/pk conjuncts: t2.o_orderkey = t3.o_orderkey +| | runtime filters: RF001 <- t3.o_orderkey +| | mem-estimate=6.29MB mem-reservation=136.00MB +| | tuple-ids=1,2,3 row-size=24B cardinality=1500000 +| | +| |--F07:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=4 +| | | Per-Host Resources: included in parent fragment +| | JOIN BUILD +| | | join-table-id=01 plan-id=02 cohort-id=02 +| | | build expressions: t3.o_orderkey +| | | mem-estimate=0B mem-reservation=0B +| | | +| | 04:HASH JOIN [INNER JOIN, PARTITIONED] +| | | hash-table-id=02 +| | | hash predicates: t3.o_orderkey = t4.o_orderkey +| | | fk/pk conjuncts: t3.o_orderkey = t4.o_orderkey +| | | runtime filters: RF002 <- t4.o_orderkey +| | | mem-estimate=3.15MB mem-reservation=136.00MB +| | | tuple-ids=2,3 row-size=16B cardinality=1500000 +| | | +| | |--F08:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=4 +| | | | Per-Host Resources: included in parent fragment +| | | JOIN BUILD +| | | | join-table-id=02 plan-id=03 cohort-id=03 +| | | | build expressions: t4.o_orderkey +| | | | mem-estimate=0B mem-reservation=0B +| | | | +| | | 08:EXCHANGE [HASH(t4.o_orderkey)] +| | | | mem-estimate=0B mem-reservation=0B +| | | | tuple-ids=3 row-size=8B cardinality=1500000 +| | | | +| | | F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| | | Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B +| | | 03:SCAN HDFS [tpch_parquet.orders t4, RANDOM] +| | | partitions=1/1 files=2 size=54.20MB +| | | stats-rows=1500000 extrapolated-rows=disabled +| | | table stats: rows=1500000 size=54.20MB +| | | column stats: all +| | | mem-estimate=40.00MB mem-reservation=0B +| | | tuple-ids=3 row-size=8B cardinality=1500000 +| | | +| | 07:EXCHANGE [HASH(t3.o_orderkey)] +| | | mem-estimate=0B mem-reservation=0B +| | | tuple-ids=2 row-size=8B cardinality=1500000 +| | | +| | F02:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| | Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B +| | 02:SCAN HDFS [tpch_parquet.orders t3, RANDOM] +| | partitions=1/1 files=2 size=54.20MB +| | runtime filters: RF002 -> t3.o_orderkey +| | stats-rows=1500000 extrapolated-rows=disabled +| | table stats: rows=1500000 size=54.20MB +| | column stats: all +| | mem-estimate=40.00MB mem-reservation=0B +| | tuple-ids=2 row-size=8B cardinality=1500000 +| | +| 09:EXCHANGE [HASH(t2.o_orderkey)] +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=1 row-size=8B cardinality=1500000 +| | +| F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B +| 01:SCAN HDFS [tpch_parquet.orders t2, RANDOM] +| partitions=1/1 files=2 size=54.20MB +| runtime filters: RF001 -> t2.o_orderkey +| stats-rows=1500000 extrapolated-rows=disabled +| table stats: rows=1500000 size=54.20MB +| column stats: all +| mem-estimate=40.00MB mem-reservation=0B +| tuple-ids=1 row-size=8B cardinality=1500000 +| +00:SCAN HDFS [tpch_parquet.orders t1, RANDOM] + partitions=1/1 files=2 size=54.20MB + runtime filters: RF000 -> t1.o_orderkey + stats-rows=1500000 extrapolated-rows=disabled + table stats: rows=1500000 size=54.20MB + column stats: all + mem-estimate=40.00MB mem-reservation=0B + tuple-ids=0 row-size=191B cardinality=1500000 +==== +# Bushy nested loop join plan. Because of IMPALA-4179 the nested loop join cannot close +# its build-side child immediately after the build phase. Therefore the whole tree of +# joins remains open until the query finishes. +# TODO: IMPALA-4179: the estimates should get lower. +select straight_join * +from tpch_parquet.nation t1 +join ( + select straight_join t2.n_nationkey k2, k3, k4 + from tpch_parquet.nation t2 + join ( + select straight_join t3.n_nationkey k3, t4.s_suppkey k4 + from tpch_parquet.nation t3 + join tpch_parquet.supplier t4 + ) v2 +) v1 +---- PLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=137.99MB + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=137.99MB mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +06:NESTED LOOP JOIN [CROSS JOIN] +| mem-estimate=71.53MB mem-reservation=0B +| tuple-ids=0,1,2,3 row-size=129B cardinality=156250000 +| +|--05:NESTED LOOP JOIN [CROSS JOIN] +| | mem-estimate=2.38MB mem-reservation=0B +| | tuple-ids=1,2,3 row-size=12B cardinality=6250000 +| | +| |--04:NESTED LOOP JOIN [CROSS JOIN] +| | | mem-estimate=78.12KB mem-reservation=0B +| | | tuple-ids=2,3 row-size=10B cardinality=250000 +| | | +| | |--03:SCAN HDFS [tpch_parquet.supplier t4] +| | | partitions=1/1 files=1 size=882.76KB +| | | stats-rows=10000 extrapolated-rows=disabled +| | | table stats: rows=10000 size=882.76KB +| | | column stats: all +| | | mem-estimate=16.00MB mem-reservation=0B +| | | tuple-ids=3 row-size=8B cardinality=10000 +| | | +| | 02:SCAN HDFS [tpch_parquet.nation t3] +| | partitions=1/1 files=1 size=2.94KB +| | stats-rows=25 extrapolated-rows=disabled +| | table stats: rows=25 size=2.94KB +| | column stats: all +| | mem-estimate=16.00MB mem-reservation=0B +| | tuple-ids=2 row-size=2B cardinality=25 +| | +| 01:SCAN HDFS [tpch_parquet.nation t2] +| partitions=1/1 files=1 size=2.94KB +| stats-rows=25 extrapolated-rows=disabled +| table stats: rows=25 size=2.94KB +| column stats: all +| mem-estimate=16.00MB mem-reservation=0B +| tuple-ids=1 row-size=2B cardinality=25 +| +00:SCAN HDFS [tpch_parquet.nation t1] + partitions=1/1 files=1 size=2.94KB + stats-rows=25 extrapolated-rows=disabled + table stats: rows=25 size=2.94KB + column stats: all + mem-estimate=16.00MB mem-reservation=0B + tuple-ids=0 row-size=117B cardinality=25 +---- DISTRIBUTEDPLAN +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=137.99MB + +F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +10:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=0,1,2,3 row-size=129B cardinality=156250000 +| +F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +Per-Host Resources: mem-estimate=87.53MB mem-reservation=0B +06:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] +| mem-estimate=71.53MB mem-reservation=0B +| tuple-ids=0,1,2,3 row-size=129B cardinality=156250000 +| +|--09:EXCHANGE [BROADCAST] +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=1,2,3 row-size=12B cardinality=6250000 +| | +| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=18.38MB mem-reservation=0B +| 05:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] +| | mem-estimate=2.38MB mem-reservation=0B +| | tuple-ids=1,2,3 row-size=12B cardinality=6250000 +| | +| |--08:EXCHANGE [BROADCAST] +| | | mem-estimate=0B mem-reservation=0B +| | | tuple-ids=2,3 row-size=10B cardinality=250000 +| | | +| | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +| | Per-Host Resources: mem-estimate=16.08MB mem-reservation=0B +| | 04:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] +| | | mem-estimate=78.12KB mem-reservation=0B +| | | tuple-ids=2,3 row-size=10B cardinality=250000 +| | | +| | |--07:EXCHANGE [BROADCAST] +| | | | mem-estimate=0B mem-reservation=0B +| | | | tuple-ids=3 row-size=8B cardinality=10000 +| | | | +| | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +| | | Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B +| | | 03:SCAN HDFS [tpch_parquet.supplier t4, RANDOM] +| | | partitions=1/1 files=1 size=882.76KB +| | | stats-rows=10000 extrapolated-rows=disabled +| | | table stats: rows=10000 size=882.76KB +| | | column stats: all +| | | mem-estimate=16.00MB mem-reservation=0B +| | | tuple-ids=3 row-size=8B cardinality=10000 +| | | +| | 02:SCAN HDFS [tpch_parquet.nation t3, RANDOM] +| | partitions=1/1 files=1 size=2.94KB +| | stats-rows=25 extrapolated-rows=disabled +| | table stats: rows=25 size=2.94KB +| | column stats: all +| | mem-estimate=16.00MB mem-reservation=0B +| | tuple-ids=2 row-size=2B cardinality=25 +| | +| 01:SCAN HDFS [tpch_parquet.nation t2, RANDOM] +| partitions=1/1 files=1 size=2.94KB +| stats-rows=25 extrapolated-rows=disabled +| table stats: rows=25 size=2.94KB +| column stats: all +| mem-estimate=16.00MB mem-reservation=0B +| tuple-ids=1 row-size=2B cardinality=25 +| +00:SCAN HDFS [tpch_parquet.nation t1, RANDOM] + partitions=1/1 files=1 size=2.94KB + stats-rows=25 extrapolated-rows=disabled + table stats: rows=25 size=2.94KB + column stats: all + mem-estimate=16.00MB mem-reservation=0B + tuple-ids=0 row-size=117B cardinality=25 +---- PARALLELPLANS +Per-Host Resource Reservation: Memory=0B +Per-Host Resource Estimates: Memory=275.97MB + +F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +10:EXCHANGE [UNPARTITIONED] +| mem-estimate=0B mem-reservation=0B +| tuple-ids=0,1,2,3 row-size=129B cardinality=156250000 +| +F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +Per-Host Resources: mem-estimate=175.05MB mem-reservation=0B +06:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] +| join table id: 00 +| mem-estimate=71.53MB mem-reservation=0B +| tuple-ids=0,1,2,3 row-size=129B cardinality=156250000 +| +|--F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| | Per-Host Resources: included in parent fragment +| JOIN BUILD +| | join-table-id=00 plan-id=01 cohort-id=01 +| | mem-estimate=0B mem-reservation=0B +| | +| 09:EXCHANGE [BROADCAST] +| | mem-estimate=0B mem-reservation=0B +| | tuple-ids=1,2,3 row-size=12B cardinality=6250000 +| | +| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| Per-Host Resources: mem-estimate=36.77MB mem-reservation=0B +| 05:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] +| | join table id: 01 +| | mem-estimate=2.38MB mem-reservation=0B +| | tuple-ids=1,2,3 row-size=12B cardinality=6250000 +| | +| |--F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| | | Per-Host Resources: included in parent fragment +| | JOIN BUILD +| | | join-table-id=01 plan-id=02 cohort-id=02 +| | | mem-estimate=0B mem-reservation=0B +| | | +| | 08:EXCHANGE [BROADCAST] +| | | mem-estimate=0B mem-reservation=0B +| | | tuple-ids=2,3 row-size=10B cardinality=250000 +| | | +| | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| | Per-Host Resources: mem-estimate=32.15MB mem-reservation=0B +| | 04:NESTED LOOP JOIN [CROSS JOIN, BROADCAST] +| | | join table id: 02 +| | | mem-estimate=78.12KB mem-reservation=0B +| | | tuple-ids=2,3 row-size=10B cardinality=250000 +| | | +| | |--F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| | | | Per-Host Resources: included in parent fragment +| | | JOIN BUILD +| | | | join-table-id=02 plan-id=03 cohort-id=03 +| | | | mem-estimate=0B mem-reservation=0B +| | | | +| | | 07:EXCHANGE [BROADCAST] +| | | | mem-estimate=0B mem-reservation=0B +| | | | tuple-ids=3 row-size=8B cardinality=10000 +| | | | +| | | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| | | Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B +| | | 03:SCAN HDFS [tpch_parquet.supplier t4, RANDOM] +| | | partitions=1/1 files=1 size=882.76KB +| | | stats-rows=10000 extrapolated-rows=disabled +| | | table stats: rows=10000 size=882.76KB +| | | column stats: all +| | | mem-estimate=16.00MB mem-reservation=0B +| | | tuple-ids=3 row-size=8B cardinality=10000 +| | | +| | 02:SCAN HDFS [tpch_parquet.nation t3, RANDOM] +| | partitions=1/1 files=1 size=2.94KB +| | stats-rows=25 extrapolated-rows=disabled +| | table stats: rows=25 size=2.94KB +| | column stats: all +| | mem-estimate=16.00MB mem-reservation=0B +| | tuple-ids=2 row-size=2B cardinality=25 +| | +| 01:SCAN HDFS [tpch_parquet.nation t2, RANDOM] +| partitions=1/1 files=1 size=2.94KB +| stats-rows=25 extrapolated-rows=disabled +| table stats: rows=25 size=2.94KB +| column stats: all +| mem-estimate=16.00MB mem-reservation=0B +| tuple-ids=1 row-size=2B cardinality=25 +| +00:SCAN HDFS [tpch_parquet.nation t1, RANDOM] + partitions=1/1 files=1 size=2.94KB + stats-rows=25 extrapolated-rows=disabled + table stats: rows=25 size=2.94KB + column stats: all + mem-estimate=16.00MB mem-reservation=0B + tuple-ids=0 row-size=117B cardinality=25 +==== +# Pipeline of analytic nodes. They should be treated as streaming nodes. +select tinyint_col, smallint_col, +sum(smallint_col) over (partition by tinyint_col order by smallint_col), +sum(smallint_col) over (partition by tinyint_col order by smallint_col + rows between unbounded preceding and 1 following), + sum(smallint_col) over (partition by tinyint_col order by smallint_col + rows between 1 following and 2 following) + from functional.alltypesagg +---- PLAN +Per-Host Resource Reservation: Memory=72.00MB +Per-Host Resource Estimates: Memory=24.00MB +Codegen disabled by planner + +F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=24.00MB mem-reservation=72.00MB +PLAN-ROOT SINK +| mem-estimate=0B mem-reservation=0B +| +04:ANALYTIC +| functions: sum(smallint_col) +| partition by: tinyint_col +| order by: smallint_col ASC +| window: ROWS BETWEEN 1 FOLLOWING AND 2 FOLLOWING +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=5,2,3,4 row-size=27B cardinality=11000 +| +03:ANALYTIC +| functions: sum(smallint_col) +| partition by: tinyint_col +| order by: smallint_col ASC +| window: ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=5,2,3 row-size=19B cardinality=11000 +| +02:ANALYTIC +| functions: sum(smallint_col) +| partition by: tinyint_col +| order by: smallint_col ASC +| window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +| mem-estimate=0B mem-reservation=16.00MB +| tuple-ids=5,2 row-size=11B cardinality=11000 +| +01:SORT +| order by: tinyint_col ASC NULLS FIRST, smallint_col ASC +| mem-estimate=8.00MB mem-reservation=24.00MB +| tuple-ids=5 row-size=3B cardinality=11000 +| +00:SCAN HDFS [functional.alltypesagg] + partitions=11/11 files=11 size=814.73KB + stats-rows=11000 extrapolated-rows=disabled + table stats: rows=11000 size=814.73KB + column stats: all + mem-estimate=16.00MB mem-reservation=0B + tuple-ids=0 row-size=3B cardinality=11000 +==== diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test b/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test index 5845af2dc..3e5fb0520 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test @@ -2,6 +2,7 @@ select * from functional.alltypes order by random() ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=144.00MB mem-reservation=48.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -23,6 +24,7 @@ PLAN-ROOT SINK select * from functional.alltypes order by abs(id) + abs(id) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=144.00MB mem-reservation=48.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -44,6 +46,7 @@ PLAN-ROOT SINK select * from functional.alltypes order by tinyint_col + 1 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=144.00MB mem-reservation=48.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -65,6 +68,7 @@ select * from functional.alltypes order by dayofweek(timestamp_col), true, id + 1, string_col = date_string_col, id = tinyint_col ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=144.00MB mem-reservation=48.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -87,6 +91,7 @@ select last_value(id) over (order by to_date(timestamp_col), bool_col is null) from functional.alltypes ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=144.00MB mem-reservation=64.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -116,6 +121,7 @@ select id from functional.alltypes order by string_col like 'a.*b', id * bigint_ regexp_replace(string_col, 'a.*b', 'c') limit 10 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=128.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -137,6 +143,7 @@ PLAN-ROOT SINK select * from functional.alltypes order by TestFn(double_col) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=144.00MB mem-reservation=48.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -158,6 +165,7 @@ PLAN-ROOT SINK select concat(date_string_col, string_col) c from functional.alltypes order by c ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=144.00MB mem-reservation=48.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | 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 fb30e09bd..58fe1bfbd 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test @@ -4,9 +4,10 @@ from tpch_parquet.customer inner join tpch_parquet.nation on c_nationkey = n_nationkey ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=1.06MB -Per-Host Resource Estimates: Memory=24.00MB +Per-Host Resource Estimates: Memory=40.00MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -15,6 +16,7 @@ PLAN-ROOT SINK | tuple-ids=0,1 row-size=355B cardinality=150000 | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +Per-Host Resources: mem-estimate=24.00MB mem-reservation=1.06MB 02:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: c_nationkey = n_nationkey | fk/pk conjuncts: c_nationkey = n_nationkey @@ -27,6 +29,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 | | tuple-ids=1 row-size=117B cardinality=25 | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B | 01:SCAN HDFS [tpch_parquet.nation, RANDOM] | partitions=1/1 files=1 size=2.94KB | stats-rows=25 extrapolated-rows=disabled @@ -48,6 +51,7 @@ Per-Host Resource Reservation: Memory=2.12MB Per-Host Resource Estimates: Memory=80.01MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -56,6 +60,7 @@ PLAN-ROOT SINK | tuple-ids=0,1 row-size=355B cardinality=150000 | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +Per-Host Resources: mem-estimate=48.01MB mem-reservation=2.12MB 02:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=00 | hash predicates: c_nationkey = n_nationkey @@ -65,6 +70,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 | tuple-ids=0,1 row-size=355B cardinality=150000 | |--F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| | Per-Host Resources: included in parent fragment | JOIN BUILD | | join-table-id=00 plan-id=01 cohort-id=01 | | build expressions: n_nationkey @@ -75,6 +81,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 | | tuple-ids=1 row-size=117B cardinality=25 | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B | 01:SCAN HDFS [tpch_parquet.nation, RANDOM] | partitions=1/1 files=1 size=2.94KB | stats-rows=25 extrapolated-rows=disabled @@ -98,9 +105,10 @@ from tpch_parquet.lineitem left join tpch_parquet.orders on l_orderkey = o_orderkey ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=136.00MB -Per-Host Resource Estimates: Memory=380.41MB +Per-Host Resource Estimates: Memory=420.41MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -109,6 +117,7 @@ PLAN-ROOT SINK | tuple-ids=0,1N row-size=454B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=380.41MB mem-reservation=136.00MB 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST] | hash predicates: l_orderkey = o_orderkey | fk/pk conjuncts: l_orderkey = o_orderkey @@ -120,6 +129,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | | tuple-ids=1 row-size=191B cardinality=1500000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B | 01:SCAN HDFS [tpch_parquet.orders, RANDOM] | partitions=1/1 files=2 size=54.20MB | stats-rows=1500000 extrapolated-rows=disabled @@ -140,6 +150,7 @@ Per-Host Resource Reservation: Memory=272.00MB Per-Host Resource Estimates: Memory=840.83MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -148,6 +159,7 @@ PLAN-ROOT SINK | tuple-ids=0,1N row-size=454B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=760.83MB mem-reservation=272.00MB 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST] | hash-table-id=00 | hash predicates: l_orderkey = o_orderkey @@ -156,6 +168,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | tuple-ids=0,1N row-size=454B cardinality=6001215 | |--F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| | Per-Host Resources: included in parent fragment | JOIN BUILD | | join-table-id=00 plan-id=01 cohort-id=01 | | build expressions: o_orderkey @@ -166,6 +179,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | tuple-ids=1 row-size=191B cardinality=1500000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B | 01:SCAN HDFS [tpch_parquet.orders, RANDOM] | partitions=1/1 files=2 size=54.20MB | stats-rows=1500000 extrapolated-rows=disabled @@ -188,9 +202,10 @@ from tpch_parquet.orders join /*+shuffle*/ tpch_parquet.customer on o_custkey = c_custkey ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=34.00MB -Per-Host Resource Estimates: Memory=58.69MB +Per-Host Resource Estimates: Memory=82.69MB F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -199,6 +214,7 @@ PLAN-ROOT SINK | tuple-ids=0,1 row-size=428B cardinality=1500000 | F02:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=2 +Per-Host Resources: mem-estimate=18.69MB mem-reservation=34.00MB 02:HASH JOIN [INNER JOIN, PARTITIONED] | hash predicates: o_custkey = c_custkey | fk/pk conjuncts: o_custkey = c_custkey @@ -211,6 +227,7 @@ F02:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=2 | | tuple-ids=1 row-size=238B cardinality=150000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=24.00MB mem-reservation=0B | 01:SCAN HDFS [tpch_parquet.customer, RANDOM] | partitions=1/1 files=1 size=12.34MB | stats-rows=150000 extrapolated-rows=disabled @@ -224,6 +241,7 @@ F02:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=2 | tuple-ids=0 row-size=191B cardinality=1500000 | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B 00:SCAN HDFS [tpch_parquet.orders, RANDOM] partitions=1/1 files=2 size=54.20MB runtime filters: RF000 -> o_custkey @@ -237,6 +255,7 @@ Per-Host Resource Reservation: Memory=34.00MB Per-Host Resource Estimates: Memory=146.69MB F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -245,6 +264,7 @@ PLAN-ROOT SINK | tuple-ids=0,1 row-size=428B cardinality=1500000 | F02:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=4 +Per-Host Resources: mem-estimate=18.69MB mem-reservation=34.00MB 02:HASH JOIN [INNER JOIN, PARTITIONED] | hash-table-id=00 | hash predicates: o_custkey = c_custkey @@ -254,6 +274,7 @@ F02:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=4 | tuple-ids=0,1 row-size=428B cardinality=1500000 | |--F04:PLAN FRAGMENT [HASH(o_custkey)] hosts=1 instances=2 +| | Per-Host Resources: included in parent fragment | JOIN BUILD | | join-table-id=00 plan-id=01 cohort-id=01 | | build expressions: c_custkey @@ -264,6 +285,7 @@ F02:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=4 | | tuple-ids=1 row-size=238B cardinality=150000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| Per-Host Resources: mem-estimate=48.00MB mem-reservation=0B | 01:SCAN HDFS [tpch_parquet.customer, RANDOM] | partitions=1/1 files=1 size=12.34MB | stats-rows=150000 extrapolated-rows=disabled @@ -277,6 +299,7 @@ F02:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=4 | tuple-ids=0 row-size=191B cardinality=1500000 | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B 00:SCAN HDFS [tpch_parquet.orders, RANDOM] partitions=1/1 files=2 size=54.20MB runtime filters: RF000 -> o_custkey @@ -292,9 +315,10 @@ from tpch_parquet.orders join /*+broadcast*/ tpch_parquet.customer on o_custkey = c_custkey ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=68.00MB -Per-Host Resource Estimates: Memory=77.38MB +Per-Host Resource Estimates: Memory=101.38MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -303,6 +327,7 @@ PLAN-ROOT SINK | tuple-ids=0,1 row-size=428B cardinality=1500000 | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +Per-Host Resources: mem-estimate=77.38MB mem-reservation=68.00MB 02:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: o_custkey = c_custkey | fk/pk conjuncts: o_custkey = c_custkey @@ -315,6 +340,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 | | tuple-ids=1 row-size=238B cardinality=150000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=24.00MB mem-reservation=0B | 01:SCAN HDFS [tpch_parquet.customer, RANDOM] | partitions=1/1 files=1 size=12.34MB | stats-rows=150000 extrapolated-rows=disabled @@ -336,6 +362,7 @@ Per-Host Resource Reservation: Memory=136.00MB Per-Host Resource Estimates: Memory=202.76MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -344,6 +371,7 @@ PLAN-ROOT SINK | tuple-ids=0,1 row-size=428B cardinality=1500000 | F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +Per-Host Resources: mem-estimate=154.76MB mem-reservation=136.00MB 02:HASH JOIN [INNER JOIN, BROADCAST] | hash-table-id=00 | hash predicates: o_custkey = c_custkey @@ -353,6 +381,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 | tuple-ids=0,1 row-size=428B cardinality=1500000 | |--F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| | Per-Host Resources: included in parent fragment | JOIN BUILD | | join-table-id=00 plan-id=01 cohort-id=01 | | build expressions: c_custkey @@ -363,6 +392,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 | | tuple-ids=1 row-size=238B cardinality=150000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +| Per-Host Resources: mem-estimate=48.00MB mem-reservation=0B | 01:SCAN HDFS [tpch_parquet.customer, RANDOM] | partitions=1/1 files=1 size=12.34MB | stats-rows=150000 extrapolated-rows=disabled @@ -386,11 +416,12 @@ from functional_parquet.alltypes left join functional_parquet.alltypestiny on alltypes.id = alltypestiny.id ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=136.00MB -Per-Host Resource Estimates: Memory=2.02GB +Per-Host Resource Estimates: Memory=2.03GB WARNING: The following tables are missing relevant table and/or column statistics. functional_parquet.alltypes, functional_parquet.alltypestiny F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -399,6 +430,7 @@ PLAN-ROOT SINK | tuple-ids=0,1N row-size=176B cardinality=unavailable | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=2.02GB mem-reservation=136.00MB 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST] | hash predicates: alltypes.id = alltypestiny.id | fk/pk conjuncts: assumed fk/pk @@ -410,6 +442,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 | | tuple-ids=1 row-size=88B cardinality=unavailable | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +| Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B | 01:SCAN HDFS [functional_parquet.alltypestiny, RANDOM] | partitions=4/4 files=4 size=10.48KB | stats-rows=unavailable extrapolated-rows=disabled @@ -432,6 +465,7 @@ WARNING: The following tables are missing relevant table and/or column statistic functional_parquet.alltypestiny F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -440,6 +474,7 @@ PLAN-ROOT SINK | tuple-ids=0,1N row-size=176B cardinality=unavailable | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=4.03GB mem-reservation=272.00MB 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST] | hash-table-id=00 | hash predicates: alltypes.id = alltypestiny.id @@ -448,6 +483,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | tuple-ids=0,1N row-size=176B cardinality=unavailable | |--F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +| | Per-Host Resources: included in parent fragment | JOIN BUILD | | join-table-id=00 plan-id=01 cohort-id=01 | | build expressions: alltypestiny.id @@ -458,6 +494,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 | | tuple-ids=1 row-size=88B cardinality=unavailable | | | F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +| Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B | 01:SCAN HDFS [functional_parquet.alltypestiny, RANDOM] | partitions=4/4 files=4 size=10.48KB | stats-rows=unavailable extrapolated-rows=disabled @@ -483,6 +520,7 @@ Per-Host Resource Reservation: Memory=2.12MB Per-Host Resource Estimates: Memory=44.00MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -491,6 +529,7 @@ PLAN-ROOT SINK | tuple-ids=2 row-size=10B cardinality=25 | F01:PLAN FRAGMENT [HASH(c_nationkey)] hosts=1 instances=1 +Per-Host Resources: mem-estimate=10.00MB mem-reservation=2.12MB 03:AGGREGATE [FINALIZE] | output: avg:merge(c_acctbal) | group by: c_nationkey @@ -502,6 +541,7 @@ F01:PLAN FRAGMENT [HASH(c_nationkey)] hosts=1 instances=1 | tuple-ids=1 row-size=10B cardinality=25 | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1 +Per-Host Resources: mem-estimate=34.00MB mem-reservation=0B 01:AGGREGATE [STREAMING] | output: avg(c_acctbal) | group by: c_nationkey @@ -520,6 +560,7 @@ Per-Host Resource Reservation: Memory=4.25MB Per-Host Resource Estimates: Memory=88.00MB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -528,6 +569,7 @@ PLAN-ROOT SINK | tuple-ids=2 row-size=10B cardinality=25 | F01:PLAN FRAGMENT [HASH(c_nationkey)] hosts=1 instances=2 +Per-Host Resources: mem-estimate=20.00MB mem-reservation=4.25MB 03:AGGREGATE [FINALIZE] | output: avg:merge(c_acctbal) | group by: c_nationkey @@ -539,6 +581,7 @@ F01:PLAN FRAGMENT [HASH(c_nationkey)] hosts=1 instances=2 | tuple-ids=1 row-size=10B cardinality=25 | F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2 +Per-Host Resources: mem-estimate=68.00MB mem-reservation=0B 01:AGGREGATE [STREAMING] | output: avg(c_acctbal) | group by: c_nationkey @@ -561,9 +604,10 @@ group by 1, 2 having count(*) = 1 ---- DISTRIBUTEDPLAN Per-Host Resource Reservation: Memory=83.00MB -Per-Host Resource Estimates: Memory=165.28MB +Per-Host Resource Estimates: Memory=205.28MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -572,6 +616,7 @@ PLAN-ROOT SINK | tuple-ids=2 row-size=33B cardinality=4690314 | F03:PLAN FRAGMENT [HASH(l_orderkey,o_orderstatus)] hosts=3 instances=3 +Per-Host Resources: mem-estimate=18.04MB mem-reservation=66.00MB 07:AGGREGATE [FINALIZE] | output: count:merge(*) | group by: l_orderkey, o_orderstatus @@ -584,6 +629,7 @@ F03:PLAN FRAGMENT [HASH(l_orderkey,o_orderstatus)] hosts=3 instances=3 | tuple-ids=2 row-size=33B cardinality=4690314 | F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 +Per-Host Resources: mem-estimate=67.24MB mem-reservation=17.00MB 03:AGGREGATE [STREAMING] | output: count(*) | group by: l_orderkey, o_orderstatus @@ -602,6 +648,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 | | tuple-ids=1 row-size=25B cardinality=1500000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2 +| Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B | 01:SCAN HDFS [tpch_parquet.orders, RANDOM] | partitions=1/1 files=2 size=54.20MB | stats-rows=1500000 extrapolated-rows=disabled @@ -615,6 +662,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3 | tuple-ids=0 row-size=8B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM] partitions=1/1 files=3 size=193.92MB runtime filters: RF000 -> l_orderkey @@ -628,6 +676,7 @@ Per-Host Resource Reservation: Memory=83.00MB Per-Host Resource Estimates: Memory=327.24MB F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -636,6 +685,7 @@ PLAN-ROOT SINK | tuple-ids=2 row-size=33B cardinality=4690314 | F03:PLAN FRAGMENT [HASH(l_orderkey,o_orderstatus)] hosts=3 instances=6 +Per-Host Resources: mem-estimate=20.00MB mem-reservation=66.00MB 07:AGGREGATE [FINALIZE] | output: count:merge(*) | group by: l_orderkey, o_orderstatus @@ -648,6 +698,7 @@ F03:PLAN FRAGMENT [HASH(l_orderkey,o_orderstatus)] hosts=3 instances=6 | tuple-ids=2 row-size=33B cardinality=4690314 | F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 +Per-Host Resources: mem-estimate=67.24MB mem-reservation=17.00MB 03:AGGREGATE [STREAMING] | output: count(*) | group by: l_orderkey, o_orderstatus @@ -663,6 +714,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 | tuple-ids=0,1 row-size=33B cardinality=5757710 | |--F05:PLAN FRAGMENT [HASH(l_orderkey)] hosts=2 instances=4 +| | Per-Host Resources: included in parent fragment | JOIN BUILD | | join-table-id=00 plan-id=01 cohort-id=01 | | build expressions: o_orderkey @@ -673,6 +725,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 | | tuple-ids=1 row-size=25B cardinality=1500000 | | | F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4 +| Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B | 01:SCAN HDFS [tpch_parquet.orders, RANDOM] | partitions=1/1 files=2 size=54.20MB | stats-rows=1500000 extrapolated-rows=disabled @@ -686,6 +739,7 @@ F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6 | tuple-ids=0 row-size=8B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=160.00MB mem-reservation=0B 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM] partitions=1/1 files=3 size=193.92MB runtime filters: RF000 -> l_orderkey @@ -703,6 +757,7 @@ Per-Host Resource Reservation: Memory=264.00MB Per-Host Resource Estimates: Memory=3.31GB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -711,6 +766,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=263B cardinality=6001215 | 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=1.62GB mem-reservation=264.00MB 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 | mem-estimate=1.62GB mem-reservation=264.00MB @@ -721,6 +777,7 @@ F01:PLAN FRAGMENT [HASH(tpch_parquet.lineitem.l_orderkey,tpch_parquet.lineitem.l | tuple-ids=1 row-size=263B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=1.69GB mem-reservation=0B 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=1.62GB mem-reservation=0B @@ -738,6 +795,7 @@ Per-Host Resource Reservation: Memory=528.00MB Per-Host Resource Estimates: Memory=6.62GB F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -746,6 +804,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=263B cardinality=6001215 | 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=6 +Per-Host Resources: mem-estimate=3.23GB mem-reservation=528.00MB 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 | mem-estimate=1.62GB mem-reservation=264.00MB @@ -756,6 +815,7 @@ F01:PLAN FRAGMENT [HASH(tpch_parquet.lineitem.l_orderkey,tpch_parquet.lineitem.l | tuple-ids=1 row-size=263B cardinality=6001215 | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=3.39GB mem-reservation=0B 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=1.62GB mem-reservation=0B @@ -780,6 +840,7 @@ WARNING: The following tables are missing relevant table and/or column statistic functional_parquet.alltypestiny F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -788,6 +849,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=24B cardinality=unavailable | F01:PLAN FRAGMENT [HASH(string_col)] hosts=3 instances=3 +Per-Host Resources: mem-estimate=128.00MB mem-reservation=264.00MB 03:AGGREGATE [FINALIZE] | output: count:merge(*) | group by: string_col @@ -799,6 +861,7 @@ F01:PLAN FRAGMENT [HASH(string_col)] hosts=3 instances=3 | tuple-ids=1 row-size=24B cardinality=unavailable | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3 +Per-Host Resources: mem-estimate=144.00MB mem-reservation=0B 01:AGGREGATE [STREAMING] | output: count(*) | group by: string_col @@ -819,6 +882,7 @@ WARNING: The following tables are missing relevant table and/or column statistic functional_parquet.alltypestiny F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -827,6 +891,7 @@ PLAN-ROOT SINK | tuple-ids=1 row-size=24B cardinality=unavailable | F01:PLAN FRAGMENT [HASH(string_col)] hosts=3 instances=6 +Per-Host Resources: mem-estimate=256.00MB mem-reservation=528.00MB 03:AGGREGATE [FINALIZE] | output: count:merge(*) | group by: string_col @@ -838,6 +903,7 @@ F01:PLAN FRAGMENT [HASH(string_col)] hosts=3 instances=6 | tuple-ids=1 row-size=24B cardinality=unavailable | F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 +Per-Host Resources: mem-estimate=288.00MB mem-reservation=0B 01:AGGREGATE [STREAMING] | output: count(*) | group by: string_col diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test b/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test index fc8778ebb..868d6ca2b 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test @@ -2,6 +2,7 @@ select * from functional.alltypes tablesample system(10) repeatable(1234) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -17,6 +18,7 @@ PLAN-ROOT SINK select * from functional.alltypes tablesample system(50) repeatable(1234) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -34,6 +36,7 @@ select * from functional.alltypes tablesample system(50) repeatable(1234) where id < 10 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -52,6 +55,7 @@ select * from functional.alltypes tablesample system(50) repeatable(1234) where year = 2009 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=48.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -67,6 +71,7 @@ PLAN-ROOT SINK select * from functional.alltypes tablesample system(0) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=0B mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -82,6 +87,7 @@ PLAN-ROOT SINK select * from functional.alltypes tablesample system(1) repeatable(1234) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -98,6 +104,7 @@ select * from functional.alltypes tablesample system(1) repeatable(1234) where year = 2010 ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -113,6 +120,7 @@ PLAN-ROOT SINK select * from functional.alltypes tablesample system (100) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=128.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -128,6 +136,7 @@ PLAN-ROOT SINK select id from functional_parquet.alltypes tablesample system(10) repeatable(1234) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -145,6 +154,7 @@ select id from functional.alltypes t1 where exists ( where t1.id = t2.id) ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=160.00MB mem-reservation=136.00MB PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | @@ -176,6 +186,7 @@ with t as (select * from functional.alltypes tablesample system(10) repeatable(1 select id from t ---- PLAN F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1 +| Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B PLAN-ROOT SINK | mem-estimate=0B mem-reservation=0B | diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test index 06d76c893..64f9b4563 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test +++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test @@ -6,7 +6,7 @@ select * from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey; ---- RESULTS: VERIFY_IS_EQUAL 'Per-Host Resource Reservation: Memory=136.00MB' -'Per-Host Resource Estimates: Memory=388.41MB' +'Per-Host Resource Estimates: Memory=476.41MB' '' 'PLAN-ROOT SINK' '04:EXCHANGE [UNPARTITIONED]' diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test index a0826032a..f59962cc2 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test +++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test @@ -6,7 +6,7 @@ select * from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey; ---- RESULTS: VERIFY_IS_EQUAL 'Per-Host Resource Reservation: Memory=136.00MB' -'Per-Host Resource Estimates: Memory=388.41MB' +'Per-Host Resource Estimates: Memory=476.41MB' '' 'PLAN-ROOT SINK' '|' diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test index 8d4ff1f0d..2736543f2 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test +++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test @@ -6,9 +6,10 @@ select * from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey; ---- RESULTS: VERIFY_IS_EQUAL 'Per-Host Resource Reservation: Memory=136.00MB' -'Per-Host Resource Estimates: Memory=388.41MB' +'Per-Host Resource Estimates: Memory=476.41MB' '' 'F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1' +'| Per-Host Resources: mem-estimate=0B mem-reservation=0B' 'PLAN-ROOT SINK' '| mem-estimate=0B mem-reservation=0B' '|' @@ -17,6 +18,7 @@ from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey; '| tuple-ids=0,1 row-size=454B cardinality=5757710' '|' 'F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3' +'Per-Host Resources: mem-estimate=388.41MB mem-reservation=136.00MB' '02:HASH JOIN [INNER JOIN, BROADCAST]' '| hash predicates: l_orderkey = o_orderkey' '| fk/pk conjuncts: l_orderkey = o_orderkey' @@ -29,6 +31,7 @@ from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey; '| | tuple-ids=1 row-size=191B cardinality=1500000' '| |' '| F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2' +'| Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B' '| 01:SCAN HDFS [tpch.orders, RANDOM]' row_regex:.*partitions=1/1 files=1 size=.* '| stats-rows=1500000 extrapolated-rows=disabled' @@ -54,7 +57,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=4.03GB' +'Per-Host Resource Estimates: Memory=4.07GB' 'WARNING: The following tables are missing relevant table and/or column statistics.' 'functional_avro.alltypes, functional_parquet.alltypessmall' ==== diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test index 4fff35adf..31f4f5bf0 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test +++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test @@ -6,9 +6,10 @@ select * from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey; ---- RESULTS: VERIFY_IS_EQUAL 'Per-Host Resource Reservation: Memory=136.00MB' -'Per-Host Resource Estimates: Memory=388.41MB' +'Per-Host Resource Estimates: Memory=476.41MB' '' 'F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1' +'Per-Host Resources: mem-estimate=0B mem-reservation=0B' ' PLAN-ROOT SINK' ' | mem-estimate=0B mem-reservation=0B' ' |' @@ -17,6 +18,7 @@ from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey; ' tuple-ids=0,1 row-size=454B cardinality=5757710' '' 'F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3' +'Per-Host Resources: mem-estimate=388.41MB mem-reservation=136.00MB' ' DATASTREAM SINK [FRAGMENT=F02, EXCHANGE=04, UNPARTITIONED]' ' | mem-estimate=0B mem-reservation=0B' ' 02:HASH JOIN [INNER JOIN, BROADCAST]' @@ -40,6 +42,7 @@ row_regex:.*partitions=1/1 files=1 size=.* ' tuple-ids=0 row-size=263B cardinality=6001215' '' 'F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2' +'Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B' ' DATASTREAM SINK [FRAGMENT=F00, EXCHANGE=03, BROADCAST]' ' | mem-estimate=0B mem-reservation=0B' ' 01:SCAN HDFS [tpch.orders, RANDOM]' diff --git a/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test b/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test index 9de548048..b31bacf11 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test +++ b/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test @@ -32,6 +32,7 @@ explain select id from alltypes; 'Codegen disabled by planner' '' 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1' +'| Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B' 'PLAN-ROOT SINK' '| mem-estimate=0B mem-reservation=0B' '|' @@ -54,6 +55,7 @@ explain select id from alltypes where month in (1, 2, 3); 'Codegen disabled by planner' '' 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1' +'| Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B' 'PLAN-ROOT SINK' '| mem-estimate=0B mem-reservation=0B' '|' @@ -77,6 +79,7 @@ explain select id from alltypes; 'Per-Host Resource Estimates: Memory=16.00MB' '' 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1' +'| Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B' 'PLAN-ROOT SINK' '| mem-estimate=0B mem-reservation=0B' '|' @@ -101,6 +104,7 @@ explain select id from alltypes where year = 2010; 'Codegen disabled by planner' '' 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1' +'| Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B' 'PLAN-ROOT SINK' '| mem-estimate=0B mem-reservation=0B' '|' @@ -124,6 +128,7 @@ explain select id from alltypes where year = 2010; 'Codegen disabled by planner' '' 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1' +'| Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B' 'PLAN-ROOT SINK' '| mem-estimate=0B mem-reservation=0B' '|'