diff --git a/.github/workflows/build-cloudberry.yml b/.github/workflows/build-cloudberry.yml index 04d5e827b6e..4a873b8d390 100644 --- a/.github/workflows/build-cloudberry.yml +++ b/.github/workflows/build-cloudberry.yml @@ -326,6 +326,12 @@ jobs: }, {"test":"ic-cbdb-parallel", "make_configs":["src/test/regress:installcheck-cbdb-parallel"] + }, + {"test":"ic-orca-parallel", + "make_configs":["src/test/regress:installcheck-orca-parallel"], + "pg_settings":{ + "optimizer_enable_parallel_append":"true" + } } ] }' diff --git a/src/backend/gpopt/config/CConfigParamMapping.cpp b/src/backend/gpopt/config/CConfigParamMapping.cpp index 603855c50ec..1a0d9588ba2 100644 --- a/src/backend/gpopt/config/CConfigParamMapping.cpp +++ b/src/backend/gpopt/config/CConfigParamMapping.cpp @@ -284,6 +284,10 @@ CConfigParamMapping::SConfigMappingElem CConfigParamMapping::m_elements[] = { GPOS_WSZ_LIT( "Enable Eager Agg transform for pushing aggregate below an innerjoin.")}, + {EopttraceEnableParallelAppendScan, &optimizer_enable_parallel_append, + false, // m_negate_param + GPOS_WSZ_LIT("Enable parallel append for scan/bitmap/index scan in partition tables.")}, + {EopttraceDisableOrderedAgg, &optimizer_enable_orderedagg, true, // m_negate_param GPOS_WSZ_LIT("Disable ordered aggregate plans.")}, @@ -449,6 +453,8 @@ CConfigParamMapping::PackConfigParamInBitset( // disable table scan if the corresponding GUC is turned off traceflag_bitset->ExchangeSet( GPOPT_DISABLE_XFORM_TF(CXform::ExfGet2TableScan)); + traceflag_bitset->ExchangeSet( + GPOPT_DISABLE_XFORM_TF(CXform::ExfGet2ParallelTableScan)); } if (!optimizer_enable_push_join_below_union_all) diff --git a/src/backend/gpopt/gpdbwrappers.cpp b/src/backend/gpopt/gpdbwrappers.cpp index 4e636a0c653..92931656cc8 100644 --- a/src/backend/gpopt/gpdbwrappers.cpp +++ b/src/backend/gpopt/gpdbwrappers.cpp @@ -25,6 +25,8 @@ #include // std::numeric_limits #include "gpos/base.h" +#include "gpopt/base/COptCtxt.h" +#include "gpopt/optimizer/COptimizerConfig.h" #include "gpos/error/CAutoExceptionStack.h" #include "gpos/error/CException.h" @@ -36,8 +38,10 @@ extern "C" { #include "access/amapi.h" #include "access/external.h" #include "access/genam.h" +#include "access/parallel.h" #include "catalog/pg_aggregate.h" #include "catalog/pg_inherits.h" +#include "cdb/cdbvars.h" #include "foreign/fdwapi.h" #include "nodes/nodeFuncs.h" #include "optimizer/clauses.h" @@ -52,6 +56,9 @@ extern "C" { #include "utils/lsyscache.h" #include "utils/memutils.h" #include "utils/partcache.h" + +extern bool enable_parallel; +extern int max_parallel_workers_per_gather; } #define GP_WRAP_START \ sigjmp_buf local_sigjmp_buf; \ @@ -2548,6 +2555,19 @@ gpdb::GetForeignServerId(Oid reloid) return 0; } +int16 +gpdb::GetAppendOnlySegmentFilesCount(Relation rel) +{ + GP_WRAP_START; + { + FormData_pg_appendonly aoFormData; + GetAppendOnlyEntry(rel, &aoFormData); + return aoFormData.segfilecount; + } + GP_WRAP_END; + return -1; +} + // Locks on partition leafs and indexes are held during optimizer (after // parse-analyze stage). ORCA need this function to lock relation. Here // we do not need to consider lock-upgrade issue, reasons are: @@ -2706,4 +2726,36 @@ gpdb::TestexprIsHashable(Node *testexpr, List *param_ids) return false; } +// check if parallel mode is OK (comprehensive check) +bool +gpdb::IsParallelModeOK(void) +{ + GP_WRAP_START; + { + if (!enable_parallel) + return false; + + if (IS_SINGLENODE()) + return false; + + if (max_parallel_workers_per_gather <= 0) + return false; + + // Check if parallel plans are enabled in current optimizer context + gpopt::COptCtxt *poctxt = gpopt::COptCtxt::PoctxtFromTLS(); + if (nullptr != poctxt) + { + gpopt::COptimizerConfig *optimizer_config = poctxt->GetOptimizerConfig(); + if (nullptr != optimizer_config) + { + if (!optimizer_config->CreateParallelPlan()) + return false; + } + } + return true; + } + GP_WRAP_END; + return false; // default to disabled if no context +} + // EOF diff --git a/src/backend/gpopt/translate/CTranslatorDXLToPlStmt.cpp b/src/backend/gpopt/translate/CTranslatorDXLToPlStmt.cpp index 4acf13bd606..fdbff9d3587 100644 --- a/src/backend/gpopt/translate/CTranslatorDXLToPlStmt.cpp +++ b/src/backend/gpopt/translate/CTranslatorDXLToPlStmt.cpp @@ -30,6 +30,7 @@ extern "C" { #include "partitioning/partdesc.h" #include "storage/lmgr.h" #include "utils/guc.h" +#include "optimizer/cost.h" #include "utils/lsyscache.h" #include "utils/partcache.h" #include "utils/rel.h" @@ -58,11 +59,15 @@ extern "C" { #include "naucrates/dxl/operators/CDXLNode.h" #include "naucrates/dxl/operators/CDXLPhysicalAgg.h" #include "naucrates/dxl/operators/CDXLPhysicalAppend.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelAppend.h" #include "naucrates/dxl/operators/CDXLPhysicalAssert.h" #include "naucrates/dxl/operators/CDXLPhysicalBitmapTableScan.h" #include "naucrates/dxl/operators/CDXLPhysicalCTAS.h" #include "naucrates/dxl/operators/CDXLPhysicalCTEConsumer.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelCTEConsumer.h" #include "naucrates/dxl/operators/CDXLPhysicalCTEProducer.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelCTEProducer.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelSequence.h" #include "naucrates/dxl/operators/CDXLPhysicalDynamicBitmapTableScan.h" #include "naucrates/dxl/operators/CDXLPhysicalDynamicForeignScan.h" #include "naucrates/dxl/operators/CDXLPhysicalDynamicIndexOnlyScan.h" @@ -83,6 +88,7 @@ extern "C" { #include "naucrates/dxl/operators/CDXLPhysicalSplit.h" #include "naucrates/dxl/operators/CDXLPhysicalTVF.h" #include "naucrates/dxl/operators/CDXLPhysicalTableScan.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelTableScan.h" #include "naucrates/dxl/operators/CDXLPhysicalValuesScan.h" #include "naucrates/dxl/operators/CDXLPhysicalWindow.h" #include "naucrates/dxl/operators/CDXLScalarBitmapBoolOp.h" @@ -348,6 +354,12 @@ CTranslatorDXLToPlStmt::TranslateDXLOperatorToPlan( ctxt_translation_prev_siblings); break; } + case EdxlopPhysicalParallelTableScan: + { + plan = TranslateDXLParallelTblScan(dxlnode, output_context, + ctxt_translation_prev_siblings); + break; + } case EdxlopPhysicalIndexScan: { plan = TranslateDXLIndexScan(dxlnode, output_context, @@ -434,6 +446,12 @@ CTranslatorDXLToPlStmt::TranslateDXLOperatorToPlan( ctxt_translation_prev_siblings); break; } + case EdxlopPhysicalParallelAppend: + { + plan = TranslateDXLParallelAppend(dxlnode, output_context, + ctxt_translation_prev_siblings); + break; + } case EdxlopPhysicalMaterialize: { plan = TranslateDXLMaterialize(dxlnode, output_context, @@ -446,6 +464,12 @@ CTranslatorDXLToPlStmt::TranslateDXLOperatorToPlan( ctxt_translation_prev_siblings); break; } + case EdxlopPhysicalParallelSequence: + { + plan = TranslateDXLParallelSequence(dxlnode, output_context, + ctxt_translation_prev_siblings); + break; + } case EdxlopPhysicalDynamicTableScan: { plan = TranslateDXLDynTblScan(dxlnode, output_context, @@ -500,12 +524,24 @@ CTranslatorDXLToPlStmt::TranslateDXLOperatorToPlan( dxlnode, output_context, ctxt_translation_prev_siblings); break; } + case EdxlopPhysicalParallelCTEProducer: + { + plan = TranslateDXLParallelCTEProducerToParallelSharedScan( + dxlnode, output_context, ctxt_translation_prev_siblings); + break; + } case EdxlopPhysicalCTEConsumer: { plan = TranslateDXLCTEConsumerToSharedScan( dxlnode, output_context, ctxt_translation_prev_siblings); break; } + case EdxlopPhysicalParallelCTEConsumer: + { + plan = TranslateDXLParallelCTEConsumerToParallelSharedScan( + dxlnode, output_context, ctxt_translation_prev_siblings); + break; + } case EdxlopPhysicalBitmapTableScan: case EdxlopPhysicalDynamicBitmapTableScan: { @@ -712,6 +748,111 @@ CTranslatorDXLToPlStmt::TranslateDXLTblScan( } +//--------------------------------------------------------------------------- +// @function: +// CTranslatorDXLToPlStmt::TranslateDXLParallelTblScan +// +// @doc: +// Translates a DXL parallel table scan node into a parallel SeqScan node +Plan * +CTranslatorDXLToPlStmt::TranslateDXLParallelTblScan( + const CDXLNode *tbl_scan_dxlnode, CDXLTranslateContext *output_context, + CDXLTranslationContextArray * /*ctxt_translation_prev_siblings*/) +{ + // translate table descriptor into a range table entry + CDXLPhysicalParallelTableScan *phy_parallel_tbl_scan_dxlop = + CDXLPhysicalParallelTableScan::Cast(tbl_scan_dxlnode->GetOperator()); + + ULONG parallel_workers = phy_parallel_tbl_scan_dxlop->UlParallelWorkers(); + + // translation context for column mappings in the base relation + CDXLTranslateContextBaseTable base_table_context(m_mp); + + const CDXLTableDescr *dxl_table_descr = + phy_parallel_tbl_scan_dxlop->GetDXLTableDescr(); + const IMDRelation *md_rel = + m_md_accessor->RetrieveRel(dxl_table_descr->MDId()); + + // Lock any table we are to scan, since it may not have been properly locked + // by the parser (e.g in case of generated scans for partitioned tables) + OID oidRel = CMDIdGPDB::CastMdid(md_rel->MDId())->Oid(); + GPOS_ASSERT(dxl_table_descr->LockMode() != -1); + gpdb::GPDBLockRelationOid(oidRel, dxl_table_descr->LockMode()); + + Index index = ProcessDXLTblDescr(dxl_table_descr, &base_table_context); + + // a table scan node must have 2 children: projection list and filter + GPOS_ASSERT(2 == tbl_scan_dxlnode->Arity()); + + // translate proj list and filter + CDXLNode *project_list_dxlnode = (*tbl_scan_dxlnode)[EdxltsIndexProjList]; + CDXLNode *filter_dxlnode = (*tbl_scan_dxlnode)[EdxltsIndexFilter]; + + List *targetlist = NIL; + + // List to hold the quals after translating filter_dxlnode node. + List *query_quals = NIL; + + TranslateProjListAndFilter( + project_list_dxlnode, filter_dxlnode, + &base_table_context, // translate context for the base table + nullptr, // translate_ctxt_left and pdxltrctxRight, + &targetlist, &query_quals, output_context); + + Plan *plan = nullptr; + Plan *plan_return = nullptr; + + // Parallel table scans are always sequential scans (not foreign scans) + SeqScan *seq_scan = MakeNode(SeqScan); + seq_scan->scanrelid = index; + plan = &(seq_scan->plan); + plan_return = (Plan *) seq_scan; + + // Set parallel execution flags + plan->parallel_aware = true; + plan->parallel_safe = true; + plan->parallel = (int) parallel_workers; + + plan->targetlist = targetlist; + + // List to hold the quals which contain both security quals and query + // quals. + List *security_query_quals = NIL; + + // Fetching the RTE of the relation from the rewritten parse tree + // based on the oidRel and adding the security quals of the RTE in + // the security_query_quals list. + AddSecurityQuals(oidRel, &security_query_quals, &index); + + // The security quals should always be executed first when + // compared to other quals. So appending query quals to the + // security_query_quals list after the security quals. + security_query_quals = + gpdb::ListConcat(security_query_quals, query_quals); + plan->qual = security_query_quals; + + if (md_rel->IsNonBlockTable()) + { + CheckSafeTargetListForAOTables(plan->targetlist); + } + + plan->plan_node_id = m_dxl_to_plstmt_context->GetNextPlanId(); + + // translate operator costs + TranslatePlanCosts(tbl_scan_dxlnode, plan); + + // Adjust row count to per-worker statistics + if (parallel_workers > 1) + { + plan->plan_rows = ceil(plan->plan_rows / parallel_workers); + } + + SetParamIds(plan); + + return plan_return; +} + + //--------------------------------------------------------------------------- // @function: // CTranslatorDXLToPlStmt::SetIndexVarAttnoWalker @@ -719,7 +860,6 @@ CTranslatorDXLToPlStmt::TranslateDXLTblScan( // @doc: // Walker to set index var attno's, // attnos of index vars are set to their relative positions in index keys, -// skip any outer references while walking the expression tree // //--------------------------------------------------------------------------- BOOL @@ -2415,15 +2555,34 @@ CTranslatorDXLToPlStmt::TranslateDXLMotion( sendslice->directDispatch.contentIds = NIL; sendslice->directDispatch.haveProcessedAnyCalculations = false; + // set parallel workers if needed + ULONG child_index = motion_dxlop->GetRelationChildIdx(); + CDXLNode *child_dxlnode = (*motion_dxlnode)[child_index]; + ULONG child_parallel_workers = ExtractParallelWorkersFromDXL(child_dxlnode); + if (child_parallel_workers > 1) + { + // Determine parallel workers based on enable_parallel and gang type + bool supports_parallel = (sendslice->gangType == GANGTYPE_PRIMARY_READER || + sendslice->gangType == GANGTYPE_PRIMARY_WRITER); + + if (supports_parallel) + { + sendslice->parallel_workers = child_parallel_workers; + } + else + { + // Disable parallel for: non-PRIMARY gang types + // (SINGLETON_READER, ENTRYDB_READER, UNALLOCATED) + sendslice->parallel_workers = 0; + } + } + motion->motionID = sendslice->sliceIndex; // translate motion child // child node is in the same position in broadcast and gather motion nodes // but different in redistribute motion nodes - - ULONG child_index = motion_dxlop->GetRelationChildIdx(); - - CDXLNode *child_dxlnode = (*motion_dxlnode)[child_index]; + // Note: child_index and child_dxlnode already defined above CDXLTranslateContext child_context(m_mp, false, output_context->GetColIdToParamIdMap()); @@ -2576,6 +2735,16 @@ CTranslatorDXLToPlStmt::TranslateDXLMotion( return nullptr; } + // Adjust row count for parallel execution in the sending slice + // The Motion node receives rows from all parallel workers, so we need to + // account for the fact that each worker processes a fraction of the rows. + // TranslatePlanCosts() already divided by numsegments, but if we have + // parallel workers, each segment is further subdivided among workers. + if (sendslice->parallel_workers > 1) + { + plan->plan_rows = ceil(plan->plan_rows / sendslice->parallel_workers); + } + SetParamIds(plan); return (Plan *) motion; @@ -4366,6 +4535,142 @@ CTranslatorDXLToPlStmt::TranslateDXLAppend( return (Plan *) append; } +//--------------------------------------------------------------------------- +// @function: +// CTranslatorDXLToPlStmt::TranslateDXLParallelAppend +// +// @doc: +// Translates a DXL parallel append node into a parallel Append node +Plan * +CTranslatorDXLToPlStmt::TranslateDXLParallelAppend( + const CDXLNode *append_dxlnode, CDXLTranslateContext *output_context, + CDXLTranslationContextArray *ctxt_translation_prev_siblings) +{ + CDXLPhysicalParallelAppend *phy_parallel_append_dxlop = + CDXLPhysicalParallelAppend::Cast(append_dxlnode->GetOperator()); + + ULONG parallel_workers = phy_parallel_append_dxlop->UlParallelWorkers(); + + // create append plan node + Append *append = MakeNode(Append); + append->first_partial_plan = 0; + + Plan *plan = &(append->plan); + plan->plan_node_id = m_dxl_to_plstmt_context->GetNextPlanId(); + + // Set parallel execution flags + plan->parallel_aware = true; + plan->parallel_safe = true; + plan->parallel = (int) parallel_workers; + + // translate operator costs + TranslatePlanCosts(append_dxlnode, plan); + + const ULONG arity = append_dxlnode->Arity(); + GPOS_ASSERT(EdxlappendIndexFirstChild < arity); + append->appendplans = NIL; + + // translate table descriptor into a range table entry + CDXLPhysicalParallelAppend *phy_append_dxlop = + CDXLPhysicalParallelAppend::Cast(append_dxlnode->GetOperator()); + + // If this append was create from a DynamicTableScan node in ORCA, it will + // contain the table descriptor of the root partitioned table. Add that to + // the range table in the PlStmt. + if (phy_append_dxlop->GetScanId() != gpos::ulong_max) + { + GPOS_ASSERT(nullptr != phy_append_dxlop->GetDXLTableDesc()); + + // translation context for column mappings in the base relation + CDXLTranslateContextBaseTable base_table_context(m_mp); + + (void) ProcessDXLTblDescr(phy_append_dxlop->GetDXLTableDesc(), + &base_table_context); + + OID oid_type = + CMDIdGPDB::CastMdid(m_md_accessor->PtMDType()->MDId()) + ->Oid(); + append->join_prune_paramids = + TranslateJoinPruneParamids(phy_append_dxlop->GetSelectorIds(), + oid_type, m_dxl_to_plstmt_context); + } + + // translate children + CDXLTranslateContext child_context(m_mp, false, + output_context->GetColIdToParamIdMap()); + for (ULONG ul = EdxlappendIndexFirstChild; ul < arity; ul++) + { + CDXLNode *child_dxlnode = (*append_dxlnode)[ul]; + + Plan *child_plan = TranslateDXLOperatorToPlan( + child_dxlnode, &child_context, ctxt_translation_prev_siblings); + + GPOS_ASSERT(nullptr != child_plan && "child plan cannot be NULL"); + + append->appendplans = gpdb::LAppend(append->appendplans, child_plan); + } + + CDXLNode *project_list_dxlnode = (*append_dxlnode)[EdxlappendIndexProjList]; + CDXLNode *filter_dxlnode = (*append_dxlnode)[EdxlappendIndexFilter]; + + plan->targetlist = NIL; + const ULONG length = project_list_dxlnode->Arity(); + for (ULONG ul = 0; ul < length; ++ul) + { + CDXLNode *proj_elem_dxlnode = (*project_list_dxlnode)[ul]; + GPOS_ASSERT(EdxlopScalarProjectElem == + proj_elem_dxlnode->GetOperator()->GetDXLOperator()); + + CDXLScalarProjElem *sc_proj_elem_dxlop = + CDXLScalarProjElem::Cast(proj_elem_dxlnode->GetOperator()); + GPOS_ASSERT(1 == proj_elem_dxlnode->Arity()); + + // translate proj element expression + CDXLNode *expr_dxlnode = (*proj_elem_dxlnode)[0]; + CDXLScalarIdent *sc_ident_dxlop = + CDXLScalarIdent::Cast(expr_dxlnode->GetOperator()); + + Index idxVarno = OUTER_VAR; + AttrNumber attno = (AttrNumber)(ul + 1); + + Var *var = gpdb::MakeVar( + idxVarno, attno, + CMDIdGPDB::CastMdid(sc_ident_dxlop->MdidType())->Oid(), + sc_ident_dxlop->TypeModifier(), + 0 // varlevelsup + ); + + TargetEntry *target_entry = MakeNode(TargetEntry); + target_entry->expr = (Expr *) var; + target_entry->resname = + CTranslatorUtils::CreateMultiByteCharStringFromWCString( + sc_proj_elem_dxlop->GetMdNameAlias()->GetMDName()->GetBuffer()); + target_entry->resno = attno; + + // add column mapping to output translation context + output_context->InsertMapping(sc_proj_elem_dxlop->Id(), target_entry); + + plan->targetlist = gpdb::LAppend(plan->targetlist, target_entry); + } + + CDXLTranslationContextArray *child_contexts = + GPOS_NEW(m_mp) CDXLTranslationContextArray(m_mp); + child_contexts->Append(output_context); + + // translate filter + plan->qual = TranslateDXLFilterToQual( + filter_dxlnode, + nullptr, // translate context for the base table + child_contexts, output_context); + + SetParamIds(plan); + + // cleanup + child_contexts->Release(); + + return (Plan *) append; +}; + //--------------------------------------------------------------------------- // @function: // CTranslatorDXLToPlStmt::TranslateDXLMaterialize @@ -4488,6 +4793,71 @@ CTranslatorDXLToPlStmt::TranslateDXLCTEProducerToSharedScan( return (Plan *) shared_input_scan; } +//--------------------------------------------------------------------------- +// @function: +// CTranslatorDXLToPlStmt::TranslateDXLParallelCTEProducerToParallelSharedScan +// +// @doc: +// Translate DXL CTE Producer node into GPDB parallel share input scan plan node +// +//--------------------------------------------------------------------------- +Plan * +CTranslatorDXLToPlStmt::TranslateDXLParallelCTEProducerToParallelSharedScan( + const CDXLNode *cte_producer_dxlnode, CDXLTranslateContext *output_context, + CDXLTranslationContextArray *ctxt_translation_prev_siblings) +{ + CDXLPhysicalParallelCTEProducer *cte_prod_dxlop = + CDXLPhysicalParallelCTEProducer::Cast(cte_producer_dxlnode->GetOperator()); + ULONG cte_id = cte_prod_dxlop->Id(); + ULONG parallel_workers = cte_prod_dxlop->UlParallelWorkers(); + + // create the Share Input Scan representing the CTE Producer + ShareInputScan *shared_input_scan = MakeNode(ShareInputScan); + shared_input_scan->share_id = cte_id; + shared_input_scan->discard_output = true; + + Plan *plan = &(shared_input_scan->scan.plan); + plan->plan_node_id = m_dxl_to_plstmt_context->GetNextPlanId(); + // Set parallel execution flags + plan->parallel_aware = true; + plan->parallel_safe = true; + plan->parallel = (int) parallel_workers; + + m_dxl_to_plstmt_context->RegisterCTEProducerInfo(cte_id, + cte_prod_dxlop->GetOutputColIdxMap(), shared_input_scan); + + // translate cost of the producer + TranslatePlanCosts(cte_producer_dxlnode, plan); + + // translate child plan + CDXLNode *project_list_dxlnode = (*cte_producer_dxlnode)[0]; + CDXLNode *child_dxlnode = (*cte_producer_dxlnode)[1]; + + CDXLTranslateContext child_context(m_mp, false, + output_context->GetColIdToParamIdMap()); + Plan *child_plan = TranslateDXLOperatorToPlan( + child_dxlnode, &child_context, ctxt_translation_prev_siblings); + GPOS_ASSERT(nullptr != child_plan && "child plan cannot be NULL"); + + CDXLTranslationContextArray *child_contexts = + GPOS_NEW(m_mp) CDXLTranslationContextArray(m_mp); + child_contexts->Append(&child_context); + // translate proj list + plan->targetlist = + TranslateDXLProjList(project_list_dxlnode, + nullptr, // base table translation context + child_contexts, output_context); + + plan->lefttree = child_plan; + plan->qual = NIL; + SetParamIds(plan); + + // cleanup + child_contexts->Release(); + + return (Plan *) shared_input_scan; +} + //--------------------------------------------------------------------------- // @function: // CTranslatorDXLToPlStmt::TranslateDXLCTEConsumerToSharedScan @@ -4590,6 +4960,114 @@ CTranslatorDXLToPlStmt::TranslateDXLCTEConsumerToSharedScan( return (Plan *) share_input_scan_cte_consumer; } +//--------------------------------------------------------------------------- +// @function: +// CTranslatorDXLToPlStmt::TranslateDXLParallelCTEConsumerToParallelSharedScan +// +// @doc: +// Translate DXL CTE Consumer node into GPDB parallel share input scan plan node +// +//--------------------------------------------------------------------------- +Plan * +CTranslatorDXLToPlStmt::TranslateDXLParallelCTEConsumerToParallelSharedScan( + const CDXLNode *cte_consumer_dxlnode, CDXLTranslateContext *output_context, + CDXLTranslationContextArray * /*ctxt_translation_prev_siblings*/) +{ + CDXLPhysicalParallelCTEConsumer *cte_consumer_dxlop = + CDXLPhysicalParallelCTEConsumer::Cast(cte_consumer_dxlnode->GetOperator()); + ULONG cte_id = cte_consumer_dxlop->Id(); + ULongPtrArray *output_colidx_map = cte_consumer_dxlop->GetOutputColIdxMap(); + ULONG parallel_workers = cte_consumer_dxlop->UlParallelWorkers(); + + // get the producer idx map + ULongPtrArray *producer_colidx_map; + ShareInputScan *share_input_scan_cte_producer; + + std::tie(producer_colidx_map, share_input_scan_cte_producer) + = m_dxl_to_plstmt_context->GetCTEProducerInfo(cte_id); + + // init the consumer plan + ShareInputScan *share_input_scan_cte_consumer = MakeNode(ShareInputScan); + share_input_scan_cte_consumer->share_id = cte_id; + share_input_scan_cte_consumer->discard_output = false; + + Plan *plan = &(share_input_scan_cte_consumer->scan.plan); + plan->plan_node_id = m_dxl_to_plstmt_context->GetNextPlanId(); + + // Set parallel execution flags + plan->parallel_aware = true; + plan->parallel_safe = true; + plan->parallel = (int) parallel_workers; + + // translate operator costs + TranslatePlanCosts(cte_consumer_dxlnode, plan); + +#ifdef GPOS_DEBUG + ULongPtrArray *output_colids_array = + cte_consumer_dxlop->GetOutputColIdsArray(); +#endif + + // generate the target list of the CTE Consumer + plan->targetlist = NIL; + CDXLNode *project_list_dxlnode = (*cte_consumer_dxlnode)[0]; + const ULONG num_of_proj_list_elem = project_list_dxlnode->Arity(); + GPOS_ASSERT(num_of_proj_list_elem == output_colids_array->Size()); + for (ULONG ul = 0; ul < num_of_proj_list_elem; ul++) + { + AttrNumber varattno = (AttrNumber)ul + 1; + if (output_colidx_map) { + ULONG remapping_idx; + remapping_idx = *(*output_colidx_map)[ul]; + if (producer_colidx_map) { + remapping_idx = *(*producer_colidx_map)[remapping_idx]; + } + GPOS_ASSERT(remapping_idx != gpos::ulong_max); + varattno = (AttrNumber)remapping_idx + 1; + } + + CDXLNode *proj_elem_dxlnode = (*project_list_dxlnode)[ul]; + CDXLScalarProjElem *sc_proj_elem_dxlop = + CDXLScalarProjElem::Cast(proj_elem_dxlnode->GetOperator()); + ULONG colid = sc_proj_elem_dxlop->Id(); + GPOS_ASSERT(colid == *(*output_colids_array)[ul]); + + CDXLNode *sc_ident_dxlnode = (*proj_elem_dxlnode)[0]; + CDXLScalarIdent *sc_ident_dxlop = + CDXLScalarIdent::Cast(sc_ident_dxlnode->GetOperator()); + OID oid_type = CMDIdGPDB::CastMdid(sc_ident_dxlop->MdidType())->Oid(); + + Var *var = + gpdb::MakeVar(OUTER_VAR, varattno, oid_type, + sc_ident_dxlop->TypeModifier(), 0 /* varlevelsup */); + + CHAR *resname = CTranslatorUtils::CreateMultiByteCharStringFromWCString( + sc_proj_elem_dxlop->GetMdNameAlias()->GetMDName()->GetBuffer()); + TargetEntry *target_entry = gpdb::MakeTargetEntry( + (Expr *) var, (AttrNumber)(ul + 1), resname, false /* resjunk */); + plan->targetlist = gpdb::LAppend(plan->targetlist, target_entry); + + output_context->InsertMapping(colid, target_entry); + } + + plan->qual = nullptr; + + SetParamIds(plan); + + // DON'T REMOVE, if current consumer need projection, then we can direct add it. + // we still keep the path of projection in consumer + + // Plan *producer_plan = &(share_input_scan_cte_producer->scan.plan); + // if (output_colidx_map != nullptr) { + // share_input_scan_cte_consumer->need_projection = true; + // share_input_scan_cte_consumer->producer_targetlist = gpdb::ListCopy(producer_plan->targetlist); + // if (!share_input_scan_cte_consumer->producer_targetlist) { + // share_input_scan_cte_consumer->need_projection = false; + // } + // } + + return (Plan *) share_input_scan_cte_consumer; +} + //--------------------------------------------------------------------------- // @function: // CTranslatorDXLToPlStmt::TranslateDXLSequence @@ -4647,6 +5125,72 @@ CTranslatorDXLToPlStmt::TranslateDXLSequence( return (Plan *) psequence; } +//--------------------------------------------------------------------------- +// @function: +// CTranslatorDXLToPlStmt::TranslateDXLParallelSequence +// +// @doc: +// Translate DXL sequence node into GPDB parallel Sequence plan node +// +//--------------------------------------------------------------------------- +Plan * +CTranslatorDXLToPlStmt::TranslateDXLParallelSequence( + const CDXLNode *sequence_dxlnode, CDXLTranslateContext *output_context, + CDXLTranslationContextArray *ctxt_translation_prev_siblings) +{ + CDXLPhysicalParallelSequence *phy_parallel_sequence_dxlop = + CDXLPhysicalParallelSequence::Cast(sequence_dxlnode->GetOperator()); + + ULONG parallel_workers = phy_parallel_sequence_dxlop->UlParallelWorkers(); + + // create append plan node + Sequence *psequence = MakeNode(Sequence); + + Plan *plan = &(psequence->plan); + plan->plan_node_id = m_dxl_to_plstmt_context->GetNextPlanId(); + + plan->parallel_aware = true; + plan->parallel_safe = true; + plan->parallel = (int) parallel_workers; + + // translate operator costs + TranslatePlanCosts(sequence_dxlnode, plan); + + ULONG arity = sequence_dxlnode->Arity(); + + CDXLTranslateContext child_context(m_mp, false, + output_context->GetColIdToParamIdMap()); + + for (ULONG ul = 1; ul < arity; ul++) + { + CDXLNode *child_dxlnode = (*sequence_dxlnode)[ul]; + + Plan *child_plan = TranslateDXLOperatorToPlan( + child_dxlnode, &child_context, ctxt_translation_prev_siblings); + + psequence->subplans = gpdb::LAppend(psequence->subplans, child_plan); + } + + CDXLNode *project_list_dxlnode = (*sequence_dxlnode)[0]; + + CDXLTranslationContextArray *child_contexts = + GPOS_NEW(m_mp) CDXLTranslationContextArray(m_mp); + child_contexts->Append(&child_context); + + // translate proj list + plan->targetlist = + TranslateDXLProjList(project_list_dxlnode, + nullptr, // base table translation context + child_contexts, output_context); + + SetParamIds(plan); + + // cleanup + child_contexts->Release(); + + return (Plan *) psequence; +} + //--------------------------------------------------------------------------- // @function: // CTranslatorDXLToPlStmt::TranslateDXLDynTblScan @@ -7282,4 +7826,75 @@ CTranslatorDXLToPlStmt::IsIndexForOrderBy( } return false; } + +//--------------------------------------------------------------------------- +// @function: +// CTranslatorDXLToPlStmt::ExtractParallelWorkersFromDXL +// +// @doc: +// Extract parallel workers count from DXL node tree recursively. +// Since parallel degree is uniform across all parallel scans in a query, +// returns the first parallel degree found from any CDXLPhysicalParallelTableScan, +// or 1 if no parallel scan exists. +// +//--------------------------------------------------------------------------- +ULONG +CTranslatorDXLToPlStmt::ExtractParallelWorkersFromDXL(const CDXLNode *dxlnode) +{ + if (nullptr == dxlnode) + { + return 1; + } + + CDXLOperator *dxlop = dxlnode->GetOperator(); + if (EdxlopPhysicalParallelTableScan == dxlop->GetDXLOperator()) + { + // Return parallel workers from the parallel table scan operator + // All parallel scans in the query share the same parallel degree + CDXLPhysicalParallelTableScan *parallel_scan_dxlop = + CDXLPhysicalParallelTableScan::Cast(dxlop); + return parallel_scan_dxlop->UlParallelWorkers(); + } + else if (EdxlopPhysicalTableScan == dxlop->GetDXLOperator() || + EdxlopPhysicalDynamicTableScan == dxlop->GetDXLOperator() || + EdxlopPhysicalIndexScan == dxlop->GetDXLOperator() || + EdxlopPhysicalIndexOnlyScan == dxlop->GetDXLOperator() || + EdxlopPhysicalBitmapTableScan == dxlop->GetDXLOperator() || + EdxlopPhysicalDynamicBitmapTableScan == dxlop->GetDXLOperator() || + EdxlopPhysicalForeignScan == dxlop->GetDXLOperator() || + EdxlopPhysicalDynamicForeignScan == dxlop->GetDXLOperator() || + EdxlopPhysicalDynamicIndexScan == dxlop->GetDXLOperator() || + EdxlopPhysicalDynamicIndexOnlyScan == dxlop->GetDXLOperator() || + EdxlopPhysicalValuesScan == dxlop->GetDXLOperator()) + { + // Non-parallel scans (table, index, bitmap, foreign, values) + // These are leaf nodes in terms of parallel worker extraction + // Return 1 to indicate no parallel workers + return 1; + } + else if (EdxlopPhysicalMotionGather == dxlop->GetDXLOperator() || + EdxlopPhysicalMotionBroadcast == dxlop->GetDXLOperator() || + EdxlopPhysicalMotionRedistribute == dxlop->GetDXLOperator() || + EdxlopPhysicalMotionRandom == dxlop->GetDXLOperator() || + EdxlopPhysicalMotionRoutedDistribute == dxlop->GetDXLOperator()) + { + // Motion node creates a slice boundary - do not recurse into child + // The child's parallel workers belong to the sending slice, not receiving slice + // Return 0 to indicate the receiving slice (current slice) has no parallel workers + return 1; + } + + // Recursively check child nodes, return early when first parallel scan is found + for (ULONG ul = 0; ul < dxlnode->Arity(); ul++) + { + ULONG child_parallel_workers = ExtractParallelWorkersFromDXL((*dxlnode)[ul]); + if (child_parallel_workers > 1) + { + return child_parallel_workers; + } + } + + return 1; +} + // EOF diff --git a/src/backend/gpopt/translate/CTranslatorRelcacheToDXL.cpp b/src/backend/gpopt/translate/CTranslatorRelcacheToDXL.cpp index 469d69fb60f..a72780a3b33 100644 --- a/src/backend/gpopt/translate/CTranslatorRelcacheToDXL.cpp +++ b/src/backend/gpopt/translate/CTranslatorRelcacheToDXL.cpp @@ -516,7 +516,6 @@ CTranslatorRelcacheToDXL::RetrieveRel(CMemoryPool *mp, CMDAccessor *md_accessor, IMdIdArray *check_constraint_mdids = nullptr; BOOL is_temporary = false; BOOL is_partitioned = false; - IMDRelation *md_rel = nullptr; IMdIdArray *partition_oids = nullptr; IMDId *foreign_server_mdid = nullptr; @@ -618,14 +617,31 @@ CTranslatorRelcacheToDXL::RetrieveRel(CMemoryPool *mp, CMDAccessor *md_accessor, CMDIdGPDB(IMDId::EmdidGeneral, gpdb::GetForeignServerId(oid)); } - md_rel = GPOS_NEW(mp) CMDRelationGPDB( + CMDRelationGPDB *md_rel_gpdb = GPOS_NEW(mp) CMDRelationGPDB( mp, mdid, mdname, is_temporary, rel_storage_type, dist, mdcol_array, distr_cols, distr_op_families, part_keys, part_types, partition_oids, convert_hash_to_random, keyset_array, md_index_info_array, check_constraint_mdids, mdpart_constraint, foreign_server_mdid, rel->rd_rel->reltuples); - return md_rel; + // Set segment file count for AO/AOCO tables + // Skip partitioned tables as they don't have physical storage (only leaf partitions do) + if ((rel_storage_type == IMDRelation::ErelstorageAppendOnlyRows || + rel_storage_type == IMDRelation::ErelstorageAppendOnlyCols) && + rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE) + { + INT seg_file_count = gpdb::GetAppendOnlySegmentFilesCount(rel.get()); + md_rel_gpdb->SetSegFileCount(seg_file_count); + } + + // Set parallel workers from table options + if (rel->rd_options != NULL) + { + INT parallel_workers = RelationGetParallelWorkers(rel.get(), -1); + md_rel_gpdb->SetParallelWorkers(parallel_workers); + } + + return md_rel_gpdb; } //--------------------------------------------------------------------------- diff --git a/src/backend/gpopt/utils/COptTasks.cpp b/src/backend/gpopt/utils/COptTasks.cpp index dd0e61116d8..ef29bb026f1 100644 --- a/src/backend/gpopt/utils/COptTasks.cpp +++ b/src/backend/gpopt/utils/COptTasks.cpp @@ -364,7 +364,7 @@ COptTasks::LoadSearchStrategy(CMemoryPool *mp, char *path) //--------------------------------------------------------------------------- COptimizerConfig * COptTasks::CreateOptimizerConfig(CMemoryPool *mp, ICostModel *cost_model, - CPlanHint *plan_hints) + CPlanHint *plan_hints, BOOL enable_parallel_plans) { // get chosen plan number, cost threshold ULLONG plan_id = (ULLONG) optimizer_plan_id; @@ -403,7 +403,8 @@ COptTasks::CreateOptimizerConfig(CMemoryPool *mp, ICostModel *cost_model, push_group_by_below_setop_threshold, xform_bind_threshold, skew_factor), plan_hints, - GPOS_NEW(mp) CWindowOids(mp, OID(F_ROW_NUMBER), OID(F_RANK_), OID(F_DENSE_RANK_))); + GPOS_NEW(mp) CWindowOids(mp, OID(F_ROW_NUMBER), OID(F_RANK_), OID(F_DENSE_RANK_)), + enable_parallel_plans); } //--------------------------------------------------------------------------- @@ -940,7 +941,7 @@ COptTasks::OptimizeTask(void *ptr) ICostModel *cost_model = GetCostModel(mp, num_segments_for_costing); CPlanHint *plan_hints = GetPlanHints(mp, opt_ctxt->m_query); COptimizerConfig *optimizer_config = - CreateOptimizerConfig(mp, cost_model, plan_hints); + CreateOptimizerConfig(mp, cost_model, plan_hints, opt_ctxt->m_create_parallel_plan); CConstExprEvaluatorProxy expr_eval_proxy(mp, &mda); IConstExprEvaluator *expr_evaluator = GPOS_NEW(mp) CConstExprEvaluatorDXL(mp, &mda, &expr_eval_proxy); @@ -1165,6 +1166,7 @@ COptTasks::GPOPTOptimizedPlan(Query *query, SOptContext *gpopt_context, Optimize gpopt_context->m_should_generate_plan_stmt = true; // Copy options in `OptimizerOptions` to `SOptContext` gpopt_context->m_create_vec_plan = opts->create_vectorization_plan; + gpopt_context->m_create_parallel_plan = opts->create_parallel_plan; Execute(&OptimizeTask, gpopt_context); return gpopt_context->m_plan_stmt; } diff --git a/src/backend/gporca/libgpdbcost/include/gpdbcost/CCostModelGPDB.h b/src/backend/gporca/libgpdbcost/include/gpdbcost/CCostModelGPDB.h index 2b44693fa4e..745292f5377 100644 --- a/src/backend/gporca/libgpdbcost/include/gpdbcost/CCostModelGPDB.h +++ b/src/backend/gporca/libgpdbcost/include/gpdbcost/CCostModelGPDB.h @@ -80,6 +80,11 @@ class CCostModelGPDB : public ICostModel const CCostModelGPDB *pcmgpdb, const SCostingInfo *pci); + // cost of parallel table scan + static CCost CostParallelTableScan(CMemoryPool *mp, CExpressionHandle &exprhdl, + const CCostModelGPDB *pcmgpdb, + const SCostingInfo *pci); + // cost of filter static CCost CostFilter(CMemoryPool *mp, CExpressionHandle &exprhdl, const CCostModelGPDB *pcmgpdb, @@ -118,11 +123,21 @@ class CCostModelGPDB : public ICostModel const CCostModelGPDB *pcmgpdb, const SCostingInfo *pci); + // cost of parallel CTE producer + static CCost CostParallelCTEProducer(CMemoryPool *mp, CExpressionHandle &exprhdl, + const CCostModelGPDB *pcmgpdb, + const SCostingInfo *pci); + // cost of CTE consumer static CCost CostCTEConsumer(CMemoryPool *mp, CExpressionHandle &exprhdl, const CCostModelGPDB *pcmgpdb, const SCostingInfo *pci); + // cost of parallel CTE consumer + static CCost CostParallelCTEConsumer(CMemoryPool *mp, CExpressionHandle &exprhdl, + const CCostModelGPDB *pcmgpdb, + const SCostingInfo *pci); + // cost of const table get static CCost CostConstTableGet(CMemoryPool *mp, CExpressionHandle &exprhdl, const CCostModelGPDB *pcmgpdb, @@ -153,6 +168,11 @@ class CCostModelGPDB : public ICostModel const CCostModelGPDB *pcmgpdb, const SCostingInfo *pci); + // cost of paralllel sequence + static CCost CostParallelSequence(CMemoryPool *mp, CExpressionHandle &exprhdl, + const CCostModelGPDB *pcmgpdb, + const SCostingInfo *pci); + // cost of sort static CCost CostSort(CMemoryPool *mp, CExpressionHandle &exprhdl, const CCostModelGPDB *pcmgpdb, @@ -225,6 +245,10 @@ class CCostModelGPDB : public ICostModel IStatistics *&stats, CMDAccessor *md_accessor, CMemoryPool *mp); + // Helper functions for parallel cost calculation + static CDouble CalculateParallelEfficiency(ULONG ulWorkers); + static CDouble GetWorkerStartupCost(const CCostModelGPDB *pcmgpdb, ULONG ulWorkers); + public: // ctor CCostModelGPDB(CMemoryPool *mp, ULONG ulSegments, diff --git a/src/backend/gporca/libgpdbcost/src/CCostModelGPDB.cpp b/src/backend/gporca/libgpdbcost/src/CCostModelGPDB.cpp index 73330059c72..3386143e394 100644 --- a/src/backend/gporca/libgpdbcost/src/CCostModelGPDB.cpp +++ b/src/backend/gporca/libgpdbcost/src/CCostModelGPDB.cpp @@ -12,6 +12,7 @@ #include "gpdbcost/CCostModelGPDB.h" #include +#include #include "gpopt/base/CColRefSetIter.h" #include "gpopt/base/COptCtxt.h" @@ -27,6 +28,10 @@ #include "gpopt/operators/CPhysicalHashAgg.h" #include "gpopt/operators/CPhysicalIndexOnlyScan.h" #include "gpopt/operators/CPhysicalIndexScan.h" +#include "gpopt/operators/CPhysicalParallelTableScan.h" +#include "gpopt/operators/CPhysicalParallelSequence.h" +#include "gpopt/operators/CPhysicalParallelCTEConsumer.h" +#include "gpopt/operators/CPhysicalParallelCTEProducer.h" #include "gpopt/operators/CPhysicalMotion.h" #include "gpopt/operators/CPhysicalMotionBroadcast.h" #include "gpopt/operators/CPhysicalPartitionSelector.h" @@ -43,6 +48,8 @@ using namespace gpos; using namespace gpdbcost; +// Forward declare PostgreSQL GUC variables +extern double parallel_setup_cost; //--------------------------------------------------------------------------- // @function: @@ -438,6 +445,77 @@ CCostModelGPDB::CostCTEProducer(CMemoryPool *mp, CExpressionHandle &exprhdl, } +//--------------------------------------------------------------------------- +// @function: +// CCostModelGPDB::CostParallelCTEProducer +// +// @doc: +// Cost of parallel CTE producer +// +//--------------------------------------------------------------------------- +CCost +CCostModelGPDB::CostParallelCTEProducer(CMemoryPool *mp, CExpressionHandle &exprhdl, + const CCostModelGPDB *pcmgpdb, + const SCostingInfo *pci) +{ + GPOS_ASSERT(nullptr != pcmgpdb); + GPOS_ASSERT(nullptr != pci); + COperator *pop = exprhdl.Pop(); + GPOS_ASSERT(COperator::EopPhysicalParallelCTEProducer == pop->Eopid()); + + // Get the parallel cte producer operator + CPhysicalParallelCTEProducer *popCTEProducer = + CPhysicalParallelCTEProducer::PopConvert(pop); + ULONG ulWorkers = popCTEProducer->UlParallelWorkers(); + + CCost cost = CostUnary(mp, exprhdl, pci, pcmgpdb->GetCostModelParams()); + + // In GPDB, the child of a ShareInputScan representing the producer can + // only be a materialize or sort. Here, we check if a materialize node + // needs to be added during DXL->PlStmt translation + + COperator *popChild = exprhdl.Pop(0 /*child_index*/); + if (nullptr == popChild) + { + // child operator is not known, this could happen when computing cost bound + return cost; + } + + COperator::EOperatorId op_id = popChild->Eopid(); + if (COperator::EopPhysicalSpool != op_id && + COperator::EopPhysicalSort != op_id) + { + // no materialize needed + return CCost(cost/ulWorkers); + } + + // a materialize (spool) node is added during DXL->PlStmt translation, + // we need to add the cost of writing the tuples to disk + const CDouble dMaterializeCostUnit = + pcmgpdb->GetCostModelParams() + ->PcpLookup(CCostModelParamsGPDB::EcpMaterializeCostUnit) + ->Get(); + GPOS_ASSERT(0 < dMaterializeCostUnit); + + + // Calculate base scan cost + CDouble dBaseCost = pci->Rows() * pci->Width() * dMaterializeCostUnit; + + // Calculate parallel efficiency (decreases with more workers) + CDouble dParallelEfficiency = CalculateParallelEfficiency(ulWorkers); + + // Parallel scan cost = base cost / (workers * efficiency) + CDouble dParallelCost = dBaseCost / (ulWorkers * dParallelEfficiency); + + // Add worker startup cost + CDouble dWorkerStartupCost = GetWorkerStartupCost(pcmgpdb, ulWorkers); + + CCost costSpooling = CCost(pci->NumRebinds() * (dParallelCost + dWorkerStartupCost)); + + return cost + costSpooling; +} + + //--------------------------------------------------------------------------- // @function: // CCostModelGPDB::CostCTEConsumer @@ -480,6 +558,62 @@ CCostModelGPDB::CostCTEConsumer(CMemoryPool *, // mp (dTableScanCostUnit + dOutputTupCostUnit))); } +//--------------------------------------------------------------------------- +// @function: +// CCostModelGPDB::CostParallelCTEConsumer +// +// @doc: +// Cost of parallel CTE consumer +// +//--------------------------------------------------------------------------- +CCost +CCostModelGPDB::CostParallelCTEConsumer(CMemoryPool *, // mp + CExpressionHandle &exprhdl, + const CCostModelGPDB *pcmgpdb, + const SCostingInfo *pci) +{ + GPOS_ASSERT(nullptr != pcmgpdb); + GPOS_ASSERT(nullptr != pci); + + COperator *pop = exprhdl.Pop(); + GPOS_ASSERT(COperator::EopPhysicalParallelCTEConsumer == pop->Eopid()); + + // Get the parallel cte consumer operator + CPhysicalParallelCTEConsumer *popCTEConsumer = + CPhysicalParallelCTEConsumer::PopConvert(pop); + ULONG ulWorkers = popCTEConsumer->UlParallelWorkers(); + + const CDouble dInitScan = + pcmgpdb->GetCostModelParams() + ->PcpLookup(CCostModelParamsGPDB::EcpInitScanFactor) + ->Get(); + const CDouble dTableScanCostUnit = + pcmgpdb->GetCostModelParams() + ->PcpLookup(CCostModelParamsGPDB::EcpTableScanCostUnit) + ->Get(); + const CDouble dOutputTupCostUnit = + pcmgpdb->GetCostModelParams() + ->PcpLookup(CCostModelParamsGPDB::EcpOutputTupCostUnit) + ->Get(); + GPOS_ASSERT(0 < dOutputTupCostUnit); + GPOS_ASSERT(0 < dTableScanCostUnit); + + // Calculate base scan cost + CDouble dBaseCost = dInitScan + pci->Rows() * pci->Width() * + (dTableScanCostUnit + dOutputTupCostUnit); + + // Calculate parallel efficiency (decreases with more workers) + CDouble dParallelEfficiency = CalculateParallelEfficiency(ulWorkers); + + // Parallel scan cost = base cost / (workers * efficiency) + CDouble dParallelCost = dBaseCost / (ulWorkers * dParallelEfficiency); + + // Add worker startup cost + CDouble dWorkerStartupCost = GetWorkerStartupCost(pcmgpdb, ulWorkers); + + return CCost(pci->NumRebinds() * (dParallelCost + dWorkerStartupCost)); +} + //--------------------------------------------------------------------------- // @function: @@ -681,6 +815,47 @@ CCostModelGPDB::CostSequence(CMemoryPool *mp, CExpressionHandle &exprhdl, return costLocal + costChild; } +//--------------------------------------------------------------------------- +// @function: +// CCostModelGPDB::CostParallelSequence +// +// @doc: +// Cost of parallel sequence +// +//--------------------------------------------------------------------------- +CCost +CCostModelGPDB::CostParallelSequence(CMemoryPool *mp, CExpressionHandle &exprhdl, + const CCostModelGPDB *pcmgpdb, + const SCostingInfo *pci) +{ + GPOS_ASSERT(nullptr != pcmgpdb); + GPOS_ASSERT(nullptr != pci); + + COperator *pop = exprhdl.Pop(); + GPOS_ASSERT(COperator::EopPhysicalParallelSequence == pop->Eopid()); + + // Get the parallel sequence operator + CPhysicalParallelSequence *popParallelSequence = + CPhysicalParallelSequence::PopConvert(pop); + ULONG ulWorkers = popParallelSequence->UlParallelWorkers(); + + // If only 1 worker, use regular scan cost + if (ulWorkers <= 1) + { + return CostSequence(mp, exprhdl, pcmgpdb, pci); + } + + CCost costLocal = CCost(pci->NumRebinds() * + CostTupleProcessing(pci->Rows(), pci->Width(), + pcmgpdb->GetCostModelParams()) + .Get()); + + CCost costChild = + CostChildren(mp, exprhdl, pci, pcmgpdb->GetCostModelParams()); + + return CCost((costLocal + costChild)/ulWorkers); +} + //--------------------------------------------------------------------------- // @function: @@ -2374,7 +2549,10 @@ CCostModelGPDB::CostScan(CMemoryPool *, // mp GPOS_ASSERT(COperator::EopPhysicalTableScan == op_id || COperator::EopPhysicalDynamicTableScan == op_id || COperator::EopPhysicalForeignScan == op_id || - COperator::EopPhysicalDynamicForeignScan == op_id); + COperator::EopPhysicalDynamicForeignScan == op_id || + COperator::EopPhysicalParallelTableScan == op_id || + COperator::EopPhysicalAppendTableScan == op_id || + COperator::EopPhysicalParallelAppendTableScan == op_id); const CDouble dInitScan = pcmgpdb->GetCostModelParams() @@ -2396,6 +2574,8 @@ CCostModelGPDB::CostScan(CMemoryPool *, // mp case COperator::EopPhysicalDynamicTableScan: case COperator::EopPhysicalForeignScan: case COperator::EopPhysicalDynamicForeignScan: + case COperator::EopPhysicalParallelTableScan: + case COperator::EopPhysicalAppendTableScan: // table scan cost considers only retrieving tuple cost, // since we scan the entire table here, the cost is correlated with table rows and table width, // since Scan's parent operator may be a filter that will be pushed into Scan node in GPDB plan, @@ -2403,6 +2583,10 @@ CCostModelGPDB::CostScan(CMemoryPool *, // mp return CCost( pci->NumRebinds() * (dInitScan + pci->Rows() * dTableWidth * dTableScanCostUnit)); + case COperator::EopPhysicalParallelAppendTableScan: + return CCost( + pci->NumRebinds() * + (dInitScan + pci->Rows() * dTableWidth * dTableScanCostUnit) - 10); default: GPOS_ASSERT(!"invalid index scan"); return CCost(0); @@ -2410,6 +2594,120 @@ CCostModelGPDB::CostScan(CMemoryPool *, // mp } +//--------------------------------------------------------------------------- +// @function: +// CCostModelGPDB::CostParallelTableScan +// +// @doc: +// Cost of parallel table scan +// +//--------------------------------------------------------------------------- +CCost +CCostModelGPDB::CostParallelTableScan(CMemoryPool *mp, + CExpressionHandle &exprhdl, + const CCostModelGPDB *pcmgpdb, + const SCostingInfo *pci) +{ + GPOS_ASSERT(nullptr != pcmgpdb); + GPOS_ASSERT(nullptr != pci); + + COperator *pop = exprhdl.Pop(); + GPOS_ASSERT(COperator::EopPhysicalParallelTableScan == pop->Eopid()); + + // Get the parallel table scan operator + CPhysicalParallelTableScan *popParallelScan = + CPhysicalParallelTableScan::PopConvert(pop); + ULONG ulWorkers = popParallelScan->UlParallelWorkers(); + + // If only 1 worker, use regular scan cost + if (ulWorkers <= 1) + { + return CostScan(mp, exprhdl, pcmgpdb, pci); + } + + // Get base scan parameters + const CDouble dInitScan = + pcmgpdb->GetCostModelParams() + ->PcpLookup(CCostModelParamsGPDB::EcpInitScanFactor) + ->Get(); + const CDouble dTableWidth = + CPhysicalScan::PopConvert(pop)->PstatsBaseTable()->Width(); + const CDouble dTableScanCostUnit = + pcmgpdb->GetCostModelParams() + ->PcpLookup(CCostModelParamsGPDB::EcpTableScanCostUnit) + ->Get(); + + // Calculate base scan cost + CDouble dBaseScanCost = dInitScan + pci->Rows() * dTableWidth * dTableScanCostUnit; + + // Calculate parallel efficiency (decreases with more workers) + CDouble dParallelEfficiency = CalculateParallelEfficiency(ulWorkers); + + // Parallel scan cost = base cost / (workers * efficiency) + CDouble dParallelScanCost = dBaseScanCost / (ulWorkers * dParallelEfficiency); + + // Add worker startup cost + CDouble dWorkerStartupCost = GetWorkerStartupCost(pcmgpdb, ulWorkers); + + // Total cost + return CCost(pci->NumRebinds() * (dParallelScanCost + dWorkerStartupCost)); +} + +//--------------------------------------------------------------------------- +// @function: +// CCostModelGPDB::CalculateParallelEfficiency +// +// @doc: +// Calculate parallel efficiency factor (0-1) based on worker count +// +//--------------------------------------------------------------------------- +CDouble +CCostModelGPDB::CalculateParallelEfficiency(ULONG ulWorkers) +{ + if (ulWorkers <= 1) + { + return 1.0; + } + + // Efficiency decreases logarithmically with more workers + // Formula: efficiency = 1 / (1 + 0.1 * log2(workers)) + // This gives: 2 workers = 0.91, 4 workers = 0.83, 8 workers = 0.77 + double dLogWorkers = std::log2(static_cast(ulWorkers)); + return CDouble(1.0 / (1.0 + 0.1 * dLogWorkers)); +} + + +//--------------------------------------------------------------------------- +// @function: +// CCostModelGPDB::GetWorkerStartupCost +// +// @doc: +// Get the cost of starting up parallel workers +// +//--------------------------------------------------------------------------- +CDouble +CCostModelGPDB::GetWorkerStartupCost(const CCostModelGPDB * /* pcmgpdb */, ULONG ulWorkers) +{ + if (ulWorkers <= 1) + { + return 0.0; + } + + // ORCA's cost units are much smaller than PostgreSQL's cost model + // PostgreSQL's parallel_setup_cost default is 1000, but ORCA's costs are: + // - InitScanFactor: 431.0 + // - HJHashTableInitCostFactor: 500.0 + // - DefaultCost: 100.0 + // + // Use a conversion factor to map parallel_setup_cost to ORCA's scale. + // With default parallel_setup_cost=1000, this gives 10.0, which is + // reasonable compared to InitScanFactor (431.0) - about 0.1% overhead + const double POSTGRES_TO_ORCA_COST_CONVERSION = 0.001; + + return CDouble(parallel_setup_cost * POSTGRES_TO_ORCA_COST_CONVERSION); +} + + //--------------------------------------------------------------------------- // @function: // CCostModelGPDB::CostFilter @@ -2483,11 +2781,18 @@ CCostModelGPDB::Cost( case COperator::EopPhysicalDynamicTableScan: case COperator::EopPhysicalForeignScan: case COperator::EopPhysicalDynamicForeignScan: - + case COperator::EopPhysicalAppendTableScan: + case COperator::EopPhysicalParallelAppendTableScan: { return CostScan(m_mp, exprhdl, this, pci); } + case COperator::EopPhysicalParallelTableScan: + //case COperator::EopPhysicalParallelAppendTableScan: + { + return CostParallelTableScan(m_mp, exprhdl, this, pci); + } + case COperator::EopPhysicalFilter: { return CostFilter(m_mp, exprhdl, this, pci); @@ -2526,11 +2831,21 @@ CCostModelGPDB::Cost( return CostCTEProducer(m_mp, exprhdl, this, pci); } + case COperator::EopPhysicalParallelCTEProducer: + { + return CostParallelCTEProducer(m_mp, exprhdl, this, pci); + } + case COperator::EopPhysicalCTEConsumer: { return CostCTEConsumer(m_mp, exprhdl, this, pci); } + case COperator::EopPhysicalParallelCTEConsumer: + { + return CostParallelCTEConsumer(m_mp, exprhdl, this, pci); + } + case COperator::EopPhysicalConstTableGet: { return CostConstTableGet(m_mp, exprhdl, this, pci); @@ -2562,6 +2877,10 @@ CCostModelGPDB::Cost( { return CostSequence(m_mp, exprhdl, this, pci); } + case COperator::EopPhysicalParallelSequence: + { + return CostParallelSequence(m_mp, exprhdl, this, pci); + } case COperator::EopPhysicalSort: { diff --git a/src/backend/gporca/libgpopt/include/gpopt/base/CDistributionSpec.h b/src/backend/gporca/libgpopt/include/gpopt/base/CDistributionSpec.h index d3b47b95f80..9f3b74bd76f 100644 --- a/src/backend/gporca/libgpopt/include/gpopt/base/CDistributionSpec.h +++ b/src/backend/gporca/libgpopt/include/gpopt/base/CDistributionSpec.h @@ -71,6 +71,7 @@ class CDistributionSpec : public CPropSpec EdtStrictSingleton, // data is on a single segment or the master (derived only, only compatible with other singleton distributions) EdtRandom, // data is randomly distributed across all segments EdtStrictRandom, // same as random, used to force multiple slices for parallel union all. + EdtWorkerRandom, // data is randomly distributed among parallel workers within segments EdtRouted, // data is routed to a segment explicitly specified in the tuple, EdtUniversal, // data is available everywhere (derived only) EdtNonSingleton, // data can have any distribution except singleton (required only) diff --git a/src/backend/gporca/libgpopt/include/gpopt/base/CDistributionSpecWorkerRandom.h b/src/backend/gporca/libgpopt/include/gpopt/base/CDistributionSpecWorkerRandom.h new file mode 100644 index 00000000000..f026f00e247 --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/base/CDistributionSpecWorkerRandom.h @@ -0,0 +1,139 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CDistributionSpecWorkerRandom.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/base/CDistributionSpecWorkerRandom.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPOPT_CDistributionSpecWorkerRandom_H +#define GPOPT_CDistributionSpecWorkerRandom_H + +#include "gpos/base.h" + +#include "gpopt/base/CDistributionSpecRandom.h" + +namespace gpopt +{ +using namespace gpos; + +//--------------------------------------------------------------------------- +// @class: +// CDistributionSpecWorkerRandom +// +// @doc: +// Class for representing worker-level random distribution. +// This class provides a specialized implementation for parallel +// worker execution with explicit worker count management. +// +//--------------------------------------------------------------------------- +class CDistributionSpecWorkerRandom : public CDistributionSpecRandom +{ +private: + // Number of workers for parallel execution + ULONG m_ulWorkers; + + // Base segment distribution (usually segment-level random) + CDistributionSpec *m_pdsSegmentBase; + + // private copy ctor + CDistributionSpecWorkerRandom(const CDistributionSpecWorkerRandom &); + +public: + // ctor + CDistributionSpecWorkerRandom(ULONG ulWorkers, CDistributionSpec *pdsSegmentBase = nullptr); + + // dtor + ~CDistributionSpecWorkerRandom() override; + + // distribution type accessor + EDistributionType + Edt() const override + { + return CDistributionSpec::EdtWorkerRandom; + } + + // distribution identifier + const CHAR * + SzId() const override + { + return "WORKER_RANDOM"; + } + + // Get worker count + ULONG + UlWorkers() const + { + return m_ulWorkers; + } + + // Get base segment distribution + CDistributionSpec * + PdsSegmentBase() const + { + return m_pdsSegmentBase; + } + + // does this distribution match the given one + BOOL Matches(const CDistributionSpec *pds) const override; + + // does this distribution satisfy the given one + BOOL FSatisfies(const CDistributionSpec *pds) const override; + + // append enforcers to dynamic array for the given plan properties + void AppendEnforcers(CMemoryPool *mp, CExpressionHandle &exprhdl, + CReqdPropPlan *prpp, CExpressionArray *pdrgpexpr, + CExpression *pexpr) override; + + // print + IOstream &OsPrint(IOstream &os) const override; + + // Factory method for creating worker-level random distribution + static CDistributionSpecWorkerRandom *PdsCreateWorkerRandom( + CMemoryPool *mp, ULONG ulWorkers, CDistributionSpec *pdsBase = nullptr); + + // conversion function + static CDistributionSpecWorkerRandom * + PdsConvert(CDistributionSpec *pds) + { + GPOS_ASSERT(nullptr != pds); + GPOS_ASSERT(EdtWorkerRandom == pds->Edt()); + + return dynamic_cast(pds); + } + + // conversion function: const argument + static const CDistributionSpecWorkerRandom * + PdsConvert(const CDistributionSpec *pds) + { + GPOS_ASSERT(nullptr != pds); + GPOS_ASSERT(EdtWorkerRandom == pds->Edt()); + + return dynamic_cast(pds); + } + +}; // class CDistributionSpecWorkerRandom + +} // namespace gpopt + +#endif // !GPOPT_CDistributionSpecWorkerRandom_H + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/include/gpopt/base/CDrvdPropPlan.h b/src/backend/gporca/libgpopt/include/gpopt/base/CDrvdPropPlan.h index 103d8aa22d2..a0283c70363 100644 --- a/src/backend/gporca/libgpopt/include/gpopt/base/CDrvdPropPlan.h +++ b/src/backend/gporca/libgpopt/include/gpopt/base/CDrvdPropPlan.h @@ -64,6 +64,9 @@ class CDrvdPropPlan : public CDrvdProp void CopyCTEProducerPlanProps(CMemoryPool *mp, CDrvdPropCtxt *pdpctxt, COperator *pop); + // copy parallel CTE producer plan properties from given context to current object + void CopyParallelCTEProducerPlanProps(CMemoryPool *mp, CDrvdPropCtxt *pdpctxt, + COperator *pop); public: CDrvdPropPlan(const CDrvdPropPlan &) = delete; diff --git a/src/backend/gporca/libgpopt/include/gpopt/base/CRewindabilitySpec.h b/src/backend/gporca/libgpopt/include/gpopt/base/CRewindabilitySpec.h index bd2dea7eec1..2959a678301 100644 --- a/src/backend/gporca/libgpopt/include/gpopt/base/CRewindabilitySpec.h +++ b/src/backend/gporca/libgpopt/include/gpopt/base/CRewindabilitySpec.h @@ -108,6 +108,7 @@ class CRewindabilitySpec : public CPropSpec EmhtSentinel }; + private: // rewindability support ERewindabilityType m_rewindability; diff --git a/src/backend/gporca/libgpopt/include/gpopt/operators/COperator.h b/src/backend/gporca/libgpopt/include/gpopt/operators/COperator.h index 657e3082ef7..9c3253072be 100644 --- a/src/backend/gporca/libgpopt/include/gpopt/operators/COperator.h +++ b/src/backend/gporca/libgpopt/include/gpopt/operators/COperator.h @@ -188,6 +188,7 @@ class COperator : public CRefCount, public DbgPrintMixin EopScalarFieldSelect, EopPhysicalTableScan, + EopPhysicalParallelTableScan, EopPhysicalForeignScan, EopPhysicalIndexScan, EopPhysicalIndexOnlyScan, @@ -209,13 +210,19 @@ class COperator : public CRefCount, public DbgPrintMixin EopPhysicalFullMergeJoin, EopPhysicalDynamicTableScan, EopPhysicalSequence, + EopPhysicalParallelSequence, EopPhysicalTVF, EopPhysicalCTEProducer, + EopPhysicalParallelCTEProducer, EopPhysicalCTEConsumer, + EopPhysicalParallelCTEConsumer, EopPhysicalSequenceProject, EopPhysicalHashSequenceProject, EopPhysicalDynamicIndexScan, + EopPhysicalAppendTableScan, + EopPhysicalParallelAppendTableScan, + EopPhysicalInnerHashJoin, EopPhysicalLeftOuterHashJoin, EopPhysicalLeftSemiHashJoin, diff --git a/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalAppendTableScan.h b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalAppendTableScan.h new file mode 100644 index 00000000000..a321f8ffc64 --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalAppendTableScan.h @@ -0,0 +1,99 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalAppendTableScan.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalAppendTableScan.h + * + *------------------------------------------------------------------------- + */ + +#ifndef GPOPT_CPhysicalAppendTableScan_H +#define GPOPT_CPhysicalAppendTableScan_H + +#include "gpos/base.h" + +#include "gpopt/operators/CPhysicalDynamicScan.h" + +namespace gpopt +{ +//--------------------------------------------------------------------------- +// @class: +// CPhysicalAppendTableScan +// +// @doc: +// Append Table scan operator +// +//--------------------------------------------------------------------------- +class CPhysicalAppendTableScan : public CPhysicalDynamicScan +{ +private: +public: + CPhysicalAppendTableScan(const CPhysicalAppendTableScan &) = delete; + + // ctors + CPhysicalAppendTableScan(CMemoryPool *mp, const CName *pnameAlias, + CTableDescriptor *ptabdesc, ULONG ulOriginOpId, + ULONG scan_id, CColRefArray *pdrgpcrOutput, + CColRef2dArray *pdrgpdrgpcrParts, + IMdIdArray *partition_mdids, + ColRefToUlongMapArray *root_col_mapping_per_part); + + // ident accessors + EOperatorId + Eopid() const override + { + return EopPhysicalAppendTableScan; + } + + // return a string for operator name + const CHAR * + SzId() const override + { + return "CPhysicalAppendTableScan"; + } + + // match function + BOOL Matches(COperator *) const override; + + // statistics derivation during costing + IStatistics *PstatsDerive(CMemoryPool *mp, CExpressionHandle &exprhdl, + CReqdPropPlan *prpplan, + IStatisticsArray *stats_ctxt) const override; + + // conversion function + static CPhysicalAppendTableScan * + PopConvert(COperator *pop) + { + GPOS_ASSERT(nullptr != pop); + GPOS_ASSERT(EopPhysicalAppendTableScan == pop->Eopid()); + + return dynamic_cast(pop); + } + + CPartitionPropagationSpec *PppsDerive( + CMemoryPool *mp, CExpressionHandle &exprhdl) const override; +}; // class CPhysicalAppendTableScan + +} // namespace gpopt + +#endif // !GPOPT_CPhysicalAppendTableScan_H + +// EOF diff --git a/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelAppendTableScan.h b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelAppendTableScan.h new file mode 100644 index 00000000000..a7c4854507a --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelAppendTableScan.h @@ -0,0 +1,143 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalParallelAppendTableScan.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelAppendTableScan.h + * + *------------------------------------------------------------------------- + */ + +#ifndef GPOPT_CPhysicalParallelAppendTableScan_H +#define GPOPT_CPhysicalParallelAppendTableScan_H + +#include "gpos/base.h" + +#include "gpopt/operators/CPhysicalDynamicScan.h" + +namespace gpopt +{ +//--------------------------------------------------------------------------- +// @class: +// CPhysicalParallelAppendTableScan +// +// @doc: +// Parallel Append Table scan operator +// +//--------------------------------------------------------------------------- +class CPhysicalParallelAppendTableScan : public CPhysicalDynamicScan +{ +private: + // number of parallel workers + ULONG m_ulParallelWorkers; + + // worker-level distribution spec + CDistributionSpec *m_pdsWorkerDistribution; +public: + CPhysicalParallelAppendTableScan(const CPhysicalParallelAppendTableScan &) = delete; + + // ctors + CPhysicalParallelAppendTableScan(CMemoryPool *mp, const CName *pnameAlias, + CTableDescriptor *ptabdesc, ULONG ulOriginOpId, + ULONG scan_id, CColRefArray *pdrgpcrOutput, + CColRef2dArray *pdrgpdrgpcrParts, + IMdIdArray *partition_mdids, + ColRefToUlongMapArray *root_col_mapping_per_part, + ULONG ulParallelWorkers); + + // dtor + ~CPhysicalParallelAppendTableScan() override; + + // ident accessors + EOperatorId + Eopid() const override + { + return EopPhysicalParallelAppendTableScan; + } + + // return a string for operator name + const CHAR * + SzId() const override + { + return "CPhysicalParallelAppendTableScan"; + } + + // match function + BOOL Matches(COperator *) const override; + + // statistics derivation during costing + IStatistics *PstatsDerive(CMemoryPool *mp, CExpressionHandle &exprhdl, + CReqdPropPlan *prpplan, + IStatisticsArray *stats_ctxt) const override; + + // number of parallel workers + ULONG UlParallelWorkers() const + { + return m_ulParallelWorkers; + } + + // conversion function + static CPhysicalParallelAppendTableScan * + PopConvert(COperator *pop) + { + GPOS_ASSERT(nullptr != pop); + GPOS_ASSERT(EopPhysicalParallelAppendTableScan == pop->Eopid()); + + return dynamic_cast(pop); + } + + CPartitionPropagationSpec *PppsDerive( + CMemoryPool *mp, CExpressionHandle &exprhdl) const override; + + CRewindabilitySpec * + PrsDerive(CMemoryPool *mp, + CExpressionHandle & // exprhdl + ) const override + { + return GPOS_NEW(mp) + CRewindabilitySpec(CRewindabilitySpec::ErtNone, + CRewindabilitySpec::EmhtNoMotion); + } + + // derive distribution + CDistributionSpec *PdsDerive(CMemoryPool *mp, CExpressionHandle &exprhdl) const override; + + // return distribution property enforcing type for this operator + CEnfdProp::EPropEnforcingType EpetDistribution( + CExpressionHandle &exprhdl, + const CEnfdDistribution *ped) const override; + + // return rewindability property enforcing type for this operator + CEnfdProp::EPropEnforcingType EpetRewindability( + CExpressionHandle &exprhdl, + const CEnfdRewindability *per) const override; + + // check if optimization contexts is valid + // Reject if parent requires REWINDABLE (e.g., for NL Join inner child) + BOOL FValidContext(CMemoryPool *mp, COptimizationContext *poc, + COptimizationContextArray *pdrgpocChild) const override; + +}; // class CPhysicalParallelAppendTableScan + +} // namespace gpopt + +#endif // !GPOPT_CPhysicalParallelAppendTableScan_H + +// EOF diff --git a/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelCTEConsumer.h b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelCTEConsumer.h new file mode 100644 index 00000000000..00e72f60b6d --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelCTEConsumer.h @@ -0,0 +1,236 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalParallelCTEConsumer.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelCTEConsumer.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPOPT_CPhysicalParallelCTEConsumer_H +#define GPOPT_CPhysicalParallelCTEConsumer_H + +#include "gpos/base.h" + +#include "gpopt/operators/CPhysical.h" + +namespace gpopt +{ +//--------------------------------------------------------------------------- +// @class: +// CPhysicalParallelCTEConsumer +// +// @doc: +// parallel CTE consumer operator +// +//--------------------------------------------------------------------------- +class CPhysicalParallelCTEConsumer : public CPhysical +{ +private: + // cte identifier + ULONG m_id; + + // cte columns + CColRefArray *m_pdrgpcr; + + // hashmap for all the columns in the CTE expression + UlongToColRefMap *m_phmulcr; + + // mapping index of colref to origin idx + // if current colref not used, then set -1 in the position + // if all used, then m_pidxmap will be nullptr + ULongPtrArray *m_pidxmap; + + // number of parallel workers + ULONG m_ulParallelWorkers; +public: + CPhysicalParallelCTEConsumer(const CPhysicalParallelCTEConsumer &) = delete; + + // ctor + CPhysicalParallelCTEConsumer(CMemoryPool *mp, ULONG id, CColRefArray *colref_array, + UlongToColRefMap *colref_mapping, ULONG ulParallelWorkers); + + // dtor + ~CPhysicalParallelCTEConsumer() override; + + // ident accessors + EOperatorId + Eopid() const override + { + return EopPhysicalParallelCTEConsumer; + } + + const CHAR * + SzId() const override + { + return "CPhysicalParallelCTEConsumer"; + } + + // cte identifier + ULONG + UlCTEId() const + { + return m_id; + } + + // cte columns + CColRefArray * + Pdrgpcr() const + { + return m_pdrgpcr; + } + + // column mapping + UlongToColRefMap * + Phmulcr() const + { + return m_phmulcr; + } + + ULongPtrArray * + PCTEIdxMap() const + { + return m_pidxmap; + } + + // operator specific hash function + ULONG HashValue() const override; + + ULONG UlParallelWorkers() const + { + return m_ulParallelWorkers; + } + + // match function + BOOL Matches(COperator *pop) const override; + + // sensitivity to order of inputs + BOOL + FInputOrderSensitive() const override + { + return false; + } + + //------------------------------------------------------------------------------------- + // Required Plan Properties + //------------------------------------------------------------------------------------- + + // compute required output columns of the n-th child + CColRefSet *PcrsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CColRefSet *pcrsRequired, ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) override; + + // compute required ctes of the n-th child + CCTEReq *PcteRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CCTEReq *pcter, ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) const override; + + // compute required sort order of the n-th child + COrderSpec *PosRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + COrderSpec *posRequired, ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) const override; + + // compute required distribution of the n-th child + CDistributionSpec *PdsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CDistributionSpec *pdsRequired, + ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) const override; + + // compute required rewindability of the n-th child + CRewindabilitySpec *PrsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CRewindabilitySpec *prsRequired, + ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) const override; + + // check if required columns are included in output columns + BOOL FProvidesReqdCols(CExpressionHandle &exprhdl, CColRefSet *pcrsRequired, + ULONG ulOptReq) const override; + + //------------------------------------------------------------------------------------- + // Derived Plan Properties + //------------------------------------------------------------------------------------- + + // derive sort order + COrderSpec *PosDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const override; + + // derive distribution + CDistributionSpec *PdsDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const override; + + // derive rewindability + CRewindabilitySpec *PrsDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const override; + + // derive cte map + CCTEMap *PcmDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const override; + + + //------------------------------------------------------------------------------------- + // Enforced Properties + //------------------------------------------------------------------------------------- + + // return order property enforcing type for this operator + CEnfdProp::EPropEnforcingType EpetOrder( + CExpressionHandle &exprhdl, const CEnfdOrder *peo) const override; + + // return rewindability property enforcing type for this operator + CEnfdProp::EPropEnforcingType EpetRewindability( + CExpressionHandle &exprhdl, + const CEnfdRewindability *per) const override; + + // return true if operator passes through stats obtained from children, + // this is used when computing stats during costing + BOOL + FPassThruStats() const override + { + return false; + } + + //------------------------------------------------------------------------------------- + //------------------------------------------------------------------------------------- + //------------------------------------------------------------------------------------- + + // conversion function + static CPhysicalParallelCTEConsumer * + PopConvert(COperator *pop) + { + GPOS_ASSERT(nullptr != pop); + GPOS_ASSERT(EopPhysicalParallelCTEConsumer == pop->Eopid()); + + return dynamic_cast(pop); + } + + // debug print + IOstream &OsPrint(IOstream &) const override; + +}; // class CPhysicalParallelCTEConsumer + +} // namespace gpopt + +#endif // !GPOPT_CPhysicalParallelCTEConsumer_H + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelCTEProducer.h b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelCTEProducer.h new file mode 100644 index 00000000000..b4325547ee4 --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelCTEProducer.h @@ -0,0 +1,236 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalParallelCTEProducer.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelCTEProducer.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPOPT_CPhysicalParallelCTEProducer_H +#define GPOPT_CPhysicalParallelCTEProducer_H + +#include "gpos/base.h" + +#include "gpopt/operators/CPhysical.h" + +namespace gpopt +{ +//--------------------------------------------------------------------------- +// @class: +// CPhysicalParallelCTEProducer +// +// @doc: +// parallel CTE producer operator +// +//--------------------------------------------------------------------------- +class CPhysicalParallelCTEProducer : public CPhysical +{ +private: + // cte identifier + ULONG m_id; + + // cte columns + CColRefArray *m_pdrgpcr; + + // set representation of cte columns + CColRefSet *m_pcrs; + + // used to record unused columns in m_pdrgpcr + ULongPtrArray *m_pdrgpcr_unused; + + // used to mapping col index + ULongPtrArray *m_pidxmap; + + // number of parallel workers + ULONG m_ulParallelWorkers; +public: + CPhysicalParallelCTEProducer(const CPhysicalParallelCTEProducer &) = delete; + + // ctor + CPhysicalParallelCTEProducer(CMemoryPool *mp, ULONG id, CColRefArray *colref_array, BOOL *umask, ULONG ulParallelWorkers); + + // dtor + ~CPhysicalParallelCTEProducer() override; + + // ident accessors + EOperatorId + Eopid() const override + { + return EopPhysicalParallelCTEProducer; + } + + const CHAR * + SzId() const override + { + return "CPhysicalParallelCTEProducer"; + } + + // cte identifier + ULONG + UlCTEId() const + { + return m_id; + } + + // cte columns + CColRefArray * + Pdrgpcr() const + { + return m_pdrgpcr; + } + + // used to make the CTE consumer colid get the right position in m_pdrgpcr. + // more details see the function CCTEInfo::PhmulcrConsumerToProducer() + ULongPtrArray * + PdrgpcrUnused() const + { + return m_pdrgpcr_unused; + } + + ULongPtrArray * + PCTEIdxMap() const + { + return m_pidxmap; + } + + ULONG UlParallelWorkers() const + { + return m_ulParallelWorkers; + } + + // operator specific hash function + ULONG HashValue() const override; + + // match function + BOOL Matches(COperator *pop) const override; + + // sensitivity to order of inputs + BOOL + FInputOrderSensitive() const override + { + return false; + } + + //------------------------------------------------------------------------------------- + // Required Plan Properties + //------------------------------------------------------------------------------------- + + // compute required output columns of the n-th child + CColRefSet *PcrsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CColRefSet *pcrsRequired, ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) override; + + // compute required ctes of the n-th child + CCTEReq *PcteRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CCTEReq *pcter, ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) const override; + + // compute required sort order of the n-th child + COrderSpec *PosRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + COrderSpec *posRequired, ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) const override; + + // compute required distribution of the n-th child + CDistributionSpec *PdsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CDistributionSpec *pdsRequired, + ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) const override; + + // compute required rewindability of the n-th child + CRewindabilitySpec *PrsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CRewindabilitySpec *prsRequired, + ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) const override; + + // check if required columns are included in output columns + BOOL FProvidesReqdCols(CExpressionHandle &exprhdl, CColRefSet *pcrsRequired, + ULONG ulOptReq) const override; + + //------------------------------------------------------------------------------------- + // Derived Plan Properties + //------------------------------------------------------------------------------------- + + // derive sort order + COrderSpec *PosDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const override; + + // derive distribution + CDistributionSpec *PdsDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const override; + + // derive rewindability + CRewindabilitySpec *PrsDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const override; + + // derive cte map + CCTEMap *PcmDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const override; + + //------------------------------------------------------------------------------------- + // Enforced Properties + //------------------------------------------------------------------------------------- + + // return order property enforcing type for this operator + CEnfdProp::EPropEnforcingType EpetOrder( + CExpressionHandle &exprhdl, const CEnfdOrder *peo) const override; + + // return rewindability property enforcing type for this operator + CEnfdProp::EPropEnforcingType EpetRewindability( + CExpressionHandle &exprhdl, + const CEnfdRewindability *per) const override; + + // return true if operator passes through stats obtained from children, + // this is used when computing stats during costing + BOOL + FPassThruStats() const override + { + return false; + } + + //------------------------------------------------------------------------------------- + //------------------------------------------------------------------------------------- + //------------------------------------------------------------------------------------- + + // conversion function + static CPhysicalParallelCTEProducer * + PopConvert(COperator *pop) + { + GPOS_ASSERT(nullptr != pop); + GPOS_ASSERT(EopPhysicalParallelCTEProducer == pop->Eopid()); + + return dynamic_cast(pop); + } + + // debug print + IOstream &OsPrint(IOstream &) const override; + +}; // class CPhysicalParallelCTEProducer + +} // namespace gpopt + +#endif // !GPOPT_CPhysicalParallelCTEProducer_H + +// EOF diff --git a/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelSequence.h b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelSequence.h new file mode 100644 index 00000000000..415a5c2b070 --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelSequence.h @@ -0,0 +1,189 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalParallelSequence.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelSequence.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPOPT_CPhysicalParallelSequence_H +#define GPOPT_CPhysicalParallelSequence_H + +#include "gpos/base.h" + +#include "gpopt/base/CDistributionSpec.h" +#include "gpopt/operators/CPhysical.h" + +namespace gpopt +{ +//--------------------------------------------------------------------------- +// @class: +// CPhysicalParallelSequence +// +// @doc: +// Physical parallel sequence operator +// +//--------------------------------------------------------------------------- +class CPhysicalParallelSequence : public CPhysical +{ +private: + //empty column set to be requested from all children except last child + CColRefSet *m_pcrsEmpty; + + // number of parallel workers + ULONG m_ulParallelWorkers; +public: + CPhysicalParallelSequence(const CPhysicalParallelSequence &) = delete; + + // ctor + explicit CPhysicalParallelSequence(CMemoryPool *mp, ULONG ulParallelWorkers); + + // dtor + ~CPhysicalParallelSequence() override; + + // ident accessors + EOperatorId + Eopid() const override + { + return EopPhysicalParallelSequence; + } + + // return a string for operator name + const CHAR * + SzId() const override + { + return "CPhysicalParallelSequence"; + } + + // match function + BOOL Matches(COperator *pop) const override; + + ULONG UlParallelWorkers() const + { + return m_ulParallelWorkers; + } + + // sensitivity to order of inputs + BOOL + FInputOrderSensitive() const override + { + return true; + } + + // conversion function + static CPhysicalParallelSequence * + PopConvert(COperator *pop) + { + GPOS_ASSERT(nullptr != pop); + GPOS_ASSERT(EopPhysicalParallelSequence == pop->Eopid()); + + return dynamic_cast(pop); + } + + //------------------------------------------------------------------------------------- + // Required Plan Properties + //------------------------------------------------------------------------------------- + + // compute required output columns of the n-th child + CColRefSet *PcrsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CColRefSet *pcrsRequired, ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) override; + + // compute required ctes of the n-th child + CCTEReq *PcteRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CCTEReq *pcter, ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) const override; + + // compute required sort columns of the n-th child + COrderSpec *PosRequired(CMemoryPool *, // mp + CExpressionHandle &, // exprhdl + COrderSpec *, // posRequired + ULONG, // child_index + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq + ) const override; + + // compute required distribution of the n-th child + CDistributionSpec *PdsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CDistributionSpec *pdsRequired, + ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG ulOptReq) const override; + + // compute required rewindability of the n-th child + CRewindabilitySpec *PrsRequired(CMemoryPool *, //mp + CExpressionHandle &, //exprhdl + CRewindabilitySpec *, //prsRequired + ULONG, // child_index + CDrvdPropArray *, // pdrgpdpCtxt + ULONG ulOptReq) const override; + + // check if required columns are included in output columns + BOOL FProvidesReqdCols(CExpressionHandle &exprhdl, CColRefSet *pcrsRequired, + ULONG ulOptReq) const override; + + //------------------------------------------------------------------------------------- + // Derived Plan Properties + //------------------------------------------------------------------------------------- + + // derive sort order from the last child + COrderSpec *PosDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const override; + + // derive distribution + CDistributionSpec *PdsDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const override; + + // derive rewindability + CRewindabilitySpec *PrsDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const override; + + //------------------------------------------------------------------------------------- + // Enforced Properties + //------------------------------------------------------------------------------------- + + // return order property enforcing type for this operator + CEnfdProp::EPropEnforcingType EpetOrder( + CExpressionHandle &exprhdl, const CEnfdOrder *peo) const override; + + // return rewindability property enforcing type for this operator + CEnfdProp::EPropEnforcingType EpetRewindability( + CExpressionHandle &exprhdl, + const CEnfdRewindability *per) const override; + + // return true if operator passes through stats obtained from children, + // this is used when computing stats during costing + BOOL + FPassThruStats() const override + { + return false; + } + + +}; // class CPhysicalParallelSequence + +} // namespace gpopt + +#endif // !GPOPT_CPhysicalParallelSequence_H + +// EOF diff --git a/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelTableScan.h b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelTableScan.h new file mode 100644 index 00000000000..2ebb916d4f3 --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelTableScan.h @@ -0,0 +1,140 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalParallelTableScan.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalParallelTableScan.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPOPT_CPhysicalParallelTableScan_H +#define GPOPT_CPhysicalParallelTableScan_H + +#include "gpos/base.h" + +#include "gpopt/operators/CPhysicalTableScan.h" + +namespace gpopt +{ +//--------------------------------------------------------------------------- +// @class: +// CPhysicalParallelTableScan +// +// @doc: +// Parallel table scan operator +// +//--------------------------------------------------------------------------- +class CPhysicalParallelTableScan : public CPhysicalTableScan +{ +private: + // number of parallel workers + ULONG m_ulParallelWorkers; + + // worker-level distribution spec + CDistributionSpec *m_pdsWorkerDistribution; + + // private copy ctor + CPhysicalParallelTableScan(const CPhysicalParallelTableScan &); + +public: + // ctors + explicit CPhysicalParallelTableScan(CMemoryPool *mp); + CPhysicalParallelTableScan(CMemoryPool *mp, const CName *pnameAlias, + CTableDescriptor *ptabdesc, + CColRefArray *pdrgpcrOutput, + ULONG ulParallelWorkers); + + // dtor + ~CPhysicalParallelTableScan() override; + + // ident accessors + EOperatorId + Eopid() const override + { + return EopPhysicalParallelTableScan; + } + + // return a string for operator name + const CHAR * + SzId() const override + { + return "CPhysicalParallelTableScan"; + } + + // number of parallel workers + ULONG UlParallelWorkers() const + { + return m_ulParallelWorkers; + } + + // operator specific hash function + ULONG HashValue() const override; + + // match function + BOOL Matches(COperator *) const override; + + // debug print + IOstream &OsPrint(IOstream &) const override; + + // conversion function + static CPhysicalParallelTableScan * + PopConvert(COperator *pop) + { + GPOS_ASSERT(nullptr != pop); + GPOS_ASSERT(EopPhysicalParallelTableScan == pop->Eopid()); + + return dynamic_cast(pop); + } + + CRewindabilitySpec * + PrsDerive(CMemoryPool *mp, + CExpressionHandle & // exprhdl + ) const override + { + return GPOS_NEW(mp) + CRewindabilitySpec(CRewindabilitySpec::ErtNone, + CRewindabilitySpec::EmhtNoMotion); + } + + // derive distribution + CDistributionSpec *PdsDerive(CMemoryPool *mp, CExpressionHandle &exprhdl) const override; + + // return distribution property enforcing type for this operator + CEnfdProp::EPropEnforcingType EpetDistribution( + CExpressionHandle &exprhdl, + const CEnfdDistribution *ped) const override; + + // return rewindability property enforcing type for this operator + CEnfdProp::EPropEnforcingType EpetRewindability( + CExpressionHandle &exprhdl, + const CEnfdRewindability *per) const override; + + // check if optimization contexts is valid + // Reject if parent requires REWINDABLE (e.g., for NL Join inner child) + BOOL FValidContext(CMemoryPool *mp, COptimizationContext *poc, + COptimizationContextArray *pdrgpocChild) const override; + +}; // class CPhysicalParallelTableScan + +} // namespace gpopt + +#endif // !GPOPT_CPhysicalParallelTableScan_H + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalTableScan.h b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalTableScan.h index ace1005d67a..d34f0150b55 100644 --- a/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalTableScan.h +++ b/src/backend/gporca/libgpopt/include/gpopt/operators/CPhysicalTableScan.h @@ -71,7 +71,8 @@ class CPhysicalTableScan : public CPhysicalScan { GPOS_ASSERT(nullptr != pop); GPOS_ASSERT(EopPhysicalTableScan == pop->Eopid() || - EopPhysicalForeignScan == pop->Eopid()); + EopPhysicalForeignScan == pop->Eopid() || + EopPhysicalParallelTableScan == pop->Eopid()); return dynamic_cast(pop); } diff --git a/src/backend/gporca/libgpopt/include/gpopt/optimizer/COptimizerConfig.h b/src/backend/gporca/libgpopt/include/gpopt/optimizer/COptimizerConfig.h index 159ddafbed7..eeee3932b3e 100644 --- a/src/backend/gporca/libgpopt/include/gpopt/optimizer/COptimizerConfig.h +++ b/src/backend/gporca/libgpopt/include/gpopt/optimizer/COptimizerConfig.h @@ -69,11 +69,15 @@ class COptimizerConfig : public CRefCount // default window oids CWindowOids *m_window_oids; + // should generate parallel plans ? + BOOL m_create_parallel_plan; + public: // ctor COptimizerConfig(CEnumeratorConfig *pec, CStatisticsConfig *stats_config, CCTEConfig *pcteconf, ICostModel *pcm, CHint *phint, - CPlanHint *pplanhint, CWindowOids *pdefoidsGPDB); + CPlanHint *pplanhint, CWindowOids *pdefoidsGPDB, + BOOL enable_parallel_plans = false); // dtor ~COptimizerConfig() override; @@ -127,6 +131,13 @@ class COptimizerConfig : public CRefCount return m_plan_hint; } + // parallel plans setting + BOOL + CreateParallelPlan() const + { + return m_create_parallel_plan; + } + // generate default optimizer configurations static COptimizerConfig *PoconfDefault(CMemoryPool *mp); diff --git a/src/backend/gporca/libgpopt/include/gpopt/search/CGroup.h b/src/backend/gporca/libgpopt/include/gpopt/search/CGroup.h index 0b94e9df7e7..eaeaf22024f 100644 --- a/src/backend/gporca/libgpopt/include/gpopt/search/CGroup.h +++ b/src/backend/gporca/libgpopt/include/gpopt/search/CGroup.h @@ -38,6 +38,7 @@ class CDrvdProp; class CDrvdPropCtxtPlan; class CReqdPropRelational; class CExpression; +class CMemo; // type definitions // array of groups @@ -160,6 +161,9 @@ class CGroup : public CRefCount, public DbgPrintMixin // memory pool CMemoryPool *m_mp; + // containing memo + CMemo *m_pmemo; + // id is used when printing memo contents ULONG m_id; @@ -257,6 +261,9 @@ class CGroup : public CRefCount, public DbgPrintMixin // setter of group state void SetState(EState estNewState); + // setter of containing memo + void SetMemo(CMemo *pmemo); + // set hash join keys void SetJoinKeys(CExpressionArray *pdrgpexprOuter, CExpressionArray *pdrgpexprInner, @@ -338,6 +345,13 @@ class CGroup : public CRefCount, public DbgPrintMixin return m_id; } + // containing memo accessor + CMemo * + Pmemo() const + { + return m_pmemo; + } + // group properties accessor CDrvdProp * Pdp() const diff --git a/src/backend/gporca/libgpopt/include/gpopt/search/CGroupProxy.h b/src/backend/gporca/libgpopt/include/gpopt/search/CGroupProxy.h index 61303f39ef7..1e77f0fbcab 100644 --- a/src/backend/gporca/libgpopt/include/gpopt/search/CGroupProxy.h +++ b/src/backend/gporca/libgpopt/include/gpopt/search/CGroupProxy.h @@ -23,6 +23,7 @@ using namespace gpos; class CGroupExpression; class CDrvdProp; class COptimizationContext; +class CMemo; //--------------------------------------------------------------------------- // @class: @@ -63,6 +64,13 @@ class CGroupProxy m_pgroup->SetState(estNewState); } + // set containing memo + void + SetMemo(CMemo *pmemo) + { + m_pgroup->SetMemo(pmemo); + } + // set hash join keys void SetJoinKeys(CExpressionArray *pdrgpexprOuter, diff --git a/src/backend/gporca/libgpopt/include/gpopt/translate/CTranslatorExprToDXL.h b/src/backend/gporca/libgpopt/include/gpopt/translate/CTranslatorExprToDXL.h index 2c50e9e812f..4e6240e3091 100644 --- a/src/backend/gporca/libgpopt/include/gpopt/translate/CTranslatorExprToDXL.h +++ b/src/backend/gporca/libgpopt/include/gpopt/translate/CTranslatorExprToDXL.h @@ -101,6 +101,9 @@ class CTranslatorExprToDXL // id of master node INT m_iMasterId; + // whether is set operator + BOOL m_isSet; + // private copy ctor CTranslatorExprToDXL(const CTranslatorExprToDXL &); @@ -268,11 +271,21 @@ class CTranslatorExprToDXL CDistributionSpecArray *pdrgpdsBaseTables, ULONG *pulNonGatherMotions, BOOL *pfDML); + CDXLNode *PdxlnParallelCTEProducer(CExpression *pexprCTEProducer, + CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, + ULONG *pulNonGatherMotions, BOOL *pfDML); + CDXLNode *PdxlnCTEConsumer(CExpression *pexprCTEConsumer, CColRefArray *colref_array, CDistributionSpecArray *pdrgpdsBaseTables, ULONG *pulNonGatherMotions, BOOL *pfDML); + CDXLNode *PdxlnParallelCTEConsumer(CExpression *pexprCTEConsumer, + CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, + ULONG *pulNonGatherMotions, BOOL *pfDML); + // store outer references in index NLJ inner child into global map void StoreIndexNLJOuterRefs(CPhysical *pop); @@ -313,6 +326,12 @@ class CTranslatorExprToDXL CDistributionSpecArray *pdrgpdsBaseTables, ULONG *pulNonGatherMotions, BOOL *pfDML); + // translate a sequence expression + CDXLNode *PdxlnParallelSequence(CExpression *pexprSequence, + CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, + ULONG *pulNonGatherMotions, BOOL *pfDML); + // translate a dynamic seq/foreign scan to append template CDXLNode *PdxlnDynamicScanToAppend( @@ -346,6 +365,32 @@ class CTranslatorExprToDXL CExpression *pexprScalarCond, CDXLPhysicalProperties *dxl_properties); + // translate an append table scan + CDXLNode *PdxlnAppendTableScan(CExpression *pexprDTS, + CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, + ULONG *pulNonGatherMotions, BOOL *pfDML); + + // translate an append table scan with a scalar condition + CDXLNode *PdxlnAppendTableScan(CExpression *pexprDTS, + CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, + CExpression *pexprScalarCond, + CDXLPhysicalProperties *dxl_properties); + + // translate a parallel append table scan + CDXLNode *PdxlnParallelAppendTableScan(CExpression *pexprDTS, + CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, + ULONG *pulNonGatherMotions, BOOL *pfDML); + + // translate a parallel append table scan with a scalar condition + CDXLNode *PdxlnParallelAppendTableScan(CExpression *pexprDTS, + CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, + CExpression *pexprScalarCond, + CDXLPhysicalProperties *dxl_properties); + // translate a dynamic bitmap table scan CDXLNode *PdxlnDynamicBitmapTableScan( CExpression *pexprDynamicBitmapTableScan, CColRefArray *colref_array, diff --git a/src/backend/gporca/libgpopt/include/gpopt/xforms/CXform.h b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXform.h index 225371097c5..e9d6878dc8e 100644 --- a/src/backend/gporca/libgpopt/include/gpopt/xforms/CXform.h +++ b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXform.h @@ -69,10 +69,14 @@ class CXform : public CRefCount, public DbgPrintMixin ExfExpandNAryJoinMinCard, ExfExpandNAryJoinDP, ExfGet2TableScan, + ExfGet2ParallelTableScan, ExfIndexGet2IndexScan, ExfDynamicGet2DynamicTableScan, + ExfDynamicGet2AppendTableScan, + ExfDynamicGet2ParallelAppendTableScan, ExfDynamicIndexGet2DynamicIndexScan, ExfImplementSequence, + ExfImplementParallelSequence, ExfImplementConstTableGet, ExfUnnestTVF, ExfImplementTVF, @@ -178,7 +182,9 @@ class CXform : public CRefCount, public DbgPrintMixin ExfInlineCTEConsumer, ExfInlineCTEConsumerUnderSelect, ExfImplementCTEProducer, + ExfImplementParallelCTEProducer, ExfImplementCTEConsumer, + ExfImplementParallelCTEConsumer, ExfExpandFullOuterJoin, ExfForeignGet2ForeignScan, ExfSelect2BitmapBoolOp, diff --git a/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformDynamicGet2AppendTableScan.h b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformDynamicGet2AppendTableScan.h new file mode 100644 index 00000000000..9bb4fca509a --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformDynamicGet2AppendTableScan.h @@ -0,0 +1,87 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformDynamicGet2AppendTableScan.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/xforms/CXformDynamicGet2AppendTableScan.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPOPT_CXformDynamicGet2AppendTableScan_H +#define GPOPT_CXformDynamicGet2AppendTableScan_H + +#include "gpos/base.h" + +#include "gpopt/operators/CLogicalDynamicGet.h" +#include "gpopt/xforms/CXformImplementation.h" + +namespace gpopt +{ +using namespace gpos; + +//--------------------------------------------------------------------------- +// @class: +// CXformDynamicGet2Append +// +// @doc: +// Transform DynamicGet to Append Table Scan +// +//--------------------------------------------------------------------------- +class CXformDynamicGet2AppendTableScan : public CXformImplementation +{ +private: +public: + CXformDynamicGet2AppendTableScan( + const CXformDynamicGet2AppendTableScan &) = delete; + + // ctor + explicit CXformDynamicGet2AppendTableScan(CMemoryPool *mp); + + // dtor + ~CXformDynamicGet2AppendTableScan() override = default; + + // ident accessors + EXformId + Exfid() const override + { + return ExfDynamicGet2AppendTableScan; + } + + // return a string for xform name + const CHAR * + SzId() const override + { + return "CXformDynamicGet2AppendTableScan"; + } + + // compute xform promise for a given expression handle + EXformPromise Exfp(CExpressionHandle &exprhdl) const override; + + // actual transform + void Transform(CXformContext *pxfctxt, CXformResult *pxfres, + CExpression *pexpr) const override; + +}; // class CXformDynamicGet2AppendTableScan + +} // namespace gpopt + +#endif // !GPOPT_CXformDynamicGet2AppendTableScan_H + +// EOF diff --git a/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformDynamicGet2ParallelAppendTableScan.h b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformDynamicGet2ParallelAppendTableScan.h new file mode 100644 index 00000000000..7036d7fb767 --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformDynamicGet2ParallelAppendTableScan.h @@ -0,0 +1,87 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformDynamicGet2ParallelAppendTableScan.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/xforms/CXformDynamicGet2ParallelAppendTableScan.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPOPT_CXformDynamicGet2ParallelAppendTableScan_H +#define GPOPT_CXformDynamicGet2ParallelAppendTableScan_H + +#include "gpos/base.h" + +#include "gpopt/operators/CLogicalDynamicGet.h" +#include "gpopt/xforms/CXformImplementation.h" + +namespace gpopt +{ +using namespace gpos; + +//--------------------------------------------------------------------------- +// @class: +// CXformDynamicGet2ParallelAppendTableScan +// +// @doc: +// Transform DynamicGet to Parallel Append Table Scan +// +//--------------------------------------------------------------------------- +class CXformDynamicGet2ParallelAppendTableScan : public CXformImplementation +{ +private: +public: + CXformDynamicGet2ParallelAppendTableScan( + const CXformDynamicGet2ParallelAppendTableScan &) = delete; + + // ctor + explicit CXformDynamicGet2ParallelAppendTableScan(CMemoryPool *mp); + + // dtor + ~CXformDynamicGet2ParallelAppendTableScan() override = default; + + // ident accessors + EXformId + Exfid() const override + { + return ExfDynamicGet2ParallelAppendTableScan; + } + + // return a string for xform name + const CHAR * + SzId() const override + { + return "CXformDynamicGet2ParallelAppendTableScan"; + } + + // compute xform promise for a given expression handle + EXformPromise Exfp(CExpressionHandle &exprhdl) const override; + + // actual transform + void Transform(CXformContext *pxfctxt, CXformResult *pxfres, + CExpression *pexpr) const override; + +}; // class CXformDynamicGet2ParallelAppendTableScan + +} // namespace gpopt + +#endif // !GPOPT_CXformDynamicGet2ParallelAppendTableScan_H + +// EOF diff --git a/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformGet2ParallelTableScan.h b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformGet2ParallelTableScan.h new file mode 100644 index 00000000000..99c8d4863d4 --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformGet2ParallelTableScan.h @@ -0,0 +1,88 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformGet2ParallelTableScan.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/xforms/CXformGet2ParallelTableScan.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPOPT_CXformGet2ParallelTableScan_H +#define GPOPT_CXformGet2ParallelTableScan_H + +#include "gpos/base.h" + +#include "gpopt/xforms/CXformImplementation.h" + +namespace gpopt +{ +using namespace gpos; + +//--------------------------------------------------------------------------- +// @class: +// CXformGet2ParallelTableScan +// +// @doc: +// Transform Get to Parallel TableScan using GUC enable_parallel +// +//--------------------------------------------------------------------------- +class CXformGet2ParallelTableScan : public CXformImplementation +{ +private: + // check if memo contains logical operators that are incompatible with parallel execution + static BOOL FHasParallelIncompatibleOps(CExpressionHandle &exprhdl); + +public: + CXformGet2ParallelTableScan(const CXformGet2ParallelTableScan &) = delete; + + // ctor + explicit CXformGet2ParallelTableScan(CMemoryPool *); + + // dtor + ~CXformGet2ParallelTableScan() override = default; + + // ident accessors + EXformId + Exfid() const override + { + return ExfGet2ParallelTableScan; + } + + // return a string for xform name + const CHAR * + SzId() const override + { + return "CXformGet2ParallelTableScan"; + } + + // compute xform promise for a given expression handle + EXformPromise Exfp(CExpressionHandle &exprhdl) const override; + + // actual transform + void Transform(CXformContext *pxfctxt, CXformResult *pxfres, + CExpression *pexpr) const override; + +}; // class CXformGet2ParallelTableScan + +} // namespace gpopt + +#endif // !GPOPT_CXformGet2ParallelTableScan_H + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelCTEConsumer.h b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelCTEConsumer.h new file mode 100644 index 00000000000..193cab8262e --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelCTEConsumer.h @@ -0,0 +1,84 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformImplementParallelCTEConsumer.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelCTEConsumer.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPOPT_CXformImplementParallelCTEConsumer_H +#define GPOPT_CXformImplementParallelCTEConsumer_H + +#include "gpos/base.h" + +#include "gpopt/xforms/CXformImplementation.h" + +namespace gpopt +{ +using namespace gpos; + +//--------------------------------------------------------------------------- +// @class: +// CXformImplementParallelCTEConsumer +// +// @doc: +// Transform Logical CTE Consumer to Physical Parallel CTE Consumer +// +//--------------------------------------------------------------------------- +class CXformImplementParallelCTEConsumer : public CXformImplementation +{ +private: +public: + CXformImplementParallelCTEConsumer(const CXformImplementParallelCTEConsumer &) = delete; + + // ctor + explicit CXformImplementParallelCTEConsumer(CMemoryPool *mp); + + // dtor + ~CXformImplementParallelCTEConsumer() override = default; + + // ident accessors + EXformId + Exfid() const override + { + return ExfImplementParallelCTEConsumer; + } + + // return a string for xform name + const CHAR * + SzId() const override + { + return "CXformImplementParallelCTEConsumer"; + } + + // compute xform promise for a given expresion handle + EXformPromise Exfp(CExpressionHandle &exprhdl) const override; + + // actual transform + void Transform(CXformContext *pxfctxt, CXformResult *pxfres, + CExpression *pexpr) const override; + +}; // class CXformImplementParallelCTEConsumer +} // namespace gpopt + +#endif // !GPOPT_CXformImplementParallelCTEConsumer_H + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelCTEProducer.h b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelCTEProducer.h new file mode 100644 index 00000000000..0640d29f77e --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelCTEProducer.h @@ -0,0 +1,84 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformImplementParallelCTEProducer.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelCTEProducer.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPOPT_CXformImplementParallelCTEProducer_H +#define GPOPT_CXformImplementParallelCTEProducer_H + +#include "gpos/base.h" + +#include "gpopt/xforms/CXformImplementation.h" + +namespace gpopt +{ +using namespace gpos; + +//--------------------------------------------------------------------------- +// @class: +// CXformImplementParallelCTEProducer +// +// @doc: +// Transform Logical CTE Producer to Physical Parallel CTE Producer +// +//--------------------------------------------------------------------------- +class CXformImplementParallelCTEProducer : public CXformImplementation +{ +private: +public: + CXformImplementParallelCTEProducer(const CXformImplementParallelCTEProducer &) = delete; + + // ctor + explicit CXformImplementParallelCTEProducer(CMemoryPool *mp); + + // dtor + ~CXformImplementParallelCTEProducer() override = default; + + // ident accessors + EXformId + Exfid() const override + { + return ExfImplementParallelCTEProducer; + } + + // return a string for xform name + const CHAR * + SzId() const override + { + return "CXformImplementParallelCTEProducer"; + } + + // compute xform promise for a given expresion handle + EXformPromise Exfp(CExpressionHandle &exprhdl) const override; + + // actual transform + void Transform(CXformContext *pxfctxt, CXformResult *pxfres, + CExpression *pexpr) const override; + +}; // class CXformImplementParallelCTEProducer +} // namespace gpopt + +#endif // !GPOPT_CXformImplementParallelCTEProducer_H + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelSequence.h b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelSequence.h new file mode 100644 index 00000000000..049615a2a5d --- /dev/null +++ b/src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelSequence.h @@ -0,0 +1,84 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformImplementParallelSequence.h + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/include/gpopt/xforms/CXformImplementParallelSequence.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPOPT_CXformImplementParallelSequence_H +#define GPOPT_CXformImplementParallelSequence_H + +#include "gpos/base.h" + +#include "gpopt/xforms/CXformImplementation.h" + +namespace gpopt +{ +using namespace gpos; + +//--------------------------------------------------------------------------- +// @class: +// CXformImplementParallelSequence +// +// @doc: +// Transform logical to physical Parallel Sequence +// +//--------------------------------------------------------------------------- +class CXformImplementParallelSequence : public CXformImplementation +{ +private: +public: + CXformImplementParallelSequence(const CXformImplementParallelSequence &) = delete; + + // ctor + explicit CXformImplementParallelSequence(CMemoryPool *); + + // dtor + ~CXformImplementParallelSequence() override = default; + + // ident accessors + EXformId + Exfid() const override + { + return ExfImplementParallelSequence; + } + + // return a string for xform name + const CHAR * + SzId() const override + { + return "CXformImplementParallelSequence"; + } + + // compute xform promise for a given expresion handle + EXformPromise Exfp(CExpressionHandle &exprhdl) const override; + + // actual transform + void Transform(CXformContext *pxfctxt, CXformResult *pxfres, + CExpression *pexpr) const override; +}; // class CXformImplementParallelSequence + +} // namespace gpopt + +#endif // !GPOPT_CXformImplementParallelSequence_H + +// EOF diff --git a/src/backend/gporca/libgpopt/include/gpopt/xforms/xforms.h b/src/backend/gporca/libgpopt/include/gpopt/xforms/xforms.h index fb5ba6b1e3c..3fb8ff44143 100644 --- a/src/backend/gporca/libgpopt/include/gpopt/xforms/xforms.h +++ b/src/backend/gporca/libgpopt/include/gpopt/xforms/xforms.h @@ -30,6 +30,8 @@ #include "gpopt/xforms/CXformDifferenceAll2LeftAntiSemiJoin.h" #include "gpopt/xforms/CXformDynamicForeignGet2DynamicForeignScan.h" #include "gpopt/xforms/CXformDynamicGet2DynamicTableScan.h" +#include "gpopt/xforms/CXformDynamicGet2AppendTableScan.h" +#include "gpopt/xforms/CXformDynamicGet2ParallelAppendTableScan.h" #include "gpopt/xforms/CXformDynamicIndexGet2DynamicIndexScan.h" #include "gpopt/xforms/CXformDynamicIndexOnlyGet2DynamicIndexOnlyScan.h" #include "gpopt/xforms/CXformEagerAgg.h" @@ -52,10 +54,13 @@ #include "gpopt/xforms/CXformGbAggDedup2StreamAggDedup.h" #include "gpopt/xforms/CXformGbAggWithMDQA2Join.h" #include "gpopt/xforms/CXformGet2TableScan.h" +#include "gpopt/xforms/CXformGet2ParallelTableScan.h" #include "gpopt/xforms/CXformImplementAssert.h" #include "gpopt/xforms/CXformImplementBitmapTableGet.h" #include "gpopt/xforms/CXformImplementCTEConsumer.h" +#include "gpopt/xforms/CXformImplementParallelCTEConsumer.h" #include "gpopt/xforms/CXformImplementCTEProducer.h" +#include "gpopt/xforms/CXformImplementParallelCTEProducer.h" #include "gpopt/xforms/CXformImplementConstTableGet.h" #include "gpopt/xforms/CXformImplementDML.h" #include "gpopt/xforms/CXformImplementDynamicBitmapTableGet.h" @@ -70,6 +75,7 @@ #include "gpopt/xforms/CXformImplementLeftSemiCorrelatedApplyIn.h" #include "gpopt/xforms/CXformImplementLimit.h" #include "gpopt/xforms/CXformImplementSequence.h" +#include "gpopt/xforms/CXformImplementParallelSequence.h" #include "gpopt/xforms/CXformImplementSequenceProject.h" #include "gpopt/xforms/CXformImplementHashSequenceProject.h" #include "gpopt/xforms/CXformImplementSplit.h" diff --git a/src/backend/gporca/libgpopt/src/base/CCostContext.cpp b/src/backend/gporca/libgpopt/src/base/CCostContext.cpp index 2d69d1d7072..36230cb6614 100644 --- a/src/backend/gporca/libgpopt/src/base/CCostContext.cpp +++ b/src/backend/gporca/libgpopt/src/base/CCostContext.cpp @@ -505,6 +505,7 @@ CCostContext::FBetterThan(const CCostContext *pcc) const if (CDistributionSpec::EdtHashed == Pdpplan()->Pds()->Edt() && CDistributionSpec::EdtRandom == pcc->Pdpplan()->Pds()->Edt()) { + // FIXME: return true; } diff --git a/src/backend/gporca/libgpopt/src/base/CDistributionSpecWorkerRandom.cpp b/src/backend/gporca/libgpopt/src/base/CDistributionSpecWorkerRandom.cpp new file mode 100644 index 00000000000..1ae4253f6fd --- /dev/null +++ b/src/backend/gporca/libgpopt/src/base/CDistributionSpecWorkerRandom.cpp @@ -0,0 +1,314 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CDistributionSpecWorkerRandom.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/base/CDistributionSpecWorkerRandom.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/base/CDistributionSpecWorkerRandom.h" + +#include "gpopt/base/CColRefSet.h" +#include "gpopt/base/CDistributionSpecHashed.h" +#include "gpopt/base/CDistributionSpecStrictRandom.h" +#include "gpopt/base/COptCtxt.h" +#include "gpopt/base/CUtils.h" +#include "gpopt/operators/CExpressionHandle.h" +#include "gpopt/operators/CPhysicalMotionHashDistribute.h" +#include "gpopt/operators/CPhysicalMotionRandom.h" +#include "naucrates/traceflags/traceflags.h" + +using namespace gpopt; + +//--------------------------------------------------------------------------- +// @function: +// CDistributionSpecWorkerRandom::CDistributionSpecWorkerRandom +// +// @doc: +// Ctor +// Note: This constructor should only be called from PdsCreateWorkerRandom +// factory method, which ensures pdsSegmentBase is properly initialized +// +//--------------------------------------------------------------------------- +CDistributionSpecWorkerRandom::CDistributionSpecWorkerRandom(ULONG ulWorkers, CDistributionSpec *pdsSegmentBase) + : m_ulWorkers(ulWorkers), m_pdsSegmentBase(pdsSegmentBase) +{ + GPOS_ASSERT(ulWorkers > 0); + if (m_pdsSegmentBase) + m_pdsSegmentBase->AddRef(); + + if (COptCtxt::PoctxtFromTLS()->FDMLQuery()) + { + // set duplicate sensitive flag to enforce Hash-Distribution of + // Const Tables in DML queries + MarkDuplicateSensitive(); + } +} + +//--------------------------------------------------------------------------- +// @function: +// CDistributionSpecWorkerRandom::~CDistributionSpecWorkerRandom +// +// @doc: +// Dtor +// +//--------------------------------------------------------------------------- +CDistributionSpecWorkerRandom::~CDistributionSpecWorkerRandom() +{ + CRefCount::SafeRelease(m_pdsSegmentBase); +} + +//--------------------------------------------------------------------------- +// @function: +// CDistributionSpecWorkerRandom::PdsCreateWorkerRandom +// +// @doc: +// Factory method for creating worker-level random distribution +// +//--------------------------------------------------------------------------- +CDistributionSpecWorkerRandom * +CDistributionSpecWorkerRandom::PdsCreateWorkerRandom(CMemoryPool *mp, ULONG ulWorkers, CDistributionSpec *pdsBase) +{ + GPOS_ASSERT(nullptr != mp); + GPOS_ASSERT(ulWorkers > 0); + + // If no base distribution provided, create a default random distribution + // using the provided memory pool (not TLS pool) + CDistributionSpec *pdsSegmentBase = pdsBase; + if (nullptr == pdsSegmentBase) + { + pdsSegmentBase = GPOS_NEW(mp) CDistributionSpecRandom(); + } + + return GPOS_NEW(mp) CDistributionSpecWorkerRandom(ulWorkers, pdsSegmentBase); +} + +//--------------------------------------------------------------------------- +// @function: +// CDistributionSpecWorkerRandom::Matches +// +// @doc: +// Match function +// +//--------------------------------------------------------------------------- +BOOL +CDistributionSpecWorkerRandom::Matches(const CDistributionSpec *pds) const +{ + if (pds->Edt() == CDistributionSpec::EdtWorkerRandom) + { + const CDistributionSpecWorkerRandom *pdsWorkerRandom = + CDistributionSpecWorkerRandom::PdsConvert(pds); + + // Check if worker counts match and base distributions are compatible + return (m_ulWorkers == pdsWorkerRandom->m_ulWorkers && + IsDuplicateSensitive() == pdsWorkerRandom->IsDuplicateSensitive() && + ((nullptr == m_pdsSegmentBase && nullptr == pdsWorkerRandom->m_pdsSegmentBase) || + (nullptr != m_pdsSegmentBase && nullptr != pdsWorkerRandom->m_pdsSegmentBase && + m_pdsSegmentBase->Matches(pdsWorkerRandom->m_pdsSegmentBase)))); + } + else if (pds->Edt() == CDistributionSpec::EdtRandom) + { + // Worker random can match regular random if base distribution matches + const CDistributionSpecRandom *pdsRandom = + CDistributionSpecRandom::PdsConvert(pds); + + return (nullptr != m_pdsSegmentBase && + m_pdsSegmentBase->Matches(pds) && + IsDuplicateSensitive() == pdsRandom->IsDuplicateSensitive()); + } + + return false; +} + +//--------------------------------------------------------------------------- +// @function: +// CDistributionSpecWorkerRandom::FSatisfies +// +// @doc: +// Check if this distribution spec satisfies the given one +// +//--------------------------------------------------------------------------- +BOOL +CDistributionSpecWorkerRandom::FSatisfies(const CDistributionSpec *pds) const +{ + if (Matches(pds)) + { + return true; + } + + // Handle different distribution types + if (EdtWorkerRandom == pds->Edt()) + { + const CDistributionSpecWorkerRandom *pdsWorkerRandom = + CDistributionSpecWorkerRandom::PdsConvert(pds); + + // Worker-level can satisfy another worker-level if it has the same number of workers + // and the base segment distribution is compatible + return (m_ulWorkers == pdsWorkerRandom->m_ulWorkers && + (nullptr == m_pdsSegmentBase || nullptr == pdsWorkerRandom->m_pdsSegmentBase || + m_pdsSegmentBase->FSatisfies(pdsWorkerRandom->m_pdsSegmentBase)) && + (IsDuplicateSensitive() || !pdsWorkerRandom->IsDuplicateSensitive())); + } + else if (EdtRandom == pds->Edt()) + { + const CDistributionSpecRandom *pdsRandom = CDistributionSpecRandom::PdsConvert(pds); + + // Worker-level can satisfy segment-level requirement + // if the base segment distribution satisfies it + return (nullptr != m_pdsSegmentBase && + m_pdsSegmentBase->FSatisfies(pds) && + (IsDuplicateSensitive() || !pdsRandom->IsDuplicateSensitive())); + } + + // Standard satisfaction logic for other distribution types + return EdtAny == pds->Edt() || EdtNonSingleton == pds->Edt() || + EdtNonReplicated == pds->Edt(); +} + +//--------------------------------------------------------------------------- +// @function: +// CDistributionSpecWorkerRandom::AppendEnforcers +// +// @doc: +// Add required enforcers to dynamic array +// +//--------------------------------------------------------------------------- +void +CDistributionSpecWorkerRandom::AppendEnforcers(CMemoryPool *mp, + CExpressionHandle &exprhdl, + CReqdPropPlan *prpp, + CExpressionArray *pdrgpexpr, + CExpression *pexpr) +{ + GPOS_ASSERT(nullptr != mp); + GPOS_ASSERT(nullptr != prpp); + GPOS_ASSERT(nullptr != pdrgpexpr); + GPOS_ASSERT(nullptr != pexpr); + GPOS_ASSERT(!GPOS_FTRACE(EopttraceDisableMotions)); + GPOS_ASSERT( + this == prpp->Ped()->PdsRequired() && + "required plan properties don't match enforced distribution spec"); + + // Get the actually required distribution specification + CDistributionSpec *pdsRequired = prpp->Ped()->PdsRequired(); + GPOS_ASSERT(nullptr != pdsRequired); + + // Get child's distribution for duplicate hazard checking + CDistributionSpec *expr_dist_spec = + CDrvdPropPlan::Pdpplan(exprhdl.Pdp())->Pds(); + BOOL fDuplicateHazard = CUtils::FDuplicateHazardDistributionSpec(expr_dist_spec); + + pexpr->AddRef(); + CExpression *pexprMotion = nullptr; + + // Generate appropriate motion based on required distribution type + switch (pdsRequired->Edt()) + { + case CDistributionSpec::EdtHashed: + { + // Required: Hashed distribution -> Generate HashDistribute Motion + if (GPOS_FTRACE(EopttraceDisableMotionHashDistribute)) + { + // Hash redistribute Motion is disabled, cannot satisfy requirement + pexpr->Release(); + return; + } + + CDistributionSpecHashed *pdsHashedRequired = + CDistributionSpecHashed::PdsConvert(pdsRequired); + pdsHashedRequired->AddRef(); + + if (fDuplicateHazard) + { + pdsHashedRequired->MarkDuplicateSensitive(); + } + + pexprMotion = GPOS_NEW(mp) CExpression( + mp, GPOS_NEW(mp) CPhysicalMotionHashDistribute(mp, pdsHashedRequired), pexpr); + break; + } + + case CDistributionSpec::EdtRandom: + case CDistributionSpec::EdtWorkerRandom: + { + // Required: Random/WorkerRandom distribution -> Generate Random Motion + if (GPOS_FTRACE(EopttraceDisableMotionRandom)) + { + // Random Motion is disabled + pexpr->Release(); + return; + } + + // Use factory method to ensure proper memory pool usage + CDistributionSpecWorkerRandom *random_dist_spec = + PdsCreateWorkerRandom(mp, m_ulWorkers, m_pdsSegmentBase); + + if (fDuplicateHazard) + { + random_dist_spec->MarkDuplicateSensitive(); + } + + pexprMotion = GPOS_NEW(mp) CExpression( + mp, GPOS_NEW(mp) CPhysicalMotionRandom(mp, random_dist_spec), pexpr); + break; + } + + default: + { + // Fallback: cannot generate appropriate motion + pexpr->Release(); + return; + } + } + + // Add the generated motion to the enforcer array + if (nullptr != pexprMotion) + { + pdrgpexpr->Append(pexprMotion); + } +} + +//--------------------------------------------------------------------------- +// @function: +// CDistributionSpecWorkerRandom::OsPrint +// +// @doc: +// Print function +// +//--------------------------------------------------------------------------- +IOstream & +CDistributionSpecWorkerRandom::OsPrint(IOstream &os) const +{ + os << SzId() << "[workers:" << m_ulWorkers << "]"; + if (nullptr != m_pdsSegmentBase) + { + os << " base:"; + m_pdsSegmentBase->OsPrint(os); + } + if (IsDuplicateSensitive()) + { + os << " (duplicate sensitive)"; + } + return os; +} + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/src/base/CDrvdPropPlan.cpp b/src/backend/gporca/libgpopt/src/base/CDrvdPropPlan.cpp index 6b773cb40fc..e3f1a9ff94d 100644 --- a/src/backend/gporca/libgpopt/src/base/CDrvdPropPlan.cpp +++ b/src/backend/gporca/libgpopt/src/base/CDrvdPropPlan.cpp @@ -19,6 +19,7 @@ #include "gpopt/operators/CExpressionHandle.h" #include "gpopt/operators/CPhysical.h" #include "gpopt/operators/CPhysicalCTEConsumer.h" +#include "gpopt/operators/CPhysicalParallelCTEConsumer.h" #include "gpopt/operators/CScalar.h" @@ -90,6 +91,11 @@ CDrvdPropPlan::Derive(CMemoryPool *mp, CExpressionHandle &exprhdl, { CopyCTEProducerPlanProps(mp, pdpctxt, popPhysical); } + else if (nullptr != pdpctxt && + COperator::EopPhysicalParallelCTEConsumer == popPhysical->Eopid()) + { + CopyParallelCTEProducerPlanProps(mp, pdpctxt, popPhysical); + } else { // call property derivation functions on the operator @@ -146,6 +152,45 @@ CDrvdPropPlan::CopyCTEProducerPlanProps(CMemoryPool *mp, CDrvdPropCtxt *pdpctxt, } } +//--------------------------------------------------------------------------- +// @function: +// CDrvdPropPlan::CopyCTEProducerPlanProps +// +// @doc: +// Copy CTE producer plan properties from given context to current object +// +//--------------------------------------------------------------------------- +void +CDrvdPropPlan::CopyParallelCTEProducerPlanProps(CMemoryPool *mp, CDrvdPropCtxt *pdpctxt, + COperator *pop) +{ + CDrvdPropCtxtPlan *pdpctxtplan = + CDrvdPropCtxtPlan::PdpctxtplanConvert(pdpctxt); + CPhysicalParallelCTEConsumer *popCTEConsumer = + CPhysicalParallelCTEConsumer::PopConvert(pop); + ULONG ulCTEId = popCTEConsumer->UlCTEId(); + UlongToColRefMap *colref_mapping = popCTEConsumer->Phmulcr(); + CDrvdPropPlan *pdpplan = pdpctxtplan->PdpplanCTEProducer(ulCTEId); + if (nullptr != pdpplan) + { + // copy producer plan properties after remapping columns + m_pos = pdpplan->Pos()->PosCopyWithRemappedColumns(mp, colref_mapping, + true /*must_exist*/); + m_pds = pdpplan->Pds()->PdsCopyWithRemappedColumns(mp, colref_mapping, + true /*must_exist*/); + // rewindability and partition filter map do not need column remapping, + // we add-ref producer's properties directly + pdpplan->Prs()->AddRef(); + m_prs = pdpplan->Prs(); + + // no need to copy the part index map. return an empty one. This is to + // distinguish between a CTE consumer and the inlined expression + m_ppps = GPOS_NEW(mp) CPartitionPropagationSpec(mp); + + GPOS_ASSERT(CDistributionSpec::EdtAny != m_pds->Edt() && + "CDistributionAny is a require-only, cannot be derived"); + } +} //--------------------------------------------------------------------------- // @function: diff --git a/src/backend/gporca/libgpopt/src/base/COptimizationContext.cpp b/src/backend/gporca/libgpopt/src/base/COptimizationContext.cpp index c718a1c50f9..806adb9fedc 100644 --- a/src/backend/gporca/libgpopt/src/base/COptimizationContext.cpp +++ b/src/backend/gporca/libgpopt/src/base/COptimizationContext.cpp @@ -17,8 +17,10 @@ #include "gpopt/base/CEnfdOrder.h" #include "gpopt/base/COptCtxt.h" #include "gpopt/base/COrderSpec.h" +#include "gpopt/operators/COperator.h" #include "gpopt/operators/CPhysicalAgg.h" #include "gpopt/operators/CPhysicalCTEProducer.h" +#include "gpopt/operators/CPhysicalParallelCTEProducer.h" #include "gpopt/operators/CPhysicalMotion.h" #include "gpopt/operators/CPhysicalNLJoin.h" #include "gpopt/operators/CPhysicalSort.h" @@ -373,7 +375,8 @@ COptimizationContext::PrppCTEProducer(CMemoryPool *mp, CCostContext *pccBest = poc->PccBest(); CGroupExpression *pgexpr = pccBest->Pgexpr(); BOOL fOptimizeCTESequence = - (COperator::EopPhysicalSequence == pgexpr->Pop()->Eopid() && + ((COperator::EopPhysicalSequence == pgexpr->Pop()->Eopid() || + COperator::EopPhysicalParallelSequence == pgexpr->Pop()->Eopid()) && (*pgexpr)[0]->FHasCTEProducer()); if (!fOptimizeCTESequence) @@ -407,15 +410,30 @@ COptimizationContext::PrppCTEProducer(CMemoryPool *mp, return nullptr; } + UlongToColRefMap *colref_mapping; + CColRefSet *pcrsInnerOutput = CDrvdPropRelational::GetRelationalProperties((*pgexpr)[1]->Pdp()) ->GetOutputColumns(); - CPhysicalCTEProducer *popProducer = - CPhysicalCTEProducer::PopConvert(pccProducer->Pgexpr()->Pop()); - UlongToColRefMap *colref_mapping = - COptCtxt::PoctxtFromTLS()->Pcteinfo()->PhmulcrConsumerToProducer( - mp, popProducer->UlCTEId(), pcrsInnerOutput, - popProducer->Pdrgpcr(), popProducer->PdrgpcrUnused()); + if (COperator::EopPhysicalCTEProducer == pccProducer->Pgexpr()->Pop()->Eopid()) + { + CPhysicalCTEProducer *popProducer = + CPhysicalCTEProducer::PopConvert(pccProducer->Pgexpr()->Pop()); + colref_mapping = + COptCtxt::PoctxtFromTLS()->Pcteinfo()->PhmulcrConsumerToProducer( + mp, popProducer->UlCTEId(), pcrsInnerOutput, + popProducer->Pdrgpcr(), popProducer->PdrgpcrUnused()); + } + else if (COperator::EopPhysicalParallelCTEProducer == pccProducer->Pgexpr()->Pop()->Eopid()) + { + CPhysicalParallelCTEProducer *popProducer = + CPhysicalParallelCTEProducer::PopConvert(pccProducer->Pgexpr()->Pop()); + colref_mapping = + COptCtxt::PoctxtFromTLS()->Pcteinfo()->PhmulcrConsumerToProducer( + mp, popProducer->UlCTEId(), pcrsInnerOutput, + popProducer->Pdrgpcr(), popProducer->PdrgpcrUnused()); + } + CReqdPropPlan *prppProducer = CReqdPropPlan::PrppRemapForCTE( mp, pocProducer->Prpp(), pccProducer->Pdpplan(), pccConsumer->Pdpplan(), colref_mapping); diff --git a/src/backend/gporca/libgpopt/src/base/CQueryContext.cpp b/src/backend/gporca/libgpopt/src/base/CQueryContext.cpp index 4c75e7202ed..b3715fc599e 100644 --- a/src/backend/gporca/libgpopt/src/base/CQueryContext.cpp +++ b/src/backend/gporca/libgpopt/src/base/CQueryContext.cpp @@ -240,8 +240,8 @@ CQueryContext::PqcGenerate(CMemoryPool *mp, CExpression *pexpr, GPOS_NEW(mp) CEnfdRewindability(prs, CEnfdRewindability::ErmSatisfy); CEnfdPartitionPropagation *pepp = GPOS_NEW(mp) CEnfdPartitionPropagation(ppps, CEnfdPartitionPropagation::EppmSatisfy); - - // Required CTEs are obtained from the CTEInfo global information in the optimizer context + + // Required CTEs are obtained from the CTEInfo global information in the optimizer context CCTEReq *pcter = poptctxt->Pcteinfo()->PcterProducers(mp); // NB: Partition propagation requirements are not initialized here. They are diff --git a/src/backend/gporca/libgpopt/src/base/CUtils.cpp b/src/backend/gporca/libgpopt/src/base/CUtils.cpp index b5f847b817a..dc7eaa08e67 100644 --- a/src/backend/gporca/libgpopt/src/base/CUtils.cpp +++ b/src/backend/gporca/libgpopt/src/base/CUtils.cpp @@ -45,7 +45,9 @@ #include "gpopt/operators/CLogicalUnary.h" #include "gpopt/operators/CPhysicalAgg.h" #include "gpopt/operators/CPhysicalCTEConsumer.h" +#include "gpopt/operators/CPhysicalParallelCTEConsumer.h" #include "gpopt/operators/CPhysicalCTEProducer.h" +#include "gpopt/operators/CPhysicalParallelCTEProducer.h" #include "gpopt/operators/CPhysicalMotionRandom.h" #include "gpopt/operators/CPhysicalNLJoin.h" #include "gpopt/operators/CPredicateUtils.h" @@ -4455,6 +4457,13 @@ CUtils::ValidateCTEProducerConsumerLocality( ULONG ulCTEID = CPhysicalCTEProducer::PopConvert(pop)->UlCTEId(); phmulul->Insert(GPOS_NEW(mp) ULONG(ulCTEID), GPOS_NEW(mp) ULONG(eelt)); } + else if (COperator::EopPhysicalParallelCTEProducer == pop->Eopid()) + { + // record the location (either master or segment or singleton) + // where the CTE producer is being executed + ULONG ulCTEID = CPhysicalParallelCTEProducer::PopConvert(pop)->UlCTEId(); + phmulul->Insert(GPOS_NEW(mp) ULONG(ulCTEID), GPOS_NEW(mp) ULONG(eelt)); + } else if (COperator::EopPhysicalCTEConsumer == pop->Eopid()) { ULONG ulCTEID = CPhysicalCTEConsumer::PopConvert(pop)->UlCTEId(); @@ -4469,6 +4478,20 @@ CUtils::ValidateCTEProducerConsumerLocality( gpopt::ExmiCTEProducerConsumerMisAligned, ulCTEID); } } + else if (COperator::EopPhysicalParallelCTEConsumer == pop->Eopid()) + { + ULONG ulCTEID = CPhysicalParallelCTEConsumer::PopConvert(pop)->UlCTEId(); + ULONG *pulLocProducer = phmulul->Find(&ulCTEID); + + // check if the CTEConsumer is being executed in the same location + // as the CTE Producer + if (nullptr == pulLocProducer || *pulLocProducer != (ULONG) eelt) + { + phmulul->Release(); + GPOS_RAISE(gpopt::ExmaGPOPT, + gpopt::ExmiCTEProducerConsumerMisAligned, ulCTEID); + } + } // In case of a Gather motion, the execution locality is set to segments // since the child of Gather motion executes on segments else if (COperator::EopPhysicalMotionGather == pop->Eopid()) diff --git a/src/backend/gporca/libgpopt/src/base/Makefile b/src/backend/gporca/libgpopt/src/base/Makefile index 604fd3a6fa4..01f2bb0a02a 100644 --- a/src/backend/gporca/libgpopt/src/base/Makefile +++ b/src/backend/gporca/libgpopt/src/base/Makefile @@ -42,6 +42,7 @@ OBJS = CAutoOptCtxt.o \ CDistributionSpecStrictRandom.o \ CDistributionSpecStrictSingleton.o \ CDistributionSpecUniversal.o \ + CDistributionSpecWorkerRandom.o \ CDrvdProp.o \ CDrvdPropCtxt.o \ CDrvdPropCtxtPlan.o \ diff --git a/src/backend/gporca/libgpopt/src/operators/CExpression.cpp b/src/backend/gporca/libgpopt/src/operators/CExpression.cpp index 4b31bdc2106..2f73b4af78a 100644 --- a/src/backend/gporca/libgpopt/src/operators/CExpression.cpp +++ b/src/backend/gporca/libgpopt/src/operators/CExpression.cpp @@ -33,6 +33,7 @@ #include "gpopt/operators/CPattern.h" #include "gpopt/operators/CPatternNode.h" #include "gpopt/operators/CPhysicalCTEProducer.h" +#include "gpopt/operators/CPhysicalParallelCTEProducer.h" #include "gpopt/search/CGroupExpression.h" #include "naucrates/statistics/CStatistics.h" #include "naucrates/traceflags/traceflags.h" @@ -1273,6 +1274,11 @@ CExpression::FValidPlan(const CReqdPropPlan *prpp, ULONG ulCTEId = CPhysicalCTEProducer::PopConvert(m_pop)->UlCTEId(); pdpctxtplan->CopyCTEProducerProps(pdpplan, ulCTEId); } + else if (COperator::EopPhysicalParallelCTEProducer == m_pop->Eopid()) + { + ULONG ulCTEId = CPhysicalParallelCTEProducer::PopConvert(m_pop)->UlCTEId(); + pdpctxtplan->CopyCTEProducerProps(pdpplan, ulCTEId); + } CDrvdPropRelational *pdprel = GetDrvdPropRelational(); diff --git a/src/backend/gporca/libgpopt/src/operators/CExpressionHandle.cpp b/src/backend/gporca/libgpopt/src/operators/CExpressionHandle.cpp index 9d6ba129010..62106071364 100644 --- a/src/backend/gporca/libgpopt/src/operators/CExpressionHandle.cpp +++ b/src/backend/gporca/libgpopt/src/operators/CExpressionHandle.cpp @@ -33,6 +33,7 @@ #include "gpopt/operators/COperator.h" #include "gpopt/operators/CPattern.h" #include "gpopt/operators/CPhysicalCTEConsumer.h" +#include "gpopt/operators/CPhysicalParallelCTEConsumer.h" #include "gpopt/operators/CPhysicalScan.h" #include "naucrates/statistics/CStatisticsUtils.h" @@ -685,6 +686,17 @@ CExpressionHandle::DerivePlanPropsForCostContext() pdpctxtplan->CopyCTEProducerProps(pdpplan, ulCTEId); } } + else if (COperator::EopPhysicalParallelCTEConsumer == pop->Eopid()) + { + // copy producer plan properties to passed derived plan properties context + ULONG ulCTEId = CPhysicalParallelCTEConsumer::PopConvert(pop)->UlCTEId(); + CDrvdPropPlan *pdpplan = + m_pcc->Poc()->Prpp()->Pcter()->Pdpplan(ulCTEId); + if (nullptr != pdpplan) + { + pdpctxtplan->CopyCTEProducerProps(pdpplan, ulCTEId); + } + } // create/derive local properties m_pdpplan = Pop()->PdpCreate(m_mp); diff --git a/src/backend/gporca/libgpopt/src/operators/CLogicalCTEConsumer.cpp b/src/backend/gporca/libgpopt/src/operators/CLogicalCTEConsumer.cpp index d1557f5454d..f422e552e20 100644 --- a/src/backend/gporca/libgpopt/src/operators/CLogicalCTEConsumer.cpp +++ b/src/backend/gporca/libgpopt/src/operators/CLogicalCTEConsumer.cpp @@ -413,6 +413,7 @@ CLogicalCTEConsumer::PxfsCandidates(CMemoryPool *mp) const CXformSet *xform_set = GPOS_NEW(mp) CXformSet(mp); (void) xform_set->ExchangeSet(CXform::ExfInlineCTEConsumer); (void) xform_set->ExchangeSet(CXform::ExfImplementCTEConsumer); + (void) xform_set->ExchangeSet(CXform::ExfImplementParallelCTEConsumer); return xform_set; } diff --git a/src/backend/gporca/libgpopt/src/operators/CLogicalCTEProducer.cpp b/src/backend/gporca/libgpopt/src/operators/CLogicalCTEProducer.cpp index a1f9a5420f4..3007834570f 100644 --- a/src/backend/gporca/libgpopt/src/operators/CLogicalCTEProducer.cpp +++ b/src/backend/gporca/libgpopt/src/operators/CLogicalCTEProducer.cpp @@ -238,6 +238,7 @@ CLogicalCTEProducer::PxfsCandidates(CMemoryPool *mp) const { CXformSet *xform_set = GPOS_NEW(mp) CXformSet(mp); (void) xform_set->ExchangeSet(CXform::ExfImplementCTEProducer); + (void) xform_set->ExchangeSet(CXform::ExfImplementParallelCTEProducer); return xform_set; } diff --git a/src/backend/gporca/libgpopt/src/operators/CLogicalDynamicGet.cpp b/src/backend/gporca/libgpopt/src/operators/CLogicalDynamicGet.cpp index 3fbefec189d..2b6adc43a8f 100644 --- a/src/backend/gporca/libgpopt/src/operators/CLogicalDynamicGet.cpp +++ b/src/backend/gporca/libgpopt/src/operators/CLogicalDynamicGet.cpp @@ -244,6 +244,8 @@ CLogicalDynamicGet::PxfsCandidates(CMemoryPool *mp) const (void) xform_set->ExchangeSet(CXform::ExfDynamicGet2DynamicTableScan); (void) xform_set->ExchangeSet( CXform::ExfExpandDynamicGetWithForeignPartitions); + (void) xform_set->ExchangeSet(CXform::ExfDynamicGet2AppendTableScan); + (void) xform_set->ExchangeSet(CXform::ExfDynamicGet2ParallelAppendTableScan); return xform_set; } diff --git a/src/backend/gporca/libgpopt/src/operators/CLogicalGet.cpp b/src/backend/gporca/libgpopt/src/operators/CLogicalGet.cpp index ff93a28522c..8e6e3461d51 100644 --- a/src/backend/gporca/libgpopt/src/operators/CLogicalGet.cpp +++ b/src/backend/gporca/libgpopt/src/operators/CLogicalGet.cpp @@ -305,6 +305,8 @@ CLogicalGet::PxfsCandidates(CMemoryPool *mp) const CXformSet *xform_set = GPOS_NEW(mp) CXformSet(mp); (void) xform_set->ExchangeSet(CXform::ExfGet2TableScan); + // add parallel table scan + (void) xform_set->ExchangeSet(CXform::ExfGet2ParallelTableScan); return xform_set; } diff --git a/src/backend/gporca/libgpopt/src/operators/CLogicalSequence.cpp b/src/backend/gporca/libgpopt/src/operators/CLogicalSequence.cpp index 6dfd2918c5e..a17ba55da15 100644 --- a/src/backend/gporca/libgpopt/src/operators/CLogicalSequence.cpp +++ b/src/backend/gporca/libgpopt/src/operators/CLogicalSequence.cpp @@ -60,6 +60,7 @@ CLogicalSequence::PxfsCandidates(CMemoryPool *mp) const { CXformSet *xform_set = GPOS_NEW(mp) CXformSet(mp); (void) xform_set->ExchangeSet(CXform::ExfImplementSequence); + (void) xform_set->ExchangeSet(CXform::ExfImplementParallelSequence); return xform_set; } diff --git a/src/backend/gporca/libgpopt/src/operators/CPhysicalAppendTableScan.cpp b/src/backend/gporca/libgpopt/src/operators/CPhysicalAppendTableScan.cpp new file mode 100644 index 00000000000..66592599609 --- /dev/null +++ b/src/backend/gporca/libgpopt/src/operators/CPhysicalAppendTableScan.cpp @@ -0,0 +1,111 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalAppendTableScan.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/operators/CPhysicalAppendTableScan.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/operators/CPhysicalAppendTableScan.h" + +#include "gpos/base.h" + +#include "gpopt/base/CDistributionSpec.h" +#include "gpopt/base/CDistributionSpecHashed.h" +#include "gpopt/base/CDistributionSpecRandom.h" +#include "gpopt/base/CDistributionSpecSingleton.h" +#include "gpopt/base/CUtils.h" +#include "gpopt/metadata/CName.h" +#include "gpopt/metadata/CTableDescriptor.h" +#include "naucrates/statistics/CStatisticsUtils.h" + +using namespace gpopt; + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalAppendTableScan::CPhysicalAppendTableScan +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CPhysicalAppendTableScan::CPhysicalAppendTableScan(CMemoryPool *mp, + const CName *pnameAlias, + CTableDescriptor *ptabdesc, + ULONG ulOriginOpId, + ULONG scan_id, + CColRefArray *pdrgpcrOutput, + CColRef2dArray *pdrgpdrgpcrParts, + IMdIdArray *partition_mdids, + ColRefToUlongMapArray *root_col_mapping_per_part) + : CPhysicalDynamicScan(mp, ptabdesc, ulOriginOpId, pnameAlias, scan_id, + pdrgpcrOutput, pdrgpdrgpcrParts, partition_mdids, + root_col_mapping_per_part) +{ +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalAppendTableScan::Matches +// +// @doc: +// match operator +// +//--------------------------------------------------------------------------- +BOOL +CPhysicalAppendTableScan::Matches(COperator *pop) const +{ + return CUtils::FMatchDynamicScan(this, pop); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalAppendTableScan::PstatsDerive +// +// @doc: +// Statistics derivation during costing +// +//--------------------------------------------------------------------------- +IStatistics * +CPhysicalAppendTableScan::PstatsDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl, + CReqdPropPlan *prpplan, + IStatisticsArray * // stats_ctxt +) const +{ + GPOS_ASSERT(nullptr != prpplan); + + return CStatisticsUtils::DeriveStatsForDynamicScan( + mp, exprhdl, ScanId(), prpplan->Pepp()->PppsRequired()); +} + +CPartitionPropagationSpec * +CPhysicalAppendTableScan::PppsDerive(CMemoryPool *mp, CExpressionHandle &) const +{ + CPartitionPropagationSpec *pps = GPOS_NEW(mp) CPartitionPropagationSpec(mp); + pps->Insert(ScanId(), CPartitionPropagationSpec::EpptConsumer, + Ptabdesc()->MDId(), nullptr, nullptr); + + return pps; +} + +// EOF diff --git a/src/backend/gporca/libgpopt/src/operators/CPhysicalCTEProducer.cpp b/src/backend/gporca/libgpopt/src/operators/CPhysicalCTEProducer.cpp index c6be21f87cd..a13e89fda1e 100644 --- a/src/backend/gporca/libgpopt/src/operators/CPhysicalCTEProducer.cpp +++ b/src/backend/gporca/libgpopt/src/operators/CPhysicalCTEProducer.cpp @@ -14,6 +14,7 @@ #include "gpos/base.h" #include "gpopt/base/CCTEMap.h" +#include "gpopt/base/CDistributionSpecRandom.h" #include "gpopt/base/COptCtxt.h" #include "gpopt/operators/CExpression.h" #include "gpopt/operators/CExpressionHandle.h" diff --git a/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelAppendTableScan.cpp b/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelAppendTableScan.cpp new file mode 100644 index 00000000000..d10974d451f --- /dev/null +++ b/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelAppendTableScan.cpp @@ -0,0 +1,260 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalAppendTableScan.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/operators/CPhysicalAppendTableScan.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/operators/CPhysicalAppendTableScan.h" +#include "gpopt/operators/CPhysicalParallelAppendTableScan.h" + +#include "gpos/base.h" + +#include "gpopt/base/CDistributionSpec.h" +#include "gpopt/base/CDistributionSpecHashed.h" +#include "gpopt/base/CDistributionSpecRandom.h" +#include "gpopt/base/CDistributionSpecSingleton.h" +#include "gpopt/base/CDistributionSpecWorkerRandom.h" +#include "gpopt/base/CUtils.h" +#include "gpopt/metadata/CName.h" +#include "gpopt/metadata/CTableDescriptor.h" +#include "naucrates/statistics/CStatisticsUtils.h" + +using namespace gpopt; + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelAppendTableScan::CPhysicalParallelAppendTableScan +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CPhysicalParallelAppendTableScan::CPhysicalParallelAppendTableScan(CMemoryPool *mp, + const CName *pnameAlias, + CTableDescriptor *ptabdesc, + ULONG ulOriginOpId, + ULONG scan_id, + CColRefArray *pdrgpcrOutput, + CColRef2dArray *pdrgpdrgpcrParts, + IMdIdArray *partition_mdids, + ColRefToUlongMapArray *root_col_mapping_per_part, + ULONG ulParallelWorkers) + : CPhysicalDynamicScan(mp, ptabdesc, ulOriginOpId, pnameAlias, scan_id, + pdrgpcrOutput, pdrgpdrgpcrParts, partition_mdids, + root_col_mapping_per_part), + m_ulParallelWorkers(ulParallelWorkers), + m_pdsWorkerDistribution(nullptr) +{ + GPOS_ASSERT(ulParallelWorkers > 0); + GPOS_ASSERT(nullptr != m_pds); + // Create worker-level distribution based on table's segment distribution + if (ulParallelWorkers > 0 && nullptr != m_pds) + { + // Create worker-level random distribution using the table's distribution as base + // The base CPhysicalScan already sets up m_pds from the table descriptor + m_pdsWorkerDistribution = CDistributionSpecWorkerRandom::PdsCreateWorkerRandom(mp, ulParallelWorkers, m_pds); + } +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelAppendTableScan::~CPhysicalParallelAppendTableScan +// +// @doc: +// dtor +// +//--------------------------------------------------------------------------- +CPhysicalParallelAppendTableScan::~CPhysicalParallelAppendTableScan() +{ + CRefCount::SafeRelease(m_pdsWorkerDistribution); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelAppendTableScan::Matches +// +// @doc: +// match operator +// +//--------------------------------------------------------------------------- +BOOL +CPhysicalParallelAppendTableScan::Matches(COperator *pop) const +{ + if (Eopid() != pop->Eopid()) + { + return false; + } + + CPhysicalParallelAppendTableScan *popParallelAppendTableScan = + CPhysicalParallelAppendTableScan::PopConvert(pop); + + return m_ptabdesc->MDId()->Equals(popParallelAppendTableScan->Ptabdesc()->MDId()) && + m_pdrgpcrOutput->Equals(popParallelAppendTableScan->PdrgpcrOutput()) && + m_ulParallelWorkers == popParallelAppendTableScan->UlParallelWorkers(); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelAppendTableScan::PstatsDerive +// +// @doc: +// Statistics derivation during costing +// +//--------------------------------------------------------------------------- +IStatistics * +CPhysicalParallelAppendTableScan::PstatsDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl, + CReqdPropPlan *prpplan, + IStatisticsArray * // stats_ctxt +) const +{ + GPOS_ASSERT(nullptr != prpplan); + + return CStatisticsUtils::DeriveStatsForDynamicScan( + mp, exprhdl, ScanId(), prpplan->Pepp()->PppsRequired()); +} + +CPartitionPropagationSpec * +CPhysicalParallelAppendTableScan::PppsDerive(CMemoryPool *mp, CExpressionHandle &) const +{ + CPartitionPropagationSpec *pps = GPOS_NEW(mp) CPartitionPropagationSpec(mp); + pps->Insert(ScanId(), CPartitionPropagationSpec::EpptConsumer, + Ptabdesc()->MDId(), nullptr, nullptr); + + return pps; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelAppendTableScan::PdsDerive +// +// @doc: +// Derive distribution for parallel append table scan +// +//--------------------------------------------------------------------------- +CDistributionSpec * +CPhysicalParallelAppendTableScan::PdsDerive(CMemoryPool *mp, CExpressionHandle &exprhdl) const +{ + // If we have a pre-computed worker distribution, use it + if (nullptr != m_pdsWorkerDistribution) + { + m_pdsWorkerDistribution->AddRef(); + return m_pdsWorkerDistribution; + } + + // Otherwise, derive from the base physical scan + // This uses the m_pds member from CPhysicalScan + return CPhysicalScan::PdsDerive(mp, exprhdl); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelAppendTableScan::EpetDistribution +// +// @doc: +// Return the enforcing type for distribution property based on this +// operator +// +//--------------------------------------------------------------------------- +CEnfdProp::EPropEnforcingType +CPhysicalParallelAppendTableScan::EpetDistribution(CExpressionHandle & /*exprhdl*/, + const CEnfdDistribution *ped) const +{ + GPOS_ASSERT(nullptr != ped); + + // First check if worker-level distribution can satisfy the requirement + // This is the primary distribution for parallel scans + if (nullptr != m_pdsWorkerDistribution && ped->FCompatible(m_pdsWorkerDistribution)) + { + return CEnfdProp::EpetUnnecessary; + } + + // Neither distribution satisfies the requirement + // Motion enforcement will be needed on the output + return CEnfdProp::EpetRequired; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelAppendTableScan::EpetRewindability +// +// @doc: +// Return rewindability property enforcing type for this operator +// +//--------------------------------------------------------------------------- +CEnfdProp::EPropEnforcingType +CPhysicalParallelAppendTableScan::EpetRewindability(CExpressionHandle &exprhdl, + const CEnfdRewindability *per) const +{ + GPOS_ASSERT(nullptr != per); + + // Get derived rewindability from this operator + CRewindabilitySpec *prs = CDrvdPropPlan::Pdpplan(exprhdl.Pdp())->Prs(); + + // Check if our derived rewindability satisfies the requirement + if (per->FCompatible(prs)) + { + // Our derived rewindability (ErtNone) satisfies the requirement + return CEnfdProp::EpetUnnecessary; + } + + // Cannot satisfy the rewindability requirement + // GPORCA will need to add an enforcer (e.g., Spool) + return CEnfdProp::EpetRequired; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelAppendTableScan::FValidContext +// +// @doc: +// Check if optimization contexts is valid; +// Reject if parent requires REWINDABLE (e.g., for NL Join inner child) +// because ParallelAppendTableScan derives NONE (not rewindable) +// +//--------------------------------------------------------------------------- +BOOL +CPhysicalParallelAppendTableScan::FValidContext(CMemoryPool *, + COptimizationContext *poc, + COptimizationContextArray *) const +{ + GPOS_ASSERT(nullptr != poc); + + CReqdPropPlan *prpp = poc->Prpp(); + CRewindabilitySpec *prsRequired = prpp->Per()->PrsRequired(); + + // If parent requires REWINDABLE or higher, reject + // ParallelTableScan can only provide ErtNone + if (prsRequired->IsOriginNLJoin()) + { + // Parent requires rewindability (e.g., NL Join inner child) + // but ParallelTableScan cannot provide it + // Reject this plan to avoid the assertion failure later + return false; + } + + return true; +} + +// EOF diff --git a/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelCTEConsumer.cpp b/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelCTEConsumer.cpp new file mode 100644 index 00000000000..91c680bffdc --- /dev/null +++ b/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelCTEConsumer.cpp @@ -0,0 +1,406 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalParallelCTEConsumer.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/operators/CPhysicalParallelCTEConsumer.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/operators/CPhysicalParallelCTEConsumer.h" + +#include "gpos/base.h" + +#include "gpopt/base/CCTEMap.h" +#include "gpopt/base/COptCtxt.h" +#include "gpopt/operators/CExpression.h" +#include "gpopt/operators/CExpressionHandle.h" +#include "gpopt/operators/CLogicalCTEProducer.h" + +using namespace gpopt; + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::CPhysicalParallelCTEConsumer +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CPhysicalParallelCTEConsumer::CPhysicalParallelCTEConsumer(CMemoryPool *mp, ULONG id, + CColRefArray *colref_array, + UlongToColRefMap *colref_mapping, + ULONG ulParallelWorkers) + : CPhysical(mp), + m_id(id), + m_pdrgpcr(nullptr), + m_phmulcr(colref_mapping), + m_pidxmap(nullptr), + m_ulParallelWorkers(ulParallelWorkers) +{ + GPOS_ASSERT(nullptr != colref_array); + GPOS_ASSERT(nullptr != colref_mapping); + + ULONG colref_size = colref_array->Size(); + m_pdrgpcr = GPOS_NEW(mp) CColRefArray(mp); + m_pidxmap = GPOS_NEW(mp) ULongPtrArray(mp); + + for (ULONG index = 0; index < colref_size; index++) { + CColRef *col_ref = (*colref_array)[index]; + if (col_ref->GetUsage(true, true) == CColRef::EUsed) { + m_pdrgpcr->Append(col_ref); + m_pidxmap->Append(GPOS_NEW(m_mp) ULONG(index)); + } + } + + if (m_pidxmap->Size() == colref_size) { + m_pidxmap->Release(); + m_pidxmap = nullptr; + } +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::~CPhysicalParallelCTEConsumer +// +// @doc: +// Dtor +// +//--------------------------------------------------------------------------- +CPhysicalParallelCTEConsumer::~CPhysicalParallelCTEConsumer() +{ + m_pdrgpcr->Release(); + m_phmulcr->Release(); + CRefCount::SafeRelease(m_pidxmap); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::PcrsRequired +// +// @doc: +// Compute required output columns of the n-th child +// +//--------------------------------------------------------------------------- +CColRefSet * +CPhysicalParallelCTEConsumer::PcrsRequired(CMemoryPool *, // mp, + CExpressionHandle &, // exprhdl, + CColRefSet *, // pcrsRequired, + ULONG, // child_index, + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq +) +{ + GPOS_ASSERT(!"CPhysicalParallelCTEConsumer has no relational children"); + return nullptr; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::PosRequired +// +// @doc: +// Compute required sort order of the n-th child +// +//--------------------------------------------------------------------------- +COrderSpec * +CPhysicalParallelCTEConsumer::PosRequired(CMemoryPool *, // mp, + CExpressionHandle &, // exprhdl, + COrderSpec *, // posRequired, + ULONG, // child_index, + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq +) const +{ + GPOS_ASSERT(!"CPhysicalParallelCTEConsumer has no relational children"); + return nullptr; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::PdsRequired +// +// @doc: +// Compute required distribution of the n-th child +// +//--------------------------------------------------------------------------- +CDistributionSpec * +CPhysicalParallelCTEConsumer::PdsRequired(CMemoryPool *, // mp, + CExpressionHandle &, // exprhdl, + CDistributionSpec *, // pdsRequired, + ULONG, //child_index + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq +) const +{ + GPOS_ASSERT(!"CPhysicalParallelCTEConsumer has no relational children"); + return nullptr; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::PrsRequired +// +// @doc: +// Compute required rewindability of the n-th child +// +//--------------------------------------------------------------------------- +CRewindabilitySpec * +CPhysicalParallelCTEConsumer::PrsRequired(CMemoryPool *, // mp, + CExpressionHandle &, // exprhdl, + CRewindabilitySpec *, // prsRequired, + ULONG, // child_index, + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq +) const +{ + GPOS_ASSERT(!"CPhysicalParallelCTEConsumer has no relational children"); + return nullptr; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::PcteRequired +// +// @doc: +// Compute required CTE map of the n-th child +// +//--------------------------------------------------------------------------- +CCTEReq * +CPhysicalParallelCTEConsumer::PcteRequired(CMemoryPool *, //mp, + CExpressionHandle &, //exprhdl, + CCTEReq *, //pcter, + ULONG, //child_index, + CDrvdPropArray *, //pdrgpdpCtxt, + ULONG //ulOptReq +) const +{ + GPOS_ASSERT(!"CPhysicalParallelCTEConsumer has no relational children"); + return nullptr; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::PosDerive +// +// @doc: +// Derive sort order +// +//--------------------------------------------------------------------------- +COrderSpec * +CPhysicalParallelCTEConsumer::PosDerive(CMemoryPool *, // mp + CExpressionHandle & //exprhdl +) const +{ + GPOS_ASSERT(!"Unexpected call to parallel CTE consumer order property derivation"); + + return nullptr; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::PdsDerive +// +// @doc: +// Derive distribution +// +//--------------------------------------------------------------------------- +CDistributionSpec * +CPhysicalParallelCTEConsumer::PdsDerive(CMemoryPool *, // mp + CExpressionHandle & //exprhdl +) const +{ + GPOS_ASSERT( + !"Unexpected call to parallel CTE consumer distribution property derivation"); + + return nullptr; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::PrsDerive +// +// @doc: +// Derive rewindability +// +//--------------------------------------------------------------------------- +CRewindabilitySpec * +CPhysicalParallelCTEConsumer::PrsDerive(CMemoryPool *, //mp + CExpressionHandle & //exprhdl +) const +{ + GPOS_ASSERT( + !"Unexpected call to parallel CTE consumer rewindability property derivation"); + + return nullptr; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::PcmDerive +// +// @doc: +// Derive cte map +// +//--------------------------------------------------------------------------- +CCTEMap * +CPhysicalParallelCTEConsumer::PcmDerive(CMemoryPool *mp, CExpressionHandle & +#ifdef GPOS_DEBUG + exprhdl +#endif +) const +{ + GPOS_ASSERT(0 == exprhdl.Arity()); + + CCTEMap *pcmConsumer = GPOS_NEW(mp) CCTEMap(mp); + pcmConsumer->Insert(m_id, CCTEMap::EctConsumer, nullptr /*pdpplan*/); + + return pcmConsumer; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::FProvidesReqdCols +// +// @doc: +// Check if required columns are included in output columns +// +//--------------------------------------------------------------------------- +BOOL +CPhysicalParallelCTEConsumer::FProvidesReqdCols(CExpressionHandle &exprhdl, + CColRefSet *pcrsRequired, + ULONG // ulOptReq +) const +{ + GPOS_ASSERT(nullptr != pcrsRequired); + + CColRefSet *pcrsOutput = exprhdl.DeriveOutputColumns(); + return pcrsOutput->ContainsAll(pcrsRequired); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::EpetOrder +// +// @doc: +// Return the enforcing type for order property based on this operator +// +//--------------------------------------------------------------------------- +CEnfdProp::EPropEnforcingType +CPhysicalParallelCTEConsumer::EpetOrder(CExpressionHandle &exprhdl, + const CEnfdOrder *peo) const +{ + GPOS_ASSERT(nullptr != peo); + GPOS_ASSERT(!peo->PosRequired()->IsEmpty()); + + COrderSpec *pos = CDrvdPropPlan::Pdpplan(exprhdl.Pdp())->Pos(); + if (peo->FCompatible(pos)) + { + return CEnfdProp::EpetUnnecessary; + } + + return CEnfdProp::EpetRequired; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::EpetRewindability +// +// @doc: +// Return the enforcing type for rewindability property based on this operator +// +//--------------------------------------------------------------------------- +CEnfdProp::EPropEnforcingType +CPhysicalParallelCTEConsumer::EpetRewindability(CExpressionHandle &exprhdl, + const CEnfdRewindability *per) const +{ + GPOS_ASSERT(nullptr != per); + + CRewindabilitySpec *prs = CDrvdPropPlan::Pdpplan(exprhdl.Pdp())->Prs(); + if (per->FCompatible(prs)) + { + return CEnfdProp::EpetUnnecessary; + } + + return CEnfdProp::EpetRequired; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::Matches +// +// @doc: +// Match function +// +//--------------------------------------------------------------------------- +BOOL +CPhysicalParallelCTEConsumer::Matches(COperator *pop) const +{ + if (pop->Eopid() != Eopid()) + { + return false; + } + + CPhysicalParallelCTEConsumer *popCTEConsumer = + CPhysicalParallelCTEConsumer::PopConvert(pop); + + return m_id == popCTEConsumer->UlCTEId() && + m_pdrgpcr->Equals(popCTEConsumer->Pdrgpcr()); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::HashValue +// +// @doc: +// Hash function +// +//--------------------------------------------------------------------------- +ULONG +CPhysicalParallelCTEConsumer::HashValue() const +{ + ULONG ulHash = gpos::CombineHashes(COperator::HashValue(), m_id); + ulHash = gpos::CombineHashes(ulHash, CUtils::UlHashColArray(m_pdrgpcr)); + + return ulHash; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEConsumer::OsPrint +// +// @doc: +// debug print +// +//--------------------------------------------------------------------------- +IOstream & +CPhysicalParallelCTEConsumer::OsPrint(IOstream &os) const +{ + os << SzId() << " ("; + os << m_id; + os << "), Columns: ["; + CUtils::OsPrintDrgPcr(os, m_pdrgpcr); + os << "]"; + + return os; +} + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelCTEProducer.cpp b/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelCTEProducer.cpp new file mode 100644 index 00000000000..bf3e09aeb78 --- /dev/null +++ b/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelCTEProducer.cpp @@ -0,0 +1,431 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalParallelCTEProducer.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/operators/CPhysicalParallelCTEProducer.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/operators/CPhysicalParallelCTEProducer.h" + +#include "gpos/base.h" + +#include "gpopt/base/CCTEMap.h" +#include "gpopt/base/COptCtxt.h" +#include "gpopt/operators/CExpression.h" +#include "gpopt/operators/CExpressionHandle.h" +#include "gpopt/operators/CPhysicalSpool.h" + +using namespace gpopt; + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::CPhysicalParallelCTEProducer +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CPhysicalParallelCTEProducer::CPhysicalParallelCTEProducer(CMemoryPool *mp, + ULONG id, + CColRefArray *colref_array, + BOOL *umask, + ULONG ulParallelWorkers) + : CPhysical(mp), m_id(id), m_pdrgpcr(nullptr), m_pcrs(nullptr), m_pdrgpcr_unused(nullptr), m_pidxmap(nullptr), m_ulParallelWorkers(ulParallelWorkers) +{ + GPOS_ASSERT(nullptr != colref_array); + +#ifdef GPOS_DEBUG + if (umask) { + ULONG colref_size = colref_array->Size(); + for (ULONG ul = 0; ul < colref_size; ul++) { + CColRef *col_ref = (*colref_array)[ul]; + if (col_ref->GetUsage(true, true) != CColRef::EUsed) { + GPOS_ASSERT(!umask[ul]); + } + } + } +#endif + + if (umask) { + ULONG colref_size = colref_array->Size(); + ULONG unused_inc = 0; + m_pcrs = GPOS_NEW(mp) CColRefSet(mp); + m_pdrgpcr = GPOS_NEW(mp) CColRefArray(mp); + m_pdrgpcr_unused = GPOS_NEW(mp) ULongPtrArray(mp, colref_size); + m_pidxmap = GPOS_NEW(mp) ULongPtrArray(mp, colref_size); + + for (ULONG ul = 0; ul < colref_size; ul++) { + CColRef *col_ref = (*colref_array)[ul]; + m_pdrgpcr_unused->Append(GPOS_NEW(m_mp) ULONG(unused_inc)); + if (umask[ul]) { + m_pidxmap->Append(GPOS_NEW(m_mp) ULONG(m_pdrgpcr->Size())); + m_pdrgpcr->Append(col_ref); + m_pcrs->Include(col_ref); + } else { + m_pidxmap->Append(GPOS_NEW(m_mp) ULONG(gpos::ulong_max)); + unused_inc++; + } + } + + } else { + m_pdrgpcr = colref_array; + m_pcrs = GPOS_NEW(mp) CColRefSet(mp, m_pdrgpcr); + } +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::~CPhysicalParallelCTEProducer +// +// @doc: +// Dtor +// +//--------------------------------------------------------------------------- +CPhysicalParallelCTEProducer::~CPhysicalParallelCTEProducer() +{ + m_pdrgpcr->Release(); + m_pcrs->Release(); + CRefCount::SafeRelease(m_pdrgpcr_unused); + CRefCount::SafeRelease(m_pidxmap); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::PcrsRequired +// +// @doc: +// Compute required output columns of the n-th child +// +//--------------------------------------------------------------------------- +CColRefSet * +CPhysicalParallelCTEProducer::PcrsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CColRefSet *pcrsRequired, ULONG child_index, + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq +) +{ + GPOS_ASSERT(0 == child_index); + GPOS_ASSERT(0 == pcrsRequired->Size()); + + CColRefSet *pcrs = GPOS_NEW(mp) CColRefSet(mp); + ULONG ccr_size = m_pdrgpcr->Size(); + for (ULONG index = 0; index < ccr_size; index++) { + CColRef *col_ref = (*m_pdrgpcr)[index]; + GPOS_ASSERT(col_ref->GetUsage() != CColRef::EUnknown); + if (col_ref->GetUsage() == CColRef::EUsed) { + pcrs->Include(col_ref); + } + } + + pcrs->Union(pcrsRequired); + CColRefSet *pcrsChildReqd = + PcrsChildReqd(mp, exprhdl, pcrs, child_index, gpos::ulong_max); + + pcrs->Release(); + + return pcrsChildReqd; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::PosRequired +// +// @doc: +// Compute required sort order of the n-th child +// +//--------------------------------------------------------------------------- +COrderSpec * +CPhysicalParallelCTEProducer::PosRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + COrderSpec *posRequired, ULONG child_index, + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq +) const +{ + GPOS_ASSERT(0 == child_index); + + return PosPassThru(mp, exprhdl, posRequired, child_index); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::PdsRequired +// +// @doc: +// Compute required distribution of the n-th child +// +//--------------------------------------------------------------------------- +CDistributionSpec * +CPhysicalParallelCTEProducer::PdsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CDistributionSpec *pdsRequired, + ULONG child_index, + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq +) const +{ + GPOS_ASSERT(0 == child_index); + + return PdsPassThru(mp, exprhdl, pdsRequired, child_index); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::PrsRequired +// +// @doc: +// Compute required rewindability of the n-th child +// +//--------------------------------------------------------------------------- +CRewindabilitySpec * +CPhysicalParallelCTEProducer::PrsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CRewindabilitySpec *prsRequired, + ULONG child_index, + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq +) const +{ + GPOS_ASSERT(0 == child_index); + + return PrsPassThru(mp, exprhdl, prsRequired, child_index); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::PcteRequired +// +// @doc: +// Compute required CTE map of the n-th child +// +//--------------------------------------------------------------------------- +CCTEReq * +CPhysicalParallelCTEProducer::PcteRequired(CMemoryPool *, //mp, + CExpressionHandle &, //exprhdl, + CCTEReq *pcter, + ULONG +#ifdef GPOS_DEBUG +child_index +#endif + , + CDrvdPropArray *, //pdrgpdpCtxt, + ULONG //ulOptReq +) const +{ + GPOS_ASSERT(0 == child_index); + return PcterPushThru(pcter); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::PosDerive +// +// @doc: +// Derive sort order +// +//--------------------------------------------------------------------------- +COrderSpec * +CPhysicalParallelCTEProducer::PosDerive(CMemoryPool *, // mp + CExpressionHandle &exprhdl) const +{ + return PosDerivePassThruOuter(exprhdl); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::PdsDerive +// +// @doc: +// Derive distribution +// +//--------------------------------------------------------------------------- +CDistributionSpec * +CPhysicalParallelCTEProducer::PdsDerive(CMemoryPool *, // mp + CExpressionHandle &exprhdl) const +{ + return PdsDerivePassThruOuter(exprhdl); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::PrsDerive +// +// @doc: +// Derive rewindability +// +//--------------------------------------------------------------------------- +CRewindabilitySpec * +CPhysicalParallelCTEProducer::PrsDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const +{ + return PrsDerivePassThruOuter(mp, exprhdl); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::PcmDerive +// +// @doc: +// Derive cte map +// +//--------------------------------------------------------------------------- +CCTEMap * +CPhysicalParallelCTEProducer::PcmDerive(CMemoryPool *mp, + CExpressionHandle &exprhdl) const +{ + GPOS_ASSERT(1 == exprhdl.Arity()); + + CCTEMap *pcmChild = exprhdl.Pdpplan(0)->GetCostModel(); + + CCTEMap *pcmProducer = GPOS_NEW(mp) CCTEMap(mp); + // store plan properties of the child in producer's CTE map + pcmProducer->Insert(m_id, CCTEMap::EctProducer, exprhdl.Pdpplan(0)); + + CCTEMap *pcmCombined = CCTEMap::PcmCombine(mp, *pcmProducer, *pcmChild); + pcmProducer->Release(); + + return pcmCombined; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::FProvidesReqdCols +// +// @doc: +// Check if required columns are included in output columns +// +//--------------------------------------------------------------------------- +BOOL +CPhysicalParallelCTEProducer::FProvidesReqdCols(CExpressionHandle &exprhdl, + CColRefSet *pcrsRequired, + ULONG // ulOptReq +) const +{ + return FUnaryProvidesReqdCols(exprhdl, pcrsRequired); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::EpetOrder +// +// @doc: +// Return the enforcing type for order property based on this operator +// +//--------------------------------------------------------------------------- +CEnfdProp::EPropEnforcingType +CPhysicalParallelCTEProducer::EpetOrder(CExpressionHandle &exprhdl, + const CEnfdOrder *peo) const +{ + GPOS_ASSERT(nullptr != peo); + GPOS_ASSERT(!peo->PosRequired()->IsEmpty()); + + COrderSpec *pos = CDrvdPropPlan::Pdpplan(exprhdl.Pdp())->Pos(); + if (peo->FCompatible(pos)) + { + return CEnfdProp::EpetUnnecessary; + } + + return CEnfdProp::EpetRequired; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::EpetRewindability +// +// @doc: +// Return the enforcing type for rewindability property based on this operator +// +//--------------------------------------------------------------------------- +CEnfdProp::EPropEnforcingType +CPhysicalParallelCTEProducer::EpetRewindability(CExpressionHandle &exprhdl, + const CEnfdRewindability *per) const +{ + GPOS_ASSERT(nullptr != per); + + CRewindabilitySpec *prs = CDrvdPropPlan::Pdpplan(exprhdl.Pdp())->Prs(); + if (per->FCompatible(prs)) + { + return CEnfdProp::EpetUnnecessary; + } + + return CEnfdProp::EpetRequired; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::Matches +// +// @doc: +// Match function +// +//--------------------------------------------------------------------------- +BOOL +CPhysicalParallelCTEProducer::Matches(COperator *pop) const +{ + if (pop->Eopid() != Eopid()) + { + return false; + } + + CPhysicalParallelCTEProducer *popCTEProducer = + CPhysicalParallelCTEProducer::PopConvert(pop); + + return m_id == popCTEProducer->UlCTEId() && + m_pdrgpcr->Equals(popCTEProducer->Pdrgpcr()); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::HashValue +// +// @doc: +// Hash function +// +//--------------------------------------------------------------------------- +ULONG +CPhysicalParallelCTEProducer::HashValue() const +{ + ULONG ulHash = gpos::CombineHashes(COperator::HashValue(), m_id); + ulHash = gpos::CombineHashes(ulHash, CUtils::UlHashColArray(m_pdrgpcr)); + + return ulHash; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelCTEProducer::OsPrint +// +// @doc: +// debug print +// +//--------------------------------------------------------------------------- +IOstream & +CPhysicalParallelCTEProducer::OsPrint(IOstream &os) const +{ + os << SzId() << " ("; + os << m_id; + os << "), Columns: ["; + CUtils::OsPrintDrgPcr(os, m_pdrgpcr); + os << "]"; + + return os; +} + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelSequence.cpp b/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelSequence.cpp new file mode 100644 index 00000000000..e101047f893 --- /dev/null +++ b/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelSequence.cpp @@ -0,0 +1,390 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalParallelSequence.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/operators/CPhysicalParallelSequence.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/operators/CPhysicalParallelSequence.h" + +#include "gpos/base.h" + +#include "gpopt/base/CCTEReq.h" +#include "gpopt/base/CDistributionSpecAny.h" +#include "gpopt/base/CDistributionSpecNonSingleton.h" +#include "gpopt/base/CDistributionSpecReplicated.h" +#include "gpopt/base/CDistributionSpecSingleton.h" +#include "gpopt/base/CDistributionSpecWorkerRandom.h" +#include "gpopt/base/COptCtxt.h" +#include "gpopt/operators/CExpressionHandle.h" + +using namespace gpopt; + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::CPhysicalParallelSequence +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CPhysicalParallelSequence::CPhysicalParallelSequence(CMemoryPool *mp, ULONG ulParallelWorkers) + : CPhysical(mp), m_pcrsEmpty(nullptr), m_ulParallelWorkers(ulParallelWorkers) +{ + GPOS_ASSERT(ulParallelWorkers > 0); + // Sequence generates two distribution requests for its children: + // (1) If incoming distribution from above is Singleton, pass it through + // to all children, otherwise request Non-Singleton (Non-Replicated) + // on all children + // + // (2) Optimize first child with Any distribution requirement, and compute + // distribution request on other children based on derived distribution + // of first child: + // * If distribution of first child is a Singleton, request Singleton + // on the second child + // * If distribution of first child is Replicated, request Replicated + // on the second child + // * Otherwise, request Non-Singleton (Non-Replicated) on the second + // child + + SetDistrRequests(2); + + m_pcrsEmpty = GPOS_NEW(mp) CColRefSet(mp); +} + + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::~CPhysicalSequence +// +// @doc: +// Dtor +// +//--------------------------------------------------------------------------- +CPhysicalParallelSequence::~CPhysicalParallelSequence() +{ + m_pcrsEmpty->Release(); +} + + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::Matches +// +// @doc: +// Match operators +// +//--------------------------------------------------------------------------- +BOOL +CPhysicalParallelSequence::Matches(COperator *pop) const +{ + return Eopid() == pop->Eopid(); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::PcrsRequired +// +// @doc: +// Compute required output columns of n-th child +// +//--------------------------------------------------------------------------- +CColRefSet * +CPhysicalParallelSequence::PcrsRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CColRefSet *pcrsRequired, ULONG child_index, + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq +) +{ + const ULONG arity = exprhdl.Arity(); + if (child_index == arity - 1) + { + // request required columns from the last child of the sequence + return PcrsChildReqd(mp, exprhdl, pcrsRequired, child_index, + gpos::ulong_max); + } + + m_pcrsEmpty->AddRef(); + GPOS_ASSERT(0 == m_pcrsEmpty->Size()); + + return m_pcrsEmpty; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::PcteRequired +// +// @doc: +// Compute required CTE map of the n-th child +// +//--------------------------------------------------------------------------- +CCTEReq * +CPhysicalParallelSequence::PcteRequired(CMemoryPool *mp, CExpressionHandle &exprhdl, + CCTEReq *pcter, ULONG child_index, + CDrvdPropArray *pdrgpdpCtxt, + ULONG //ulOptReq +) const +{ + GPOS_ASSERT(nullptr != pcter); + if (child_index < exprhdl.Arity() - 1) + { + return pcter->PcterAllOptional(mp); + } + + // derived CTE maps from previous children + CCTEMap *pcmCombined = PcmCombine(mp, pdrgpdpCtxt); + + // pass the remaining requirements that have not been resolved + CCTEReq *pcterUnresolved = + pcter->PcterUnresolvedSequence(mp, pcmCombined, pdrgpdpCtxt); + pcmCombined->Release(); + + return pcterUnresolved; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::FProvidesReqdCols +// +// @doc: +// Helper for checking if required columns are included in output columns +// +//--------------------------------------------------------------------------- +BOOL +CPhysicalParallelSequence::FProvidesReqdCols(CExpressionHandle &exprhdl, + CColRefSet *pcrsRequired, + ULONG // ulOptReq +) const +{ + GPOS_ASSERT(nullptr != pcrsRequired); + + // last child must provide required columns + ULONG arity = exprhdl.Arity(); + GPOS_ASSERT(0 < arity); + + CColRefSet *pcrsChild = exprhdl.DeriveOutputColumns(arity - 1); + + return pcrsChild->ContainsAll(pcrsRequired); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::PdsRequired +// +// @doc: +// Compute required distribution of the n-th child +// +//--------------------------------------------------------------------------- +CDistributionSpec * +CPhysicalParallelSequence::PdsRequired(CMemoryPool *mp, + CExpressionHandle & +#ifdef GPOS_DEBUG + exprhdl +#endif // GPOS_DEBUG + , + CDistributionSpec *, + ULONG child_index, CDrvdPropArray *, + ULONG ulOptReq) const +{ + GPOS_ASSERT(2 == exprhdl.Arity()); + GPOS_ASSERT(child_index < exprhdl.Arity()); + GPOS_ASSERT(ulOptReq < UlDistrRequests()); + // always WorkerRandom distribution spec + // TODO compute m_ulParallelWorkers from children nodes + return GPOS_NEW(mp) CDistributionSpecWorkerRandom(m_ulParallelWorkers, nullptr); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::PosRequired +// +// @doc: +// Compute required sort order of the n-th child +// +//--------------------------------------------------------------------------- +COrderSpec * +CPhysicalParallelSequence::PosRequired(CMemoryPool *mp, + CExpressionHandle &, // exprhdl, + COrderSpec *, // posRequired, + ULONG, // child_index, + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq +) const +{ + // no order requirement on the children + return GPOS_NEW(mp) COrderSpec(mp); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::PrsRequired +// +// @doc: +// Compute required rewindability order of the n-th child +// +//--------------------------------------------------------------------------- +CRewindabilitySpec * +CPhysicalParallelSequence::PrsRequired(CMemoryPool *, // mp, + CExpressionHandle &, // exprhdl, + CRewindabilitySpec *prsRequired, + ULONG, // child_index, + CDrvdPropArray *, // pdrgpdpCtxt + ULONG // ulOptReq +) const +{ + // TODO: shardikar; Handle outer refs in the subtree correctly, by passing + // "Rescannable' Also, maybe it should pass through the prsRequired, since it + // doesn't materialize any results? It's important to consider performance + // consequences of that also. + return GPOS_NEW(m_mp) + CRewindabilitySpec(CRewindabilitySpec::ErtNone, prsRequired->Emht()); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::PosDerive +// +// @doc: +// Derive sort order +// +//--------------------------------------------------------------------------- +COrderSpec * +CPhysicalParallelSequence::PosDerive(CMemoryPool *, // mp, + CExpressionHandle &exprhdl) const +{ + // pass through sort order from last child + const ULONG arity = exprhdl.Arity(); + + GPOS_ASSERT(1 <= arity); + + COrderSpec *pos = exprhdl.Pdpplan(arity - 1 /*child_index*/)->Pos(); + pos->AddRef(); + + return pos; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::PdsDerive +// +// @doc: +// Derive distribution +// +//--------------------------------------------------------------------------- +CDistributionSpec * +CPhysicalParallelSequence::PdsDerive(CMemoryPool *, // mp, + CExpressionHandle &exprhdl) const +{ + // pass through distribution from last child + const ULONG arity = exprhdl.Arity(); + + GPOS_ASSERT(1 <= arity); + + CDistributionSpec *pds = exprhdl.Pdpplan(arity - 1 /*child_index*/)->Pds(); + pds->AddRef(); + + return pds; +} + + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::PrsDerive +// +// @doc: +// Derive rewindability +// +//--------------------------------------------------------------------------- +CRewindabilitySpec * +CPhysicalParallelSequence::PrsDerive(CMemoryPool *, //mp + CExpressionHandle &exprhdl) const +{ + const ULONG arity = exprhdl.Arity(); + GPOS_ASSERT(1 <= arity); + + CRewindabilitySpec::EMotionHazardType motion_hazard = + CRewindabilitySpec::EmhtNoMotion; + for (ULONG ul = 0; ul < arity; ul++) + { + CRewindabilitySpec *prs = exprhdl.Pdpplan(ul)->Prs(); + if (prs->HasMotionHazard()) + { + motion_hazard = CRewindabilitySpec::EmhtMotion; + break; + } + } + + // TODO: shardikar; Fix this implementation. Although CPhysicalSequence is + // not rewindable, all its children might be rewindable. This implementation + // ignores the rewindability of the op's children + return GPOS_NEW(m_mp) + CRewindabilitySpec(CRewindabilitySpec::ErtNone, motion_hazard); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::EpetOrder +// +// @doc: +// Return the enforcing type for the order property based on this operator +// +//--------------------------------------------------------------------------- +CEnfdProp::EPropEnforcingType +CPhysicalParallelSequence::EpetOrder(CExpressionHandle &exprhdl, + const CEnfdOrder *peo) const +{ + GPOS_ASSERT(nullptr != peo); + + // get order delivered by the sequence node + COrderSpec *pos = CDrvdPropPlan::Pdpplan(exprhdl.Pdp())->Pos(); + + if (peo->FCompatible(pos)) + { + // required order will be established by the sequence operator + return CEnfdProp::EpetUnnecessary; + } + + // required distribution will be enforced on sequence's output + return CEnfdProp::EpetRequired; +} + + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelSequence::EpetRewindability +// +// @doc: +// Return the enforcing type for rewindability property based on this operator +// +//--------------------------------------------------------------------------- +CEnfdProp::EPropEnforcingType +CPhysicalParallelSequence::EpetRewindability(CExpressionHandle &, // exprhdl + const CEnfdRewindability * // per +) const +{ + // rewindability must be enforced on operator's output + return CEnfdProp::EpetRequired; +} + + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelTableScan.cpp b/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelTableScan.cpp new file mode 100644 index 00000000000..a70a35981c6 --- /dev/null +++ b/src/backend/gporca/libgpopt/src/operators/CPhysicalParallelTableScan.cpp @@ -0,0 +1,284 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalParallelTableScan.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/operators/CPhysicalParallelTableScan.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/operators/CPhysicalParallelTableScan.h" + +#include "gpos/base.h" + +#include "gpopt/base/CDistributionSpec.h" +#include "gpopt/base/CDistributionSpecHashed.h" +#include "gpopt/base/CDistributionSpecRandom.h" +#include "gpopt/base/CDistributionSpecWorkerRandom.h" +#include "gpopt/base/CDistributionSpecSingleton.h" +#include "gpopt/base/CUtils.h" +#include "gpopt/base/CEnfdDistribution.h" +#include "gpopt/base/CEnfdRewindability.h" +#include "gpopt/base/COptimizationContext.h" +#include "gpopt/base/CRewindabilitySpec.h" +#include "gpopt/base/CDrvdPropPlan.h" +#include "gpopt/metadata/CName.h" +#include "gpopt/metadata/CTableDescriptor.h" +#include "gpopt/operators/CExpressionHandle.h" + +using namespace gpopt; + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelTableScan::CPhysicalParallelTableScan +// +// @doc: +// ctor +// +//--------------------------------------------------------------------------- +CPhysicalParallelTableScan::CPhysicalParallelTableScan(CMemoryPool *mp) + : CPhysicalTableScan(mp, GPOS_NEW(mp) CName(GPOS_NEW(mp) CWStringConst(GPOS_WSZ_LIT("parallel_table"))), nullptr, nullptr), + m_ulParallelWorkers(1), + m_pdsWorkerDistribution(nullptr) +{ +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelTableScan::CPhysicalParallelTableScan +// +// @doc: +// ctor +// +//--------------------------------------------------------------------------- +CPhysicalParallelTableScan::CPhysicalParallelTableScan(CMemoryPool *mp, + const CName *pnameAlias, + CTableDescriptor *ptabdesc, + CColRefArray *pdrgpcrOutput, + ULONG ulParallelWorkers) + : CPhysicalTableScan(mp, pnameAlias, ptabdesc, pdrgpcrOutput), + m_ulParallelWorkers(ulParallelWorkers), + m_pdsWorkerDistribution(nullptr) +{ + GPOS_ASSERT(ulParallelWorkers > 0); + GPOS_ASSERT(nullptr != m_pds); + // Create worker-level distribution based on table's segment distribution + if (ulParallelWorkers > 0 && nullptr != m_pds) + { + // Create worker-level random distribution using the table's distribution as base + // The base CPhysicalScan already sets up m_pds from the table descriptor + m_pdsWorkerDistribution = CDistributionSpecWorkerRandom::PdsCreateWorkerRandom(mp, ulParallelWorkers, m_pds); + } +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelTableScan::~CPhysicalParallelTableScan +// +// @doc: +// dtor +// +//--------------------------------------------------------------------------- +CPhysicalParallelTableScan::~CPhysicalParallelTableScan() +{ + CRefCount::SafeRelease(m_pdsWorkerDistribution); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelTableScan::HashValue +// +// @doc: +// Combine pointer for table descriptor, parallel workers and Eop +// +//--------------------------------------------------------------------------- +ULONG +CPhysicalParallelTableScan::HashValue() const +{ + ULONG ulHash = gpos::CombineHashes(CPhysicalTableScan::HashValue(), + gpos::HashValue(&m_ulParallelWorkers)); + return ulHash; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelTableScan::Matches +// +// @doc: +// match operator +// +//--------------------------------------------------------------------------- +BOOL +CPhysicalParallelTableScan::Matches(COperator *pop) const +{ + if (Eopid() != pop->Eopid()) + { + return false; + } + + CPhysicalParallelTableScan *popParallelTableScan = + CPhysicalParallelTableScan::PopConvert(pop); + + return CPhysicalTableScan::Matches(pop) && + m_ulParallelWorkers == popParallelTableScan->UlParallelWorkers(); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelTableScan::OsPrint +// +// @doc: +// debug print +// +//--------------------------------------------------------------------------- +IOstream & +CPhysicalParallelTableScan::OsPrint(IOstream &os) const +{ + os << SzId() << " "; + + // alias of table as referenced in the query + m_pnameAlias->OsPrint(os); + + // actual name of table in catalog and columns + os << " ("; + m_ptabdesc->Name().OsPrint(os); + os << "), Columns: ["; + + CUtils::OsPrintDrgPcr(os, m_pdrgpcrOutput); + os << "], Workers: " << m_ulParallelWorkers; + + return os; +} + + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelTableScan::PdsDerive +// +// @doc: +// Derive distribution for parallel table scan +// +//--------------------------------------------------------------------------- +CDistributionSpec * +CPhysicalParallelTableScan::PdsDerive(CMemoryPool *mp, CExpressionHandle &exprhdl) const +{ + // If we have a pre-computed worker distribution, use it + if (nullptr != m_pdsWorkerDistribution) + { + m_pdsWorkerDistribution->AddRef(); + return m_pdsWorkerDistribution; + } + + // Otherwise, derive from the base physical scan + // This uses the m_pds member from CPhysicalScan + return CPhysicalScan::PdsDerive(mp, exprhdl); +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelTableScan::EpetDistribution +// +// @doc: +// Return the enforcing type for distribution property based on this +// operator +// +//--------------------------------------------------------------------------- +CEnfdProp::EPropEnforcingType +CPhysicalParallelTableScan::EpetDistribution(CExpressionHandle & /*exprhdl*/, + const CEnfdDistribution *ped) const +{ + GPOS_ASSERT(nullptr != ped); + + // First check if worker-level distribution can satisfy the requirement + // This is the primary distribution for parallel scans + if (nullptr != m_pdsWorkerDistribution && ped->FCompatible(m_pdsWorkerDistribution)) + { + return CEnfdProp::EpetUnnecessary; + } + + // Neither distribution satisfies the requirement + // Motion enforcement will be needed on the output + return CEnfdProp::EpetRequired; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelTableScan::EpetRewindability +// +// @doc: +// Return rewindability property enforcing type for this operator +// +//--------------------------------------------------------------------------- +CEnfdProp::EPropEnforcingType +CPhysicalParallelTableScan::EpetRewindability(CExpressionHandle &exprhdl, + const CEnfdRewindability *per) const +{ + GPOS_ASSERT(nullptr != per); + + // Get derived rewindability from this operator + CRewindabilitySpec *prs = CDrvdPropPlan::Pdpplan(exprhdl.Pdp())->Prs(); + + // Check if our derived rewindability satisfies the requirement + if (per->FCompatible(prs)) + { + // Our derived rewindability (ErtNone) satisfies the requirement + return CEnfdProp::EpetUnnecessary; + } + + // Cannot satisfy the rewindability requirement + // GPORCA will need to add an enforcer (e.g., Spool) + return CEnfdProp::EpetRequired; +} + +//--------------------------------------------------------------------------- +// @function: +// CPhysicalParallelTableScan::FValidContext +// +// @doc: +// Check if optimization contexts is valid; +// Reject if parent requires REWINDABLE (e.g., for NL Join inner child) +// because ParallelTableScan derives NONE (not rewindable) +// +//--------------------------------------------------------------------------- +BOOL +CPhysicalParallelTableScan::FValidContext(CMemoryPool *, + COptimizationContext *poc, + COptimizationContextArray *) const +{ + GPOS_ASSERT(nullptr != poc); + + CReqdPropPlan *prpp = poc->Prpp(); + CRewindabilitySpec *prsRequired = prpp->Per()->PrsRequired(); + + // If parent requires REWINDABLE or higher, reject + // ParallelTableScan can only provide ErtNone + if (prsRequired->IsOriginNLJoin()) + { + // Parent requires rewindability (e.g., NL Join inner child) + // but ParallelTableScan cannot provide it + // Reject this plan to avoid the assertion failure later + return false; + } + + return true; +} + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/src/operators/Makefile b/src/backend/gporca/libgpopt/src/operators/Makefile index faa0f9a70d6..ecad1665ff3 100644 --- a/src/backend/gporca/libgpopt/src/operators/Makefile +++ b/src/backend/gporca/libgpopt/src/operators/Makefile @@ -85,6 +85,8 @@ OBJS = CExpression.o \ CPhysicalBitmapTableScan.o \ CPhysicalCTEConsumer.o \ CPhysicalCTEProducer.o \ + CPhysicalParallelCTEConsumer.o \ + CPhysicalParallelCTEProducer.o \ CPhysicalComputeScalar.o \ CPhysicalConstTableGet.o \ CPhysicalDML.o \ @@ -93,6 +95,8 @@ OBJS = CExpression.o \ CPhysicalDynamicIndexScan.o \ CPhysicalDynamicScan.o \ CPhysicalDynamicTableScan.o \ + CPhysicalAppendTableScan.o \ + CPhysicalParallelAppendTableScan.o \ CPhysicalForeignScan.o \ CPhysicalFilter.o \ CPhysicalFullMergeJoin.o \ @@ -123,12 +127,14 @@ OBJS = CExpression.o \ CPhysicalMotionRoutedDistribute.o \ CPhysicalDynamicForeignScan.o \ CPhysicalNLJoin.o \ + CPhysicalParallelTableScan.o \ CPhysicalParallelUnionAll.o \ CPhysicalPartitionSelector.o \ CPhysicalRightOuterHashJoin.o \ CPhysicalScalarAgg.o \ CPhysicalScan.o \ CPhysicalSequence.o \ + CPhysicalParallelSequence.o \ CPhysicalSequenceProject.o \ CPhysicalHashSequenceProject.o \ CPhysicalSerialUnionAll.o \ diff --git a/src/backend/gporca/libgpopt/src/optimizer/COptimizerConfig.cpp b/src/backend/gporca/libgpopt/src/optimizer/COptimizerConfig.cpp index e55a04862aa..a9486f33e09 100644 --- a/src/backend/gporca/libgpopt/src/optimizer/COptimizerConfig.cpp +++ b/src/backend/gporca/libgpopt/src/optimizer/COptimizerConfig.cpp @@ -33,14 +33,16 @@ COptimizerConfig::COptimizerConfig(CEnumeratorConfig *pec, CStatisticsConfig *stats_config, CCTEConfig *pcteconf, ICostModel *cost_model, CHint *phint, CPlanHint *pplanhint, - CWindowOids *pwindowoids) + CWindowOids *pwindowoids, + BOOL enable_parallel_plans) : m_enumerator_cfg(pec), m_stats_conf(stats_config), m_cte_conf(pcteconf), m_cost_model(cost_model), m_hint(phint), m_plan_hint(pplanhint), - m_window_oids(pwindowoids) + m_window_oids(pwindowoids), + m_create_parallel_plan(enable_parallel_plans) { GPOS_ASSERT(nullptr != pec); GPOS_ASSERT(nullptr != stats_config); @@ -85,7 +87,7 @@ COptimizerConfig::PoconfDefault(CMemoryPool *mp) CStatisticsConfig::PstatsconfDefault(mp), CCTEConfig::PcteconfDefault(mp), ICostModel::PcmDefault(mp), CHint::PhintDefault(mp), nullptr /* pplanhint */, - CWindowOids::GetWindowOids(mp)); + CWindowOids::GetWindowOids(mp), false /* enable_parallel_plans */); } //--------------------------------------------------------------------------- @@ -105,7 +107,7 @@ COptimizerConfig::PoconfDefault(CMemoryPool *mp, ICostModel *pcm) GPOS_NEW(mp) CEnumeratorConfig(mp, 0 /*plan_id*/, 0 /*ullSamples*/), CStatisticsConfig::PstatsconfDefault(mp), CCTEConfig::PcteconfDefault(mp), pcm, CHint::PhintDefault(mp), - nullptr /* pplanhint */, CWindowOids::GetWindowOids(mp)); + nullptr /* pplanhint */, CWindowOids::GetWindowOids(mp), false /* enable_parallel_plans */); } //--------------------------------------------------------------------------- diff --git a/src/backend/gporca/libgpopt/src/search/CGroup.cpp b/src/backend/gporca/libgpopt/src/search/CGroup.cpp index 18b4cf44e5b..02fd7fa9ccc 100644 --- a/src/backend/gporca/libgpopt/src/search/CGroup.cpp +++ b/src/backend/gporca/libgpopt/src/search/CGroup.cpp @@ -152,6 +152,7 @@ CGroup::SContextLink::Equals(const SContextLink *pclink1, //--------------------------------------------------------------------------- CGroup::CGroup(CMemoryPool *mp, BOOL fScalar) : m_mp(mp), + m_pmemo(nullptr), m_id(GPOPT_INVALID_GROUP_ID), m_fScalar(fScalar), m_pdrgpexprJoinKeysOuter(nullptr), @@ -534,6 +535,25 @@ CGroup::SetState(EState estNewState) } +//--------------------------------------------------------------------------- +// @function: +// CGroup::SetMemo +// +// @doc: +// Set containing memo reference; +// +//--------------------------------------------------------------------------- +void +CGroup::SetMemo(CMemo *pmemo) +{ + GPOS_ASSERT(nullptr != pmemo); + GPOS_ASSERT(nullptr == m_pmemo && + "Overwriting previously assigned memo reference"); + + m_pmemo = pmemo; +} + + void CGroup::SetJoinKeys(CExpressionArray *pdrgpexprOuter, CExpressionArray *pdrgpexprInner, diff --git a/src/backend/gporca/libgpopt/src/search/CGroupExpression.cpp b/src/backend/gporca/libgpopt/src/search/CGroupExpression.cpp index dffd10cb35f..bff8ef7068f 100644 --- a/src/backend/gporca/libgpopt/src/search/CGroupExpression.cpp +++ b/src/backend/gporca/libgpopt/src/search/CGroupExpression.cpp @@ -179,7 +179,8 @@ CGroupExpression::SetOptimizationLevel() // a sequence expression with a first child group that contains a CTE // producer gets a higher optimization level. This is to be sure that the // producer gets optimized before its consumers - if (COperator::EopPhysicalSequence == m_pop->Eopid()) + if (COperator::EopPhysicalSequence == m_pop->Eopid() || + COperator::EopPhysicalParallelSequence == m_pop->Eopid()) { CGroup *pgroupFirst = (*this)[0]; if (pgroupFirst->FHasCTEProducer()) diff --git a/src/backend/gporca/libgpopt/src/search/CJobGroupExpressionOptimization.cpp b/src/backend/gporca/libgpopt/src/search/CJobGroupExpressionOptimization.cpp index 404a772c1d3..b0559433d72 100644 --- a/src/backend/gporca/libgpopt/src/search/CJobGroupExpressionOptimization.cpp +++ b/src/backend/gporca/libgpopt/src/search/CJobGroupExpressionOptimization.cpp @@ -219,7 +219,8 @@ CJobGroupExpressionOptimization::Init(CGroupExpression *pgexpr, m_ulOptReq = ulOptReq; m_fChildOptimizationFailed = false; m_fOptimizeCTESequence = - (COperator::EopPhysicalSequence == pgexpr->Pop()->Eopid() && + ((COperator::EopPhysicalSequence == pgexpr->Pop()->Eopid() || + COperator::EopPhysicalParallelSequence == pgexpr->Pop()->Eopid()) && (*pgexpr)[0]->FHasCTEProducer()); if (nullptr != prppCTEProducer) { diff --git a/src/backend/gporca/libgpopt/src/search/CMemo.cpp b/src/backend/gporca/libgpopt/src/search/CMemo.cpp index 5248300aa86..d92da7a9ed9 100644 --- a/src/backend/gporca/libgpopt/src/search/CMemo.cpp +++ b/src/backend/gporca/libgpopt/src/search/CMemo.cpp @@ -140,6 +140,7 @@ CMemo::Add( GPOS_ASSERT(nullptr != gp.PgexprFirst()); gp.SetId(id); + gp.SetMemo(this); gp.InitProperties(pdp); } diff --git a/src/backend/gporca/libgpopt/src/translate/CTranslatorExprToDXL.cpp b/src/backend/gporca/libgpopt/src/translate/CTranslatorExprToDXL.cpp index 6119e2ba71f..c47b2d3bc55 100644 --- a/src/backend/gporca/libgpopt/src/translate/CTranslatorExprToDXL.cpp +++ b/src/backend/gporca/libgpopt/src/translate/CTranslatorExprToDXL.cpp @@ -30,10 +30,15 @@ #include "gpopt/exception.h" #include "gpopt/mdcache/CMDAccessorUtils.h" #include "gpopt/operators/CPhysicalAgg.h" +#include "gpopt/operators/CPhysicalAppendTableScan.h" +#include "gpopt/operators/CPhysicalParallelAppendTableScan.h" #include "gpopt/operators/CPhysicalAssert.h" #include "gpopt/operators/CPhysicalBitmapTableScan.h" #include "gpopt/operators/CPhysicalCTEConsumer.h" #include "gpopt/operators/CPhysicalCTEProducer.h" +#include "gpopt/operators/CPhysicalParallelCTEConsumer.h" +#include "gpopt/operators/CPhysicalParallelCTEProducer.h" +#include "gpopt/operators/CPhysicalParallelSequence.h" #include "gpopt/operators/CPhysicalConstTableGet.h" #include "gpopt/operators/CPhysicalCorrelatedLeftOuterNLJoin.h" #include "gpopt/operators/CPhysicalDML.h" @@ -66,6 +71,7 @@ #include "gpopt/operators/CPhysicalStreamAggDeduplicate.h" #include "gpopt/operators/CPhysicalTVF.h" #include "gpopt/operators/CPhysicalTableScan.h" +#include "gpopt/operators/CPhysicalParallelTableScan.h" #include "gpopt/operators/CPhysicalUnionAll.h" #include "gpopt/operators/CPredicateUtils.h" #include "gpopt/operators/CScalarArray.h" @@ -99,12 +105,15 @@ #include "naucrates/dxl/operators/CDXLDatumBool.h" #include "naucrates/dxl/operators/CDXLDirectDispatchInfo.h" #include "naucrates/dxl/operators/CDXLPhysicalAppend.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelAppend.h" #include "naucrates/dxl/operators/CDXLPhysicalAssert.h" #include "naucrates/dxl/operators/CDXLPhysicalBitmapTableScan.h" #include "naucrates/dxl/operators/CDXLPhysicalBroadcastMotion.h" #include "naucrates/dxl/operators/CDXLPhysicalCTAS.h" #include "naucrates/dxl/operators/CDXLPhysicalCTEConsumer.h" #include "naucrates/dxl/operators/CDXLPhysicalCTEProducer.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelCTEConsumer.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelCTEProducer.h" #include "naucrates/dxl/operators/CDXLPhysicalDynamicBitmapTableScan.h" #include "naucrates/dxl/operators/CDXLPhysicalDynamicForeignScan.h" #include "naucrates/dxl/operators/CDXLPhysicalDynamicIndexOnlyScan.h" @@ -125,10 +134,12 @@ #include "naucrates/dxl/operators/CDXLPhysicalResult.h" #include "naucrates/dxl/operators/CDXLPhysicalRoutedDistributeMotion.h" #include "naucrates/dxl/operators/CDXLPhysicalSequence.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelSequence.h" #include "naucrates/dxl/operators/CDXLPhysicalSort.h" #include "naucrates/dxl/operators/CDXLPhysicalSplit.h" #include "naucrates/dxl/operators/CDXLPhysicalTVF.h" #include "naucrates/dxl/operators/CDXLPhysicalTableScan.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelTableScan.h" #include "naucrates/dxl/operators/CDXLPhysicalWindow.h" #include "naucrates/dxl/operators/CDXLScalarAggref.h" #include "naucrates/dxl/operators/CDXLScalarArray.h" @@ -342,6 +353,7 @@ CTranslatorExprToDXL::CreateDXLNode(CExpression *pexpr, GPOS_ASSERT(nullptr != pexpr); ULONG ulOpId = (ULONG) pexpr->Pop()->Eopid(); if (COperator::EopPhysicalTableScan == ulOpId || + COperator::EopPhysicalParallelTableScan == ulOpId || COperator::EopPhysicalForeignScan == ulOpId) { CDXLNode *dxlnode = PdxlnTblScan( @@ -352,6 +364,13 @@ CTranslatorExprToDXL::CreateDXLNode(CExpression *pexpr, return dxlnode; } + else if (COperator::EopPhysicalParallelAppendTableScan == ulOpId && m_isSet) + { + CDXLNode *dxlnode = PdxlnAppendTableScan( + pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, pfDML); + + return dxlnode; + } // add a result node on top to project out columns not needed any further, // for instance, if the grouping /order by /partition/ distribution columns // are no longer needed @@ -465,11 +484,26 @@ CTranslatorExprToDXL::CreateDXLNode(CExpression *pexpr, pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, pfDML); break; + case COperator::EopPhysicalParallelSequence: + dxlnode = CTranslatorExprToDXL::PdxlnParallelSequence( + pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, + pfDML); + break; case COperator::EopPhysicalDynamicTableScan: dxlnode = CTranslatorExprToDXL::PdxlnDynamicTableScan( pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, pfDML); break; + case COperator::EopPhysicalAppendTableScan: + dxlnode = CTranslatorExprToDXL::PdxlnAppendTableScan( + pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, + pfDML); + break; + case COperator::EopPhysicalParallelAppendTableScan: + dxlnode = CTranslatorExprToDXL::PdxlnParallelAppendTableScan( + pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, + pfDML); + break; case COperator::EopPhysicalDynamicBitmapTableScan: dxlnode = CTranslatorExprToDXL::PdxlnDynamicBitmapTableScan( pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, @@ -531,11 +565,21 @@ CTranslatorExprToDXL::CreateDXLNode(CExpression *pexpr, pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, pfDML); break; + case COperator::EopPhysicalParallelCTEProducer: + dxlnode = CTranslatorExprToDXL::PdxlnParallelCTEProducer( + pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, + pfDML); + break; case COperator::EopPhysicalCTEConsumer: dxlnode = CTranslatorExprToDXL::PdxlnCTEConsumer( pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, pfDML); break; + case COperator::EopPhysicalParallelCTEConsumer: + dxlnode = CTranslatorExprToDXL::PdxlnParallelCTEConsumer( + pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, + pfDML); + break; case COperator::EopPhysicalFullMergeJoin: dxlnode = CTranslatorExprToDXL::PdxlnMergeJoin( pexpr, colref_array, pdrgpdsBaseTables, pulNonGatherMotions, @@ -711,6 +755,13 @@ CTranslatorExprToDXL::PdxlnTblScan(CExpression *pexprTblScan, { pdxlopTS = GPOS_NEW(m_mp) CDXLPhysicalTableScan(m_mp, table_descr); } + else if (COperator::EopPhysicalParallelTableScan == op_id) + { + CPhysicalParallelTableScan *parallel_scan = + CPhysicalParallelTableScan::PopConvert(pexprTblScan->Pop()); + pdxlopTS = GPOS_NEW(m_mp) CDXLPhysicalParallelTableScan( + m_mp, table_descr, parallel_scan->UlParallelWorkers()); + } else { GPOS_ASSERT(COperator::EopPhysicalForeignScan == op_id); @@ -1264,6 +1315,52 @@ CTranslatorExprToDXL::PdxlnDynamicTableScan( pexprScalarCond, dxl_properties); } +//--------------------------------------------------------------------------- +// @function: +// CTranslatorExprToDXL::PdxlnAppendTableScan +// +// @doc: +// Create a DXL append table scan node from an optimizer +// append table scan node. +// +//--------------------------------------------------------------------------- +CDXLNode * +CTranslatorExprToDXL::PdxlnAppendTableScan( + CExpression *pexprDTS, CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, + ULONG *, // pulNonGatherMotions, + BOOL * // pfDML +) +{ + CExpression *pexprScalarCond = nullptr; + CDXLPhysicalProperties *dxl_properties = nullptr; + return PdxlnAppendTableScan(pexprDTS, colref_array, pdrgpdsBaseTables, + pexprScalarCond, dxl_properties); +} + +//--------------------------------------------------------------------------- +// @function: +// CTranslatorExprToDXL::PdxlnParallelAppendTableScan +// +// @doc: +// Create a DXL parallel append table scan node from an optimizer +// parallel append table scan node. +// +//--------------------------------------------------------------------------- +CDXLNode * +CTranslatorExprToDXL::PdxlnParallelAppendTableScan( + CExpression *pexprDTS, CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, + ULONG *, // pulNonGatherMotions, + BOOL * // pfDML +) +{ + CExpression *pexprScalarCond = nullptr; + CDXLPhysicalProperties *dxl_properties = nullptr; + return PdxlnParallelAppendTableScan(pexprDTS, colref_array, pdrgpdsBaseTables, + pexprScalarCond, dxl_properties); +} + // Construct a dxl table descr for a child partition CTableDescriptor * CTranslatorExprToDXL::MakeTableDescForPart(const IMDRelation *part, @@ -1434,6 +1531,291 @@ CTranslatorExprToDXL::PdxlnDynamicTableScan( return pdxlnDTS; } +// Translate CPhysicalAppendTableScan node. It creates a CDXLPhysicalAppend +// node over a number of CDXLPhysicalTableScan nodes - include all child partition +// tables of the root partition. +// +// To handle dropped and re-ordered columns, the project list and any filter +// expression from the root table are modified using the per partition mappings +// for each child CDXLPhysicalTableScan +CDXLNode * +CTranslatorExprToDXL::PdxlnAppendTableScan( + CExpression *pexprDTS, CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, CExpression *pexprScalarCond, + CDXLPhysicalProperties *dxl_properties) +{ + CPhysicalDynamicScan *popDTS = + CPhysicalDynamicScan::PopConvert(pexprDTS->Pop()); + + ULongPtrArray *selector_ids = GPOS_NEW(m_mp) ULongPtrArray(m_mp); + CPartitionPropagationSpec *pps_reqd = + pexprDTS->Prpp()->Pepp()->PppsRequired(); + if (pps_reqd->Contains(popDTS->ScanId())) + { + const CBitSet *bs = pps_reqd->SelectorIds(popDTS->ScanId()); + CBitSetIter bsi(*bs); + for (ULONG ul = 0; bsi.Advance(); ul++) + { + selector_ids->Append(GPOS_NEW(m_mp) ULONG(bsi.Bit())); + } + } + + // construct plan costs + CDXLPhysicalProperties *pdxlpropDTS = GetProperties(pexprDTS); + + if (nullptr != dxl_properties) + { + CWStringDynamic *rows_out_str = GPOS_NEW(m_mp) CWStringDynamic( + m_mp, + dxl_properties->GetDXLOperatorCost()->GetRowsOutStr()->GetBuffer()); + CWStringDynamic *pstrCost = GPOS_NEW(m_mp) + CWStringDynamic(m_mp, dxl_properties->GetDXLOperatorCost() + ->GetTotalCostStr() + ->GetBuffer()); + + pdxlpropDTS->GetDXLOperatorCost()->SetRows(rows_out_str); + pdxlpropDTS->GetDXLOperatorCost()->SetCost(pstrCost); + dxl_properties->Release(); + } + GPOS_ASSERT(nullptr != pexprDTS->Prpp()); + + // construct projection list for top-level Append node + CColRefSet *pcrsOutput = pexprDTS->Prpp()->PcrsRequired(); + CDXLNode *pdxlnPrLAppend = PdxlnProjList(pcrsOutput, colref_array); + CDXLTableDescr *root_dxl_table_descr = MakeDXLTableDescr( + popDTS->Ptabdesc(), popDTS->PdrgpcrOutput(), pexprDTS->Prpp()); + + // Construct the Append node - even when there is only one child partition. + // This is done for two reasons: + // * Dynamic partition pruning + // Even if one partition is present in the statically pruned plan, we could + // still dynamically prune it away. This needs an Append node. + // * Col mappings issues + // When the first selected child partition's cols have different types/order + // than the root partition, we can no longer re-use the colrefs of the root + // partition, since colrefs are immutable. Thus, we create new colrefs for + // this partition. But, if there is no Append (in case of just one selected + // partition), then we also go through update all references above the DTS + // with the new colrefs. For simplicity, we decided to keep the Append + // around to maintain this projection (mapping) from the old root colrefs + // to the first selected partition colrefs. + // + // GPDB_12_MERGE_FIXME: An Append on a single TableScan can be removed in + // CTranslatorDXLToPlstmt since these points do not apply there. + CDXLNode *pdxlnAppend = GPOS_NEW(m_mp) CDXLNode( + m_mp, + GPOS_NEW(m_mp) CDXLPhysicalAppend(m_mp, false, false, popDTS->ScanId(), + root_dxl_table_descr, selector_ids)); + pdxlnAppend->SetProperties(pdxlpropDTS); + pdxlnAppend->AddChild(pdxlnPrLAppend); + pdxlnAppend->AddChild(PdxlnFilter(nullptr)); + + IMdIdArray *part_mdids = popDTS->GetPartitionMdids(); + for (ULONG ul = 0; ul < part_mdids->Size(); ++ul) + { + IMDId *part_mdid = (*part_mdids)[ul]; + const IMDRelation *part = m_pmda->RetrieveRel(part_mdid); + + CTableDescriptor *part_tabdesc = + MakeTableDescForPart(part, popDTS->Ptabdesc()); + + // Create new colrefs for the child partition. The ColRefs from root + // DTS, which may be used in any parent node, can no longer be exported + // by a child of the Append node. Thus it is exported by the Append + // node itself, and new colrefs are created here. + CColRefArray *part_colrefs = GPOS_NEW(m_mp) CColRefArray(m_mp); + for (ULONG ul_col = 0; ul_col < part_tabdesc->ColumnCount(); ++ul_col) + { + const CColumnDescriptor *cd = part_tabdesc->Pcoldesc(ul_col); + CColRef *cr = m_pcf->PcrCreate(cd->RetrieveType(), + cd->TypeModifier(), cd->Name()); + part_colrefs->Append(cr); + } + + CDXLTableDescr *dxl_table_descr = + MakeDXLTableDescr(part_tabdesc, part_colrefs, pexprDTS->Prpp()); + part_tabdesc->Release(); + + CDXLNode *dxlnode = GPOS_NEW(m_mp) CDXLNode( + m_mp, GPOS_NEW(m_mp) CDXLPhysicalTableScan(m_mp, dxl_table_descr)); + + // GPDB_12_MERGE_FIXME: Compute stats & properties per scan + pdxlpropDTS->AddRef(); + dxlnode->SetProperties(pdxlpropDTS); + + // ColRef -> index in child table desc (per partition) + auto root_col_mapping = (*popDTS->GetRootColMappingPerPart())[ul]; + + // construct projection list, re-ordered to match root DTS + CDXLNode *pdxlnPrL = PdxlnProjListForChildPart( + root_col_mapping, part_colrefs, pcrsOutput, colref_array); + dxlnode->AddChild(pdxlnPrL); // project list + + // construct the filter + CDXLNode *filter_dxlnode = PdxlnFilter( + PdxlnCondForChildPart(root_col_mapping, part_colrefs, + popDTS->PdrgpcrOutput(), pexprScalarCond)); + dxlnode->AddChild(filter_dxlnode); // filter + + // add to the other scans under the created Append node + pdxlnAppend->AddChild(dxlnode); + + // cleanup + part_colrefs->Release(); + } + + CDistributionSpec *pds = pexprDTS->GetDrvdPropPlan()->Pds(); + pds->AddRef(); + pdrgpdsBaseTables->Append(pds); + + GPOS_ASSERT(pdxlnAppend); + return pdxlnAppend; +} + +// Translate CPhysicalParallelAppendTableScan node. It creates a CDXLPhysicalParallelAppend +// node over a number of CDXLPhysicalParallelTableScan nodes - include all child partition +// tables of the root partition. +// +// To handle dropped and re-ordered columns, the project list and any filter +// expression from the root table are modified using the per partition mappings +// for each child CDXLPhysicalParallelTableScan +CDXLNode * +CTranslatorExprToDXL::PdxlnParallelAppendTableScan( + CExpression *pexprDTS, CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, CExpression *pexprScalarCond, + CDXLPhysicalProperties *dxl_properties) +{ + ULONG ulParallelWorkers = 0; + + CPhysicalParallelAppendTableScan *popDTS = + CPhysicalParallelAppendTableScan::PopConvert(pexprDTS->Pop()); + + ulParallelWorkers = popDTS->UlParallelWorkers(); + ULongPtrArray *selector_ids = GPOS_NEW(m_mp) ULongPtrArray(m_mp); + CPartitionPropagationSpec *pps_reqd = + pexprDTS->Prpp()->Pepp()->PppsRequired(); + if (pps_reqd->Contains(popDTS->ScanId())) + { + const CBitSet *bs = pps_reqd->SelectorIds(popDTS->ScanId()); + CBitSetIter bsi(*bs); + for (ULONG ul = 0; bsi.Advance(); ul++) + { + selector_ids->Append(GPOS_NEW(m_mp) ULONG(bsi.Bit())); + } + } + + // construct plan costs + CDXLPhysicalProperties *pdxlpropDTS = GetProperties(pexprDTS); + + if (nullptr != dxl_properties) + { + CWStringDynamic *rows_out_str = GPOS_NEW(m_mp) CWStringDynamic( + m_mp, + dxl_properties->GetDXLOperatorCost()->GetRowsOutStr()->GetBuffer()); + CWStringDynamic *pstrCost = GPOS_NEW(m_mp) + CWStringDynamic(m_mp, dxl_properties->GetDXLOperatorCost() + ->GetTotalCostStr() + ->GetBuffer()); + + pdxlpropDTS->GetDXLOperatorCost()->SetRows(rows_out_str); + pdxlpropDTS->GetDXLOperatorCost()->SetCost(pstrCost); + dxl_properties->Release(); + } + GPOS_ASSERT(nullptr != pexprDTS->Prpp()); + + // construct projection list for top-level Append node + CColRefSet *pcrsOutput = pexprDTS->Prpp()->PcrsRequired(); + CDXLNode *pdxlnPrLAppend = PdxlnProjList(pcrsOutput, colref_array); + CDXLTableDescr *root_dxl_table_descr = MakeDXLTableDescr( + popDTS->Ptabdesc(), popDTS->PdrgpcrOutput(), pexprDTS->Prpp()); + + // Construct the Append node - even when there is only one child partition. + // This is done for two reasons: + // * Dynamic partition pruning + // Even if one partition is present in the statically pruned plan, we could + // still dynamically prune it away. This needs an Append node. + // * Col mappings issues + // When the first selected child partition's cols have different types/order + // than the root partition, we can no longer re-use the colrefs of the root + // partition, since colrefs are immutable. Thus, we create new colrefs for + // this partition. But, if there is no Append (in case of just one selected + // partition), then we also go through update all references above the DTS + // with the new colrefs. For simplicity, we decided to keep the Append + // around to maintain this projection (mapping) from the old root colrefs + // to the first selected partition colrefs. + // + // GPDB_12_MERGE_FIXME: An Append on a single TableScan can be removed in + // CTranslatorDXLToPlstmt since these points do not apply there. + CDXLNode *pdxlnAppend = GPOS_NEW(m_mp) CDXLNode( + m_mp, + GPOS_NEW(m_mp) CDXLPhysicalParallelAppend(m_mp, false, false, popDTS->ScanId(), + root_dxl_table_descr, selector_ids, ulParallelWorkers)); + pdxlnAppend->SetProperties(pdxlpropDTS); + pdxlnAppend->AddChild(pdxlnPrLAppend); + pdxlnAppend->AddChild(PdxlnFilter(nullptr)); + + IMdIdArray *part_mdids = popDTS->GetPartitionMdids(); + for (ULONG ul = 0; ul < part_mdids->Size(); ++ul) + { + IMDId *part_mdid = (*part_mdids)[ul]; + const IMDRelation *part = m_pmda->RetrieveRel(part_mdid); + + CTableDescriptor *part_tabdesc = + MakeTableDescForPart(part, popDTS->Ptabdesc()); + + // Create new colrefs for the child partition. The ColRefs from root + // DTS, which may be used in any parent node, can no longer be exported + // by a child of the Append node. Thus it is exported by the Append + // node itself, and new colrefs are created here. + CColRefArray *part_colrefs = GPOS_NEW(m_mp) CColRefArray(m_mp); + for (ULONG ul_col = 0; ul_col < part_tabdesc->ColumnCount(); ++ul_col) + { + const CColumnDescriptor *cd = part_tabdesc->Pcoldesc(ul_col); + CColRef *cr = m_pcf->PcrCreate(cd->RetrieveType(), + cd->TypeModifier(), cd->Name()); + part_colrefs->Append(cr); + } + + CDXLTableDescr *dxl_table_descr = + MakeDXLTableDescr(part_tabdesc, part_colrefs, pexprDTS->Prpp()); + part_tabdesc->Release(); + + CDXLNode *dxlnode = GPOS_NEW(m_mp) CDXLNode( + m_mp, GPOS_NEW(m_mp) CDXLPhysicalParallelTableScan(m_mp, dxl_table_descr, ulParallelWorkers)); + + // GPDB_12_MERGE_FIXME: Compute stats & properties per scan + pdxlpropDTS->AddRef(); + dxlnode->SetProperties(pdxlpropDTS); + + // ColRef -> index in child table desc (per partition) + auto root_col_mapping = (*popDTS->GetRootColMappingPerPart())[ul]; + + // construct projection list, re-ordered to match root DTS + CDXLNode *pdxlnPrL = PdxlnProjListForChildPart( + root_col_mapping, part_colrefs, pcrsOutput, colref_array); + dxlnode->AddChild(pdxlnPrL); // project list + + // construct the filter + CDXLNode *filter_dxlnode = PdxlnFilter( + PdxlnCondForChildPart(root_col_mapping, part_colrefs, + popDTS->PdrgpcrOutput(), pexprScalarCond)); + dxlnode->AddChild(filter_dxlnode); // filter + + // add to the other scans under the created Append node + pdxlnAppend->AddChild(dxlnode); + + // cleanup + part_colrefs->Release(); + } + + CDistributionSpec *pds = pexprDTS->GetDrvdPropPlan()->Pds(); + pds->AddRef(); + pdrgpdsBaseTables->Append(pds); + + GPOS_ASSERT(pdxlnAppend); + return pdxlnAppend; +} + //--------------------------------------------------------------------------- // @function: // CTranslatorExprToDXL::PdxlnDynamicScanToAppend @@ -2556,6 +2938,7 @@ CTranslatorExprToDXL::PdxlnFromFilter(CExpression *pexprFilter, switch (eopidRelational) { case COperator::EopPhysicalTableScan: + case COperator::EopPhysicalParallelTableScan: case COperator::EopPhysicalForeignScan: { // if there is a structure of the form @@ -2613,6 +2996,20 @@ CTranslatorExprToDXL::PdxlnFromFilter(CExpression *pexprFilter, pdrgpdsBaseTables, pexprScalar, dxl_properties); } + case COperator::EopPhysicalAppendTableScan: + { + dxl_properties->AddRef(); + + return PdxlnAppendTableScan(pexprRelational, colref_array, + pdrgpdsBaseTables, pexprScalar, dxl_properties); + } + case COperator::EopPhysicalParallelAppendTableScan: + { + dxl_properties->AddRef(); + + return PdxlnParallelAppendTableScan(pexprRelational, colref_array, + pdrgpdsBaseTables, pexprScalar, dxl_properties); + } default: { return PdxlnResultFromFilter(pexprFilter, colref_array, @@ -2847,6 +3244,66 @@ CTranslatorExprToDXL::PdxlnCTEProducer( return pdxlnCTEProducer; } +//--------------------------------------------------------------------------- +// @function: +// CTranslatorExprToDXL::PdxlnParallelCTEProducer +// +// @doc: +// Translate a physical parallel cte producer expression +// +//--------------------------------------------------------------------------- +CDXLNode * +CTranslatorExprToDXL::PdxlnParallelCTEProducer( + CExpression *pexprCTEProducer, + CColRefArray *, //colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, ULONG *pulNonGatherMotions, + BOOL *pfDML) +{ + GPOS_ASSERT(nullptr != pexprCTEProducer); + + ULONG ulParallelWorkers = 0; + + // extract components + CExpression *pexprRelational = (*pexprCTEProducer)[0]; + CPhysicalParallelCTEProducer *popCTEProducer = + CPhysicalParallelCTEProducer::PopConvert(pexprCTEProducer->Pop()); + + ulParallelWorkers = popCTEProducer->UlParallelWorkers(); + + // extract physical properties from cte producer + CDXLPhysicalProperties *dxl_properties = GetProperties(pexprCTEProducer); + + // extract the CTE id and the array of colids + const ULONG ulCTEId = popCTEProducer->UlCTEId(); + ULongPtrArray *colids = CUtils::Pdrgpul(m_mp, popCTEProducer->Pdrgpcr()); + + GPOS_ASSERT(nullptr != pexprCTEProducer->Prpp()); + CColRefArray *pdrgpcrRequired = popCTEProducer->Pdrgpcr(); + CColRefSet *pcrsOutput = GPOS_NEW(m_mp) CColRefSet(m_mp); + pcrsOutput->Include(pdrgpcrRequired); + + // translate relational child expression + CDXLNode *child_dxlnode = CreateDXLNode( + pexprRelational, pdrgpcrRequired, pdrgpdsBaseTables, + pulNonGatherMotions, pfDML, true /*fRemap*/, false /*fRoot */); + + CDXLNode *pdxlnPrL = PdxlnProjList(pcrsOutput, pdrgpcrRequired); + pcrsOutput->Release(); + + ULongPtrArray *producer_idx_map = popCTEProducer->PCTEIdxMap(); + if (producer_idx_map) { + producer_idx_map->AddRef(); + } + + CDXLNode *pdxlnCTEProducer = GPOS_NEW(m_mp) CDXLNode( + m_mp, GPOS_NEW(m_mp) CDXLPhysicalParallelCTEProducer(m_mp, ulCTEId, colids, producer_idx_map, ulParallelWorkers), + pdxlnPrL, child_dxlnode); + + pdxlnCTEProducer->SetProperties(dxl_properties); + + return pdxlnCTEProducer; +} + //--------------------------------------------------------------------------- // @function: // CTranslatorExprToDXL::PdxlnCTEConsumer @@ -2899,6 +3356,62 @@ CTranslatorExprToDXL::PdxlnCTEConsumer( return pdxlnCTEConsumer; } +//--------------------------------------------------------------------------- +// @function: +// CTranslatorExprToDXL::PdxlnParallelCTEConsumer +// +// @doc: +// Translate a physical parallel cte consumer expression +// +//--------------------------------------------------------------------------- +CDXLNode * +CTranslatorExprToDXL::PdxlnParallelCTEConsumer( + CExpression *pexprCTEConsumer, + CColRefArray *, //colref_array, + CDistributionSpecArray *, // pdrgpdsBaseTables, + ULONG *, // pulNonGatherMotions, + BOOL * // pfDML +) +{ + GPOS_ASSERT(nullptr != pexprCTEConsumer); + + ULONG ulParallelWorkers = 0; + + // extract components + CPhysicalParallelCTEConsumer *popCTEConsumer = + CPhysicalParallelCTEConsumer::PopConvert(pexprCTEConsumer->Pop()); + + ulParallelWorkers = popCTEConsumer->UlParallelWorkers(); + + // extract physical properties from cte consumer + CDXLPhysicalProperties *dxl_properties = GetProperties(pexprCTEConsumer); + + // extract the CTE id and the array of colids + const ULONG ulCTEId = popCTEConsumer->UlCTEId(); + CColRefArray *colref_array = popCTEConsumer->Pdrgpcr(); + ULongPtrArray *colids = CUtils::Pdrgpul(m_mp, colref_array); + + CColRefSet *pcrsOutput = GPOS_NEW(m_mp) CColRefSet(m_mp); + pcrsOutput->Include(colref_array); + + // translate relational child expression + CDXLNode *pdxlnPrL = PdxlnProjList(pcrsOutput, colref_array); + ULongPtrArray *pidxmap = popCTEConsumer->PCTEIdxMap(); + if (pidxmap) { + pidxmap->AddRef(); + } + + CDXLNode *pdxlnCTEConsumer = GPOS_NEW(m_mp) CDXLNode( + m_mp, GPOS_NEW(m_mp) CDXLPhysicalParallelCTEConsumer(m_mp, ulCTEId, colids, pidxmap, ulParallelWorkers), + pdxlnPrL); + + pcrsOutput->Release(); + + pdxlnCTEConsumer->SetProperties(dxl_properties); + + return pdxlnCTEConsumer; +} + //--------------------------------------------------------------------------- // @function: // CTranslatorExprToDXL::PdxlnAppend @@ -2919,6 +3432,7 @@ CTranslatorExprToDXL::PdxlnAppend(CExpression *pexprUnionAll, CPhysicalUnionAll::PopConvert(pexprUnionAll->Pop()); CColRefArray *pdrgpcrOutputAll = popUnionAll->PdrgpcrOutput(); CColRefSet *reqdCols = pexprUnionAll->Prpp()->PcrsRequired(); + m_isSet = true; CDXLPhysicalAppend *dxl_op = GPOS_NEW(m_mp) CDXLPhysicalAppend(m_mp, false, false); @@ -2995,6 +3509,7 @@ CTranslatorExprToDXL::PdxlnAppend(CExpression *pexprUnionAll, requiredInput->Release(); } reqd_col_positions->Release(); + m_isSet = false; return pdxlnAppend; } @@ -4383,6 +4898,7 @@ CTranslatorExprToDXL::PdxlnCorrelatedNLJoin( switch (pexprOuterChild->Pop()->Eopid()) { case COperator::EopPhysicalTableScan: + case COperator::EopPhysicalParallelTableScan: { dxl_properties->AddRef(); // create and return a table scan node @@ -4571,6 +5087,7 @@ UlIndexFilter(Edxlopid edxlopid) { case EdxlopPhysicalTableScan: case EdxlopPhysicalForeignScan: + case EdxlopPhysicalParallelTableScan: return EdxltsIndexFilter; case EdxlopPhysicalDynamicForeignScan: return EdxldfsIndexFilter; @@ -4626,6 +5143,7 @@ CTranslatorExprToDXL::PdxlnResultFromNLJoinOuter( case EdxlopPhysicalDynamicIndexScan: case EdxlopPhysicalDynamicBitmapTableScan: case EdxlopPhysicalResult: + case EdxlopPhysicalParallelTableScan: { // if the scalar join condition is a constant TRUE, just translate the child, no need to create an AND expression if (CTranslatorExprToDXLUtils::FScalarConstTrue(m_pmda, @@ -5454,6 +5972,85 @@ CTranslatorExprToDXL::PdxlnSequence(CExpression *pexprSequence, return pdxlnSequence; } +//--------------------------------------------------------------------------- +// @function: +// CTranslatorExprToDXL::PdxlnParallelSequence +// +// @doc: +// Create a DXL parallel sequence node from an optimizer parallel sequence expression +// +//--------------------------------------------------------------------------- +CDXLNode * +CTranslatorExprToDXL::PdxlnParallelSequence(CExpression *pexprSequence, + CColRefArray *colref_array, + CDistributionSpecArray *pdrgpdsBaseTables, + ULONG *pulNonGatherMotions, BOOL *pfDML) +{ + GPOS_ASSERT(nullptr != pexprSequence); + + const ULONG arity = pexprSequence->Arity(); + GPOS_ASSERT(0 < arity); + + ULONG ulParallelWorkers = 0; + + CPhysicalParallelSequence *popDTS = + CPhysicalParallelSequence::PopConvert(pexprSequence->Pop()); + + ulParallelWorkers = popDTS->UlParallelWorkers(); + + // construct parallel sequence node + CDXLPhysicalParallelSequence *pdxlopSequence = + GPOS_NEW(m_mp) CDXLPhysicalParallelSequence(m_mp, ulParallelWorkers); + CDXLNode *pdxlnSequence = GPOS_NEW(m_mp) CDXLNode(m_mp, pdxlopSequence); + CDXLPhysicalProperties *dxl_properties = GetProperties(pexprSequence); + pdxlnSequence->SetProperties(dxl_properties); + + // translate children + CDXLNodeArray *pdrgpdxlnChildren = GPOS_NEW(m_mp) CDXLNodeArray(m_mp); + + for (ULONG ul = 0; ul < arity; ul++) + { + CExpression *pexprChild = (*pexprSequence)[ul]; + + CColRefArray *pdrgpcrChildOutput = nullptr; + if (ul == arity - 1) + { + // impose output columns on last child + pdrgpcrChildOutput = colref_array; + } + + CDXLNode *child_dxlnode = CreateDXLNode( + pexprChild, pdrgpcrChildOutput, pdrgpdsBaseTables, + pulNonGatherMotions, pfDML, false /*fRemap*/, false /*fRoot*/); + pdrgpdxlnChildren->Append(child_dxlnode); + } + + // construct project list from the project list of the last child + CDXLNode *pdxlnLastChild = (*pdrgpdxlnChildren)[arity - 1]; + CDXLNode *pdxlnProjListChild = (*pdxlnLastChild)[0]; + + CDXLNode *proj_list_dxlnode = + CTranslatorExprToDXLUtils::PdxlnProjListFromChildProjList( + m_mp, m_pcf, m_phmcrdxln, pdxlnProjListChild); + pdxlnSequence->AddChild(proj_list_dxlnode); + + // add children + for (ULONG ul = 0; ul < arity; ul++) + { + CDXLNode *pdxlnChid = (*pdrgpdxlnChildren)[ul]; + pdxlnChid->AddRef(); + pdxlnSequence->AddChild(pdxlnChid); + } + + pdrgpdxlnChildren->Release(); + +#ifdef GPOS_DEBUG + pdxlopSequence->AssertValid(pdxlnSequence, false /* validate_children */); +#endif + + return pdxlnSequence; +} + //--------------------------------------------------------------------------- // @function: // CTranslatorExprToDXL::PdxlnPartitionSelector diff --git a/src/backend/gporca/libgpopt/src/translate/CTranslatorExprToDXLUtils.cpp b/src/backend/gporca/libgpopt/src/translate/CTranslatorExprToDXLUtils.cpp index 27f5cb688fe..8d9bc83351b 100644 --- a/src/backend/gporca/libgpopt/src/translate/CTranslatorExprToDXLUtils.cpp +++ b/src/backend/gporca/libgpopt/src/translate/CTranslatorExprToDXLUtils.cpp @@ -17,6 +17,7 @@ #include "gpopt/base/CConstraintDisjunction.h" #include "gpopt/base/CConstraintInterval.h" #include "gpopt/base/CConstraintNegation.h" +#include "gpopt/base/CDistributionSpecWorkerRandom.h" #include "gpopt/base/COptCtxt.h" #include "gpopt/exception.h" #include "gpopt/mdcache/CMDAccessorUtils.h" @@ -759,10 +760,11 @@ CTranslatorExprToDXLUtils::SetDirectDispatchInfo( // +--CScalarConst (5) if (CDistributionSpec::EdtHashed == pds->Edt() || - CDistributionSpec::EdtRandom == pds->Edt()) + CDistributionSpec::EdtRandom == pds->Edt() || + CDistributionSpec::EdtWorkerRandom == pds->Edt()) { // direct dispatch supported for scans over - // hash & random distributed tables + // hash, random & worker-random distributed tables for (ULONG i = 0; i < size; i++) { CExpression *pexprFilter = (*pexprFilterArray)[i]; @@ -816,6 +818,62 @@ CTranslatorExprToDXLUtils::SetDirectDispatchInfo( dxl_direct_dispatch_info = GetDXLDirectDispatchInfoRandDist( mp, md_accessor, pcrDistrCol, pcnstrDistrCol); } + else if (CDistributionSpec::EdtWorkerRandom == pds->Edt()) + { + CConstraint *pcnstr = ppc->Pcnstr(); + + CDistributionSpecWorkerRandom *pdsWorkerRandom = + CDistributionSpecWorkerRandom::PdsConvert(pds); + + // Get the base segment distribution for worker-random distribution + CDistributionSpec *pdsSegmentBase = pdsWorkerRandom->PdsSegmentBase(); + + if (nullptr == pdsSegmentBase) + { + // No base segment distribution available, cannot proceed with direct dispatch + continue; + } + + // Handle direct dispatch based on the base segment distribution type + if (CDistributionSpec::EdtHashed == pdsSegmentBase->Edt()) + { + // Base distribution is hashed - use hash distribution keys + CDistributionSpecHashed *pdsHashed = + CDistributionSpecHashed::PdsConvert(pdsSegmentBase); + CExpressionArray *pdrgpexprHashed = pdsHashed->Pdrgpexpr(); + + dxl_direct_dispatch_info = GetDXLDirectDispatchInfo( + mp, md_accessor, pdrgpexprHashed, pcnstr); + } + else if (CDistributionSpec::EdtRandom == pdsSegmentBase->Edt()) + { + // Base distribution is random - use gp_segment_id + CDistributionSpecRandom *pdsRandom = + CDistributionSpecRandom::PdsConvert(pdsSegmentBase); + + // Extracting GpSegmentID for base random distribution + const CColRef *pcrDistrCol = pdsRandom->GetGpSegmentId(); + + if (pcrDistrCol == nullptr) + { + // Direct Dispatch not feasible - no gp_segment_id available + continue; + } + + CConstraint *pcnstrDistrCol = pcnstr->Pcnstr(mp, pcrDistrCol); + + if (pcnstrDistrCol == nullptr) + { + // Direct Dispatch not feasible - no constraint on gp_segment_id + continue; + } + + dxl_direct_dispatch_info = GetDXLDirectDispatchInfoRandDist( + mp, md_accessor, pcrDistrCol, pcnstrDistrCol); + } + // Note: Other base distribution types (Singleton, Replicated, etc.) + // are not supported for direct dispatch in worker-random context + } if (nullptr != dxl_direct_dispatch_info) { diff --git a/src/backend/gporca/libgpopt/src/xforms/CXformDynamicGet2AppendTableScan.cpp b/src/backend/gporca/libgpopt/src/xforms/CXformDynamicGet2AppendTableScan.cpp new file mode 100644 index 00000000000..8fc244e76ce --- /dev/null +++ b/src/backend/gporca/libgpopt/src/xforms/CXformDynamicGet2AppendTableScan.cpp @@ -0,0 +1,130 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformDynamicGet2AppendTableScan.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/xforms/CXformDynamicGet2AppendTableScan.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/xforms/CXformDynamicGet2AppendTableScan.h" + +#include "gpos/base.h" + +#include "gpopt/hints/CHintUtils.h" +#include "gpopt/metadata/CTableDescriptor.h" +#include "gpopt/operators/CLogicalDynamicGet.h" +#include "gpopt/operators/CPhysicalAppendTableScan.h" +#include "gpopt/optimizer/COptimizerConfig.h" + +using namespace gpopt; + + +//--------------------------------------------------------------------------- +// @function: +// CXformDynamicGet2AppendTableScan::CXformDynamicGet2AppendTableScan +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CXformDynamicGet2AppendTableScan::CXformDynamicGet2AppendTableScan( + CMemoryPool *mp) + : CXformImplementation( + // pattern + GPOS_NEW(mp) CExpression(mp, GPOS_NEW(mp) CLogicalDynamicGet(mp))) +{ +} + +// compute xform promise for a given expression handle +CXform::EXformPromise +CXformDynamicGet2AppendTableScan::Exfp(CExpressionHandle &exprhdl) const +{ + CLogicalDynamicGet *popGet = CLogicalDynamicGet::PopConvert(exprhdl.Pop()); + // Do not run if contains foreign partitions, instead run CXformExpandDynamicGetWithForeignPartitions + if (popGet->ContainsForeignParts()) + { + return CXform::ExfpNone; + } + + if (!GPOS_FTRACE(EopttraceEnableParallelAppendScan)) + return CXform::ExfpNone; + + return CXform::ExfpHigh; +} + +//--------------------------------------------------------------------------- +// @function: +// CXformDynamicGet2AppendTableScan::Transform +// +// @doc: +// Actual transformation +// +//--------------------------------------------------------------------------- +void +CXformDynamicGet2AppendTableScan::Transform(CXformContext *pxfctxt, + CXformResult *pxfres, + CExpression *pexpr) const +{ + GPOS_ASSERT(nullptr != pxfctxt); + GPOS_ASSERT(FPromising(pxfctxt->Pmp(), this, pexpr)); + GPOS_ASSERT(FCheckPattern(pexpr)); + + CLogicalDynamicGet *popGet = CLogicalDynamicGet::PopConvert(pexpr->Pop()); + + if (!CHintUtils::SatisfiesPlanHints( + popGet, + COptCtxt::PoctxtFromTLS()->GetOptimizerConfig()->GetPlanHint())) + { + return; + } + + CMemoryPool *mp = pxfctxt->Pmp(); + + // create/extract components for alternative + CName *pname = GPOS_NEW(mp) CName(mp, popGet->Name()); + + CTableDescriptor *ptabdesc = popGet->Ptabdesc(); + ptabdesc->AddRef(); + + CColRefArray *pdrgpcrOutput = popGet->PdrgpcrOutput(); + GPOS_ASSERT(nullptr != pdrgpcrOutput); + + pdrgpcrOutput->AddRef(); + + CColRef2dArray *pdrgpdrgpcrPart = popGet->PdrgpdrgpcrPart(); + pdrgpdrgpcrPart->AddRef(); + + popGet->GetPartitionMdids()->AddRef(); + popGet->GetRootColMappingPerPart()->AddRef(); + + // create alternative expression + CExpression *pexprAlt = GPOS_NEW(mp) CExpression( + mp, GPOS_NEW(mp) CPhysicalAppendTableScan( + mp, pname, ptabdesc, popGet->UlOpId(), popGet->ScanId(), + pdrgpcrOutput, pdrgpdrgpcrPart, popGet->GetPartitionMdids(), + popGet->GetRootColMappingPerPart())); + // add alternative to transformation result + pxfres->Add(pexprAlt); +} + + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/src/xforms/CXformDynamicGet2ParallelAppendTableScan.cpp b/src/backend/gporca/libgpopt/src/xforms/CXformDynamicGet2ParallelAppendTableScan.cpp new file mode 100644 index 00000000000..ba50e7cd955 --- /dev/null +++ b/src/backend/gporca/libgpopt/src/xforms/CXformDynamicGet2ParallelAppendTableScan.cpp @@ -0,0 +1,195 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformDynamicGet2ParallelAppendTableScan.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/xforms/CXformDynamicGet2ParallelAppendTableScan.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/xforms/CXformDynamicGet2ParallelAppendTableScan.h" + +#include "gpos/base.h" + +#include "gpopt/hints/CHintUtils.h" +#include "gpopt/metadata/CTableDescriptor.h" +#include "gpopt/operators/CLogicalDynamicGet.h" +#include "gpopt/operators/CPhysicalParallelAppendTableScan.h" +#include "gpopt/optimizer/COptimizerConfig.h" + +// Forward declarations for gpdbwrappers functions +namespace gpdb { +bool IsParallelModeOK(void); +} + +using namespace gpopt; + +// Use gpdbwrappers for parallel checks +extern int max_parallel_workers_per_gather; + +//--------------------------------------------------------------------------- +// @function: +// CXformDynamicGet2ParallelAppendTableScan::CXformDynamicGet2ParallelAppendTableScan +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CXformDynamicGet2ParallelAppendTableScan::CXformDynamicGet2ParallelAppendTableScan( + CMemoryPool *mp) + : CXformImplementation( + // pattern + GPOS_NEW(mp) CExpression(mp, GPOS_NEW(mp) CLogicalDynamicGet(mp))) +{ +} + +// compute xform promise for a given expression handle +CXform::EXformPromise +CXformDynamicGet2ParallelAppendTableScan::Exfp(CExpressionHandle &exprhdl) const +{ + // Check if parallel plans are enabled in context and parallel processing in safe + if (!gpdb::IsParallelModeOK()) + { + return CXform::ExfpNone; + } + + if (!GPOS_FTRACE(EopttraceEnableParallelAppendScan)) + return CXform::ExfpNone; + + CLogicalDynamicGet *popGet = CLogicalDynamicGet::PopConvert(exprhdl.Pop()); + CTableDescriptor *ptabdesc = popGet->Ptabdesc(); + + // Don't use parallel append for replicated tables + if (ptabdesc->GetRelDistribution() == IMDRelation::EreldistrReplicated || + ptabdesc->GetRelDistribution() == IMDRelation::EreldistrMasterOnly || + COptCtxt::PoctxtFromTLS()->HasReplicatedTables()) + { + return CXform::ExfpNone; + } + + // For AO/AOCO tables, check segfilecount early to avoid useless transformation + CMDAccessor *md_accessor = COptCtxt::PoctxtFromTLS()->Pmda(); + const IMDRelation *pmdrel = md_accessor->RetrieveRel(ptabdesc->MDId()); + IMDRelation::Erelstoragetype storage_type = pmdrel->RetrieveRelStorageType(); + + // Check if this is an AO/AOCO table + if (storage_type == IMDRelation::ErelstorageAppendOnlyRows || + storage_type == IMDRelation::ErelstorageAppendOnlyCols) + { + INT seg_file_count = pmdrel->SegFileCount(); + // Only reject if segfilecount is explicitly known to be 0 or 1 + // -1 means unknown (e.g., from DXL deserialization), so allow parallel in that case + if (seg_file_count >= 0 && seg_file_count <= 1) + { + // If segfilecount is 0 or 1, parallel execution is pointless + // Reject parallel scan early in promise phase + GPOS_TRACE_FORMAT("CXformGet2ParallelAppendTableScan rejected for table %ls: AO/AOCO table has segfilecount=%d (needs >1 for parallel scan)", + ptabdesc->Name().Pstr()->GetBuffer(), seg_file_count); + return CXform::ExfpNone; + } + } + + // Do not run if contains foreign partitions, instead run CXformExpandDynamicGetWithForeignPartitions + if (popGet->ContainsForeignParts()) + { + return CXform::ExfpNone; + } + + return CXform::ExfpHigh; +} + +//--------------------------------------------------------------------------- +// @function: +// CXformDynamicGet2ParallelAppendTableScan::Transform +// +// @doc: +// Actual transformation +// +//--------------------------------------------------------------------------- +void +CXformDynamicGet2ParallelAppendTableScan::Transform(CXformContext *pxfctxt, + CXformResult *pxfres, + CExpression *pexpr) const +{ + GPOS_ASSERT(nullptr != pxfctxt); + GPOS_ASSERT(FPromising(pxfctxt->Pmp(), this, pexpr)); + GPOS_ASSERT(FCheckPattern(pexpr)); + + CLogicalDynamicGet *popGet = CLogicalDynamicGet::PopConvert(pexpr->Pop()); + + if (!CHintUtils::SatisfiesPlanHints( + popGet, + COptCtxt::PoctxtFromTLS()->GetOptimizerConfig()->GetPlanHint())) + { + return; + } + + CMemoryPool *mp = pxfctxt->Pmp(); + + // create/extract components for alternative + CName *pname = GPOS_NEW(mp) CName(mp, popGet->Name()); + + CTableDescriptor *ptabdesc = popGet->Ptabdesc(); + ptabdesc->AddRef(); + + CColRefArray *pdrgpcrOutput = popGet->PdrgpcrOutput(); + GPOS_ASSERT(nullptr != pdrgpcrOutput); + + pdrgpcrOutput->AddRef(); + + CColRef2dArray *pdrgpdrgpcrPart = popGet->PdrgpdrgpcrPart(); + pdrgpdrgpcrPart->AddRef(); + + popGet->GetPartitionMdids()->AddRef(); + popGet->GetRootColMappingPerPart()->AddRef(); + + // Determine parallel workers degree + // Priority: table-level parallel_workers setting > GUC max_parallel_workers_per_gather > default + ULONG ulParallelWorkers = 2; // default + + // Check if table has a specific parallel_workers setting + CMDAccessor *md_accessor = COptCtxt::PoctxtFromTLS()->Pmda(); + const IMDRelation *pmdrel = md_accessor->RetrieveRel(ptabdesc->MDId()); + INT table_parallel_workers = pmdrel->ParallelWorkers(); + + if (table_parallel_workers > 0) + { + // Use table-level setting if explicitly configured + ulParallelWorkers = (ULONG)table_parallel_workers; + } + else if (max_parallel_workers_per_gather > 0) + { + // Fall back to GUC setting + ulParallelWorkers = (ULONG)max_parallel_workers_per_gather; + } + + // create alternative expression + CExpression *pexprAlt = GPOS_NEW(mp) CExpression( + mp, GPOS_NEW(mp) CPhysicalParallelAppendTableScan( + mp, pname, ptabdesc, popGet->UlOpId(), popGet->ScanId(), + pdrgpcrOutput, pdrgpdrgpcrPart, popGet->GetPartitionMdids(), + popGet->GetRootColMappingPerPart(), ulParallelWorkers)); + // add alternative to transformation result + pxfres->Add(pexprAlt); +} + + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/src/xforms/CXformFactory.cpp b/src/backend/gporca/libgpopt/src/xforms/CXformFactory.cpp index e21c24511dd..0fa765c43b8 100644 --- a/src/backend/gporca/libgpopt/src/xforms/CXformFactory.cpp +++ b/src/backend/gporca/libgpopt/src/xforms/CXformFactory.cpp @@ -144,10 +144,14 @@ CXformFactory::Instantiate() Add(GPOS_NEW(m_mp) CXformExpandNAryJoinMinCard(m_mp)); Add(GPOS_NEW(m_mp) CXformExpandNAryJoinDP(m_mp)); Add(GPOS_NEW(m_mp) CXformGet2TableScan(m_mp)); + Add(GPOS_NEW(m_mp) CXformGet2ParallelTableScan(m_mp)); Add(GPOS_NEW(m_mp) CXformIndexGet2IndexScan(m_mp)); Add(GPOS_NEW(m_mp) CXformDynamicGet2DynamicTableScan(m_mp)); + Add(GPOS_NEW(m_mp) CXformDynamicGet2AppendTableScan(m_mp)); + Add(GPOS_NEW(m_mp) CXformDynamicGet2ParallelAppendTableScan(m_mp)); Add(GPOS_NEW(m_mp) CXformDynamicIndexGet2DynamicIndexScan(m_mp)); Add(GPOS_NEW(m_mp) CXformImplementSequence(m_mp)); + Add(GPOS_NEW(m_mp) CXformImplementParallelSequence(m_mp)); Add(GPOS_NEW(m_mp) CXformImplementConstTableGet(m_mp)); Add(GPOS_NEW(m_mp) CXformUnnestTVF(m_mp)); Add(GPOS_NEW(m_mp) CXformImplementTVF(m_mp)); @@ -256,7 +260,9 @@ CXformFactory::Instantiate() Add(GPOS_NEW(m_mp) CXformInlineCTEConsumer(m_mp)); Add(GPOS_NEW(m_mp) CXformInlineCTEConsumerUnderSelect(m_mp)); Add(GPOS_NEW(m_mp) CXformImplementCTEProducer(m_mp)); + Add(GPOS_NEW(m_mp) CXformImplementParallelCTEProducer(m_mp)); Add(GPOS_NEW(m_mp) CXformImplementCTEConsumer(m_mp)); + Add(GPOS_NEW(m_mp) CXformImplementParallelCTEConsumer(m_mp)); Add(GPOS_NEW(m_mp) CXformExpandFullOuterJoin(m_mp)); Add(GPOS_NEW(m_mp) CXformForeignGet2ForeignScan(m_mp)); Add(GPOS_NEW(m_mp) CXformSelect2BitmapBoolOp(m_mp)); diff --git a/src/backend/gporca/libgpopt/src/xforms/CXformGet2ParallelTableScan.cpp b/src/backend/gporca/libgpopt/src/xforms/CXformGet2ParallelTableScan.cpp new file mode 100644 index 00000000000..7e023ae6f3d --- /dev/null +++ b/src/backend/gporca/libgpopt/src/xforms/CXformGet2ParallelTableScan.cpp @@ -0,0 +1,265 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformGet2ParallelTableScan.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/xforms/CXformGet2ParallelTableScan.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/xforms/CXformGet2ParallelTableScan.h" + +#include "gpos/base.h" + +#include "gpopt/base/COptCtxt.h" +#include "gpopt/hints/CHintUtils.h" +#include "gpopt/metadata/CTableDescriptor.h" +#include "gpopt/operators/CExpressionHandle.h" +#include "gpopt/operators/CLogicalGet.h" +#include "gpopt/operators/CPhysicalParallelTableScan.h" +#include "gpopt/optimizer/COptimizerConfig.h" +#include "naucrates/md/IMDRelation.h" +#include "gpopt/search/CGroupProxy.h" +#include "gpopt/search/CMemo.h" + + +// Use gpdbwrappers for parallel checks +extern int max_parallel_workers_per_gather; + +// Forward declarations for gpdbwrappers functions +namespace gpdb { + bool IsParallelModeOK(void); +} + +using namespace gpopt; + +//--------------------------------------------------------------------------- +// @function: +// CXformGet2ParallelTableScan::FHasParallelIncompatibleOps +// +// @doc: +// Check if memo contains logical operators that are incompatible +// with parallel execution (CTE, Dynamic scans, Foreign scans, etc.) +// +//--------------------------------------------------------------------------- +BOOL +CXformGet2ParallelTableScan::FHasParallelIncompatibleOps(CExpressionHandle &exprhdl) +{ + CGroupExpression *pgexprHandle = exprhdl.Pgexpr(); + if (nullptr == pgexprHandle) + { + return false; + } + + CGroup *pgroup = pgexprHandle->Pgroup(); + if (nullptr == pgroup) + { + return false; + } + + CMemo *pmemo = pgroup->Pmemo(); + if (nullptr == pmemo) + { + return false; + } + + // Iterate through all groups in memo to check for parallel-incompatible operations + const ULONG_PTR ulGroups = pmemo->UlpGroups(); + for (ULONG_PTR ul = 0; ul < ulGroups; ul++) + { + CGroup *pgroupCurrent = pmemo->Pgroup(ul); + if (nullptr == pgroupCurrent) + { + continue; + } + + // Check all group expressions in this group using CGroupProxy + CGroupProxy gp(pgroupCurrent); + CGroupExpression *pgexpr = gp.PgexprFirst(); + while (nullptr != pgexpr) + { + COperator::EOperatorId eopid = pgexpr->Pop()->Eopid(); + +// // Check for CTE-related operators (incompatible with parallel execution) +// if (COperator::EopLogicalCTEProducer == eopid || +// COperator::EopLogicalCTEConsumer == eopid || +// COperator::EopLogicalSequence == eopid || +// COperator::EopLogicalSequenceProject == eopid) +// { +// return true; +// } + + if (COperator::EopLogicalUnion == eopid || + COperator::EopLogicalUnionAll == eopid || + COperator::EopLogicalIntersect == eopid || + COperator::EopLogicalIntersectAll == eopid || + COperator::EopLogicalDifference == eopid || + COperator::EopLogicalDifferenceAll == eopid) + { + // Set operations are not supported in parallel plans + return true; + } + + pgexpr = gp.PgexprNext(pgexpr); + } + } + + return false; +} + +//--------------------------------------------------------------------------- +// @function: +// CXformGet2ParallelTableScan::CXformGet2ParallelTableScan +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CXformGet2ParallelTableScan::CXformGet2ParallelTableScan(CMemoryPool *mp) + : CXformImplementation( + // pattern + GPOS_NEW(mp) CExpression(mp, GPOS_NEW(mp) CLogicalGet(mp))) +{ +} + +//--------------------------------------------------------------------------- +// @function: +// CXformGet2ParallelTableScan::Exfp +// +// @doc: +// Compute promise of xform based on GUC enable_parallel +// Uses unified parallel degree from max_parallel_workers_per_gather +// +//--------------------------------------------------------------------------- +CXform::EXformPromise +CXformGet2ParallelTableScan::Exfp(CExpressionHandle &exprhdl) const +{ + // Check if parallel plans are enabled in context and parallel processing is safe + if (!gpdb::IsParallelModeOK()) + { + return CXform::ExfpNone; + } + + // Check for parallel-incompatible operations that would conflict with parallel scans + if (FHasParallelIncompatibleOps(exprhdl)) + { + return CXform::ExfpNone; + } + + CLogicalGet *popGet = CLogicalGet::PopConvert(exprhdl.Pop()); + CTableDescriptor *ptabdesc = popGet->Ptabdesc(); + + // Don't use parallel scan for replicated tables + if (ptabdesc->GetRelDistribution() == IMDRelation::EreldistrReplicated || + ptabdesc->GetRelDistribution() == IMDRelation::EreldistrMasterOnly || + COptCtxt::PoctxtFromTLS()->HasReplicatedTables()) + { + //FIXME: Should we consider replicated tables. + return CXform::ExfpNone; + } + + // For AO/AOCO tables, check segfilecount early to avoid useless transformation + CMDAccessor *md_accessor = COptCtxt::PoctxtFromTLS()->Pmda(); + const IMDRelation *pmdrel = md_accessor->RetrieveRel(ptabdesc->MDId()); + IMDRelation::Erelstoragetype storage_type = pmdrel->RetrieveRelStorageType(); + + // Check if this is an AO/AOCO table + if (storage_type == IMDRelation::ErelstorageAppendOnlyRows || + storage_type == IMDRelation::ErelstorageAppendOnlyCols) + { + INT seg_file_count = pmdrel->SegFileCount(); + // Only reject if segfilecount is explicitly known to be 0 or 1 + // -1 means unknown (e.g., from DXL deserialization), so allow parallel in that case + if (seg_file_count >= 0 && seg_file_count <= 1) + { + // If segfilecount is 0 or 1, parallel execution is pointless + // Reject parallel scan early in promise phase + GPOS_TRACE_FORMAT("CXformGet2ParallelTableScan rejected for table %ls: AO/AOCO table has segfilecount=%d (needs >1 for parallel scan)", + ptabdesc->Name().Pstr()->GetBuffer(), seg_file_count); + return CXform::ExfpNone; + } + } + + // High promise for parallel scan when enabled + // All tables will use the same parallel degree from max_parallel_workers_per_gather + return CXform::ExfpHigh; +} + +//--------------------------------------------------------------------------- +// @function: +// CXformGet2ParallelTableScan::Transform +// +// @doc: +// Actual transformation +// +//--------------------------------------------------------------------------- +void +CXformGet2ParallelTableScan::Transform(CXformContext *pxfctxt, CXformResult *pxfres, + CExpression *pexpr) const +{ + GPOS_ASSERT(nullptr != pxfctxt); + GPOS_ASSERT(FPromising(pxfctxt->Pmp(), this, pexpr)); + GPOS_ASSERT(FCheckPattern(pexpr)); + + CLogicalGet *popGet = CLogicalGet::PopConvert(pexpr->Pop()); + + CMemoryPool *mp = pxfctxt->Pmp(); + + // create/extract components for alternative + CName *pname = GPOS_NEW(mp) CName(mp, popGet->Name()); + + CTableDescriptor *ptabdesc = popGet->Ptabdesc(); + ptabdesc->AddRef(); + + CColRefArray *pdrgpcrOutput = popGet->PdrgpcrOutput(); + GPOS_ASSERT(nullptr != pdrgpcrOutput); + pdrgpcrOutput->AddRef(); + + // Determine parallel workers degree + // Priority: table-level parallel_workers setting > GUC max_parallel_workers_per_gather > default + ULONG ulParallelWorkers = 2; // default + + // Check if table has a specific parallel_workers setting + CMDAccessor *md_accessor = COptCtxt::PoctxtFromTLS()->Pmda(); + const IMDRelation *pmdrel = md_accessor->RetrieveRel(ptabdesc->MDId()); + INT table_parallel_workers = pmdrel->ParallelWorkers(); + + if (table_parallel_workers > 0) + { + // Use table-level setting if explicitly configured + ulParallelWorkers = (ULONG)table_parallel_workers; + } + else if (max_parallel_workers_per_gather > 0) + { + // Fall back to GUC setting + ulParallelWorkers = (ULONG)max_parallel_workers_per_gather; + } + + // create alternative expression + CExpression *pexprAlt = GPOS_NEW(mp) CExpression( + mp, + GPOS_NEW(mp) CPhysicalParallelTableScan(mp, pname, ptabdesc, pdrgpcrOutput, ulParallelWorkers)); + + // add alternative to transformation result + pxfres->Add(pexprAlt); +} + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/src/xforms/CXformGet2TableScan.cpp b/src/backend/gporca/libgpopt/src/xforms/CXformGet2TableScan.cpp index 05e826aec92..d34068a3dfc 100644 --- a/src/backend/gporca/libgpopt/src/xforms/CXformGet2TableScan.cpp +++ b/src/backend/gporca/libgpopt/src/xforms/CXformGet2TableScan.cpp @@ -20,6 +20,10 @@ #include "gpopt/operators/CPhysicalTableScan.h" #include "gpopt/optimizer/COptimizerConfig.h" +namespace gpdb { + bool IsParallelModeOK(void); +} + using namespace gpopt; @@ -57,6 +61,13 @@ CXformGet2TableScan::Exfp(CExpressionHandle &exprhdl) const return CXform::ExfpNone; } + // If parallel processing is enabled, give lower priority to regular table scan + // to allow parallel table scan to take precedence + if (gpdb::IsParallelModeOK()) + { + return CXform::ExfpLow; + } + return CXform::ExfpHigh; } diff --git a/src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelCTEConsumer.cpp b/src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelCTEConsumer.cpp new file mode 100644 index 00000000000..93454fc91a7 --- /dev/null +++ b/src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelCTEConsumer.cpp @@ -0,0 +1,119 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformImplementParallelCTEConsumer.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelCTEConsumer.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/xforms/CXformImplementParallelCTEConsumer.h" + +#include "gpos/base.h" + +#include "gpopt/operators/CLogicalCTEConsumer.h" +#include "gpopt/operators/CPatternLeaf.h" +#include "gpopt/operators/CPhysicalParallelCTEConsumer.h" + +using namespace gpopt; +namespace gpdb { +bool IsParallelModeOK(void); +} + + +//--------------------------------------------------------------------------- +// @function: +// CXformImplementParallelCTEConsumer::CXformImplementParallelCTEConsumer +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CXformImplementParallelCTEConsumer::CXformImplementParallelCTEConsumer(CMemoryPool *mp) + : CXformImplementation( + // pattern + GPOS_NEW(mp) CExpression(mp, GPOS_NEW(mp) CLogicalCTEConsumer(mp))) +{ +} + +//--------------------------------------------------------------------------- +// @function: +// CXformImplementParallelCTEConsumer::Exfp +// +// @doc: +// Compute promise of xform +// +//--------------------------------------------------------------------------- +CXform::EXformPromise +CXformImplementParallelCTEConsumer::Exfp(CExpressionHandle & // exprhdl +) const +{ + // Check if parallel plans are enabled in context and parallel processing in safe + if (!gpdb::IsParallelModeOK()) + { + return CXform::ExfpNone; + } + + return CXform::ExfpHigh; +} + + +//--------------------------------------------------------------------------- +// @function: +// CXformImplementParallelCTEConsumer::Transform +// +// @doc: +// Actual transformation +// +//--------------------------------------------------------------------------- +void +CXformImplementParallelCTEConsumer::Transform(CXformContext *pxfctxt, + CXformResult *pxfres, + CExpression *pexpr) const +{ + GPOS_ASSERT(nullptr != pxfctxt); + GPOS_ASSERT(FPromising(pxfctxt->Pmp(), this, pexpr)); + GPOS_ASSERT(FCheckPattern(pexpr)); + + CLogicalCTEConsumer *popCTEConsumer = + CLogicalCTEConsumer::PopConvert(pexpr->Pop()); + CMemoryPool *mp = pxfctxt->Pmp(); + + // extract components for alternative + ULONG id = popCTEConsumer->UlCTEId(); + + CColRefArray *colref_array = popCTEConsumer->Pdrgpcr(); + colref_array->AddRef(); + + UlongToColRefMap *colref_mapping = popCTEConsumer->Phmulcr(); + GPOS_ASSERT(nullptr != colref_mapping); + colref_mapping->AddRef(); + + // create physical CTE Consumer + CExpression *pexprAlt = + GPOS_NEW(mp) CExpression(mp, GPOS_NEW(mp) CPhysicalParallelCTEConsumer( + mp, id, colref_array, colref_mapping, 2)); + + // add alternative to transformation result + pxfres->Add(pexprAlt); +} + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelCTEProducer.cpp b/src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelCTEProducer.cpp new file mode 100644 index 00000000000..21330755749 --- /dev/null +++ b/src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelCTEProducer.cpp @@ -0,0 +1,120 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformImplementParallelCTEProducer.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelCTEProducer.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/xforms/CXformImplementParallelCTEProducer.h" + +#include "gpos/base.h" + +#include "gpopt/operators/CLogicalCTEProducer.h" +#include "gpopt/operators/CPatternLeaf.h" +#include "gpopt/operators/CPhysicalParallelCTEProducer.h" + +using namespace gpopt; +namespace gpdb { +bool IsParallelModeOK(void); +} + +//--------------------------------------------------------------------------- +// @function: +// CXformImplementParallelCTEProducer::CXformImplementParallelCTEProducer +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CXformImplementParallelCTEProducer::CXformImplementParallelCTEProducer(CMemoryPool *mp) + : CXformImplementation( + // pattern + GPOS_NEW(mp) CExpression( + mp, GPOS_NEW(mp) CLogicalCTEProducer(mp), + GPOS_NEW(mp) CExpression(mp, GPOS_NEW(mp) CPatternLeaf(mp)))) +{ +} + +//--------------------------------------------------------------------------- +// @function: +// CXformImplementParallelCTEProducer::Exfp +// +// @doc: +// Compute promise of xform +// +//--------------------------------------------------------------------------- +CXform::EXformPromise +CXformImplementParallelCTEProducer::Exfp(CExpressionHandle & // exprhdl +) const +{ + // Check if parallel plans are enabled in context and parallel processing in safe + if (!gpdb::IsParallelModeOK()) + { + return CXform::ExfpNone; + } + + return CXform::ExfpHigh; +} + + +//--------------------------------------------------------------------------- +// @function: +// CXformImplementParallelCTEProducer::Transform +// +// @doc: +// Actual transformation +// +//--------------------------------------------------------------------------- +void +CXformImplementParallelCTEProducer::Transform(CXformContext *pxfctxt, + CXformResult *pxfres, + CExpression *pexpr) const +{ + GPOS_ASSERT(nullptr != pxfctxt); + GPOS_ASSERT(FPromising(pxfctxt->Pmp(), this, pexpr)); + GPOS_ASSERT(FCheckPattern(pexpr)); + + CLogicalCTEProducer *popCTEProducer = + CLogicalCTEProducer::PopConvert(pexpr->Pop()); + CMemoryPool *mp = pxfctxt->Pmp(); + + // extract components for alternative + ULONG id = popCTEProducer->UlCTEId(); + + CColRefArray *colref_array = popCTEProducer->Pdrgpcr(); + colref_array->AddRef(); + + // child of CTEProducer operator + CExpression *pexprChild = (*pexpr)[0]; + pexprChild->AddRef(); + + // create physical CTE Producer + CExpression *pexprAlt = GPOS_NEW(mp) + CExpression(mp, GPOS_NEW(mp) CPhysicalParallelCTEProducer(mp, id, colref_array, popCTEProducer->UsedMask(), 2), + pexprChild); + + // add alternative to transformation result + pxfres->Add(pexprAlt); +} + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelSequence.cpp b/src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelSequence.cpp new file mode 100644 index 00000000000..62caa68e57c --- /dev/null +++ b/src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelSequence.cpp @@ -0,0 +1,111 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CXformImplementParallelSequence.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/xforms/CXformImplementParallelSequence.cpp + * + *------------------------------------------------------------------------- + */ + +#include "gpopt/xforms/CXformImplementParallelSequence.h" + +#include "gpos/base.h" + +#include "gpopt/metadata/CTableDescriptor.h" +#include "gpopt/operators/CLogicalSequence.h" +#include "gpopt/operators/CPatternMultiLeaf.h" +#include "gpopt/operators/CPhysicalParallelSequence.h" + +using namespace gpopt; +namespace gpdb { +bool IsParallelModeOK(void); +} + + +//--------------------------------------------------------------------------- +// @function: +// CXformImplementParallelSequence::CXformImplementParallelSequence +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CXformImplementParallelSequence::CXformImplementParallelSequence(CMemoryPool *mp) + : CXformImplementation( + // pattern + GPOS_NEW(mp) CExpression( + mp, GPOS_NEW(mp) CLogicalSequence(mp), + GPOS_NEW(mp) CExpression(mp, GPOS_NEW(mp) CPatternMultiLeaf(mp)))) +{ +} + + +//--------------------------------------------------------------------------- +// @function: +// CXformImplementParallelCTEProducer::Exfp +// +// @doc: +// Compute promise of xform +// +//--------------------------------------------------------------------------- +CXform::EXformPromise +CXformImplementParallelSequence::Exfp(CExpressionHandle & // exprhdl +) const +{ + // Check if parallel plans are enabled in context and parallel processing in safe + if (!gpdb::IsParallelModeOK()) + { + return CXform::ExfpNone; + } + + return CXform::ExfpHigh; +} + +//--------------------------------------------------------------------------- +// @function: +// CXformImplementParallelSequence::Transform +// +// @doc: +// Actual transformation +// +//--------------------------------------------------------------------------- +void +CXformImplementParallelSequence::Transform(CXformContext *pxfctxt, CXformResult *pxfres, + CExpression *pexpr) const +{ + GPOS_ASSERT(nullptr != pxfctxt); + GPOS_ASSERT(FPromising(pxfctxt->Pmp(), this, pexpr)); + GPOS_ASSERT(FCheckPattern(pexpr)); + + CMemoryPool *mp = pxfctxt->Pmp(); + + CExpressionArray *pdrgpexpr = pexpr->PdrgPexpr(); + pdrgpexpr->AddRef(); + + // create alternative expression + CExpression *pexprAlt = GPOS_NEW(mp) + CExpression(mp, GPOS_NEW(mp) CPhysicalParallelSequence(mp, 2), pdrgpexpr); + // add alternative to transformation result + pxfres->Add(pexprAlt); +} + + +// EOF diff --git a/src/backend/gporca/libgpopt/src/xforms/Makefile b/src/backend/gporca/libgpopt/src/xforms/Makefile index 03f6293b36d..4b8e0e80922 100644 --- a/src/backend/gporca/libgpopt/src/xforms/Makefile +++ b/src/backend/gporca/libgpopt/src/xforms/Makefile @@ -26,6 +26,8 @@ OBJS = CDecorrelator.o \ CXformDifference2LeftAntiSemiJoin.o \ CXformDifferenceAll2LeftAntiSemiJoin.o \ CXformDynamicGet2DynamicTableScan.o \ + CXformDynamicGet2AppendTableScan.o \ + CXformDynamicGet2ParallelAppendTableScan.o \ CXformDynamicIndexOnlyGet2DynamicIndexOnlyScan.o \ CXformDynamicIndexGet2DynamicIndexScan.o \ CXformEagerAgg.o \ @@ -47,16 +49,20 @@ OBJS = CDecorrelator.o \ CXformGbAggDedup2StreamAggDedup.o \ CXformGbAggWithMDQA2Join.o \ CXformGet2TableScan.o \ + CXformGet2ParallelTableScan.o \ CXformImplementAssert.o \ CXformImplementBitmapTableGet.o \ CXformImplementCTEConsumer.o \ CXformImplementCTEProducer.o \ + CXformImplementParallelCTEConsumer.o \ + CXformImplementParallelCTEProducer.o \ CXformImplementConstTableGet.o \ CXformImplementDML.o \ CXformImplementDynamicBitmapTableGet.o \ CXformImplementFullOuterMergeJoin.o \ CXformImplementLimit.o \ CXformImplementSequence.o \ + CXformImplementParallelSequence.o \ CXformImplementSequenceProject.o \ CXformImplementHashSequenceProject.o \ CXformImplementSplit.o \ diff --git a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLOperator.h b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLOperator.h index 8fb27307c36..d9ded5726e6 100644 --- a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLOperator.h +++ b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLOperator.h @@ -125,6 +125,7 @@ enum Edxlopid EdxlopPhysicalValuesScan, EdxlopPhysicalProjection, EdxlopPhysicalTableScan, + EdxlopPhysicalParallelTableScan, EdxlopPhysicalBitmapTableScan, EdxlopPhysicalDynamicBitmapTableScan, EdxlopPhysicalForeignScan, @@ -144,9 +145,11 @@ enum Edxlopid EdxlopPhysicalAgg, EdxlopPhysicalSort, EdxlopPhysicalAppend, + EdxlopPhysicalParallelAppend, EdxlopPhysicalMaterialize, EdxlopPhysicalDynamicForeignScan, EdxlopPhysicalSequence, + EdxlopPhysicalParallelSequence, EdxlopPhysicalDynamicTableScan, EdxlopPhysicalDynamicIndexScan, EdxlopPhysicalDynamicIndexOnlyScan, @@ -156,6 +159,8 @@ enum Edxlopid EdxlopPhysicalCTEProducer, EdxlopPhysicalCTEConsumer, + EdxlopPhysicalParallelCTEProducer, + EdxlopPhysicalParallelCTEConsumer, EdxlopPhysicalDML, EdxlopPhysicalSplit, diff --git a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLOperatorFactory.h b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLOperatorFactory.h index ed27db129fe..5ffb517fb55 100644 --- a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLOperatorFactory.h +++ b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLOperatorFactory.h @@ -134,6 +134,10 @@ class CDXLOperatorFactory static CDXLPhysical *MakeDXLTblScan(CDXLMemoryManager *dxl_memory_manager, const Attributes &attrs); + // create a parallel table scan operator + static CDXLPhysical *MakeDXLParallelTblScan(CDXLMemoryManager *dxl_memory_manager, + const Attributes &attrs); + // create a result operator static CDXLPhysical *MakeDXLResult(CDXLMemoryManager *dxl_memory_manager); @@ -173,6 +177,10 @@ class CDXLOperatorFactory static CDXLPhysical *MakeDXLAppend(CDXLMemoryManager *dxl_memory_manager, const Attributes &attrs); + // create a parallel append operator + static CDXLPhysical *MakeDXLParallelAppend(CDXLMemoryManager *dxl_memory_manager, + const Attributes &attrs); + // create a limit operator static CDXLPhysical *MakeDXLLimit(CDXLMemoryManager *dxl_memory_manager, const Attributes &attrs); diff --git a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalAppend.h b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalAppend.h index 2b547015447..c53d1466ce2 100644 --- a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalAppend.h +++ b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalAppend.h @@ -64,7 +64,7 @@ class CDXLPhysicalAppend : public CDXLPhysical // ctor/dtor CDXLPhysicalAppend(CMemoryPool *mp, BOOL fIsTarget, BOOL fIsZapped); - // ctor for partitioned table scan + // ctor for paritioned table scan CDXLPhysicalAppend(CMemoryPool *mp, BOOL fIsTarget, BOOL fIsZapped, ULONG scan_id, CDXLTableDescr *dxl_table_desc, ULongPtrArray *selector_ids); diff --git a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelAppend.h b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelAppend.h new file mode 100644 index 00000000000..54bd7c80cc4 --- /dev/null +++ b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelAppend.h @@ -0,0 +1,143 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CDXLPhysicalParallelAppend.h + * + * IDENTIFICATION + * src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelAppend.h + * + *------------------------------------------------------------------------- + */ + +#ifndef GPDXL_CDXLPhysicalParallelAppend_H +#define GPDXL_CDXLPhysicalParallelAppend_H + +#include "gpos/base.h" + +#include "naucrates/dxl/operators/CDXLPhysicalAppend.h" + +namespace gpdxl +{ +using namespace gpos; + +//--------------------------------------------------------------------------- +// @class: +// CDXLPhysicalParallelAppend +// +// @doc: +// Class for representing DXL parallel append operators +// +//--------------------------------------------------------------------------- +class CDXLPhysicalParallelAppend : public CDXLPhysical +{ +private: + // is the append node used in an update/delete statement + BOOL m_used_in_upd_del = false; + + // TODO: - Apr 12, 2011; find a better name (and comments) for this variable + BOOL m_is_zapped = false; + + // scan id from the CPhysicalDynamicTableScan (a.k.a part_index_id) + // when m_scan_id != gpos::ulong_max + ULONG m_scan_id = gpos::ulong_max; + + // table descr of the root partitioned table (when translated from a CPhysicalDynamicTableScan) + CDXLTableDescr *m_dxl_table_descr = nullptr; + + ULongPtrArray *m_selector_ids = nullptr; + + // number of parallel workers + ULONG m_ulParallelWorkers; + +public: + CDXLPhysicalParallelAppend(const CDXLPhysicalParallelAppend &) = delete; + + //ctor + CDXLPhysicalParallelAppend(CMemoryPool *mp, BOOL fIsTarget, BOOL fIsZapped, ULONG ulParallelWorkers); + + // ctor for paritioned table scan + CDXLPhysicalParallelAppend(CMemoryPool *mp, BOOL fIsTarget, BOOL fIsZapped, + ULONG scan_id, CDXLTableDescr *dxl_table_desc, + ULongPtrArray *selector_ids, ULONG ulParallelWorkers); + + // dtor + ~CDXLPhysicalParallelAppend() override; + + // accessors + Edxlopid GetDXLOperator() const override; + const CWStringConst *GetOpNameStr() const override; + + BOOL IsUsedInUpdDel() const; + BOOL IsZapped() const; + + CDXLTableDescr * + GetDXLTableDesc() const + { + return m_dxl_table_descr; + } + + void + SetDXLTableDesc(CDXLTableDescr *dxl_table_desc) + { + m_dxl_table_descr = dxl_table_desc; + } + + ULONG + GetScanId() const + { + return m_scan_id; + } + + const ULongPtrArray * + GetSelectorIds() const + { + return m_selector_ids; + } + + // get number of parallel workers + ULONG UlParallelWorkers() const + { + return m_ulParallelWorkers; + } + + // serialize operator in DXL format + void SerializeToDXL(CXMLSerializer *xml_serializer, + const CDXLNode *dxlnode) const override; + + // conversion function + static CDXLPhysicalParallelAppend * + Cast(CDXLOperator *dxl_op) + { + GPOS_ASSERT(nullptr != dxl_op); + GPOS_ASSERT(EdxlopPhysicalParallelAppend == dxl_op->GetDXLOperator()); + + return dynamic_cast(dxl_op); + } + +#ifdef GPOS_DEBUG + // checks whether the operator has valid structure, i.e. number and + // types of child nodes + void AssertValid(const CDXLNode *, BOOL validate_children) const override; +#endif // GPOS_DEBUG +}; // class CDXLPhysicalParallelAppend + +} // namespace gpdxl +#endif // !GPDXL_CDXLPhysicalParallelAppend_H + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelCTEConsumer.h b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelCTEConsumer.h new file mode 100644 index 00000000000..5ee6d266dad --- /dev/null +++ b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelCTEConsumer.h @@ -0,0 +1,126 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CDXLPhysicalParallelCTEConsumer.h + * + * IDENTIFICATION + * src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelCTEConsumer.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPDXL_CDXLPhysicalParallelCTEConsumer_H +#define GPDXL_CDXLPhysicalParallelCTEConsumer_H + +#include "gpos/base.h" +#include "gpos/common/CDynamicPtrArray.h" + +#include "naucrates/dxl/operators/CDXLPhysicalCTEConsumer.h" + +namespace gpdxl +{ +//--------------------------------------------------------------------------- +// @class: +// CDXLPhysicalParallelCTEConsumer +// +// @doc: +// Class for representing DXL physical parallel CTE Consumers +// +//--------------------------------------------------------------------------- +class CDXLPhysicalParallelCTEConsumer : public CDXLPhysical +{ +private: + // cte id + ULONG m_id; + + // output column ids + ULongPtrArray *m_output_colids_array; + + // output column index mapping + ULongPtrArray *m_output_colidx_map; + + // number of parallel workers + ULONG m_ulParallelWorkers; + +public: + CDXLPhysicalParallelCTEConsumer(CDXLPhysicalParallelCTEConsumer &) = delete; + + // ctor + CDXLPhysicalParallelCTEConsumer(CMemoryPool *mp, ULONG id, + ULongPtrArray *output_colids_array, + ULongPtrArray *output_colidx_map, + ULONG ulParallelWorkers); + + // dtor + ~CDXLPhysicalParallelCTEConsumer() override; + + // operator type + Edxlopid GetDXLOperator() const override; + + // operator name + const CWStringConst *GetOpNameStr() const override; + + // cte identifier + ULONG + Id() const + { + return m_id; + } + + ULongPtrArray * + GetOutputColIdsArray() const + { + return m_output_colids_array; + } + + ULongPtrArray * + GetOutputColIdxMap() const + { + return m_output_colidx_map; + } + + // get number of parallel workers + ULONG UlParallelWorkers() const + { + return m_ulParallelWorkers; + } + + // serialize operator in DXL format + void SerializeToDXL(CXMLSerializer *xml_serializer, + const CDXLNode *dxlnode) const override; + +#ifdef GPOS_DEBUG + // checks whether the operator has valid structure, i.e. number and + // types of child nodes + void AssertValid(const CDXLNode *dxlnode, + BOOL validate_children) const override; +#endif // GPOS_DEBUG + + // conversion function + static CDXLPhysicalParallelCTEConsumer * + Cast(CDXLOperator *dxl_op) + { + GPOS_ASSERT(nullptr != dxl_op); + GPOS_ASSERT(EdxlopPhysicalParallelCTEConsumer == dxl_op->GetDXLOperator()); + return dynamic_cast(dxl_op); + } +}; +} // namespace gpdxl +#endif // !GPDXL_CDXLPhysicalParallelCTEConsumer_H + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelCTEProducer.h b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelCTEProducer.h new file mode 100644 index 00000000000..307757946e8 --- /dev/null +++ b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelCTEProducer.h @@ -0,0 +1,124 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CDXLPhysicalParallelCTEProducer.h + * + * IDENTIFICATION + * src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelCTEProducer.h + * + *------------------------------------------------------------------------- + */ +#ifndef GPDXL_CDXLPhysicalParallelCTEProducer_H +#define GPDXL_CDXLPhysicalParallelCTEProducer_H +#include "gpos/base.h" +#include "gpos/common/CDynamicPtrArray.h" + +#include "naucrates/dxl/operators/CDXLPhysical.h" + +namespace gpdxl +{ +//--------------------------------------------------------------------------- +// @class: +// CDXLPhysicalParallelCTEProducer +// +// @doc: +// Class for representing DXL physical parallel CTE producers +// +//--------------------------------------------------------------------------- +class CDXLPhysicalParallelCTEProducer : public CDXLPhysical +{ +private: + // cte id + ULONG m_id; + + // output column ids + ULongPtrArray *m_output_colids_array; + + // output column index mapping + ULongPtrArray *m_output_colidx_map; + + // number of parallel workers + ULONG m_ulParallelWorkers; + +public: + CDXLPhysicalParallelCTEProducer(CDXLPhysicalParallelCTEProducer &) = delete; + + // ctor + CDXLPhysicalParallelCTEProducer(CMemoryPool *mp, ULONG id, + ULongPtrArray *output_colids_array, + ULongPtrArray *output_colidx_map, + ULONG ulParallelWorkers); + + // dtor + ~CDXLPhysicalParallelCTEProducer() override; + + // operator type + Edxlopid GetDXLOperator() const override; + + // operator name + const CWStringConst *GetOpNameStr() const override; + + // cte identifier + ULONG + Id() const + { + return m_id; + } + + ULongPtrArray * + GetOutputColIdsArray() const + { + return m_output_colids_array; + } + + + ULongPtrArray * + GetOutputColIdxMap() const + { + return m_output_colidx_map; + } + + // get number of parallel workers + ULONG UlParallelWorkers() const + { + return m_ulParallelWorkers; + } + + // serialize operator in DXL format + void SerializeToDXL(CXMLSerializer *xml_serializer, + const CDXLNode *dxlnode) const override; + +#ifdef GPOS_DEBUG + // checks whether the operator has valid structure, i.e. number and + // types of child nodes + void AssertValid(const CDXLNode *dxlnode, + BOOL validate_children) const override; +#endif // GPOS_DEBUG + + // conversion function + static CDXLPhysicalParallelCTEProducer * + Cast(CDXLOperator *dxl_op) + { + GPOS_ASSERT(nullptr != dxl_op); + GPOS_ASSERT(EdxlopPhysicalParallelCTEProducer == dxl_op->GetDXLOperator()); + return dynamic_cast(dxl_op); + } +}; +} +#endif // !GPDXL_CDXLPhysicalParallelCTEProducer_H \ No newline at end of file diff --git a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelSequence.h b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelSequence.h new file mode 100644 index 00000000000..8d87d900708 --- /dev/null +++ b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelSequence.h @@ -0,0 +1,94 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CDXLPhysicalParallelSequence.h + * + * IDENTIFICATION + * src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelSequence.h + * + *------------------------------------------------------------------------- + */ + +#ifndef GPDXL_CDXLPhysicalParallelSequence_H +#define GPDXL_CDXLPhysicalParallelSequence_H + +#include "gpos/base.h" + +#include "naucrates/dxl/operators/CDXLPhysical.h" +#include "naucrates/dxl/operators/CDXLSpoolInfo.h" + +namespace gpdxl +{ +//--------------------------------------------------------------------------- +// @class: +// CDXLPhysicalParallelSequence +// +// @doc: +// Class for representing DXL physical parallel sequence operators +// +//--------------------------------------------------------------------------- +class CDXLPhysicalParallelSequence : public CDXLPhysical +{ +private: + // number of parallel workers + ULONG m_ulParallelWorkers; + +public: + CDXLPhysicalParallelSequence(CDXLPhysicalParallelSequence &) = delete; + + // ctor + CDXLPhysicalParallelSequence(CMemoryPool *mp, ULONG ulParallelWorkers); + + // dtor + ~CDXLPhysicalParallelSequence() override; + + // accessors + Edxlopid GetDXLOperator() const override; + const CWStringConst *GetOpNameStr() const override; + + // get number of parallel workers + ULONG UlParallelWorkers() const + { + return m_ulParallelWorkers; + } + + // serialize operator in DXL format + void SerializeToDXL(CXMLSerializer *xml_serializer, + const CDXLNode *dxlnode) const override; + + // conversion function + static CDXLPhysicalParallelSequence * + Cast(CDXLOperator *dxl_op) + { + GPOS_ASSERT(nullptr != dxl_op); + GPOS_ASSERT(EdxlopPhysicalParallelSequence == dxl_op->GetDXLOperator()); + + return dynamic_cast(dxl_op); + } + +#ifdef GPOS_DEBUG + // checks whether the operator has valid structure, i.e. number and + // types of child nodes + void AssertValid(const CDXLNode *, BOOL validate_children) const override; +#endif // GPOS_DEBUG +}; +} // namespace gpdxl +#endif // !GPDXL_CDXLPhysicalParallelSequence_H + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelTableScan.h b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelTableScan.h new file mode 100644 index 00000000000..ff3afa68df0 --- /dev/null +++ b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelTableScan.h @@ -0,0 +1,104 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CDXLPhysicalParallelTableScan.h + * + * IDENTIFICATION + * src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalParallelTableScan.h + * + *------------------------------------------------------------------------- + */ + +#ifndef GPDXL_CDXLPhysicalParallelTableScan_H +#define GPDXL_CDXLPhysicalParallelTableScan_H + +#include "gpos/base.h" + +#include "naucrates/dxl/operators/CDXLPhysicalTableScan.h" + +namespace gpdxl +{ +using namespace gpos; + +//--------------------------------------------------------------------------- +// @class: +// CDXLPhysicalParallelTableScan +// +// @doc: +// Class for representing DXL parallel table scan operators +// +//--------------------------------------------------------------------------- +class CDXLPhysicalParallelTableScan : public CDXLPhysicalTableScan +{ +private: + // number of parallel workers + ULONG m_ulParallelWorkers; + +public: + CDXLPhysicalParallelTableScan(const CDXLPhysicalParallelTableScan &) = delete; + + // ctor + CDXLPhysicalParallelTableScan(CMemoryPool *mp, CDXLTableDescr *table_descr, + ULONG ulParallelWorkers); + + // ctor with uninitialized table descriptor + CDXLPhysicalParallelTableScan(CMemoryPool *mp, ULONG ulParallelWorkers); + + // dtor + ~CDXLPhysicalParallelTableScan() override = default; + + // get operator type + Edxlopid GetDXLOperator() const override; + + // get operator name + const CWStringConst *GetOpNameStr() const override; + + // get number of parallel workers + ULONG UlParallelWorkers() const + { + return m_ulParallelWorkers; + } + + // serialize operator in DXL format + void SerializeToDXL(CXMLSerializer *xml_serializer, + const CDXLNode *dxlnode) const override; + + // conversion function + static CDXLPhysicalParallelTableScan * + Cast(CDXLOperator *dxl_op) + { + GPOS_ASSERT(nullptr != dxl_op); + GPOS_ASSERT(EdxlopPhysicalParallelTableScan == dxl_op->GetDXLOperator()); + + return dynamic_cast(dxl_op); + } + +#ifdef GPOS_DEBUG + // checks whether the operator has valid structure, i.e. number and + // types of child nodes + void AssertValid(const CDXLNode *dxlnode, BOOL validate_children) const override; +#endif // GPOS_DEBUG + +}; // class CDXLPhysicalParallelTableScan + +} // namespace gpdxl + +#endif // !GPDXL_CDXLPhysicalParallelTableScan_H + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalTableScan.h b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalTableScan.h index a043e137614..aef1695612a 100644 --- a/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalTableScan.h +++ b/src/backend/gporca/libnaucrates/include/naucrates/dxl/operators/CDXLPhysicalTableScan.h @@ -76,7 +76,8 @@ class CDXLPhysicalTableScan : public CDXLPhysical { GPOS_ASSERT(nullptr != dxl_op); GPOS_ASSERT(EdxlopPhysicalTableScan == dxl_op->GetDXLOperator() || - EdxlopPhysicalForeignScan == dxl_op->GetDXLOperator()); + EdxlopPhysicalForeignScan == dxl_op->GetDXLOperator() || + EdxlopPhysicalParallelTableScan == dxl_op->GetDXLOperator()); return dynamic_cast(dxl_op); } diff --git a/src/backend/gporca/libnaucrates/include/naucrates/dxl/xml/dxltokens.h b/src/backend/gporca/libnaucrates/include/naucrates/dxl/xml/dxltokens.h index 5a61801511a..af6793fd655 100644 --- a/src/backend/gporca/libnaucrates/include/naucrates/dxl/xml/dxltokens.h +++ b/src/backend/gporca/libnaucrates/include/naucrates/dxl/xml/dxltokens.h @@ -96,6 +96,7 @@ enum Edxltoken EdxltokenPhysical, EdxltokenPhysicalTableScan, + EdxltokenPhysicalParallelTableScan, EdxltokenPhysicalBitmapTableScan, EdxltokenPhysicalDynamicBitmapTableScan, EdxltokenPhysicalForeignScan, @@ -115,9 +116,11 @@ enum Edxltoken EdxltokenPhysicalResult, EdxltokenPhysicalAggregate, EdxltokenPhysicalAppend, + EdxltokenPhysicalParallelAppend, EdxltokenPhysicalMaterialize, EdxltokenPhysicalDynamicForeignScan, EdxltokenPhysicalSequence, + EdxltokenPhysicalParallelSequence, EdxltokenPhysicalDynamicTableScan, EdxltokenPhysicalDynamicIndexScan, EdxltokenPhysicalTVF, @@ -129,6 +132,8 @@ enum Edxltoken EdxltokenPhysicalCTEProducer, EdxltokenPhysicalCTEConsumer, + EdxltokenPhysicalParallelCTEProducer, + EdxltokenPhysicalParallelCTEConsumer, EdxltokenDuplicateSensitive, @@ -374,6 +379,7 @@ enum Edxltoken EdxltokenColNdvBySeg, EdxltokenParamId, + EdxltokenParallelWorkers, EdxltokenCtidColName, EdxltokenOidColName, diff --git a/src/backend/gporca/libnaucrates/include/naucrates/md/CMDRelationCtasGPDB.h b/src/backend/gporca/libnaucrates/include/naucrates/md/CMDRelationCtasGPDB.h index 780c1be5b11..195a17f9e94 100644 --- a/src/backend/gporca/libnaucrates/include/naucrates/md/CMDRelationCtasGPDB.h +++ b/src/backend/gporca/libnaucrates/include/naucrates/md/CMDRelationCtasGPDB.h @@ -264,6 +264,12 @@ class CMDRelationCtasGPDB : public IMDRelationCtas return 0; } + INT SegFileCount() const override + { + GPOS_ASSERT("Function should not be called for CTAS tables"); + return -1; + } + #ifdef GPOS_DEBUG // debug print of the metadata relation void DebugPrint(IOstream &os) const override; diff --git a/src/backend/gporca/libnaucrates/include/naucrates/md/CMDRelationGPDB.h b/src/backend/gporca/libnaucrates/include/naucrates/md/CMDRelationGPDB.h index f15b0f26ecc..ecb4bb4a4d4 100644 --- a/src/backend/gporca/libnaucrates/include/naucrates/md/CMDRelationGPDB.h +++ b/src/backend/gporca/libnaucrates/include/naucrates/md/CMDRelationGPDB.h @@ -122,13 +122,19 @@ class CMDRelationGPDB : public IMDRelation // rows CDouble m_rows; + // segment file count for AO/AOCO tables (-1 for non-AO tables) + INT m_seg_file_count; + + // parallel workers setting from table options (-1 if not set) + INT m_parallel_workers; + public: CMDRelationGPDB(const CMDRelationGPDB &) = delete; // ctor CMDRelationGPDB( CMemoryPool *mp, IMDId *mdid, CMDName *mdname, BOOL is_temp_table, - Erelstoragetype rel_storage_type, + Erelstoragetype rel_storage_type, Ereldistrpolicy rel_distr_policy, CMDColumnArray *mdcol_array, ULongPtrArray *distr_col_array, IMdIdArray *distr_opfamilies, ULongPtrArray *partition_cols_array, CharPtrArray *str_part_types_array, @@ -242,6 +248,16 @@ class CMDRelationGPDB : public IMDRelation CDouble Rows() const override; + INT SegFileCount() const override; + + // Set segment file count for AO/AOCO tables + void SetSegFileCount(INT seg_file_count); + + INT ParallelWorkers() const override; + + // Set parallel workers from table options + void SetParallelWorkers(INT parallel_workers); + #ifdef GPOS_DEBUG // debug print of the metadata relation void DebugPrint(IOstream &os) const override; diff --git a/src/backend/gporca/libnaucrates/include/naucrates/md/IMDRelation.h b/src/backend/gporca/libnaucrates/include/naucrates/md/IMDRelation.h index 036af05597a..b5e7a8f22ee 100644 --- a/src/backend/gporca/libnaucrates/include/naucrates/md/IMDRelation.h +++ b/src/backend/gporca/libnaucrates/include/naucrates/md/IMDRelation.h @@ -203,6 +203,12 @@ class IMDRelation : public IMDCacheObject // rows virtual CDouble Rows() const = 0; + + // segment file count for AO/AOCO tables (returns -1 for non-AO tables) + virtual INT SegFileCount() const = 0; + + // parallel workers setting from table options (returns -1 if not set) + virtual INT ParallelWorkers() const = 0; }; // common structure over relation and external relation metadata for index info diff --git a/src/backend/gporca/libnaucrates/include/naucrates/md/IMDRelationCtas.h b/src/backend/gporca/libnaucrates/include/naucrates/md/IMDRelationCtas.h index 18aa6b8e30c..ae805c9cc17 100644 --- a/src/backend/gporca/libnaucrates/include/naucrates/md/IMDRelationCtas.h +++ b/src/backend/gporca/libnaucrates/include/naucrates/md/IMDRelationCtas.h @@ -108,6 +108,13 @@ class IMDRelationCtas : public IMDRelation // CTAS storage options virtual CDXLCtasStorageOptions *GetDxlCtasStorageOption() const = 0; + + // parallel workers - CTAS tables don't have this setting yet + INT + ParallelWorkers() const override + { + return -1; // not set + } }; } // namespace gpmd diff --git a/src/backend/gporca/libnaucrates/include/naucrates/traceflags/traceflags.h b/src/backend/gporca/libnaucrates/include/naucrates/traceflags/traceflags.h index 2e489f214e5..a7a298d9825 100644 --- a/src/backend/gporca/libnaucrates/include/naucrates/traceflags/traceflags.h +++ b/src/backend/gporca/libnaucrates/include/naucrates/traceflags/traceflags.h @@ -188,6 +188,9 @@ enum EOptTraceFlag // Eager Agg EopttraceEnableEagerAgg = 103030, + // Enable parallel append scan + EopttraceEnableParallelAppendScan = 103031, + // ExpandFullJoin EopttraceExpandFullJoin = 103032, diff --git a/src/backend/gporca/libnaucrates/src/md/CMDRelationGPDB.cpp b/src/backend/gporca/libnaucrates/src/md/CMDRelationGPDB.cpp index 03f52c3fdd0..dceb24d0c55 100644 --- a/src/backend/gporca/libnaucrates/src/md/CMDRelationGPDB.cpp +++ b/src/backend/gporca/libnaucrates/src/md/CMDRelationGPDB.cpp @@ -62,7 +62,9 @@ CMDRelationGPDB::CMDRelationGPDB( m_colpos_nondrop_colpos_map(nullptr), m_attrno_nondrop_col_pos_map(nullptr), m_nondrop_col_pos_array(nullptr), - m_rows(rows) + m_rows(rows), + m_seg_file_count(-1), + m_parallel_workers(-1) { GPOS_ASSERT(mdid->IsValid()); GPOS_ASSERT(nullptr != mdcol_array); @@ -595,6 +597,46 @@ CMDRelationGPDB::Rows() const return m_rows; } +INT +CMDRelationGPDB::SegFileCount() const +{ + return m_seg_file_count; +} + +//--------------------------------------------------------------------------- +// @function: +// CMDRelationGPDB::SetSegFileCount +// +// @doc: +// Set segment file count for AO/AOCO tables +// +//--------------------------------------------------------------------------- +void +CMDRelationGPDB::SetSegFileCount(INT seg_file_count) +{ + m_seg_file_count = seg_file_count; +} + +INT +CMDRelationGPDB::ParallelWorkers() const +{ + return m_parallel_workers; +} + +//--------------------------------------------------------------------------- +// @function: +// CMDRelationGPDB::SetParallelWorkers +// +// @doc: +// Set parallel workers from table options +// +//--------------------------------------------------------------------------- +void +CMDRelationGPDB::SetParallelWorkers(INT parallel_workers) +{ + m_parallel_workers = parallel_workers; +} + //--------------------------------------------------------------------------- // @function: // CMDRelationGPDB::Serialize diff --git a/src/backend/gporca/libnaucrates/src/operators/CDXLOperatorFactory.cpp b/src/backend/gporca/libnaucrates/src/operators/CDXLOperatorFactory.cpp index 8b037b833f0..94cd44bc49e 100644 --- a/src/backend/gporca/libnaucrates/src/operators/CDXLOperatorFactory.cpp +++ b/src/backend/gporca/libnaucrates/src/operators/CDXLOperatorFactory.cpp @@ -28,8 +28,10 @@ #include "naucrates/dxl/operators/CDXLDatumStatsDoubleMappable.h" #include "naucrates/dxl/operators/CDXLDatumStatsLintMappable.h" #include "naucrates/dxl/operators/CDXLLogicalJoin.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelTableScan.h" #include "naucrates/dxl/operators/CDXLPhysicalAgg.h" #include "naucrates/dxl/operators/CDXLPhysicalAppend.h" +#include "naucrates/dxl/operators/CDXLPhysicalParallelAppend.h" #include "naucrates/dxl/operators/CDXLPhysicalBroadcastMotion.h" #include "naucrates/dxl/operators/CDXLPhysicalGatherMotion.h" #include "naucrates/dxl/operators/CDXLPhysicalHashJoin.h" @@ -101,6 +103,31 @@ CDXLOperatorFactory::MakeDXLTblScan(CDXLMemoryManager *dxl_memory_manager, return GPOS_NEW(mp) CDXLPhysicalTableScan(mp); } +//--------------------------------------------------------------------------- +// @function: +// CDXLOperatorFactory::MakeDXLParallelTblScan +// +// @doc: +// Construct a parallel table scan operator +// +//--------------------------------------------------------------------------- +CDXLPhysical * +CDXLOperatorFactory::MakeDXLParallelTblScan(CDXLMemoryManager *dxl_memory_manager, + const Attributes &attrs) +{ + // get the memory pool from the memory manager + CMemoryPool *mp = dxl_memory_manager->Pmp(); + + // extract number of parallel workers + const XMLCh *parallel_workers_xml = ExtractAttrValue(attrs, EdxltokenParallelWorkers, + EdxltokenPhysicalParallelTableScan); + ULONG ulParallelWorkers = CDXLOperatorFactory::ConvertAttrValueToUlong( + dxl_memory_manager, parallel_workers_xml, EdxltokenParallelWorkers, + EdxltokenPhysicalParallelTableScan); + + return GPOS_NEW(mp) CDXLPhysicalParallelTableScan(mp, ulParallelWorkers); +} + //--------------------------------------------------------------------------- // @function: // CDXLOperatorFactory::MakeDXLResult @@ -382,6 +409,48 @@ CDXLOperatorFactory::MakeDXLAppend(CDXLMemoryManager *dxl_memory_manager, nullptr, selector_ids); } +//--------------------------------------------------------------------------- +// @function: +// CDXLOperatorFactory::MakeDXLParallelAppend +// @doc: +// Construct a Parallel Append operator +// +//--------------------------------------------------------------------------- +CDXLPhysical * +CDXLOperatorFactory::MakeDXLParallelAppend(CDXLMemoryManager *dxl_memory_manager, + const Attributes &attrs) +{ + // get the memory pool from the memory manager + CMemoryPool *mp = dxl_memory_manager->Pmp(); + + BOOL is_target = ExtractConvertAttrValueToBool(dxl_memory_manager, attrs, + EdxltokenAppendIsTarget, + EdxltokenPhysicalParallelAppend); + + BOOL is_zapped = ExtractConvertAttrValueToBool(dxl_memory_manager, attrs, + EdxltokenAppendIsZapped, + EdxltokenPhysicalParallelAppend); + + ULONG scan_id = ExtractConvertAttrValueToUlong(dxl_memory_manager, attrs, EdxltokenPartIndexId, + EdxltokenPhysicalParallelAppend, true /* is_optional */, + gpos::ulong_max /* default_value */); + + ULONG parallel_workers = ExtractConvertAttrValueToUlong(dxl_memory_manager, attrs, EdxltokenParallelWorkers, + EdxltokenPhysicalParallelAppend, true, + 0); + + ULongPtrArray *selector_ids = nullptr; + if (scan_id != gpos::ulong_max) + { + selector_ids = ExtractConvertValuesToArray(dxl_memory_manager, attrs, + EdxltokenSelectorIds, + EdxltokenPhysicalParallelAppend); + } + + return GPOS_NEW(mp) CDXLPhysicalParallelAppend(mp, is_target, is_zapped, scan_id, + nullptr, selector_ids, parallel_workers); +} + //--------------------------------------------------------------------------- // @function: // CDXLOperatorFactory::MakeDXLLimit diff --git a/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalAppend.cpp b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalAppend.cpp index 8e90a279cd8..adbdbd9bec1 100644 --- a/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalAppend.cpp +++ b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalAppend.cpp @@ -34,8 +34,10 @@ CDXLPhysicalAppend::CDXLPhysicalAppend(CMemoryPool *mp, BOOL fIsTarget, { } -CDXLPhysicalAppend::CDXLPhysicalAppend(CMemoryPool *mp, BOOL fIsTarget, - BOOL fIsZapped, ULONG scan_id, +CDXLPhysicalAppend::CDXLPhysicalAppend(CMemoryPool *mp, + BOOL fIsTarget, + BOOL fIsZapped, + ULONG scan_id, CDXLTableDescr *dxl_table_desc, ULongPtrArray *selector_ids) : CDXLPhysical(mp), diff --git a/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelAppend.cpp b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelAppend.cpp new file mode 100644 index 00000000000..55289774440 --- /dev/null +++ b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelAppend.cpp @@ -0,0 +1,187 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CPhysicalParallelAppend.cpp + * + * IDENTIFICATION + * src/backend/gporca/libgpopt/src/operators/CPhysicalParallelAppend.cpp + * + *------------------------------------------------------------------------- + */ + +#include "naucrates/dxl/operators/CDXLPhysicalParallelAppend.h" + +#include "gpos/common/CBitSetIter.h" + +#include "naucrates/dxl/CDXLUtils.h" +#include "naucrates/dxl/operators/CDXLNode.h" +#include "naucrates/dxl/xml/CXMLSerializer.h" + +using namespace gpos; +using namespace gpdxl; + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelAppend::CDXLPhysicalParallelAppend +// +// @doc: +// Constructor +// +//--------------------------------------------------------------------------- +CDXLPhysicalParallelAppend::CDXLPhysicalParallelAppend(CMemoryPool *mp, BOOL fIsTarget, + BOOL fIsZapped, ULONG ulParallelWorkers) + : CDXLPhysical(mp), m_used_in_upd_del(fIsTarget), m_is_zapped(fIsZapped), m_ulParallelWorkers(ulParallelWorkers) +{ +} + +CDXLPhysicalParallelAppend::CDXLPhysicalParallelAppend(CMemoryPool *mp, + BOOL fIsTarget, + BOOL fIsZapped, + ULONG scan_id, + CDXLTableDescr *dxl_table_desc, + ULongPtrArray *selector_ids, + ULONG ulParallelWorkers) + : CDXLPhysical(mp), + m_used_in_upd_del(fIsTarget), + m_is_zapped(fIsZapped), + m_scan_id(scan_id), + m_dxl_table_descr(dxl_table_desc), + m_selector_ids(selector_ids), + m_ulParallelWorkers(ulParallelWorkers) +{ +} + +CDXLPhysicalParallelAppend::~CDXLPhysicalParallelAppend() +{ + CRefCount::SafeRelease(m_dxl_table_descr); + CRefCount::SafeRelease(m_selector_ids); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelAppend::GetDXLOperator +// +// @doc: +// Operator type +// +//--------------------------------------------------------------------------- +Edxlopid +CDXLPhysicalParallelAppend::GetDXLOperator() const +{ + return EdxlopPhysicalParallelAppend; +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelAppend::GetOpNameStr +// +// @doc: +// Operator name +// +//--------------------------------------------------------------------------- +const CWStringConst * +CDXLPhysicalParallelAppend::GetOpNameStr() const +{ + return CDXLTokens::GetDXLTokenStr(EdxltokenPhysicalParallelAppend); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelAppend::SerializeToDXL +// +// @doc: +// Serialize operator in DXL format +// +//--------------------------------------------------------------------------- +void +CDXLPhysicalParallelAppend::SerializeToDXL(CXMLSerializer *xml_serializer, const CDXLNode *dxlnode) const +{ + const CWStringConst *element_name = GetOpNameStr(); + + xml_serializer->OpenElement( + CDXLTokens::GetDXLTokenStr(EdxltokenNamespacePrefix), element_name); + + xml_serializer->AddAttribute( + CDXLTokens::GetDXLTokenStr(EdxltokenAppendIsTarget), m_used_in_upd_del); + xml_serializer->AddAttribute( + CDXLTokens::GetDXLTokenStr(EdxltokenAppendIsZapped), m_is_zapped); + xml_serializer->AddAttribute( + CDXLTokens::GetDXLTokenStr(EdxltokenParallelWorkers), m_ulParallelWorkers); + + if (m_scan_id != gpos::ulong_max) + { + xml_serializer->AddAttribute( + CDXLTokens::GetDXLTokenStr(EdxltokenPartIndexId), m_scan_id); + + CWStringDynamic *serialized_selector_ids = + CDXLUtils::Serialize(m_mp, m_selector_ids); + xml_serializer->AddAttribute( + CDXLTokens::GetDXLTokenStr(EdxltokenSelectorIds), + serialized_selector_ids); + GPOS_DELETE(serialized_selector_ids); + } + // serialize properties + dxlnode->SerializePropertiesToDXL(xml_serializer); + + if (m_dxl_table_descr != nullptr) + { + GPOS_ASSERT(m_scan_id != gpos::ulong_max); + m_dxl_table_descr->SerializeToDXL(xml_serializer); + } + + // serialize children + dxlnode->SerializeChildrenToDXL(xml_serializer); + + xml_serializer->CloseElement( + CDXLTokens::GetDXLTokenStr(EdxltokenNamespacePrefix), element_name); +} + +#ifdef GPOS_DEBUG +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalAppend::AssertValid +// +// @doc: +// Checks whether operator node is well-structured +// +//--------------------------------------------------------------------------- +void +CDXLPhysicalParallelAppend::AssertValid(const CDXLNode *dxlnode, + BOOL validate_children) const +{ + // assert proj list and filter are valid + CDXLPhysical::AssertValid(dxlnode, validate_children); + + const ULONG ulChildren = dxlnode->Arity(); + for (ULONG ul = EdxlappendIndexFirstChild; ul < ulChildren; ul++) + { + CDXLNode *child_dxlnode = (*dxlnode)[ul]; + GPOS_ASSERT(EdxloptypePhysical == + child_dxlnode->GetOperator()->GetDXLOperatorType()); + + if (validate_children) + { + child_dxlnode->GetOperator()->AssertValid(child_dxlnode, + validate_children); + } + } +} +#endif // GPOS_DEBUG + +// EOF diff --git a/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelCTEConsumer.cpp b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelCTEConsumer.cpp new file mode 100644 index 00000000000..005e3f43970 --- /dev/null +++ b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelCTEConsumer.cpp @@ -0,0 +1,166 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CDXLPhysicalParallelCTEConsumer.cpp + * + * IDENTIFICATION + * src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelCTEConsumer.cpp + * + *------------------------------------------------------------------------- + */ + +#include "naucrates/dxl/operators/CDXLPhysicalParallelCTEConsumer.h" + +#include "gpos/string/CWStringDynamic.h" + +#include "naucrates/dxl/CDXLUtils.h" +#include "naucrates/dxl/operators/CDXLNode.h" +#include "naucrates/dxl/xml/CXMLSerializer.h" +#include "naucrates/dxl/xml/dxltokens.h" + +using namespace gpos; +using namespace gpdxl; + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEConsumer::CDXLPhysicalParallelCTEConsumer +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CDXLPhysicalParallelCTEConsumer::CDXLPhysicalParallelCTEConsumer( + CMemoryPool *mp, ULONG id, ULongPtrArray *output_colids_array, ULongPtrArray *output_colidx_map, ULONG ulParallelWorkers) + : CDXLPhysical(mp), m_id(id), m_output_colids_array(output_colids_array), m_output_colidx_map(output_colidx_map), m_ulParallelWorkers(ulParallelWorkers) +{ + GPOS_ASSERT(nullptr != output_colids_array); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEConsumer::~CDXLPhysicalParallelCTEConsumer +// +// @doc: +// Dtor +// +//--------------------------------------------------------------------------- +CDXLPhysicalParallelCTEConsumer::~CDXLPhysicalParallelCTEConsumer() +{ + m_output_colids_array->Release(); + CRefCount::SafeRelease(m_output_colidx_map); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEConsumer::GetDXLOperator +// +// @doc: +// Operator type +// +//--------------------------------------------------------------------------- +Edxlopid +CDXLPhysicalParallelCTEConsumer::GetDXLOperator() const +{ + return EdxlopPhysicalParallelCTEConsumer; +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEConsumer::GetOpNameStr +// +// @doc: +// Operator name +// +//--------------------------------------------------------------------------- +const CWStringConst * +CDXLPhysicalParallelCTEConsumer::GetOpNameStr() const +{ + return CDXLTokens::GetDXLTokenStr(EdxltokenPhysicalParallelCTEConsumer); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEConsumer::SerializeToDXL +// +// @doc: +// Serialize operator in DXL format +// +//--------------------------------------------------------------------------- +void +CDXLPhysicalParallelCTEConsumer::SerializeToDXL(CXMLSerializer *xml_serializer, + const CDXLNode *dxlnode) const +{ + const CWStringConst *element_name = GetOpNameStr(); + + xml_serializer->OpenElement( + CDXLTokens::GetDXLTokenStr(EdxltokenNamespacePrefix), element_name); + xml_serializer->AddAttribute(CDXLTokens::GetDXLTokenStr(EdxltokenCTEId), + Id()); + + CWStringDynamic *str_colids = + CDXLUtils::Serialize(m_mp, m_output_colids_array); + xml_serializer->AddAttribute(CDXLTokens::GetDXLTokenStr(EdxltokenColumns), + str_colids); + GPOS_DELETE(str_colids); + + if (m_output_colidx_map) { + CWStringDynamic *str_colidx_map = + CDXLUtils::Serialize(m_mp, m_output_colidx_map); + + xml_serializer->AddAttribute(CDXLTokens::GetDXLTokenStr(EdxltokenColIdxmap), + str_colidx_map); + GPOS_DELETE(str_colidx_map); + } + + // serialize properties + dxlnode->SerializePropertiesToDXL(xml_serializer); + + dxlnode->SerializeChildrenToDXL(xml_serializer); + xml_serializer->CloseElement( + CDXLTokens::GetDXLTokenStr(EdxltokenNamespacePrefix), element_name); +} + +#ifdef GPOS_DEBUG +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEConsumer::AssertValid +// +// @doc: +// Checks whether operator node is well-structured +// +//--------------------------------------------------------------------------- +void +CDXLPhysicalParallelCTEConsumer::AssertValid(const CDXLNode *dxlnode, + BOOL validate_children) const +{ + GPOS_ASSERT(1 == dxlnode->Arity()); + + CDXLNode *dxlnode_proj_list = (*dxlnode)[0]; + GPOS_ASSERT(EdxlopScalarProjectList == + dxlnode_proj_list->GetOperator()->GetDXLOperator()); + + if (validate_children) + { + dxlnode_proj_list->GetOperator()->AssertValid(dxlnode_proj_list, + validate_children); + } +} +#endif // GPOS_DEBUG + +// EOF diff --git a/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelCTEProducer.cpp b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelCTEProducer.cpp new file mode 100644 index 00000000000..738f23fe11b --- /dev/null +++ b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelCTEProducer.cpp @@ -0,0 +1,171 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CDXLPhysicalParallelCTEProducer.cpp + * + * IDENTIFICATION + * src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelCTEProducer.cpp + * + *------------------------------------------------------------------------- + */ + +#include "naucrates/dxl/operators/CDXLPhysicalParallelCTEProducer.h" + +#include "gpos/string/CWStringDynamic.h" + +#include "naucrates/dxl/CDXLUtils.h" +#include "naucrates/dxl/operators/CDXLNode.h" +#include "naucrates/dxl/xml/CXMLSerializer.h" +#include "naucrates/dxl/xml/dxltokens.h" + +using namespace gpos; +using namespace gpdxl; + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEProducer::CDXLPhysicalParallelCTEProducer +// +// @doc: +// Ctor +// +//--------------------------------------------------------------------------- +CDXLPhysicalParallelCTEProducer::CDXLPhysicalParallelCTEProducer( + CMemoryPool *mp, ULONG id, ULongPtrArray *output_colids_array, ULongPtrArray *output_colidx_map, ULONG ulParallelWorkers) + : CDXLPhysical(mp), m_id(id), m_output_colids_array(output_colids_array), m_output_colidx_map(output_colidx_map), m_ulParallelWorkers(ulParallelWorkers) +{ + GPOS_ASSERT(nullptr != output_colids_array); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEProducer::~CDXLPhysicalParallelCTEProducer +// +// @doc: +// Dtor +// +//--------------------------------------------------------------------------- +CDXLPhysicalParallelCTEProducer::~CDXLPhysicalParallelCTEProducer() +{ + m_output_colids_array->Release(); + CRefCount::SafeRelease(m_output_colidx_map); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEProducer::GetDXLOperator +// +// @doc: +// Operator type +// +//--------------------------------------------------------------------------- +Edxlopid +CDXLPhysicalParallelCTEProducer::GetDXLOperator() const +{ + return EdxlopPhysicalParallelCTEProducer; +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEProducer::GetOpNameStr +// +// @doc: +// Operator name +// +//--------------------------------------------------------------------------- +const CWStringConst * +CDXLPhysicalParallelCTEProducer::GetOpNameStr() const +{ + return CDXLTokens::GetDXLTokenStr(EdxltokenPhysicalParallelCTEProducer); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEProducer::SerializeToDXL +// +// @doc: +// Serialize operator in DXL format +// +//--------------------------------------------------------------------------- +void +CDXLPhysicalParallelCTEProducer::SerializeToDXL(CXMLSerializer *xml_serializer, + const CDXLNode *dxlnode) const +{ + const CWStringConst *element_name = GetOpNameStr(); + + xml_serializer->OpenElement( + CDXLTokens::GetDXLTokenStr(EdxltokenNamespacePrefix), element_name); + xml_serializer->AddAttribute(CDXLTokens::GetDXLTokenStr(EdxltokenCTEId), + Id()); + + CWStringDynamic *pstrColIds = + CDXLUtils::Serialize(m_mp, m_output_colids_array); + xml_serializer->AddAttribute(CDXLTokens::GetDXLTokenStr(EdxltokenColumns), + pstrColIds); + GPOS_DELETE(pstrColIds); + + if (m_output_colidx_map) { + CWStringDynamic *str_colidx_map = + CDXLUtils::Serialize(m_mp, m_output_colidx_map); + + xml_serializer->AddAttribute(CDXLTokens::GetDXLTokenStr(EdxltokenColIdxmap), + str_colidx_map); + GPOS_DELETE(str_colidx_map); + } + + // serialize properties + dxlnode->SerializePropertiesToDXL(xml_serializer); + + dxlnode->SerializeChildrenToDXL(xml_serializer); + xml_serializer->CloseElement( + CDXLTokens::GetDXLTokenStr(EdxltokenNamespacePrefix), element_name); +} + +#ifdef GPOS_DEBUG +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelCTEProducer::AssertValid +// +// @doc: +// Checks whether operator node is well-structured +// +//--------------------------------------------------------------------------- +void +CDXLPhysicalParallelCTEProducer::AssertValid(const CDXLNode *dxlnode, + BOOL validate_children) const +{ + GPOS_ASSERT(2 == dxlnode->Arity()); + + CDXLNode *pdxlnPrL = (*dxlnode)[0]; + CDXLNode *child_dxlnode = (*dxlnode)[1]; + + GPOS_ASSERT(EdxlopScalarProjectList == + pdxlnPrL->GetOperator()->GetDXLOperator()); + GPOS_ASSERT(EdxloptypePhysical == + child_dxlnode->GetOperator()->GetDXLOperatorType()); + + if (validate_children) + { + pdxlnPrL->GetOperator()->AssertValid(pdxlnPrL, validate_children); + child_dxlnode->GetOperator()->AssertValid(child_dxlnode, + validate_children); + } +} +#endif // GPOS_DEBUG + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelSequence.cpp b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelSequence.cpp new file mode 100644 index 00000000000..968f4867777 --- /dev/null +++ b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelSequence.cpp @@ -0,0 +1,150 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CDXLPhysicalParallelSequence.cpp + * + * IDENTIFICATION + * src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelSequence.cpp + * + *------------------------------------------------------------------------- + */ + +#ifndef GPOPT_CPhysicalParallelSequence_H +#define GPOPT_CPhysicalParallelSequence_H + +#include "naucrates/dxl/operators/CDXLPhysicalParallelSequence.h" + +#include "naucrates/dxl/operators/CDXLNode.h" +#include "naucrates/dxl/xml/CXMLSerializer.h" + +using namespace gpos; +using namespace gpdxl; + +//--------------------------------------------------------------------------- +// @class: +// CDXLPhysicalParallelSequence::CDXLPhysicalParallelSequence +// +// @doc: +// Constructor +// +//--------------------------------------------------------------------------- +CDXLPhysicalParallelSequence::CDXLPhysicalParallelSequence(CMemoryPool *mp, ULONG ulParallelWorkers) + : CDXLPhysical(mp), m_ulParallelWorkers(ulParallelWorkers) +{ +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelSequence::~CDXLPhysicalParallelSequence +// +// @doc: +// Destructor +// +//--------------------------------------------------------------------------- +CDXLPhysicalParallelSequence::~CDXLPhysicalParallelSequence() = default; + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelSequence::GetDXLOperator +// +// @doc: +// Operator type +// +//--------------------------------------------------------------------------- +Edxlopid +CDXLPhysicalParallelSequence::GetDXLOperator() const +{ + return EdxlopPhysicalParallelSequence; +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelSequence::GetOpNameStr +// +// @doc: +// Operator name +// +//--------------------------------------------------------------------------- +const CWStringConst * +CDXLPhysicalParallelSequence::GetOpNameStr() const +{ + return CDXLTokens::GetDXLTokenStr(EdxltokenPhysicalParallelSequence); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelSequence::SerializeToDXL +// +// @doc: +// Serialize operator in DXL format +// +//--------------------------------------------------------------------------- +void +CDXLPhysicalParallelSequence::SerializeToDXL(CXMLSerializer *xml_serializer, + const CDXLNode *dxlnode) const +{ + const CWStringConst *element_name = GetOpNameStr(); + + xml_serializer->OpenElement( + CDXLTokens::GetDXLTokenStr(EdxltokenPhysicalParallelSequence), element_name); + + // serialize properties + dxlnode->SerializePropertiesToDXL(xml_serializer); + + // serialize children + dxlnode->SerializeChildrenToDXL(xml_serializer); + + xml_serializer->CloseElement( + CDXLTokens::GetDXLTokenStr(EdxltokenNamespacePrefix), element_name); +} + +#ifdef GPOS_DEBUG +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelSequence::AssertValid +// +// @doc: +// Checks whether operator node is well-structured +// +//--------------------------------------------------------------------------- +void +CDXLPhysicalParallelSequence::AssertValid(const CDXLNode *dxlnode, + BOOL validate_children) const +{ + const ULONG arity = dxlnode->Arity(); + GPOS_ASSERT(1 < arity); + + for (ULONG ul = 1; ul < arity; ul++) + { + CDXLNode *child_dxlnode = (*dxlnode)[ul]; + GPOS_ASSERT(EdxloptypePhysical == + child_dxlnode->GetOperator()->GetDXLOperatorType()); + + if (validate_children) + { + child_dxlnode->GetOperator()->AssertValid(child_dxlnode, + validate_children); + } + } +} +#endif // GPOS_DEBUG + +#endif // !GPOPT_CPhysicalParallelSequence_H + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelTableScan.cpp b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelTableScan.cpp new file mode 100644 index 00000000000..a96657031e9 --- /dev/null +++ b/src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelTableScan.cpp @@ -0,0 +1,162 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * CDXLPhysicalParallelTableScan.cpp + * + * IDENTIFICATION + * src/backend/gporca/libnaucrates/src/operators/CDXLPhysicalParallelTableScan.cpp + * + *------------------------------------------------------------------------- + */ + +#include "naucrates/dxl/operators/CDXLPhysicalParallelTableScan.h" + +#include "naucrates/dxl/operators/CDXLNode.h" +#include "naucrates/dxl/xml/CXMLSerializer.h" + +using namespace gpos; +using namespace gpdxl; + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelTableScan::CDXLPhysicalParallelTableScan +// +// @doc: +// Constructor +// +//--------------------------------------------------------------------------- +CDXLPhysicalParallelTableScan::CDXLPhysicalParallelTableScan(CMemoryPool *mp, + CDXLTableDescr *table_descr, + ULONG ulParallelWorkers) + : CDXLPhysicalTableScan(mp, table_descr), + m_ulParallelWorkers(ulParallelWorkers) +{ + GPOS_ASSERT(ulParallelWorkers > 0); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelTableScan::CDXLPhysicalParallelTableScan +// +// @doc: +// Constructor with uninitialized table descriptor +// +//--------------------------------------------------------------------------- +CDXLPhysicalParallelTableScan::CDXLPhysicalParallelTableScan(CMemoryPool *mp, + ULONG ulParallelWorkers) + : CDXLPhysicalTableScan(mp), + m_ulParallelWorkers(ulParallelWorkers) +{ + GPOS_ASSERT(ulParallelWorkers > 0); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelTableScan::GetDXLOperator +// +// @doc: +// Operator type +// +//--------------------------------------------------------------------------- +Edxlopid +CDXLPhysicalParallelTableScan::GetDXLOperator() const +{ + return EdxlopPhysicalParallelTableScan; +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelTableScan::GetOpNameStr +// +// @doc: +// Operator name +// +//--------------------------------------------------------------------------- +const CWStringConst * +CDXLPhysicalParallelTableScan::GetOpNameStr() const +{ + return CDXLTokens::GetDXLTokenStr(EdxltokenPhysicalParallelTableScan); +} + +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelTableScan::SerializeToDXL +// +// @doc: +// Serialize operator in DXL format +// +//--------------------------------------------------------------------------- +void +CDXLPhysicalParallelTableScan::SerializeToDXL(CXMLSerializer *xml_serializer, + const CDXLNode *dxlnode) const +{ + const CWStringConst *element_name = GetOpNameStr(); + xml_serializer->OpenElement(CDXLTokens::GetDXLTokenStr(EdxltokenNamespacePrefix), + element_name); + + // serialize parallel workers attribute + xml_serializer->AddAttribute(CDXLTokens::GetDXLTokenStr(EdxltokenParallelWorkers), + m_ulParallelWorkers); + + // serialize properties + dxlnode->SerializePropertiesToDXL(xml_serializer); + + // serialize projection list and filter + dxlnode->SerializeChildrenToDXL(xml_serializer); + + xml_serializer->CloseElement(CDXLTokens::GetDXLTokenStr(EdxltokenNamespacePrefix), + element_name); +} + +#ifdef GPOS_DEBUG +//--------------------------------------------------------------------------- +// @function: +// CDXLPhysicalParallelTableScan::AssertValid +// +// @doc: +// Checks whether operator node is well-structured +// +//--------------------------------------------------------------------------- +void +CDXLPhysicalParallelTableScan::AssertValid(const CDXLNode *dxlnode, + BOOL validate_children) const +{ + // assert proj list and filter are valid + CDXLPhysical::AssertValid(dxlnode, validate_children); + + // parallel table scan has only 2 children (proj list and filter) + GPOS_ASSERT(2 == dxlnode->Arity()); + + CDXLNode *proj_list_dxlnode = (*dxlnode)[0]; // First child is projection list + CDXLNode *filter_dxlnode = (*dxlnode)[1]; // Second child is filter + + GPOS_ASSERT(EdxlopScalarProjectList == + proj_list_dxlnode->GetOperator()->GetDXLOperator()); + GPOS_ASSERT(EdxlopScalarFilter == + filter_dxlnode->GetOperator()->GetDXLOperator()); + + if (validate_children) + { + proj_list_dxlnode->GetOperator()->AssertValid(proj_list_dxlnode, validate_children); + filter_dxlnode->GetOperator()->AssertValid(filter_dxlnode, validate_children); + } +} +#endif // GPOS_DEBUG + +// EOF \ No newline at end of file diff --git a/src/backend/gporca/libnaucrates/src/operators/Makefile b/src/backend/gporca/libnaucrates/src/operators/Makefile index 418bd5a0fd2..fb8a37e4c0c 100644 --- a/src/backend/gporca/libnaucrates/src/operators/Makefile +++ b/src/backend/gporca/libnaucrates/src/operators/Makefile @@ -51,12 +51,15 @@ OBJS = CDXLColDescr.o \ CDXLPhysicalAbstractBitmapScan.o \ CDXLPhysicalAgg.o \ CDXLPhysicalAppend.o \ + CDXLPhysicalParallelAppend.o \ CDXLPhysicalAssert.o \ CDXLPhysicalBitmapTableScan.o \ CDXLPhysicalBroadcastMotion.o \ CDXLPhysicalCTAS.o \ CDXLPhysicalCTEConsumer.o \ + CDXLPhysicalParallelCTEConsumer.o \ CDXLPhysicalCTEProducer.o \ + CDXLPhysicalParallelCTEProducer.o \ CDXLPhysicalDML.o \ CDXLPhysicalDynamicBitmapTableScan.o \ CDXLPhysicalDynamicIndexScan.o \ @@ -81,10 +84,12 @@ OBJS = CDXLColDescr.o \ CDXLPhysicalResult.o \ CDXLPhysicalRoutedDistributeMotion.o \ CDXLPhysicalSequence.o \ + CDXLPhysicalParallelSequence.o \ CDXLPhysicalSort.o \ CDXLPhysicalSplit.o \ CDXLPhysicalTVF.o \ CDXLPhysicalTableScan.o \ + CDXLPhysicalParallelTableScan.o \ CDXLPhysicalValuesScan.o \ CDXLPhysicalWindow.o \ CDXLProperties.o \ diff --git a/src/backend/gporca/libnaucrates/src/parser/CParseHandlerOptimizerConfig.cpp b/src/backend/gporca/libnaucrates/src/parser/CParseHandlerOptimizerConfig.cpp index 3b64d409bfe..49b4d786b55 100644 --- a/src/backend/gporca/libnaucrates/src/parser/CParseHandlerOptimizerConfig.cpp +++ b/src/backend/gporca/libnaucrates/src/parser/CParseHandlerOptimizerConfig.cpp @@ -273,7 +273,7 @@ CParseHandlerOptimizerConfig::EndElement(const XMLCh *const, // element_uri, } m_optimizer_config = GPOS_NEW(m_mp) COptimizerConfig( - pec, stats_config, pcteconfig, pcm, phint, pplanhint, pwindowoidsGPDB); + pec, stats_config, pcteconfig, pcm, phint, pplanhint, pwindowoidsGPDB, false); CParseHandlerTraceFlags *pphTraceFlags = dynamic_cast((*this)[this->Length() - 1]); diff --git a/src/backend/gporca/libnaucrates/src/xml/dxltokens.cpp b/src/backend/gporca/libnaucrates/src/xml/dxltokens.cpp index 0f59d689a05..1fdc5159753 100644 --- a/src/backend/gporca/libnaucrates/src/xml/dxltokens.cpp +++ b/src/backend/gporca/libnaucrates/src/xml/dxltokens.cpp @@ -112,6 +112,7 @@ CDXLTokens::Init(CMemoryPool *mp) {EdxltokenPhysical, GPOS_WSZ_LIT("OpPhysical")}, {EdxltokenPhysicalTableScan, GPOS_WSZ_LIT("TableScan")}, + {EdxltokenPhysicalParallelTableScan, GPOS_WSZ_LIT("ParallelTableScan")}, {EdxltokenPhysicalBitmapTableScan, GPOS_WSZ_LIT("BitmapTableScan")}, {EdxltokenPhysicalDynamicBitmapTableScan, GPOS_WSZ_LIT("DynamicBitmapTableScan")}, @@ -136,6 +137,7 @@ CDXLTokens::Init(CMemoryPool *mp) {EdxltokenPhysicalResult, GPOS_WSZ_LIT("Result")}, {EdxltokenPhysicalValuesScan, GPOS_WSZ_LIT("Values")}, {EdxltokenPhysicalAppend, GPOS_WSZ_LIT("Append")}, + {EdxltokenPhysicalParallelAppend, GPOS_WSZ_LIT("Parallel Append")}, {EdxltokenPhysicalMaterialize, GPOS_WSZ_LIT("Materialize")}, {EdxltokenPhysicalDynamicForeignScan, GPOS_WSZ_LIT("DynamicForeignScan")}, @@ -420,6 +422,7 @@ CDXLTokens::Init(CMemoryPool *mp) {EdxltokenColNdvBySeg, GPOS_WSZ_LIT("NdvBySeg")}, {EdxltokenParamId, GPOS_WSZ_LIT("ParamId")}, + {EdxltokenParallelWorkers, GPOS_WSZ_LIT("ParallelWorkers")}, {EdxltokenCtidColName, GPOS_WSZ_LIT("ctid")}, {EdxltokenOidColName, GPOS_WSZ_LIT("oid")}, diff --git a/src/backend/gporca/server/src/startup/main.cpp b/src/backend/gporca/server/src/startup/main.cpp index 29be9dc9994..9418fa94abb 100644 --- a/src/backend/gporca/server/src/startup/main.cpp +++ b/src/backend/gporca/server/src/startup/main.cpp @@ -381,6 +381,24 @@ PvExec(void *pv) return nullptr; } +//--------------------------------------------------------------------------- +// Stub implementations for PostgreSQL symbols required by GPORCA +// These are needed for standalone testing without linking to PostgreSQL +//--------------------------------------------------------------------------- + +// Stub for parallel mode check +namespace gpdb { + bool IsParallelModeOK(void) + { + // For unittest, we enable parallel mode by default + return true; + } +} + +// Stub GUC variables for parallel execution +int max_parallel_workers_per_gather = 2; +bool enable_parallel = true; +double parallel_setup_cost = 1000.0; //--------------------------------------------------------------------------- // @function: diff --git a/src/backend/gporca/server/src/unittest/CTestUtils.cpp b/src/backend/gporca/server/src/unittest/CTestUtils.cpp index bca99766fda..5ce83305b32 100644 --- a/src/backend/gporca/server/src/unittest/CTestUtils.cpp +++ b/src/backend/gporca/server/src/unittest/CTestUtils.cpp @@ -3539,7 +3539,7 @@ CTestUtils::EresSamplePlans(const CHAR *rgszFileNames[], ULONG ulTests, CStatisticsConfig::PstatsconfDefault(mp), CCTEConfig::PcteconfDefault(mp), ICostModel::PcmDefault(mp), CHint::PhintDefault(mp), nullptr, - CWindowOids::GetWindowOids(mp)); + CWindowOids::GetWindowOids(mp), false); } else { @@ -3678,7 +3678,7 @@ CTestUtils::EresCheckPlans(const CHAR *rgszFileNames[], ULONG ulTests, CStatisticsConfig::PstatsconfDefault(mp), CCTEConfig::PcteconfDefault(mp), ICostModel::PcmDefault(mp), CHint::PhintDefault(mp), nullptr, - CWindowOids::GetWindowOids(mp)); + CWindowOids::GetWindowOids(mp), false); } else { diff --git a/src/backend/gporca/server/src/unittest/dxl/statistics/CCardinalityTestUtils.cpp b/src/backend/gporca/server/src/unittest/dxl/statistics/CCardinalityTestUtils.cpp index f95ecb14ee4..24da702044c 100644 --- a/src/backend/gporca/server/src/unittest/dxl/statistics/CCardinalityTestUtils.cpp +++ b/src/backend/gporca/server/src/unittest/dxl/statistics/CCardinalityTestUtils.cpp @@ -112,7 +112,8 @@ CCardinalityTestUtils::PhistInt4Remain(CMemoryPool *mp, ULONG num_of_buckets, } return GPOS_NEW(mp) CHistogram(mp, histogram_buckets, true, null_freq, - num_NDV_remain, freq_remaining); + num_NDV_remain, freq_remaining, + num_NDV_remain /* distinct_by_segs */); } // helper function to generate an example int histogram diff --git a/src/backend/gporca/server/src/unittest/dxl/statistics/CHistogramTest.cpp b/src/backend/gporca/server/src/unittest/dxl/statistics/CHistogramTest.cpp index b6eb14a8378..900e16a3f7d 100644 --- a/src/backend/gporca/server/src/unittest/dxl/statistics/CHistogramTest.cpp +++ b/src/backend/gporca/server/src/unittest/dxl/statistics/CHistogramTest.cpp @@ -268,7 +268,8 @@ CHistogramTest::PhistExampleInt4Remain(CMemoryPool *mp) return GPOS_NEW(mp) CHistogram(mp, histogram_buckets, true, 0.1 /*null_freq*/, - 2.0 /*distinct_remaining*/, 0.4 /*freq_remaining*/); + 2.0 /*distinct_remaining*/, 0.4 /*freq_remaining*/, + 2.0 /*distinct_by_segs*/); } // basis skew test diff --git a/src/backend/gporca/server/src/unittest/dxl/statistics/CStatisticsTest.cpp b/src/backend/gporca/server/src/unittest/dxl/statistics/CStatisticsTest.cpp index e596add7f0d..6b0345b98e6 100644 --- a/src/backend/gporca/server/src/unittest/dxl/statistics/CStatisticsTest.cpp +++ b/src/backend/gporca/server/src/unittest/dxl/statistics/CStatisticsTest.cpp @@ -504,7 +504,7 @@ CStatisticsTest::EresUnittest_CStatisticsBasic() ULongPtrArray *aggs = GPOS_NEW(mp) ULongPtrArray(mp); CStatistics *pstats4 = CGroupByStatsProcessor::CalcGroupByStats( - mp, stats, GCs, aggs, nullptr /*keys*/); + mp, stats, GCs, aggs, nullptr /*keys*/, false /*is_partial*/); GPOS_TRACE(GPOS_WSZ_LIT("pstats4 = stats group by")); CCardinalityTestUtils::PrintStats(mp, pstats4); diff --git a/src/backend/gporca/server/src/unittest/gpopt/minidump/CICGTest.cpp b/src/backend/gporca/server/src/unittest/gpopt/minidump/CICGTest.cpp index 58d3971cfcf..facd4126ea2 100644 --- a/src/backend/gporca/server/src/unittest/gpopt/minidump/CICGTest.cpp +++ b/src/backend/gporca/server/src/unittest/gpopt/minidump/CICGTest.cpp @@ -303,7 +303,7 @@ CICGTest::EresUnittest_NegativeIndexApplyTests() CEnumeratorConfig::GetEnumeratorCfg(mp, 0 /*plan_id*/), CStatisticsConfig::PstatsconfDefault(mp), CCTEConfig::PcteconfDefault(mp), pcm, CHint::PhintDefault(mp), - nullptr /* pplanhint */, CWindowOids::GetWindowOids(mp)); + nullptr /* pplanhint */, CWindowOids::GetWindowOids(mp), false); CDXLNode *pdxlnPlan = CMinidumperUtils::PdxlnExecuteMinidump( mp, rgszNegativeIndexApplyFileNames[ul], GPOPT_TEST_SEGMENTS /*ulSegments*/, 1 /*ulSessionId*/, diff --git a/src/backend/gporca/server/src/unittest/gpopt/minidump/CMiniDumperDXLTest.cpp b/src/backend/gporca/server/src/unittest/gpopt/minidump/CMiniDumperDXLTest.cpp index f046a185f29..b088aa5f6c5 100644 --- a/src/backend/gporca/server/src/unittest/gpopt/minidump/CMiniDumperDXLTest.cpp +++ b/src/backend/gporca/server/src/unittest/gpopt/minidump/CMiniDumperDXLTest.cpp @@ -127,7 +127,7 @@ CMiniDumperDXLTest::EresUnittest_Basic() CStatisticsConfig::PstatsconfDefault(mp), CCTEConfig::PcteconfDefault(mp), ICostModel::PcmDefault(mp), CHint::PhintDefault(mp), nullptr /* pplanhint */, - CWindowOids::GetWindowOids(mp)); + CWindowOids::GetWindowOids(mp), false); // setup opt ctx CAutoOptCtxt aoc(mp, &mda, nullptr, /* pceeval */ diff --git a/src/backend/gporca/server/src/unittest/gpopt/minidump/CMissingStatsTest.cpp b/src/backend/gporca/server/src/unittest/gpopt/minidump/CMissingStatsTest.cpp index a2d362aa5a2..6dc10137187 100644 --- a/src/backend/gporca/server/src/unittest/gpopt/minidump/CMissingStatsTest.cpp +++ b/src/backend/gporca/server/src/unittest/gpopt/minidump/CMissingStatsTest.cpp @@ -88,7 +88,7 @@ CMissingStatsTest::EresUnittest_RunTests() CEnumeratorConfig::GetEnumeratorCfg(mp, 0 /*plan_id*/), CStatisticsConfig::PstatsconfDefault(mp), CCTEConfig::PcteconfDefault(mp), pcm, CHint::PhintDefault(mp), - nullptr /* pplanhint */, CWindowOids::GetWindowOids(mp)); + nullptr /* pplanhint */, CWindowOids::GetWindowOids(mp), false); SMissingStatsTestCase testCase = rgtc[ul]; CDXLNode *pdxlnPlan = CMinidumperUtils::PdxlnExecuteMinidump( diff --git a/src/backend/gporca/server/src/unittest/gpopt/xforms/CXformTest.cpp b/src/backend/gporca/server/src/unittest/gpopt/xforms/CXformTest.cpp index 910a10ad2ab..ba8e554bf08 100644 --- a/src/backend/gporca/server/src/unittest/gpopt/xforms/CXformTest.cpp +++ b/src/backend/gporca/server/src/unittest/gpopt/xforms/CXformTest.cpp @@ -172,7 +172,7 @@ CXformTest::EresUnittest_ApplyXforms_CTE() pdrgpexpr->Append(pexprConsumer); COptCtxt::PoctxtFromTLS()->Pcteinfo()->IncrementConsumers(ulCTEId); - COptCtxt::PoctxtFromTLS()->Pcteinfo()->AddCTEConsumer(pexprNewConsumer); + COptCtxt::PoctxtFromTLS()->Pcteinfo()->AddCTEConsumer(pexprConsumer); pexprConsumer->AddRef(); CExpression *pexprSelect = diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c index da5acb23ebf..232a71aabf5 100644 --- a/src/backend/optimizer/plan/planner.c +++ b/src/backend/optimizer/plan/planner.c @@ -332,6 +332,27 @@ planner(Query *parse, const char *query_string, int cursorOptions, optimizer_options = palloc(sizeof(OptimizerOptions)); optimizer_options->create_vectorization_plan = false; + + /* + * Set parallel plan creation based on PostgreSQL planner's parallel safety checks. + * This synchronizes with the same conditions used in standard_planner(). + */ + if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 && + IsUnderPostmaster && + parse->commandType == CMD_SELECT && + !parse->hasModifyingCTE && + max_parallel_workers_per_gather > 0 && + !IsParallelWorker()) + { + /* All cheap tests pass, check query tree for parallel safety */ + char maxParallelHazard = max_parallel_hazard(parse); + optimizer_options->create_parallel_plan = (maxParallelHazard != PROPARALLEL_UNSAFE); + } + else + { + /* Skip the query tree scan, assume unsafe */ + optimizer_options->create_parallel_plan = false; + } if (planner_hook) { if (gp_log_optimization_time) diff --git a/src/backend/utils/misc/guc_gp.c b/src/backend/utils/misc/guc_gp.c index d2fa30ea762..22faebd58c5 100644 --- a/src/backend/utils/misc/guc_gp.c +++ b/src/backend/utils/misc/guc_gp.c @@ -331,6 +331,7 @@ bool optimizer_enable_motion_gather; bool optimizer_enable_motion_redistribute; bool optimizer_enable_sort; bool optimizer_enable_materialize; +bool optimizer_enable_parallel_append; bool optimizer_enable_partition_propagation; bool optimizer_enable_partition_selection; bool optimizer_enable_outerjoin_rewrite; @@ -2250,6 +2251,16 @@ struct config_bool ConfigureNamesBool_gp[] = true, NULL, NULL, NULL }, + { + {"optimizer_enable_parallel_append", PGC_USERSET, DEVELOPER_OPTIONS, + gettext_noop("Enable parallel append for seq/bitmap/index scan in partition table."), + NULL, + GUC_NOT_IN_SAMPLE + }, + &optimizer_enable_parallel_append, + false, + NULL, NULL, NULL + }, { {"optimizer_enable_partition_propagation", PGC_USERSET, DEVELOPER_OPTIONS, gettext_noop("Enable plans with Partition Propagation operators in the optimizer."), diff --git a/src/include/gpopt/gpdbwrappers.h b/src/include/gpopt/gpdbwrappers.h index 261cd28b5f0..d59225645f4 100644 --- a/src/include/gpopt/gpdbwrappers.h +++ b/src/include/gpopt/gpdbwrappers.h @@ -717,6 +717,9 @@ List *GetMergeJoinOpFamilies(Oid opno); // get the OID of base elementtype fora given typid Oid GetBaseType(Oid typid); +// check if parallel mode is OK (comprehensive check) +bool IsParallelModeOK(void); + // returns the result of evaluating 'expr' as an Expr. Caller keeps ownership of 'expr' // and takes ownership of the result Expr *EvaluateExpr(Expr *expr, Oid result_type, int32 typmod); @@ -768,6 +771,8 @@ List *GetRelChildIndexes(Oid reloid); Oid GetForeignServerId(Oid reloid); +int16 GetAppendOnlySegmentFilesCount(Relation rel); + void GPDBLockRelationOid(Oid reloid, int lockmode); char *GetRelFdwName(Oid reloid); diff --git a/src/include/gpopt/translate/CTranslatorDXLToPlStmt.h b/src/include/gpopt/translate/CTranslatorDXLToPlStmt.h index 625ed5cd0a3..eb4d11c34a6 100644 --- a/src/include/gpopt/translate/CTranslatorDXLToPlStmt.h +++ b/src/include/gpopt/translate/CTranslatorDXLToPlStmt.h @@ -213,6 +213,13 @@ class CTranslatorDXLToPlStmt ctxt_translation_prev_siblings // translation contexts of previous siblings ); + // translate DXL parallel table scan node into a parallel SeqScan node + Plan *TranslateDXLParallelTblScan( + const CDXLNode *tbl_scan_dxlnode, CDXLTranslateContext *output_context, + CDXLTranslationContextArray * + ctxt_translation_prev_siblings // translation contexts of previous siblings + ); + // translate DXL index scan node into a IndexScan node Plan *TranslateDXLIndexScan( const CDXLNode *index_scan_dxlnode, @@ -354,6 +361,12 @@ class CTranslatorDXLToPlStmt ctxt_translation_prev_siblings // translation contexts of previous siblings ); + Plan *TranslateDXLParallelAppend( + const CDXLNode *append_dxlnode, CDXLTranslateContext *output_context, + CDXLTranslationContextArray * + ctxt_translation_prev_siblings // translation contexts of previous siblings + ); + Plan *TranslateDXLMaterialize( const CDXLNode *materialize_dxlnode, CDXLTranslateContext *output_context, @@ -375,6 +388,13 @@ class CTranslatorDXLToPlStmt ctxt_translation_prev_siblings // translation contexts of previous siblings ); + // translate a parallel sequence operator + Plan *TranslateDXLParallelSequence( + const CDXLNode *sequence_dxlnode, CDXLTranslateContext *output_context, + CDXLTranslationContextArray * + ctxt_translation_prev_siblings // translation contexts of previous siblings + ); + // translate a dynamic table scan operator Plan *TranslateDXLDynTblScan( const CDXLNode *dyn_tbl_scan_dxlnode, @@ -436,12 +456,28 @@ class CTranslatorDXLToPlStmt ctxt_translation_prev_siblings // translation contexts of previous siblings ); + // translate a CTE producer into a GPDB parallel share input scan + Plan *TranslateDXLParallelCTEProducerToParallelSharedScan( + const CDXLNode *cte_producer_dxlnode, + CDXLTranslateContext *output_context, + CDXLTranslationContextArray * + ctxt_translation_prev_siblings // translation contexts of previous siblings + ); + // translate a CTE consumer into a GPDB share input scan Plan *TranslateDXLCTEConsumerToSharedScan( const CDXLNode *cte_consumer_dxlnode, CDXLTranslateContext *output_context, CDXLTranslationContextArray * - ctxt_translation_prev_siblings // translation contexts of previous siblings + ctxt_translation_prev_siblings // translation contexts of previous siblings + ); + + // translate a CTE consumer into a GPDB parallel share input scan + Plan *TranslateDXLParallelCTEConsumerToParallelSharedScan( + const CDXLNode *cte_consumer_dxlnode, + CDXLTranslateContext *output_context, + CDXLTranslationContextArray * + ctxt_translation_prev_siblings // translation contexts of previous siblings ); // translate a (dynamic) bitmap table scan operator @@ -657,6 +693,10 @@ class CTranslatorDXLToPlStmt // fill the aggno and transno for the aggnode static void TranslateAggFillInfo(CContextDXLToPlStmt *context, Aggref *aggref); + + // extract parallel workers from DXL node tree + static ULONG ExtractParallelWorkersFromDXL(const CDXLNode *dxlnode); + }; } // namespace gpdxl diff --git a/src/include/gpopt/utils/COptTasks.h b/src/include/gpopt/utils/COptTasks.h index 3fa5f91216c..3e0c61aeb59 100644 --- a/src/include/gpopt/utils/COptTasks.h +++ b/src/include/gpopt/utils/COptTasks.h @@ -90,6 +90,9 @@ struct SOptContext // is serializing a plan to DXL required ? BOOL m_should_serialize_plan_dxl{false}; + // should generate parallel plans ? + BOOL m_create_parallel_plan{false}; + // did the optimizer fail unexpectedly? BOOL m_is_unexpected_failure{false}; @@ -129,7 +132,8 @@ class COptTasks // create optimizer configuration object static COptimizerConfig *CreateOptimizerConfig(CMemoryPool *mp, ICostModel *cost_model, - CPlanHint *plan_hints); + CPlanHint *plan_hints, + BOOL enable_parallel_plans = false); // optimize a query to a physical DXL static void *OptimizeTask(void *ptr); diff --git a/src/include/optimizer/orcaopt.h b/src/include/optimizer/orcaopt.h index 4a045b632c0..6c1f3075bc4 100644 --- a/src/include/optimizer/orcaopt.h +++ b/src/include/optimizer/orcaopt.h @@ -33,6 +33,7 @@ typedef struct OptimizerOptions { bool create_vectorization_plan; + bool create_parallel_plan; } OptimizerOptions; diff --git a/src/include/utils/guc.h b/src/include/utils/guc.h index 4b62b663f0b..6d2700899af 100644 --- a/src/include/utils/guc.h +++ b/src/include/utils/guc.h @@ -514,6 +514,7 @@ extern bool optimizer_enable_motion_gather; extern bool optimizer_enable_motion_redistribute; extern bool optimizer_enable_sort; extern bool optimizer_enable_materialize; +extern bool optimizer_enable_parallel_append; extern bool optimizer_enable_partition_propagation; extern bool optimizer_enable_partition_selection; extern bool optimizer_enable_outerjoin_rewrite; diff --git a/src/include/utils/unsync_guc_name.h b/src/include/utils/unsync_guc_name.h index 4c956e14fbf..7f51230c865 100644 --- a/src/include/utils/unsync_guc_name.h +++ b/src/include/utils/unsync_guc_name.h @@ -427,6 +427,7 @@ "optimizer_enable_nljoin", "optimizer_enable_outerjoin_rewrite", "optimizer_enable_outerjoin_to_unionall_rewrite", + "optimizer_enable_parallel_append", "optimizer_enable_partition_propagation", "optimizer_enable_partition_selection", "optimizer_enable_push_join_below_union_all", diff --git a/src/test/regress/GNUmakefile b/src/test/regress/GNUmakefile index 9bc8b67591e..60f8d491efb 100644 --- a/src/test/regress/GNUmakefile +++ b/src/test/regress/GNUmakefile @@ -223,6 +223,12 @@ installcheck-cbdb-parallel: all tablespace-setup twophase_pqexecparams $(pg_regress_installcheck) $(REGRESS_OPTS) --schedule=$(srcdir)/parallel_schedule --schedule=$(srcdir)/greenplum_schedule --max-connections=5 $(EXTRA_TESTS) --exclude-tests explain \ ) +installcheck-orca-parallel: all tablespace-setup twophase_pqexecparams + ( \ + export PGOPTIONS='-c optimizer=on -c enable_parallel=true -c min_parallel_table_scan_size=0 -c min_parallel_index_scan_size=0 -c parallel_setup_cost=0'; \ + $(pg_regress_installcheck) $(REGRESS_OPTS) --exclude-file=$(srcdir)/excluded_tests.conf --schedule=$(srcdir)/parallel_schedule --schedule=$(srcdir)/greenplum_schedule --max-connections=4 --ignore-plans $(EXTRA_TESTS) \ + ) + installcheck-tests: all $(pg_regress_installcheck) $(REGRESS_OPTS) $(TESTS) $(EXTRA_TESTS) diff --git a/src/test/regress/excluded_tests.conf b/src/test/regress/excluded_tests.conf new file mode 100644 index 00000000000..e29197946d4 --- /dev/null +++ b/src/test/regress/excluded_tests.conf @@ -0,0 +1,14 @@ +brin_bloom +join_hash +explain +memoize +cte_prune +tuplesort +gporca +brin_ao +brin_aocs +direct_dispatch +bfv_dd +bfv_dd_multicolumn +planhints +rowhints diff --git a/src/test/regress/expected/brin.out b/src/test/regress/expected/brin.out index 20807f31997..2a0d670300a 100644 --- a/src/test/regress/expected/brin.out +++ b/src/test/regress/expected/brin.out @@ -323,6 +323,7 @@ BEGIN SET enable_bitmapscan = 1; SET optimizer_enable_tablescan = 0; SET optimizer_enable_bitmapscan = 1; + SET enable_parallel = 0; plan_ok := false; is_planner_plan := false; diff --git a/src/test/regress/expected/brin_bloom.out b/src/test/regress/expected/brin_bloom.out index 241847a2707..e636c9ba4ae 100644 --- a/src/test/regress/expected/brin_bloom.out +++ b/src/test/regress/expected/brin_bloom.out @@ -230,6 +230,7 @@ BEGIN -- run the query using the brin index SET enable_seqscan = 0; SET enable_bitmapscan = 1; + SET enable_parallel = 0; plan_ok := false; FOR plan_line IN EXECUTE format($y$EXPLAIN SELECT array_agg(ctid) FROM brintest_bloom WHERE %s $y$, cond) LOOP diff --git a/src/test/regress/expected/brin_bloom_optimizer.out b/src/test/regress/expected/brin_bloom_optimizer.out index 65622f51b4b..2fb93d2819f 100644 --- a/src/test/regress/expected/brin_bloom_optimizer.out +++ b/src/test/regress/expected/brin_bloom_optimizer.out @@ -230,6 +230,7 @@ BEGIN -- run the query using the brin index SET enable_seqscan = 0; SET enable_bitmapscan = 1; + SET enable_parallel = 0; plan_ok := false; FOR plan_line IN EXECUTE format($y$EXPLAIN SELECT array_agg(ctid) FROM brintest_bloom WHERE %s $y$, cond) LOOP diff --git a/src/test/regress/expected/brin_multi.out b/src/test/regress/expected/brin_multi.out index 51277fdb887..037e331f8ba 100644 --- a/src/test/regress/expected/brin_multi.out +++ b/src/test/regress/expected/brin_multi.out @@ -260,6 +260,7 @@ BEGIN -- run the query using the brin index SET enable_seqscan = 0; SET enable_bitmapscan = 1; + SET enable_parallel = 0; plan_ok := false; FOR plan_line IN EXECUTE format($y$EXPLAIN SELECT array_agg(ctid) FROM brintest_multi WHERE %s $y$, cond) LOOP diff --git a/src/test/regress/expected/brin_multi_optimizer.out b/src/test/regress/expected/brin_multi_optimizer.out index 39100bfeb80..ff372c1c66e 100644 --- a/src/test/regress/expected/brin_multi_optimizer.out +++ b/src/test/regress/expected/brin_multi_optimizer.out @@ -260,6 +260,7 @@ BEGIN -- run the query using the brin index SET enable_seqscan = 0; SET enable_bitmapscan = 1; + SET enable_parallel = 0; plan_ok := false; FOR plan_line IN EXECUTE format($y$EXPLAIN SELECT array_agg(ctid) FROM brintest_multi WHERE %s $y$, cond) LOOP diff --git a/src/test/regress/expected/brin_multi_optimizer_1.out b/src/test/regress/expected/brin_multi_optimizer_1.out index d995e7888d2..da6ed1d404a 100644 --- a/src/test/regress/expected/brin_multi_optimizer_1.out +++ b/src/test/regress/expected/brin_multi_optimizer_1.out @@ -260,6 +260,7 @@ BEGIN -- run the query using the brin index SET enable_seqscan = 0; SET enable_bitmapscan = 1; + SET enable_parallel = 0; plan_ok := false; FOR plan_line IN EXECUTE format($y$EXPLAIN SELECT array_agg(ctid) FROM brintest_multi WHERE %s $y$, cond) LOOP diff --git a/src/test/regress/expected/brin_optimizer.out b/src/test/regress/expected/brin_optimizer.out index 37d3d21fb10..b749771b08e 100644 --- a/src/test/regress/expected/brin_optimizer.out +++ b/src/test/regress/expected/brin_optimizer.out @@ -323,6 +323,7 @@ BEGIN SET enable_bitmapscan = 1; SET optimizer_enable_tablescan = 0; SET optimizer_enable_bitmapscan = 1; + SET enable_parallel = 0; plan_ok := false; is_planner_plan := false; diff --git a/src/test/regress/expected/create_index.out b/src/test/regress/expected/create_index.out index f795078e761..bab603be36b 100644 --- a/src/test/regress/expected/create_index.out +++ b/src/test/regress/expected/create_index.out @@ -1705,7 +1705,7 @@ DROP TABLE syscol_table; -- -- Tests for IS NULL/IS NOT NULL with b-tree indexes -- -CREATE TABLE onek_with_null AS SELECT unique1, unique2 FROM onek; +CREATE TABLE onek_with_null AS SELECT unique1, unique2 FROM onek DISTRIBUTED BY (unique1); INSERT INTO onek_with_null (unique1,unique2) VALUES (NULL, -1), (NULL, NULL); CREATE UNIQUE INDEX onek_nulltest ON onek_with_null (unique2,unique1); SET enable_seqscan = OFF; diff --git a/src/test/regress/expected/create_index_optimizer.out b/src/test/regress/expected/create_index_optimizer.out index f661e53f487..3f3b41aea64 100644 --- a/src/test/regress/expected/create_index_optimizer.out +++ b/src/test/regress/expected/create_index_optimizer.out @@ -1734,7 +1734,7 @@ DROP TABLE syscol_table; -- -- Tests for IS NULL/IS NOT NULL with b-tree indexes -- -CREATE TABLE onek_with_null AS SELECT unique1, unique2 FROM onek; +CREATE TABLE onek_with_null AS SELECT unique1, unique2 FROM onek DISTRIBUTED BY (unique1); INSERT INTO onek_with_null (unique1,unique2) VALUES (NULL, -1), (NULL, NULL); CREATE UNIQUE INDEX onek_nulltest ON onek_with_null (unique2,unique1); SET enable_seqscan = OFF; diff --git a/src/test/regress/expected/rpt_optimizer.out b/src/test/regress/expected/rpt_optimizer.out index 24ee61ff642..a07e2c8f8dc 100644 --- a/src/test/regress/expected/rpt_optimizer.out +++ b/src/test/regress/expected/rpt_optimizer.out @@ -933,16 +933,17 @@ explain (costs off) select a from t_replicate_volatile union all select * from n -- CTAS explain (costs off) create table rpt_ctas as select random() from generate_series(1, 10) distributed replicated; -QUERY PLAN -___________ + QUERY PLAN +---------------------------------------------- Result -> Broadcast Motion 1:3 (slice1) -> Function Scan on generate_series -GP_IGNORE:(4 rows) + Optimizer: GPORCA +(4 rows) explain (costs off) create table rpt_ctas as select a from generate_series(1, 10) a group by a having sum(a) > random() distributed replicated; -QUERY PLAN -___________ + QUERY PLAN +------------------------------------------------------------------------------- Result -> Broadcast Motion 3:3 (slice1; segments: 3) -> Result @@ -951,8 +952,8 @@ ___________ Group Key: generate_series -> Result -> Function Scan on generate_series - Optimizer: Pivotal Optimizer (GPORCA) -GP_IGNORE:(9 rows) + Optimizer: GPORCA +(9 rows) -- update & delete explain (costs off) update t_replicate_volatile set a = 1 where b > random(); diff --git a/src/test/regress/expected/tuplesort.out b/src/test/regress/expected/tuplesort.out index 47b5a8ddb21..e6cd26efdb1 100644 --- a/src/test/regress/expected/tuplesort.out +++ b/src/test/regress/expected/tuplesort.out @@ -236,7 +236,7 @@ CLUSTER abbrev_abort_uuids USING abbrev_abort_uuids__abort_increasing_idx; -- head SELECT id, abort_increasing, abort_decreasing, noabort_increasing, noabort_decreasing FROM abbrev_abort_uuids -ORDER BY ctid LIMIT 5; +ORDER BY ctid, id LIMIT 5; id | abort_increasing | abort_decreasing | noabort_increasing | noabort_decreasing ----+--------------------------------------+--------------------------------------+--------------------------------------+-------------------------------------- 2 | 00000000-0000-0000-0000-000000000001 | 00000000-0000-0000-0000-000000019999 | 00000001-0000-0000-0000-000000000001 | 00009990-0000-0000-0000-000000019999 diff --git a/src/test/regress/expected/tuplesort_optimizer.out b/src/test/regress/expected/tuplesort_optimizer.out index d62e0a0e71b..4eaabea61b5 100644 --- a/src/test/regress/expected/tuplesort_optimizer.out +++ b/src/test/regress/expected/tuplesort_optimizer.out @@ -232,7 +232,7 @@ CLUSTER abbrev_abort_uuids USING abbrev_abort_uuids__abort_increasing_idx; -- head SELECT id, abort_increasing, abort_decreasing, noabort_increasing, noabort_decreasing FROM abbrev_abort_uuids -ORDER BY ctid LIMIT 5; +ORDER BY ctid, id LIMIT 5; id | abort_increasing | abort_decreasing | noabort_increasing | noabort_decreasing ----+--------------------------------------+--------------------------------------+--------------------------------------+-------------------------------------- 2 | 00000000-0000-0000-0000-000000000001 | 00000000-0000-0000-0000-000000019999 | 00000001-0000-0000-0000-000000000001 | 00009990-0000-0000-0000-000000019999 diff --git a/src/test/regress/expected/workfile/hashjoin_spill.out b/src/test/regress/expected/workfile/hashjoin_spill.out index 5633e670954..053f457294b 100644 --- a/src/test/regress/expected/workfile/hashjoin_spill.out +++ b/src/test/regress/expected/workfile/hashjoin_spill.out @@ -37,6 +37,8 @@ HINT: The 'DISTRIBUTED BY' clause determines the distribution of data. Make sur insert into test_hj_spill SELECT i,i,i%1000,i,i,i,i,i from (select generate_series(1, nsegments * 15000) as i from (select count(*) as nsegments from gp_segment_configuration where role='p' and content >= 0) foo) bar; +-- Collect statistics to ensure ORCA generates correct execution plan +ANALYZE test_hj_spill; SET statement_mem=1024; set gp_resqueue_print_operator_memory_limits=on; set gp_workfile_compression = on; diff --git a/src/test/regress/sql/brin.sql b/src/test/regress/sql/brin.sql index 9a446efdc82..fade709eaa4 100644 --- a/src/test/regress/sql/brin.sql +++ b/src/test/regress/sql/brin.sql @@ -330,6 +330,7 @@ BEGIN SET enable_bitmapscan = 1; SET optimizer_enable_tablescan = 0; SET optimizer_enable_bitmapscan = 1; + SET enable_parallel = 0; plan_ok := false; is_planner_plan := false; diff --git a/src/test/regress/sql/brin_bloom.sql b/src/test/regress/sql/brin_bloom.sql index 84572fdc2fb..558ca5730b7 100644 --- a/src/test/regress/sql/brin_bloom.sql +++ b/src/test/regress/sql/brin_bloom.sql @@ -231,6 +231,7 @@ BEGIN -- run the query using the brin index SET enable_seqscan = 0; SET enable_bitmapscan = 1; + SET enable_parallel = 0; plan_ok := false; FOR plan_line IN EXECUTE format($y$EXPLAIN SELECT array_agg(ctid) FROM brintest_bloom WHERE %s $y$, cond) LOOP diff --git a/src/test/regress/sql/brin_multi.sql b/src/test/regress/sql/brin_multi.sql index 9deb8d2573d..66028aaf1d5 100644 --- a/src/test/regress/sql/brin_multi.sql +++ b/src/test/regress/sql/brin_multi.sql @@ -266,6 +266,7 @@ BEGIN -- run the query using the brin index SET enable_seqscan = 0; SET enable_bitmapscan = 1; + SET enable_parallel = 0; plan_ok := false; FOR plan_line IN EXECUTE format($y$EXPLAIN SELECT array_agg(ctid) FROM brintest_multi WHERE %s $y$, cond) LOOP diff --git a/src/test/regress/sql/create_index.sql b/src/test/regress/sql/create_index.sql index 1b598d13b27..746e1a267b2 100644 --- a/src/test/regress/sql/create_index.sql +++ b/src/test/regress/sql/create_index.sql @@ -649,7 +649,7 @@ DROP TABLE syscol_table; -- Tests for IS NULL/IS NOT NULL with b-tree indexes -- -CREATE TABLE onek_with_null AS SELECT unique1, unique2 FROM onek; +CREATE TABLE onek_with_null AS SELECT unique1, unique2 FROM onek DISTRIBUTED BY (unique1); INSERT INTO onek_with_null (unique1,unique2) VALUES (NULL, -1), (NULL, NULL); CREATE UNIQUE INDEX onek_nulltest ON onek_with_null (unique2,unique1); diff --git a/src/test/regress/sql/tuplesort.sql b/src/test/regress/sql/tuplesort.sql index 45c7c027cd8..fa5f87c7ec9 100644 --- a/src/test/regress/sql/tuplesort.sql +++ b/src/test/regress/sql/tuplesort.sql @@ -95,7 +95,7 @@ CLUSTER abbrev_abort_uuids USING abbrev_abort_uuids__abort_increasing_idx; -- head SELECT id, abort_increasing, abort_decreasing, noabort_increasing, noabort_decreasing FROM abbrev_abort_uuids -ORDER BY ctid LIMIT 5; +ORDER BY ctid, id LIMIT 5; -- tail SELECT id, abort_increasing, abort_decreasing, noabort_increasing, noabort_decreasing diff --git a/src/test/regress/sql/workfile/hashjoin_spill.sql b/src/test/regress/sql/workfile/hashjoin_spill.sql index 08e30501ef2..7166842dae5 100644 --- a/src/test/regress/sql/workfile/hashjoin_spill.sql +++ b/src/test/regress/sql/workfile/hashjoin_spill.sql @@ -40,6 +40,10 @@ CREATE TABLE test_hj_spill (i1 int, i2 int, i3 int, i4 int, i5 int, i6 int, i7 i insert into test_hj_spill SELECT i,i,i%1000,i,i,i,i,i from (select generate_series(1, nsegments * 15000) as i from (select count(*) as nsegments from gp_segment_configuration where role='p' and content >= 0) foo) bar; + +-- Collect statistics to ensure ORCA generates correct execution plan +ANALYZE test_hj_spill; + SET statement_mem=1024; set gp_resqueue_print_operator_memory_limits=on;