From c1b7d28de9923d693c71de84afd65adb581a85b9 Mon Sep 17 00:00:00 2001 From: Orri Erling Date: Thu, 9 Jan 2025 15:33:59 -0800 Subject: [PATCH] Velox side Verax support (#1) Summary: Pull Request resolved: https://github.com/facebookexperimental/verax/pull/1 X-link: https://github.com/facebookincubator/velox/pull/11931 Reviewed By: pedroerp Differential Revision: D67571079 Pulled By: oerling fbshipit-source-id: f817e19d98ab80a64fbd3d26d33edd3027003a93 --- verax/ArenaCache.h | 67 + verax/CMakeLists.txt | 38 + verax/Cost.cpp | 182 ++ verax/Cost.h | 74 + verax/Filters.cpp | 29 + verax/Filters.h | 37 + verax/Plan.cpp | 1467 +++++++++++++++++ verax/Plan.h | 782 +++++++++ verax/PlanObject.cpp | 164 ++ verax/PlanObject.h | 244 +++ verax/PlanUtils.cpp | 62 + verax/PlanUtils.h | 121 ++ verax/QueryGraph.cpp | 1072 ++++++++++++ verax/QueryGraph.h | 755 +++++++++ verax/QueryGraphContext.cpp | 105 ++ verax/QueryGraphContext.h | 241 +++ verax/RelationOp.cpp | 266 +++ verax/RelationOp.h | 410 +++++ verax/Schema.cpp | 381 +++++ verax/Schema.h | 462 ++++++ verax/SchemaResolver.cpp | 50 + verax/SchemaResolver.h | 45 + verax/ToGraph.cpp | 599 +++++++ verax/ToVelox.cpp | 620 +++++++ verax/VeloxHistory.cpp | 83 + verax/VeloxHistory.h | 48 + verax/connectors/CMakeLists.txt | 36 + verax/connectors/ConnectorMetadata.cpp | 23 + verax/connectors/ConnectorMetadata.h | 502 ++++++ verax/connectors/ConnectorSplitSource.cpp | 45 + verax/connectors/ConnectorSplitSource.h | 43 + verax/connectors/hive/CMakeLists.txt | 42 + .../connectors/hive/HiveConnectorMetadata.cpp | 114 ++ verax/connectors/hive/HiveConnectorMetadata.h | 120 ++ .../hive/LocalHiveConnectorMetadata.cpp | 577 +++++++ .../hive/LocalHiveConnectorMetadata.h | 241 +++ verax/connectors/hive/tests/CMakeLists.txt | 30 + .../hive/tests/HiveConnectorMetadataTest.cpp | 97 ++ verax/tests/CMakeLists.txt | 43 + verax/tests/ParquetTpchTest.cpp | 162 ++ verax/tests/ParquetTpchTest.h | 69 + verax/tests/PlanTest.cpp | 185 +++ verax/tests/Tpch.cpp | 143 ++ verax/tests/Tpch.h | 26 + verax/tests/VeloxSql.cpp | 775 +++++++++ 45 files changed, 11677 insertions(+) create mode 100644 verax/ArenaCache.h create mode 100644 verax/CMakeLists.txt create mode 100644 verax/Cost.cpp create mode 100644 verax/Cost.h create mode 100644 verax/Filters.cpp create mode 100644 verax/Filters.h create mode 100644 verax/Plan.cpp create mode 100644 verax/Plan.h create mode 100644 verax/PlanObject.cpp create mode 100644 verax/PlanObject.h create mode 100644 verax/PlanUtils.cpp create mode 100644 verax/PlanUtils.h create mode 100644 verax/QueryGraph.cpp create mode 100644 verax/QueryGraph.h create mode 100644 verax/QueryGraphContext.cpp create mode 100644 verax/QueryGraphContext.h create mode 100644 verax/RelationOp.cpp create mode 100644 verax/RelationOp.h create mode 100644 verax/Schema.cpp create mode 100644 verax/Schema.h create mode 100644 verax/SchemaResolver.cpp create mode 100644 verax/SchemaResolver.h create mode 100644 verax/ToGraph.cpp create mode 100644 verax/ToVelox.cpp create mode 100644 verax/VeloxHistory.cpp create mode 100644 verax/VeloxHistory.h create mode 100644 verax/connectors/CMakeLists.txt create mode 100644 verax/connectors/ConnectorMetadata.cpp create mode 100644 verax/connectors/ConnectorMetadata.h create mode 100644 verax/connectors/ConnectorSplitSource.cpp create mode 100644 verax/connectors/ConnectorSplitSource.h create mode 100644 verax/connectors/hive/CMakeLists.txt create mode 100644 verax/connectors/hive/HiveConnectorMetadata.cpp create mode 100644 verax/connectors/hive/HiveConnectorMetadata.h create mode 100644 verax/connectors/hive/LocalHiveConnectorMetadata.cpp create mode 100644 verax/connectors/hive/LocalHiveConnectorMetadata.h create mode 100644 verax/connectors/hive/tests/CMakeLists.txt create mode 100644 verax/connectors/hive/tests/HiveConnectorMetadataTest.cpp create mode 100644 verax/tests/CMakeLists.txt create mode 100644 verax/tests/ParquetTpchTest.cpp create mode 100644 verax/tests/ParquetTpchTest.h create mode 100644 verax/tests/PlanTest.cpp create mode 100644 verax/tests/Tpch.cpp create mode 100644 verax/tests/Tpch.h create mode 100644 verax/tests/VeloxSql.cpp diff --git a/verax/ArenaCache.h b/verax/ArenaCache.h new file mode 100644 index 0000000..1407944 --- /dev/null +++ b/verax/ArenaCache.h @@ -0,0 +1,67 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include +#include "velox/common/base/BitUtil.h" +#include "velox/common/memory/HashStringAllocator.h" + +namespace facebook::velox::optimizer { + +/// Single threaded cache in front of HashStringAllocator free +/// list. Speeds up allocation and fre of plan candidates. This is +/// about 2x faster than uncached HashStringAllocator. Overall amounts +/// to ~2% of optimization time. +class ArenaCache { + static constexpr int32_t kMaxSize = 512; + static constexpr int32_t kGranularity = 16; + + public: + explicit ArenaCache(velox::HashStringAllocator& allocator) + : allocator_(allocator), allocated_(kMaxSize / kGranularity) {} + + void* allocate(size_t size) { + auto sizeClass = velox::bits::roundUp(size, kGranularity) / kGranularity; + if (sizeClass < kMaxSize / kGranularity) { + if (!allocated_[sizeClass].empty()) { + void* result = allocated_[sizeClass].back(); + allocated_[sizeClass].pop_back(); + totalSize_ -= sizeClass; + return result; + } + } + return allocator_.allocate(sizeClass * kGranularity)->begin(); + } + + void free(void* ptr) { + auto header = velox::HashStringAllocator::headerOf(ptr); + int32_t sizeClass = header->size() / kGranularity; + if (sizeClass < kMaxSize / kGranularity) { + totalSize_ += sizeClass; + allocated_[sizeClass].push_back(ptr); + return; + } + allocator_.free(header); + } + + private: + velox::HashStringAllocator& allocator_; + std::vector> allocated_; + uint64_t totalSize_{0}; +}; + +} // namespace facebook::velox::optimizer diff --git a/verax/CMakeLists.txt b/verax/CMakeLists.txt new file mode 100644 index 0000000..884f2b2 --- /dev/null +++ b/verax/CMakeLists.txt @@ -0,0 +1,38 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed 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. + +add_subdirectory(tests) + +add_subdirectory(connectors) + +add_library( + velox_verax + ToGraph.cpp + Plan.cpp + PlanObject.cpp + Schema.cpp + SchemaResolver.cpp + QueryGraph.cpp + QueryGraphContext.cpp + Filters.cpp + Cost.cpp + PlanUtils.cpp + RelationOp.cpp + ToVelox.cpp + VeloxHistory.cpp) + +add_dependencies(velox_verax velox_hive_connector) + +target_link_libraries(velox_verax velox_core velox_connector_metadata + velox_multifragment_plan velox_connector) diff --git a/verax/Cost.cpp b/verax/Cost.cpp new file mode 100644 index 0000000..b92cfa6 --- /dev/null +++ b/verax/Cost.cpp @@ -0,0 +1,182 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/Cost.h" //@manual +#include "optimizer/Plan.h" //@manual +#include "optimizer/PlanUtils.h" //@manual + +namespace facebook::velox::optimizer { + +using namespace facebook::velox; + +// Collection of per operation costs for a target system. The base +// unit is the time to memcpy a cache line in a large memcpy on one +// core. This is ~6GB/s, so ~10ns. Other times are expressed as +// multiples of that. +struct Costs { + static float byteShuffleCost() { + return 12; // ~500MB/s + } + + static float hashProbeCost(float cardinality) { + return cardinality < 10000 ? kArrayProbeCost + : cardinality < 500000 ? kSmallHashCost + : kLargeHashCost; + } + + static constexpr float kKeyCompareCost = + 6; // ~30 instructions to find, decode and an compare + static constexpr float kArrayProbeCost = 2; // ~10 instructions. + static constexpr float kSmallHashCost = 10; // 50 instructions + static constexpr float kLargeHashCost = 40; // 2 LLC misses + static constexpr float kColumnRowCost = 5; + static constexpr float kColumnByteCost = 0.1; + + // Cost of hash function on one column. + static constexpr float kHashColumnCost = 0.5; + + // Cost of getting a column from a hash table + static constexpr float kHashExtractColumnCost = 0.5; + + // Minimal cost of calling a filter function, e.g. comparing two numeric + // exprss. + static constexpr float kMinimumFilterCost = 2; +}; + +void RelationOp::setCost(const PlanState& state) { + cost_.inputCardinality = state.cost.fanout; +} + +float ColumnGroup::lookupCost(float range) const { + // Add 2 because it takes a compare and access also if hitting the + // same row. log(1) == 0, so this would other wise be zero cost. + return Costs::kKeyCompareCost * log(range + 2) / log(2); +} + +float orderPrefixDistance( + RelationOpPtr input, + ColumnGroupP index, + const ExprVector& keys) { + int32_t i = 0; + float selection = 1; + for (; i < input->distribution().order.size() && + i < index->distribution().order.size() && i < keys.size(); + ++i) { + if (input->distribution().order[i]->sameOrEqual(*keys[i])) { + selection *= index->distribution().order[i]->value().cardinality; + } + } + return selection; +} + +void TableScan::setCost(const PlanState& input) { + RelationOp::setCost(input); + float size = byteSize(columns_); + if (!keys.empty()) { + float lookupRange(index->distribution().cardinality); + float orderSelectivity = orderPrefixDistance(this->input(), index, keys); + auto distance = lookupRange / std::max(1, orderSelectivity); + float batchSize = std::min(cost_.inputCardinality, 10000); + if (orderSelectivity == 1) { + // The data does not come in key order. + float batchCost = index->lookupCost(lookupRange) + + index->lookupCost(lookupRange / batchSize) * + std::max(1, batchSize); + cost_.unitCost = batchCost / batchSize; + } else { + float batchCost = index->lookupCost(lookupRange) + + index->lookupCost(distance) * std::max(1, batchSize); + cost_.unitCost = batchCost / batchSize; + } + return; + } else { + cost_.fanout = + index->distribution().cardinality * baseTable->filterSelectivity; + } + auto numColumns = columns_.size(); + auto rowCost = numColumns * Costs::kColumnRowCost + + std::max(0, size - 8 * numColumns) * Costs::kColumnByteCost; + cost_.unitCost += cost_.fanout * rowCost; +} + +void Aggregation::setCost(const PlanState& input) { + RelationOp::setCost(input); + float cardinality = 1; + for (auto key : grouping) { + cardinality *= key->value().cardinality; + } + // The estimated output is input minus the times an input is a + // duplicate of a key already in the input. The cardinality of the + // result is (d - d * 1 - (1 / d))^n. where d is the number of + // potentially distinct keys and n is the number of elements in the + // input. This approaches d as n goes to infinity. The chance of one in d + // being unique after n values is 1 - (1/d)^n. + auto nOut = cardinality - + cardinality * pow(1.0 - (1.0 / cardinality), cost_.inputCardinality); + cost_.fanout = nOut / cost_.inputCardinality; + cost_.unitCost = grouping.size() * Costs::hashProbeCost(nOut); + float rowBytes = byteSize(grouping) + byteSize(aggregates); + cost_.totalBytes = nOut * rowBytes; +} + +template +std::pair shuffleCostV(const V& columns) { + float size = byteSize(columns); + return {size * Costs::byteShuffleCost(), size}; +} + +float shuffleCost(const ColumnVector& columns) { + return shuffleCostV(columns).second; +} + +float shuffleCost(const ExprVector& columns) { + return shuffleCostV(columns).second; +} + +void Repartition::setCost(const PlanState& input) { + RelationOp::setCost(input); + auto pair = shuffleCostV(columns_); + cost_.unitCost = pair.second; + cost_.totalBytes = cost_.inputCardinality * pair.first; +} + +void HashBuild::setCost(const PlanState& input) { + RelationOp::setCost(input); + cost_.unitCost = keys.size() * Costs::kHashColumnCost + + Costs::hashProbeCost(cost_.inputCardinality) + + this->input()->columns().size() * Costs::kHashExtractColumnCost * 2; + cost_.totalBytes = + cost_.inputCardinality * byteSize(this->input()->columns()); +} + +void Join::setCost(const PlanState& input) { + RelationOp::setCost(input); + float buildSize = right->cost().inputCardinality; + auto rowCost = + right->input()->columns().size() * Costs::kHashExtractColumnCost; + cost_.unitCost = Costs::hashProbeCost(buildSize) + cost_.fanout * rowCost + + leftKeys.size() * Costs::kHashColumnCost; +} + +void Filter::setCost(const PlanState& /*input*/) { + cost_.unitCost = Costs::kMinimumFilterCost * exprs_.size(); + // We assume each filter selects 4/5. Small effect makes it so + // join and scan selectivities that are better known have more + // influence on plan cardinality. To be filled in from history. + cost_.fanout = pow(0.8, exprs_.size()); +} + +} // namespace facebook::velox::optimizer diff --git a/verax/Cost.h b/verax/Cost.h new file mode 100644 index 0000000..55eb0c2 --- /dev/null +++ b/verax/Cost.h @@ -0,0 +1,74 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "optimizer/RelationOp.h" //@manual + +namespace facebook::velox::optimizer { + +/// Interface to historical query cost and cardinality +/// information. There is one long lived instance per +/// process. Public functions are thread safe since multiple +/// concurrent Optimizations may access and update the same History. +class History { + public: + virtual ~History() = default; + + /// Returns a historical cost for 'op' or nullopt ir if no data. Matches the + /// full 'op' tree to history. The exactness of the match depends on the + /// History implementation. Implementations may extrapolate from approximate + /// matches. A Cost from this will be used over a non-historical cost if + /// available. Not const since some extrapolated data can be kept. + virtual std::optional findCost(RelationOp& op) = 0; + + /// Records that the cost and cardinality of 'op' was 'cost' as observed from + /// execution. + virtual void recordCost(const RelationOp& op, Cost cost) = 0; + + /// Sets 'filterSelectivity' of 'baseTable' from historical data. Considers + /// filters only and does not return a cost since the cost depends on the + /// columns extracted. This is used first for coming up with join orders. The + /// plan candidates are then made and findCost() is used to access historical + /// cost and plan cardinality. + virtual bool setLeafSelectivity(BaseTable& baseTable) = 0; + + virtual void recordLeafSelectivity( + const std::string& handle, + float selectivity, + bool overwrite = true) { + std::lock_guard l(mutex_); + if (!overwrite && + leafSelectivities_.find(handle) != leafSelectivities_.end()) { + return; + } + leafSelectivities_[handle] = selectivity; + } + + protected: + // serializes access to all data members. + std::mutex mutex_; + + /// Memo for selectivity keyed on ConnectorTableHandle::toString(). + /// Values between 0 and 1. + std::unordered_map leafSelectivities_; +}; + +float shuffleCost(const ColumnVector& columns); + +float shuffleCost(const ExprVector& columns); + +} // namespace facebook::velox::optimizer diff --git a/verax/Filters.cpp b/verax/Filters.cpp new file mode 100644 index 0000000..24b8412 --- /dev/null +++ b/verax/Filters.cpp @@ -0,0 +1,29 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/Plan.h" //@manual +#include "optimizer/PlanUtils.h" //@manual +#include "optimizer/QueryGraph.h" //@manual +#include "velox/common/base/SimdUtil.h" +#include "velox/common/base/SuccinctPrinter.h" + +namespace facebook::velox::optimizer { + +Cost filterCost(CPSpan conjuncts) { + return Cost(); +} + +} // namespace facebook::velox::optimizer diff --git a/verax/Filters.h b/verax/Filters.h new file mode 100644 index 0000000..3391114 --- /dev/null +++ b/verax/Filters.h @@ -0,0 +1,37 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/Plan.h" //@manual +#include "optimizer/PlanUtils.h" //@manual +#include "optimizer/QueryGraph.h" //@manual +#include "velox/common/base/SimdUtil.h" +#include "velox/common/base/SuccinctPrinter.h" + +namespace facebook::velox::optimizer { + +// Returns the cost and cardinality ('unitCost' and 'fanout') for 'conjuncts'. +Cost filterCost(CPSpan conjuncts); + +/// Returns 'conjuncts' wit all items that are common between all disjuncts of +/// each OR are pulled to top level. +ExprVector extractCommonConjuncts(ExprVector conjuncts) {} + +// Extracts an OR that can be resolved for 'table'. This has a result +// if each disjunct of 'or' is an and that specifies some condition +// that can be resolved within 'table'. +disjunctsForTable(ExprPtr or, PlanObjectCP table); + +} // namespace facebook::velox::optimizer diff --git a/verax/Plan.cpp b/verax/Plan.cpp new file mode 100644 index 0000000..aba3129 --- /dev/null +++ b/verax/Plan.cpp @@ -0,0 +1,1467 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/Plan.h" //@manual +#include "optimizer/Cost.h" //@manual +#include "optimizer/PlanUtils.h" //@manual + +#include + +namespace facebook::velox::optimizer { + +using namespace facebook::velox; +using facebook::velox::core::JoinType; + +Optimization::Optimization( + const core::PlanNode& plan, + const Schema& schema, + History& history, + velox::core::ExpressionEvaluator& evaluator, + int32_t traceFlags) + : schema_(schema), + inputPlan_(plan), + history_(history), + evaluator_(evaluator), + traceFlags_(traceFlags) { + queryCtx()->optimization() = this; + root_ = makeQueryGraph(); + root_->distributeConjuncts(); + root_->addImpliedJoins(); + root_->linkTablesToJoins(); + setDerivedTableOutput(root_, inputPlan_); +} + +void Optimization::trace( + int32_t event, + int32_t id, + const Cost& cost, + RelationOp& plan) { + if (event & traceFlags_) { + std::cout << (event == kRetained ? "Retained: " : "Abandoned: ") << id + << ":" << " " << succinctNumber(cost.unitCost + cost.setupCost) + << " " << plan.toString(true, false) << std::endl; + } +} + +PlanPtr Optimization::bestPlan() { + topState_.dt = root_; + PlanObjectSet targetColumns; + targetColumns.unionColumns(root_->columns); + topState_.setTargetColumnsForDt(targetColumns); + makeJoins(nullptr, topState_); + Distribution empty; + bool ignore; + return topState_.plans.best(empty, ignore); +} + +std::unordered_map& baseSelectivities() { + static std::unordered_map map; + return map; +} + +FunctionSet functionBits(Name /*name*/) { + return FunctionSet(0); +} + +Plan::Plan(RelationOpPtr _op, const PlanState& state) + : op(_op), + cost(state.cost), + tables(state.placed), + columns(state.targetColumns), + fullyImported(state.dt->fullyImported) {} + +bool Plan::isStateBetter(const PlanState& state) const { + return cost.unitCost * cost.inputCardinality + cost.setupCost > + state.cost.unitCost * state.cost.inputCardinality + state.cost.setupCost; +} + +std::string Plan::printCost() const { + return cost.toString(true, false); +} + +std::string Plan::toString(bool detail) const { + queryCtx()->contextPlan() = const_cast(this); + auto result = op->toString(true, detail); + queryCtx()->contextPlan() = nullptr; + return result; +} + +void PlanState::addCost(RelationOp& op) { + if (!static_cast(op.cost().unitCost)) { + op.setCost(*this); + } + cost.unitCost += cost.inputCardinality * cost.fanout * op.cost().unitCost; + cost.setupCost += op.cost().setupCost; + cost.fanout *= op.cost().fanout; +} + +void PlanState::addNextJoin( + const JoinCandidate* candidate, + RelationOpPtr plan, + BuildSet builds, + std::vector& toTry) const { + if (!isOverBest()) { + toTry.emplace_back(candidate, plan, cost, placed, columns, builds); + } else { + optimization.trace(Optimization::kExceededBest, dt->id(), cost, *plan); + } +} + +void PlanState::addBuilds(const BuildSet& added) { + for (auto build : added) { + if (std::find(builds.begin(), builds.end(), build) == builds.end()) { + builds.push_back(build); + } + } +} + +void PlanState::setTargetColumnsForDt(const PlanObjectSet& target) { + targetColumns = target; + for (auto i = 0; i < dt->columns.size(); ++i) { + if (target.contains(dt->columns[i])) { + targetColumns.unionColumns(dt->exprs[i]); + } + } + for (auto& having : dt->having) { + targetColumns.unionColumns(having); + } +} + +PlanObjectSet PlanState::downstreamColumns() const { + auto it = downstreamPrecomputed.find(placed); + if (it != downstreamPrecomputed.end()) { + return it->second; + } + PlanObjectSet result; + for (auto join : dt->joins) { + bool addFilter = false; + if (!placed.contains(join->rightTable())) { + addFilter = true; + result.unionColumns(join->leftKeys()); + } + if (join->leftTable() && !placed.contains(join->leftTable())) { + addFilter = true; + result.unionColumns(join->rightKeys()); + } + if (addFilter && !join->filter().empty()) { + result.unionColumns(join->filter()); + } + } + for (auto& filter : dt->conjuncts) { + if (!placed.contains(filter)) { + result.unionColumns(filter); + } + } + for (auto& conjunct : dt->conjuncts) { + if (!placed.contains(conjunct)) { + result.unionColumns(conjunct); + } + } + if (dt->aggregation && !placed.contains(dt->aggregation)) { + auto aggToPlace = dt->aggregation->aggregation; + for (auto i = 0; i < aggToPlace->columns().size(); ++i) { + // Grouping columns must be computed anyway, aggregates only if referenced + // by enclosing. + if (i < aggToPlace->grouping.size()) { + result.unionColumns(aggToPlace->grouping[i]); + } else if (targetColumns.contains(aggToPlace->columns()[i])) { + result.unionColumns( + aggToPlace->aggregates[i - aggToPlace->grouping.size()]); + } + } + } + result.unionSet(targetColumns); + downstreamPrecomputed[placed] = result; + return result; +} + +std::string PlanState::printCost() const { + return cost.toString(true, true); +} + +std::string PlanState::printPlan(RelationOpPtr op, bool detail) const { + auto plan = std::make_unique(op, *this); + return plan->toString(detail); +} + +PlanPtr PlanSet::addPlan(RelationOpPtr plan, PlanState& state) { + bool insert = plans.empty(); + int32_t replaceIndex = -1; + if (!insert) { + // Compare with existing. If there is one with same distribution + // and new is better, replace. If there is one with a different + // distribution and the new one can produce the same distribution + // by repartition, for cheaper, add the new one and delete the old + // one. + for (auto i = 0; i < plans.size(); ++i) { + auto old = plans[i].get(); + if (!(state.input == old->input)) { + continue; + } + if (!old->isStateBetter(state)) { + continue; + } + if (old->op->distribution().isSamePartition(plan->distribution())) { + replaceIndex = i; + continue; + } + } + } + if (insert || replaceIndex != -1) { + auto newPlan = std::make_unique(plan, state); + auto result = newPlan.get(); + if (!bestPlan || + bestPlan->cost.unitCost + bestPlan->cost.setupCost > + result->cost.unitCost + result->cost.setupCost) { + bestPlan = result; + bestCostWithShuffle = result->cost.unitCost + result->cost.setupCost + + shuffleCost(result->op->columns()) * result->cost.fanout; + } + if (replaceIndex >= 0) { + plans[replaceIndex] = std::move(newPlan); + } else { + plans.push_back(std::move(newPlan)); + } + return result; + } + return nullptr; +} + +PlanPtr PlanSet::best(const Distribution& distribution, bool& needsShuffle) { + PlanPtr best = nullptr; + PlanPtr match = nullptr; + float bestCost = -1; + float matchCost = -1; + for (auto i = 0; i < plans.size(); ++i) { + float cost = plans[i]->cost.fanout * plans[i]->cost.unitCost + + plans[i]->cost.setupCost; + if (!best || bestCost > cost) { + best = plans[i].get(); + bestCost = cost; + } + if (plans[i]->op->distribution().isSamePartition(distribution)) { + match = plans[i].get(); + matchCost = cost; + } + } + if (best != match && match) { + float shuffle = shuffleCost(best->op->columns()) * best->cost.fanout; + if (bestCost + shuffle < matchCost) { + needsShuffle = true; + assert(best != nullptr); + return best; + } + } + needsShuffle = best != match; + assert(best != nullptr); + return best; +} + +float startingScore(PlanObjectCP table, DerivedTableP /*dt*/) { + if (table->type() == PlanType::kTable) { + return table->as() + ->schemaTable->columnGroups[0] + ->distribution() + .cardinality; + } + return 10; +} + +const JoinEdgeVector& joinedBy(PlanObjectCP table) { + if (table->type() == PlanType::kTable) { + return table->as()->joinedBy; + } + VELOX_DCHECK(table->type() == PlanType::kDerivedTable); + return table->as()->joinedBy; +} + +// Traverses joins from 'candidate'. Follows any join that goes to a +// table not in 'visited' with a fanout < +// 'maxFanout'. 'fanoutFromRoot' is the product of the fanouts +// between 'candidate' and the 'candidate' of the top level call to +// this. 'path' is the set of joined tables between this invocation +// and the top level. 'fanoutFromRoot' is thus the selectivity of +// the linear join sequence in 'path'. When a reducing join +// sequence is found, the tables on the path are added to +// 'result'. 'reduction' is the product of the fanouts of all the +// reducing join paths added to 'result'. +void reducingJoinsRecursive( + const PlanState& state, + PlanObjectCP candidate, + float fanoutFromRoot, + float maxFanout, + std::vector& path, + PlanObjectSet& visited, + PlanObjectSet& result, + float& reduction, + std::function& path, float reduction)> + resultFunc = nullptr) { + bool isLeaf = true; + for (auto join : joinedBy(candidate)) { + if (join->leftOptional() || join->rightOptional()) { + continue; + } + JoinSide other = join->sideOf(candidate, true); + if (!state.dt->tableSet.contains(other.table) || !state.dt->hasJoin(join)) { + continue; + } + if (other.table->type() != PlanType::kTable) { + continue; + } + if (visited.contains(other.table)) { + continue; + } + if (other.fanout > maxFanout) { + continue; + } + visited.add(other.table); + auto fanout = fanoutFromRoot * other.fanout; + if (fanout < 0.9) { + result.add(other.table); + for (auto step : path) { + result.add(step); + maxFanout = 1; + } + } + path.push_back(other.table); + isLeaf = false; + reducingJoinsRecursive( + state, + other.table, + fanout, + maxFanout, + path, + visited, + result, + reduction, + resultFunc); + path.pop_back(); + } + if (fanoutFromRoot < 1 && isLeaf) { + // We are at the end of a reducing sequence of joins. Update the total + // fanout for the set of all reducing join paths from the top level + // 'candidate'. + reduction *= fanoutFromRoot; + if (resultFunc) { + resultFunc(path, fanoutFromRoot); + } + } +} + +JoinCandidate reducingJoins( + const PlanState& state, + const JoinCandidate& candidate) { + // For an inner join, see if can bundle reducing joins on the build. + JoinCandidate reducing; + reducing.join = candidate.join; + reducing.fanout = candidate.fanout; + PlanObjectSet reducingSet; + if (candidate.join->isInner()) { + PlanObjectSet visited = state.placed; + assert(!candidate.tables.empty()); // lint + visited.add(candidate.tables[0]); + reducingSet.add(candidate.tables[0]); + std::vector path{candidate.tables[0]}; + float reduction = 1; + reducingJoinsRecursive( + state, + candidate.tables[0], + 1, + 1.2, + path, + visited, + reducingSet, + reduction); + if (reduction < 0.9) { + // The only table in 'candidate' must be first in the bushy table list. + reducing.tables = candidate.tables; + reducingSet.forEach([&](auto object) { + if (object != reducing.tables[0]) { + reducing.tables.push_back(object); + } + }); + reducing.fanout = candidate.fanout * reduction; + } + } + if (!state.dt->noImportOfExists) { + PlanObjectSet exists; + float reduction = 1; + assert(!candidate.tables.empty()); + std::vector path{candidate.tables[0]}; + // Look for reducing joins that were not added before, also covering already + // placed tables. This may copy reducing joins from a probe to the + // corresponding build. + reducingSet.add(candidate.tables[0]); + reducingSet.unionSet(state.dt->importedExistences); + reducingJoinsRecursive( + state, + candidate.tables[0], + 1, + 10, + path, + reducingSet, + exists, + reduction, + [&](auto& path, float reduction) { + if (reduction < 0.7) { + // The original table is added to the reducing existences because + // the path starts with it but it is not joined twice since it + // already is the start of the main join. + PlanObjectSet added; + for (auto i = 1; i < path.size(); ++i) { + added.add(path[i]); + } + reducing.existences.push_back(std::move(added)); + } + }); + } + if (reducing.tables.empty() && reducing.existences.empty()) { + // No reduction. + return JoinCandidate{}; + } + if (reducing.tables.empty()) { + // No reducing joins but reducing existences from probe side. + reducing.tables = candidate.tables; + } + return reducing; +} + +// Calls 'func' with join, joined table and fanout for the joinable tables. +template +void forJoinedTables(const PlanState& state, Func func) { + std::unordered_set visited; + state.placed.forEach([&](PlanObjectCP placedTable) { + if (!placedTable->isTable()) { + return; + } + for (auto join : joinedBy(placedTable)) { + if (join->isNonCommutative()) { + if (!visited.insert(join).second) { + continue; + } + bool usable = true; + for (auto key : join->leftKeys()) { + if (!state.placed.isSubset(key->allTables())) { + usable = false; + break; + } + } + if (usable) { + func(join, join->rightTable(), join->lrFanout()); + } + } else { + auto [table, fanout] = join->otherTable(placedTable); + if (!state.dt->tableSet.contains(table)) { + continue; + } + func(join, table, fanout); + } + } + }); +} + +JoinSide JoinCandidate::sideOf(PlanObjectCP side, bool other) const { + return join->sideOf(side, other); +} + +std::string JoinCandidate::toString() const { + std::stringstream out; + out << join->toString() << " fanout " << fanout; + for (auto i = 1; i < tables.size(); ++i) { + out << " + " << tables[i]->toString(); + } + if (!existences.empty()) { + out << " exists " << existences[0].toString(false); + } + return out.str(); +} + +bool NextJoin::isWorse(const NextJoin& other) const { + float shuffle = + plan->distribution().isSamePartition(other.plan->distribution()) + ? 0 + : plan->cost().fanout * shuffleCost(plan->columns()); + return cost.unitCost + cost.setupCost + shuffle > + other.cost.unitCost + other.cost.setupCost; +} + +std::vector Optimization::nextJoins(PlanState& state) { + std::vector candidates; + candidates.reserve(state.dt->tables.size()); + forJoinedTables( + state, [&](JoinEdgeP join, PlanObjectCP joined, float fanout) { + if (!state.placed.contains(joined) && state.dt->hasJoin(join) && + state.dt->hasTable(joined)) { + candidates.emplace_back(join, joined, fanout); + } + }); + + std::vector bushes; + // Take the first hand joined tables and bundle them with reducing joins that + // can go on the build side. + for (auto& candidate : candidates) { + auto bush = reducingJoins(state, candidate); + if (!bush.tables.empty()) { + bushes.push_back(std::move(bush)); + } + } + candidates.insert(candidates.begin(), bushes.begin(), bushes.end()); + std::sort( + candidates.begin(), + candidates.end(), + [](const JoinCandidate& left, const JoinCandidate& right) { + return left.fanout < right.fanout; + }); + if (candidates.empty()) { + // There are no join edges. There could still be cross joins. + state.dt->startTables.forEach([&](PlanObjectCP object) { + if (!state.placed.contains(object)) { + candidates.emplace_back(nullptr, object, tableCardinality(object)); + } + }); + } + return candidates; +} + +size_t MemoKey::hash() const { + size_t hash = tables.hash(); + for (auto& exists : existences) { + hash = bits::commutativeHashMix(hash, exists.hash()); + } + return hash; +} + +bool MemoKey::operator==(const MemoKey& other) const { + if (firstTable == other.firstTable && columns == other.columns && + tables == other.tables) { + if (existences.size() != other.existences.size()) { + return false; + } + for (auto& e : existences) { + bool found = true; + for (auto& e2 : other.existences) { + if (e2 == e) { + found = true; + break; + } + if (!found) { + return false; + } + } + } + return true; + } + return false; +} + +RelationOpPtr repartitionForAgg(const RelationOpPtr& plan, PlanState& state) { + // No shuffle if all grouping keys are in partitioning. + bool shuffle = false; + const ExprVector& keyValues = state.dt->aggregation->aggregation->grouping; + // If no grouping and not yet gathered on a single node, add a gather before + // final agg. + if (keyValues.empty() && !plan->distribution().distributionType.isGather) { + auto* gather = make( + plan, + Distribution::gather(plan->distribution().distributionType), + plan->columns()); + state.addCost(*gather); + return gather; + } + for (auto& key : keyValues) { + auto nthKey = position(plan->distribution().partition, *key); + if (nthKey == kNotFound) { + shuffle = true; + break; + } + } + if (!shuffle) { + return plan; + } + + Distribution distribution( + plan->distribution().distributionType, + plan->resultCardinality(), + keyValues); + auto* repartition = + make(plan, std::move(distribution), plan->columns()); + state.addCost(*repartition); + return repartition; +} + +void Optimization::addPostprocess( + DerivedTableP dt, + RelationOpPtr& plan, + PlanState& state) { + if (dt->aggregation) { + auto* partialAgg = make( + *dt->aggregation->aggregation, + plan, + core::AggregationNode::Step::kPartial); + state.placed.add(dt->aggregation); + state.addCost(*partialAgg); + plan = repartitionForAgg(partialAgg, state); + auto* finalAgg = make( + *dt->aggregation->aggregation, + plan, + core::AggregationNode::Step::kFinal); + state.addCost(*finalAgg); + plan = finalAgg; + } + if (dt->orderBy) { + auto* orderBy = make( + plan, + dt->orderBy->distribution().order, + dt->orderBy->distribution().orderType); + state.addCost(*orderBy); + plan = orderBy; + } + if (!dt->columns.empty()) { + auto* project = make(plan, dt->exprs, dt->columns); + plan = project; + } +} + +std::vector chooseLeafIndex(const BaseTable* table) { + assert(!table->schemaTable->columnGroups.empty()); + return {table->schemaTable->columnGroups[0]}; +} + +template +CPSpan leadingColumns(V& exprs) { + int32_t i = 0; + for (; i < exprs.size(); ++i) { + if (exprs[i]->type() != PlanType::kColumn) { + break; + } + } + return CPSpan(reinterpret_cast(&exprs[0]), i); +} + +bool isIndexColocated( + const IndexInfo& info, + const ExprVector& lookupValues, + const RelationOpPtr& input) { + if (info.index->distribution().isBroadcast && + input->distribution().distributionType.locus == + info.index->distribution().distributionType.locus) { + return true; + } + + // True if 'input' is partitioned so that each partitioning key is joined to + // the corresponding partition key in 'info'. + if (!(input->distribution().distributionType == + info.index->distribution().distributionType)) { + return false; + } + if (input->distribution().partition.empty()) { + return false; + } + if (input->distribution().partition.size() != + info.index->distribution().partition.size()) { + return false; + } + for (auto i = 0; i < input->distribution().partition.size(); ++i) { + auto nthKey = position(lookupValues, *input->distribution().partition[i]); + if (nthKey != kNotFound) { + if (info.schemaColumn(info.lookupKeys.at(nthKey)) != + info.index->distribution().partition.at(i)) { + return false; + } + } else { + return false; + } + } + return true; +} + +RelationOpPtr repartitionForIndex( + const IndexInfo& info, + const ExprVector& lookupValues, + const RelationOpPtr& plan, + PlanState& state) { + if (isIndexColocated(info, lookupValues, plan)) { + return plan; + } + ExprVector keyExprs; + auto& partition = info.index->distribution().partition; + for (auto key : partition) { + // partition is in schema columns, lookupKeys is in BaseTable columns. Use + // the schema column of lookup key for matching. + auto nthKey = position( + info.lookupKeys, + [](auto c) { + return c->type() == PlanType::kColumn + ? c->template as()->schemaColumn() + : c; + }, + *key); + if (nthKey != kNotFound) { + keyExprs.push_back(lookupValues[nthKey]); + } else { + return nullptr; + } + } + + Distribution distribution( + info.index->distribution().distributionType, + plan->resultCardinality(), + std::move(keyExprs)); + auto* repartition = + make(plan, std::move(distribution), plan->columns()); + state.addCost(*repartition); + return repartition; +} + +float fanoutJoinTypeLimit(JoinType joinType, float fanout) { + switch (joinType) { + case JoinType::kLeft: + return std::max(1, fanout); + case JoinType::kLeftSemiFilter: + return std::min(1, fanout); + case JoinType::kAnti: + return 1 - std::min(1, fanout); + case JoinType::kLeftSemiProject: + case JoinType::kRightSemiProject: + return 1; + default: + return fanout; + } +} + +void Optimization::joinByIndex( + const RelationOpPtr& plan, + const JoinCandidate& candidate, + PlanState& state, + std::vector& toTry) { + if (candidate.tables.at(0)->type() != PlanType::kTable || + candidate.tables.size() > 1 || !candidate.existences.empty()) { + // Index applies to single base tables. + return; + } + auto rightTable = candidate.tables.at(0)->as(); + auto left = candidate.sideOf(rightTable, true); + auto right = candidate.sideOf(rightTable); + auto& keys = right.keys; + auto keyColumns = leadingColumns(keys); + if (keyColumns.empty()) { + return; + } + for (auto& index : rightTable->schemaTable->columnGroups) { + auto info = rightTable->schemaTable->indexInfo(index, keyColumns); + if (info.lookupKeys.empty()) { + continue; + } + PlanStateSaver save(state); + auto newPartition = repartitionForIndex(info, left.keys, plan, state); + if (!newPartition) { + continue; + } + state.placed.add(candidate.tables.at(0)); + auto joinType = right.leftJoinType(); + if (joinType == JoinType::kFull || joinType == JoinType::kRight) { + // Not available by index. + return; + } + auto fanout = fanoutJoinTypeLimit( + joinType, info.scanCardinality * rightTable->filterSelectivity); + + auto lookupKeys = left.keys; + // The number of keys is the prefix that matches index order. + lookupKeys.resize(info.lookupKeys.size()); + state.columns.unionSet(TableScan::availableColumns(rightTable, index)); + PlanObjectSet c = state.downstreamColumns(); + c.intersect(state.columns); + for (auto& filter : rightTable->filter) { + c.unionSet(filter->columns()); + } + + ColumnVector columns; + c.forEach([&](PlanObjectCP o) { columns.push_back(o->as()); }); + + auto* scan = make( + newPartition, + newPartition->distribution(), + rightTable, + info.index, + fanout, + columns, + lookupKeys, + joinType, + candidate.join->filter()); + + state.columns.unionSet(c); + state.addCost(*scan); + state.addNextJoin(&candidate, scan, {}, toTry); + } +} + +// Returns the positions in 'keys' for the expressions that determine the +// partition. empty if the partition is not decided by 'keys' +std::vector joinKeyPartition( + const RelationOpPtr& op, + const ExprVector& keys) { + std::vector positions; + for (unsigned i = 0; i < op->distribution().partition.size(); ++i) { + auto nthKey = position(keys, *op->distribution().partition[i]); + if (nthKey == kNotFound) { + return {}; + } + positions.push_back(nthKey); + } + return positions; +} + +PlanObjectSet availableColumns(PlanObjectCP object) { + PlanObjectSet set; + if (object->type() == PlanType::kTable) { + for (auto& c : object->as()->columns) { + set.add(c); + } + } else if (object->type() == PlanType::kDerivedTable) { + for (auto& c : object->as()->columns) { + set.add(c); + } + } else { + VELOX_UNREACHABLE("Joinable must be a table or derived table"); + } + return set; +} + +bool isBroadcastableSize(PlanPtr build, PlanState& /*state*/) { + return build->cost.fanout < 100000; +} + +void Optimization::joinByHash( + const RelationOpPtr& plan, + const JoinCandidate& candidate, + PlanState& state, + std::vector& toTry) { + assert(!candidate.tables.empty()); + auto build = candidate.sideOf(candidate.tables[0]); + auto probe = candidate.sideOf(candidate.tables[0], true); + ExprVector copartition; + auto partKeys = joinKeyPartition(plan, probe.keys); + if (partKeys.empty()) { + // Prefer to make a build partitioned on join keys and shuffle probe to + // align with build. + copartition = build.keys; + } + PlanStateSaver save(state); + PlanObjectSet buildTables; + PlanObjectSet buildColumns; + PlanObjectSet buildFilterColumns; + for (auto& filter : candidate.join->filter()) { + buildFilterColumns.unionColumns(filter); + } + buildFilterColumns.intersect(availableColumns(candidate.tables[0])); + for (auto buildTable : candidate.tables) { + buildColumns.unionSet(availableColumns(buildTable)); + buildTables.add(buildTable); + } + auto downstream = state.downstreamColumns(); + buildColumns.intersect(downstream); + buildColumns.unionColumns(build.keys); + buildColumns.unionSet(buildFilterColumns); + state.columns.unionSet(buildColumns); + auto memoKey = MemoKey{ + candidate.tables[0], buildColumns, buildTables, candidate.existences}; + PlanObjectSet empty; + bool needsShuffle = false; + auto buildPlan = makePlan( + memoKey, + Distribution(plan->distribution().distributionType, 0, copartition), + empty, + candidate.existsFanout, + state, + needsShuffle); + // the build side tables are all joined if the first build is a + // table but if it is a derived table (most often with aggregation), + // only some of the tables may be fully joined. + if (candidate.tables[0]->type() == PlanType::kDerivedTable) { + state.placed.add(candidate.tables[0]); + state.placed.unionSet(buildPlan->fullyImported); + } else { + state.placed.unionSet(buildTables); + } + PlanState buildState(state.optimization, state.dt, buildPlan); + bool partitionByProbe = !partKeys.empty(); + RelationOpPtr buildInput = buildPlan->op; + RelationOpPtr probeInput = plan; + if (partitionByProbe) { + if (needsShuffle) { + if (copartition.empty()) { + for (auto i : partKeys) { + copartition.push_back(build.keys[i]); + } + } + Distribution dist(plan->distribution().distributionType, 0, copartition); + auto* shuffleTemp = + make(buildInput, dist, buildInput->columns()); + buildState.addCost(*shuffleTemp); + buildInput = shuffleTemp; + } + } else if ( + candidate.join->isBroadcastableType() && + isBroadcastableSize(buildPlan, state)) { + auto* broadcast = make( + buildInput, + Distribution::broadcast( + plan->distribution().distributionType, plan->resultCardinality()), + buildInput->columns()); + buildState.addCost(*broadcast); + buildInput = broadcast; + } else { + // The probe gets shuffled to align with build. If build is not partitioned + // on its keys, shuffle the build too. + auto buildPart = joinKeyPartition(buildInput, build.keys); + if (buildPart.empty()) { + // The build is not aligned on join keys. + Distribution buildDist( + plan->distribution().distributionType, + plan->resultCardinality(), + build.keys); + auto* buildShuffle = + make(buildInput, buildDist, buildInput->columns()); + buildState.addCost(*buildShuffle); + buildInput = buildShuffle; + } + + ExprVector distCols; + for (size_t i = 0; i < probe.keys.size(); ++i) { + auto key = build.keys[i]; + auto nthKey = position(buildInput->distribution().partition, *key); + if (nthKey != kNotFound) { + if (distCols.size() <= nthKey) { + distCols.resize(nthKey + 1); + } + assert(!distCols.empty()); + distCols[nthKey] = probe.keys[i]; + } + } + Distribution probeDist( + probeInput->distribution().distributionType, + probeInput->resultCardinality(), + std::move(distCols)); + auto* probeShuffle = + make(plan, std::move(probeDist), plan->columns()); + state.addCost(*probeShuffle); + probeInput = probeShuffle; + } + auto* buildOp = + make(buildInput, ++buildCounter_, build.keys, buildPlan); + buildState.addCost(*buildOp); + + ColumnVector columns; + PlanObjectSet columnSet; + ColumnCP mark = nullptr; + PlanObjectSet probeColumns; + probeColumns.unionColumns(plan->columns()); + auto joinType = build.leftJoinType(); + bool probeOnly = joinType == JoinType::kLeftSemiFilter || + joinType == JoinType::kLeftSemiProject || joinType == JoinType::kAnti || + joinType == JoinType::kLeftSemiProject; + downstream = state.downstreamColumns(); + downstream.forEach([&](auto object) { + auto column = reinterpret_cast(object); + if (column == build.markColumn) { + mark = column; + columnSet.add(object); + return; + } + if (!(!probeOnly && buildColumns.contains(column)) && + !probeColumns.contains(column)) { + return; + } + columnSet.add(object); + columns.push_back(column); + }); + // If there is an existence flag, it is the rightmost result column. + if (mark) { + const_cast(&mark->value())->trueFraction = + std::min(1, candidate.fanout); + columns.push_back(mark); + } + state.columns = columnSet; + auto fanout = fanoutJoinTypeLimit(joinType, candidate.fanout); + auto* join = make( + JoinMethod::kHash, + joinType, + probeInput, + buildOp, + probe.keys, + build.keys, + candidate.join->filter(), + fanout, + std::move(columns)); + state.addCost(*join); + state.cost.setupCost += buildState.cost.unitCost; + state.addNextJoin(&candidate, join, {buildOp}, toTry); +} + +void Optimization::joinByHashRight( + const RelationOpPtr& plan, + const JoinCandidate& candidate, + PlanState& state, + std::vector& toTry) { + assert(!candidate.tables.empty()); + auto probe = candidate.sideOf(candidate.tables[0]); + auto build = candidate.sideOf(candidate.tables[0], true); + PlanStateSaver save(state); + PlanObjectSet probeTables; + PlanObjectSet probeColumns; + PlanObjectSet probeFilterColumns; + for (auto& filter : candidate.join->filter()) { + probeFilterColumns.unionColumns(filter); + } + probeFilterColumns.intersect(availableColumns(candidate.tables[0])); + + for (auto probeTable : candidate.tables) { + probeColumns.unionSet(availableColumns(probeTable)); + state.placed.add(probeTable); + probeTables.add(probeTable); + } + auto downstream = state.downstreamColumns(); + probeColumns.intersect(downstream); + probeColumns.unionColumns(probe.keys); + probeColumns.unionSet(probeFilterColumns); + state.columns.unionSet(probeColumns); + auto memoKey = MemoKey{ + candidate.tables[0], probeColumns, probeTables, candidate.existences}; + PlanObjectSet empty; + bool needsShuffle = false; + auto probePlan = makePlan( + memoKey, + Distribution(plan->distribution().distributionType, 0, {}), + empty, + candidate.existsFanout, + state, + needsShuffle); + PlanState probeState(state.optimization, state.dt, probePlan); + + RelationOpPtr probeInput = probePlan->op; + RelationOpPtr buildInput = plan; + // The build gets shuffled to align with probe. If probe is not partitioned + // on its keys, shuffle the probe too. + auto probePart = joinKeyPartition(probeInput, probe.keys); + if (probePart.empty()) { + Distribution probeDist( + buildInput->distribution().distributionType, + probeInput->resultCardinality(), + probe.keys); + auto* probeShuffle = + make(probeInput, probeDist, probeInput->columns()); + probeInput = probeShuffle; + } + ExprVector buildPartCols; + for (size_t i = 0; i < probe.keys.size(); ++i) { + auto key = probe.keys[i]; + auto nthKey = position(probeInput->distribution().partition, *key); + if (nthKey != kNotFound) { + if (buildPartCols.size() <= nthKey) { + buildPartCols.resize(nthKey + 1); + } + assert(!buildPartCols.empty()); + buildPartCols[nthKey] = build.keys[i]; + } + } + Distribution buildDist( + probeInput->distribution().distributionType, + buildInput->resultCardinality(), + std::move(buildPartCols)); + auto* buildShuffle = + make(plan, std::move(buildDist), plan->columns()); + state.addCost(*buildShuffle); + buildInput = buildShuffle; + + auto* buildOp = + make(buildInput, ++buildCounter_, build.keys, nullptr); + state.addCost(*buildOp); + + ColumnVector columns; + PlanObjectSet columnSet; + ColumnCP mark = nullptr; + PlanObjectSet buildColumns; + buildColumns.unionColumns(plan->columns()); + auto joinType = probe.leftJoinType(); + auto fanout = fanoutJoinTypeLimit(joinType, candidate.fanout); + // Change the join type to the right join variant. + switch (joinType) { + case JoinType::kLeft: + joinType = JoinType::kRight; + break; + case JoinType::kLeftSemiFilter: + joinType = JoinType::kRightSemiFilter; + break; + case JoinType::kLeftSemiProject: + joinType = JoinType::kRightSemiProject; + break; + default: + VELOX_FAIL("Join type does not have right hash join variant"); + } + + bool buildOnly = joinType == JoinType::kRightSemiFilter || + joinType == JoinType::kRightSemiProject; + downstream = state.downstreamColumns(); + downstream.forEach([&](auto object) { + auto column = reinterpret_cast(object); + if (column == probe.markColumn) { + mark = column; + return; + } + if (!buildColumns.contains(column) && + !(!buildOnly && probeColumns.contains(column))) { + return; + } + columnSet.add(object); + columns.push_back(column); + }); + if (mark) { + const_cast(&mark->value())->trueFraction = + std::min(1, candidate.fanout); + columns.push_back(mark); + } + + state.columns = columnSet; + auto buildCost = state.cost.unitCost; + state.cost = probeState.cost; + state.cost.setupCost += buildCost; + + auto* join = make( + JoinMethod::kHash, + joinType, + probeInput, + buildOp, + probe.keys, + build.keys, + candidate.join->filter(), + fanout, + std::move(columns)); + state.addCost(*join); + + state.addNextJoin(&candidate, join, {buildOp}, toTry); +} + +void Optimization::crossJoin( + const RelationOpPtr& plan, + const JoinCandidate& candidate, + PlanState& state, + std::vector& toTry) { + VELOX_NYI("No cross joins"); +} + +void Optimization::addJoin( + const JoinCandidate& candidate, + const RelationOpPtr& plan, + PlanState& state, + std::vector& result) { + std::vector toTry; + if (!candidate.join) { + crossJoin(plan, candidate, state, toTry); + return; + } + joinByIndex(plan, candidate, state, toTry); + auto sizeAfterIndex = toTry.size(); + joinByHash(plan, candidate, state, toTry); + if (toTry.size() > sizeAfterIndex && candidate.join->isNonCommutative()) { + // There is a hash based candidate with a non-commutative join. Try a right + // join variant. + joinByHashRight(plan, candidate, state, toTry); + } + // If one is much better do not try the other. + if (toTry.size() == 2 && candidate.tables.size() == 1) { + if (toTry[0].isWorse(toTry[1])) { + toTry.erase(toTry.begin()); + } else if (toTry[1].isWorse(toTry[0])) { + toTry.erase(toTry.begin() + 1); + } + } + result.insert(result.end(), toTry.begin(), toTry.end()); +} + +// Sets 'columns' to the columns in 'downstream' that exist +// in 'index' of 'table'. +ColumnVector indexColumns(const PlanObjectSet& downstream, ColumnGroupP index) { + ColumnVector result; + downstream.forEach([&](PlanObjectCP object) { + if (!object->as()->schemaColumn()) { + return; + } + if (position(index->columns(), *object->as()->schemaColumn()) != + kNotFound) { + result.push_back(object->as()); + } + }); + return result; +} + +void Optimization::tryNextJoins( + PlanState& state, + const std::vector& nextJoins) { + for (auto& next : nextJoins) { + PlanStateSaver save(state); + state.placed = next.placed; + state.columns = next.columns; + state.cost = next.cost; + state.addBuilds(next.newBuilds); + makeJoins(next.plan, state); + } +} + +RelationOpPtr Optimization::placeSingleRowDt( + RelationOpPtr plan, + const DerivedTable* subq, + ExprCP filter, + PlanState& state) { + auto broadcast = Distribution::broadcast(DistributionType(), 1); + MemoKey memoKey; + memoKey.firstTable = subq; + memoKey.tables.add(subq); + for (auto& column : subq->columns) { + memoKey.columns.add(column); + } + PlanObjectSet empty; + bool needsShuffle = false; + auto rightPlan = makePlan(memoKey, broadcast, empty, 1, state, needsShuffle); + auto rightOp = rightPlan->op; + if (needsShuffle) { + auto* repartition = + make(rightOp, broadcast, rightOp->columns()); + rightOp = repartition; + } + auto resultColumns = plan->columns(); + resultColumns.insert( + resultColumns.end(), + rightOp->columns().begin(), + rightOp->columns().end()); + auto* join = new (queryCtx()->allocate(sizeof(Join))) Join( + JoinMethod::kCross, + JoinType::kInner, + plan, + rightOp, + {}, + {}, + {filter}, + 0.5, + resultColumns); + state.addCost(*join); + return join; +} + +void Optimization::placeDerivedTable( + const DerivedTable* from, + PlanState& state) { + PlanStateSaver save(state); + + state.placed.add(from); + PlanObjectSet columns = state.downstreamColumns(); + PlanObjectSet dtColumns; + for (auto column : from->columns) { + dtColumns.add(column); + } + columns.intersect(dtColumns); + state.columns.unionSet(columns); + MemoKey key; + key.columns = columns; + key.firstTable = from; + key.tables.add(from); + bool ignore; + auto plan = makePlan(key, Distribution(), PlanObjectSet(), 1, state, ignore); + // Make plans based on the dt alone as first. + makeJoins(plan->op, state); + + // We see if there are reducing joins to import inside the dt. + PlanObjectSet visited = state.placed; + visited.add(from); + visited.unionSet(state.dt->importedExistences); + visited.unionSet(state.dt->fullyImported); + PlanObjectSet reducingSet; + reducingSet.add(from); + std::vector path{from}; + float reduction = 1; + reducingJoinsRecursive( + state, from, 1, 1.2, path, visited, reducingSet, reduction); + if (reduction < 0.9) { + key.tables = reducingSet; + auto savedPlaced = state.placed; + key.columns = state.downstreamColumns(); + plan = makePlan(key, Distribution(), PlanObjectSet(), 1, state, ignore); + // Not all reducing joins are necessarily retained in the plan. Only mark + // the ones fully imported as placed. + state.placed.unionSet(plan->fullyImported); + makeJoins(plan->op, state); + } +} + +bool Optimization::placeConjuncts(RelationOpPtr plan, PlanState& state) { + PlanStateSaver save(state); + ExprVector filters; + PlanObjectSet columnsAndSingles = state.columns; + state.dt->singleRowDts.forEach([&](PlanObjectCP object) { + columnsAndSingles.unionColumns(object->as()->columns); + }); + for (auto& conjunct : state.dt->conjuncts) { + if (state.placed.contains(conjunct)) { + continue; + } + if (conjunct->columns().isSubset(state.columns)) { + state.columns.add(conjunct); + filters.push_back(conjunct); + continue; + } + if (conjunct->columns().isSubset(columnsAndSingles)) { + // The filter depends on placed tables and non-correlated single row + // subqueries. + std::vector placeable; + auto subqColumns = conjunct->columns(); + subqColumns.except(state.columns); + subqColumns.forEach([&](PlanObjectCP object) { + state.dt->singleRowDts.forEach([&](PlanObjectCP dtObject) { + auto subq = dtObject->as(); + // If the subq provides columns for the filter, place it. + auto conjunctColumns = conjunct->columns(); + for (auto subqColumn : subq->columns) { + if (conjunctColumns.contains(subqColumn)) { + placeable.push_back(subq); + break; + } + } + }); + }); + for (auto i = 0; i < placeable.size(); ++i) { + state.placed.add(conjunct); + plan = placeSingleRowDt( + plan, + placeable[i], + (i == placeable.size() - 1 ? conjunct : nullptr), + state); + makeJoins(plan, state); + return true; + } + } + } + if (!filters.empty()) { + for (auto& filter : filters) { + state.placed.add(filter); + } + auto* filter = make(plan, std::move(filters)); + state.addCost(*filter); + makeJoins(filter, state); + return true; + } + return false; +} + +void Optimization::makeJoins(RelationOpPtr plan, PlanState& state) { + auto& dt = state.dt; + if (!plan) { + std::vector firstTables; + dt->startTables.forEach([&](auto table) { firstTables.push_back(table); }); + std::vector scores(firstTables.size()); + for (auto i = 0; i < firstTables.size(); ++i) { + auto table = firstTables[i]; + scores.at(i) = startingScore(table, dt); + } + std::vector ids(firstTables.size()); + std::iota(ids.begin(), ids.end(), 0); + std::sort(ids.begin(), ids.end(), [&](int32_t left, int32_t right) { + return scores[left] > scores[right]; + }); + for (auto i : ids) { + auto from = firstTables.at(i); + if (from->type() == PlanType::kTable) { + auto table = from->as(); + auto indices = chooseLeafIndex(table->as()); + // Make plan starting with each relevant index of the table. + auto downstream = state.downstreamColumns(); + for (auto index : indices) { + PlanStateSaver save(state); + state.placed.add(table); + auto columns = indexColumns(downstream, index); + + auto* scan = make( + nullptr, + TableScan::outputDistribution(table, index, columns), + table, + index, + index->distribution().cardinality * table->filterSelectivity, + columns); + + state.columns.unionObjects(columns); + state.addCost(*scan); + makeJoins(scan, state); + } + } else { + // Start with a derived table. + placeDerivedTable(from->as(), state); + } + } + } else { + if (state.isOverBest()) { + trace(kExceededBest, dt->id(), state.cost, *plan); + return; + } + // Add multitable filters not associated to a non-inner join. + if (placeConjuncts(plan, state)) { + return; + } + auto candidates = nextJoins(state); + if (candidates.empty()) { + addPostprocess(dt, plan, state); + auto kept = state.plans.addPlan(plan, state); + if (kept) { + trace(kRetained, dt->id(), state.cost, *kept->op); + } + return; + } + std::vector nextJoins; + nextJoins.reserve(candidates.size()); + for (auto& candidate : candidates) { + addJoin(candidate, plan, state, nextJoins); + } + tryNextJoins(state, nextJoins); + } +} + +PlanPtr Optimization::makePlan( + const MemoKey& key, + const Distribution& distribution, + const PlanObjectSet& /*boundColumns*/, + float existsFanout, + PlanState& state, + bool& needsShuffle) { + auto it = memo_.find(key); + PlanSet* plans; + if (it == memo_.end()) { + DerivedTable dt; + dt.import( + *state.dt, key.firstTable, key.tables, key.existences, existsFanout); + PlanState inner(*this, &dt); + if (key.firstTable->type() == PlanType::kDerivedTable) { + inner.setTargetColumnsForDt(key.columns); + } else { + inner.targetColumns = key.columns; + } + makeJoins(nullptr, inner); + memo_[key] = std::move(inner.plans); + plans = &memo_[key]; + } else { + plans = &it->second; + } + return plans->best(distribution, needsShuffle); +} + +} // namespace facebook::velox::optimizer diff --git a/verax/Plan.h b/verax/Plan.h new file mode 100644 index 0000000..00ce17c --- /dev/null +++ b/verax/Plan.h @@ -0,0 +1,782 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ +#pragma once + +#include "optimizer/Cost.h" //@manual +#include "optimizer/RelationOp.h" //@manual +#include "velox/connectors/Connector.h" +#include "velox/core/PlanNode.h" +#include "velox/runner/MultiFragmentPlan.h" + +/// Planning-time data structures. Represent the state of the planning process +/// plus utilities. +namespace facebook::velox::optimizer { + +struct ITypedExprHasher { + size_t operator()(const velox::core::ITypedExpr* expr) const { + return expr->hash(); + } +}; + +struct ITypedExprComparer { + bool operator()( + const velox::core::ITypedExpr* lhs, + const velox::core::ITypedExpr* rhs) const { + return *lhs == *rhs; + } +}; + +// Map for deduplicating ITypedExpr trees. +using ExprDedupMap = folly::F14FastMap< + const velox::core::ITypedExpr*, + ExprCP, + ITypedExprHasher, + ITypedExprComparer>; + +struct Plan; +struct PlanState; + +using PlanPtr = Plan*; + +/// A set of build sides. a candidate plan tracks all builds so that they can be +/// reused +using BuildSet = std::vector; + +/// Item produced by optimization and kept in memo. Corresponds to +/// pre-costed physical plan with costs and data properties. +struct Plan { + Plan(RelationOpPtr op, const PlanState& state); + + /// True if 'state' has a lower cost than 'this'. + bool isStateBetter(const PlanState& state) const; + + // Root of the plan tree. + RelationOpPtr op; + + // Total cost of 'op'. Setup costs and memory sizes are added up. The unit + // cost is the sum of the unit costs of the left-deep branch of 'op', where + // each unit cost is multiplied by the product of the fanouts of its inputs. + Cost cost; + + // The tables from original join graph that are included in this + // plan. If this is a derived table in the original plan, the + // covered object is the derived table, not its constituent + // tables. + PlanObjectSet tables; + + // The produced columns. Includes input columns. + PlanObjectSet columns; + + // Columns that are fixed on input. Applies to index path for a derived + // table, e.g. a left (t1 left t2) dt on dt.t1pk = a.fk. In a memo of dt + // inputs is dt.pkt1. + PlanObjectSet input; + + // hash join builds placed in the plan. Allows reusing a build. + BuildSet builds; + + // the tables/derived tables that are contained in this plan and need not be + // addressed by enclosing plans. This is all the tables in a build side join + // but not necessarily all tables that were added to a group by derived table. + PlanObjectSet fullyImported; + + std::string printCost() const; + std::string toString(bool detail) const; +}; + +/// The set of plans produced for a set of tables and columns. The plans may +/// have different output orders and distributions. +struct PlanSet { + // Interesting equivalent plans. + std::vector> plans; + + // plan with lowest cost + setupCost. Member of 'plans'. + PlanPtr bestPlan{nullptr}; + + // Cost of 'bestPlan' plus shuffle. If a cutoff is applicable, nothing more + // expensive than this should be tried. + float bestCostWithShuffle{0}; + + // Returns the best plan that produces 'distribution'. If the best plan has + // some other distribution, sets 'needsShuffle ' to true. + PlanPtr best(const Distribution& distribution, bool& needShuffle); + + /// Compares 'plan' to already seen plans and retains it if it is + /// interesting, e.g. better than the best so far or has an interesting + /// order. Returns the plan if retained, nullptr if not. + PlanPtr addPlan(RelationOpPtr plan, PlanState& state); +}; + +// Represents the next table/derived table to join. May consist of several +// tables for a bushy build side. +struct JoinCandidate { + JoinCandidate() = default; + + JoinCandidate(JoinEdgeP _join, PlanObjectCP _right, float _fanout) + : join(_join), tables({_right}), fanout(_fanout) {} + + // Returns the join side info for 'table'. If 'other' is set, returns the + // other side. + JoinSide sideOf(PlanObjectCP side, bool other = false) const; + + std::string toString() const; + + // The join between already placed tables and the table(s) in 'this'. + JoinEdgeP join{nullptr}; + + // Tables to join on the build side. The tables must not be already placed in + // the plan. side, i.e. be alread + std::vector tables; + + // Joins imported from the left side for reducing a build + // size. These could be ignored without affecting the result but can + // be included to restrict the size of build, e.g. lineitem join + // part left (partsupp exists part) would have the second part in + // 'existences' and partsupp in 'tables' because we know that + // partsupp will not be probed with keys that are not in part, so + // there is no point building with these. This may involve tables already + // placed in the plan. + std::vector existences; + + // Number of right side hits for one row on the left. The join + // selectivity in 'tables' affects this but the selectivity in + // 'existences' does not. + float fanout; + + // the selectivity from 'existences'. 0.2 means that the join of 'tables' is + // reduced 5x. + float existsFanout{1}; +}; + +/// Represents a join to add to a partial plan. One join candidate can make +/// many NextJoins, e.g, for different join methods. If one is clearly best, +/// not all need be tried. If many NextJoins are disconnected (no JoinEdge +/// between them), these may be statically orderable without going through +/// permutations. +struct NextJoin { + NextJoin( + const JoinCandidate* candidate, + const RelationOpPtr& plan, + const Cost& cost, + const PlanObjectSet& placed, + const PlanObjectSet& columns, + const BuildSet& builds) + : candidate(candidate), + plan(plan), + cost(cost), + placed(placed), + columns(columns), + newBuilds(builds) {} + + const JoinCandidate* candidate; + RelationOpPtr plan; + Cost cost; + PlanObjectSet placed; + PlanObjectSet columns; + BuildSet newBuilds; + + /// If true, only 'other' should be tried. Use to compare equivalent joins + /// with different join method or partitioning. + bool isWorse(const NextJoin& other) const; +}; + +class Optimization; + +/// Tracks the set of tables / columns that have been placed or are still needed +/// when constructing a partial plan. +struct PlanState { + PlanState(Optimization& optimization, DerivedTableP dt) + : optimization(optimization), dt(dt) {} + + PlanState(Optimization& optimization, DerivedTableP dt, PlanPtr plan) + : optimization(optimization), dt(dt), cost(plan->cost) {} + + Optimization& optimization; + // The derived table from which the tables are drawn. + DerivedTableP dt{nullptr}; + + // The tables that have been placed so far. + PlanObjectSet placed; + + // The columns that have a value from placed tables. + PlanObjectSet columns; + + // The columns that need a value at the end of the plan. A dt can be + // planned for just join/filter columns or all payload. Initially, + // columns the selected columns of the dt depend on. + PlanObjectSet targetColumns; + + // lookup keys for an index based derived table. + PlanObjectSet input; + + // The total cost for the PlanObjects placed thus far. + Cost cost; + + // All the hash join builds in any branch of the partial plan constructed so + // far. + BuildSet builds; + + // True if we should backtrack when 'costs' exceeds the best cost with shuffle + // from already generated plans. + bool hasCutoff{true}; + + // Interesting completed plans for the dt being planned. For + // example, best by cost and maybe plans with interesting orders. + PlanSet plans; + + // Caches results of downstreamColumns(). This is a pure function of + // 'placed' a'targetColumns' and 'dt'. + mutable std::unordered_map + downstreamPrecomputed; + + /// Updates 'cost_' to reflect 'op' being placed on top of the partial plan. + void addCost(RelationOp& op); + + /// Adds 'added' to all hash join builds. + void addBuilds(const BuildSet& added); + + // Specifies that the plan to make only references 'target' columns and + // whatever these depend on. These refer to 'columns' of 'dt'. + void setTargetColumnsForDt(const PlanObjectSet& target); + + /// Returns the set of columns referenced in unplaced joins/filters union + /// targetColumns. Gets smaller as more tables are placed. + PlanObjectSet downstreamColumns() const; + + // Adds a placed join to the set of partial queries to be developed. No op if + // cost exceeds best so far and cutoff is enabled. + void addNextJoin( + const JoinCandidate* candidate, + RelationOpPtr plan, + BuildSet builds, + std::vector& toTry) const; + + std::string printCost() const; + + /// Makes a string of 'op' with 'details'. Costs are annotated with percentage + /// of total in 'this->cost'. + std::string printPlan(RelationOpPtr op, bool detail) const; + + /// True if the costs accumulated so far are so high that this should not be + /// explored further. + bool isOverBest() const { + return hasCutoff && plans.bestPlan && + cost.unitCost + cost.setupCost > plans.bestCostWithShuffle; + } +}; + +/// A scoped guard that restores fields of PlanState on destruction. +struct PlanStateSaver { + public: + explicit PlanStateSaver(PlanState& state) + : state_(state), + placed_(state.placed), + columns_(state.columns), + cost_(state.cost), + numBuilds_(state.builds.size()) {} + + ~PlanStateSaver() { + state_.placed = std::move(placed_); + state_.columns = std::move(columns_); + state_.cost = cost_; + state_.builds.resize(numBuilds_); + } + + private: + PlanState& state_; + PlanObjectSet placed_; + PlanObjectSet columns_; + const Cost cost_; + const int32_t numBuilds_; +}; + +/// Key for collection of memoized partial plans. These are all made for hash +/// join builds with different cardinality reducing joins pushed down. The first +/// table is the table for which the key represents the build side. The 'tables' +/// set is the set of reducing joins applied to 'firstTable', including the +/// table itself. 'existences' is another set of reducing joins that are +/// semijoined to the join of 'tables' in order to restrict the build side. +struct MemoKey { + bool operator==(const MemoKey& other) const; + size_t hash() const; + + PlanObjectCP firstTable; + PlanObjectSet columns; + PlanObjectSet tables; + std::vector existences; +}; + +} // namespace facebook::velox::optimizer + +namespace std { +template <> +struct hash<::facebook::velox::optimizer::MemoKey> { + size_t operator()(const ::facebook::velox::optimizer::MemoKey& key) const { + return key.hash(); + } +}; +} // namespace std + +namespace facebook::velox::optimizer { + +/// Instance of query optimization. Comverts a plan and schema into an +/// optimized plan. Depends on QueryGraphContext being set on the +/// calling thread. There is one instance per query to plan. The +/// instance must stay live as long as a returned plan is live. +class Optimization { + public: + static constexpr int32_t kRetained = 1; + static constexpr int32_t kExceededBest = 2; + + using PlanCostMap = std::unordered_map< + velox::core::PlanNodeId, + std::vector>>; + + Optimization( + const velox::core::PlanNode& plan, + const Schema& schema, + History& history, + velox::core::ExpressionEvaluator& evaluator, + int32_t traceFlags = 0); + + /// Returns the optimized RelationOp plan for 'plan' given at construction. + PlanPtr bestPlan(); + + /// Returns a set of per-stage Velox PlanNode trees. + velox::runner::MultiFragmentPlanPtr toVeloxPlan( + RelationOpPtr plan, + const velox::runner::MultiFragmentPlan::Options& options); + + // Produces trace output if event matches 'traceFlags_'. + void trace(int32_t event, int32_t id, const Cost& cost, RelationOp& plan); + + void setLeafHandle( + int32_t id, + connector::ConnectorTableHandlePtr handle, + std::vector extraFilters) { + leafHandles_[id] = std::make_pair(handle, extraFilters); + } + + std::pair> + leafHandle(int32_t id) { + auto it = leafHandles_.find(id); + return it != leafHandles_.end() + ? it->second + : std::make_pair< + std::shared_ptr, + std::vector>(nullptr, {}); + } + + // Translates from Expr to Velox. + velox::core::TypedExprPtr toTypedExpr(ExprCP expr); + auto& idGenerator() { + return idGenerator_; + } + + /// Sets 'filterSelectivity' of 'baseTable' from history. Returns True if set. + bool setLeafSelectivity(BaseTable& baseTable) { + return history_.setLeafSelectivity(baseTable); + } + + auto& memo() { + return memo_; + } + + auto& existenceDts() { + return existenceDts_; + } + + // Lists the possible joins based on 'state.placed' and adds each on top of + // 'plan'. This is a set of plans extending 'plan' by one join (single table + // or bush). Calls itself on the interesting next plans. If all tables have + // been used, adds postprocess and adds the plan to 'plans' in 'state'. If + // 'state' enables cutoff and a partial plan is worse than the best so far, + // discards the candidate. + void makeJoins(RelationOpPtr plan, PlanState& state); + + velox::core::ExpressionEvaluator* evaluator() { + return &evaluator_; + } + + Name newCName(const std::string& prefix) { + return toName(fmt::format("{}{}", prefix, ++nameCounter_)); + } + + PlanCostMap planCostMap() { + return costEstimates_; + } + + bool& makeVeloxExprWithNoAlias() { + return makeVeloxExprWithNoAlias_; + } + + // Makes an output type for use in PlanNode et al. If 'columnType' is set, + // only considers base relation columns of the given type. + velox::RowTypePtr makeOutputType(const ColumnVector& columns); + + private: + static constexpr uint64_t kAllAllowedInDt = ~0UL; + + // True if 'op' is in 'mask. + static bool contains(uint64_t mask, PlanType op) { + return 0 != (mask & (1UL << static_cast(op))); + } + + // Returns a mask that allows 'op' in the same derived table. + uint64_t allow(PlanType op) { + return 1UL << static_cast(op); + } + + // Removes 'op' from the set of operators allowed in the current derived + // table. makeQueryGraph() starts a new derived table if it finds an operator + // that does not belong to the mask. + static uint64_t makeDtIf(uint64_t mask, PlanType op) { + return mask & ~(1UL << static_cast(op)); + } + + // Initializes a tree of DerivedTables with JoinEdges from 'plan' given at + // construction. Sets 'root_' to the root DerivedTable. + DerivedTableP makeQueryGraph(); + + // Converts 'plan' to PlanObjects and records join edges into + // 'currentSelect_'. If 'node' does not match allowedInDt, wraps 'node' in a + // new DerivedTable. + PlanObjectP makeQueryGraph( + const velox::core::PlanNode& node, + uint64_t allowedInDt); + + // Converts a table scan into a BaseTable wen building a DerivedTable. + PlanObjectP makeBaseTable(const core::TableScanNode* tableScan); + + // Interprets a Project node and adds its information into the DerivedTable + // being assembled. + void addProjection(const core::ProjectNode* project); + + // Interprets a Filter node and adds its information into the DerivedTable + // being assembled. + void addFilter(const core::FilterNode* Filter); + + // Interprets an AggregationNode and adds its information to the DerivedTable + // being assembled. + PlanObjectP addAggregation( + const core::AggregationNode& aggNode, + uint64_t allowedInDt); + + // Sets the columns to project out from the root DerivedTable based on + // 'plan'. + void setDerivedTableOutput( + DerivedTableP dt, + const velox::core::PlanNode& planNode); + + // Returns a literal from applying 'call' or 'cast' to 'literals'. nullptr if + // not successful. + ExprCP tryFoldConstant( + const velox::core::CallTypedExpr* call, + const velox::core::CastTypedExpr* cast, + const ExprVector& literals); + + // Makes a deduplicated Expr tree from 'expr'. + ExprCP translateExpr(const velox::core::TypedExprPtr& expr); + + // Adds conjuncts combined by any number of enclosing ands from 'input' to + // 'flat'. + void translateConjuncts( + const velox::core::TypedExprPtr& input, + ExprVector& flat); + + // Converts 'name' to a deduplicated ExprCP. If 'name' is assigned to an + // expression in a projection, returns the deduplicated ExprPtr of the + // expression. + ExprCP translateColumn(const std::string& name); + + // Applies translateColumn to a 'source'. + ExprVector translateColumns( + const std::vector& source); + + // Adds a JoinEdge corresponding to 'join' to the enclosing DerivedTable. + void translateJoin(const velox::core::AbstractJoinNode& join); + + // Makes an extra column for existence flag. + ColumnCP makeMark(const velox::core::AbstractJoinNode& join); + + // Adds a join edge for a join with no equalities. + void translateNonEqualityJoin(const velox::core::NestedLoopJoinNode& join); + + // Adds order by information to the enclosing DerivedTable. + OrderByP translateOrderBy(const velox::core::OrderByNode& order); + + // Adds aggregation information to the enclosing DerivedTable. + AggregationP translateAggregation( + const velox::core::AggregationNode& aggregation); + + // Adds 'node' and descendants to query graph wrapped inside a + // DerivedTable. Done for joins to the right of non-inner joins, + // group bys as non-top operators, whenever descendents of 'node' + // are not freely reorderable with its parents' descendents. + PlanObjectP wrapInDt(const velox::core::PlanNode& node); + + /// Retrieves or makes a plan from 'key'. 'key' specifies a set of + /// top level joined tables or a hash join build side table or + /// join. 'distribution' is the desired output distribution or a + /// distribution with no partitioning if this does + /// matter. 'boundColumns' is a set of columns that are lookup keys + /// for an index based path through the joins in + /// 'key'. 'existsFanout' is the selectivity for the 'existences' in + /// 'key', i.e. extra reducing joins for a hash join build side, + /// reflecting reducing joins on the probe side, 1 if none. 'state' + /// is the state of the caller, empty for a top level call and the + /// state with the planned objects so far if planning a derived + /// table. 'needsShuffle' is set to true if a shuffle is needed to + /// align the result of the made plan with 'distribution'. + PlanPtr makePlan( + const MemoKey& key, + const Distribution& distribution, + const PlanObjectSet& boundColumns, + float existsFanout, + PlanState& state, + bool& needsShuffle); + + // Returns a sorted list of candidates to add to the plan in + // 'state'. The joinable tables depend on the tables already present + // in 'plan'. A candidate will be a single table for all the single + // tables that can be joined. Additionally, when the single table + // can be joined to more tables not in 'state' to form a reducing + // join, this is produced as a candidate for a bushy hash join. When + // a single table or join to be used as a hash build side is made, + // we further check if reducing joins applying to the probe can be + // used to furthr reduce the build. These last joins are added as + // 'existences' in the candidate. + std::vector nextJoins(PlanState& state); + + // Adds group by, order by, top k to 'plan'. Updates 'plan' if + // relation ops added. Sets cost in 'state'. + void addPostprocess(DerivedTableP dt, RelationOpPtr& plan, PlanState& state); + + // Places a derived table as first table in a plan. Imports possibly reducing + // joins into the plan if can. + void placeDerivedTable(const DerivedTable* from, PlanState& state); + + // Adds the items from 'dt.conjuncts' that are not placed in 'state' + // and whose prerequisite columns are placed. If conjuncts can be + // placed, adds them to 'state.placed' and calls makeJoins() + // recursively to make the rest of the plan. Returns false if no + // unplaced conjuncts were found and and plan construction should + // proceed. + bool placeConjuncts(RelationOpPtr plan, PlanState& state); + + // Helper function that calls makeJoins recursively for each of + // 'nextJoins'. The point of making 'nextJoins' first and only then + // calling makeJoins is to allow detecting a star pattern of a fact + // table and independently joined dimensions. These can be ordered + // based on partitioning and size and we do not need to evaluate + // their different permutations. + void tryNextJoins(PlanState& state, const std::vector& nextJoins); + + // Adds a cross join to access a single row from a non-correlated subquery. + RelationOpPtr placeSingleRowDt( + RelationOpPtr plan, + const DerivedTable* subq, + ExprCP filter, + PlanState& state); + + // Adds the join represented by'candidate' on top of 'plan'. Tries index and + // hash based methods and adds the index and hash based plans to 'result'. If + // one of these is clearly superior, only adds the better one. + void addJoin( + const JoinCandidate& candidate, + const RelationOpPtr& plan, + PlanState& state, + std::vector& result); + + // If 'candidate' can be added on top 'plan' as a merge/index lookup, adds the + // plan to 'toTry'. Adds any necessary repartitioning. + void joinByIndex( + const RelationOpPtr& plan, + const JoinCandidate& candidate, + PlanState& state, + std::vector& toTry); + + // Adds 'candidate' on top of 'plan as a hash join. Adds possibly needed + // repartitioning to both probe and build and makes a broadcast build if + // indicated. If 'candidate' calls for a join on the build ide, plans a + // derived table with the build side tables and optionl 'existences' from + // 'candidate'. + void joinByHash( + const RelationOpPtr& plan, + const JoinCandidate& candidate, + PlanState& state, + std::vector& toTry); + + /// Tries a right hash join variant of left outer or left semijoin. + void joinByHashRight( + const RelationOpPtr& plan, + const JoinCandidate& candidate, + PlanState& state, + std::vector& toTry); + + void crossJoin( + const RelationOpPtr& plan, + const JoinCandidate& candidate, + PlanState& state, + std::vector& toTry); + + // Returns a filter expr that ands 'exprs'. nullptr if 'exprs' is empty. + velox::core::TypedExprPtr toAnd(const ExprVector& exprs); + + // Translates 'exprs' and returns them in 'result'. If an expr is + // other than a column, adds a projection node to evaluate the + // expression. The projection is added on top of 'source' and + // returned. If no projection is added, 'source' is returned. + velox::core::PlanNodePtr maybeProject( + const ExprVector& exprs, + velox::core::PlanNodePtr source, + std::vector& result); + + // Makes a Velox AggregationNode for a RelationOp. + velox::core::PlanNodePtr makeAggregation( + Aggregation& agg, + velox::runner::ExecutableFragment& fragment, + std::vector& stages); + + // Makes partial + final order by fragments for order by with and without + // limit. + velox::core::PlanNodePtr makeOrderBy( + OrderBy& op, + velox::runner::ExecutableFragment& fragment, + std::vector& stages); + + // Makes a tree of PlanNode for a tree of + // RelationOp. 'fragment' is the fragment that 'op' + // belongs to. If op or children are repartitions then the + // source of each makes a separate fragment. These + // fragments are referenced from 'fragment' via + // 'inputStages' and are returned in 'stages'. + velox::core::PlanNodePtr makeFragment( + RelationOpPtr op, + velox::runner::ExecutableFragment& fragment, + std::vector& stages); + + // Returns a new PlanNodeId and associates the Cost of 'op' with it. + velox::core::PlanNodeId nextId(const RelationOp& op); + + // Records 'cost' for 'id'. 'role' can be e.g. 'build; or + // 'probe'. for nodes that produce multiple operators. + void recordPlanNodeEstimate( + const velox::core::PlanNodeId id, + Cost cost, + const std::string& role); + + const Schema& schema_; + + // Top level plan to optimize. + const velox::core::PlanNode& inputPlan_; + + // Source of historical cost/cardinality information. + History& history_; + velox::core::ExpressionEvaluator& evaluator_; + // Top DerivedTable when making a QueryGraph from PlanNode. + DerivedTableP root_; + + // Innermost DerivedTable when making a QueryGraph from PlanNode. + DerivedTableP currentSelect_; + + // Maps names in project noes of 'inputPlan_' to deduplicated Exprs. + std::unordered_map renames_; + + // Maps unique core::TypedExprs from 'inputPlan_' to deduplicated Exps. + ExprDedupMap exprDedup_; + + // Counter for generating unique correlation names for BaseTables and + // DerivedTables. + int32_t nameCounter_{0}; + + // Serial number for columns created for projections that name Exprs, e.g. in + // join or grouping keys. + int32_t resultNameCounter_{0}; + + // Serial number for stages in executable plan. + int32_t stageCounter_{0}; + + std::unordered_map memo_; + + // Set of previously planned dts for importing probe side reducing joins to a + // build side + std::unordered_map existenceDts_; + + // The top level PlanState. Contains the set of top level interesting plans. + // Must stay alive as long as the Plans and RelationOps are reeferenced. + PlanState topState_{*this, nullptr}; + + // Controls tracing. + int32_t traceFlags_{0}; + + // Generates unique ids for build sides. + int32_t buildCounter_{0}; + + // When making a graph from 'inputPlan_' the output of an aggregation comes + // from the topmost (final) and the input from the lefmost (whichever consumes + // raw values). Records the output type of the final aggregation. + velox::RowTypePtr aggFinalType_; + + // Map from plan object id to pair of handle with pushdown filters and list of + // filters to eval on the result from the handle. + std::unordered_map< + int32_t, + std::pair< + connector::ConnectorTableHandlePtr, + std::vector>> + leafHandles_; + + class PlanNodeIdGenerator { + public: + explicit PlanNodeIdGenerator(int startId = 0) : nextId_{startId} {} + + core::PlanNodeId next() { + return fmt::format("{}", nextId_++); + } + + void reset(int startId = 0) { + nextId_ = startId; + } + + private: + int nextId_; + }; + + velox::runner::MultiFragmentPlan::Options options_; + PlanNodeIdGenerator idGenerator_; + // Limit for a possible limit/top k order by for while making a Velox plan. -1 + // means no limit. + int32_t toVeloxLimit_{-1}; + int32_t toVeloxOffset_{0}; + + // map from Velox PlanNode ids to RelationOp. For cases that have multiple + // operators, e.g. probe and build, both RelationOps are mentioned. + PlanCostMap costEstimates_; + + // On when producing a remaining filter for table scan, where columns must + // correspond 1:1 to the schema. + bool makeVeloxExprWithNoAlias_{false}; +}; + +/// Returns bits describing function 'name'. +FunctionSet functionBits(Name name); + +const JoinEdgeVector& joinedBy(PlanObjectCP table); + +void filterUpdated(BaseTableCP baseTable); + +} // namespace facebook::velox::optimizer diff --git a/verax/PlanObject.cpp b/verax/PlanObject.cpp new file mode 100644 index 0000000..a220a80 --- /dev/null +++ b/verax/PlanObject.cpp @@ -0,0 +1,164 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/QueryGraph.h" //@manual +#include "velox/common/base/SimdUtil.h" + +namespace facebook::velox::optimizer { + +size_t PlanObjectPHasher::operator()(const PlanObjectCP& object) const { + return object->hash(); +} + +bool PlanObjectPComparer::operator()( + const PlanObjectCP& lhs, + const PlanObjectCP& rhs) const { + if (rhs == lhs) { + return true; + } + return rhs && lhs && lhs->isExpr() && rhs->isExpr() && + reinterpret_cast(lhs)->sameOrEqual( + *reinterpret_cast(rhs)); +} + +size_t PlanObject::hash() const { + size_t h = static_cast(id_); + for (auto& child : children()) { + h = velox::bits::hashMix(h, child->hash()); + } + return h; +} + +namespace { +template +bool isZero(const V& bits, size_t begin, size_t end) { + for (size_t i = begin; i < end; ++i) { + if (bits[i]) { + return false; + } + } + return true; +} +} // namespace + +bool PlanObjectSet::operator==(const PlanObjectSet& other) const { + // The sets are equal if they have the same bits set. Trailing words of zeros + // do not count. + auto l1 = bits_.size(); + auto l2 = other.bits_.size(); + for (unsigned i = 0; i < l1 && i < l2; ++i) { + if (bits_[i] != other.bits_[i]) { + return false; + } + } + if (l1 < l2) { + return isZero(other.bits_, l1, l2); + } + if (l2 < l1) { + return isZero(bits_, l2, l1); + } + return true; +} + +bool PlanObjectSet::isSubset(const PlanObjectSet& super) const { + auto l1 = bits_.size(); + auto l2 = super.bits_.size(); + for (unsigned i = 0; i < l1 && i < l2; ++i) { + if (bits_[i] & ~super.bits_[i]) { + return false; + } + } + if (l2 < l1) { + return isZero(bits_, l2, l1); + } + return true; +} + +size_t PlanObjectSet::hash() const { + // The hash is a mix of the hashes of all non-zero words. + size_t hash = 123; + for (unsigned i = 0; i < bits_.size(); ++i) { + hash = velox::simd::crc32U64(hash, bits_[i]); + } + return hash * hash; +} + +void PlanObjectSet::unionColumns(ExprCP expr) { + switch (expr->type()) { + case PlanType::kLiteral: + return; + case PlanType::kColumn: + add(expr); + return; + case PlanType::kAggregate: { + auto condition = expr->as()->condition(); + if (condition) { + unionColumns(condition); + } + } + // Fall through. + FMT_FALLTHROUGH; + case PlanType::kCall: { + auto call = reinterpret_cast(expr); + unionSet(call->columns()); + return; + } + default: + VELOX_UNREACHABLE(); + } +} + +void PlanObjectSet::unionColumns(const ExprVector& exprs) { + for (auto& expr : exprs) { + unionColumns(expr); + } +} + +void PlanObjectSet::unionColumns(const ColumnVector& exprs) { + for (auto& expr : exprs) { + unionColumns(expr); + } +} + +void PlanObjectSet::unionSet(const PlanObjectSet& other) { + ensureWords(other.bits_.size()); + for (auto i = 0; i < other.bits_.size(); ++i) { + bits_[i] |= other.bits_[i]; + } +} + +void PlanObjectSet::intersect(const PlanObjectSet& other) { + bits_.resize(std::min(bits_.size(), other.bits_.size())); + for (auto i = 0; i < bits_.size(); ++i) { + assert(!other.bits_.empty()); + bits_[i] &= other.bits_[i]; + } +} + +std::string PlanObjectSet::toString(bool names) const { + std::stringstream out; + forEach([&](auto object) { + out << object->id(); + if (names) { + out << ": " << object->toString() << std::endl; + } else { + out << " "; + } + }); + return out.str(); +} + +} // namespace facebook::velox::optimizer diff --git a/verax/PlanObject.h b/verax/PlanObject.h new file mode 100644 index 0000000..aa2a6c2 --- /dev/null +++ b/verax/PlanObject.h @@ -0,0 +1,244 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "optimizer/QueryGraphContext.h" //@manual + +namespace facebook::velox::optimizer { + +/// Enum for types of query graph nodes. Used when making a tree into +/// a query graph and later to differentiate between tables, derived +/// tables and different expressions. +enum class PlanType { + kTable, + kDerivedTable, + kColumn, + kLiteral, + kCall, + kAggregate, + kAggregation, + kProject, + kFilter, + kJoin, + kOrderBy, + kLimit +}; + +/// True if 'type' is an expression with a value. +inline bool isExprType(PlanType type) { + return type == PlanType::kColumn || type == PlanType::kCall || + type == PlanType::kLiteral; +} + +/// Common superclass of all vertices of a query graph. This +/// includes tables, columns, expressions, derived tables etc. These +/// all have a unique small integer id. Ids are often to track +/// membership in PlanObjectSets. These stand for e.g. the columns +/// assigned by an operator or the tables in a partial plan. Joins +/// are edges of the graph but do not have ids, rather, they are +/// identified by the ids of their end points. PlanObjects are created at the +/// start of planning and are arena allocated to be all dropped when the +/// planning is complete. +class PlanObject { + public: + explicit PlanObject(PlanType _type) + : type_(_type), id_(queryCtx()->newId(this)) {} + + virtual ~PlanObject() = default; + + void operator delete(void* /*ptr*/) { + LOG(FATAL) << "Plan objects are not deletable"; + } + + int32_t id() const { + return id_; + } + + PlanType type() const { + return type_; + } + + template + T* as() { + return reinterpret_cast(this); + } + + /// Returns 'this' as const T. + template + const T* as() const { + return reinterpret_cast(this); + } + + /// Returns a view on children, e.g. arguments of a function call. + virtual CPSpan children() const { + return CPSpan(nullptr, nullptr); + } + + /// Returns true if 'this' is an expression with a value. + virtual bool isExpr() const { + return false; + } + + /// Returns true if this is a table or derived table. + virtual bool isTable() const { + return false; + } + + /// Returns true if 'this' is a function call or aggregate function call. + virtual bool isFunction() const { + return false; + } + + size_t hash() const; + + virtual std::string toString() const { + return fmt::format("#{}", id_); + } + + private: + const PlanType type_; + const int32_t id_; +}; + +/// Set of PlanObjects. Uses the objects id() as an index into a bitmap. +class PlanObjectSet { + public: + /// True if id of 'object' is in 'this'. + bool contains(PlanObjectCP object) const { + return object->id() < bits_.size() * 64 && + velox::bits::isBitSet(bits_.data(), object->id()); + } + + bool operator==(const PlanObjectSet& other) const; + + size_t hash() const; + + // True if no members. + bool empty() const { + for (auto word : bits_) { + if (word) { + return false; + } + } + return true; + } + + /// Inserts id of 'object'. + void add(PlanObjectCP object) { + auto id = object->id(); + ensureSize(id); + velox::bits::setBit(bits_.data(), id); + } + + /// Returns true if 'this' is a subset of 'super'. + bool isSubset(const PlanObjectSet& super) const; + + /// Erases id of 'object'. + void erase(PlanObjectCP object) { + if (object->id() < bits_.size() * 64) { + velox::bits::clearBit(bits_.data(), object->id()); + } + } + + void except(const PlanObjectSet& other) { + velox::bits::forEachSetBit( + other.bits_.data(), 0, other.bits_.size() * 64, [&](auto id) { + if (id < bits_.size() * 64) { + velox::bits::clearBit(bits_.data(), id); + } + }); + } + + /// Adds ids of all columns 'expr' depends on. + void unionColumns(ExprCP expr); + + /// Adds ids of all columns 'exprs' depend on. + void unionColumns(const ExprVector& exprs); + void unionColumns(const ColumnVector& exprs); + + /// Adds all ids in 'other'. + void unionSet(const PlanObjectSet& other); + + /// Erases all ids not in 'other'. + void intersect(const PlanObjectSet& other); + + /// Adds ids of all objects in 'objects'. + template + void unionObjects(const V& objects) { + for (auto& object : objects) { + add(object); + } + } + + /// Applies 'func' to each object in 'this'. + template + void forEach(Func func) const { + auto ctx = queryCtx(); + velox::bits::forEachSetBit(bits_.data(), 0, bits_.size() * 64, [&](auto i) { + func(ctx->objectAt(i)); + }); + } + + template + void forEachMutable(Func func) const { + auto ctx = queryCtx(); + velox::bits::forEachSetBit(bits_.data(), 0, bits_.size() * 64, [&](auto i) { + func(ctx->mutableObjectAt(i)); + }); + } + + /// Returns the objects corresponding to ids in 'this' as a vector of T. + template + std::vector objects() const { + std::vector result; + forEach( + [&](auto object) { result.push_back(reinterpret_cast(object)); }); + return result; + } + + /// Prnts the contents with ids and the string representation of the objects + /// if 'names' is true. + std::string toString(bool names) const; + + private: + void ensureSize(int32_t id) { + ensureWords(velox::bits::nwords(id + 1)); + } + + void ensureWords(int32_t size) { + if (bits_.size() < size) { + bits_.resize(size); + } + } + + // A one bit corresponds to the id of each member. + std::vector> bits_; +}; + +using PlanObjectVector = std::vector>; + +} // namespace facebook::velox::optimizer + +namespace std { +template <> +struct hash<::facebook::velox::optimizer::PlanObjectSet> { + size_t operator()( + const ::facebook::velox::optimizer::PlanObjectSet& set) const { + return set.hash(); + } +}; +} // namespace std diff --git a/verax/PlanUtils.cpp b/verax/PlanUtils.cpp new file mode 100644 index 0000000..387a1d7 --- /dev/null +++ b/verax/PlanUtils.cpp @@ -0,0 +1,62 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/PlanUtils.h" //@manual +#include "optimizer/QueryGraph.h" //@manual + +namespace facebook::velox::optimizer { + +/// Match the input 'value' to the most appropriate unit and return +/// a string value. The units are specified in the 'units' array. +/// unitOffset is used to skip the starting units. +/// unitScale is used to determine the unit. +/// precision is used to set the decimal digits in the final output. +std::string succinctPrint( + double decimalValue, + const std::string_view* units, + int unitsSize, + int unitOffset, + double unitScale, + int precision) { + std::stringstream out; + int offset = unitOffset; + while ((decimalValue / unitScale) >= 1 && offset < (unitsSize - 1)) { + decimalValue = decimalValue / unitScale; + offset++; + } + if (offset == unitOffset) { + // Print the default value. + precision = 0; + } + out << std::fixed << std::setprecision(precision) << decimalValue + << units[offset]; + return out.str(); +} + +std::string succinctNumber(double value, int32_t precision) { + static constexpr std::string_view kUnit[] = { + "n", "u", "m", "", "k", "M", "G", "T", "P"}; + + return succinctPrint( + value * 1e9, + kUnit, + sizeof(kUnit) / sizeof(std::string_view), + 0, + 1000, + precision); +} + +} // namespace facebook::velox::optimizer diff --git a/verax/PlanUtils.h b/verax/PlanUtils.h new file mode 100644 index 0000000..9a28cbb --- /dev/null +++ b/verax/PlanUtils.h @@ -0,0 +1,121 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include +#include "optimizer/QueryGraph.h" //@manual + +namespace facebook::velox::optimizer { + +template +bool isSubset(const T& subset, const U& superset) { + for (auto item : subset) { + if (std::find(superset.begin(), superset.end(), item) == superset.end()) { + return false; + } + } + return true; +} + +// Returns how many leading members of 'ordered' are covered by 'set' +template +uint32_t prefixSize(Ordered ordered, Set set) { + for (auto i = 0; i < ordered.size(); ++i) { + if (std::find(set.begin(), set.end(), ordered[i]) == set.end()) { + return i; + } + } + return ordered.size(); +} + +// Replaces each element of 'set' that matches an element of 'originals' with +// the corresponding element of 'replaceWith'. Returns true if all elements were +// replaced. +template +bool replace(Set& set, Old& originals, New replaceWith) { + bool allReplaced = true; + for (auto& element : set) { + auto it = std::find(originals.begin(), originals.end(), element); + if (it == originals.end()) { + allReplaced = false; + continue; + } + element = replaceWith[it - originals.begin()]; + } + return allReplaced; +} + +template +void appendToVector(T& destination, U& source) { + for (auto i : source) { + destination.push_back(i); + } +} + +constexpr uint32_t kNotFound = ~0U; + +/// Returns index of 'expr' in collection 'exprs'. kNotFound if not found. +/// Compares with equivalence classes, so that equal columns are +/// interchangeable. +template +uint32_t position(const V& exprs, const Expr& expr) { + for (auto i = 0; i < exprs.size(); ++i) { + if (exprs[i]->sameOrEqual(expr)) { + return i; + } + } + return kNotFound; +} + +/// Returns index of 'expr' in collection 'exprs'. kNotFound if not found. +/// Compares with equivalence classes, so that equal columns are +/// interchangeable. Applies 'getter' to each element of 'exprs' before +/// comparison. +template +uint32_t position(const V& exprs, Getter getter, const Expr& expr) { + for (auto i = 0; i < exprs.size(); ++i) { + if (getter(exprs[i])->sameOrEqual(expr)) { + return i; + } + } + return kNotFound; +} + +/// Prints a number with precision' digits followed by a scale letter (n, u, m, +/// , k, M, G T, P. +std::string succinctNumber(double value, int32_t precision = 2); + +/// Returns the sum of the sizes of 'exprs'. +template +float byteSize(const V& exprs) { + float size = 0; + for (auto& expr : exprs) { + size += expr->value().byteSize(); + } + return size; +} + +template +Target transform(const V& set, Func func) { + Target result; + for (auto& elt : set) { + result.push_back(func(elt)); + } + return result; +} + +} // namespace facebook::velox::optimizer diff --git a/verax/QueryGraph.cpp b/verax/QueryGraph.cpp new file mode 100644 index 0000000..7e1e01a --- /dev/null +++ b/verax/QueryGraph.cpp @@ -0,0 +1,1072 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/QueryGraph.h" //@manual +#include "optimizer/Plan.h" //@manual +#include "optimizer/PlanUtils.h" //@manual +#include "velox/common/base/SimdUtil.h" +#include "velox/common/base/SuccinctPrinter.h" + +namespace facebook::velox::optimizer { + +void Column::equals(ColumnCP other) const { + if (!equivalence_ && !other->equivalence_) { + auto* equiv = make(); + equiv->columns.push_back(this); + equiv->columns.push_back(other); + equivalence_ = equiv; + other->equivalence_ = equiv; + return; + } + if (!other->equivalence_) { + other->equivalence_ = equivalence_; + equivalence_->columns.push_back(other); + return; + } + if (!equivalence_) { + other->equals(this); + return; + } + for (auto& column : other->equivalence_->columns) { + equivalence_->columns.push_back(column); + column->equivalence_ = equivalence_; + } +} + +std::string Column::toString() const { + Name cname = !relation_ ? "" + : relation_->type() == PlanType::kTable + ? relation_->as()->cname + : relation_->type() == PlanType::kDerivedTable + ? relation_->as()->cname + : "--"; + + return fmt::format("{}.{}", cname, name_); +} + +std::string Call::toString() const { + std::stringstream out; + out << name_ << "("; + for (auto i = 0; i < args_.size(); ++i) { + out << args_[i]->toString() << (i == args_.size() - 1 ? ")" : ", "); + } + return out.str(); +} + +std::string conjunctsToString(const ExprVector& conjuncts) { + std::stringstream out; + for (auto i = 0; i < conjuncts.size(); ++i) { + out << conjuncts[i]->toString() + << (i == conjuncts.size() - 1 ? "" : " and "); + } + return out.str(); +} + +std::string BaseTable::toString() const { + std::stringstream out; + out << "{" << PlanObject::toString(); + out << schemaTable->name << " " << cname << "}"; + return out.str(); +} + +const JoinSide JoinEdge::sideOf(PlanObjectCP side, bool other) const { + if ((side == rightTable_ && !other) || (side == leftTable_ && other)) { + return { + rightTable_, + rightKeys_, + lrFanout_, + rightOptional_, + rightExists_, + rightNotExists_, + markColumn_, + rightUnique_}; + } + return { + leftTable_, + leftKeys_, + rlFanout_, + leftOptional_, + false, + false, + nullptr, + leftUnique_}; +} + +bool JoinEdge::isBroadcastableType() const { + return !leftOptional_; +} + +void JoinEdge::addEquality(ExprCP left, ExprCP right) { + leftKeys_.push_back(left); + rightKeys_.push_back(right); + guessFanout(); +} + +std::string JoinEdge::toString() const { + std::stringstream out; + out << "toString() : " multiple tables "); + if (leftOptional_ && rightOptional_) { + out << " full outr "; + } else if (markColumn_) { + out << " exists project "; + } else if (rightOptional_) { + out << " left"; + } else if (rightExists_) { + out << " exists "; + } else if (rightNotExists_) { + out << " not exists "; + } else { + out << " inner "; + } + out << rightTable_->toString(); + out << " on "; + for (auto i = 0; i < leftKeys_.size(); ++i) { + out << leftKeys_[i]->toString() << " = " << rightKeys_[i]->toString() + << (i < leftKeys_.size() - 1 ? " and " : ""); + } + if (!filter_.empty()) { + out << " filter " << conjunctsToString(filter_); + } + out << ">"; + return out.str(); +} + +bool Expr::sameOrEqual(const Expr& other) const { + if (this == &other) { + return true; + } + if (type() != other.type()) { + return false; + } + switch (type()) { + case PlanType::kColumn: + return as()->equivalence() && + as()->equivalence() == other.as()->equivalence(); + case PlanType::kAggregate: { + auto a = reinterpret_cast(this); + auto b = reinterpret_cast(&other); + if (a->isDistinct() != b->isDistinct() || + a->isAccumulator() != b->isAccumulator() || + !(a->condition() == b->condition() || + (a->condition() && b->condition() && + a->condition()->sameOrEqual(*b->condition())))) { + return false; + } + } + // Fall through. + FMT_FALLTHROUGH; + case PlanType::kCall: { + if (as()->name() != other.as()->name()) { + return false; + } + auto numArgs = as()->args().size(); + if (numArgs != other.as()->args().size()) { + return false; + } + for (auto i = 0; i < numArgs; ++i) { + if (as()->args()[i]->sameOrEqual(*other.as()->args()[i])) { + return false; + } + } + return true; + } + default: + return false; + } +} + +PlanObjectCP singleTable(PlanObjectCP object) { + if (isExprType(object->type())) { + return object->as()->singleTable(); + } + return nullptr; +} + +PlanObjectCP Expr::singleTable() const { + if (type() == PlanType::kColumn) { + return as()->relation(); + } + PlanObjectCP table = nullptr; + bool multiple = false; + columns_.forEach([&](PlanObjectCP object) { + VELOX_CHECK(object->type() == PlanType::kColumn); + if (!table) { + table = object->template as()->relation(); + } else if (table != object->as()->relation()) { + multiple = true; + } + }); + return multiple ? nullptr : table; +} + +PlanObjectSet Expr::allTables() const { + PlanObjectSet set; + columns_.forEach( + [&](PlanObjectCP object) { set.add(object->as()->relation()); }); + return set; +} + +PlanObjectSet allTables(CPSpan exprs) { + PlanObjectSet all; + for (auto expr : exprs) { + auto set = expr->allTables(); + all.unionSet(set); + } + return all; +} + +Column::Column(Name name, PlanObjectP relation, const Value& value) + : Expr(PlanType::kColumn, value), name_(name), relation_(relation) { + columns_.add(this); + if (relation_ && relation_->type() == PlanType::kTable) { + schemaColumn_ = relation->as()->schemaTable->findColumn(name_); + VELOX_CHECK(schemaColumn_); + } +} + +void DerivedTable::addJoinEquality( + ExprCP left, + ExprCP right, + const ExprVector& filter, + bool leftOptional, + bool rightOptional, + bool rightExists, + bool rightNotExists) { + auto leftTable = singleTable(left); + auto rightTable = singleTable(right); + for (auto& join : joins) { + if (join->leftTable() == leftTable && join->rightTable() == rightTable) { + join->addEquality(left, right); + return; + } else if ( + join->rightTable() == leftTable && join->leftTable() == rightTable) { + join->addEquality(right, left); + return; + } + } + auto* join = make( + leftTable, + rightTable, + filter, + leftOptional, + rightOptional, + rightExists, + rightNotExists); + join->addEquality(left, right); + joins.push_back(join); +} + +using EdgeSet = std::unordered_set>; + +bool hasEdge(const EdgeSet& edges, int32_t id1, int32_t id2) { + if (id1 == id2) { + return true; + } + auto it = edges.find( + id1 > id2 ? std::pair(id2, id1) + : std::pair(id1, id2)); + return it != edges.end(); +} + +void addEdge(EdgeSet& edges, int32_t id1, int32_t id2) { + if (id1 > id2) { + edges.insert(std::pair(id2, id1)); + } else { + edges.insert(std::pair(id1, id2)); + } +} + +void fillJoins( + PlanObjectCP column, + const Equivalence& equivalence, + EdgeSet& edges, + DerivedTableP dt) { + for (auto& other : equivalence.columns) { + if (!hasEdge(edges, column->id(), other->id())) { + addEdge(edges, column->id(), other->id()); + dt->addJoinEquality( + column->as(), + other->as(), + {}, + false, + false, + false, + false); + } + } +} + +void DerivedTable::addImpliedJoins() { + EdgeSet edges; + for (auto& join : joins) { + if (join->isInner()) { + for (auto i = 0; i < join->leftKeys().size(); ++i) { + if (join->leftKeys()[i]->type() == PlanType::kColumn && + join->rightKeys()[i]->type() == PlanType::kColumn) { + addEdge(edges, join->leftKeys()[i]->id(), join->rightKeys()[i]->id()); + } + } + } + } + // The loop appends to 'joins', so loop over a copy. + JoinEdgeVector joinsCopy = joins; + for (auto& join : joinsCopy) { + if (join->isInner()) { + for (auto i = 0; i < join->leftKeys().size(); ++i) { + if (join->leftKeys()[i]->type() == PlanType::kColumn && + join->rightKeys()[i]->type() == PlanType::kColumn) { + auto leftEq = join->leftKeys()[i]->as()->equivalence(); + auto rightEq = join->rightKeys()[i]->as()->equivalence(); + if (rightEq && leftEq) { + for (auto& left : leftEq->columns) { + fillJoins(left, *rightEq, edges, this); + } + } else if (leftEq) { + fillJoins(join->rightKeys()[i], *leftEq, edges, this); + } else if (rightEq) { + fillJoins(join->leftKeys()[i], *rightEq, edges, this); + } + } + } + } + } +} + +void DerivedTable::setStartTables() { + findSingleRowDts(); + startTables = tableSet; + startTables.except(singleRowDts); + for (auto join : joins) { + if (join->isNonCommutative()) { + startTables.erase(join->rightTable()); + } + } +} + +bool isSingleRowDt(PlanObjectCP object) { + if (object->type() == PlanType::kDerivedTable) { + auto dt = object->as(); + return dt->limit == 1 || + (dt->aggregation && dt->aggregation->aggregation->grouping.empty()); + } + return false; +} + +void DerivedTable::findSingleRowDts() { + auto tablesCopy = tableSet; + int32_t numSingle = 0; + for (auto& join : joins) { + tablesCopy.erase(join->rightTable()); + for (auto& key : join->leftKeys()) { + tablesCopy.except(key->allTables()); + } + for (auto& filter : join->filter()) { + tablesCopy.except(filter->allTables()); + } + } + tablesCopy.forEach([&](PlanObjectCP object) { + if (isSingleRowDt(object)) { + ++numSingle; + singleRowDts.add(object); + } + }); + // if everything is a single row dt, then process tese as cross products and + // not as placed with filters. + if (numSingle == tables.size()) { + singleRowDts = PlanObjectSet(); + } +} + +void DerivedTable::linkTablesToJoins() { + setStartTables(); + + // All tables directly mentioned by a join link to the join. A non-inner + // that depends on multiple left tables has no leftTable but is still linked + // from all the tables it depends on. + for (auto join : joins) { + PlanObjectSet tables; + for (auto key : join->leftKeys()) { + tables.unionSet(key->allTables()); + } + for (auto key : join->rightKeys()) { + tables.unionSet(key->allTables()); + } + if (!join->filter().empty()) { + for (auto& conjunct : join->filter()) { + tables.unionSet(conjunct->allTables()); + } + } + tables.forEachMutable([&](PlanObjectP table) { + if (table->type() == PlanType::kTable) { + table->as()->addJoinedBy(join); + } else { + VELOX_CHECK(table->type() == PlanType::kDerivedTable); + table->as()->addJoinedBy(join); + } + }); + } +} + +// Returns a right exists (semijoin) with 'table' on the left and one of +// 'tables' on the right. +JoinEdgeP makeExists(PlanObjectCP table, PlanObjectSet tables) { + for (auto join : joinedBy(table)) { + if (join->leftTable() == table) { + if (!tables.contains(join->rightTable())) { + continue; + } + auto* exists = new (queryCtx()->allocate(sizeof(JoinEdge))) + JoinEdge(table, join->rightTable(), {}, false, false, true, false); + for (auto i = 0; i < join->leftKeys().size(); ++i) { + exists->addEquality(join->leftKeys()[i], join->rightKeys()[i]); + } + return exists; + } + if (join->rightTable() == table) { + if (!join->leftTable() || !tables.contains(join->leftTable())) { + continue; + } + + auto* exists = QGC_MAKE_IN_ARENA(JoinEdge)( + table, join->leftTable(), {}, false, false, true, false); + for (auto i = 0; i < join->leftKeys().size(); ++i) { + exists->addEquality(join->rightKeys()[i], join->leftKeys()[i]); + } + return exists; + } + } + VELOX_UNREACHABLE("No join to make an exists build side restriction"); +} + +std::pair makeExistsDtAndJoin( + const DerivedTable& super, + PlanObjectCP firstTable, + float existsFanout, + PlanObjectVector& existsTables, + JoinEdgeP existsJoin) { + auto firstExistsTable = existsJoin->rightKeys()[0]->singleTable(); + VELOX_CHECK(firstExistsTable); + MemoKey existsDtKey; + existsDtKey.firstTable = firstExistsTable; + for (auto& column : existsJoin->rightKeys()) { + existsDtKey.columns.unionColumns(column); + } + auto optimization = queryCtx()->optimization(); + existsDtKey.tables.unionObjects(existsTables); + auto it = optimization->existenceDts().find(existsDtKey); + DerivedTableP existsDt; + if (it == optimization->existenceDts().end()) { + auto* newDt = make(); + existsDt = newDt; + existsDt->cname = queryCtx()->optimization()->newCName("edt"); + existsDt->import(super, firstExistsTable, existsDtKey.tables, {}); + for (auto& k : existsJoin->rightKeys()) { + auto* existsColumn = make( + toName(fmt::format("{}.{}", existsDt->cname, k->toString())), + existsDt, + k->value()); + existsDt->columns.push_back(existsColumn); + existsDt->exprs.push_back(k); + } + existsDt->noImportOfExists = true; + existsDt->makeInitialPlan(); + optimization->existenceDts()[existsDtKey] = existsDt; + } else { + existsDt = it->second; + } + auto* joinWithDt = QGC_MAKE_IN_ARENA(JoinEdge)( + firstTable, existsDt, {}, false, false, true, false); + joinWithDt->setFanouts(existsFanout, 1); + for (auto i = 0; i < existsJoin->leftKeys().size(); ++i) { + joinWithDt->addEquality(existsJoin->leftKeys()[i], existsDt->columns[i]); + } + return std::make_pair(existsDt, joinWithDt); +} + +void DerivedTable::import( + const DerivedTable& super, + PlanObjectCP firstTable, + const PlanObjectSet& _tables, + const std::vector& existences, + float existsFanout) { + tableSet = _tables; + _tables.forEach([&](auto table) { tables.push_back(table); }); + for (auto join : super.joins) { + if (_tables.contains(join->rightTable()) && join->leftTable() && + _tables.contains(join->leftTable())) { + joins.push_back(join); + } + } + for (auto& exists : existences) { + // We filter the derived table by importing reducing semijoins. + // These are based on joins on the outer query but become + // existences so as not to change cardinality. The reducing join + // is against one or more tables. If more than one table, the join + // of these tables goes into its own derived table which is joined + // with exists to the main table(s) in the 'this'. + importedExistences.unionSet(exists); + PlanObjectVector existsTables; + exists.forEach([&](auto object) { existsTables.push_back(object); }); + auto existsJoin = makeExists(firstTable, exists); + if (existsTables.size() > 1) { + // There is a join on the right of exists. Needs its own dt. + auto [existsDt, joinWithDt] = makeExistsDtAndJoin( + super, firstTable, existsFanout, existsTables, existsJoin); + joins.push_back(joinWithDt); + tables.push_back(existsDt); + tableSet.add(existsDt); + noImportOfExists = true; + } else { + joins.push_back(existsJoin); + assert(!existsTables.empty()); + tables.push_back(existsTables[0]); + tableSet.add(existsTables[0]); + noImportOfExists = true; + } + } + if (firstTable->type() == PlanType::kDerivedTable) { + importJoinsIntoFirstDt(firstTable->as()); + } else { + fullyImported = _tables; + } + linkTablesToJoins(); +} + +// Returns a copy of 'expr,, replacing instances of columns in 'outer' with the +// corresponding expression from 'inner' +ExprCP +importExpr(ExprCP expr, const ColumnVector& outer, const ExprVector& inner) { + if (!expr) { + return nullptr; + } + switch (expr->type()) { + case PlanType::kColumn: + for (auto i = 0; i < inner.size(); ++i) { + if (outer[i] == expr) { + return inner[i]; + } + } + return expr; + case PlanType::kLiteral: + return expr; + case PlanType::kCall: + case PlanType::kAggregate: { + auto children = expr->children(); + std::vector newChildren(children.size()); + FunctionSet functions; + bool anyChange = false; + for (auto i = 0; i < children.size(); ++i) { + newChildren[i] = importExpr(children[i]->as(), outer, inner); + anyChange |= newChildren[i] != children[i]; + if (newChildren[i]->isFunction()) { + functions = functions | newChildren[i]->as()->functions(); + } + } + ExprCP newCondition = nullptr; + if (expr->type() == PlanType::kAggregate) { + newCondition = + importExpr(expr->as()->condition(), outer, inner); + anyChange |= newCondition != expr->as()->condition(); + + if (newCondition && newCondition->isFunction()) { + functions = functions | newCondition->as()->functions(); + } + } + if (!anyChange) { + return expr; + } + ExprVector childVector; + childVector.insert( + childVector.begin(), newChildren.begin(), newChildren.end()); + if (expr->type() == PlanType::kCall) { + auto call = expr->as(); + auto* copy = make( + call->name(), call->value(), std::move(childVector), functions); + return copy; + } else if (expr->type() == PlanType::kAggregate) { + auto aggregate = expr->as(); + auto* copy = make( + aggregate->name(), + aggregate->value(), + std::move(childVector), + functions, + aggregate->isDistinct(), + newCondition, + aggregate->isAccumulator(), + aggregate->intermediateType()); + return copy; + } + } + FMT_FALLTHROUGH; + default: + VELOX_UNREACHABLE(); + } +} + +PlanObjectCP otherSide(JoinEdgeP join, PlanObjectCP side) { + if (side == join->leftTable()) { + return join->rightTable(); + } else if (join->rightTable() == side) { + return join->leftTable(); + } + return nullptr; +} + +bool isProjected(PlanObjectCP table, PlanObjectSet columns) { + bool projected = false; + columns.forEach([&](PlanObjectCP column) { + projected |= column->as()->relation() == table; + }); + return projected; +} + +// True if 'join' has max 1 match for a row of 'side'. +bool isUnique(JoinEdgeP join, PlanObjectCP side) { + return join->sideOf(side, true).isUnique; +} + +// Returns a join partner of 'startin 'joins' ' where the partner is +// not in 'visited' Sets 'isFullyImported' to false if the partner is +// not guaranteed n:1 reducing or has columns that are projected out. +PlanObjectCP nextJoin( + PlanObjectCP start, + const JoinEdgeVector& joins, + PlanObjectSet columns, + PlanObjectSet visited, + bool& fullyImported) { + for (auto& join : joins) { + auto other = otherSide(join, start); + if (!other) { + continue; + } + if (visited.contains(other)) { + continue; + } + if (!isUnique(join, other) || isProjected(other, columns)) { + fullyImported = false; + } + return other; + } + return nullptr; +} + +void joinChain( + PlanObjectCP start, + const JoinEdgeVector& joins, + PlanObjectSet columns, + PlanObjectSet visited, + bool& fullyImported, + std::vector& path) { + auto next = nextJoin(start, joins, columns, visited, fullyImported); + if (!next) { + return; + } + visited.add(next); + path.push_back(next); + joinChain(next, joins, columns, visited, fullyImported, path); +} + +JoinEdgeP importedJoin( + JoinEdgeP join, + PlanObjectCP other, + ExprCP innerKey, + bool fullyImported) { + auto left = singleTable(innerKey); + VELOX_CHECK(left); + auto otherKey = join->sideOf(other).keys[0]; + auto* newJoin = QGC_MAKE_IN_ARENA(JoinEdge)( + left, other, {}, false, false, !fullyImported, false); + newJoin->addEquality(innerKey, otherKey); + return newJoin; +} + +JoinEdgeP importedDtJoin( + JoinEdgeP join, + DerivedTableP dt, + ExprCP innerKey, + bool fullyImported) { + auto left = singleTable(innerKey); + VELOX_CHECK(left); + auto otherKey = dt->columns[0]; + auto* newJoin = QGC_MAKE_IN_ARENA(JoinEdge)( + left, dt, {}, false, false, !fullyImported, false); + newJoin->addEquality(innerKey, otherKey); + return newJoin; +} + +template +void eraseFirst(V& set, E element) { + auto it = std::find(set.begin(), set.end(), element); + if (it != set.end()) { + set.erase(it); + } else { + LOG(INFO) << "suspect erase"; + } +} + +void DerivedTable::makeProjection(ExprVector exprs) { + auto optimization = queryCtx()->optimization(); + for (auto& expr : exprs) { + auto* column = + make(optimization->newCName("ec"), this, expr->value()); + columns.push_back(column); + this->exprs.push_back(expr); + } +} + +void DerivedTable::importJoinsIntoFirstDt(const DerivedTable* firstDt) { + if (tables.size() == 1 && tables[0]->type() == PlanType::kDerivedTable) { + flattenDt(tables[0]->as()); + return; + } + auto initialTables = tables; + if (firstDt->limit != -1 || firstDt->orderBy) { + // tables can't be imported but are marked as used so not tried again. + for (auto i = 1; i < tables.size(); ++i) { + importedExistences.add(tables[i]); + } + return; + } + auto& outer = firstDt->columns; + auto& inner = firstDt->exprs; + PlanObjectSet projected; + for (auto& expr : exprs) { + projected.unionColumns(expr); + } + + auto* newFirst = make(*firstDt->as()); + newFirst->cname = firstDt->as()->cname; + for (auto& join : joins) { + auto other = otherSide(join, firstDt); + if (!other) { + continue; + } + if (!tableSet.contains(other)) { + // Already placed in some previous join chain. + continue; + } + auto side = join->sideOf(firstDt); + if (side.keys.size() > 1 || !join->filter().empty()) { + continue; + } + auto innerKey = importExpr(side.keys[0], outer, inner); + assert(innerKey); + if (innerKey->containsFunction(FunctionSet::kAggregate)) { + // If the join key is an aggregate, the join can't be moved below the agg. + continue; + } + auto otherSide = join->sideOf(firstDt, true); + PlanObjectSet visited; + visited.add(firstDt); + visited.add(other); + std::vector path; + bool fullyImported = otherSide.isUnique; + joinChain(other, joins, projected, visited, fullyImported, path); + if (path.empty()) { + if (other->type() == PlanType::kDerivedTable) { + const_cast(other)->as()->makeInitialPlan(); + } + + newFirst->tables.push_back(other); + newFirst->tableSet.add(other); + newFirst->joins.push_back( + importedJoin(join, other, innerKey, fullyImported)); + if (fullyImported) { + newFirst->fullyImported.add(other); + } + } else { + auto* chainDt = make(); + PlanObjectSet chainSet; + chainSet.add(other); + if (fullyImported) { + newFirst->fullyImported.add(other); + } + for (auto& object : path) { + chainSet.add(object); + if (fullyImported) { + newFirst->fullyImported.add(object); + } + } + chainDt->makeProjection(otherSide.keys); + chainDt->import(*this, other, chainSet, {}, 1); + chainDt->makeInitialPlan(); + newFirst->tables.push_back(chainDt); + newFirst->tableSet.add(chainDt); + newFirst->joins.push_back( + importedDtJoin(join, chainDt, innerKey, fullyImported)); + } + eraseFirst(tables, other); + tableSet.erase(other); + for (auto& table : path) { + eraseFirst(tables, table); + tableSet.erase(table); + } + } + + VELOX_CHECK_EQ(tables.size(), 1); + for (auto i = 0; i < initialTables.size(); ++i) { + if (!newFirst->fullyImported.contains(initialTables[i])) { + newFirst->importedExistences.add(initialTables[i]); + } + } + tables[0] = newFirst; + flattenDt(newFirst); +} + +void DerivedTable::flattenDt(const DerivedTable* dt) { + tables = dt->tables; + cname = dt->cname; + tableSet = dt->tableSet; + joins = dt->joins; + columns = dt->columns; + exprs = dt->exprs; + fullyImported = dt->fullyImported; + importedExistences.unionSet(dt->importedExistences); + aggregation = dt->aggregation; + having = dt->having; +} + +void BaseTable::addFilter(ExprCP expr) { + auto columns = expr->columns(); + bool isMultiColumn = false; + bool isSingleColumn = false; + columns.forEach([&](PlanObjectCP object) { + if (!isMultiColumn) { + if (isSingleColumn) { + isMultiColumn = true; + } else { + isSingleColumn = true; + } + }; + }); + if (isSingleColumn) { + columnFilters.push_back(expr); + filterUpdated(this); + return; + } + filter.push_back(expr); + filterUpdated(this); +} + +// Finds a JoinEdge between tables[0] and tables[1]. Sets tables[0] to the left +// and [1] to the right table of the found join. Returns the JoinEdge. If +// 'create' is true and no edge is found, makes a new edge with tables[0] as +// left and [1] as right. +JoinEdgeP +findJoin(DerivedTableP dt, std::vector& tables, bool create) { + for (auto& join : dt->joins) { + if (join->leftTable() == tables[0] && join->rightTable() == tables[1]) { + return join; + } + if (join->leftTable() == tables[1] && join->rightTable() == tables[0]) { + std::swap(tables[0], tables[1]); + return join; + } + } + if (create) { + auto* join = QGC_MAKE_IN_ARENA(JoinEdge)( + tables[0], tables[1], {}, false, false, false, false); + dt->joins.push_back(join); + return join; + } + return nullptr; +} + +// True if 'expr' is of the form a = b where a depends on one of ''tables' and b +// on the other. If true, returns the side depending on tables[0] in 'left' and +// the other in 'right'. +bool isJoinEquality( + ExprCP expr, + std::vector& tables, + ExprCP& left, + ExprCP& right) { + if (expr->type() == PlanType::kCall) { + auto call = expr->as(); + if (call->name() == toName("eq")) { + left = call->args()[0]; + right = call->args()[1]; + auto leftTable = singleTable(left); + auto rightTable = singleTable(right); + if (!leftTable || !rightTable) { + return false; + } + if (leftTable == tables[1]) { + std::swap(left, right); + } + return true; + } + } + return false; +} + +void DerivedTable::distributeConjuncts() { + std::vector changedDts; + for (auto i = 0; i < conjuncts.size(); ++i) { + PlanObjectSet tableSet = conjuncts[i]->allTables(); + std::vector tables; + tableSet.forEachMutable([&](auto table) { tables.push_back(table); }); + if (tables.size() == 1) { + if (tables[0] == this) { + continue; // the conjunct depends on containing dt, like grouping or + // existence flags. Leave in place. + } else if (tables[0]->type() == PlanType::kDerivedTable) { + // Translate the column names and add the condition to the conjuncts in + // the dt. + VELOX_NYI(); + } else { + VELOX_CHECK(tables[0]->type() == PlanType::kTable); + tables[0]->as()->addFilter(conjuncts[i]); + } + conjuncts.erase(conjuncts.begin() + i); + --i; + continue; + } + if (tables.size() == 2) { + ExprCP left = nullptr; + ExprCP right = nullptr; + // expr depends on 2 tables. If it is left = right or right = left and + // there is no edge or the edge is inner, add the equality. For other + // cases, leave the conjunct in place, to be evaluated when its + // dependences are known. + if (isJoinEquality(conjuncts[i], tables, left, right)) { + auto join = findJoin(this, tables, true); + if (join->isInner()) { + if (left->type() == PlanType::kColumn && + right->type() == PlanType::kColumn) { + left->as()->equals(right->as()); + } + if (join->leftTable() == tables[0]) { + join->addEquality(left, right); + } else { + join->addEquality(right, left); + } + conjuncts.erase(conjuncts.begin() + i); + --i; + } + } + } + } + // Re-guess fanouts after all single table filters are pushed down. + for (auto& join : joins) { + join->guessFanout(); + } +} + +void DerivedTable::makeInitialPlan() { + auto optimization = queryCtx()->optimization(); + MemoKey key; + key.firstTable = this; + key.tables.add(this); + for (auto& column : columns) { + key.columns.add(column); + } + bool found = false; + auto it = optimization->memo().find(key); + if (it != optimization->memo().end()) { + found = true; + } + distributeConjuncts(); + addImpliedJoins(); + linkTablesToJoins(); + setStartTables(); + PlanState state(*optimization, this); + for (auto expr : exprs) { + state.targetColumns.unionColumns(expr); + } + + optimization->makeJoins(nullptr, state); + Distribution emptyDistribution; + bool needsShuffle; + auto plan = state.plans.best(emptyDistribution, needsShuffle)->op; + auto& distribution = plan->distribution(); + ExprVector partition = distribution.partition; + ExprVector order = distribution.order; + auto orderType = distribution.orderType; + replace(partition, exprs, columns.data()); + replace(order, exprs, columns.data()); + auto* dtDist = make( + distribution.distributionType, + distribution.cardinality, + partition, + order, + orderType); + this->distribution = dtDist; + if (!found) { + optimization->memo()[key] = std::move(state.plans); + } +} + +std::string DerivedTable::toString() const { + std::stringstream out; + out << "{dt " << cname << " from "; + for (auto& table : tables) { + out << table->toString() << " "; + } + out << " where "; + for (auto& join : joins) { + out << join->toString(); + } + if (!conjuncts.empty()) { + out << " where " << conjunctsToString(conjuncts); + } + out << "}"; + return out.str(); +} + +float tableCardinality(PlanObjectCP table) { + if (table->type() == PlanType::kTable) { + return table->as() + ->schemaTable->columnGroups[0] + ->distribution() + .cardinality; + } + VELOX_CHECK(table->type() == PlanType::kDerivedTable); + return table->as()->distribution->cardinality; +} + +void JoinEdge::guessFanout() { + if (fanoutsFixed_) { + return; + } + auto left = joinCardinality(leftTable_, toRangeCast(leftKeys_)); + auto right = joinCardinality(rightTable_, toRangeCast(rightKeys_)); + leftUnique_ = left.unique; + rightUnique_ = right.unique; + lrFanout_ = right.joinCardinality * baseSelectivity(rightTable_); + rlFanout_ = left.joinCardinality * baseSelectivity(leftTable_); + // If one side is unique, the other side is a pk to fk join, with fanout = + // fk-table-card / pk-table-card. + if (rightUnique_) { + lrFanout_ = baseSelectivity(rightTable_); + rlFanout_ = tableCardinality(leftTable_) / tableCardinality(rightTable_) * + baseSelectivity(leftTable_); + } + if (leftUnique_) { + rlFanout_ = baseSelectivity(leftTable_); + lrFanout_ = tableCardinality(rightTable_) / tableCardinality(leftTable_) * + baseSelectivity(rightTable_); + } +} + +void exprsToString(const ExprVector& exprs, std::stringstream& out) { + int32_t size = exprs.size(); + for (auto i = 0; i < size; ++i) { + out << exprs[i]->toString() << (i < size - 1 ? ", " : ""); + } +} + +} // namespace facebook::velox::optimizer diff --git a/verax/QueryGraph.h b/verax/QueryGraph.h new file mode 100644 index 0000000..b4a4072 --- /dev/null +++ b/verax/QueryGraph.h @@ -0,0 +1,755 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "optimizer/Schema.h" //@manual + +#include "velox/core/PlanNode.h" + +/// Defines subclasses of PlanObject for describing the logical +/// structure of queries. These are the constraints that guide +/// generation of plan candidates. These are referenced from +/// candidates but stay immutable acrosss the candidate +/// generation. Sometimes new derived tables may be added for +/// representing constraints on partial plans but otherwise these stay +/// constant. +namespace facebook::velox::optimizer { + +/// The join structure is described as a tree of derived tables with +/// base tables as leaves. Joins are described as join graph +/// edges. Edges describe direction for non-inner joins. Scalar and +/// existence subqueries are flattened into derived tables or base +/// tables. The join graph would represent select ... from t where +/// exists(x) or exists(y) as a derived table of three joined tables +/// where the edge from t to x and t to y is directed and qualified as +/// left semijoin. The semijoins project out one column, an existence +/// flag. The filter would be expresssed as a conjunct under the top +/// derived table with x-exists or y-exists. + +/// Superclass for all expressions. +class Expr : public PlanObject { + public: + Expr(PlanType type, const Value& value) : PlanObject(type), value_(value) {} + + bool isExpr() const override { + return true; + } + + // Returns the single base or derived table 'this' depends on, nullptr if + // 'this' depends on none or multiple tables. + PlanObjectCP singleTable() const; + + /// Returns all tables 'this' depends on. + PlanObjectSet allTables() const; + + /// True if '&other == this' or is recursively equal with column + /// leaves either same or in same equivalence. + bool sameOrEqual(const Expr& other) const; + + const PlanObjectSet& columns() const { + return columns_; + } + + const Value& value() const { + return value_; + } + + /// True if 'this' contains any function from 'set'. See FunctionSet. + virtual bool containsFunction(uint64_t /*set*/) const { + return false; + } + + protected: + // The columns this depends on. + PlanObjectSet columns_; + + // Type Constraints on the value of 'this'. + Value value_; +}; + +/// If 'object' is an Expr, returns Expr::singleTable, else nullptr. +PlanObjectCP singleTable(PlanObjectCP object); + +struct Equivalence; +using EquivalenceP = Equivalence*; + +/// Represents a literal. +class Literal : public Expr { + public: + Literal(const Value& value, const velox::variant* literal) + : Expr(PlanType::kLiteral, value), literal_(literal) {} + + const velox::variant& literal() const { + return *literal_; + } + + private: + const velox::variant* const literal_; +}; + +/// Represents a column. A column is always defined by a relation, whether table +/// or derived table. +class Column : public Expr { + public: + Column(Name _name, PlanObjectP _relation, const Value& value); + + Name name() const { + return name_; + } + + PlanObjectCP relation() const { + return relation_; + } + + ColumnCP schemaColumn() const { + return schemaColumn_; + } + + /// Asserts that 'this' and 'other' are joined on equality. This has a + /// transitive effect, so if a and b are previously asserted equal and c is + /// asserted equal to b, a and c are also equal. + void equals(ColumnCP other) const; + + std::string toString() const override; + + struct Equivalence* equivalence() const { + return equivalence_; + } + + private: + // Last part of qualified name. + Name name_; + + // The defining BaseTable or DerivedTable. + PlanObjectP relation_; + + // Equivalence class. Lists all columns directly or indirectly asserted equal + // to 'this'. + mutable EquivalenceP equivalence_{nullptr}; + + // If this is a column of a BaseTable, points to the corresponding + // column in the SchemaTable. Used for matching with + // ordering/partitioning columns in the SchemaTable. + ColumnCP schemaColumn_{nullptr}; +}; + +template +inline folly::Range toRange(const std::vector>& v) { + return folly::Range(v.data(), v.size()); +} + +template +inline CPSpan toRangeCast(U v) { + return CPSpan(reinterpret_cast(v.data()), v.size()); +} + +/// A bit set that qualifies a function call. Represents which functions/kinds +/// of functions are found inside the children of a function call. +class FunctionSet { + public: + /// Indicates and aggregate function in the set. + static constexpr uint64_t kAggregate = 1; + + FunctionSet() : set_(0) {} + explicit FunctionSet(uint64_t set) : set_(set) {} + + /// True if 'item' is in 'this'. + bool contains(int64_t item) const { + return 0 != (set_ & (1UL << item)); + } + + /// Unions 'this' and 'other' and returns the result. + FunctionSet operator|(const FunctionSet& other) const { + return FunctionSet(set_ | other.set_); + } + + /// Unions 'this' and 'other' and returns the result. + FunctionSet operator|(uint64_t other) const { + return FunctionSet(set_ | other); + } + + private: + uint64_t set_; +}; + +/// Represents a function call or a special form, any expression with +/// subexpressions. +class Call : public Expr { + public: + Call( + PlanType type, + Name name, + const Value& value, + ExprVector args, + FunctionSet functions) + : Expr(type, value), + name_(name), + args_(std::move(args)), + functions_(functions) { + for (auto arg : args_) { + columns_.unionSet(arg->columns()); + } + } + + Call(Name name, Value value, ExprVector args, FunctionSet functions) + : Call(PlanType::kCall, name, value, args, functions) {} + + Name name() const { + return name_; + } + + const FunctionSet functions() const { + return functions_; + } + + bool isFunction() const override { + return true; + } + + bool containsFunction(uint64_t set) const override { + return functions_.contains(set); + } + + const ExprVector& args() const { + return args_; + } + + CPSpan children() const override { + return folly::Range( + reinterpret_cast(args_.data()), args_.size()); + } + + std::string toString() const override; + + private: + // name of function. + Name const name_; + + // Arguments. + const ExprVector args_; + + // Set of functions used in 'this' and 'args'. + const FunctionSet functions_; +}; + +using CallCP = const Call*; + +/// Represens a set of transitively equal columns. +struct Equivalence { + // Each element has a direct or implied equality edge to every other. + ColumnVector columns; +}; + +/// Represents one side of a join. See Join below for the meaning of the +/// members. +struct JoinSide { + PlanObjectCP table; + const ExprVector& keys; + float fanout; + const bool isOptional; + const bool isExists; + const bool isNotExists; + ColumnCP markColumn; + const bool isUnique; + + /// Returns the join type to use if 'this' is the right side. + velox::core::JoinType leftJoinType() const { + if (isNotExists) { + return velox::core::JoinType::kAnti; + } + if (isExists) { + return velox::core::JoinType::kLeftSemiFilter; + } + if (isOptional) { + return velox::core::JoinType::kLeft; + } + if (markColumn) { + return velox::core::JoinType::kLeftSemiProject; + } + return velox::core::JoinType::kInner; + } +}; + +/// Represents a possibly directional equality join edge. +/// 'rightTable' is always set. 'leftTable' is nullptr if 'leftKeys' come from +/// different tables. If so, 'this' must be non-inner and not full outer. +/// 'filter' is a list of post join conjuncts. This should be present only in +/// non-inner joins. For inner joins these are representable as freely +/// decomposable and reorderable conjuncts. +class JoinEdge { + public: + JoinEdge( + PlanObjectCP leftTable, + PlanObjectCP rightTable, + ExprVector filter, + bool leftOptional, + bool rightOptional, + bool rightExists, + bool rightNotExists, + ColumnCP markColumn = nullptr) + : leftTable_(leftTable), + rightTable_(rightTable), + filter_(std::move(filter)), + leftOptional_(leftOptional), + rightOptional_(rightOptional), + rightExists_(rightExists), + rightNotExists_(rightNotExists), + markColumn_(markColumn) { + if (isInner()) { + VELOX_CHECK(filter_.empty()); + } + } + + PlanObjectCP leftTable() const { + return leftTable_; + } + + PlanObjectCP rightTable() const { + return rightTable_; + } + + const ExprVector& leftKeys() const { + return leftKeys_; + } + + const ExprVector& rightKeys() const { + return rightKeys_; + } + + float lrFanout() const { + return lrFanout_; + } + + bool leftOptional() const { + return leftOptional_; + } + + bool rightOptional() const { + return rightOptional_; + } + + void addEquality(ExprCP left, ExprCP right); + + /// True if inner join. + bool isInner() const { + return !leftOptional_ && !rightOptional_ && !rightExists_ && + !rightNotExists_; + } + + // True if all tables referenced from 'leftKeys' must be placed before placing + // this. + bool isNonCommutative() const { + // Inner and full outer joins are commutative. + if (rightOptional_ && leftOptional_) { + return false; + } + return !leftTable_ || rightOptional_ || leftOptional_ || rightExists_ || + rightNotExists_ || markColumn_; + } + // Returns the join side info for 'table'. If 'other' is set, returns the + // other side. + const JoinSide sideOf(PlanObjectCP side, bool other = false) const; + + /// Returns the table on the otherside of 'table' and the number of rows in + /// the returned table for one row in 'table'. If the join is not inner + /// returns nullptr, 0. + std::pair otherTable(PlanObjectCP table) const { + return leftTable_ == table && !leftOptional_ + ? std::pair{rightTable_, lrFanout_} + : rightTable_ == table && !rightOptional_ && !rightExists_ + ? std::pair{leftTable_, rlFanout_} + : std::pair{nullptr, 0}; + } + + const ExprVector& filter() const { + return filter_; + } + + void setFanouts(float rl, float lr) { + fanoutsFixed_ = true; + lrFanout_ = lr; + rlFanout_ = rl; + } + + std::string toString() const; + + //// Fills in 'lrFanout' and 'rlFanout', 'leftUnique', 'rightUnique'. + void guessFanout(); + + // True if a hash join build can be broadcastable. Used when building on the + // right. None of the right hash join variants is broadcastable. + bool isBroadcastableType() const; + + private: + // Leading left side join keys. + ExprVector leftKeys_; + // Leading right side join keys, compared equals to 1:1 to 'leftKeys'. + ExprVector rightKeys_; + + PlanObjectCP const leftTable_; + PlanObjectCP const rightTable_; + + // 'rightKeys' select max 1 'leftTable' row. + bool leftUnique_{false}; + + // 'leftKeys' select max 1 'rightTable' row. + bool rightUnique_{false}; + + // number of right side rows selected for one row on the left. + float lrFanout_{1}; + + // Number of left side rows selected for one row on the right. + float rlFanout_{1}; + + // True if 'lrFanout_' and 'rlFanout_' are set by setFanouts. + bool fanoutsFixed_{false}; + + // Join condition for any non-equality conditions for non-inner joins. + const ExprVector filter_; + + // True if an unprobed right side row produces a result with right side + // columns set and left side columns as null. Possible only be hash or + // merge. + const bool leftOptional_; + + // True if a right side miss produces a row with left side columns + // and a null for right side columns (left outer join). A full outer + // join has both left and right optional. + const bool rightOptional_; + + // True if the right side is only checked for existence of a match. If + // rightOptional is set, this can project out a null for misses. + const bool rightExists_; + + // True if produces a result for left if no match on the right. + const bool rightNotExists_; + + // Flag to set if right side has a match. + const ColumnCP markColumn_; +}; + +using JoinEdgeP = JoinEdge*; + +using JoinEdgeVector = std::vector>; + +/// Adds 'element' to 'vector' if it is not in it. +template +inline void pushBackUnique(V& vector, E& element) { + if (std::find(vector.begin(), vector.end(), element) == vector.end()) { + vector.push_back(element); + } +} + +/// Represents a reference to a table from a query. The There is one of these +/// for each occurrence of the schema table. A TableScan references one +/// baseTable but the same BaseTable can be referenced from many TableScans, for +/// example if accessing different indices in a secondary to primary key lookup. +struct BaseTable : public PlanObject { + BaseTable() : PlanObject(PlanType::kTable) {} + + // Correlation name, distinguishes between uses of the same schema table. + Name cname{nullptr}; + + SchemaTableCP schemaTable{nullptr}; + + /// All columns referenced from 'schemaTable' under this correlation name. + /// Different indices may have to be combined in different TableScans to cover + /// 'columns'. + ColumnVector columns; + + // All joins where 'this' is an end point. + JoinEdgeVector joinedBy; + + // Top level conjuncts on single columns and literals, column to the left. + ExprVector columnFilters; + + // Multicolumn filters dependent on 'this' alone. + ExprVector filter; + + // the fraction of base table rows selected by all filters involving this + // table only. + float filterSelectivity{1}; + + bool isTable() const override { + return true; + } + + void addJoinedBy(JoinEdgeP join) { + pushBackUnique(joinedBy, join); + } + + /// Adds 'expr' to 'filters' or 'columnFilters'. + void addFilter(ExprCP expr); + + std::string toString() const override; +}; + +using BaseTableCP = const BaseTable*; + +using TypeVector = + std::vector>; + +// Aggregate function. The aggregation and arguments are in the +// inherited Call. The Value pertains to the aggregation +// result or accumulator. +class Aggregate : public Call { + public: + Aggregate( + Name name, + const Value& value, + ExprVector args, + FunctionSet functions, + bool isDistinct, + ExprCP condition, + bool isAccumulator, + const velox::Type* intermediateType) + : Call( + PlanType::kAggregate, + name, + value, + std::move(args), + functions | FunctionSet::kAggregate), + isDistinct_(isDistinct), + condition_(condition), + isAccumulator_(isAccumulator), + intermediateType_(intermediateType) { + for (auto& arg : this->args()) { + rawInputType_.push_back(arg->value().type); + } + if (condition_) { + columns_.unionSet(condition_->columns()); + } + } + + ExprCP condition() const { + return condition_; + } + + bool isDistinct() const { + return isDistinct_; + } + + bool isAccumulator() const { + return isAccumulator_; + } + + const velox::Type* intermediateType() const { + return intermediateType_; + } + + const TypeVector rawInputType() const { + return rawInputType_; + } + + private: + bool isDistinct_; + ExprCP condition_; + bool isAccumulator_; + const velox::Type* intermediateType_; + TypeVector rawInputType_; +}; + +using AggregateCP = const Aggregate*; + +struct Aggregation; +using AggregationP = Aggregation*; + +/// Wraps an Aggregation RelationOp. This gives the aggregation a PlanObject id +struct AggregationPlan : public PlanObject { + AggregationPlan(AggregationP agg) + : PlanObject(PlanType::kAggregate), aggregation(agg) {} + + AggregationP aggregation; +}; + +using AggregationPlanCP = const AggregationPlan*; + +struct OrderBy; +using OrderByP = OrderBy*; + +/// Represents a derived table, i.e. a select in a from clause. This is the +/// basic unit of planning. Derived tables can be merged and split apart from +/// other ones. Join types and orders are decided within each derived table. A +/// derived table is likewise a reorderable unit inside its parent derived +/// table. Joins can move between derived tables within limits, considering the +/// semantics of e.g. group by. +struct DerivedTable : public PlanObject { + DerivedTable() : PlanObject(PlanType::kDerivedTable) {} + + // Distribution that gives partition, cardinality and + // order/uniqueness for the dt alone. This is expressed in terms of + // outside visible 'columns'. Actual uses of the dt in candidate + // plans may be modified from this by e.g. importing restrictions + // from enclosing query. Set for a non-top level dt. + Distribution* distribution{nullptr}; + + // Correlation name. + Name cname{nullptr}; + + // Columns projected out. Visible in the enclosing query. + ColumnVector columns; + + // Exprs projected out.1:1 to 'columns'. + ExprVector exprs; + + // References all joins where 'this' is an end point. + JoinEdgeVector joinedBy; + + // All tables in from, either Table or DerivedTable. If Table, all + // filters resolvable with the table alone are in single column filters or + // 'filter' of BaseTable. + std::vector> tables; + + // Repeats the contents of 'tables'. Used for membership check. A DerivedTable + // can be a subset of another, for example when planning a join for a build + // side. In this case joins that refer to tables not in 'tableSet' are not + // considered. + PlanObjectSet tableSet; + + // Single row tables from non-correlated scalar subqueries. + PlanObjectSet singleRowDts; + + // Tables that are not to the right sides of non-commutative joins. + PlanObjectSet startTables; + + // Joins between 'tables'. + JoinEdgeVector joins; + + // Filters in where for that are not single table expressions and not join + // filters of explicit joins and not equalities between columns of joined + // tables. + ExprVector conjuncts; + + // Set of reducing joined tables imported to reduce build size. Set if 'this' + // represents a build side join. + PlanObjectSet importedExistences; + + // The set of tables in import() '_tables' that are fully covered by this dt + // and need not be considered outside of it. If 'firstTable' in import is a + // group by dt, for example, some joins may be imported as reducing existences + // but will still have to be considered by the enclosing query. Such tables + // are not included in 'fullyImported' If 'firstTable' in import is a base + // table, then 'fullyImported' is '_tables'. + PlanObjectSet fullyImported; + + // + // True if this dt is already a reducing join imported to a build side. Do not + // try to further restrict this with probe side. + bool noImportOfExists{false}; + // Postprocessing clauses, group by, having, order by, limit, offset. + AggregationPlanCP aggregation{nullptr}; + ExprVector having; + OrderByP orderBy{nullptr}; + int32_t limit{-1}; + int32_t offset{0}; + + /// Adds an equijoin edge between 'left' and 'right'. The flags correspond to + /// the like-named members in Join. + void addJoinEquality( + ExprCP left, + ExprCP right, + const ExprVector& filter, + bool leftOptional, + bool rightOptional, + bool rightExists, + bool rightNotExists); + + // after 'joins' is filled in, links tables to their direct and + // equivalence-implied joins. + void linkTablesToJoins(); + + /// Completes 'joins' with edges implied by column equivalences. + void addImpliedJoins(); + + /// Initializes 'this' to join 'tables' from 'super'. Adds the joins from + /// 'existences' as semijoins to limit cardinality when making a hash join + /// build side. Allows importing a reducing join from probe to build. + /// 'firstTable' is the joined table that is restricted by the other tables in + /// 'tables' and 'existences'. 'existsFanout' us the reduction from joining + /// 'firstTable' with 'existences'. + void import( + const DerivedTable& super, + PlanObjectCP firstTable, + const PlanObjectSet& tables, + const std::vector& existences, + float existsFanout = 1); + + bool isTable() const override { + return true; + } + + //// True if 'table' is of 'this'. + bool hasTable(PlanObjectCP table) const { + return std::find(tables.begin(), tables.end(), table) != tables.end(); + } + + // True if 'join' exists in 'this'. Tables link to joins that may be + // in different speculative candidate dts. So only consider joins + // inside the current dt wen planning. + bool hasJoin(JoinEdgeP join) const { + return std::find(joins.begin(), joins.end(), join) != joins.end(); + } + + /// Fills in 'startTables_' to 'tables_' that are not to the right of + /// non-commutative joins. + void setStartTables(); + + std::string toString() const override; + void addJoinedBy(JoinEdgeP join) { + pushBackUnique(joinedBy, join); + } + + /// Moves suitable elements of 'conjuncts' into join edges or single + /// table filters. May be called repeatedly if enclosing dt's add + /// more conjuncts. May call itself recursively on component dts. + void distributeConjuncts(); + + /// memoizes plans for 'this' and fills in 'distribution_'. Needed + /// before adding 'this' as a join side because join sides must have + /// a cardinality guess. + void makeInitialPlan(); + + private: + // Imports the joins in 'this' inside 'firstDt', which must be a + // member of 'this'. The import is possible if the join is not + // through aggregates in 'firstDt'. On return, all joins that can go + // inside firstDt are imported below aggregation in + // firstDt. 'firstDt' is not modified, its original contents are + // copied in a new dt before the import. + void importJoinsIntoFirstDt(const DerivedTable* firstDt); + + // Sets 'dt' to be the complete contents of 'this'. + void flattenDt(const DerivedTable* dt); + + // Finds single row dts from non-correlated scalar subqueries. + void findSingleRowDts(); + + // Sets 'columns' abd 'exprs'. + void makeProjection(ExprVector exprs); +}; + +using DerivedTableP = DerivedTable*; + +float tableCardinality(PlanObjectCP table); + +/// Returns all distinct tables 'exprs' depend on. +PlanObjectSet allTables(CPSpan exprs); + +/// Appends the string representation of 'exprs' to 'out'. +void exprsToString(const ExprVector& exprs, std::stringstream& out); + +} // namespace facebook::velox::optimizer diff --git a/verax/QueryGraphContext.cpp b/verax/QueryGraphContext.cpp new file mode 100644 index 0000000..cd4f012 --- /dev/null +++ b/verax/QueryGraphContext.cpp @@ -0,0 +1,105 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/QueryGraphContext.h" //@manual + +namespace facebook::velox::optimizer { + +QueryGraphContext*& queryCtx() { + thread_local QueryGraphContext* context; + return context; +} + +PlanObjectP QueryGraphContext::dedup(PlanObjectP object) { + auto pair = deduppedObjects_.insert(object); + return *pair.first; +} + +const char* QueryGraphContext::toName(std::string_view str) { + auto it = names_.find(str); + if (it != names_.end()) { + return it->data(); + } + char* data = allocator_.allocate(str.size() + 1)->begin(); // NOLINT + memcpy(data, str.data(), str.size()); + data[str.size()] = 0; + names_.insert(std::string_view(data, str.size())); + return data; +} + +Name toName(std::string_view string) { + return queryCtx()->toName(string); +} + +const Type* QueryGraphContext::toType(const TypePtr& type) { + return dedupType(type).get(); +} +TypePtr QueryGraphContext::dedupType(const TypePtr& type) { + auto it = deduppedTypes_.find(type); + if (it != deduppedTypes_.end()) { + return *it; + } + auto size = type->size(); + if (size == 0) { + deduppedTypes_.insert(type); + toTypePtr_[type.get()] = type; + return type; + } + std::vector children; + for (auto i = 0; i < size; ++i) { + children.push_back(dedupType(type->childAt(i))); + } + TypePtr newType; + switch (type->kind()) { + case TypeKind::ROW: { + std::vector names; + for (auto i = 0; i < size; ++i) { + names.push_back(type->as().nameOf(i)); + } + newType = ROW(std::move(names), std::move(children)); + break; + } + case TypeKind::ARRAY: + newType = ARRAY(children[0]); + break; + case TypeKind::MAP: + newType = MAP(children[0], children[1]); + break; + default: + VELOX_FAIL("Type has size > 0 and is not row/array/map"); + } + deduppedTypes_.insert(newType); + toTypePtr_[newType.get()] = newType; + return newType; +} + +const TypePtr& QueryGraphContext::toTypePtr(const Type* type) { + auto it = toTypePtr_.find(type); + if (it != toTypePtr_.end()) { + return it->second; + } + VELOX_FAIL("Cannot translate {} back to TypePtr", type->toString()); +} + +const Type* toType(const TypePtr& type) { + return queryCtx()->toType(type); +} + +const TypePtr& toTypePtr(const Type* type) { + return queryCtx()->toTypePtr(type); +} + +} // namespace facebook::velox::optimizer diff --git a/verax/QueryGraphContext.h b/verax/QueryGraphContext.h new file mode 100644 index 0000000..d47888b --- /dev/null +++ b/verax/QueryGraphContext.h @@ -0,0 +1,241 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "optimizer/ArenaCache.h" //@manual +#include "velox/common/memory/HashStringAllocator.h" + +// #define QG_USE_MALLOC +#define QG_CACHE_ARENA + +/// Thread local context and utilities for query planning. + +namespace facebook::velox::optimizer { + +/// Pointer to an arena allocated interned copy of a null terminated string. +/// Used for identifiers. Allows comparing strings by comparing pointers. +using Name = const char*; + +/// Shorthand for a view on an array of T* +template +using CPSpan = folly::Range; + +class PlanObject; + +using PlanObjectP = PlanObject*; +using PlanObjectCP = const PlanObject*; + +struct PlanObjectPHasher { + size_t operator()(const PlanObjectCP& object) const; +}; + +struct PlanObjectPComparer { + bool operator()(const PlanObjectCP& lhs, const PlanObjectCP& rhs) const; +}; + +struct TypeHasher { + size_t operator()(const TypePtr& type) const { + // hash on recursive TypeKind. Structs that differ in field names + // only or decimals with different precisions will collide, no + // other collisions expected. + return type->hashKind(); + } +}; + +struct TypeComparer { + bool operator()(const TypePtr& lhs, const TypePtr& rhs) const { + return *lhs == *rhs; + } +}; + +struct Plan; +using PlanPtr = Plan*; +class Optimization; + +/// Context for making a query plan. Owns all memory associated to +/// planning, except for the input PlanNode tree. The result of +/// planning is also owned by 'this', so the planning result must be +/// copied into a non-owned target specific representation before +/// destroying 'this'. QueryGraphContext is not thread safe and may +/// be accessed from one thread at a time. Memory allocation +/// references this via a thread local through queryCtx(). +class QueryGraphContext { + public: + explicit QueryGraphContext(velox::HashStringAllocator& allocator) + : allocator_(allocator), cache_(allocator_) {} + + /// Returns the interned representation of 'str', i.e. Returns a + /// pointer to a canonical null terminated const char* with the same + /// characters as 'str'. Allows comparing names by comparing + /// pointers. + Name toName(std::string_view str); + + /// Returns a new unique id to use for 'object' and associates 'object' to + /// this id. Tagging objects with integere ids is useful for efficiently + /// representing sets of objects as bitmaps. + int32_t newId(PlanObject* object) { + objects_.push_back(object); + return objects_.size() - 1; + } + + /// Allocates 'size' bytes from the arena of 'this'. The allocation lives + /// until free() is called on it or the arena is destroyed. + void* allocate(size_t size) { +#ifdef QG_TEST_USE_MALLOC + // Benchmark-only. Dropping the arena will not free un-free'd allocs. + return ::malloc(size); +#elif defined(QG_CACHE_ARENA) + return cache_.allocate(size); +#else + return allocator_.allocate(size)->begin(); +#endif + } + + /// Frees ptr, which must have been allocated with allocate() above. Calling + /// this is not mandatory since objects from the arena get freed at latest + /// when the arena is destroyed. + void free(void* ptr) { +#ifdef QG_TEST_USE_MALLOC + ::free(ptr); +#elif defined(QG_CACHE_ARENA) + cache_.free(ptr); +#else + allocator_.free(velox::HashStringAllocator::headerOf(ptr)); +#endif + } + + /// Returns a canonical instance for all logically equal values of 'object'. + /// Returns 'object' on first call with object, thereafter the same physical + /// object if the argument is equal. + PlanObjectP dedup(PlanObjectP object); + + /// Returns the object associated to 'id'. See newId() + PlanObjectCP objectAt(int32_t id) { + return objects_[id]; + } + + PlanObjectP mutableObjectAt(int32_t id) { + return objects_[id]; + } + + /// Returns the top level plan being processed when printing operator trees. + /// If non-null, allows showing percentages. + Plan*& contextPlan() { + return contextPlan_; + } + + /// The top level Optimization instance. + Optimization*& optimization() { + return optimization_; + } + + // Records the use of a TypePtr in optimization. Returns a canonical + // representative of the type, allowing pointer equality for exact match. + // Allows mapping from the Type* back to TypePtr. + const Type* toType(const velox::TypePtr& type); + + /// Returns the canonical TypePtr corresponding to 'type'. 'type' must have + /// been previously returned by toType(). + const TypePtr& toTypePtr(const Type* type); + + private: + TypePtr dedupType(const TypePtr& type); + + velox::HashStringAllocator& allocator_; + ArenaCache cache_; + + // PlanObjects are stored at the index given by their id. + std::vector objects_; + + // Set of interned copies of identifiers. insert() into this returns the + // canonical interned copy of any string. Lifetime is limited to 'allocator_'. + std::unordered_set names_; + + // Set for deduplicating planObject trees. + std::unordered_set + deduppedObjects_; + + std::unordered_set deduppedTypes_; + // Maps raw Type* back to shared TypePtr. Used in toType()() and toTypePtr(). + std::unordered_map toTypePtr_; + + Plan* contextPlan_{nullptr}; + Optimization* optimization_{nullptr}; +}; + +/// Returns a mutable reference to the calling thread's QueryGraphContext. +QueryGraphContext*& queryCtx(); + +template +inline _Tp* make(_Args&&... __args) { + return new (queryCtx()->allocate(sizeof(_Tp))) + _Tp(std::forward<_Args>(__args)...); +} + +/// Macro to use instead of make() when make() errors out from too +/// many arguments. +#define QGC_MAKE_IN_ARENA(_Tp) new (queryCtx()->allocate(sizeof(_Tp))) _Tp + +/// Converts std::string to name used in query graph objects. raw pointer to +/// arena allocated const chars. +// Name toName(const std::string& string); +Name toName(std::string_view string); + +/// Shorthand for toType() in thread's QueryGraphContext. +const Type* toType(const TypePtr& type); +/// Shorthand for toTypePtr() in thread's QueryGraphContext. +const TypePtr& toTypePtr(const Type* type); + +/// STL compatible allocator that manages std:: containers allocated in the +/// QueryGraphContext arena. +template +struct QGAllocator { + using value_type = T; + QGAllocator() = default; + + template + explicit QGAllocator(QGAllocator) {} + + T* allocate(std::size_t n) { + return reinterpret_cast( + queryCtx()->allocate(velox::checkedMultiply(n, sizeof(T)))); // NOLINT + } + + void deallocate(T* p, std::size_t /*n*/) noexcept { + queryCtx()->free(p); + } + + friend bool operator==( + const QGAllocator& /*lhs*/, + const QGAllocator& /*rhs*/) { + return true; + } + + friend bool operator!=(const QGAllocator& lhs, const QGAllocator& rhs) { + return !(lhs == rhs); + } +}; + +// Forward declarations of common types and collections. +class Expr; +using ExprCP = const Expr*; +class Column; +using ColumnCP = const Column*; +using ExprVector = std::vector>; +using ColumnVector = std::vector>; + +} // namespace facebook::velox::optimizer diff --git a/verax/RelationOp.cpp b/verax/RelationOp.cpp new file mode 100644 index 0000000..0042ef6 --- /dev/null +++ b/verax/RelationOp.cpp @@ -0,0 +1,266 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/Plan.h" //@manual +#include "optimizer/PlanUtils.h" //@manual +#include "optimizer/QueryGraph.h" //@manual +#include "velox/common/base/SuccinctPrinter.h" + +namespace facebook::velox::optimizer { + +const Value& RelationOp::value(ExprCP expr) const { + // Compute new Value by applying restrictions from operators + // between the place Expr is first defined and the output of + // 'this'. Memoize the result in 'this'. + return expr->value(); +} + +std::string RelationOp::toString(bool recursive, bool detail) const { + if (input_ && recursive) { + return input_->toString(true, detail); + } + return ""; +} + +// static +Distribution TableScan::outputDistribution( + const BaseTable* baseTable, + ColumnGroupP index, + const ColumnVector& columns) { + auto schemaColumns = transform( + columns, [](auto& c) { return c->schemaColumn(); }); + + ExprVector partition; + ExprVector order; + OrderTypeVector orderType; + // if all partitioning columns are projected, the output is partitioned. + if (isSubset(index->distribution().partition, schemaColumns)) { + partition = index->distribution().partition; + replace(partition, schemaColumns, columns.data()); + } + auto numPrefix = prefixSize(index->distribution().order, schemaColumns); + if (numPrefix > 0) { + order = index->distribution().order; + order.resize(numPrefix); + orderType = index->distribution().orderType; + orderType.resize(numPrefix); + replace(order, schemaColumns, columns.data()); + } + return Distribution( + index->distribution().distributionType, + index->distribution().cardinality * baseTable->filterSelectivity, + std::move(partition), + std::move(order), + std::move(orderType), + index->distribution().numKeysUnique <= numPrefix + ? index->distribution().numKeysUnique + : 0, + 1.0 / baseTable->filterSelectivity); +} + +// static +PlanObjectSet TableScan::availableColumns( + const BaseTable* baseTable, + ColumnGroupP index) { + // The columns of base table that exist in 'index'. + PlanObjectSet result; + for (auto column : index->columns()) { + for (auto baseColumn : baseTable->columns) { + if (baseColumn->name() == column->name()) { + result.add(baseColumn); + break; + } + } + } + return result; +} + +std::string Cost::toString(bool /*detail*/, bool isUnit) const { + std::stringstream out; + float multiplier = isUnit ? 1 : inputCardinality; + out << succinctNumber(fanout * multiplier) << " rows " + << succinctNumber(unitCost * multiplier) << "CU"; + if (setupCost > 0) { + out << ", setup " << succinctNumber(setupCost) << "CU"; + } + if (static_cast(totalBytes)) { + out << " " << velox::succinctBytes(totalBytes); + } + return out.str(); +} + +void RelationOp::printCost(bool detail, std::stringstream& out) const { + auto ctx = queryCtx(); + if (ctx && ctx->contextPlan()) { + auto plan = ctx->contextPlan(); + auto totalCost = plan->cost.unitCost + plan->cost.setupCost; + auto pct = 100 * cost_.inputCardinality * cost_.unitCost / totalCost; + out << " " << std::fixed << std::setprecision(2) << pct << "% "; + } + if (detail) { + out << " " << cost_.toString(detail, false) << std::endl; + } +} + +const char* joinTypeLabel(velox::core::JoinType type) { + switch (type) { + case velox::core::JoinType::kLeft: + return "left"; + case velox::core::JoinType::kRight: + return "right"; + case velox::core::JoinType::kRightSemiFilter: + return "right exists"; + case velox::core::JoinType::kRightSemiProject: + return "right exists-flag"; + case velox::core::JoinType::kLeftSemiFilter: + return "exists"; + case velox::core::JoinType::kLeftSemiProject: + return "exists-flag"; + case velox::core::JoinType::kAnti: + return "not exists"; + default: + return ""; + } +} + +std::string TableScan::toString(bool /*recursive*/, bool detail) const { + std::stringstream out; + if (input()) { + out << input()->toString(true, detail); + out << " *I " << joinTypeLabel(joinType); + } + out << baseTable->schemaTable->name << " " << baseTable->cname; + if (detail) { + printCost(detail, out); + if (!input()) { + out << distribution_.toString() << std::endl; + } + } + return out.str(); +} + +std::string Join::toString(bool recursive, bool detail) const { + std::stringstream out; + if (recursive) { + out << input()->toString(true, detail); + } + out << "*" << (method == JoinMethod::kHash ? "H" : "M") << " " + << joinTypeLabel(joinType); + printCost(detail, out); + if (recursive) { + out << " (" << right->toString(true, detail) << ")"; + if (detail) { + out << std::endl; + } + } + return out.str(); +} + +std::string Repartition::toString(bool recursive, bool detail) const { + std::stringstream out; + if (recursive) { + out << input()->toString(true, detail) << " "; + } + out << (distribution().isBroadcast ? "broadcast" : "shuffle") << " "; + if (detail && !distribution().isBroadcast) { + out << distribution().toString(); + printCost(detail, out); + } else if (detail) { + printCost(detail, out); + } + return out.str(); +} + +Aggregation::Aggregation( + const Aggregation& other, + RelationOpPtr input, + velox::core::AggregationNode::Step _step) + : Aggregation(other) { + *const_cast(&distribution_) = input->distribution(); + input_ = std::move(input); + step = _step; + using velox::core::AggregationNode; + if (step == AggregationNode::Step::kPartial || + step == AggregationNode::Step::kIntermediate) { + *const_cast(&columns_) = intermediateColumns; + } else if (step == AggregationNode::Step::kFinal) { + for (auto i = 0; i < grouping.size(); ++i) { + grouping[i] = intermediateColumns[i]; + } + } +} + +std::string Aggregation::toString(bool recursive, bool detail) const { + std::stringstream out; + if (recursive) { + out << input()->toString(true, detail) << " "; + } + out << velox::core::AggregationNode::stepName(step) << " agg"; + printCost(detail, out); + return out.str(); +} + +std::string HashBuild::toString(bool recursive, bool detail) const { + std::stringstream out; + if (recursive) { + out << input()->toString(true, detail) << " "; + } + out << " Build "; + printCost(detail, out); + return out.str(); +} + +std::string Filter::toString(bool recursive, bool detail) const { + std::stringstream out; + if (recursive) { + out << input()->toString(true, detail) << " "; + } + if (detail) { + out << "Filter ("; + for (auto i = 0; i < exprs_.size(); ++i) { + out << exprs_[i]->toString(); + if (i < exprs_.size() - 1) { + out << " and "; + } + } + out << ")\n"; + } else { + out << "filter " << exprs_.size() << " exprs "; + } + return out.str(); +} + +std::string Project::toString(bool recursive, bool detail) const { + std::stringstream out; + if (recursive) { + out << input()->toString(true, detail) << " "; + } + if (detail) { + out << "Project ("; + for (auto i = 0; i < exprs_.size(); ++i) { + out << columns_[i]->toString() << " = " << exprs_[i]->toString(); + if (i < exprs_.size() - 1) { + out << ", "; + } + } + out << ")\n"; + } else { + out << "project " << exprs_.size() << " columns "; + } + return out.str(); +} + +} // namespace facebook::velox::optimizer diff --git a/verax/RelationOp.h b/verax/RelationOp.h new file mode 100644 index 0000000..e783c6c --- /dev/null +++ b/verax/RelationOp.h @@ -0,0 +1,410 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "optimizer/QueryGraph.h" //@manual +#include "optimizer/Schema.h" //@manual + +/// Plan candidates. +/// A candidate plan is constructed based on the join graph/derived table +/// tree. + +namespace facebook::velox::optimizer { + +struct PlanState; + +// Represents the cost and cardinality of a RelationOp or Plan. A Cost has a +// per-row cost, a per-row fanout and a one-time setup cost. For example, a hash +// join probe has a fanout of 0.3 if 3 of 10 input rows are expected to hit, a +// constant small per-row cost that is fixed and a setup cost that is +// the one time cost of the build side subplan. The inputCardinality +// is a precalculated product of the left deep inputs for the hash +// probe. For a leaf table scan, input cardinality is 1 and the fanout +// is the estimated cardinality after filters, the unitCost is the +// cost of the scan and all filters. For an index lookup, the unit +// cost is a function of the index size and the input spacing and +// input cardinality. A lookup that hits densely is cheaper than one +// that hits sparsely. An index lookup has no setup cost. +struct Cost { + // Cardinality of the output of the left deep input tree. 1 for a leaf + // scan. + float inputCardinality{1}; + + // Cost of processing one input tuple. Complete cost of the operation for a + // leaf. + float unitCost{0}; + + // 'fanout * inputCardinality' is the number of result rows. For a leaf scan, + // this is the number of rows. + float fanout{1}; + + // One time setup cost. Cost of build subplan for the first use of a hash + // build side. 0 for the second use of a hash build side. 0 for table scan + // or index access. + float setupCost{0}; + + // Estimate of total data volume for a hash join build or group/order + // by/distinct / repartition. The memory footprint may not be this if the + // operation is streaming or spills. + float totalBytes{0}; + + // Maximum memory occupancy. If the operation is blocking, e.g. group by, the + // amount of spill is 'totalBytes' - 'peakResidentBytes'. + float peakResidentBytes{0}; + + /// If 'isUnit' shows the cost/cardinality for one row, else for + /// 'inputCardinality' rows. + std::string toString(bool detail, bool isUnit = false) const; +}; + +/// Physical relational operator. This is the common base class of all elements +/// of plan candidates. The immutable Exprs, Columns and BaseTables in the query +/// graph are referenced from these. RelationOp instances are also arena +/// allocated but are reference counted so that no longer interesting +/// candidate plans can be freed, since a very large number of these +/// could be generated. +class RelationOp : public Relation { + public: + RelationOp( + RelType type, + boost::intrusive_ptr input, + Distribution distribution, + ColumnVector columns = {}) + : Relation(type, std::move(distribution), std::move(columns)), + input_(std::move(input)) {} + + virtual ~RelationOp() = default; + + void operator delete(void* ptr) { + queryCtx()->free(ptr); + } + + const boost::intrusive_ptr& input() const { + return input_; + } + + const Cost& cost() const { + return cost_; + } + + /// Returns the number of output rows. + float resultCardinality() const { + return cost_.inputCardinality * cost_.fanout; + } + + /// Returns the value constraints of 'expr' at the output of + /// 'this'. For example, a filter or join may limit values. An Expr + /// will for example have no more distinct values than the number of + /// rows. This is computed on first use. + const Value& value(ExprCP expr) const; + + /// Fills in 'cost_' after construction. Depends on 'input' and is defined for + /// each subclass. + virtual void setCost(const PlanState& input); + + /// Returns human redable string for 'this' and inputs if 'recursive' is true. + /// If 'detail' is true, includes cost and other details. + virtual std::string toString(bool recursive, bool detail) const; + + protected: + // adds a line of cost information to 'out' + void printCost(bool detail, std::stringstream& out) const; + + // Input of filter/project/group by etc., Left side of join, nullptr for a + // leaf table scan. + boost::intrusive_ptr input_; + + Cost cost_; + + private: + // thread local reference count. PlanObjects are freed when the + // QueryGraphContext arena is freed, candidate plans are freed when no longer + // referenced. + mutable int32_t refCount_{0}; + + friend void intrusive_ptr_add_ref(RelationOp* op); + friend void intrusive_ptr_release(RelationOp* op); +}; + +using RelationOpPtr = boost::intrusive_ptr; + +inline void intrusive_ptr_add_ref(RelationOp* op) { + ++op->refCount_; +} + +inline void intrusive_ptr_release(RelationOp* op) { + if (0 == --op->refCount_) { + delete op; + } +} + +/// Represents a full table scan or an index lookup. +struct TableScan : public RelationOp { + TableScan( + RelationOpPtr input, + Distribution _distribution, + const BaseTable* table, + ColumnGroupP _index, + float fanout, + ColumnVector columns, + ExprVector lookupKeys = {}, + velox::core::JoinType joinType = velox::core::JoinType::kInner, + ExprVector joinFilter = {}) + : RelationOp( + RelType::kTableScan, + input, + std::move(_distribution), + std::move(columns)), + baseTable(table), + index(_index), + keys(std::move(lookupKeys)), + joinType(joinType), + joinFilter(std::move(joinFilter)) { + cost_.fanout = fanout; + } + + /// Columns of base table available in 'index'. + static PlanObjectSet availableColumns( + const BaseTable* baseTable, + ColumnGroupP index); + + /// Returns the distribution given the table, index and columns. If + /// partitioning/ordering columns are in the output columns, the + /// distribution reflects the distribution of the index. + static Distribution outputDistribution( + const BaseTable* baseTable, + ColumnGroupP index, + const ColumnVector& columns); + + void setCost(const PlanState& input) override; + + std::string toString(bool recursive, bool detail) const override; + + // The base table reference. May occur in multiple scans if the base + // table decomposes into access via secondary index joined to pk or + // if doing another pass for late materialization. + const BaseTable* baseTable; + + // Index (or other materialization of table) used for the physical data + // access. + ColumnGroupP index; + + // Columns read from 'baseTable'. Can be more than 'columns' if + // there are filters that need columns that are not projected out to + // next op. + PlanObjectSet extractedColumns; + + // Lookup keys, empty if full table scan. + ExprVector keys; + + // If this is a lookup, 'joinType' can be inner, left or anti. + velox::core::JoinType joinType{velox::core::JoinType::kInner}; + + // If this is a non-inner join, extra filter for the join. + const ExprVector joinFilter; +}; + +/// Represents a repartition, i.e. query fragment boundary. The distribution of +/// the output is '_distribution'. +class Repartition : public RelationOp { + public: + Repartition( + RelationOpPtr input, + Distribution distribution, + ColumnVector columns) + : RelationOp( + RelType::kRepartition, + std::move(input), + std::move(distribution), + std::move(columns)) {} + + void setCost(const PlanState& input) override; + std::string toString(bool recursive, bool detail) const override; +}; + +using RepartitionPtr = const Repartition*; + +/// Represents a usually multitable filter not associated with any non-inner +/// join. Non-equality constraints over inner joins become Filters. +class Filter : public RelationOp { + public: + Filter(RelationOpPtr input, ExprVector exprs) + : RelationOp( + RelType::kFilter, + input, + input->distribution(), + input->columns()), + exprs_(std::move(exprs)) {} + const ExprVector& exprs() const { + return exprs_; + } + + void setCost(const PlanState& input) override; + std::string toString(bool recursive, bool detail) const override; + + private: + const ExprVector exprs_; +}; + +/// Assigns names to expressions. Used to rename output from a derived table. +class Project : public RelationOp { + public: + Project(RelationOpPtr input, ExprVector exprs, ColumnVector columns) + : RelationOp( + RelType::kProject, + input, + input->distribution().rename(exprs, columns), + columns), + exprs_(std::move(exprs)), + columns_(std::move(columns)) {} + + const ExprVector& exprs() const { + return exprs_; + } + + const ColumnVector& columns() const { + return columns_; + } + + std::string toString(bool recursive, bool detail) const override; + + private: + const ExprVector exprs_; + const ColumnVector columns_; +}; + +enum class JoinMethod { kHash, kMerge, kCross }; + +/// Represents a hash or merge join. +struct Join : public RelationOp { + Join( + JoinMethod _method, + velox::core::JoinType _joinType, + RelationOpPtr input, + RelationOpPtr right, + ExprVector leftKeys, + ExprVector rightKeys, + ExprVector filter, + float fanout, + ColumnVector columns) + : RelationOp(RelType::kJoin, input, input->distribution(), columns), + method(_method), + joinType(_joinType), + right(std::move(right)), + leftKeys(std::move(leftKeys)), + rightKeys(std::move(rightKeys)), + filter(std::move(filter)) { + cost_.fanout = fanout; + } + + JoinMethod method; + velox::core::JoinType joinType; + RelationOpPtr right; + ExprVector leftKeys; + ExprVector rightKeys; + ExprVector filter; + + void setCost(const PlanState& input) override; + std::string toString(bool recursive, bool detail) const override; +}; + +using JoinPtr = Join*; + +/// Occurs as right input of JoinOp with type kHash. Contains the +/// cost and memory specific to building the table. Can be +/// referenced from multiple JoinOps. The unit cost * input +/// cardinality of this is counted as setup cost in the first +/// referencing join and not counted in subsequent ones. +struct HashBuild : public RelationOp { + HashBuild(RelationOpPtr input, int32_t id, ExprVector _keys, PlanPtr plan) + : RelationOp( + RelType::kHashBuild, + input, + input->distribution(), + input->columns()), + buildId(id), + keys(std::move(_keys)), + plan(plan) {} + + int32_t buildId{0}; + ExprVector keys; + // The plan producing the build data. Used for deduplicating joins. + PlanPtr plan; + + void setCost(const PlanState& input) override; + + std::string toString(bool recursive, bool detail) const override; +}; + +using HashBuildPtr = HashBuild*; + +/// Represents aggregation with or without grouping. +struct Aggregation : public RelationOp { + Aggregation( + const Aggregation& other, + RelationOpPtr input, + velox::core::AggregationNode::Step _step); + + Aggregation(RelationOpPtr input, ExprVector _grouping) + : RelationOp( + RelType::kAggregation, + input, + input ? input->distribution() : Distribution()), + grouping(std::move(_grouping)) {} + + // Grouping keys + ExprVector grouping; + + // Keys where the key expression is functionally dependent on + // another key or keys. These can be late materialized or converted + // to any() aggregates. + PlanObjectSet dependentKeys; + + std::vector> aggregates; + + velox::core::AggregationNode::Step step{ + velox::core::AggregationNode::Step::kSingle}; + + // 'columns' of RelationOp is the final columns. 'intermediateColumns is the + // output of the corresponding partial aggregation. + ColumnVector intermediateColumns; + + void setCost(const PlanState& input) override; + std::string toString(bool recursive, bool detail) const override; +}; + +/// Represents an order by. The order is given by the distribution. +struct OrderBy : public RelationOp { + OrderBy( + RelationOpPtr input, + ExprVector keys, + OrderTypeVector orderType, + PlanObjectSet dependentKeys = {}) + : RelationOp( + RelType::kOrderBy, + input, + input ? input->distribution().copyWithOrder(keys, orderType) + : Distribution(DistributionType(), 1, {}, keys, orderType)), + dependentKeys(dependentKeys) {} + + // Keys where the key expression is functionally dependent on + // another key or keys. These can be late materialized or converted + // to payload. + PlanObjectSet dependentKeys; +}; + +} // namespace facebook::velox::optimizer diff --git a/verax/Schema.cpp b/verax/Schema.cpp new file mode 100644 index 0000000..fe1ce75 --- /dev/null +++ b/verax/Schema.cpp @@ -0,0 +1,381 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/Schema.h" //@manual +#include "optimizer/Plan.h" //@manual +#include "optimizer/PlanUtils.h" //@manual +#include "velox/common/base/SimdUtil.h" +#include "velox/common/base/SuccinctPrinter.h" + +namespace facebook::velox::optimizer { + +float Value::byteSize() const { + if (type->isFixedWidth()) { + return type->cppSizeInBytes(); + } + switch (type->kind()) { + // Add complex types here. + default: + return 16; + } +} + +std::vector SchemaTable::toColumns( + const std::vector& names) { + std::vector columns(names.size()); + assert(!columns.empty()); // lint + for (auto i = 0; i < names.size(); ++i) { + columns[i] = findColumn(name); + } + + return columns; +} + +ColumnGroupP SchemaTable::addIndex( + const char* name, + float cardinality, + int32_t numKeysUnique, + int32_t numOrdering, + const ColumnVector& keys, + DistributionType distType, + const ColumnVector& partition, + const ColumnVector& columns) { + Distribution distribution; + distribution.cardinality = cardinality; + for (auto i = 0; i < numOrdering; ++i) { + distribution.orderType.push_back(OrderType::kAscNullsFirst); + } + distribution.numKeysUnique = numKeysUnique; + appendToVector(distribution.order, keys); + distribution.distributionType = distType; + appendToVector(distribution.partition, partition); + columnGroups.push_back(make(name, this, distribution, columns)); + return columnGroups.back(); +} + +ColumnCP SchemaTable::column(const std::string& name, const Value& value) { + auto it = columns.find(toName(name)); + if (it != columns.end()) { + return it->second; + } + auto* column = make(toName(name), nullptr, value); + columns[toName(name)] = column; + return column; +} + +ColumnCP SchemaTable::findColumn(const std::string& name) const { + auto it = columns.find(toName(name)); + VELOX_CHECK(it != columns.end()); + return it->second; +} + +Schema::Schema( + const char* _name, + std::vector tables, + LocusCP locus) + : name_(_name), defaultLocus_(locus) { + for (auto& table : tables) { + tables_[table->name] = table; + } +} + +Schema::Schema(const char* _name, SchemaResolver* source, LocusCP locus) + : name_(_name), source_(source), defaultLocus_(locus) {} + +SchemaTableCP Schema::findTable(std::string_view name) const { + auto internedName = toName(name); + auto it = tables_.find(internedName); + if (it != tables_.end()) { + return it->second; + } + VELOX_CHECK_NOT_NULL(source_); + auto* table = source_->findTable(std::string(name)); + if (!table) { + return nullptr; + } + auto* schemaTable = make(internedName, table->rowType()); + schemaTable->connectorTable = table; + ColumnVector columns; + for (auto& pair : table->columnMap()) { + auto& tableColumn = *pair.second; + float cardinality = tableColumn.approxNumDistinct(table->numRows()); + Value value(tableColumn.type().get(), cardinality); + auto columnName = toName(pair.first); + auto* column = make(columnName, nullptr, value); + schemaTable->columns[columnName] = column; + columns.push_back(column); + } + DistributionType defaultDist; + defaultDist.locus = defaultLocus_; + auto* pk = schemaTable->addIndex( + toName("pk"), table->numRows(), 0, 0, {}, defaultDist, {}, columns); + addTable(schemaTable); + pk->layout = table->layouts()[0]; + return schemaTable; +} + +void Schema::addTable(SchemaTableCP table) const { + tables_[table->name] = table; +} + +// The fraction of rows of a base table selected by non-join filters. 0.2 +// means 1 in 5 are selected. +float baseSelectivity(PlanObjectCP object) { + if (object->type() == PlanType::kTable) { + return object->as()->filterSelectivity; + } + return 1; +} + +template +ColumnCP findColumnByName(const T& columns, Name name) { + for (auto column : columns) { + if (column->type() == PlanType::kColumn && + column->template as()->name() == name) { + return column->template as(); + } + } + return nullptr; +} + +bool SchemaTable::isUnique(CPSpan columns) const { + for (auto& column : columns) { + if (column->type() != PlanType::kColumn) { + return false; + } + } + for (auto index : columnGroups) { + auto nUnique = index->distribution().numKeysUnique; + if (!nUnique) { + continue; + } + bool unique = true; + for (auto i = 0; i < nUnique; ++i) { + auto part = findColumnByName(columns, index->columns()[i]->name()); + if (!part) { + unique = false; + break; + } + } + if (unique) { + return true; + } + } + return false; +} + +float combine(float card, int32_t ith, float otherCard) { + if (ith == 0) { + return card / otherCard; + } + if (otherCard > card) { + return 1; + } + return card / otherCard; +} + +IndexInfo SchemaTable::indexInfo(ColumnGroupP index, CPSpan columns) + const { + IndexInfo info; + info.index = index; + info.scanCardinality = index->distribution().cardinality; + info.joinCardinality = index->distribution().cardinality; + PlanObjectSet covered; + int32_t numCovered = 0; + int32_t numSorting = index->distribution().orderType.size(); + int32_t numUnique = index->distribution().numKeysUnique; + for (auto i = 0; i < numSorting || i < numUnique; ++i) { + auto part = findColumnByName( + columns, index->distribution().order[i]->as()->name()); + if (!part) { + break; + } + ++numCovered; + covered.add(part); + if (i < numSorting) { + info.scanCardinality = combine( + info.scanCardinality, + i, + index->distribution().order[i]->value().cardinality); + info.lookupKeys.push_back(part); + info.joinCardinality = info.scanCardinality; + } else { + info.joinCardinality = combine( + info.joinCardinality, + i, + index->distribution().order[i]->value().cardinality); + } + if (i == numUnique - 1) { + info.unique = true; + } + } + + for (auto i = 0; i < columns.size(); ++i) { + auto column = columns[i]; + if (column->type() != PlanType::kColumn) { + // Join key is an expression dependent on the table. + covered.unionColumns(column->as()); + info.joinCardinality = combine( + info.joinCardinality, numCovered, column->value().cardinality); + continue; + } + if (covered.contains(column)) { + continue; + } + auto part = findColumnByName(index->columns(), column->name()); + if (!part) { + continue; + } + covered.add(column); + ++numCovered; + info.joinCardinality = + combine(info.joinCardinality, numCovered, column->value().cardinality); + } + info.coveredColumns = std::move(covered); + return info; +} + +IndexInfo SchemaTable::indexByColumns(CPSpan columns) const { + // Match 'columns' against all indices. Pick the one that has the + // longest prefix intersection with 'columns'. If 'columns' are a + // unique combination on any index, then unique is true of the + // result. + IndexInfo pkInfo; + IndexInfo best; + bool unique = isUnique(columns); + float bestPrediction = 0; + for (auto iIndex = 0; iIndex < columnGroups.size(); ++iIndex) { + auto index = columnGroups[iIndex]; + auto candidate = indexInfo(index, columns); + if (iIndex == 0) { + pkInfo = candidate; + best = candidate; + bestPrediction = best.joinCardinality; + continue; + } + if (candidate.lookupKeys.empty()) { + // No prefix match for secondary index. + continue; + } + // The join cardinality estimate from the longest prefix is preferred for + // the estimate. The index with the least scan cardinality is preferred + if (candidate.lookupKeys.size() > best.lookupKeys.size()) { + bestPrediction = candidate.joinCardinality; + } + if (candidate.scanCardinality < best.scanCardinality) { + best = candidate; + } + } + best.joinCardinality = bestPrediction; + best.unique = unique; + return best; +} + +IndexInfo joinCardinality(PlanObjectCP table, CPSpan keys) { + if (table->type() == PlanType::kTable) { + auto schemaTable = table->as()->schemaTable; + return schemaTable->indexByColumns(keys); + } + VELOX_CHECK(table->type() == PlanType::kDerivedTable); + auto dt = table->as(); + auto distribution = dt->distribution; + assert(distribution); + IndexInfo result; + result.scanCardinality = distribution->cardinality; + const ExprVector* groupingKeys = nullptr; + if (dt->aggregation) { + groupingKeys = &dt->aggregation->aggregation->grouping; + } + result.joinCardinality = result.scanCardinality; + for (auto i = 0; i < keys.size(); ++i) { + result.joinCardinality = + combine(result.joinCardinality, i, keys[i]->value().cardinality); + } + if (groupingKeys && keys.size() >= groupingKeys->size()) { + result.unique = true; + } + return result; +} + +ColumnCP IndexInfo::schemaColumn(ColumnCP keyValue) const { + for (auto& column : index->columns()) { + if (column->name() == keyValue->name()) { + return column; + } + } + return nullptr; +} + +bool Distribution::isSamePartition(const Distribution& other) const { + if (!(distributionType == other.distributionType)) { + return false; + } + if (isBroadcast || other.isBroadcast) { + return true; + } + if (partition.size() != other.partition.size()) { + return false; + } + if (partition.size() == 0) { + // If the partitioning columns are not in the columns or if there + // are no partitioning columns, there can be no copartitioning. + return false; + } + for (auto i = 0; i < partition.size(); ++i) { + if (!partition[i]->sameOrEqual(*other.partition[i])) { + return false; + } + } + return true; +} + +Distribution Distribution::rename( + const ExprVector& exprs, + const ColumnVector& names) const { + Distribution result = *this; + // Partitioning survives projection if all partitioning columns are projected + // out. + if (!replace(result.partition, exprs, names)) { + result.partition.clear(); + } + // Ordering survives if a prefix of the previous order continues to be + // projected out. + result.order.resize(prefixSize(result.order, exprs)); + replace(result.order, exprs, names); + return result; +} + +std::string Distribution::toString() const { + if (isBroadcast) { + return "broadcast"; + } + std::stringstream out; + if (!partition.empty()) { + out << "P "; + exprsToString(partition, out); + out << " " << distributionType.numPartitions << " ways"; + } + if (!order.empty()) { + out << " O "; + exprsToString(order, out); + } + if (numKeysUnique && numKeysUnique >= order.size()) { + out << " first " << numKeysUnique << " unique"; + } + return out.str(); +} + +} // namespace facebook::velox::optimizer diff --git a/verax/Schema.h b/verax/Schema.h new file mode 100644 index 0000000..24d3994 --- /dev/null +++ b/verax/Schema.h @@ -0,0 +1,462 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "optimizer/PlanObject.h" //@manual +#include "optimizer/SchemaResolver.h" //@manual + +/// Schema representation for use in query planning. All objects are +/// arena allocated for the duration of planning the query. We do +/// not expect to keep a full schema in memory, rather we expect to +/// instantiate the relevant schema objects based on the query. The +/// arena for these can be different from that for the PlanObjects, +/// though, so that a schema cache can have its own lifetime. +namespace facebook::velox::optimizer { + +template +using NameMap = std::unordered_map< + Name, + T, + std::hash, + std::equal_to, + QGAllocator>>; + +/// Represents constraints on a column value or intermediate result. +struct Value { + Value(const velox::Type* _type, float _cardinality) + : type(_type), cardinality(_cardinality) {} + + /// Returns the average byte size of a value when it occurs as an intermediate + /// result without dictionary or other encoding. + float byteSize() const; + + const velox::Type* type; + const velox::variant* min{nullptr}; + const velox::variant* max{nullptr}; + + // Count of distinct values. Is not exact and is used for estimating + // cardinalities of group bys or joins. + const float cardinality{1}; + + // Estimate of true fraction for booleans. 0 means always + // false. This is an estimate and 1 or 0 do not allow pruning + // dependent code paths. + float trueFraction{1}; + + // 0 means no nulls, 0.5 means half are null. + float nullFraction{0}; + + // True if nulls may occur. 'false' means that plans that allow no nulls may + // be generated. + bool nullable{true}; +}; + +/// Describes order in an order by or index. +enum class OrderType { + kAscNullsFirst, + kAscNullsLast, + kDescNullsFirst, + kDescNullsLast +}; + +using OrderTypeVector = std::vector>; + +class RelationOp; + +/// Represents a system that contains or produces data.For cases of federation +/// where data is only accessible via a specific instance of a specific type of +/// system, the locus represents the instance and the subclass of Locus +/// represents the type of system for a schema object. For a +/// RelationOp, the locus of its distribution means that the op is performed by +/// the corresponding system. Distributions can be copartitioned only +/// if their locus is equal (==) to the other locus. A Locus is referenced by +/// raw pointer and may be allocated from outside the optimization arena. It is +/// immutable and lives past the optimizer arena. +class Locus { + public: + explicit Locus(Name name, connector::Connector* connector) + : name_(name), connector_(connector) {} + + virtual ~Locus() = default; + + Name name() const { + // Make sure the name is in the current optimization + // arena. 'this' may live across several arenas. + return toName(name_); + } + + const connector::Connector* connector() const { + // // 'connector_' can be nullptr if no executable plans are made. + VELOX_CHECK_NOT_NULL(connector_); + return connector_; + } + + /// Sets the cardinality in op. Returns true if set. If false, default + /// cardinality determination. + virtual bool setCardinality(RelationOp& /*op*/) const { + return false; + } + + /// Sets the cost. Returns true if set. If false, the default cost is set with + /// RelationOp::setCost. + virtual bool setCost(RelationOp& /*op*/) const { + return false; + } + + std::string toString() const { + return name_; + } + + private: + const Name name_; + const connector::Connector* connector_; +}; + +using LocusCP = const Locus*; + +/// Method for determining a partition given an ordered list of partitioning +/// keys. Hive hash is an example, range partitioning is another. Add values +/// here for more types. +enum class ShuffleMode { kNone, kHive }; + +/// Distribution of data. 'numPartitions' is 1 if the data is not partitioned. +/// There is copartitioning if the DistributionType is the same on both sides +/// and both sides have an equal number of 1:1 type matched partitioning keys. +struct DistributionType { + bool operator==(const DistributionType& other) const { + return mode == other.mode && numPartitions == other.numPartitions && + locus == other.locus && isGather == other.isGather; + } + + ShuffleMode mode{ShuffleMode::kNone}; + int32_t numPartitions{1}; + LocusCP locus{nullptr}; + bool isGather{false}; +}; + +// Describes output of relational operator. If base table, cardinality is +// after filtering. +struct Distribution { + Distribution() = default; + Distribution( + DistributionType type, + float cardinality, + ExprVector _partition, + ExprVector _order = {}, + OrderTypeVector _orderType = {}, + int32_t uniquePrefix = 0, + float _spacing = 0) + : distributionType(std::move(type)), + cardinality(cardinality), + partition(std::move(_partition)), + order(std::move(_order)), + orderType(std::move(_orderType)), + numKeysUnique(uniquePrefix), + spacing(_spacing) {} + + /// Returns a Distribution for use in a broadcast shuffle. + static Distribution broadcast(DistributionType type, float cardinality) { + Distribution result(type, cardinality, {}); + result.isBroadcast = true; + return result; + } + + /// Returns a distribution for an end of query gather from last stage + /// fragments. Specifying order will create a merging exchange when the + /// Distribution occurs in a Repartition. + static Distribution gather( + DistributionType type, + const ExprVector& order = {}, + const OrderTypeVector& orderType = {}) { + auto singleType = type; + singleType.numPartitions = 1; + singleType.isGather = true; + return Distribution(singleType, 1, {}, order, orderType); + } + + /// Returns a copy of 'this' with 'order' and 'orderType' set from + /// arguments. + Distribution copyWithOrder(ExprVector order, OrderTypeVector orderType) + const { + Distribution copy = *this; + copy.order = order; + copy.orderType = orderType; + return copy; + } + + /// True if 'this' and 'other' have the same number/type of keys and same + /// distribution type. Data is copartitioned if both sides have a 1:1 + /// equality on all partitioning key columns. + bool isSamePartition(const Distribution& other) const; + + Distribution rename(const ExprVector& exprs, const ColumnVector& names) const; + + std::string toString() const; + + DistributionType distributionType; + + // Number of rows 'this' applies to. This is the size in rows if 'this' + // occurs in a table or index. + float cardinality; + + // Partitioning columns. The values of these columns determine which of + // 'numPartitions' contains any given row. This does not specify the + // partition function (e.g. Hive bucket or range partition). + ExprVector partition; + + // Ordering columns. Each partition is ordered by these. Specifies that + // streaming group by or merge join are possible. + ExprVector order; + + // Corresponds 1:1 to 'order'. The size of this gives the number of leading + // columns of 'order' on which the data is sorted. + OrderTypeVector orderType; + + // Number of leading elements of 'order' such that these uniquely + // identify a row. 0 if there is no uniqueness. This can be non-0 also if + // data is not sorted. This indicates a uniqueness for joining. + int32_t numKeysUnique{0}; + + // Specifies the selectivity between the source of the ordered data + // and 'this'. For example, if orders join lineitem and both are + // ordered on orderkey and there is a 1/1000 selection on orders, + // the distribution after the filter would have a spacing of 1000, + // meaning that lineitem is hit every 1000 orders, meaning that an + // index join with lineitem would skip 4000 rows between hits + // because lineitem has an average of 4 repeats of orderkey. + float spacing{-1}; + + // True if the data is replicated to 'numPartitions'. + bool isBroadcast{false}; +}; + +/// Identifies a base table or the operator type producing the relation. Base +/// data as in Index has type kBase. The result of a table scan is kTableScan. +enum class RelType { + kBase, + kTableScan, + kRepartition, + kFilter, + kProject, + kJoin, + kHashBuild, + kAggregation, + kOrderBy +}; + +/// Represents a relation (table) that is either physically stored or is the +/// streaming output of a query operator. This has a distribution describing +/// partitioning and data order and a set of columns describing the payload. +class Relation { + public: + Relation( + RelType relType, + Distribution distribution, + const ColumnVector& columns) + : relType_(relType), + distribution_(std::move(distribution)), + columns_(columns) {} + + RelType relType() const { + return relType_; + } + + const Distribution& distribution() const { + return distribution_; + } + + const ColumnVector& columns() const { + return columns_; + } + + ColumnVector& mutableColumns() { + return columns_; + } + + template + const T* as() const { + return static_cast(this); + } + + template + T* as() { + return static_cast(this); + } + + protected: + const RelType relType_; + const Distribution distribution_; + ColumnVector columns_; +}; + +struct SchemaTable; +using SchemaTableCP = const SchemaTable*; + +/// Represents a stored collection of rows with part of or all columns +/// of a table. A ColumnGroup may have a uniqueness constraint over a +/// set of columns, a partitioning and an ordering plus a set of +/// payload columns. An index is a ColumnGroup that may not have all +/// columns but is organized to facilitate retrievel. We use the name +/// index for ColumnGroup when using it for lookup. +struct ColumnGroup : public Relation { + ColumnGroup( + Name _name, + SchemaTableCP _table, + Distribution distribution, + const ColumnVector& _columns, + const connector::TableLayout* layout = nullptr) + : Relation(RelType::kBase, distribution, _columns), + name(_name), + table(_table), + layout(layout) {} + + Name name; + SchemaTableCP table; + const connector::TableLayout* layout; + + /// Returns cost of next lookup when the hit is within 'range' rows + /// of the previous hit. If lookups are not batched or not ordered, + /// then 'range' should be the cardinality of the index. + float lookupCost(float range) const; +}; + +using ColumnGroupP = ColumnGroup*; + +// Describes the number of rows to look at and the number of expected matches +// given equality constraints for a set of columns. See +// SchemaTable::indexInfo(). +struct IndexInfo { + // Index chosen based on columns. + ColumnGroupP index; + + // True if the column combination is unique. This can be true even if there + // is no key order in 'index'. + bool unique{false}; + + // The number of rows selected after index lookup based on 'lookupKeys'. For + // empty 'lookupKeys', this is the cardinality of 'index'. + float scanCardinality; + + // The expected number of hits for an equality match of lookup keys. This is + // the expected number of rows given the lookup column combination + // regardless of whether an index order can be used. + float joinCardinality; + + // The lookup columns that match 'index'. These match 1:1 the leading keys + // of 'index'. If 'index' has no ordering columns or if the lookup columns + // are not a prefix of these, this is empty. + std::vector lookupKeys; + + // The columns that were considered in 'scanCardinality' and + // 'joinCardinality'. This may be fewer columns than given to + // indexInfo() if the index does not cover some columns. + PlanObjectSet coveredColumns; + + /// Returns the schema column for the BaseTable column 'column' or nullptr + /// if not in the index. + ColumnCP schemaColumn(ColumnCP keyValue) const; +}; + +IndexInfo joinCardinality(PlanObjectCP table, CPSpan keys); + +float baseSelectivity(PlanObjectCP object); + +/// A table in a schema. The table may have multiple differently ordered and +/// partitioned physical representations (ColumnGroups). Not all ColumnGroups +/// (aka indices) need to contain all columns. +struct SchemaTable { + SchemaTable(Name _name, const velox::RowTypePtr& _type) + : name(_name), type(_type) {} + + /// Adds an index. The arguments set the corresponding members of a + /// Distribution. + ColumnGroupP addIndex( + Name name, + float cardinality, + int32_t numKeysUnique, + int32_t numOrdering, + const ColumnVector& keys, + DistributionType distType, + const ColumnVector& partition, + const ColumnVector& columns); + + /// Finds or adds a column with 'name' and 'value'. + ColumnCP column(const std::string& name, const Value& value); + + ColumnCP findColumn(const std::string& name) const; + + /// True if 'columns' match no more than one row. + bool isUnique(CPSpan columns) const; + + /// Returns uniqueness and cardinality information for a lookup on 'index' + /// where 'columns' have an equality constraint. + IndexInfo indexInfo(ColumnGroupP index, CPSpan columns) const; + + /// Returns the best index to use for lookup where 'columns' have an + /// equality constraint. + IndexInfo indexByColumns(CPSpan columns) const; + + std::vector toColumns(const std::vector& names); + Name name; + const velox::RowTypePtr& type; + + // Lookup from name to column. + NameMap columns; + + // All indices. Must contain at least one. + std::vector> columnGroups; + + // Table description from external schema. This is the + // source-dependent representation from which 'this' was created. + const velox::connector::Table* connectorTable{nullptr}; +}; + +/// Represents a collection of tables. Normally filled in ad hoc given +/// the set of tables referenced by a query. The lifetime is a single +/// optimization run. The owned objects are from the optimizer +/// arena. Schema is owned by the application and is not from the +/// optimization arena. Objects of different catalogs/schemas get +/// added to 'this' on first use. The Schema feeds from a +/// SchemaResolver which interfaces to a local/remote metadata +/// repository. The objects have a default Locus for convenience. +class Schema { + public: + /// Constructs a testing schema without SchemaResolver. + Schema(Name _name, std::vector tables, LocusCP locus); + + /// Constructs a Schema for producing executable plans, backed by 'source'. + Schema(Name _name, SchemaResolver* source, LocusCP locus); + + /// Returns the table with 'name' or nullptr if not found. + SchemaTableCP findTable(std::string_view name) const; + + Name name() const { + return name_; + } + + void addTable(SchemaTableCP table) const; + + private: + Name name_; + mutable NameMap tables_; + SchemaResolver* source_{nullptr}; + LocusCP defaultLocus_; +}; + +using SchemaP = Schema*; + +} // namespace facebook::velox::optimizer diff --git a/verax/SchemaResolver.cpp b/verax/SchemaResolver.cpp new file mode 100644 index 0000000..9690f30 --- /dev/null +++ b/verax/SchemaResolver.cpp @@ -0,0 +1,50 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/SchemaResolver.h" //@manual + +namespace facebook::velox::optimizer { + +const connector::Table* SchemaResolver::findTable(const std::string& name) { + std::vector dots; + std::string lookupName; + connector::Connector* connector = nullptr; + for (auto i = 0; i < name.size(); ++i) { + if (name[i] == '.') { + dots.push_back(i); + } + } + if (dots.empty()) { + lookupName = defaultSchema_.empty() + ? name + : fmt::format("{}.{}", defaultSchema_, name); + connector = defaultConnector_.get(); + } else if (dots.back() == name.size() - 1) { + VELOX_USER_FAIL("Table name ends in '.': {}", name); + } else if (dots.size() == 1) { + lookupName = name; + connector = defaultConnector_.get(); + } else if (dots.size() > 2) { + VELOX_USER_FAIL("Table name has more than 3 parts: {}", name); + } else { + connector = connector::getConnector(name.substr(0, dots[0])).get(); + lookupName = name.substr(dots[0], name.size()); + } + + return connector->metadata()->findTable(lookupName); +} + +} // namespace facebook::velox::optimizer diff --git a/verax/SchemaResolver.h b/verax/SchemaResolver.h new file mode 100644 index 0000000..84e05b8 --- /dev/null +++ b/verax/SchemaResolver.h @@ -0,0 +1,45 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include +#include "optimizer/connectors/ConnectorMetadata.h" //@manual +#include "velox/vector/ComplexVector.h" + +namespace facebook::velox::optimizer { + +/// Translates from table name to table. 'defaultConnector is used +/// to look up the name if the name has no catalog. 'defaultSchema' +/// is used to fill in the schema if the name has no schema. +class SchemaResolver { + public: + SchemaResolver( + const std::shared_ptr& defaultConnector, + const std::string& defaultSchema) + : defaultConnector_(defaultConnector), defaultSchema_(defaultSchema) {} + + virtual ~SchemaResolver() = default; + + virtual const connector::Table* findTable(const std::string& name); + + private: + // Connector to use if name does not specify a catalog. + const std::shared_ptr defaultConnector_; + const std::string defaultSchema_; +}; + +} // namespace facebook::velox::optimizer diff --git a/verax/ToGraph.cpp b/verax/ToGraph.cpp new file mode 100644 index 0000000..9d1de19 --- /dev/null +++ b/verax/ToGraph.cpp @@ -0,0 +1,599 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/Plan.h" //@manual +#include "optimizer/PlanUtils.h" //@manual +#include "velox/exec/Aggregate.h" +#include "velox/expression/ConstantExpr.h" + +namespace facebook::velox::optimizer { + +using namespace facebook::velox; + +std::string veloxToString(core::PlanNode* plan) { + return plan->toString(true, true); +} + +void Optimization::setDerivedTableOutput( + DerivedTableP dt, + const velox::core::PlanNode& planNode) { + auto& outputType = planNode.outputType(); + for (auto i = 0; i < outputType->size(); ++i) { + auto fieldType = outputType->childAt(i); + auto fieldName = outputType->nameOf(i); + auto expr = translateColumn(fieldName); + Value value(toType(fieldType), 0); + auto* column = make(toName(fieldName), dt, value); + dt->columns.push_back(column); + dt->exprs.push_back(expr); + renames_[fieldName] = column; + } +} + +DerivedTableP Optimization::makeQueryGraph() { + auto* root = make(); + root_ = root; + currentSelect_ = root_; + root->cname = toName(fmt::format("dt{}", ++nameCounter_)); + makeQueryGraph(inputPlan_, kAllAllowedInDt); + return root_; +} + +const std::string* columnName(const core::TypedExprPtr& expr) { + if (auto column = + dynamic_cast(expr.get())) { + if (column->inputs().empty() || + dynamic_cast(column->inputs()[0].get())) { + return &column->name(); + } + } + return nullptr; +} + +bool isCall(const core::TypedExprPtr& expr, const std::string& name) { + if (auto call = std::dynamic_pointer_cast(expr)) { + return call->name() == name; + } + return false; +} + +void Optimization::translateConjuncts( + const core::TypedExprPtr& input, + ExprVector& flat) { + if (!input) { + return; + } + if (isCall(input, "and")) { + for (auto& child : input->inputs()) { + translateConjuncts(child, flat); + } + } else { + flat.push_back(translateExpr(input)); + } +} + +template +variant toVariant(BaseVector& constantVector) { + using T = typename TypeTraits::NativeType; + if (auto typed = dynamic_cast*>(&constantVector)) { + return variant(typed->valueAt(0)); + } + VELOX_FAIL("Literal not of foldable type"); +} + +ExprCP Optimization::tryFoldConstant( + const core::CallTypedExpr* call, + const core::CastTypedExpr* cast, + const ExprVector& literals) { + try { + Value value(call ? toType(call->type()) : toType(cast->type()), 1); + auto* veraxExpr = make( + PlanType::kCall, + cast ? toName("cast") : toName(call->name()), + value, + literals, + FunctionSet()); + auto typedExpr = toTypedExpr(veraxExpr); + auto exprSet = evaluator_.compile(typedExpr); + auto first = exprSet->exprs().front().get(); + if (auto constantExpr = dynamic_cast(first)) { + auto variantLiteral = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH( + toVariant, constantExpr->value()->typeKind(), *constantExpr->value()); + Value value(toType(constantExpr->value()->type()), 1); + // Copy the variant from value to allocated in arena. + auto* copy = make(variantLiteral); + auto* literal = make(value, copy); + return literal; + } + return nullptr; + } catch (const std::exception& e) { + return nullptr; + } +} + +ExprCP Optimization::translateExpr(const core::TypedExprPtr& expr) { + if (auto name = columnName(expr)) { + return translateColumn(*name); + } + if (auto constant = + dynamic_cast(expr.get())) { + auto* literal = + make(Value(toType(constant->type()), 1), &constant->value()); + return literal; + } + auto it = exprDedup_.find(expr.get()); + if (it != exprDedup_.end()) { + return it->second; + } + ExprVector args{expr->inputs().size()}; + PlanObjectSet columns; + FunctionSet funcs; + auto& inputs = expr->inputs(); + float cardinality = 1; + bool allConstant = true; + for (auto i = 0; i < inputs.size(); ++i) { + args[i] = translateExpr(inputs[i]); + allConstant &= args[i]->type() == PlanType::kLiteral; + cardinality = std::max(cardinality, args[i]->value().cardinality); + if (args[i]->type() == PlanType::kCall) { + funcs = funcs | args[i]->as()->functions(); + } + } + auto call = dynamic_cast(expr.get()); + auto cast = dynamic_cast(expr.get()); + if (allConstant && (call || cast)) { + auto literal = tryFoldConstant(call, cast, args); + if (literal) { + return literal; + } + } + + if (call) { + auto name = toName(call->name()); + funcs = funcs | functionBits(name); + + auto* callExpr = + make(name, Value(toType(call->type()), cardinality), args, funcs); + exprDedup_[expr.get()] = callExpr; + return callExpr; + } + if (cast) { + auto name = toName("cast"); + funcs = funcs | functionBits(name); + + auto* callExpr = + make(name, Value(toType(cast->type()), cardinality), args, funcs); + exprDedup_[expr.get()] = callExpr; + return callExpr; + } + + VELOX_NYI(); + return nullptr; +} + +ExprCP Optimization::translateColumn(const std::string& name) { + auto column = renames_.find(name); + if (column != renames_.end()) { + return column->second; + } + VELOX_FAIL("could not resolve name {}", name); +} + +ExprVector Optimization::translateColumns( + const std::vector& source) { + ExprVector result{source.size()}; + for (auto i = 0; i < source.size(); ++i) { + result[i] = translateColumn(source[i]->name()); // NOLINT + } + return result; +} + +TypePtr intermediateType(const core::CallTypedExprPtr& call) { + std::vector types; + for (auto& arg : call->inputs()) { + types.push_back(arg->type()); + } + return exec::Aggregate::intermediateType(call->name(), types); +} + +AggregationP Optimization::translateAggregation( + const core::AggregationNode& source) { + using velox::core::AggregationNode; + + if (source.step() == AggregationNode::Step::kPartial || + source.step() == AggregationNode::Step::kSingle) { + auto* aggregation = + make(nullptr, translateColumns(source.groupingKeys())); + for (auto i = 0; i < source.groupingKeys().size(); ++i) { + if (aggregation->grouping[i]->type() == PlanType::kColumn) { + aggregation->mutableColumns().push_back( + aggregation->grouping[i]->as()); + } else { + auto name = toName(source.outputType()->nameOf(i)); + toType(source.outputType()->childAt(i)); + + auto* column = make( + name, currentSelect_, aggregation->grouping[i]->value()); + aggregation->mutableColumns().push_back(column); + } + } + // The keys for intermediate are the same as for final. + aggregation->intermediateColumns = aggregation->columns(); + for (auto i = 0; i < source.aggregateNames().size(); ++i) { + auto rawFunc = translateExpr(source.aggregates()[i].call)->as(); + ExprCP condition = nullptr; + if (source.aggregates()[i].mask) { + condition = translateExpr(source.aggregates()[i].mask); + } + VELOX_CHECK(source.aggregates()[i].sortingKeys.empty()); + auto accumulatorType = + toType(intermediateType(source.aggregates()[i].call)); + auto* agg = make( + rawFunc->name(), + rawFunc->value(), + rawFunc->args(), + rawFunc->functions(), + false, + condition, + false, + accumulatorType); + auto name = toName(source.aggregateNames()[i]); + auto* column = make(name, currentSelect_, agg->value()); + aggregation->mutableColumns().push_back(column); + auto intermediateValue = agg->value(); + intermediateValue.type = accumulatorType; + auto* intermediateColumn = + make(name, currentSelect_, intermediateValue); + aggregation->intermediateColumns.push_back(intermediateColumn); + auto dedupped = queryCtx()->dedup(agg); + aggregation->aggregates.push_back(dedupped->as()); + auto resultName = toName(source.aggregateNames()[i]); + renames_[resultName] = aggregation->columns().back(); + } + return aggregation; + } + return nullptr; +} + +OrderByP Optimization::translateOrderBy(const core::OrderByNode& order) { + OrderTypeVector orderType; + for (auto& sort : order.sortingOrders()) { + orderType.push_back( + sort.isAscending() ? (sort.isNullsFirst() ? OrderType::kAscNullsFirst + : OrderType::kAscNullsLast) + : (sort.isNullsFirst() ? OrderType::kDescNullsFirst + : OrderType::kDescNullsLast)); + } + auto keys = translateColumns(order.sortingKeys()); + auto* orderBy = QGC_MAKE_IN_ARENA(OrderBy)(nullptr, keys, orderType, {}); + return orderBy; +} + +ColumnCP Optimization::makeMark(const core::AbstractJoinNode& join) { + auto type = join.outputType(); + auto name = toName(type->nameOf(type->size() - 1)); + Value value(toType(type->childAt(type->size() - 1)), 2); + auto* column = make(name, currentSelect_, value); + return column; +} + +void Optimization::translateJoin(const core::AbstractJoinNode& join) { + bool isInner = join.isInnerJoin(); + makeQueryGraph(*join.sources()[0], allow(PlanType::kJoin)); + auto leftKeys = translateColumns(join.leftKeys()); + // For an inner join a join tree on the right can be flattened, for all other + // kinds it must be kept together in its own dt. + makeQueryGraph(*join.sources()[1], isInner ? allow(PlanType::kJoin) : 0); + auto rightKeys = translateColumns(join.rightKeys()); + ExprVector conjuncts; + translateConjuncts(join.filter(), conjuncts); + if (isInner) { + // Every column to column equality adds to an equivalence class and is an + // independent bidirectional join edge. + for (auto i = 0; i < leftKeys.size(); ++i) { + auto l = leftKeys[i]; + auto r = rightKeys.at(i); + if (l->type() == PlanType::kColumn && r->type() == PlanType::kColumn) { + l->as()->equals(r->as()); + currentSelect_->addJoinEquality(l, r, {}, false, false, false, false); + } else { + currentSelect_->addJoinEquality(l, r, {}, false, false, false, false); + } + } + currentSelect_->conjuncts.insert( + currentSelect_->conjuncts.end(), conjuncts.begin(), conjuncts.end()); + } else { + auto joinType = join.joinType(); + bool leftOptional = + joinType == core::JoinType::kRight || joinType == core::JoinType::kFull; + bool rightOptional = + joinType == core::JoinType::kLeft || joinType == core::JoinType::kFull; + bool rightExists = joinType == core::JoinType::kLeftSemiFilter; + bool rightNotExists = joinType == core::JoinType::kAnti; + ColumnCP markColumn = + joinType == core::JoinType::kLeftSemiProject ? makeMark(join) : nullptr; + ; + + PlanObjectSet leftTables; + PlanObjectCP rightTable = nullptr; + + for (auto i = 0; i < leftKeys.size(); ++i) { + auto l = leftKeys[i]; + leftTables.unionSet(l->allTables()); + auto r = rightKeys.at(i); + auto rightKeyTable = r->singleTable(); + if (rightTable) { + VELOX_CHECK(rightKeyTable == rightTable); + } else { + rightTable = rightKeyTable; + } + } + VELOX_CHECK(rightTable, "No right side in join"); + std::vector leftTableVector; + leftTables.forEach( + [&](PlanObjectCP table) { leftTableVector.push_back(table); }); + auto* edge = make( + leftTableVector.size() == 1 ? leftTableVector[0] : nullptr, + rightTable, + conjuncts, + leftOptional, + rightOptional, + rightExists, + rightNotExists, + markColumn); + if (markColumn) { + renames_[markColumn->name()] = markColumn; + } + currentSelect_->joins.push_back(edge); + for (auto i = 0; i < leftKeys.size(); ++i) { + edge->addEquality(leftKeys[i], rightKeys[i]); + } + } +} + +void Optimization::translateNonEqualityJoin( + const core::NestedLoopJoinNode& join) { + auto joinType = join.joinType(); + bool isInner = joinType == core::JoinType::kInner; + makeQueryGraph(*join.sources()[0], allow(PlanType::kJoin)); + // For an inner join a join tree on the right can be flattened, for all other + // kinds it must be kept together in its own dt. + makeQueryGraph(*join.sources()[1], isInner ? allow(PlanType::kJoin) : 0); + ExprVector conjuncts; + translateConjuncts(join.joinCondition(), conjuncts); + if (conjuncts.empty()) { + // Inner cross product. Join conditions may be added from + // conjuncts of the enclosing DerivedTable. + return; + } + PlanObjectSet tables; + for (auto& conjunct : conjuncts) { + tables.unionColumns(conjunct); + } + std::vector tableVector; + tables.forEach([&](PlanObjectCP table) { tableVector.push_back(table); }); + if (tableVector.size() == 2) { + auto* edge = make( + tableVector[0], tableVector[1], conjuncts, false, false, false, false); + edge->guessFanout(); + currentSelect_->joins.push_back(edge); + + } else { + VELOX_NYI("Multiway non-equality join not supported"); + currentSelect_->conjuncts.insert( + currentSelect_->conjuncts.end(), conjuncts.begin(), conjuncts.end()); + } +} + +bool isJoin(const core::PlanNode& node) { + auto name = node.name(); + if (name == "HashJoin" || name == "MergeJoin" || name == "NestedLoopJoin") { + return true; + } + if (name == "Project" || name == "Filter") { + return isJoin(*node.sources()[0]); + } + return false; +} + +bool isDirectOver(const core::PlanNode& node, const std::string& name) { + auto source = node.sources()[0]; + if (source && source->name() == name) { + return true; + } + return false; +} + +PlanObjectP Optimization::wrapInDt(const core::PlanNode& node) { + DerivedTableP previousDt = currentSelect_; + auto* newDt = make(); + auto cname = toName(fmt::format("dt{}", ++nameCounter_)); + newDt->cname = cname; + currentSelect_ = newDt; + makeQueryGraph(node, kAllAllowedInDt); + + currentSelect_ = previousDt; + velox::RowTypePtr type = node.outputType(); + // node.name() == "Aggregation" ? aggFinalType_ : node.outputType(); + for (auto i = 0; i < type->size(); ++i) { + ExprCP inner = translateColumn(type->nameOf(i)); + newDt->exprs.push_back(inner); + auto* outer = make(toName(type->nameOf(i)), newDt, inner->value()); + newDt->columns.push_back(outer); + renames_[type->nameOf(i)] = outer; + } + currentSelect_->tables.push_back(newDt); + currentSelect_->tableSet.add(newDt); + newDt->makeInitialPlan(); + + return newDt; +} + +PlanObjectP Optimization::makeBaseTable(const core::TableScanNode* tableScan) { + auto tableHandle = tableScan->tableHandle().get(); + auto assignments = tableScan->assignments(); + auto schemaTable = schema_.findTable(tableHandle->name()); + auto cname = fmt::format("t{}", ++nameCounter_); + + auto* baseTable = make(); + baseTable->cname = toName(cname); + baseTable->schemaTable = schemaTable; + ColumnVector columns; + ColumnVector schemaColumns; + for (auto& pair : assignments) { + auto schemaColumn = schemaTable->findColumn(pair.second->name()); + schemaColumns.push_back(schemaColumn); + auto value = schemaColumn->value(); + auto* column = make(toName(pair.second->name()), baseTable, value); + columns.push_back(column); + renames_[pair.first] = column; + } + baseTable->columns = columns; + + setLeafHandle(baseTable->id(), tableScan->tableHandle(), {}); + setLeafSelectivity(*baseTable); + currentSelect_->tables.push_back(baseTable); + currentSelect_->tableSet.add(baseTable); + return baseTable; +} + +void Optimization::addProjection(const core::ProjectNode* project) { + auto names = project->names(); + auto exprs = project->projections(); + for (auto i = 0; i < names.size(); ++i) { + if (auto field = dynamic_cast( + exprs.at(i).get())) { + // A variable projected to itself adds no renames. Inputs contain this + // all the time. + if (field->name() == names[i]) { + continue; + } + } + auto expr = translateExpr(exprs.at(i)); + renames_[names[i]] = expr; + } +} + +void Optimization::addFilter(const core::FilterNode* filter) { + ExprVector flat; + translateConjuncts(filter->filter(), flat); + if (isDirectOver(*filter, "Aggregation")) { + VELOX_CHECK( + currentSelect_->having.empty(), + "Must have aall of HAVING in one filter"); + currentSelect_->having = flat; + } else { + currentSelect_->conjuncts.insert( + currentSelect_->conjuncts.end(), flat.begin(), flat.end()); + } +} + +PlanObjectP Optimization::addAggregation( + const core::AggregationNode& aggNode, + uint64_t allowedInDt) { + using AggregationNode = velox::core::AggregationNode; + if (aggNode.step() == AggregationNode::Step::kPartial || + aggNode.step() == AggregationNode::Step::kSingle) { + if (!contains(allowedInDt, PlanType::kAggregation)) { + return wrapInDt(aggNode); + } + if (aggNode.step() == AggregationNode::Step::kSingle) { + aggFinalType_ = aggNode.outputType(); + } + makeQueryGraph( + *aggNode.sources()[0], makeDtIf(allowedInDt, PlanType::kAggregation)); + auto agg = translateAggregation(aggNode); + if (agg) { + auto* aggPlan = make(agg); + currentSelect_->aggregation = aggPlan; + } + } else { + if (aggNode.step() == AggregationNode::Step::kFinal) { + aggFinalType_ = aggNode.outputType(); + } + makeQueryGraph(*aggNode.sources()[0], allowedInDt); + } + return currentSelect_; +} + +PlanObjectP Optimization::makeQueryGraph( + const core::PlanNode& node, + uint64_t allowedInDt) { + auto name = node.name(); + if (isJoin(node) && !contains(allowedInDt, PlanType::kJoin)) { + return wrapInDt(node); + } + if (name == "TableScan") { + return makeBaseTable(reinterpret_cast(&node)); + } + if (name == "Project") { + makeQueryGraph(*node.sources()[0], allowedInDt); + addProjection(reinterpret_cast(&node)); + return currentSelect_; + } + if (name == "Filter") { + makeQueryGraph(*node.sources()[0], allowedInDt); + addFilter(reinterpret_cast(&node)); + return currentSelect_; + } + if (name == "HashJoin" || name == "MergeJoin") { + if (!contains(allowedInDt, PlanType::kJoin)) { + return wrapInDt(node); + } + translateJoin(*reinterpret_cast(&node)); + return currentSelect_; + } + if (name == "NestedLoopJoin") { + if (!contains(allowedInDt, PlanType::kJoin)) { + return wrapInDt(node); + } + translateNonEqualityJoin( + *reinterpret_cast(&node)); + return currentSelect_; + } + if (name == "LocalPartition") { + makeQueryGraph(*node.sources()[0], allowedInDt); + return currentSelect_; + } + if (name == "Aggregation") { + return addAggregation( + *reinterpret_cast(&node), allowedInDt); + } + if (name == "OrderBy") { + if (!contains(allowedInDt, PlanType::kOrderBy)) { + return wrapInDt(node); + } + makeQueryGraph( + *node.sources()[0], makeDtIf(allowedInDt, PlanType::kOrderBy)); + currentSelect_->orderBy = + translateOrderBy(*reinterpret_cast(&node)); + return currentSelect_; + } + if (name == "Limit") { + if (!contains(allowedInDt, PlanType::kLimit)) { + return wrapInDt(node); + } + makeQueryGraph(*node.sources()[0], makeDtIf(allowedInDt, PlanType::kLimit)); + auto limit = reinterpret_cast(&node); + currentSelect_->limit = limit->count(); + currentSelect_->offset = limit->offset(); + } else { + VELOX_NYI("Unsupported PlanNode {}", name); + } + return currentSelect_; +} + +} // namespace facebook::velox::optimizer diff --git a/verax/ToVelox.cpp b/verax/ToVelox.cpp new file mode 100644 index 0000000..179639e --- /dev/null +++ b/verax/ToVelox.cpp @@ -0,0 +1,620 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/Plan.h" //@manual +#include "optimizer/PlanUtils.h" //@manual +#include "velox/core/PlanNode.h" +#include "velox/exec/HashPartitionFunction.h" +#include "velox/expression/ExprToSubfieldFilter.h" +#include "velox/expression/ScopedVarSetter.h" + +namespace facebook::velox::optimizer { + +using namespace facebook::velox; +using namespace facebook::velox::exec; +using namespace facebook::velox::runner; + +void filterUpdated(BaseTableCP table) { + auto optimization = queryCtx()->optimization(); + std::vector remainingConjuncts; + std::vector pushdownConjuncts; + ScopedVarSetter noAlias(&optimization->makeVeloxExprWithNoAlias(), true); + for (auto filter : table->columnFilters) { + auto typedExpr = optimization->toTypedExpr(filter); + try { + auto evaluator = optimization->evaluator(); + auto pair = velox::exec::toSubfieldFilter(typedExpr, evaluator); + if (!pair.second) { + remainingConjuncts.push_back(std::move(typedExpr)); + continue; + } + pushdownConjuncts.push_back(typedExpr); + } catch (const std::exception& e) { + remainingConjuncts.push_back(std::move(typedExpr)); + } + } + for (auto expr : table->filter) { + remainingConjuncts.push_back(optimization->toTypedExpr(expr)); + } + core::TypedExprPtr remainingFilter; + for (auto conjunct : remainingConjuncts) { + if (!remainingFilter) { + remainingFilter = conjunct; + } else { + remainingFilter = std::make_shared( + BOOLEAN(), + std::vector{remainingFilter, conjunct}, + "and"); + } + } + auto& dataColumns = table->schemaTable->connectorTable->rowType(); + auto* layout = table->schemaTable->columnGroups[0]->layout; + auto connector = layout->connector(); + std::vector columns; + for (int32_t i = 0; i < dataColumns->size(); ++i) { + // Add subfield pruning here. + columns.push_back(connector->metadata()->createColumnHandle( + *layout, dataColumns->nameOf(i))); + } + auto allFilters = std::move(pushdownConjuncts); + if (remainingFilter) { + allFilters.push_back(remainingFilter); + } + std::vector rejectedFilters; + auto handle = connector->metadata()->createTableHandle( + *layout, + columns, + *optimization->evaluator(), + std::move(allFilters), + rejectedFilters); + + optimization->setLeafHandle(table->id(), handle, std::move(rejectedFilters)); + optimization->setLeafSelectivity(*const_cast(table)); +} + +core::PlanNodeId Optimization::nextId(const RelationOp& op) { + auto id = idGenerator_.next(); + recordPlanNodeEstimate(id, op.cost(), ""); + return id; +} + +void Optimization::recordPlanNodeEstimate( + const core::PlanNodeId id, + Cost cost, + const std::string& role) { + auto it = costEstimates_.find(id); + if (it == costEstimates_.end()) { + costEstimates_[id] = {std::make_pair(role, cost)}; + } else { + it->second.push_back(std::make_pair(role, cost)); + } +} + +RelationOpPtr addGather(RelationOpPtr op) { + if (op->distribution().distributionType.isGather) { + return op; + } + if (op->relType() == RelType::kOrderBy) { + auto order = op->distribution(); + Distribution final = Distribution::gather( + op->distribution().distributionType, order.order, order.orderType); + auto* gather = make(op, final, op->columns()); + auto* orderBy = make(gather, order.order, order.orderType); + return orderBy; + } + auto* gather = make( + op, + Distribution::gather(op->distribution().distributionType), + op->columns()); + return gather; +} + +MultiFragmentPlanPtr Optimization::toVeloxPlan( + RelationOpPtr plan, + const MultiFragmentPlan::Options& options) { + options_ = options; + std::vector stages; + if (options_.numWorkers > 1) { + plan = addGather(plan); + } + ExecutableFragment top; + top.fragment.planNode = makeFragment(plan, top, stages); + stages.push_back(std::move(top)); + return std::make_shared( + std::move(stages), options); +} + +RowTypePtr Optimization::makeOutputType(const ColumnVector& columns) { + std::vector names; + std::vector types; + for (auto i = 0; i < columns.size(); ++i) { + auto* column = columns[i]; + auto relation = column->relation(); + if (relation->type() == PlanType::kTable) { + auto* schemaTable = relation->as()->schemaTable; + if (!schemaTable) { + continue; + } + auto* runnerTable = schemaTable->connectorTable; + if (runnerTable) { + auto* runnerColumn = + runnerTable->findColumn(std::string(column->name())); + VELOX_CHECK_NOT_NULL(runnerColumn); + } + } + auto name = makeVeloxExprWithNoAlias_ ? std::string(column->name()) + : column->toString(); + names.push_back(name); + types.push_back(toTypePtr(columns[i]->value().type)); + } + return ROW(std::move(names), std::move(types)); +} + +core::TypedExprPtr Optimization::toAnd(const ExprVector& exprs) { + core::TypedExprPtr result; + for (auto expr : exprs) { + auto conjunct = toTypedExpr(expr); + if (!result) { + result = conjunct; + } else { + result = std::make_shared( + BOOLEAN(), std::vector{result, conjunct}, "and"); + } + } + return result; +} + +core::TypedExprPtr Optimization::toTypedExpr(ExprCP expr) { + switch (expr->type()) { + case PlanType::kColumn: { + auto column = expr->as(); + auto name = makeVeloxExprWithNoAlias_ ? std::string(column->name()) + : column->toString(); + return std::make_shared( + toTypePtr(expr->value().type), name); + } + case PlanType::kCall: { + std::vector inputs; + auto call = expr->as(); + for (auto arg : call->args()) { + inputs.push_back(toTypedExpr(arg)); + } + if (call->name() == toName("cast")) { + return std::make_shared( + toTypePtr(expr->value().type), std::move(inputs), false); + } + return std::make_shared( + toTypePtr(expr->value().type), std::move(inputs), call->name()); + } + case PlanType::kLiteral: { + auto literal = expr->as(); + return std::make_shared( + toTypePtr(literal->value().type), literal->literal()); + } + + default: + VELOX_FAIL("Cannot translate {} to TypeExpr", expr->toString()); + } +} + +// Translates ExprPtrs to FieldAccessTypedExprs. Maintains a set of +// projections and produces a ProjectNode to evaluate distinct +// expressions for non-column Exprs given to toFieldref() and +// related functions. +class TempProjections { + public: + TempProjections(Optimization& optimization, const RelationOp& input) + : optimization_(optimization), input_(input) { + for (auto& column : input_.columns()) { + exprChannel_[column] = nextChannel_++; + names_.push_back(column->toString()); + fieldRefs_.push_back(std::make_shared( + toTypePtr(column->value().type), column->toString())); + } + exprs_.insert(exprs_.begin(), fieldRefs_.begin(), fieldRefs_.end()); + } + + core::FieldAccessTypedExprPtr toFieldRef(ExprCP expr) { + auto it = exprChannel_.find(expr); + if (it == exprChannel_.end()) { + VELOX_CHECK(expr->type() != PlanType::kColumn); + exprChannel_[expr] = nextChannel_++; + exprs_.push_back(optimization_.toTypedExpr(expr)); + names_.push_back(fmt::format("__r{}", nextChannel_ - 1)); + fieldRefs_.push_back(std::make_shared( + toTypePtr(expr->value().type), names_.back())); + return fieldRefs_.back(); + } + return fieldRefs_[it->second]; + } + + template + std::vector toFieldRefs(const ExprVector& exprs) { + std::vector result; + for (auto expr : exprs) { + result.push_back(toFieldRef(expr)); + } + return result; + } + + core::PlanNodePtr maybeProject(core::PlanNodePtr inputNode) { + if (nextChannel_ == input_.columns().size()) { + return inputNode; + } + return std::make_shared( + optimization_.idGenerator().next(), + std::move(names_), + std::move(exprs_), + inputNode); + } + + private: + Optimization& optimization_; + const RelationOp& input_; + int32_t nextChannel_{0}; + std::vector fieldRefs_; + std::vector names_; + std::vector exprs_; + std::unordered_map exprChannel_; +}; + +core::PlanNodePtr Optimization::makeAggregation( + Aggregation& op, + ExecutableFragment& fragment, + std::vector& stages) { + auto input = makeFragment(op.input(), fragment, stages); + TempProjections projections(*this, *op.input()); + + std::vector aggregateNames; + std::vector aggregates; + bool isRawInput = op.step == core::AggregationNode::Step::kPartial || + op.step == core::AggregationNode::Step::kSingle; + int32_t numKeys = op.grouping.size(); + for (auto i = 0; i < op.aggregates.size(); ++i) { + aggregateNames.push_back(op.columns()[i + numKeys]->toString()); + + auto aggregate = op.aggregates[i]; + core::FieldAccessTypedExprPtr mask; + std::vector rawInputTypes; + for (auto type : aggregate->rawInputType()) { + rawInputTypes.push_back(toTypePtr(type)); + } + if (isRawInput) { + if (aggregate->condition()) { + mask = projections.toFieldRef(aggregate->condition()); + } + auto call = std::make_shared( + toTypePtr(op.columns()[numKeys + i]->value().type), + projections.toFieldRefs(aggregate->args()), + aggregate->name()); + aggregates.push_back({call, rawInputTypes, mask, {}, {}, false}); + } else { + auto call = std::make_shared( + toTypePtr(op.columns()[numKeys + i]->value().type), + std::vector{ + std::make_shared( + toTypePtr(aggregate->intermediateType()), + aggregateNames.back())}, + aggregate->name()); + aggregates.push_back({call, rawInputTypes, mask, {}, {}, false}); + } + } + auto keys = projections.toFieldRefs(op.grouping); + auto project = projections.maybeProject(input); + auto r = new core::AggregationNode( + nextId(op), + op.step, + keys, + {}, + aggregateNames, + aggregates, + false, + project); + core::PlanNodePtr ptr; + ptr.reset(r); + return ptr; +} + +core::PlanNodePtr Optimization::makeOrderBy( + OrderBy& op, + ExecutableFragment& fragment, + std::vector& stages) { + if (root_->limit > 0) { + toVeloxLimit_ = root_->limit; + toVeloxOffset_ = root_->offset; + } + ExecutableFragment source; + source.width = options_.numWorkers; + source.taskPrefix = fmt::format("stage{}", ++stageCounter_); + auto input = makeFragment(op.input(), source, stages); + TempProjections projections(*this, *op.input()); + std::vector sortOrder; + for (auto order : op.distribution().orderType) { + sortOrder.push_back( + order == OrderType::kAscNullsFirst ? core::SortOrder(true, true) + : order == OrderType ::kAscNullsLast ? core::SortOrder(true, false) + : order == OrderType::kDescNullsFirst + ? core::SortOrder(false, true) + : core::SortOrder(false, false)); + } + auto keys = projections.toFieldRefs(op.distribution().order); + auto project = projections.maybeProject(input); + core::PlanNodePtr orderByNode; + if (toVeloxLimit_ <= 0) { + orderByNode = std::make_shared( + nextId(op), keys, sortOrder, true, project); + } else { + orderByNode = std::make_shared( + nextId(op), + keys, + sortOrder, + toVeloxLimit_ + toVeloxOffset_, + true, + project); + } + auto localMerge = std::make_shared( + idGenerator_.next(), + keys, + sortOrder, + std::vector{orderByNode}); + + source.fragment.planNode = std::make_shared( + idGenerator_.next(), + core::PartitionedOutputNode::Kind::kPartitioned, + std::vector{}, + 1, + false, + std::make_shared(), + localMerge->outputType(), + VectorSerde::Kind::kPresto, + localMerge); + + core::PlanNodePtr merge = std::make_shared( + idGenerator_.next(), + localMerge->outputType(), + keys, + sortOrder, + VectorSerde::Kind::kPresto); + fragment.width = 1; + fragment.inputStages.push_back(InputStage{merge->id(), source.taskPrefix}); + stages.push_back(std::move(source)); + if (toVeloxLimit_ > 0 || toVeloxOffset_ != 0) { + return std::make_shared( + idGenerator().next(), toVeloxOffset_, toVeloxLimit_, false, merge); + } + return merge; +} + +class HashPartitionFunctionSpec : public core::PartitionFunctionSpec { + public: + HashPartitionFunctionSpec( + RowTypePtr inputType, + std::vector keys) + : inputType_{inputType}, keys_{keys} {} + + std::unique_ptr create( + int numPartitions, + bool localExchange = false) const override { + return std::make_unique( + localExchange, numPartitions, inputType_, keys_); + } + + folly::dynamic serialize() const override { + VELOX_UNREACHABLE(); + } + + std::string toString() const override { + return ""; + } + + private: + const RowTypePtr inputType_; + const std::vector keys_; +}; + +class BroadcastPartitionFunctionSpec : public core::PartitionFunctionSpec { + public: + std::unique_ptr create( + int /* numPartitions */, + bool /*localExchange*/) const override { + return nullptr; + } + + std::string toString() const override { + return "broadcast"; + } + + folly::dynamic serialize() const override { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "BroadcastPartitionFunctionSpec"; + return obj; + } + + static core::PartitionFunctionSpecPtr deserialize( + const folly::dynamic& /* obj */, + void* /* context */) { + return std::make_shared(); + } +}; + +core::PartitionFunctionSpecPtr createPartitionFunctionSpec( + const RowTypePtr& inputType, + const std::vector& keys, + bool isBroadcast) { + if (isBroadcast) { + return std::make_shared(); + } + if (keys.empty()) { + return std::make_shared(); + } else { + std::vector keyIndices; + keyIndices.reserve(keys.size()); + for (const auto& key : keys) { + keyIndices.push_back(inputType->getChildIdx( + dynamic_cast(key.get())->name())); + } + return std::make_shared( + inputType, std::move(keyIndices)); + } +} + +core::PlanNodePtr Optimization::makeFragment( + RelationOpPtr op, + ExecutableFragment& fragment, + std::vector& stages) { + switch (op->relType()) { + case RelType::kProject: { + auto input = makeFragment(op->input(), fragment, stages); + auto project = op->as(); + std::vector names; + std::vector exprs; + for (auto i = 0; i < project->exprs().size(); ++i) { + names.push_back(project->columns()[i]->toString()); + exprs.push_back(toTypedExpr(project->exprs()[i])); + } + return std::make_shared( + nextId(*project), std::move(names), std::move(exprs), input); + } + case RelType::kFilter: { + auto filter = op->as(); + return std::make_shared( + idGenerator_.next(), + toAnd(filter->exprs()), + makeFragment(filter->input(), fragment, stages)); + } + case RelType::kAggregation: { + return makeAggregation(*op->as(), fragment, stages); + } + case RelType::kOrderBy: { + return makeOrderBy(*op->as(), fragment, stages); + } + case RelType::kRepartition: { + ExecutableFragment source; + source.width = options_.numWorkers; + source.taskPrefix = fmt::format("stage{}", ++stageCounter_); + auto sourcePlan = makeFragment(op->input(), source, stages); + TempProjections project(*this, *op->input()); + + auto repartition = op->as(); + auto keys = project.toFieldRefs( + repartition->distribution().partition); + auto& distribution = repartition->distribution(); + if (distribution.distributionType.isGather) { + fragment.width = 1; + } + auto partitioningInput = project.maybeProject(sourcePlan); + auto partitionFunctionFactory = createPartitionFunctionSpec( + partitioningInput->outputType(), keys, distribution.isBroadcast); + if (distribution.isBroadcast) { + source.numBroadcastDestinations = options_.numWorkers; + } + source.fragment.planNode = std::make_shared( + nextId(*op), + distribution.isBroadcast + ? core::PartitionedOutputNode::Kind::kBroadcast + : core::PartitionedOutputNode::Kind::kPartitioned, + keys, + (keys.empty()) ? 1 : options_.numWorkers, + false, + std::move(partitionFunctionFactory), + makeOutputType(repartition->columns()), + VectorSerde::Kind::kPresto, + partitioningInput); + auto exchange = std::make_shared( + idGenerator_.next(), + sourcePlan->outputType(), + VectorSerde::Kind::kPresto); + fragment.inputStages.push_back( + InputStage{exchange->id(), source.taskPrefix}); + stages.push_back(std::move(source)); + return exchange; + } + case RelType::kTableScan: { + auto scan = op->as(); + auto outputType = makeOutputType(scan->columns()); + auto handlePair = leafHandle(scan->baseTable->id()); + if (!handlePair.first) { + filterUpdated(scan->baseTable); + handlePair = leafHandle(scan->baseTable->id()); + VELOX_CHECK_NOT_NULL( + handlePair.first, + "No table for scan {}", + scan->toString(true, true)); + } + std::unordered_map> + assignments; + for (auto column : scan->columns()) { + // TODO: Make assignments have a ConnectorTableHandlePtr instead of + // non-const shared_ptr. + assignments[column->toString()] = + std::const_pointer_cast( + scan->index->layout->connector() + ->metadata() + ->createColumnHandle(*scan->index->layout, column->name())); + } + auto scanNode = std::make_shared( + nextId(*op), + outputType, + std::const_pointer_cast( + handlePair.first), + assignments); + VELOX_CHECK(handlePair.second.empty(), "Expecting no rejected filters"); + fragment.scans.push_back(scanNode); + return scanNode; + } + case RelType::kJoin: { + auto join = op->as(); + TempProjections leftProjections(*this, *op->input()); + TempProjections rightProjections(*this, *join->right); + auto left = makeFragment(op->input(), fragment, stages); + auto right = makeFragment(join->right, fragment, stages); + if (join->method == JoinMethod::kCross) { + auto joinNode = std::make_shared( + nextId(*join), + join->joinType, + nullptr, + leftProjections.maybeProject(left), + rightProjections.maybeProject(right), + makeOutputType(join->columns())); + if (join->filter.empty()) { + return joinNode; + } + return std::make_shared( + idGenerator().next(), toAnd(join->filter), joinNode); + } + auto leftKeys = leftProjections.toFieldRefs(join->leftKeys); + auto rightKeys = rightProjections.toFieldRefs(join->rightKeys); + return std::make_shared( + nextId(*join), + join->joinType, + false, + leftKeys, + rightKeys, + toAnd(join->filter), + leftProjections.maybeProject(left), + rightProjections.maybeProject(right), + makeOutputType(join->columns())); + } + case RelType::kHashBuild: + return makeFragment(op->input(), fragment, stages); + default: + VELOX_FAIL( + "Unsupported RelationOp {}", static_cast(op->relType())); + } + return nullptr; +} + +} // namespace facebook::velox::optimizer diff --git a/verax/VeloxHistory.cpp b/verax/VeloxHistory.cpp new file mode 100644 index 0000000..c095fa2 --- /dev/null +++ b/verax/VeloxHistory.cpp @@ -0,0 +1,83 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/VeloxHistory.h" //@manual +#include "velox/exec/Operator.h" +#include "velox/exec/TaskStats.h" + +namespace facebook::velox::optimizer { + +using namespace facebook::velox::exec; +using namespace facebook::velox::runner; + +bool VeloxHistory::setLeafSelectivity(BaseTable& table) { + auto optimization = queryCtx()->optimization(); + auto handlePair = optimization->leafHandle(table.id()); + auto handle = handlePair.first; + auto string = handle->toString(); + { + auto it = leafSelectivities_.find(string); + if (it != leafSelectivities_.end()) { + std::lock_guard l(mutex_); + table.filterSelectivity = it->second; + return true; + } + } + auto* runnerTable = table.schemaTable->connectorTable; + if (!runnerTable) { + // If there is no physical table to go to: Assume 1/10 if any filters. + if (table.columnFilters.empty() && table.filter.empty()) { + table.filterSelectivity = 1; + } else { + table.filterSelectivity = 0.1; + } + return false; + } + + auto sample = + runnerTable->layouts()[0]->sample(handlePair.first, 1, handlePair.second); + table.filterSelectivity = + static_cast(sample.second) / (sample.first + 1); + recordLeafSelectivity(string, table.filterSelectivity, false); + return true; +} + +void VeloxHistory::recordVeloxExecution( + const RelationOp* op, + const std::vector& plan, + const std::vector& stats) { + std::unordered_map map; + for (auto& task : stats) { + for (auto& pipeline : task.pipelineStats) { + for (auto& op : pipeline.operatorStats) { + map[op.planNodeId] = &op; + } + } + } + for (auto& fragment : plan) { + for (auto& scan : fragment.scans) { + auto scanStats = map[scan->id()]; + std::string handle = scan->tableHandle()->toString(); + recordLeafSelectivity( + handle, + scanStats->outputPositions / + std::max(1, scanStats->rawInputPositions), + true); + } + } +} + +} // namespace facebook::velox::optimizer diff --git a/verax/VeloxHistory.h b/verax/VeloxHistory.h new file mode 100644 index 0000000..2362e11 --- /dev/null +++ b/verax/VeloxHistory.h @@ -0,0 +1,48 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "optimizer/Plan.h" //@manual +#include "velox/exec/TaskStats.h" +#include "velox/runner/MultiFragmentPlan.h" + +namespace facebook::velox::optimizer { + +/// Records and retrieves estimated and actual cardinalities based on Velox +/// handles and execution stats. +class VeloxHistory : public History { + public: + virtual std::optional findCost(RelationOp& op) override { + return std::nullopt; + } + + void recordCost(const RelationOp& op, Cost cost) override {} + + /// Sets the filter selectivity of a table scan. Returns true if there is data + /// to back the estimate and false if this is a pure guess. + bool setLeafSelectivity(BaseTable& table) override; + + /// Stores observed costs and cardinalities from a query execution. If 'op' is + /// non-null, non-leaf costs from non-leaf levels are recorded. Otherwise only + /// leaf scan selectivities are recorded. + virtual void recordVeloxExecution( + const RelationOp* op, + const std::vector& plan, + const std::vector& stats); +}; + +} // namespace facebook::velox::optimizer diff --git a/verax/connectors/CMakeLists.txt b/verax/connectors/CMakeLists.txt new file mode 100644 index 0000000..bb78e36 --- /dev/null +++ b/verax/connectors/CMakeLists.txt @@ -0,0 +1,36 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed 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. + +velox_add_library(velox_connector_metadata ConnectorMetadata.cpp) + +velox_link_libraries( + velox_connector_metadata + velox_common_base velox_memory velox_connector) + +add_subdirectory(hive) + +velox_add_library(velox_connector_split_source ConnectorSplitSource.cpp) + +velox_link_libraries( + velox_connector_split_source + velox_connector_metadata + velox_local_runner + velox_multifragment_plan + velox_common_base + velox_memory + velox_hive_connector + velox_dwio_common + velox_dwio_dwrf_writer + velox_exec + velox_cursor) diff --git a/verax/connectors/ConnectorMetadata.cpp b/verax/connectors/ConnectorMetadata.cpp new file mode 100644 index 0000000..ed95baf --- /dev/null +++ b/verax/connectors/ConnectorMetadata.cpp @@ -0,0 +1,23 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/connectors/ConnectorMetadata.h" //@manual + +namespace facebook::velox::connectors { + +void dummy() {} + +} // namespace facebook::velox::connectors diff --git a/verax/connectors/ConnectorMetadata.h b/verax/connectors/ConnectorMetadata.h new file mode 100644 index 0000000..e7415e0 --- /dev/null +++ b/verax/connectors/ConnectorMetadata.h @@ -0,0 +1,502 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ +#pragma once + +#include "velox/common/memory/HashStringAllocator.h" +#include "velox/connectors/Connector.h" +#include "velox/type/Subfield.h" +#include "velox/type/Type.h" +#include "velox/type/Variant.h" + +namespace facebook::velox::core { +// Forward declare because used in sampling and filtering APIs in +// abstract Connector. The abstract interface does not depend on +// core:: but implementations do. +class ITypedExpr; +using TypedExprPtr = std::shared_ptr; +} // namespace facebook::velox::core + +/// Base classes for schema elements used in execution. A +/// ConnectorMetadata provides access to table information. A Table has a +/// TableLayout for each of its physical organizations, e.g. base table, index, +/// column group, sorted projection etc. A TableLayout has partitioning and +/// ordering properties and a set of Columns. A Column has ColumnStatistics. A +/// TableLayout combined with Column and Subfield selection and +/// optional filters and lookup keys produces a ConnectorTableHandle. A +/// ConnectorTableHandle can be used to build a table scan or index +/// lookup PlanNode. A ConnectorTableHandle can be used for split +/// enumeration. Derived classes of the above connect to different +/// metadata stores and provide different metadata, e.g. order, +/// partitioning, bucketing etc. +namespace facebook::velox::connector { + +/// Represents statistics of a column. The statistics may represent the column +/// across the table or may be calculated over a sample of a layout of the +/// table. All fields are optional. +struct ColumnStatistics { + /// Empty for top level column. Struct member name or string of key for + /// struct or flat map subfield. + std::string name; + + /// If true, the column cannot have nulls. + bool nonNull{false}; + + /// Observed percentage of nulls. 0 does not mean that there are no nulls. + float nullPct{0}; + + /// Minimum observed value for comparable scalar columns. + std::optional min; + + /// Maximum observed value for a comparable scalar. + std::optional max; + + /// For string, varbinary, array and map, the maximum observed number of + /// characters/bytes/elements/key-value pairs. + std::optional maxLength; + + /// Average count of characters/bytes/elements/key-value pairs. + std::optional avgLength; + + /// Estimated number of distinct values. Not specified for complex types. + std::optional numDistinct; + + /// For complex type columns, statistics of children. For array, contains one + /// element describing the array elements. For struct, has one element for + /// each member. For map, has an element for keys and one for values. For flat + /// map, may have one element for each key. In all cases, stats may be + /// missing. + std::vector children; +}; + +/// Base class for column. The column's name and type are immutable but the +/// stats may be set multiple times. +class Column { + public: + virtual ~Column() = default; + + Column(const std::string& name, TypePtr type) : name_(name), type_(type) {} + + const ColumnStatistics* stats() const { + return latestStats_; + } + + ColumnStatistics* mutableStats() { + std::lock_guard l(mutex_); + if (!latestStats_) { + allStats_.push_back(std::make_unique()); + latestStats_ = allStats_.back().get(); + } + return latestStats_; + } + + /// Sets statistics. May be called multipl times if table contents change. + void setStats(std::unique_ptr stats) { + std::lock_guard l(mutex_); + allStats_.push_back(std::move(stats)); + latestStats_ = allStats_.back().get(); + } + + const std::string& name() const { + return name_; + } + + const TypePtr& type() const { + return type_; + } + + /// Returns approximate number of distinct values. Returns 'deflt' if no + /// information. + int64_t approxNumDistinct(int64_t deflt = 1000) const { + auto* s = stats(); + return s && s->numDistinct.has_value() ? s->numDistinct.value() : deflt; + } + + protected: + const std::string name_; + const TypePtr type_; + + // The latest element added to 'allStats_'. + tsan_atomic latestStats_{nullptr}; + + // All statistics recorded for this column. Old values can be purged when the + // containing Schema is not in use. + std::vector> allStats_; + + private: + // Serializes changes to statistics. + std::mutex mutex_; +}; + +class Table; + +/// Represents sorting order. Duplicate of core::SortOrder. Connectors +struct SortOrder { + bool isAscending{true}; + bool isNullsFirst{true}; +}; + +/// Represents a physical manifestation of a table. There is at least +/// one layout but for tables that have multiple sort orders, +/// partitionings, indices, column groups, etc, there is a separate +/// layout for each. The layout represents data at rest. The +/// ConnectorTableHandle represents the query's constraints on the layout a scan +/// or lookup is accessing. +class TableLayout { + public: + TableLayout( + const std::string& name, + const Table* table, + connector::Connector* connector, + std::vector columns, + std::vector partitionColumns, + std::vector orderColumns, + std::vector sortOrder, + std::vector lookupKeys, + bool supportsScan) + : name_(name), + table_(table), + connector_(connector), + columns_(std::move(columns)), + partitionColumns_(std::move(partitionColumns)), + orderColumns_(std::move(orderColumns)), + sortOrder_(std::move(sortOrder)), + lookupKeys_(lookupKeys), + supportsScan_(supportsScan) { + std::vector names; + std::vector types; + for (auto& column : columns_) { + names.push_back(column->name()); + types.push_back(column->type()); + } + rowType_ = ROW(std::move(names), std::move(types)); + } + + virtual ~TableLayout() = default; + + /// Name for documentation. If there are multiple layouts, this is unique + /// within the table. + const std::string name() const { + return name_; + } + + /// Returns the Connector to use for generating ColumnHandles and TableHandles + /// for operations against this layout. + connector::Connector* connector() const { + return connector_; + } + + /// Returns the containing Table. + const Table* table() const { + return table_; + } + + /// List of columns present in this layout. + const std::vector& columns() const; + + /// Set of partitioning columns. The values in partitioning columns determine + /// the location of the row. Joins on equality of partitioning columns are + /// co-located. + const std::vector& partitionColumns() const { + return partitionColumns_; + } + + /// Columns on which content is ordered within the range of rows covered by a + /// Split. + const std::vector& orderColumns() const { + return orderColumns_; + } + + /// Sorting order. Corresponds 1:1 to orderColumns(). + const std::vector& sortOrder() const { + return sortOrder_; + } + + /// Returns the key columns usable for index lookup. This is modeled + /// separately from sortedness since some sorted files may not + /// support lookup. An index lookup has 0 or more equalities + /// followed by up to one range. The equalities need to be on + /// contiguous, leading parts of the column list and the range must + /// be on the next. This coresponds to a multipart key. + const std::vector& lookupKeys() const { + return lookupKeys_; + } + + /// True if a full table scan is supported. Some lookup sources prohibit this. + /// At the same time the dataset may be available in a scannable form in + /// another layout. + bool supportsScan() const { + return supportsScan_; + } + + /// Returns the columns and their names as a RowType. + const RowTypePtr& rowType() const { + return rowType_; + } + + /// Samples 'pct' percent of rows. Applies filters in 'handle' + /// before sampling. Returns {count of sampled, count matching + /// filters}. 'extraFilters' is a list of conjuncts to evaluate in + /// addition to the filters in 'handle'. If 'statistics' is + /// non-nullptr, fills it with post-filter statistics for the + /// subfields in 'fields'. When sampling on demand, it is usually sufficient + /// to look at a subset of all accessed columns, so we specify these instead + /// of defaulting to the columns in 'handle'. 'allocator' is used for + /// temporary memory in gathering statistics. + virtual std::pair sample( + const connector::ConnectorTableHandlePtr& handle, + float pct, + std::vector extraFilters, + const std::vector& fields = {}, + HashStringAllocator* allocator = nullptr, + std::vector* statistics = nullptr) const { + VELOX_UNSUPPORTED("Table class does not support sampling."); + } + + const Column* findColumn(const std::string& name) const { + for (auto& column : columns_) { + if (column->name() == name) { + return column; + } + } + return nullptr; + } + + private: + const std::string name_; + const Table* table_; + connector::Connector* connector_; + std::vector columns_; + const std::vector partitionColumns_; + const std::vector orderColumns_; + const std::vector sortOrder_; + const std::vector lookupKeys_; + const bool supportsScan_; + RowTypePtr rowType_; +}; + +class Schema; + +/// Base class for table. This is used for name resolution. A TableLayout is +/// used for accessing physical organization like partitioning and sort order. +class Table { + public: + virtual ~Table() = default; + + Table(const std::string& name) : name_(name) {} + + const std::string& name() const { + return name_; + } + + /// Returns all columns as RowType. + const RowTypePtr& rowType() const { + return type_; + } + + /// Returns the set of columns as abstract, non-owned + /// columns. Implementations may hav different Column + /// implementations with different options, so we do not return the + /// implementation's columns but an abstract form. + virtual const std::unordered_map& columnMap() + const = 0; + + const Column* findColumn(const std::string& name) const { + auto& map = columnMap(); + auto it = map.find(name); + return it == map.end() ? nullptr : it->second; + } + + virtual const std::vector& layouts() const = 0; + + /// Returns an estimate of the number of rows in 'this'. + virtual uint64_t numRows() const = 0; + + protected: + const std::string name_; + + // Discovered from data. In the event of different types, we take the + // latest (i.e. widest) table type. + RowTypePtr type_; +}; + +/// Describes a single partition of a TableLayout. A TableLayout has at least +/// one partition, even if it has no partitioning columns. +class PartitionHandle { + public: + virtual ~PartitionHandle() = default; +}; + +/// Enumerates splits. The table and partitions to cover are given to +/// ConnectorSplitManager. +class SplitSource { + public: + static constexpr uint32_t kUngroupedGroupId = + std::numeric_limits::max(); + + /// Result of getSplits. Each split belongs to a group. A nullptr split for + /// group means that there are no more splits for the group. In ungrouped + /// execution, the group is always kUngroupedGroupId. + struct SplitAndGroup { + std::shared_ptr split; + uint32_t group{kUngroupedGroupId}; + }; + + virtual ~SplitSource() = default; + + /// Returns a set of splits that cover up to 'targetBytes' of data. + virtual std::vector getSplits(uint64_t targetBytes) = 0; +}; + +class ConnectorSplitManager { + public: + virtual ~ConnectorSplitManager() = default; + + /// Returns the list of all partitions that match the filters in + /// 'tableHandle'. A non-partitioned table returns one partition. + virtual std::vector> listPartitions( + const ConnectorTableHandlePtr& tableHandle) = 0; + + /// Returns a SplitSource that covers the contents of 'partitions'. The set of + /// partitions is exposed separately so that the caller may process the + /// partitions in a specific order or distribute them to specific nodes in a + /// cluster. + virtual std::shared_ptr getSplitSource( + const ConnectorTableHandlePtr& tableHandle, + std::vector> partitions) = 0; +}; + +using SubfieldPtr = std::shared_ptr; + +struct SubfieldPtrHasher { + size_t operator()(const SubfieldPtr& subfield) const { + return subfield->hash(); + } +}; + +struct SubfieldPtrComparer { + bool operator()(const SubfieldPtr& lhs, const SubfieldPtr& rhs) const { + return *lhs == *rhs; + } +}; + +/// Subfield and default value for use in pushing down a complex type cast into +/// a ColumnHandle. +struct TargetSubfield { + SubfieldPtr target; + variant defaultValue; +}; + +using SubfieldMapping = std::unordered_map< + SubfieldPtr, + TargetSubfield, + SubfieldPtrHasher, + SubfieldPtrComparer>; + +/// Describes a set of lookup keys. Lookup keys can be specified for +/// supporting connector types when creating a +/// ConnectorTableHandle. The corresponding DataSource will then be +/// used with a lookup API. The keys should match a prefix of +/// lookupKeys() of the TableLayout when making a +/// ConnectorTableHandle. The leading keys are compared with +/// equality. A trailing key part may be compared with range +/// constraints. The flags have the same meaning as in +/// common::BigintRange and related. +struct LookupKeys { + /// Columns with equality constraints. Must be a prefix of the lookupKeys() in + /// TableLayout. + std::vector equalityColumns; + + /// Column on which a range condition is applied in lookup. Must be the + /// immediately following key in lookupKeys() order after the last column in + /// 'equalities. If 'equalities' is empty, 'rangeColumn' must be the first in + /// lookupKeys() order. + std::optional rangeColumn; + + // True if the lookup has no lower bound for 'rangeColumn'. + bool lowerUnbounded{true}; + + /// true if the lookup specifies no upper bound for 'rangeColumn'. + bool upperUnbounded{true}; + + /// True if rangeColumn > range lookup lower bound. + bool lowerExclusive{false}; + + /// 'true' if rangeColum < upper range lookup value. + bool upperExclusive{false}; + + /// true if matches for a range lookup should be returned in ascending order + /// of the range column. Some lookup sources may support descending order. + bool isAscending{true}; +}; + +class ConnectorMetadata { + public: + virtual ~ConnectorMetadata() = default; + + /// Post-construction initialization. This is called after adding + /// the ConnectorMetadata to the connector so that Connector methods + /// that refer to metadata are available. + virtual void initialize() = 0; + + /// Creates a ColumnHandle for 'columnName'. If the type is a + /// complex type, 'subfields' specifies which subfields need to be + /// retrievd. empty 'subfields' means all are returned. If + /// 'castToType' is present, this can be a type that the column can + /// be cast to. The set of supported casts depends on the + /// connector. In specific, a map may be cast to a struct. For casts + /// between complex types, 'subfieldMapping' maps from the subfield + /// in the data to the subfield in 'castToType'. The defaultValue is + /// produced if the key Subfield does not occur in the + /// data. Subfields of 'castToType that are not covered by + /// 'subfieldMapping' are set to null if 'castToType' is a struct + /// and are absent if 'castToType' is a map. See implementing + /// Connector for exact set of cast and subfield semantics. + virtual ColumnHandlePtr createColumnHandle( + const TableLayout& layoutData, + const std::string& columnName, + std::vector subfields = {}, + std::optional castToType = std::nullopt, + SubfieldMapping subfieldMapping = {}) { + VELOX_UNSUPPORTED(); + } + + /// Returns a ConnectorTableHandle for use in + /// createDataSource. 'filters' are pushed down into the + /// DataSource. 'filters' are expressions involving literals and + /// columns of 'layout'. The filters not supported by the target + /// system are returned in 'rejectedFilters'. 'rejectedFilters' will + /// have to be applied to the data returned by the + /// DataSource. 'rejectedFilters' may or may not be a subset of + /// 'filters' or subexpressions thereof. If 'lookupKeys' is present, + /// these must match the lookupKeys() in 'layout'. + virtual ConnectorTableHandlePtr createTableHandle( + const TableLayout& layout, + std::vector columnHandles, + core::ExpressionEvaluator& evaluator, + std::vector filters, + std::vector& rejectedFilters, + std::optional = std::nullopt) { + VELOX_UNSUPPORTED(); + } + + virtual const Table* findTable(const std::string& name) = 0; + + /// Returns a SplitManager for split enumeration for TableLayouts accessed + /// through 'this'. + virtual ConnectorSplitManager* splitManager() = 0; +}; + +} // namespace facebook::velox::connector diff --git a/verax/connectors/ConnectorSplitSource.cpp b/verax/connectors/ConnectorSplitSource.cpp new file mode 100644 index 0000000..fc963a9 --- /dev/null +++ b/verax/connectors/ConnectorSplitSource.cpp @@ -0,0 +1,45 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/connectors/ConnectorSplitSource.h" //@manual + +namespace facebook::velox::connector { + +std::vector ConnectorSplitSource::getSplits( + uint64_t targetBytes) { + auto splits = source_->getSplits(targetBytes); + std::vector runnerSplits; + // convert the connector::SplitSource::SplitAndGroup to + // runner::SplitSource::SplitAndGroup. + for (auto& s : splits) { + runnerSplits.push_back({s.split, s.group}); + } + return runnerSplits; +} + +std::shared_ptr +ConnectorSplitSourceFactory::splitSourceForScan( + const core::TableScanNode& scan) { + auto handle = scan.tableHandle(); + auto connector = connector::getConnector(handle->connectorId()); + auto partitions = + connector->metadata()->splitManager()->listPartitions(handle); + auto source = + connector->metadata()->splitManager()->getSplitSource(handle, partitions); + return std::make_shared(std::move(source)); +} + +} // namespace facebook::velox::connector diff --git a/verax/connectors/ConnectorSplitSource.h b/verax/connectors/ConnectorSplitSource.h new file mode 100644 index 0000000..6f2d893 --- /dev/null +++ b/verax/connectors/ConnectorSplitSource.h @@ -0,0 +1,43 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/connectors/ConnectorMetadata.h" //@manual +#include "velox/runner/Runner.h" + +namespace facebook::velox::connector { + +/// A runner::SplitSource that encapsulates a connector::SplitSource. +/// runner::SplitSource does not depend on ConnectorMetadata.h, thus we have a +/// proxy between the two. +class ConnectorSplitSource : public runner::SplitSource { + public: + ConnectorSplitSource(std::shared_ptr source) + : source_(std::move(source)) {} + + std::vector getSplits(uint64_t targetBytes) override; + + private: + std::shared_ptr source_; +}; + +/// Generic SplitSourceFactory that delegates the work to ConnectorMetadata. +class ConnectorSplitSourceFactory : public runner::SplitSourceFactory { + public: + std::shared_ptr splitSourceForScan( + const core::TableScanNode& scan) override; +}; + +} // namespace facebook::velox::connector diff --git a/verax/connectors/hive/CMakeLists.txt b/verax/connectors/hive/CMakeLists.txt new file mode 100644 index 0000000..c4685b0 --- /dev/null +++ b/verax/connectors/hive/CMakeLists.txt @@ -0,0 +1,42 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed 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. + +velox_add_library( + velox_hive_connector_metadata + OBJECT + HiveConnectorMetadata.cpp + LocalHiveConnectorMetadata.cpp) + +velox_link_libraries( + velox_hive_connector_metadata + velox_hive_connector + velox_dwio_catalog_fbhive + velox_dwio_dwrf_reader + velox_dwio_dwrf_writer + velox_dwio_orc_reader + velox_dwio_parquet_reader + velox_dwio_parquet_writer + velox_file + velox_hive_partition_function + velox_type_tz + velox_s3fs + velox_hdfs + velox_gcs + velox_abfs) + + + +if(${VELOX_BUILD_TESTING}) + add_subdirectory(tests) +endif() diff --git a/verax/connectors/hive/HiveConnectorMetadata.cpp b/verax/connectors/hive/HiveConnectorMetadata.cpp new file mode 100644 index 0000000..1742574 --- /dev/null +++ b/verax/connectors/hive/HiveConnectorMetadata.cpp @@ -0,0 +1,114 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/connectors/hive/HiveConnectorMetadata.h" //@manual +#include "velox/connectors/hive/HiveConnector.h" +#include "velox/connectors/hive/TableHandle.h" +#include "velox/expression/ExprToSubfieldFilter.h" +#include "velox/expression/FieldReference.h" + +namespace facebook::velox::connector::hive { + +namespace { +HiveColumnHandle::ColumnType columnType( + const HiveTableLayout& layout, + const std::string& columnName) { + auto& columns = layout.hivePartitionColumns(); + for (auto& c : columns) { + if (c->name() == columnName) { + return HiveColumnHandle::ColumnType::kPartitionKey; + } + } + // TODO recognize special names like $path, $bucket etc. + return HiveColumnHandle::ColumnType::kRegular; +} +} // namespace + +ColumnHandlePtr HiveConnectorMetadata::createColumnHandle( + const TableLayout& layout, + const std::string& columnName, + std::vector subfields, + std::optional castToType, + SubfieldMapping subfieldMapping) { + // castToType and subfieldMapping are not yet supported. + VELOX_CHECK(subfieldMapping.empty()); + VELOX_CHECK(!castToType.has_value()); + auto* hiveLayout = reinterpret_cast(&layout); + auto* column = hiveLayout->findColumn(columnName); + auto handle = std::make_shared( + columnName, + columnType(*hiveLayout, columnName), + column->type(), + column->type(), + std::move(subfields)); + return std::dynamic_pointer_cast(handle); +} + +ConnectorTableHandlePtr HiveConnectorMetadata::createTableHandle( + const TableLayout& layout, + std::vector columnHandles, + velox::core::ExpressionEvaluator& evaluator, + std::vector filters, + std::vector& rejectedFilters, + std::optional lookupKeys) { + VELOX_CHECK(!lookupKeys.has_value(), "Hive does not support lookup keys"); + auto* hiveLayout = dynamic_cast(&layout); + + std::vector names; + std::vector types; + for (auto& columnHandle : columnHandles) { + auto* hiveColumn = + reinterpret_cast(columnHandle.get()); + names.push_back(hiveColumn->name()); + types.push_back(hiveColumn->dataType()); + } + auto dataColumns = ROW(std::move(names), std::move(types)); + std::vector remainingConjuncts; + SubfieldFilters subfieldFilters; + for (auto& typedExpr : filters) { + try { + auto pair = velox::exec::toSubfieldFilter(typedExpr, &evaluator); + if (!pair.second) { + remainingConjuncts.push_back(std::move(typedExpr)); + continue; + } + subfieldFilters[std::move(pair.first)] = std::move(pair.second); + } catch (const std::exception& e) { + remainingConjuncts.push_back(std::move(typedExpr)); + } + } + core::TypedExprPtr remainingFilter; + for (auto conjunct : remainingConjuncts) { + if (!remainingFilter) { + remainingFilter = conjunct; + } else { + remainingFilter = std::make_shared( + BOOLEAN(), + std::vector{remainingFilter, conjunct}, + "and"); + } + } + return std::dynamic_pointer_cast( + std::make_shared( + hiveConnector_->connectorId(), + hiveLayout->table()->name(), + true, + std::move(subfieldFilters), + remainingFilter, + std::move(dataColumns))); +} + +} // namespace facebook::velox::connector::hive diff --git a/verax/connectors/hive/HiveConnectorMetadata.h b/verax/connectors/hive/HiveConnectorMetadata.h new file mode 100644 index 0000000..7d4f68e --- /dev/null +++ b/verax/connectors/hive/HiveConnectorMetadata.h @@ -0,0 +1,120 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "optimizer/connectors/ConnectorMetadata.h" //@manual +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/HiveConnector.h" +#include "velox/dwio/common/Options.h" + +namespace facebook::velox::connector::hive { + +/// Describes a single partition of a Hive table. If the table is +/// bucketed, this resolves to a single file. If the table is +/// partitioned and not bucketed, this resolves to a leaf +/// directory. If the table is not bucketed and not partitioned, +/// this resolves to the directory corresponding to the table. +struct HivePartitionHandle : public PartitionHandle { + HivePartitionHandle( + const std::unordered_map> + partitionKeys, + std::optional tableBucketNumber) + : partitionKeys(partitionKeys), tableBucketNumber(tableBucketNumber) {} + + const std::unordered_map> + partitionKeys; + const std::optional tableBucketNumber; +}; + +/// Describes a Hive table layout. Adds a file format and a list of +/// Hive partitioning columns and an optional bucket count to the base +/// TableLayout. The partitioning in TableLayout does not differentiate between +/// bucketing and Hive partitioning columns. The bucketing columns +/// are the 'partitioning' columns minus the +/// 'hivePartitioningColumns'. 'numBuckets' is the number of Hive buckets if +/// 'partitionColumns' differs from 'hivePartitionColumns'. +class HiveTableLayout : public TableLayout { + public: + HiveTableLayout( + const std::string& name, + const Table* table, + connector::Connector* connector, + std::vector columns, + std::vector partitioning, + std::vector orderColumns, + std::vector sortOrder, + std::vector lookupKeys, + std::vector hivePartitionColumns, + dwio::common::FileFormat fileFormat, + std::optional numBuckets = std::nullopt) + : TableLayout( + name, + table, + connector, + columns, + partitioning, + orderColumns, + sortOrder, + lookupKeys, + true), + fileFormat_(fileFormat), + hivePartitionColumns_(hivePartitionColumns), + numBuckets_(numBuckets) {} + + dwio::common::FileFormat fileFormat() const { + return fileFormat_; + } + + const std::vector& hivePartitionColumns() const { + return hivePartitionColumns_; + } + + std::optional numBuckets() const { + return numBuckets_; + } + + protected: + const dwio::common::FileFormat fileFormat_; + const std::vector hivePartitionColumns_; + std::optional numBuckets_; +}; + +class HiveConnectorMetadata : public ConnectorMetadata { + public: + explicit HiveConnectorMetadata(HiveConnector* hiveConnector) + : hiveConnector_(hiveConnector) {} + + ColumnHandlePtr createColumnHandle( + const TableLayout& layout, + const std::string& columnName, + std::vector subfields = {}, + std::optional castToType = std::nullopt, + SubfieldMapping subfieldMapping = {}) override; + + ConnectorTableHandlePtr createTableHandle( + const TableLayout& layout, + std::vector columnHandles, + core::ExpressionEvaluator& evaluator, + std::vector filters, + std::vector& rejectedFilters, + std::optional lookupKeys = std::nullopt) override; + + protected: + HiveConnector* const hiveConnector_; +}; + +} // namespace facebook::velox::connector::hive diff --git a/verax/connectors/hive/LocalHiveConnectorMetadata.cpp b/verax/connectors/hive/LocalHiveConnectorMetadata.cpp new file mode 100644 index 0000000..6556a55 --- /dev/null +++ b/verax/connectors/hive/LocalHiveConnectorMetadata.cpp @@ -0,0 +1,577 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/connectors/hive/LocalHiveConnectorMetadata.h" //@manual +#include "velox/common/base/Fs.h" +#include "velox/connectors/Connector.h" +#include "velox/connectors/hive/HiveConnectorSplit.h" +#include "velox/dwio/common/BufferedInput.h" +#include "velox/dwio/common/Reader.h" +#include "velox/dwio/common/ReaderFactory.h" +#include "velox/dwio/dwrf/common/Statistics.h" +#include "velox/expression/Expr.h" + +namespace facebook::velox::connector::hive { + +std::vector> +LocalHiveSplitManager::listPartitions( + const ConnectorTableHandlePtr& tableHandle) { + // All tables are unpartitioned. + std::unordered_map> empty; + return {std::make_shared(empty, std::nullopt)}; +} + +std::shared_ptr LocalHiveSplitManager::getSplitSource( + const ConnectorTableHandlePtr& tableHandle, + std::vector> partitions) { + // Since there are only unpartitioned tables now, always makes a SplitSource + // that goes over all the files in the handle's layout. + auto tableName = tableHandle->name(); + auto* metadata = getConnector(tableHandle->connectorId())->metadata(); + auto* table = metadata->findTable(tableName); + VELOX_CHECK_NOT_NULL( + table, "Could not find {} in its ConnectorMetadata", tableName); + auto* layout = dynamic_cast(table->layouts()[0]); + VELOX_CHECK_NOT_NULL(layout); + auto files = layout->files(); + return std::make_shared( + files, 2, layout->fileFormat(), layout->connector()->connectorId()); +} + +std::vector LocalHiveSplitSource::getSplits( + uint64_t targetBytes) { + std::vector result; + uint64_t bytes = 0; + for (;;) { + if (currentFile_ >= static_cast(files_.size())) { + result.push_back(SplitSource::SplitAndGroup{nullptr, 0}); + return result; + } + + if (currentSplit_ >= fileSplits_.size()) { + fileSplits_.clear(); + ++currentFile_; + if (currentFile_ >= files_.size()) { + result.push_back(SplitSource::SplitAndGroup{nullptr, 0}); + return result; + } + + currentSplit_ = 0; + auto filePath = files_[currentFile_]; + const auto fileSize = fs::file_size(filePath); + // Take the upper bound. + const int splitSize = std::ceil((fileSize) / splitsPerFile_); + for (int i = 0; i < splitsPerFile_; ++i) { + fileSplits_.push_back( + connector::hive::HiveConnectorSplitBuilder(filePath) + .connectorId(connectorId_) + .fileFormat(format_) + .start(i * splitSize) + .length(splitSize) + .build()); + } + } + result.push_back(SplitAndGroup{std::move(fileSplits_[currentSplit_++]), 0}); + bytes += + reinterpret_cast(result.back().split.get()) + ->length; + if (bytes > targetBytes) { + return result; + } + } +} + +LocalHiveConnectorMetadata::LocalHiveConnectorMetadata( + HiveConnector* hiveConnector) + : HiveConnectorMetadata(hiveConnector), + hiveConfig_( + std::make_shared(hiveConnector_->connectorConfig())), + splitManager_(this) {} + +void LocalHiveConnectorMetadata::initialize() { + auto formatName = hiveConfig_->hiveLocalFileFormat(); + auto path = hiveConfig_->hiveLocalDataPath(); + format_ = formatName == "dwrf" ? dwio::common::FileFormat::DWRF + : formatName == "parquet" ? dwio::common::FileFormat::PARQUET + : dwio::common::FileFormat::UNKNOWN; + makeQueryCtx(); + makeConnectorQueryCtx(); + readTables(path); +} + +void LocalHiveConnectorMetadata::ensureInitialized() const { + std::lock_guard l(mutex_); + if (initialized_) { + return; + } + const_cast(this)->initialize(); + initialized_ = true; +} + +void LocalHiveConnectorMetadata::makeQueryCtx() { + std::unordered_map config; + std::unordered_map> + connectorConfigs; + connectorConfigs[hiveConnector_->connectorId()] = + std::const_pointer_cast(hiveConfig_->config()); + + queryCtx_ = core::QueryCtx::create( + hiveConnector_->executor(), + core::QueryConfig(config), + std::move(connectorConfigs), + cache::AsyncDataCache::getInstance(), + rootPool_->shared_from_this(), + nullptr, + "local_hive_metadata"); +} + +void LocalHiveConnectorMetadata::makeConnectorQueryCtx() { + common::SpillConfig spillConfig; + common::PrefixSortConfig prefixSortConfig; + schemaPool_ = queryCtx_->pool()->addLeafChild("schemaReader"); + connectorQueryCtx_ = std::make_shared( + schemaPool_.get(), + queryCtx_->pool(), + queryCtx_->connectorSessionProperties(hiveConnector_->connectorId()), + &spillConfig, + prefixSortConfig, + std::make_unique( + queryCtx_.get(), schemaPool_.get()), + queryCtx_->cache(), + "scan_for_schema", + "schema", + "N/a", + 0, + queryCtx_->queryConfig().sessionTimezone()); +} + +void LocalHiveConnectorMetadata::readTables(const std::string& path) { + for (auto const& dirEntry : fs::directory_iterator{path}) { + if (!dirEntry.is_directory() || + dirEntry.path().filename().c_str()[0] == '.') { + continue; + } + loadTable(dirEntry.path().filename(), dirEntry.path()); + } +} + +// Feeds the values in 'vector' into 'builder'. +template +void addStats( + velox::dwrf::StatisticsBuilder* builder, + const BaseVector& vector) { + auto* typedVector = vector.asUnchecked>(); + for (auto i = 0; i < typedVector->size(); ++i) { + if (!typedVector->isNullAt(i)) { + reinterpret_cast(builder)->addValues(typedVector->valueAt(i)); + } + } +} + +std::unique_ptr toRunnerStats( + std::unique_ptr dwioStats) { + auto result = std::make_unique(); + // result->numDistinct = dwioStats->numDistinct(); + + return result; +} + +std::pair LocalHiveTableLayout::sample( + const connector::ConnectorTableHandlePtr& handle, + float pct, + std::vector extraFilters, + const std::vector& fields, + HashStringAllocator* allocator, + std::vector* statistics) const { + std::vector> builders; + VELOX_CHECK(extraFilters.empty()); + auto result = sample(handle, pct, fields, allocator, &builders); + if (!statistics) { + return result; + } + statistics->resize(builders.size()); + for (auto i = 0; i < builders.size(); ++i) { + ColumnStatistics runnerStats; + if (builders[i]) { + dwrf::proto::ColumnStatistics proto; + builders[i]->toProto(proto); + dwrf::StatsContext context("", dwrf::WriterVersion::ORIGINAL); + auto wrapper = dwrf::ColumnStatisticsWrapper(&proto); + auto stats = buildColumnStatisticsFromProto(wrapper, context); + runnerStats = *toRunnerStats(std::move(stats)); + } + + (*statistics)[i] = std::move(runnerStats); + } + return result; +} + +std::pair LocalHiveTableLayout::sample( + const connector::ConnectorTableHandlePtr& tableHandle, + float pct, + const std::vector& fields, + HashStringAllocator* /*allocator*/, + std::vector>* statsBuilders) + const { + dwrf::StatisticsBuilderOptions options( + /*stringLengthLimit=*/100, /*initialSize=*/0); + std::vector> builders; + + std::unordered_map< + std::string, + std::shared_ptr> + columnHandles; + std::vector names; + std::vector types; + for (auto& field : fields) { + auto& path = field.path(); + auto column = + dynamic_cast(path[0].get()) + ->name(); + const auto idx = rowType()->getChildIdx(column); + names.push_back(rowType()->nameOf(idx)); + types.push_back(rowType()->childAt(idx)); + columnHandles[names.back()] = + std::make_shared( + names.back(), + connector::hive::HiveColumnHandle::ColumnType::kRegular, + types.back(), + types.back()); + switch (types.back()->kind()) { + case TypeKind::BIGINT: + case TypeKind::INTEGER: + case TypeKind::SMALLINT: + builders.push_back( + std::make_unique(options)); + break; + case TypeKind::REAL: + case TypeKind::DOUBLE: + builders.push_back( + std::make_unique(options)); + break; + case TypeKind::VARCHAR: + builders.push_back( + std::make_unique(options)); + break; + + default: + builders.push_back(nullptr); + } + } + + const auto outputType = ROW(std::move(names), std::move(types)); + int64_t passingRows = 0; + int64_t scannedRows = 0; + for (auto& file : files_) { + // TODO: make createDataSource take a ConnectorTableHandlePtr instead of a + // shared_ptr to mutable handle. + auto handleCopy = + std::const_pointer_cast(tableHandle); + auto connectorQueryCtx = + reinterpret_cast(connector()->metadata()) + ->connectorQueryCtx(); + auto dataSource = connector()->createDataSource( + outputType, handleCopy, columnHandles, connectorQueryCtx.get()); + + auto split = connector::hive::HiveConnectorSplitBuilder(file) + .fileFormat(fileFormat_) + .connectorId(connector()->connectorId()) + .build(); + dataSource->addSplit(split); + constexpr int32_t kBatchSize = 1000; + for (;;) { + ContinueFuture ignore{ContinueFuture::makeEmpty()}; + + auto data = dataSource->next(kBatchSize, ignore).value(); + if (data == nullptr) { + scannedRows += dataSource->getCompletedRows(); + break; + } + passingRows += data->size(); + for (auto column = 0; column < builders.size(); ++column) { + if (!builders[column]) { + continue; + } + auto* builder = builders[column].get(); + auto loadChild = [](RowVectorPtr data, int32_t column) { + data->childAt(column) = + BaseVector::loadedVectorShared(data->childAt(column)); + }; + switch (rowType()->childAt(column)->kind()) { + case TypeKind::SMALLINT: + loadChild(data, column); + addStats( + builder, *data->childAt(column)); + break; + case TypeKind::INTEGER: + loadChild(data, column); + addStats( + builder, *data->childAt(column)); + break; + case TypeKind::BIGINT: + loadChild(data, column); + addStats( + builder, *data->childAt(column)); + break; + case TypeKind::REAL: + loadChild(data, column); + addStats( + builder, *data->childAt(column)); + break; + case TypeKind::DOUBLE: + loadChild(data, column); + addStats( + builder, *data->childAt(column)); + break; + case TypeKind::VARCHAR: + loadChild(data, column); + addStats( + builder, *data->childAt(column)); + break; + + default: + break; + } + } + if (scannedRows + dataSource->getCompletedRows() > + table()->numRows() * (pct / 100)) { + break; + } + } + } + if (statsBuilders) { + *statsBuilders = std::move(builders); + } + return std::pair(scannedRows, passingRows); +} + +void LocalTable::makeDefaultLayout( + std::vector files, + LocalHiveConnectorMetadata& metadata) { + std::vector columns; + for (auto i = 0; i < type_->size(); ++i) { + auto name = type_->nameOf(i); + columns.push_back(columns_[name].get()); + } + auto* connector = metadata.hiveConnector(); + auto format = metadata.fileFormat(); + std::vector empty; + auto layout = std::make_unique( + name_, + this, + connector, + std::move(columns), + empty, + empty, + std::vector{}, + empty, + empty, + format, + std::nullopt); + layout->setFiles(std::move(files)); + exportedLayouts_.push_back(layout.get()); + layouts_.push_back(std::move(layout)); +} + +void LocalHiveConnectorMetadata::loadTable( + const std::string& tableName, + const fs::path& tablePath) { + // open each file in the directory and check their type and add up the row + // counts. + RowTypePtr tableType; + LocalTable* table = nullptr; + std::vector files; + + for (auto const& dirEntry : fs::directory_iterator{tablePath}) { + if (!dirEntry.is_regular_file()) { + continue; + } + // Ignore hidden files. + if (dirEntry.path().filename().c_str()[0] == '.') { + continue; + } + auto it = tables_.find(tableName); + if (it != tables_.end()) { + table = reinterpret_cast(it->second.get()); + } else { + tables_[tableName] = std::make_unique(tableName, format_); + table = tables_[tableName].get(); + } + dwio::common::ReaderOptions readerOptions{schemaPool_.get()}; + readerOptions.setFileFormat(format_); + auto input = std::make_unique( + std::make_shared(dirEntry.path().string()), + readerOptions.memoryPool()); + std::unique_ptr reader = + dwio::common::getReaderFactory(readerOptions.fileFormat()) + ->createReader(std::move(input), readerOptions); + const auto fileType = reader->rowType(); + if (!tableType) { + tableType = fileType; + } else if (fileType->size() > tableType->size()) { + // The larger type is the later since there is only addition of columns. + // TODO: Check the column types are compatible where they overlap. + tableType = fileType; + } + const auto rows = reader->numberOfRows(); + + if (rows.has_value()) { + table->numRows_ += rows.value(); + } + for (auto i = 0; i < fileType->size(); ++i) { + auto name = fileType->nameOf(i); + Column* column; + auto columnIt = table->columns().find(name); + if (columnIt != table->columns().end()) { + column = columnIt->second.get(); + } else { + table->columns()[name] = + std::make_unique(name, fileType->childAt(i)); + column = table->columns()[name].get(); + } + // Initialize the stats from the first file. + if (column->stats() == nullptr) { + auto readerStats = reader->columnStatistics(i); + if (readerStats) { + auto numValues = readerStats->getNumberOfValues(); + column->setStats(toRunnerStats(std::move(readerStats))); + if (rows.has_value() && rows.value() > 0 && numValues.has_value()) { + column->mutableStats()->nullPct = + 100 * (rows.value() - numValues.value()) / rows.value(); + } + } + } + } + files.push_back(dirEntry.path()); + } + VELOX_CHECK_NOT_NULL(table, "Table directory {} is empty", tablePath); + + table->setType(tableType); + table->makeDefaultLayout(std::move(files), *this); + float pct = 10; + if (table->numRows() > 1000000) { + // Set pct to sample ~100K rows. + pct = 100 * 100000 / table->numRows(); + } + table->sampleNumDistincts(pct, schemaPool_.get()); +} + +void LocalTable::sampleNumDistincts(float samplePct, memory::MemoryPool* pool) { + std::vector fields; + for (auto i = 0; i < type_->size(); ++i) { + fields.push_back(common::Subfield(type_->nameOf(i))); + } + + // Sample the table. Adjust distinct values according to the samples. + auto allocator = std::make_unique(pool); + auto* layout = layouts_[0].get(); + std::vector columns; + for (auto i = 0; i < type_->size(); ++i) { + columns.push_back(layout->connector()->metadata()->createColumnHandle( + *layout, type_->nameOf(i))); + } + auto* metadata = dynamic_cast( + layout->connector()->metadata()); + auto& evaluator = *metadata->connectorQueryCtx()->expressionEvaluator(); + std::vector ignore; + auto handle = layout->connector()->metadata()->createTableHandle( + *layout, columns, evaluator, {}, ignore); + std::vector> statsBuilders; + auto* localLayout = dynamic_cast(layout); + VELOX_CHECK_NOT_NULL(localLayout, "Expecting a local hive layout"); + auto [sampled, passed] = localLayout->sample( + handle, samplePct, fields, allocator.get(), &statsBuilders); + numSampledRows_ = sampled; + for (auto i = 0; i < statsBuilders.size(); ++i) { + if (statsBuilders[i]) { + // TODO: Use HLL estimate of distinct values here after this is added to + // the stats builder. Now assume that all rows have a different value. + // Later refine this by observed min-max range. + int64_t approxNumDistinct = numRows_; + // For tiny tables the sample is 100% and the approxNumDistinct is + // accurate. For partial samples, the distinct estimate is left to be the + // distinct estimate of the sample if there are few distincts. This is an + // enumeration where values in unsampled rows are likely the same. If + // there are many distincts, we multiply by 1/sample rate assuming that + // unsampled rows will mostly have new values. + + if (numSampledRows_ < numRows_) { + if (approxNumDistinct > sampled / 50) { + float numDups = + numSampledRows_ / static_cast(approxNumDistinct); + approxNumDistinct = std::min(numRows_, numRows_ / numDups); + + // If the type is an integer type, num distincts cannot be larger than + // max - min. + if (auto* ints = dynamic_cast( + statsBuilders[i].get())) { + auto min = ints->getMinimum(); + auto max = ints->getMaximum(); + if (min.has_value() && max.has_value()) { + auto range = max.value() - min.value(); + approxNumDistinct = std::min(approxNumDistinct, range); + } + } + } + + const_cast(findColumn(type_->nameOf(i))) + ->mutableStats() + ->numDistinct = approxNumDistinct; + } + } + } +} + +const std::unordered_map& LocalTable::columnMap() + const { + std::lock_guard l(mutex_); + if (columns_.empty()) { + return exportedColumns_; + } + for (auto& pair : columns_) { + exportedColumns_[pair.first] = pair.second.get(); + } + return exportedColumns_; +} + +const Table* LocalHiveConnectorMetadata::findTable(const std::string& name) { + ensureInitialized(); + auto it = tables_.find(name); + if (it == tables_.end()) { + return nullptr; + } + return it->second.get(); +} + +namespace { +class LocalHiveConnectorMetadataFactory : public HiveConnectorMetadataFactory { + public: + std::shared_ptr create(HiveConnector* connector) override { + auto hiveConfig = + std::make_shared(connector->connectorConfig()); + auto path = hiveConfig->hiveLocalDataPath(); + if (path.empty()) { + return nullptr; + } + return std::make_shared(connector); + } +}; + +bool dummy = registerHiveConnectorMetadataFactory( + std::make_unique()); +} // namespace + +} // namespace facebook::velox::connector::hive diff --git a/verax/connectors/hive/LocalHiveConnectorMetadata.h b/verax/connectors/hive/LocalHiveConnectorMetadata.h new file mode 100644 index 0000000..81e8335 --- /dev/null +++ b/verax/connectors/hive/LocalHiveConnectorMetadata.h @@ -0,0 +1,241 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "optimizer/connectors/hive/HiveConnectorMetadata.h" //@manual +#include "velox/common/base/Fs.h" +#include "velox/common/memory/HashStringAllocator.h" +#include "velox/connectors/hive/HiveConnector.h" +#include "velox/connectors/hive/TableHandle.h" +#include "velox/core/QueryCtx.h" +#include "velox/dwio/common/Options.h" +#include "velox/dwio/dwrf/writer/StatisticsBuilder.h" + +namespace facebook::velox::connector::hive { + +class LocalHiveSplitSource : public SplitSource { + public: + LocalHiveSplitSource( + std::vector files, + int32_t splitsPerFile, + dwio::common::FileFormat format, + const std::string& connectorId) + : splitsPerFile_(splitsPerFile), + format_(format), + connectorId_(connectorId), + files_(files) {} + + std::vector getSplits( + uint64_t targetBytes) override; + + private: + const int32_t splitsPerFile_; + const dwio::common::FileFormat format_; + const std::string connectorId_; + std::vector files_; + std::vector> fileSplits_; + int32_t currentFile_{-1}; + int32_t currentSplit_{0}; +}; + +class LocalHiveConnectorMetadata; + +class LocalHiveSplitManager : public ConnectorSplitManager { + public: + LocalHiveSplitManager(LocalHiveConnectorMetadata* metadata) + : metadata_(metadata) {} + std::vector> listPartitions( + const ConnectorTableHandlePtr& tableHandle) override; + + std::shared_ptr getSplitSource( + const ConnectorTableHandlePtr& tableHandle, + std::vector> partitions) override; + + private: + LocalHiveConnectorMetadata* metadata_; +}; + +/// A HiveTableLayout backed by local files. Implements sampling by reading +/// local files and stores the file list inside 'this'. +class LocalHiveTableLayout : public HiveTableLayout { + public: + LocalHiveTableLayout( + const std::string& name, + const Table* table, + connector::Connector* connector, + std::vector columns, + std::vector partitioning, + std::vector orderColumns, + std::vector sortOrder, + std::vector lookupKeys, + std::vector hivePartitionColumns, + dwio::common::FileFormat fileFormat, + std::optional numBuckets = std::nullopt) + : HiveTableLayout( + name, + table, + connector, + columns, + partitioning, + orderColumns, + sortOrder, + lookupKeys, + hivePartitionColumns, + fileFormat, + numBuckets) {} + + std::pair sample( + const connector::ConnectorTableHandlePtr& handle, + float pct, + std::vector extraFilters, + const std::vector& fields, + HashStringAllocator* allocator = nullptr, + std::vector* statistics = nullptr) const override; + + const std::vector& files() const { + return files_; + } + + void setFiles(std::vector files) { + files_ = std::move(files); + } + + /// Like sample() above, but fills 'builders' with the data. + std::pair sample( + const connector::ConnectorTableHandlePtr& handle, + float pct, + const std::vector& fields, + HashStringAllocator* allocator, + std::vector>* statsBuilders) + const; + + private: + std::vector files_; +}; + +class LocalTable : public Table { + public: + LocalTable(const std::string& name, dwio::common::FileFormat format) + : Table(name) {} + + std::unordered_map>& columns() { + return columns_; + } + const std::vector& layouts() const override { + return exportedLayouts_; + } + + const std::unordered_map& columnMap() + const override; + + void setType(const RowTypePtr& type) { + type_ = type; + } + + void makeDefaultLayout( + std::vector files, + LocalHiveConnectorMetadata& metadata); + + uint64_t numRows() const override { + return numRows_; + } + + /// Samples 'samplePct' % rows of the table and sets the num distincts + /// estimate for the columns. uses 'pool' for temporary data. + void sampleNumDistincts(float samplePct, memory::MemoryPool* pool); + + private: + // Serializes initialization, e.g. exportedColumns_. + mutable std::mutex mutex_; + + // All columns. Filled by loadTable(). + std::unordered_map> columns_; + + // Non-owning columns map used for exporting the column set as abstract + // columns. + mutable std::unordered_map exportedColumns_; + + /// Table layouts. For a Hive table this is normally one layout with all + /// columns included. + std::vector> layouts_; + + // Copy of 'llayouts_' for use in layouts(). + std::vector exportedLayouts_; + + int64_t numRows_{0}; + int64_t numSampledRows_{0}; + + friend class LocalHiveConnectorMetadata; +}; + +class LocalHiveConnectorMetadata : public HiveConnectorMetadata { + public: + LocalHiveConnectorMetadata(HiveConnector* hiveConector); + + void initialize() override; + + const Table* findTable(const std::string& name) override; + + ConnectorSplitManager* splitManager() override { + ensureInitialized(); + return &splitManager_; + } + + dwio::common::FileFormat fileFormat() const { + return format_; + } + + const std::shared_ptr& connectorQueryCtx() const { + return connectorQueryCtx_; + } + + HiveConnector* hiveConnector() const { + return hiveConnector_; + } + + /// returns the set of known tables. This is not part of the + /// ConnectorMetadata API. This This is only needed for running the + /// DuckDB parser on testing queries since the latter needs a set of + /// tables for name resolution. + const std::unordered_map>& tables() + const { + ensureInitialized(); + return tables_; + } + + private: + void ensureInitialized() const; + void makeQueryCtx(); + void makeConnectorQueryCtx(); + void readTables(const std::string& path); + + void loadTable(const std::string& tableName, const fs::path& tablePath); + + mutable std::mutex mutex_; + mutable bool initialized_{false}; + std::shared_ptr hiveConfig_; + std::shared_ptr rootPool_{ + memory::memoryManager()->addRootPool()}; + std::shared_ptr schemaPool_; + std::shared_ptr queryCtx_; + std::shared_ptr connectorQueryCtx_; + dwio::common::FileFormat format_; + std::unordered_map> tables_; + LocalHiveSplitManager splitManager_; +}; + +} // namespace facebook::velox::connector::hive diff --git a/verax/connectors/hive/tests/CMakeLists.txt b/verax/connectors/hive/tests/CMakeLists.txt new file mode 100644 index 0000000..ca981c0 --- /dev/null +++ b/verax/connectors/hive/tests/CMakeLists.txt @@ -0,0 +1,30 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed 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. +add_executable(velox_hive_connector_metadata_test HiveConnectorMetadataTest.cpp) + +add_test(velox_hive_connector_metadata_test velox_hive_connector_metadata_test) + +target_link_libraries( + velox_hive_connector_metadata_test + velox_hive_connector_metadata + velox_hive_connector + velox_exec_runner_test_util + velox_hive_partition_function + velox_dwio_common_exception + velox_vector_fuzzer + velox_vector_test_lib + velox_exec + velox_exec_test_lib + GTest::gtest + GTest::gtest_main) diff --git a/verax/connectors/hive/tests/HiveConnectorMetadataTest.cpp b/verax/connectors/hive/tests/HiveConnectorMetadataTest.cpp new file mode 100644 index 0000000..4bb8d9e --- /dev/null +++ b/verax/connectors/hive/tests/HiveConnectorMetadataTest.cpp @@ -0,0 +1,97 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/connectors/hive/LocalHiveConnectorMetadata.h" //@manual //@manual +#include "velox/exec/tests/utils/DistributedPlanBuilder.h" +#include "velox/exec/tests/utils/LocalRunnerTestBase.h" +#include "velox/exec/tests/utils/QueryAssertions.h" + +using namespace facebook::velox; +using namespace facebook::velox::exec; +using namespace facebook::velox::exec::test; +using namespace facebook::velox::connector; + +class HiveConnectorMetadataTest : public LocalRunnerTestBase { + protected: + static constexpr int32_t kNumFiles = 5; + static constexpr int32_t kNumVectors = 5; + static constexpr int32_t kRowsPerVector = 10000; + static constexpr int32_t kNumRows = kNumFiles * kNumVectors * kRowsPerVector; + + static void SetUpTestCase() { + // The lambdas will be run after this scope returns, so make captures + // static. + static int32_t counter1; + // Clear 'counter1' so that --gtest_repeat runs get the same data. + counter1 = 0; + auto customize1 = [&](const RowVectorPtr& rows) { + makeAscending(rows, counter1); + }; + + rowType_ = ROW({"c0"}, {BIGINT()}); + testTables_ = {TableSpec{ + .name = "T", + .columns = rowType_, + .rowsPerVector = kRowsPerVector, + .numVectorsPerFile = kNumVectors, + .numFiles = kNumFiles, + .customizeData = customize1}}; + + // Creates the data and schema from 'testTables_'. These are created on the + // first test fixture initialization. + LocalRunnerTestBase::SetUpTestCase(); + } + + static void makeAscending(const RowVectorPtr& rows, int32_t& counter) { + auto ints = rows->childAt(0)->as>(); + for (auto i = 0; i < ints->size(); ++i) { + ints->set(i, counter + i); + } + counter += ints->size(); + } + + inline static RowTypePtr rowType_; +}; + +TEST_F(HiveConnectorMetadataTest, basic) { + auto connector = getConnector(kHiveConnectorId); + auto metadata = connector->metadata(); + ASSERT_TRUE(metadata != nullptr); + auto table = metadata->findTable("T"); + ASSERT_TRUE(table != nullptr); + auto column = table->findColumn("c0"); + ASSERT_TRUE(column != nullptr); + EXPECT_EQ(250'000, table->numRows()); + auto* layout = table->layouts()[0]; + auto columnHandle = metadata->createColumnHandle(*layout, "c0"); + std::vector columns = {columnHandle}; + std::vector filters; + std::vector rejectedFilters; + auto ctx = dynamic_cast(metadata) + ->connectorQueryCtx(); + + auto tableHandle = metadata->createTableHandle( + *layout, columns, *ctx->expressionEvaluator(), filters, rejectedFilters); + EXPECT_TRUE(rejectedFilters.empty()); + std::vector stats; + std::vector fields; + auto c0 = common::Subfield::create("c0"); + fields.push_back(std::move(*c0)); + HashStringAllocator allocator(pool_.get()); + auto pair = layout->sample(tableHandle, 100, {}, fields, &allocator, &stats); + EXPECT_EQ(250'000, pair.first); + EXPECT_EQ(250'000, pair.second); +} diff --git a/verax/tests/CMakeLists.txt b/verax/tests/CMakeLists.txt new file mode 100644 index 0000000..0f9dd31 --- /dev/null +++ b/verax/tests/CMakeLists.txt @@ -0,0 +1,43 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed 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. + +add_executable(velox_plan_test PlanTest.cpp Tpch.cpp ParquetTpchTest.cpp) + +add_test(velox_plan_test velox_plan_test) + +target_link_libraries( + velox_plan_test + velox_verax + velox_tpch_gen + velox_exec_test_lib + velox_dwio_parquet_reader + velox_dwio_native_parquet_reader + gtest + gtest_main) + +add_executable(velox_sql VeloxSql.cpp) + +target_link_libraries( + velox_sql + velox_local_runner + velox_verax + velox_connector_split_source + velox_hive_connector_metadata + velox_exec_test_lib + velox_dwio_common + velox_dwio_parquet_reader + velox_dwio_native_parquet_reader + velox_parse_parser + velox_parse_expression + velox_parse_utils) diff --git a/verax/tests/ParquetTpchTest.cpp b/verax/tests/ParquetTpchTest.cpp new file mode 100644 index 0000000..5c89f5b --- /dev/null +++ b/verax/tests/ParquetTpchTest.cpp @@ -0,0 +1,162 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/tests/ParquetTpchTest.h" //@manual + +DEFINE_string( + data_path, + "", + "Path to TPCH data directory. If empty, the test creates a temp directory and deletes it on exit"); +DEFINE_bool(create_dataset, true, "Creates the TPCH tables"); + +namespace facebook::velox::optimizer::test { +using namespace facebook::velox::exec; +using namespace facebook::velox::exec::test; + +std::shared_ptr ParquetTpchTest::duckDb_; +std::string ParquetTpchTest::createPath_; +std::string ParquetTpchTest::path_; +std::shared_ptr ParquetTpchTest::tempDirectory_; +std::shared_ptr ParquetTpchTest::tpchBuilder_; + +// static +void ParquetTpchTest::SetUpTestSuite() { + memory::MemoryManager::testingSetInstance({}); + + duckDb_ = std::make_shared(); + if (FLAGS_data_path.empty()) { + tempDirectory_ = TempDirectoryPath::create(); + createPath_ = tempDirectory_->getPath(); + path_ = createPath_; + FLAGS_data_path = createPath_; + } else if (FLAGS_create_dataset) { + VELOX_CHECK(!FLAGS_data_path.empty()); + createPath_ = FLAGS_data_path; + path_ = createPath_; + } + tpchBuilder_ = + std::make_shared(dwio::common::FileFormat::PARQUET); + + functions::prestosql::registerAllScalarFunctions(); + aggregate::prestosql::registerAllAggregateFunctions(); + + parse::registerTypeResolver(); + filesystems::registerLocalFileSystem(); + dwio::common::registerFileSinks(); + + parquet::registerParquetReaderFactory(); + parquet::registerParquetWriterFactory(); + + connector::registerConnectorFactory( + std::make_shared()); + auto hiveConnector = + connector::getConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName) + ->newConnector( + kHiveConnectorId, + std::make_shared( + std::unordered_map())); + connector::registerConnector(hiveConnector); + + connector::registerConnectorFactory( + std::make_shared()); + auto tpchConnector = + connector::getConnectorFactory( + connector::tpch::TpchConnectorFactory::kTpchConnectorName) + ->newConnector( + kTpchConnectorId, + std::make_shared( + std::unordered_map())); + connector::registerConnector(tpchConnector); + + if (!createPath_.empty()) { + saveTpchTablesAsParquet(); + } + tpchBuilder_->initialize(path_); +} + +// static +void ParquetTpchTest::TearDownTestSuite() { + connector::unregisterConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName); + connector::unregisterConnectorFactory( + connector::tpch::TpchConnectorFactory::kTpchConnectorName); + connector::unregisterConnector(kHiveConnectorId); + connector::unregisterConnector(kTpchConnectorId); + parquet::unregisterParquetReaderFactory(); + parquet::unregisterParquetWriterFactory(); +} + +void ParquetTpchTest::saveTpchTablesAsParquet() { + std::shared_ptr rootPool{ + memory::memoryManager()->addRootPool()}; + std::shared_ptr pool{rootPool->addLeafChild("leaf")}; + + for (const auto& table : tpch::tables) { + auto tableName = toTableName(table); + auto tableDirectory = fmt::format("{}/{}", createPath_, tableName); + auto tableSchema = tpch::getTableSchema(table); + auto columnNames = tableSchema->names(); + auto plan = PlanBuilder() + .tpchTableScan(table, std::move(columnNames), 0.01) + .planNode(); + auto split = + exec::Split(std::make_shared( + kTpchConnectorId, 1, 0)); + + auto rows = + AssertQueryBuilder(plan).splits({split}).copyResults(pool.get()); + duckDb_->createTable(tableName.data(), {rows}); + + plan = PlanBuilder() + .values({rows}) + .tableWrite(tableDirectory, dwio::common::FileFormat::PARQUET) + .planNode(); + + AssertQueryBuilder(plan).copyResults(pool.get()); + } +} + +std::shared_ptr ParquetTpchTest::assertQuery( + const TpchPlan& tpchPlan, + const std::string& duckQuery, + const std::optional>& sortingKeys) const { + bool noMoreSplits = false; + constexpr int kNumSplits = 10; + constexpr int kNumDrivers = 4; + auto addSplits = [&](Task* task) { + if (!noMoreSplits) { + for (const auto& entry : tpchPlan.dataFiles) { + for (const auto& path : entry.second) { + auto const splits = HiveConnectorTestBase::makeHiveConnectorSplits( + path, kNumSplits, tpchPlan.dataFileFormat); + for (const auto& split : splits) { + task->addSplit(entry.first, Split(split)); + } + } + task->noMoreSplits(entry.first); + } + } + noMoreSplits = true; + }; + CursorParameters params; + params.maxDrivers = kNumDrivers; + params.planNode = tpchPlan.plan; + return exec::test::assertQuery( + params, addSplits, duckQuery, *duckDb_, sortingKeys); +} + +} // namespace facebook::velox::optimizer::test diff --git a/verax/tests/ParquetTpchTest.h b/verax/tests/ParquetTpchTest.h new file mode 100644 index 0000000..315d8e0 --- /dev/null +++ b/verax/tests/ParquetTpchTest.h @@ -0,0 +1,69 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include + +#include "velox/common/file/FileSystems.h" +#include "velox/connectors/tpch/TpchConnector.h" +#include "velox/dwio/parquet/RegisterParquetReader.h" +#include "velox/dwio/parquet/RegisterParquetWriter.h" +#include "velox/exec/tests/utils/AssertQueryBuilder.h" +#include "velox/exec/tests/utils/HiveConnectorTestBase.h" +#include "velox/exec/tests/utils/PlanBuilder.h" +#include "velox/exec/tests/utils/TempDirectoryPath.h" +#include "velox/exec/tests/utils/TpchQueryBuilder.h" +#include "velox/functions/prestosql/aggregates/RegisterAggregateFunctions.h" +#include "velox/functions/prestosql/registration/RegistrationFunctions.h" +#include "velox/parse/TypeResolver.h" + +DECLARE_string(data_path); +DECLARE_bool(create_dataset); + +namespace facebook::velox::optimizer::test { + +class ParquetTpchTest : public testing::Test { + protected: + static void SetUpTestSuite(); + + static void TearDownTestSuite(); + + static void saveTpchTablesAsParquet(); + + void assertQuery( + int queryId, + const std::optional>& sortingKeys = {}) { + auto tpchPlan = tpchBuilder_->getQueryPlan(queryId); + auto duckDbSql = tpch::getQuery(queryId); + assertQuery(tpchPlan, duckDbSql, sortingKeys); + } + + std::shared_ptr assertQuery( + const exec::test::TpchPlan& tpchPlan, + const std::string& duckQuery, + const std::optional>& sortingKeys) const; + + static std::shared_ptr duckDb_; + static std::string createPath_; + static std::string path_; + static std::shared_ptr tempDirectory_; + static std::shared_ptr tpchBuilder_; + + static constexpr char const* kTpchConnectorId{"test-tpch"}; +}; + +} // namespace facebook::velox::optimizer::test diff --git a/verax/tests/PlanTest.cpp b/verax/tests/PlanTest.cpp new file mode 100644 index 0000000..dca3e40 --- /dev/null +++ b/verax/tests/PlanTest.cpp @@ -0,0 +1,185 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/Plan.h" //@manual +#include "optimizer/VeloxHistory.h" //@manual + +#include +#include +#include "optimizer/tests/ParquetTpchTest.h" //@manual +#include "optimizer/tests/Tpch.h" //@manual +#include "velox/common/file/FileSystems.h" +#include "velox/dwio/parquet/RegisterParquetReader.h" +#include "velox/exec/tests/utils/TpchQueryBuilder.h" +#include "velox/expression/Expr.h" +#include "velox/functions/prestosql/aggregates/RegisterAggregateFunctions.h" +#include "velox/functions/prestosql/registration/RegistrationFunctions.h" +#include "velox/parse/TypeResolver.h" + +DEFINE_int32(trace, 0, "Enable trace 1=retained plans, 2=abandoned, 3=both"); + +DEFINE_int32(num_repeats, 1, "Number of repeats for optimization timing"); + +using namespace facebook::velox; +using namespace facebook::velox::optimizer; +using namespace facebook::velox::optimizer::test; + +std::string nodeString(core::PlanNode* node) { + return node->toString(true, true); +} + +class PlanTest : public ParquetTpchTest { + protected: + void SetUp() override { + memory::MemoryManager::testingSetInstance({}); + rootPool_ = memory::memoryManager()->addRootPool("velox_sql"); + pool_ = rootPool_->addLeafChild("optimizer"); + allocator_ = std::make_unique(pool_.get()); + context_ = std::make_unique(*allocator_); + queryCtx() = context_.get(); + functions::prestosql::registerAllScalarFunctions(); + aggregate::prestosql::registerAllAggregateFunctions(); + parse::registerTypeResolver(); + filesystems::registerLocalFileSystem(); + if (!registered) { + registered = true; + parquet::registerParquetReaderFactory(); + } + builder_ = std::make_unique( + dwio::common::FileFormat::PARQUET); + builder_->initialize(FLAGS_data_path); + history_ = std::make_unique(); + makeCheats(); + queryCtx_ = core::QueryCtx::create(); + + evaluator_ = std::make_unique( + queryCtx_.get(), pool_.get()); + } + + void makeCheats() { + history_->recordLeafSelectivity( + "table: lineitem, range filters: [(l_shipdate, BigintRange: [9205, 9223372036854775807] no nulls)]", + 0.5); + history_->recordLeafSelectivity( + "table: orders, range filters: [(o_orderdate, BigintRange: [-9223372036854775808, 9203] no nulls)]", + 0.5); + history_->recordLeafSelectivity( + "table: customer, range filters: [(c_mktsegment, Filter(BytesValues, deterministic, null not allowed))]", + 0.2); + history_->recordLeafSelectivity( + "table: part, remaining filter: (like(ROW[\"p_name\"],\"%green%\"))", + 1.0 / 17); + } + + std::string makePlan( + std::shared_ptr plan, + bool partitioned, + bool ordered, + int numRepeats = FLAGS_num_repeats) { + auto schema = tpchSchema(100, partitioned, ordered, false); + std::string string; + for (auto counter = 0; counter < numRepeats; ++counter) { + Optimization opt(*plan, *schema, *history_, *evaluator_, FLAGS_trace); + auto result = opt.bestPlan(); + if (counter == numRepeats - 1) { + string = result->toString(true); + } + } + return fmt::format( + "=== {} {}:\n{}\n", + partitioned ? "Partitioned on PK" : "Not partitioned", + ordered ? "sorted on PK" : "not sorted", + string); + } + + std::shared_ptr rootPool_; + std::shared_ptr pool_; + + std::unique_ptr allocator_; + + std::unique_ptr context_; + std::unique_ptr history_; + std::shared_ptr queryCtx_; + std::unique_ptr evaluator_; + std::unique_ptr builder_; + static inline bool registered; +}; + +TEST_F(PlanTest, queryGraph) { + TypePtr row1 = ROW({{"c1", ROW({{"c1a", INTEGER()}})}, {"c2", DOUBLE()}}); + TypePtr row2 = row1 = + ROW({{"c1", ROW({{"c1a", INTEGER()}})}, {"c2", DOUBLE()}}); + TypePtr largeRow = ROW( + {{"c1", ROW({{"c1a", INTEGER()}})}, + {"c2", DOUBLE()}, + {"m1", MAP(INTEGER(), ARRAY(INTEGER()))}}); + TypePtr differentNames = + ROW({{"different", ROW({{"c1a", INTEGER()}})}, {"c2", DOUBLE()}}); + + auto* dedupRow1 = toType(row1); + auto* dedupRow2 = toType(row2); + auto* dedupLargeRow = toType(largeRow); + auto* dedupDifferentNames = toType(differentNames); + + // dedupped complex types make a copy. + EXPECT_NE(row1.get(), dedupRow1); + + // Identical types get equal pointers. + EXPECT_EQ(dedupRow1, dedupRow2); + + // Different names differentiate types. + EXPECT_NE(dedupDifferentNames, dedupRow1); + + // Shared complex substructure makes equal pointers. + EXPECT_EQ(dedupRow1->childAt(0).get(), dedupLargeRow->childAt(0).get()); + + // Identical child types with different names get equal pointers. + EXPECT_EQ(dedupRow1->childAt(0).get(), dedupDifferentNames->childAt(0).get()); +} + +TEST_F(PlanTest, q3) { + auto q = builder_->getQueryPlan(3).plan; + auto result = makePlan(q, true, true); + std::cout << result; + result = makePlan(q, true, false); + std::cout << result; +} + +TEST_F(PlanTest, q9) { + auto q = builder_->getQueryPlan(9).plan; + auto result = makePlan(q, true, true); + std::cout << result; + result = makePlan(q, true, false); + std::cout << result; +} + +TEST_F(PlanTest, q17) { + auto q = builder_->getQueryPlan(17).plan; + auto result = makePlan(q, true, true); + std::cout << result; + result = makePlan(q, true, false); + std::cout << result; +} + +void printPlan(core::PlanNode* plan, bool r, bool d) { + std::cout << plan->toString(r, d) << std::endl; +} + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::Init init(&argc, &argv, false); + return RUN_ALL_TESTS(); +} diff --git a/verax/tests/Tpch.cpp b/verax/tests/Tpch.cpp new file mode 100644 index 0000000..9745afe --- /dev/null +++ b/verax/tests/Tpch.cpp @@ -0,0 +1,143 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "optimizer/tests/Tpch.h" //@manual +#include "velox/tpch/gen/TpchGen.h" + +namespace facebook::velox::optimizer { + +using namespace facebook::velox; +// Returns a map from column name to cardinality. Positive means fixed value +// count, negative means the count is scale * (-count). +const std::unordered_map& tpchColumns() { + static std::unordered_map columns; + if (columns.empty()) { + columns["o_orderkey"] = -1500000; + columns["o_custkey"] = -150000; + + columns["l_orderkey"] = -1500000; + columns["l_linenumber"] = 4; + columns["l_partkey"] = -200000; + columns["l_suppkey"] = 10000; + columns["l_discount"] = 10; + columns["l_tax"] = 10; + columns["l_shipmode"] = 7; + columns["l_shipdate"] = 7 * 365; + columns["l_commitdate"] = 7 * 365; + columns["l_receiptdate"] = 7 * 365; + + columns["c_custkey"] = -150000; + columns["c_mktsegment"] = 5; + + columns["p_partkey"] = -200000; + columns["s_suppkey"] = -10000; + columns["ps_partkey"] = -200000; + columns["ps_suppkey"] = -10000; + + columns["n_nationkey"] = 25; + columns["n_name"] = 25; + columns["n_regionkey"] = 5; + columns["r_regionkey"] = 5; + columns["r_name"] = 5; + } + return columns; +} + +Value columnValue( + const std::string& name, + const velox::TypePtr& type, + int32_t scale, + int64_t cardinality) { + auto& columns = tpchColumns(); + auto it = columns.find(name); + if (it == columns.end()) { + return Value(type.get(), std::min(5, cardinality / 1000)); + } + return Value(type.get(), it->second > 0 ? it->second : -it->second * scale); +} + +SchemaTableCP makeTable( + tpch::Table id, + int32_t scale, + bool partitioned, + bool ordered, + bool secondary) { + VELOX_CHECK(!secondary, "Secondary indices not implemented"); + auto cardinality = tpch::getRowCount(id, scale); + + auto type = tpch::getTableSchema(id); + auto tableName = tpch::toTableName(id); + auto* table = make( + toName(std::string(tableName.data(), tableName.size())), type); + ColumnVector orderedColumns; + for (auto i = 0; i < type->size(); ++i) { + auto name = toName(type->nameOf(i)); + auto value = columnValue(name, type->childAt(i), scale, cardinality); + orderedColumns.push_back(table->column(name, value)); + } + int32_t numOrder = + id == tpch::Table::TBL_LINEITEM || id == tpch::Table::TBL_PARTSUPP ? 2 + : 1; + auto pkColumns = orderedColumns; + if (id == tpch::Table::TBL_LINEITEM) { + // Swap 2nd and 4th so that l_linenumber is second, since sorting or + // uniqueness defining columns must be first in index. + std::swap(pkColumns[1], pkColumns[3]); + } + ColumnVector partition; + DistributionType dist; + if (partitioned) { + partition.push_back(pkColumns[0]); + dist = DistributionType{ShuffleMode::kHive, 100}; + } + table->addIndex( + "pk", + cardinality, + numOrder, + ordered ? numOrder : 0, + pkColumns, + dist, + partition, + orderedColumns); + return table; +} + +SchemaP +tpchSchema(int32_t scale, bool partitioned, bool ordered, bool secondary) { + auto title = + fmt::format("tpch{}{}", partitioned ? "p" : "", ordered ? "o" : ""); + std::vector tables{ + makeTable( + tpch::Table::TBL_LINEITEM, scale, partitioned, ordered, secondary), + makeTable( + tpch::Table::TBL_ORDERS, scale, partitioned, ordered, secondary), + makeTable( + tpch::Table::TBL_CUSTOMER, scale, partitioned, ordered, secondary), + makeTable(tpch::Table::TBL_PART, scale, partitioned, ordered, secondary), + makeTable( + tpch::Table::TBL_PARTSUPP, scale, partitioned, ordered, secondary), + makeTable( + tpch::Table::TBL_SUPPLIER, scale, partitioned, ordered, secondary), + makeTable( + tpch::Table::TBL_NATION, scale, partitioned, ordered, secondary), + makeTable( + tpch::Table::TBL_REGION, scale, partitioned, ordered, secondary), + }; + + return make(toName(title), std::move(tables), nullptr); +} + +} // namespace facebook::velox::optimizer diff --git a/verax/tests/Tpch.h b/verax/tests/Tpch.h new file mode 100644 index 0000000..1d69f57 --- /dev/null +++ b/verax/tests/Tpch.h @@ -0,0 +1,26 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "optimizer/QueryGraph.h" //@manual + +namespace facebook::velox::optimizer { + +SchemaP +tpchSchema(int32_t scale, bool partitioned, bool ordered, bool secondary); + +} diff --git a/verax/tests/VeloxSql.cpp b/verax/tests/VeloxSql.cpp new file mode 100644 index 0000000..0381bc6 --- /dev/null +++ b/verax/tests/VeloxSql.cpp @@ -0,0 +1,775 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include +#include +#include +#include +#include + +#include "optimizer/connectors/hive/LocalHiveConnectorMetadata.h" //@manual +#include "velox/common/base/SuccinctPrinter.h" +#include "velox/common/file/FileSystems.h" +#include "velox/common/memory/MmapAllocator.h" +#include "velox/connectors/hive/HiveConnector.h" +#include "velox/dwio/common/Options.h" +#include "velox/dwio/dwrf/RegisterDwrfReader.h" +#include "velox/dwio/dwrf/reader/DwrfReader.h" +#include "velox/dwio/parquet/RegisterParquetReader.h" +#include "velox/exec/Exchange.h" + +#include "optimizer/Plan.h" //@manual +#include "optimizer/SchemaResolver.h" //@manual +#include "optimizer/VeloxHistory.h" //@manual +#include "optimizer/connectors/ConnectorSplitSource.h" //@manual +#include "velox/exec/PlanNodeStats.h" +#include "velox/exec/Split.h" +#include "velox/exec/tests/utils/HiveConnectorTestBase.h" +#include "velox/exec/tests/utils/LocalExchangeSource.h" +#include "velox/expression/Expr.h" +#include "velox/functions/prestosql/aggregates/RegisterAggregateFunctions.h" +#include "velox/functions/prestosql/registration/RegistrationFunctions.h" +#include "velox/parse/QueryPlanner.h" +#include "velox/parse/TypeResolver.h" +#include "velox/runner/LocalRunner.h" +#include "velox/serializers/PrestoSerializer.h" +#include "velox/vector/VectorSaver.h" + +using namespace facebook::velox; +using namespace facebook::velox::exec; +using namespace facebook::velox::runner; +using namespace facebook::velox::exec::test; +using namespace facebook::velox::dwio::common; + +namespace { +static bool notEmpty(const char* /*flagName*/, const std::string& value) { + return !value.empty(); +} + +static bool validateDataFormat(const char* flagname, const std::string& value) { + if ((value.compare("parquet") == 0) || (value.compare("dwrf") == 0)) { + return true; + } + std::cout + << fmt::format( + "Invalid value for --{}: {}. Allowed values are [\"parquet\", \"dwrf\"]", + flagname, + value) + << std::endl; + return false; +} + +int32_t printResults(const std::vector& results) { + int32_t numRows = 0; + std::cout << "Results:" << std::endl; + bool printType = true; + for (const auto& vector : results) { + // Print RowType only once. + if (printType) { + std::cout << vector->type()->asRow().toString() << std::endl; + printType = false; + } + for (vector_size_t i = 0; i < vector->size(); ++i) { + std::cout << vector->toString(i) << std::endl; + ++numRows; + } + } + return numRows; +} +} // namespace + +DEFINE_string( + data_path, + "", + "Root path of data. Data layout must follow Hive-style partitioning. "); + +DEFINE_string(ssd_path, "", "Directory for local SSD cache"); +DEFINE_int32(ssd_cache_gb, 0, "Size of local SSD cache in GB"); +DEFINE_int32( + ssd_checkpoint_interval_gb, + 8, + "Make a checkpoint after every n GB added to SSD cache"); + +DEFINE_int32(optimizer_trace, 0, "Optimizer trace level"); + +DEFINE_bool(print_plan, false, "Print optimizer results"); + +DEFINE_bool(print_stats, false, "print statistics"); +DEFINE_bool( + include_custom_stats, + false, + "Include custom statistics along with execution statistics"); +DEFINE_int32(max_rows, 100, "Max number of printed result rows"); +DEFINE_int32(num_drivers, 4, "Number of drivers"); +DEFINE_int32(num_workers, 4, "Number of in-process workers"); + +DEFINE_string(data_format, "parquet", "Data format"); +DEFINE_int32(num_splits_per_file, 10, "Number of splits per file"); +DEFINE_int32( + cache_gb, + 0, + "GB of process memory for cache and query.. if " + "non-0, uses mmap to allocator and in-process data cache."); +DEFINE_int32(num_repeats, 1, "Number of times to run --query"); +DEFINE_string( + query, + "", + "Text of query. If empty, reads ';' separated queries from standard input"); + +DEFINE_string( + record, + "", + "Name of SQL file with a single query. Writes the " + "output to .ref for use with --check"); +DEFINE_string( + check, + "", + "Name of SQL file with a single query. Runs and " + "compares with .ref, previously recorded with --record"); + +DEFINE_validator(data_path, ¬Empty); +DEFINE_validator(data_format, &validateDataFormat); + +struct RunStats { + std::map flags; + int64_t micros{0}; + int64_t rawInputBytes{0}; + int64_t userNanos{0}; + int64_t systemNanos{0}; + std::string output; + + std::string toString(bool detail) { + std::stringstream out; + out << succinctNanos(micros * 1000) << " " + << succinctBytes(rawInputBytes / (micros / 1000000.0)) << "/s raw, " + << succinctNanos(userNanos) << " user " << succinctNanos(systemNanos) + << " system (" << (100 * (userNanos + systemNanos) / (micros * 1000)) + << "%)"; + if (!flags.empty()) { + out << ", flags: "; + for (auto& pair : flags) { + out << pair.first << "=" << pair.second << " "; + } + } + out << std::endl << std::endl; + if (detail) { + out << std::endl << output << std::endl; + } + return out.str(); + } +}; + +class VeloxRunner { + public: + void initialize() { + if (FLAGS_cache_gb) { + memory::MemoryManagerOptions options; + int64_t memoryBytes = FLAGS_cache_gb * (1LL << 30); + options.useMmapAllocator = true; + options.allocatorCapacity = memoryBytes; + options.useMmapArena = true; + options.mmapArenaCapacityRatio = 1; + memory::MemoryManager::testingSetInstance(options); + std::unique_ptr ssdCache; + if (FLAGS_ssd_cache_gb) { + constexpr int32_t kNumSsdShards = 16; + cacheExecutor_ = + std::make_unique(kNumSsdShards); + const cache::SsdCache::Config config( + FLAGS_ssd_path, + static_cast(FLAGS_ssd_cache_gb) << 30, + kNumSsdShards, + cacheExecutor_.get(), + static_cast(FLAGS_ssd_checkpoint_interval_gb) << 30); + ssdCache = std::make_unique(config); + } + + cache_ = cache::AsyncDataCache::create( + memory::memoryManager()->allocator(), std::move(ssdCache)); + cache::AsyncDataCache::setInstance(cache_.get()); + } else { + memory::MemoryManager::testingSetInstance({}); + } + + rootPool_ = memory::memoryManager()->addRootPool("velox_sql"); + + optimizerPool_ = rootPool_->addLeafChild("optimizer"); + schemaPool_ = rootPool_->addLeafChild("schema"); + checkPool_ = rootPool_->addLeafChild("check"); + + functions::prestosql::registerAllScalarFunctions(); + aggregate::prestosql::registerAllAggregateFunctions(); + parse::registerTypeResolver(); + filesystems::registerLocalFileSystem(); + parquet::registerParquetReaderFactory(); + dwrf::registerDwrfReaderFactory(); + exec::ExchangeSource::registerFactory( + exec::test::createLocalExchangeSource); + serializer::presto::PrestoVectorSerde::registerVectorSerde(); + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kPresto)) { + serializer::presto::PrestoVectorSerde::registerNamedVectorSerde(); + } + ioExecutor_ = std::make_unique(8); + std::unordered_map connectorConfig; + connectorConfig[connector::hive::HiveConfig::kLocalDataPath] = + FLAGS_data_path; + connectorConfig[connector::hive::HiveConfig::kLocalFileFormat] = + FLAGS_data_format; + auto config = + std::make_shared(std::move(connectorConfig)); + connector::registerConnectorFactory( + std::make_shared()); + connector_ = + connector::getConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName) + ->newConnector(kHiveConnectorId, config, ioExecutor_.get()); + connector::registerConnector(connector_); + + std::unordered_map> + connectorConfigs; + auto copy = hiveConfig_; + connectorConfigs[kHiveConnectorId] = + std::make_shared(std::move(copy)); + + schemaQueryCtx_ = core::QueryCtx::create( + executor_.get(), + core::QueryConfig(config_), + std::move(connectorConfigs), + cache::AsyncDataCache::getInstance(), + rootPool_->shared_from_this(), + spillExecutor_.get(), + "schema"); + common::SpillConfig spillConfig; + common::PrefixSortConfig prefixSortConfig; + + schemaRootPool_ = rootPool_->addAggregateChild("schemaRoot"); + connectorQueryCtx_ = std::make_shared( + schemaPool_.get(), + schemaRootPool_.get(), + schemaQueryCtx_->connectorSessionProperties(kHiveConnectorId), + &spillConfig, + prefixSortConfig, + std::make_unique( + schemaQueryCtx_.get(), schemaPool_.get()), + schemaQueryCtx_->cache(), + "scan_for_schema", + "schema", + "N/a", + 0, + schemaQueryCtx_->queryConfig().sessionTimezone()); + + schema_ = std::make_shared( + connector_, ""); + + planner_ = std::make_unique(optimizerPool_.get()); + auto& tables = dynamic_cast( + connector_->metadata()) + ->tables(); + for (auto& pair : tables) { + planner_->registerTable(pair.first, pair.second->rowType()); + } + planner_->registerTableScan( + [this]( + const std::string& id, + const std::string& name, + const RowTypePtr& rowType, + const std::vector& columnNames) { + return toTableScan(id, name, rowType, columnNames); + }); + history_ = std::make_unique(); + executor_ = std::make_shared( + FLAGS_num_drivers * 2 + 2); + spillExecutor_ = std::make_shared(4); + } + + core::PlanNodePtr toTableScan( + const std::string& id, + const std::string& name, + const RowTypePtr& rowType, + const std::vector& columnNames) { + using namespace connector::hive; + auto handle = std::make_shared( + kHiveConnectorId, name, true, SubfieldFilters{}, nullptr); + std::unordered_map> + assignments; + + auto table = connector_->metadata()->findTable(name); + for (auto i = 0; i < rowType->size(); ++i) { + auto projectedName = rowType->nameOf(i); + auto& columnName = columnNames[i]; + VELOX_CHECK( + table->columnMap().find(columnName) != table->columnMap().end(), + "No column {} in {}", + columnName, + name); + assignments[projectedName] = std::make_shared( + columnName, + HiveColumnHandle::ColumnType::kRegular, + rowType->childAt(i), + rowType->childAt(i)); + } + return std::make_shared( + id, rowType, handle, assignments); + } + + void runInner( + LocalRunner& runner, + std::vector& result, + RunStats& stats) { + uint64_t micros = 0; + { + struct rusage start; + getrusage(RUSAGE_SELF, &start); + MicrosecondTimer timer(µs); + while (auto rows = runner.next()) { + result.push_back(rows); + } + + struct rusage final; + getrusage(RUSAGE_SELF, &final); + auto tvNanos = [](struct timeval tv) { + return tv.tv_sec * 1000000000 + tv.tv_usec * 1000; + }; + stats.userNanos = tvNanos(final.ru_utime) - tvNanos(start.ru_utime); + stats.systemNanos = tvNanos(final.ru_stime) - tvNanos(start.ru_stime); + } + stats.micros = micros; + } + + /// stores results and plans to 'ref', to be used with --check. + void setRecordStream(std::ofstream* ref) { + record_ = ref; + } + + /// Compares results to data in 'ref'. 'ref' is produced with --record. + void setCheckStream(std::ifstream* ref) { + check_ = ref; + } + + void run(const std::string& sql) { + if (record_ || check_) { + std::string error; + std::string plan; + std::vector result; + run1(sql, nullptr, nullptr, &error); + if (error.empty()) { + run1(sql, &result, &plan, &error); + } + if (record_) { + if (!error.empty()) { + writeString(error, *record_); + } else { + writeString("", *record_); + writeString(plan, *record_); + writeVectors(result, *record_); + } + } else if (check_) { + auto refError = readString(*check_); + if (refError != error) { + ++numFailed_; + std::cerr << "Expected error " + << (refError.empty() ? std::string("no error") : refError) + << " got " + << (error.empty() ? std::string("no error") : error) + << std::endl; + if (!refError.empty()) { + readString(*check_); + readVectors(*check_); + } + return; + } + if (!error.empty()) { + // errors matched. + return; + } + auto refPlan = readString(*check_); + auto refResult = readVectors(*check_); + bool planMiss = false; + bool resultMiss = false; + if (plan != refPlan) { + std::cerr << "Plan mismatch: Expected " << refPlan << std::endl + << " got " << plan << std::endl; + ++numPlanMismatch_; + planMiss = true; + } + if (!assertEqualResults(refResult, result)) { + ++numResultMismatch_; + resultMiss = true; + } + if (!resultMiss && !planMiss) { + ++numPassed_; + } else { + ++numFailed_; + } + } + } else { + run1(sql); + } + } + + std::string planCostString( + const core::PlanNodeId& id, + const facebook::velox::optimizer::Optimization::PlanCostMap& estimates) { + auto it = estimates.find(id); + if (it == estimates.end()) { + return ""; + } + std::stringstream out; + for (auto& pair : it->second) { + out << pair.first << ": " << pair.second.toString(true); + } + return out.str(); + } + + /// Runs a query and returns the result as a single vector in *resultVector, + /// the plan text in *planString and the error message in *errorString. + /// *errorString is not set if no error. Any of these may be nullptr. + std::shared_ptr run1( + const std::string& sql, + std::vector* resultVector = nullptr, + std::string* planString = nullptr, + std::string* errorString = nullptr, + std::vector* statsReturn = nullptr) { + std::shared_ptr runner; + std::unordered_map> + connectorConfigs; + auto copy = hiveConfig_; + connectorConfigs[kHiveConnectorId] = + std::make_shared(std::move(copy)); + ++queryCounter_; + auto queryCtx = core::QueryCtx::create( + executor_.get(), + core::QueryConfig(config_), + std::move(connectorConfigs), + cache::AsyncDataCache::getInstance(), + rootPool_->shared_from_this(), + spillExecutor_.get(), + fmt::format("query_{}", queryCounter_)); + + // The default Locus for planning is the system and data of 'connector_'. + optimizer::Locus locus(connector_->connectorId().c_str(), connector_.get()); + core::PlanNodePtr plan; + try { + plan = planner_->plan(sql); + } catch (std::exception& e) { + std::cerr << "parse error: " << e.what() << std::endl; + if (errorString) { + *errorString = fmt::format("Parse error: {}", e.what()); + } + return nullptr; + } + facebook::velox::optimizer::Optimization::PlanCostMap estimates; + MultiFragmentPlan::Options opts; + opts.numWorkers = FLAGS_num_workers; + opts.numDrivers = FLAGS_num_drivers; + auto allocator = + std::make_unique(optimizerPool_.get()); + auto context = + std::make_unique( + *allocator); + facebook::velox::optimizer::queryCtx() = context.get(); + exec::SimpleExpressionEvaluator evaluator( + queryCtx.get(), optimizerPool_.get()); + MultiFragmentPlanPtr fragmentedPlan; + try { + facebook::velox::optimizer::Schema veraxSchema( + "test", schema_.get(), &locus); + facebook::velox::optimizer::Optimization opt( + *plan, veraxSchema, *history_, evaluator, FLAGS_optimizer_trace); + auto best = opt.bestPlan(); + if (planString) { + *planString = best->op->toString(true, false); + } + if (FLAGS_print_plan) { + std::cout << "Plan: " << best->toString(true); + } + if (FLAGS_print_stats) { + estimates = opt.planCostMap(); + } + fragmentedPlan = opt.toVeloxPlan(best->op, opts); + } catch (const std::exception& e) { + facebook::velox::optimizer::queryCtx() = nullptr; + std::cerr << "optimizer error: " << e.what() << std::endl; + if (errorString) { + *errorString = fmt::format("optimizer error: {}", e.what()); + } + return nullptr; + } + facebook::velox::optimizer::queryCtx() = nullptr; + RunStats runStats; + try { + runner = std::make_shared( + fragmentedPlan, + queryCtx, + std::make_shared()); + std::vector results; + runInner(*runner, results, runStats); + + int numRows = printResults(results); + if (resultVector) { + *resultVector = results; + } + auto stats = runner->stats(); + if (statsReturn) { + *statsReturn = stats; + } + auto& fragments = fragmentedPlan->fragments(); + for (int32_t i = fragments.size() - 1; i >= 0; --i) { + for (auto& pipeline : stats[i].pipelineStats) { + auto& first = pipeline.operatorStats[0]; + if (first.operatorType == "TableScan") { + runStats.rawInputBytes += first.rawInputBytes; + } + } + if (FLAGS_print_stats) { + std::cout << "Fragment " << i << ":" << std::endl; + std::cout << printPlanWithStats( + *fragments[i].fragment.planNode, + stats[i], + FLAGS_include_custom_stats, + [&](auto id) { return planCostString(id, estimates); }); + std::cout << std::endl; + } + } + history_->recordVeloxExecution(nullptr, fragments, stats); + std::cout << numRows << " rows " << runStats.toString(false) << std::endl; + } catch (const std::exception& e) { + std::cerr << "Query terminated with: " << e.what() << std::endl; + if (errorString) { + *errorString = fmt::format("Runtime error: {}", e.what()); + } + waitForCompletion(runner); + return nullptr; + } + waitForCompletion(runner); + return runner; + } + + void waitForCompletion(const std::shared_ptr& runner) { + if (runner) { + try { + runner->waitForCompletion(50000); + } catch (const std::exception& /*ignore*/) { + } + } + } + + /// Returns exit status for run. 0 is passed, 1 is plan differences only, 2 is + /// result differences. + int32_t checkStatus() { + std::cerr << numPassed_ << " passed " << numFailed_ << " failed " + << numPlanMismatch_ << " plan mismatch " << numResultMismatch_ + << " result mismatch" << std::endl; + if (!numFailed_) { + return 0; + } + return numResultMismatch_ ? 2 : 1; + } + + private: + template + static void write(const T& value, std::ostream& out) { + out.write((char*)&value, sizeof(T)); + } + + template + static T read(std::istream& in) { + T value; + in.read((char*)&value, sizeof(T)); + return value; + } + + static std::string readString(std::istream& in) { + auto len = read(in); + std::string result; + result.resize(len); + in.read(result.data(), result.size()); + return result; + } + + static void writeString(const std::string& string, std::ostream& out) { + write(string.size(), out); + out.write(string.data(), string.size()); + } + + std::vector readVectors(std::istream& in) { + auto size = read(in); + std::vector result(size); + for (auto i = 0; i < size; ++i) { + result[i] = std::dynamic_pointer_cast( + restoreVector(in, checkPool_.get())); + } + return result; + } + + void writeVectors(std::vector& vectors, std::ostream& out) { + write(vectors.size(), out); + for (auto& vector : vectors) { + saveVector(*vector, out); + } + } + + int32_t printResults(const std::vector& results) { + std::cout << "Results:" << std::endl; + bool printType = true; + int32_t numRows = 0; + for (auto vectorIndex = 0; vectorIndex < results.size(); ++vectorIndex) { + const auto& vector = results[vectorIndex]; + // Print RowType only once. + if (printType) { + std::cout << vector->type()->asRow().toString() << std::endl; + printType = false; + } + for (vector_size_t i = 0; i < vector->size(); ++i) { + std::cout << vector->toString(i) << std::endl; + if (++numRows >= FLAGS_max_rows) { + int32_t numLeft = (vector->size() - (i - 1)); + ++vectorIndex; + for (; vectorIndex < results.size(); ++vectorIndex) { + numLeft += results[vectorIndex]->size(); + } + if (numLeft) { + std::cout << fmt::format("[Omitted {} more rows.", numLeft) + << std::endl; + } + return numRows + numLeft; + } + } + } + return numRows; + } + + std::shared_ptr allocator_; + std::shared_ptr cache_; + std::shared_ptr rootPool_; + std::shared_ptr optimizerPool_; + std::shared_ptr schemaPool_; + std::shared_ptr schemaRootPool_; + std::shared_ptr checkPool_; + std::unique_ptr ioExecutor_; + std::unique_ptr cacheExecutor_; + std::shared_ptr executor_; + std::shared_ptr spillExecutor_; + std::shared_ptr schemaQueryCtx_; + std::shared_ptr connectorQueryCtx_; + std::shared_ptr connector_; + std::shared_ptr schema_; + std::unique_ptr history_; + std::unique_ptr planner_; + std::unordered_map config_; + std::unordered_map hiveConfig_; + std::ofstream* record_{nullptr}; + std::ifstream* check_{nullptr}; + int32_t numPassed_{0}; + int32_t numFailed_{0}; + int32_t numPlanMismatch_{0}; + int32_t numResultMismatch_{0}; + int32_t queryCounter_{0}; +}; + +std::string readCommand(std::istream& in, bool& end) { + std::string line; + std::stringstream command; + end = false; + while (std::getline(in, line)) { + if (!line.empty() && line.back() == ';') { + command << line.substr(0, line.size() - 1); + return command.str(); + } + command << line << std::endl; + } + end = true; + return ""; +} + +void readCommands( + VeloxRunner& runner, + const std::string& prompt, + std::istream& in) { + for (;;) { + std::cout << prompt; + bool end; + std::string command = readCommand(in, end); + if (end) { + break; + } + if (command.empty()) { + continue; + } + auto cstr = command.c_str(); + char* flag = nullptr; + char* value = nullptr; + if (sscanf(cstr, "flag %ms = %ms", &flag, &value) == 2) { + std::cout << gflags::SetCommandLineOption(flag, value); + free(flag); + free(value); + continue; + } + runner.run(command); + } +} + +void recordQueries(VeloxRunner& runner) { + std::ifstream in(FLAGS_record); + std::ofstream ref; + ref.open(FLAGS_record + ".ref", std::ios_base::out | std::ios_base::trunc); + runner.setRecordStream(&ref); + readCommands(runner, "", in); +} + +void checkQueries(VeloxRunner& runner) { + std::ifstream in(FLAGS_check); + std::ifstream ref(FLAGS_check + ".ref"); + runner.setCheckStream(&ref); + readCommands(runner, "", in); + exit(runner.checkStatus()); +} + +std::string sevenBit(std::string& in) { + for (auto i = 0; i < in.size(); ++i) { + if ((uint8_t)in[i] > 127) { + in[i] = ' '; + } + } + return in; +} + +int main(int argc, char** argv) { + std::string kUsage( + "Velox local SQL command line. Run 'velox_sql --help' for available options.\n"); + gflags::SetUsageMessage(kUsage); + folly::Init init(&argc, &argv, false); + VeloxRunner runner; + try { + runner.initialize(); + if (!FLAGS_query.empty()) { + runner.run(FLAGS_query); + } else if (!FLAGS_record.empty()) { + recordQueries(runner); + } else if (!FLAGS_check.empty()) { + checkQueries(runner); + } else { + std::cout + << "Velox SQL. Type statement and end with ;. flag name = value; sets a gflag." + << std::endl; + readCommands(runner, "SQL> ", std::cin); + } + } catch (std::exception& e) { + std::cerr << "Error: " << e.what() << std::endl; + exit(-1); + } + return 0; +}