diff --git a/src/common/algorithm/ReservoirSampling.h b/src/common/algorithm/ReservoirSampling.h index 9540b61b275..7ad90edcbad 100644 --- a/src/common/algorithm/ReservoirSampling.h +++ b/src/common/algorithm/ReservoirSampling.h @@ -19,6 +19,14 @@ class ReservoirSampling final { samples_.reserve(num); } + explicit ReservoirSampling(uint64_t num, uint64_t count) { + num_ = num; + samples_.reserve(num); + for (uint64_t i = 0; i < count; ++i) { + sampling(i); + } + } + bool sampling(T&& sample) { if (cnt_ < num_) { samples_.emplace_back(std::move(sample)); diff --git a/src/graph/context/ast/QueryAstContext.h b/src/graph/context/ast/QueryAstContext.h index 9d1dacfd08f..5e8d60fdcda 100644 --- a/src/graph/context/ast/QueryAstContext.h +++ b/src/graph/context/ast/QueryAstContext.h @@ -93,17 +93,15 @@ struct GoContext final : AstContext { bool joinInput{false}; // true when $$.tag.prop exist bool joinDst{false}; - // Optimize for some simple go sentence which only need dst id. - bool isSimple{false}; - // The column name used by plan node`GetDstBySrc` - std::string dstIdColName{kDst}; ExpressionProps exprProps; // save dst prop YieldColumns* dstPropsExpr; - // save src and edge prop - YieldColumns* srcEdgePropsExpr; + // save src prop + YieldColumns* srcPropsExpr; + // save edge prop + YieldColumns* edgePropsExpr; // for track vid in Nsteps std::string srcVidColName; std::string dstVidColName; diff --git a/src/graph/context/iterator/GetNbrsRespDataSetIter.cpp b/src/graph/context/iterator/GetNbrsRespDataSetIter.cpp index 369f4e5acb9..0759736df7b 100644 --- a/src/graph/context/iterator/GetNbrsRespDataSetIter.cpp +++ b/src/graph/context/iterator/GetNbrsRespDataSetIter.cpp @@ -69,6 +69,12 @@ void GetNbrsRespDataSetIter::buildPropIndex(const std::string& colName, size_t c } } +Value GetNbrsRespDataSetIter::getVid() const { + DCHECK(valid()); + const Row& curRow = dataset_->rows[curRowIdx_]; + return curRow[0]; +} + Value GetNbrsRespDataSetIter::getVertex() const { // Always check the valid() before getVertex DCHECK(valid()); @@ -150,5 +156,45 @@ std::vector GetNbrsRespDataSetIter::getAdjEdges(VidHashSet* dstSet) const return adjEdges; } +std::unordered_set GetNbrsRespDataSetIter::getAdjDsts() const { + DCHECK(valid()); + + std::unordered_set adjDsts; + const Row& curRow = dataset_->rows[curRowIdx_]; + for (const auto& [edgeName, propIdx] : edgePropsMap_) { + DCHECK_LT(propIdx.colIdx, curRow.size()); + const Value& edgeColumn = curRow[propIdx.colIdx]; + if (edgeColumn.isList()) { + for (const Value& edgeVal : edgeColumn.getList().values) { + // skip the edge direction symbol: `-/+` + auto name = edgeName.substr(1); + if (!edgeVal.isList() || edgeVal.getList().empty()) { + continue; + } + const List& propList = edgeVal.getList(); + DCHECK_LT(propIdx.edgeDstIdx, propList.size()); + auto& dst = propList[propIdx.edgeDstIdx]; + adjDsts.emplace(dst); + } + } + } + return adjDsts; +} + +size_t GetNbrsRespDataSetIter::size() { + size_t size = 0; + for (; valid(); next()) { + const Row& curRow = dataset_->rows[curRowIdx_]; + for (const auto& [edgeName, propIdx] : edgePropsMap_) { + DCHECK_LT(propIdx.colIdx, curRow.size()); + const Value& edgeColumn = curRow[propIdx.colIdx]; + if (edgeColumn.isList()) { + size += edgeColumn.getList().values.size(); + } + } + } + return size; +} + } // namespace graph } // namespace nebula diff --git a/src/graph/context/iterator/GetNbrsRespDataSetIter.h b/src/graph/context/iterator/GetNbrsRespDataSetIter.h index c44188e0211..4a87b91552a 100644 --- a/src/graph/context/iterator/GetNbrsRespDataSetIter.h +++ b/src/graph/context/iterator/GetNbrsRespDataSetIter.h @@ -27,6 +27,12 @@ class GetNbrsRespDataSetIter final { Value getVertex() const; std::vector getAdjEdges(VidHashSet* dstSet) const; + std::unordered_set getAdjDsts() const; + + Value getVid() const; + + size_t size(); + private: struct PropIndex { size_t colIdx; diff --git a/src/graph/executor/CMakeLists.txt b/src/graph/executor/CMakeLists.txt index 67cb40bf4d7..9177df1b0d2 100644 --- a/src/graph/executor/CMakeLists.txt +++ b/src/graph/executor/CMakeLists.txt @@ -39,10 +39,11 @@ nebula_add_library( query/ScanVerticesExecutor.cpp query/ScanEdgesExecutor.cpp query/TraverseExecutor.cpp + query/ExpandExecutor.cpp + query/ExpandAllExecutor.cpp query/AppendVerticesExecutor.cpp query/RollUpApplyExecutor.cpp query/PatternApplyExecutor.cpp - query/GetDstBySrcExecutor.cpp algo/BFSShortestPathExecutor.cpp algo/MultiShortestPathExecutor.cpp algo/ProduceAllPathsExecutor.cpp diff --git a/src/graph/executor/Executor.cpp b/src/graph/executor/Executor.cpp index 1e68050a3b5..a8a31b9a8f5 100644 --- a/src/graph/executor/Executor.cpp +++ b/src/graph/executor/Executor.cpp @@ -69,9 +69,10 @@ #include "graph/executor/query/AssignExecutor.h" #include "graph/executor/query/DataCollectExecutor.h" #include "graph/executor/query/DedupExecutor.h" +#include "graph/executor/query/ExpandAllExecutor.h" +#include "graph/executor/query/ExpandExecutor.h" #include "graph/executor/query/FilterExecutor.h" #include "graph/executor/query/FulltextIndexScanExecutor.h" -#include "graph/executor/query/GetDstBySrcExecutor.h" #include "graph/executor/query/GetEdgesExecutor.h" #include "graph/executor/query/GetNeighborsExecutor.h" #include "graph/executor/query/GetVerticesExecutor.h" @@ -197,6 +198,12 @@ Executor *Executor::makeExecutor(QueryContext *qctx, const PlanNode *node) { case PlanNode::Kind::kGetNeighbors: { return pool->makeAndAdd(node, qctx); } + case PlanNode::Kind::kExpand: { + return pool->makeAndAdd(node, qctx); + } + case PlanNode::Kind::kExpandAll: { + return pool->makeAndAdd(node, qctx); + } case PlanNode::Kind::kFulltextIndexScan: { return pool->makeAndAdd(node, qctx); } @@ -559,9 +566,6 @@ Executor *Executor::makeExecutor(QueryContext *qctx, const PlanNode *node) { case PlanNode::Kind::kShortestPath: { return pool->makeAndAdd(node, qctx); } - case PlanNode::Kind::kGetDstBySrc: { - return pool->makeAndAdd(node, qctx); - } case PlanNode::Kind::kUnknown: { DLOG(FATAL) << "Unknown plan node kind " << static_cast(node->kind()); break; diff --git a/src/graph/executor/logic/ArgumentExecutor.cpp b/src/graph/executor/logic/ArgumentExecutor.cpp index ed2ab753602..36d1eeff39f 100644 --- a/src/graph/executor/logic/ArgumentExecutor.cpp +++ b/src/graph/executor/logic/ArgumentExecutor.cpp @@ -20,8 +20,8 @@ folly::Future ArgumentExecutor::execute() { DCHECK(iter != nullptr); const auto &successor = successors(); - auto sucessorExecutor = *successor.begin(); - bool flag = sucessorExecutor->node()->kind() != PlanNode::Kind::kGetVertices; + auto kind = (*successor.begin())->node()->kind(); + bool flag = (kind != PlanNode::Kind::kGetVertices && kind != PlanNode::Kind::kExpand); DataSet ds; ds.colNames = argNode->colNames(); diff --git a/src/graph/executor/query/DataCollectExecutor.cpp b/src/graph/executor/query/DataCollectExecutor.cpp index 0b51e36ee03..5441ec0517e 100644 --- a/src/graph/executor/query/DataCollectExecutor.cpp +++ b/src/graph/executor/query/DataCollectExecutor.cpp @@ -32,10 +32,6 @@ folly::Future DataCollectExecutor::doCollect() { NG_RETURN_IF_ERROR(rowBasedMove(vars)); break; } - case DataCollect::DCKind::kMToN: { - NG_RETURN_IF_ERROR(collectMToN(vars, dc->step(), dc->distinct())); - break; - } case DataCollect::DCKind::kBFSShortest: { NG_RETURN_IF_ERROR(collectBFSShortest(vars)); break; @@ -124,53 +120,6 @@ Status DataCollectExecutor::rowBasedMove(const std::vector& vars) { return Status::OK(); } -Status DataCollectExecutor::collectMToN(const std::vector& vars, - const StepClause& mToN, - bool distinct) { - DataSet ds; - ds.colNames = std::move(colNames_); - DCHECK(!ds.colNames.empty()); - robin_hood::unordered_flat_set> unique; - // itersHolder keep life cycle of iters util this method return. - std::vector> itersHolder; - for (auto& var : vars) { - auto& hist = ectx_->getHistory(var); - std::size_t histSize = hist.size(); - DCHECK_GE(mToN.mSteps(), 1); - std::size_t n = mToN.nSteps() > histSize ? histSize : mToN.nSteps(); - for (auto i = mToN.mSteps() - 1; i < n; ++i) { - auto iter = hist[i].iter(); - if (iter->isSequentialIter()) { - auto* seqIter = static_cast(iter.get()); - unique.reserve(seqIter->size()); - while (seqIter->valid()) { - if (distinct && !unique.emplace(seqIter->row()).second) { - seqIter->unstableErase(); - } else { - seqIter->next(); - } - } - } else { - std::stringstream msg; - msg << "Iterator should be kind of SequentialIter, but was: " << iter->kind(); - return Status::Error(msg.str()); - } - itersHolder.emplace_back(std::move(iter)); - } - } - - for (auto& iter : itersHolder) { - if (iter->isSequentialIter()) { - auto* seqIter = static_cast(iter.get()); - for (seqIter->reset(); seqIter->valid(); seqIter->next()) { - ds.rows.emplace_back(seqIter->moveRow()); - } - } - } - result_.setDataSet(std::move(ds)); - return Status::OK(); -} - Status DataCollectExecutor::collectBFSShortest(const std::vector& vars) { // Will rewrite this method once we implement returning the props for the // path. diff --git a/src/graph/executor/query/DataCollectExecutor.h b/src/graph/executor/query/DataCollectExecutor.h index 386862511c4..113dd3a3930 100644 --- a/src/graph/executor/query/DataCollectExecutor.h +++ b/src/graph/executor/query/DataCollectExecutor.h @@ -17,9 +17,6 @@ // // `rowBaseMove` : Collect the latest version of the results // -// `collectMToN` : Only used in GO MToN scenarios. Collect the results generated by MTON -// steps (multi-version) -// // `collectAllPaths` : Only used in FindPath scenarios. Collect the paths generated by the FindPath // operator (multi-version) // @@ -42,8 +39,6 @@ class DataCollectExecutor final : public Executor { Status rowBasedMove(const std::vector& vars); - Status collectMToN(const std::vector& vars, const StepClause& mToN, bool distinct); - Status collectBFSShortest(const std::vector& vars); Status collectAllPaths(const std::vector& vars); diff --git a/src/graph/executor/query/ExpandAllExecutor.cpp b/src/graph/executor/query/ExpandAllExecutor.cpp new file mode 100644 index 00000000000..dd85469bbb4 --- /dev/null +++ b/src/graph/executor/query/ExpandAllExecutor.cpp @@ -0,0 +1,443 @@ +// Copyright (c) 2022 vesoft inc. All rights reserved. +// +// This source code is licensed under Apache 2.0 License. + +#include "graph/executor/query/ExpandAllExecutor.h" + +#include "common/algorithm/ReservoirSampling.h" +#include "graph/service/GraphFlags.h" +#include "graph/util/SchemaUtil.h" +#include "graph/util/Utils.h" + +using nebula::storage::StorageClient; +using nebula::storage::StorageRpcResponse; +using nebula::storage::cpp2::GetNeighborsResponse; + +namespace nebula { +namespace graph { + +Status ExpandAllExecutor::buildRequestVids() { + SCOPED_TIMER(&execTime_); + const auto& inputVar = expand_->inputVar(); + auto inputIter = ectx_->getResult(inputVar).iterRef(); + auto iter = static_cast(inputIter); + size_t iterSize = iter->size(); + nextStepVids_.reserve(iterSize); + if (joinInput_) { + for (; iter->valid(); iter->next()) { + const auto& src = iter->getColumn(0); + const auto& dst = iter->getColumn(1); + nextStepVids_.emplace(dst); + // update preDst2vidMap_ + auto findDst = preDst2VidsMap_.find(dst); + if (findDst == preDst2VidsMap_.end()) { + std::unordered_set tmp({src}); + preDst2VidsMap_.emplace(dst, std::move(tmp)); + } else { + findDst->second.emplace(src); + } + } + } else { + for (; iter->valid(); iter->next()) { + const auto& dst = iter->getColumn(-1); + nextStepVids_.emplace(dst); + } + } + return Status::OK(); +} + +folly::Future ExpandAllExecutor::execute() { + currentStep_ = expand_->minSteps(); + maxSteps_ = expand_->maxSteps(); + vertexColumns_ = expand_->vertexColumns(); + edgeColumns_ = expand_->edgeColumns(); + sample_ = expand_->sample(); + stepLimits_ = expand_->stepLimits(); + joinInput_ = expand_->joinInput(); + result_.colNames = expand_->colNames(); + + NG_RETURN_IF_ERROR(buildRequestVids()); + if (nextStepVids_.empty()) { + return finish(ResultBuilder().value(Value(std::move(result_))).build()); + } + return getNeighbors(); +} + +folly::Future ExpandAllExecutor::getNeighbors() { + currentStep_++; + StorageClient* storageClient = qctx_->getStorageClient(); + StorageClient::CommonRequestParam param(expand_->space(), + qctx_->rctx()->session()->id(), + qctx_->plan()->id(), + qctx_->plan()->isProfileEnabled()); + std::vector vids(nextStepVids_.size()); + std::move(nextStepVids_.begin(), nextStepVids_.end(), vids.begin()); + QueryExpressionContext qec(qctx()->ectx()); + return storageClient + ->getNeighbors(param, + {nebula::kVid}, + std::move(vids), + {}, + storage::cpp2::EdgeDirection::OUT_EDGE, + nullptr, + expand_->vertexProps(), + expand_->edgeProps(), + nullptr, + false, + false, + std::vector(), + expand_->limit(qec), + expand_->filter(), + nullptr) + .via(runner()) + .thenValue([this](RpcResponse&& resp) mutable { + // MemoryTrackerVerified + memory::MemoryCheckGuard guard; + nextStepVids_.clear(); + SCOPED_TIMER(&execTime_); + addStats(resp); + time::Duration expandTime; + curLimit_ = 0; + curMaxLimit_ = stepLimits_.empty() ? std::numeric_limits::max() + : stepLimits_[currentStep_ - 2]; + return handleResponse(std::move(resp)).ensure([this, expandTime]() { + std::string timeName = "graphExpandAllTime+" + folly::to(currentStep_); + otherStats_.emplace(timeName, folly::sformat("{}(us)", expandTime.elapsedInUSec())); + }); + }) + .thenValue([this](Status s) -> folly::Future { + NG_RETURN_IF_ERROR(s); + if (qctx()->isKilled()) { + return Status::Error("Execution had been killed"); + } + if (currentStep_ <= maxSteps_) { + if (!nextStepVids_.empty()) { + return getNeighbors(); + } + if (!preVisitedVids_.empty()) { + return expandFromCache(); + } + } + return finish(ResultBuilder().value(Value(std::move(result_))).build()); + }); +} + +folly::Future ExpandAllExecutor::expandFromCache() { + for (; currentStep_ <= maxSteps_; ++currentStep_) { + time::Duration expandTime; + if (qctx()->isKilled()) { + return Status::Error("Execution had been killed"); + } + curLimit_ = 0; + curMaxLimit_ = + stepLimits_.empty() ? std::numeric_limits::max() : stepLimits_[currentStep_ - 2]; + + std::vector samples; + if (sample_) { + int64_t size = 0; + for (auto& vid : preVisitedVids_) { + size += adjList_[vid].size(); + } + algorithm::ReservoirSampling sampler(curMaxLimit_, size); + samples = sampler.samples(); + std::sort(samples.begin(), samples.end(), [](int64_t a, int64_t b) { return a > b; }); + } + + std::unordered_map> dst2VidsMap; + std::unordered_set visitedVids; + getNeighborsFromCache(dst2VidsMap, visitedVids, samples); + preVisitedVids_.swap(visitedVids); + preDst2VidsMap_.swap(dst2VidsMap); + std::string timeName = "graphCacheExpandAllTime+" + folly::to(currentStep_); + otherStats_.emplace(timeName, folly::sformat("{}(us)", expandTime.elapsedInUSec())); + if (!nextStepVids_.empty()) { + return getNeighbors(); + } + } + return finish(ResultBuilder().value(Value(std::move(result_))).build()); +} + +void ExpandAllExecutor::getNeighborsFromCache( + std::unordered_map>& dst2VidsMap, + std::unordered_set& visitedVids, + std::vector& samples) { + for (const auto& vid : preVisitedVids_) { + auto findVid = adjList_.find(vid); + if (findVid == adjList_.end()) { + continue; + } + auto& adjEdgeProps = findVid->second; + auto& vertexProps = adjEdgeProps.back(); + for (auto edgeIter = adjEdgeProps.begin(); edgeIter != adjEdgeProps.end() - 1; ++edgeIter) { + if (sample_) { + if (samples.empty()) { + break; + } + if (curLimit_++ != samples.back()) { + continue; + } else { + samples.pop_back(); + } + } else { + if (curLimit_++ >= curMaxLimit_) { + break; + } + } + auto& dst = (*edgeIter).values.back(); + if (adjList_.find(dst) == adjList_.end()) { + nextStepVids_.emplace(dst); + } else { + visitedVids.emplace(dst); + } + if (joinInput_) { + auto findInitVid = preDst2VidsMap_.find(vid); + buildResult(findInitVid->second, vertexProps, *edgeIter); + updateDst2VidsMap(dst2VidsMap, vid, dst); + } else { + buildResult(vertexProps, *edgeIter); + } + } + } + if (currentStep_ <= maxSteps_) { + resetNextStepVids(visitedVids); + } +} + +Status ExpandAllExecutor::handleLastStep(GetNeighborsIter* iter, std::vector& samples) { + QueryExpressionContext ctx(ectx_); + List curVertexProps; + Value curVid; + std::unordered_map> dst2VidsMap; + std::unordered_set visitedVids; + if (iter->valid() && vertexColumns_) { + for (auto& col : vertexColumns_->columns()) { + Value val = col->expr()->eval(ctx(iter)); + curVertexProps.values.emplace_back(std::move(val)); + } + } + for (; iter->valid(); iter->next()) { + List edgeProps; + if (edgeColumns_) { + for (auto& col : edgeColumns_->columns()) { + Value val = col->expr()->eval(ctx(iter)); + edgeProps.values.emplace_back(std::move(val)); + } + } + const auto& vid = iter->getColumn(0); + curVid = curVid.empty() ? vid : curVid; + if (curVid != vid) { + curVid = vid; + curVertexProps.values.clear(); + if (vertexColumns_) { + for (auto& col : vertexColumns_->columns()) { + Value val = col->expr()->eval(ctx(iter)); + curVertexProps.values.emplace_back(std::move(val)); + } + } + } + + if (limitORsample(samples)) { + continue; + } + + if (joinInput_) { + auto findVid = preDst2VidsMap_.find(vid); + buildResult(findVid->second, curVertexProps, edgeProps, true); + } else { + buildResult(curVertexProps, edgeProps, true); + } + } + if (!preVisitedVids_.empty()) { + getNeighborsFromCache(dst2VidsMap, visitedVids, samples); + } + return Status::OK(); +} + +folly::Future ExpandAllExecutor::handleResponse(RpcResponse&& resps) { + NG_RETURN_IF_ERROR(handleCompleteness(resps, FLAGS_accept_partial_success)); + List list; + for (auto& resp : resps.responses()) { + auto dataset = resp.get_vertices(); + if (dataset) { + list.values.emplace_back(std::move(*dataset)); + } + } + auto listVal = std::make_shared(std::move(list)); + auto iter = std::make_unique(listVal); + if (iter->numRows() == 0) { + return Status::OK(); + } + + std::vector samples; + if (sample_) { + auto size = iter->size(); + algorithm::ReservoirSampling sampler(curMaxLimit_, size); + samples = sampler.samples(); + std::sort(samples.begin(), samples.end(), [](int64_t a, int64_t b) { return a > b; }); + } + if (currentStep_ > maxSteps_) { + return handleLastStep(iter.get(), samples); + } + + QueryExpressionContext ctx(ectx_); + std::unordered_map> dst2VidsMap; + std::unordered_set visitedVids; + std::vector adjEdgeProps; + List curVertexProps; + Value curVid; + if (iter->valid() && vertexColumns_) { + for (auto& col : vertexColumns_->columns()) { + Value val = col->expr()->eval(ctx(iter.get())); + curVertexProps.values.emplace_back(std::move(val)); + } + } + for (; iter->valid(); iter->next()) { + List edgeProps; + if (edgeColumns_) { + for (auto& col : edgeColumns_->columns()) { + Value val = col->expr()->eval(ctx(iter.get())); + edgeProps.values.emplace_back(std::move(val)); + } + } + // get the start vid of the next step through dst when expanding in the cache + auto dst = iter->getEdgeProp("*", nebula::kDst); + if (dst.isNull()) { + // (TEST) + continue; + } + edgeProps.values.emplace_back(dst); + const auto& vid = iter->getColumn(0); + curVid = curVid.empty() ? vid : curVid; + if (curVid != vid) { + adjEdgeProps.emplace_back(std::move(curVertexProps)); + adjList_.emplace(curVid, std::move(adjEdgeProps)); + curVid = vid; + if (vertexColumns_) { + for (auto& col : vertexColumns_->columns()) { + Value val = col->expr()->eval(ctx(iter.get())); + curVertexProps.values.emplace_back(std::move(val)); + } + } + } + adjEdgeProps.emplace_back(edgeProps); + + if (limitORsample(samples)) { + continue; + } + + if (joinInput_) { + auto findVid = preDst2VidsMap_.find(vid); + buildResult(findVid->second, curVertexProps, edgeProps); + updateDst2VidsMap(dst2VidsMap, vid, dst); + } else { + buildResult(curVertexProps, edgeProps); + } + + if (adjList_.find(dst) == adjList_.end()) { + nextStepVids_.emplace(dst); + } else { + visitedVids.emplace(dst); + } + } + if (!curVid.empty()) { + adjEdgeProps.emplace_back(std::move(curVertexProps)); + adjList_.emplace(curVid, std::move(adjEdgeProps)); + } + + resetNextStepVids(visitedVids); + + if (!preVisitedVids_.empty()) { + getNeighborsFromCache(dst2VidsMap, visitedVids, samples); + } + visitedVids.swap(preVisitedVids_); + dst2VidsMap.swap(preDst2VidsMap_); + return Status::OK(); +} + +void ExpandAllExecutor::buildResult(const std::unordered_set& vids, + const List& vList, + const List& eList, + bool isLastStep) { + std::vector list = vList.values; + if (!eList.values.empty()) { + list.reserve(list.size() + eList.values.size()); + if (isLastStep) { + list.insert(list.end(), eList.values.begin(), eList.values.end()); + } else { + list.insert(list.end(), eList.values.begin(), eList.values.end() - 1); + } + } + for (auto& vid : vids) { + Row row; + row.values.emplace_back(vid); + if (!list.empty()) { + row.values.insert(row.values.end(), list.begin(), list.end()); + } + result_.rows.emplace_back(std::move(row)); + } +} + +void ExpandAllExecutor::buildResult(const List& vList, const List& eList, bool isLastStep) { + if (vList.values.empty() && eList.values.empty()) { + return; + } + Row row = vList; + if (isLastStep) { + row.values.insert(row.values.end(), eList.values.begin(), eList.values.end()); + } else { + row.values.insert(row.values.end(), eList.values.begin(), eList.values.end() - 1); + } + result_.rows.emplace_back(std::move(row)); +} + +void ExpandAllExecutor::resetNextStepVids(std::unordered_set& visitedVids) { + auto vidIter = nextStepVids_.begin(); + while (vidIter != nextStepVids_.end()) { + if (adjList_.find(*vidIter) != adjList_.end()) { + visitedVids.emplace(*vidIter); + vidIter = nextStepVids_.erase(vidIter); + } else { + vidIter++; + } + } +} + +void ExpandAllExecutor::updateDst2VidsMap( + std::unordered_map>& dst2VidsMap, + const Value& src, + const Value& dst) { + auto findSrc = preDst2VidsMap_.find(src); + if (findSrc == preDst2VidsMap_.end()) { + auto findDst = dst2VidsMap.find(dst); + if (findDst == dst2VidsMap.end()) { + std::unordered_set tmp({src}); + dst2VidsMap.emplace(dst, std::move(tmp)); + } else { + findDst->second.emplace(src); + } + } else { + auto findDst = dst2VidsMap.find(dst); + if (findDst == dst2VidsMap.end()) { + dst2VidsMap.emplace(dst, findSrc->second); + } else { + findDst->second.insert(findSrc->second.begin(), findSrc->second.end()); + } + } +} + +bool ExpandAllExecutor::limitORsample(std::vector& samples) { + if (sample_) { + if (samples.empty() || curLimit_++ != samples.back()) { + return true; + } + samples.pop_back(); + return false; + } + if (curLimit_++ >= curMaxLimit_) { + return true; + } + return false; +} + +} // namespace graph +} // namespace nebula diff --git a/src/graph/executor/query/ExpandAllExecutor.h b/src/graph/executor/query/ExpandAllExecutor.h new file mode 100644 index 00000000000..6f5388b6eac --- /dev/null +++ b/src/graph/executor/query/ExpandAllExecutor.h @@ -0,0 +1,114 @@ +// Copyright (c) 2020 vesoft inc. All rights reserved. +// +// This source code is licensed under Apache 2.0 License. + +#ifndef GRAPH_EXECUTOR_QUERY_EXPAND_ALL_H_ +#define GRAPH_EXECUTOR_QUERY_EXPAND_ALL_H_ + +#include "graph/executor/StorageAccessExecutor.h" +#include "graph/planner/plan/Query.h" + +// The go statement is divided into two operators, expand operator and expandAll operator. +// expandAll is responsible for expansion and take attributes that user need. + +// The semantics of GO N STEPS FROM 'xxx' OVER edge WHERE condition YIELD yyy is +// First : expand n-1 steps from 'xxx' to get the result set vids(deduplication) +// Second: expand the last step from vids. get the attributes required by the users during expansion +// then execute filter condition. finally return the results to the next operator + +// The semantics of GO M TO N STEPS is +// GO M STEPS +// UNION ALL +// GO M+1 STEPS +// ... +// UNION ALL +// GO N STEPS + +// therefore. each step in expandAll operator. we need adds the result to the global dataset result_ +// and returns the result_ after the expansion is completed. + +// if need to join with the previous statement, we need save the mapping relationship +// between the init vid and the destination vid during the expansion +// finally add a column (column name `expand_vid`, store the init vids) to the result_ +// for join previous statement + +// if expression contains $$.tag.propName、 $$ +// we need add a column(colume name `_expandall_dst`, store the destination vid) +// for join getVertices's dataset + +// adjList is an adjacency list structure +// which saves the vids and the attributes of edge and src vertex that user need when expasion +// when expanding, if the vid has already been visited, do not need to go through RPC +// just get the result directly through adjList_ + +// when minSteps == maxSteps, only need to expand one step +// then filter and limit information can be pushed down to storage + +namespace nebula { +namespace graph { +class ExpandAllExecutor final : public StorageAccessExecutor { + public: + ExpandAllExecutor(const PlanNode* node, QueryContext* qctx) + : StorageAccessExecutor("ExpandAllExecutor", node, qctx) { + expand_ = asNode(node); + } + + Status buildRequestVids(); + + folly::Future execute() override; + + folly::Future getNeighbors(); + + void getNeighborsFromCache(std::unordered_map>& dst2VidsMap, + std::unordered_set& visitedVids, + std::vector& samples); + + folly::Future expandFromCache(); + + void updateDst2VidsMap(std::unordered_map>& dst2VidMap, + const Value& src, + const Value& dst); + + void buildResult(const List& vList, const List& eList, bool isLastStep = false); + + void buildResult(const std::unordered_set& vids, + const List& vList, + const List& eList, + bool isLastStep = false); + + Status handleLastStep(GetNeighborsIter* iter, std::vector& samples); + + using RpcResponse = storage::StorageRpcResponse; + folly::Future handleResponse(RpcResponse&& resps); + + void resetNextStepVids(std::unordered_set& visitedVids); + + bool limitORsample(std::vector& samples); + + private: + const ExpandAll* expand_; + bool joinInput_{false}; + size_t currentStep_{0}; + size_t maxSteps_{0}; + DataSet result_; + YieldColumns* edgeColumns_{nullptr}; + YieldColumns* vertexColumns_{nullptr}; + + bool sample_{false}; + int64_t curLimit_{0}; + int64_t curMaxLimit_{std::numeric_limits::max()}; + std::vector stepLimits_; + + std::unordered_set nextStepVids_; + std::unordered_set preVisitedVids_; + std::unordered_map> adjList_; + // keep the mapping relationship between the init vid and the destination vid + // during the expansion. KEY : edge's dst, VALUE : init vids + // then we can know which init vids can reach the current destination point + std::unordered_map> preDst2VidsMap_; +}; + +} // namespace graph +} // namespace nebula + +#endif // GRAPH_EXECUTOR_QUERY_ExpandAllExecutor_H_ diff --git a/src/graph/executor/query/ExpandExecutor.cpp b/src/graph/executor/query/ExpandExecutor.cpp new file mode 100644 index 00000000000..9cc1be79dc4 --- /dev/null +++ b/src/graph/executor/query/ExpandExecutor.cpp @@ -0,0 +1,381 @@ +// Copyright (c) 2022 vesoft inc. All rights reserved. +// +// This source code is licensed under Apache 2.0 License. + +#include "graph/executor/query/ExpandExecutor.h" + +#include "common/algorithm/ReservoirSampling.h" +#include "graph/service/GraphFlags.h" +#include "graph/util/SchemaUtil.h" +#include "graph/util/Utils.h" + +using nebula::storage::StorageClient; +using nebula::storage::StorageRpcResponse; +using nebula::storage::cpp2::GetDstBySrcResponse; +using nebula::storage::cpp2::GetNeighborsResponse; + +namespace nebula { +namespace graph { + +Status ExpandExecutor::buildRequestVids() { + SCOPED_TIMER(&execTime_); + const auto& inputVar = expand_->inputVar(); + auto inputIter = ectx_->getResult(inputVar).iterRef(); + auto iter = static_cast(inputIter); + size_t iterSize = iter->size(); + nextStepVids_.reserve(iterSize); + QueryExpressionContext ctx(ectx_); + + const auto& spaceInfo = qctx()->rctx()->session()->space(); + const auto& metaVidType = *(spaceInfo.spaceDesc.vid_type_ref()); + auto vidType = SchemaUtil::propTypeToValueType(metaVidType.get_type()); + for (; iter->valid(); iter->next()) { + const auto& vid = iter->getColumn(0); + if (vid.type() == vidType) { + nextStepVids_.emplace(vid); + } + } + return Status::OK(); +} + +folly::Future ExpandExecutor::execute() { + maxSteps_ = expand_->maxSteps(); + sample_ = expand_->sample(); + stepLimits_ = expand_->stepLimits(); + NG_RETURN_IF_ERROR(buildRequestVids()); + if (nextStepVids_.empty()) { + DataSet emptyDs; + return finish(ResultBuilder().value(Value(std::move(emptyDs))).build()); + } + if (maxSteps_ == 0) { + DataSet ds; + ds.colNames = expand_->colNames(); + for (const auto& vid : nextStepVids_) { + Row row; + row.values.emplace_back(vid); + if (expand_->joinInput()) { + row.values.emplace_back(vid); + } + ds.rows.emplace_back(std::move(row)); + } + return finish(ResultBuilder().value(Value(std::move(ds))).build()); + } + if (expand_->joinInput() || !stepLimits_.empty()) { + return getNeighbors(); + } + return GetDstBySrc(); +} + +folly::Future ExpandExecutor::GetDstBySrc() { + currentStep_++; + time::Duration getDstTime; + StorageClient* storageClient = qctx_->getStorageClient(); + StorageClient::CommonRequestParam param(expand_->space(), + qctx_->rctx()->session()->id(), + qctx_->plan()->id(), + qctx_->plan()->isProfileEnabled()); + std::vector vids(nextStepVids_.size()); + std::move(nextStepVids_.begin(), nextStepVids_.end(), vids.begin()); + return storageClient->getDstBySrc(param, std::move(vids), expand_->edgeTypes()) + .via(runner()) + .ensure([this, getDstTime]() { + SCOPED_TIMER(&execTime_); + otherStats_.emplace("total_rpc_time", folly::sformat("{}(us)", getDstTime.elapsedInUSec())); + }) + .thenValue([this](StorageRpcResponse&& resps) { + memory::MemoryCheckGuard guard; + nextStepVids_.clear(); + SCOPED_TIMER(&execTime_); + auto& hostLatency = resps.hostLatency(); + for (size_t i = 0; i < hostLatency.size(); ++i) { + size_t size = 0u; + auto& result = resps.responses()[i]; + if (result.dsts_ref().has_value()) { + size = (*result.dsts_ref()).size(); + } + auto info = util::collectRespProfileData(result.result, hostLatency[i], size); + otherStats_.emplace(folly::sformat("resp[{}]", i), folly::toPrettyJson(info)); + } + auto result = handleCompleteness(resps, FLAGS_accept_partial_success); + if (!result.ok()) { + return folly::makeFuture(result.status()); + } + auto& responses = resps.responses(); + if (currentStep_ < maxSteps_) { + for (auto& resp : responses) { + auto* dataset = resp.get_dsts(); + if (dataset == nullptr) continue; + for (auto& row : dataset->rows) { + nextStepVids_.insert(row.values.begin(), row.values.end()); + } + } + if (nextStepVids_.empty()) { + DataSet emptyDs; + finish(ResultBuilder().value(Value(std::move(emptyDs))).build()); + return folly::makeFuture(Status::OK()); + } + return GetDstBySrc(); + } else { + ResultBuilder builder; + builder.state(result.value()); + DataSet ds; + for (auto& resp : responses) { + auto* dataset = resp.get_dsts(); + if (dataset == nullptr) continue; + dataset->colNames = expand_->colNames(); + ds.append(std::move(*dataset)); + } + builder.value(Value(std::move(ds))).iter(Iterator::Kind::kSequential); + finish(builder.build()); + return folly::makeFuture(Status::OK()); + } + }); +} + +folly::Future ExpandExecutor::getNeighbors() { + currentStep_++; + StorageClient* storageClient = qctx_->getStorageClient(); + StorageClient::CommonRequestParam param(expand_->space(), + qctx_->rctx()->session()->id(), + qctx_->plan()->id(), + qctx_->plan()->isProfileEnabled()); + std::vector vids(nextStepVids_.size()); + std::move(nextStepVids_.begin(), nextStepVids_.end(), vids.begin()); + return storageClient + ->getNeighbors(param, + {nebula::kVid}, + std::move(vids), + {}, + storage::cpp2::EdgeDirection::OUT_EDGE, + nullptr, + nullptr, + expand_->edgeProps(), + nullptr, + false, + false, + std::vector(), + -1, + nullptr, + nullptr) + .via(runner()) + .thenValue([this](RpcResponse&& resp) mutable { + // MemoryTrackerVerified + memory::MemoryCheckGuard guard; + nextStepVids_.clear(); + SCOPED_TIMER(&execTime_); + addStats(resp); + time::Duration expandTime; + curLimit_ = 0; + curMaxLimit_ = stepLimits_.empty() ? std::numeric_limits::max() + : stepLimits_[currentStep_ - 1]; + return handleResponse(std::move(resp)).ensure([this, expandTime]() { + std::string timeName = "graphExpandTime+" + folly::to(currentStep_); + otherStats_.emplace(timeName, folly::sformat("{}(us)", expandTime.elapsedInUSec())); + }); + }) + .thenValue([this](Status s) -> folly::Future { + NG_RETURN_IF_ERROR(s); + if (qctx()->isKilled()) { + return Status::Error("Execution had been killed"); + } + if (currentStep_ < maxSteps_) { + if (!nextStepVids_.empty()) { + return getNeighbors(); + } + if (!preVisitedVids_.empty()) { + return expandFromCache(); + } + } + return buildResult(); + }); +} + +folly::Future ExpandExecutor::expandFromCache() { + for (; currentStep_ < maxSteps_; ++currentStep_) { + time::Duration expandTime; + if (qctx()->isKilled()) { + return Status::Error("Execution had been killed"); + } + curLimit_ = 0; + curMaxLimit_ = + stepLimits_.empty() ? std::numeric_limits::max() : stepLimits_[currentStep_ - 1]; + std::unordered_map> dst2VidsMap; + std::unordered_set visitedVids; + + std::vector samples; + if (sample_) { + int64_t size = 0; + for (auto& vid : preVisitedVids_) { + size += adjDsts_[vid].size(); + } + algorithm::ReservoirSampling sampler(curMaxLimit_, size); + samples = sampler.samples(); + std::sort(samples.begin(), samples.end(), [](int64_t a, int64_t b) { return a > b; }); + } + + getNeighborsFromCache(dst2VidsMap, visitedVids, samples); + dst2VidsMap.swap(preDst2VidsMap_); + visitedVids.swap(preVisitedVids_); + std::string timeName = "graphCacheExpandTime+" + folly::to(currentStep_); + otherStats_.emplace(timeName, folly::sformat("{}(us)", expandTime.elapsedInUSec())); + if (!nextStepVids_.empty()) { + return getNeighbors(); + } + } + return buildResult(); +} + +void ExpandExecutor::getNeighborsFromCache( + std::unordered_map>& dst2VidsMap, + std::unordered_set& visitedVids, + std::vector& samples) { + for (const auto& vid : preVisitedVids_) { + auto findVid = adjDsts_.find(vid); + if (findVid == adjDsts_.end()) { + continue; + } + auto& dsts = findVid->second; + + for (const auto& dst : dsts) { + if (sample_) { + if (samples.empty()) { + break; + } + if (curLimit_++ != samples.back()) { + continue; + } else { + samples.pop_back(); + } + } else { + if (curLimit_++ >= curMaxLimit_) { + break; + } + } + updateDst2VidsMap(dst2VidsMap, vid, dst); + if (currentStep_ >= maxSteps_) { + continue; + } + if (adjDsts_.find(dst) == adjDsts_.end()) { + nextStepVids_.emplace(dst); + } else { + visitedVids.emplace(dst); + } + } + } +} + +// 1、 update adjDsts_, cache vid and the corresponding dsts +// 2、 get next step's vids +// 3、 handle the situation when limit OR sample exists +// 4、 cache already visited vids +// 5、 get the dsts corresponding to the vid that has been visited in the previous step by adjDsts_ +folly::Future ExpandExecutor::handleResponse(RpcResponse&& resps) { + NG_RETURN_IF_ERROR(handleCompleteness(resps, FLAGS_accept_partial_success)); + std::unordered_map> dst2VidsMap; + std::unordered_set visitedVids; + std::vector samples; + if (sample_) { + size_t size = 0; + for (auto& resp : resps.responses()) { + auto dataset = resp.get_vertices(); + if (!dataset) continue; + GetNbrsRespDataSetIter iter(dataset); + size += iter.size(); + } + algorithm::ReservoirSampling sampler(curMaxLimit_, size); + samples = sampler.samples(); + std::sort(samples.begin(), samples.end(), [](int64_t a, int64_t b) { return a > b; }); + } + + for (auto& resp : resps.responses()) { + auto dataset = resp.get_vertices(); + if (!dataset) { + continue; + } + for (GetNbrsRespDataSetIter iter(dataset); iter.valid(); iter.next()) { + auto dsts = iter.getAdjDsts(); + if (dsts.empty()) { + continue; + } + const auto& src = iter.getVid(); + // do not cache in the last step + if (currentStep_ < maxSteps_) { + adjDsts_.emplace(src, dsts); + } + + for (const auto& dst : dsts) { + if (sample_) { + if (samples.empty()) { + break; + } + if (curLimit_++ != samples.back()) { + continue; + } else { + samples.pop_back(); + } + } else { + if (curLimit_++ >= curMaxLimit_) { + break; + } + } + updateDst2VidsMap(dst2VidsMap, src, dst); + + if (currentStep_ >= maxSteps_) { + continue; + } + if (adjDsts_.find(dst) == adjDsts_.end()) { + nextStepVids_.emplace(dst); + } else { + visitedVids.emplace(dst); + } + } + } + } + if (!preVisitedVids_.empty()) { + getNeighborsFromCache(dst2VidsMap, visitedVids, samples); + } + dst2VidsMap.swap(preDst2VidsMap_); + visitedVids.swap(preVisitedVids_); + return Status::OK(); +} + +void ExpandExecutor::updateDst2VidsMap( + std::unordered_map>& dst2VidsMap, + const Value& src, + const Value& dst) { + auto findSrc = preDst2VidsMap_.find(src); + if (findSrc == preDst2VidsMap_.end()) { + auto findDst = dst2VidsMap.find(dst); + if (findDst == dst2VidsMap.end()) { + std::unordered_set tmp({src}); + dst2VidsMap.emplace(dst, std::move(tmp)); + } else { + findDst->second.emplace(src); + } + } else { + auto findDst = dst2VidsMap.find(dst); + if (findDst == dst2VidsMap.end()) { + dst2VidsMap.emplace(dst, findSrc->second); + } else { + findDst->second.insert(findSrc->second.begin(), findSrc->second.end()); + } + } +} + +folly::Future ExpandExecutor::buildResult() { + DataSet ds; + ds.colNames = expand_->colNames(); + for (auto pair : preDst2VidsMap_) { + auto& dst = pair.first; + for (auto src : pair.second) { + Row row; + row.values.emplace_back(src); + row.values.emplace_back(dst); + ds.rows.emplace_back(std::move(row)); + } + } + return finish(ResultBuilder().value(Value(std::move(ds))).build()); +} + +} // namespace graph +} // namespace nebula diff --git a/src/graph/executor/query/ExpandExecutor.h b/src/graph/executor/query/ExpandExecutor.h new file mode 100644 index 00000000000..8ac3dfac92a --- /dev/null +++ b/src/graph/executor/query/ExpandExecutor.h @@ -0,0 +1,84 @@ +// Copyright (c) 2020 vesoft inc. All rights reserved. +// +// This source code is licensed under Apache 2.0 License. + +#ifndef GRAPH_EXECUTOR_QUERY_EXPAND_H_ +#define GRAPH_EXECUTOR_QUERY_EXPAND_H_ + +#include "graph/executor/StorageAccessExecutor.h" +#include "graph/planner/plan/Query.h" + +// The go statement is divided into two operators, expand operator and expandAll operator. +// expand is responsible for expansion and does not take attributes. + +// if no need join, invoke the getDstBySrc interface to output only one column, +// which is the set of destination vids(deduplication) after maxSteps expansion + +// if need to join with the previous statement, invoke the getNeighbors interface, +// and we need save the mapping relationship between the init vid and the destination vid +// during the expansion. finally output two columns, the first column is the init vid +// the second column is the destination vid after maxSteps expansion from this vid + +// If maxSteps == 0, no expansion, and output after checking the type of vids + +// adjList is an adjacency list structure +// which saves the vids and all destination vids that expand one step +// when expanding, if the vid has already been visited, do not need to go through RPC +// just get the result directly through adjList_ + +namespace nebula { +namespace graph { +class ExpandExecutor final : public StorageAccessExecutor { + public: + ExpandExecutor(const PlanNode* node, QueryContext* qctx) + : StorageAccessExecutor("ExpandExecutor", node, qctx) { + expand_ = asNode(node); + } + + Status buildRequestVids(); + + folly::Future execute() override; + + folly::Future getNeighbors(); + + folly::Future GetDstBySrc(); + + void getNeighborsFromCache(std::unordered_map>& dst2VidMap, + std::unordered_set& visitedVids, + std::vector& samples); + + folly::Future expandFromCache(); + + void updateDst2VidsMap(std::unordered_map>& dst2VidMap, + const Value& src, + const Value& dst); + + folly::Future buildResult(); + + using RpcResponse = storage::StorageRpcResponse; + folly::Future handleResponse(RpcResponse&& resps); + + private: + const Expand* expand_; + size_t currentStep_{0}; + size_t maxSteps_{0}; + + bool sample_{false}; + int64_t curLimit_{0}; + int64_t curMaxLimit_{std::numeric_limits::max()}; + std::vector stepLimits_; + + std::unordered_set nextStepVids_; + std::unordered_set preVisitedVids_; + std::unordered_map> adjDsts_; + + // keep the mapping relationship between the init vid and the destination vid + // during the expansion. KEY : edge's dst, VALUE : init vids + // then we can know which init vids can reach the current destination point + std::unordered_map> preDst2VidsMap_; +}; + +} // namespace graph +} // namespace nebula + +#endif // GRAPH_EXECUTOR_QUERY_ExpandExecutor_H_ diff --git a/src/graph/executor/query/GetDstBySrcExecutor.cpp b/src/graph/executor/query/GetDstBySrcExecutor.cpp deleted file mode 100644 index 5635d8106ef..00000000000 --- a/src/graph/executor/query/GetDstBySrcExecutor.cpp +++ /dev/null @@ -1,89 +0,0 @@ -// Copyright (c) 2020 vesoft inc. All rights reserved. -// -// This source code is licensed under Apache 2.0 License. - -#include "graph/executor/query/GetDstBySrcExecutor.h" - -#include "graph/service/GraphFlags.h" -#include "graph/util/Utils.h" - -using nebula::storage::StorageClient; -using nebula::storage::StorageRpcResponse; -using nebula::storage::cpp2::GetDstBySrcResponse; - -namespace nebula { -namespace graph { - -StatusOr> GetDstBySrcExecutor::buildRequestList() { - SCOPED_TIMER(&execTime_); - auto inputVar = gd_->inputVar(); - auto iter = ectx_->getResult(inputVar).iter(); - return buildRequestListByVidType(iter.get(), gd_->src(), gd_->dedup()); -} - -folly::Future GetDstBySrcExecutor::execute() { - auto res = buildRequestList(); - NG_RETURN_IF_ERROR(res); - auto reqList = std::move(res).value(); - if (reqList.empty()) { - DataSet emptyResult; - emptyResult.colNames = gd_->colNames(); - return finish(ResultBuilder() - .value(Value(std::move(emptyResult))) - .iter(Iterator::Kind::kSequential) - .build()); - } - - time::Duration getDstTime; - StorageClient* storageClient = qctx_->getStorageClient(); - QueryExpressionContext qec(qctx()->ectx()); - StorageClient::CommonRequestParam param(gd_->space(), - qctx()->rctx()->session()->id(), - qctx()->plan()->id(), - qctx()->plan()->isProfileEnabled()); - return storageClient->getDstBySrc(param, std::move(reqList), gd_->edgeTypes()) - .via(runner()) - .ensure([this, getDstTime]() { - SCOPED_TIMER(&execTime_); - otherStats_.emplace("total_rpc_time", folly::sformat("{}(us)", getDstTime.elapsedInUSec())); - }) - .thenValue([this](StorageRpcResponse&& resp) { - memory::MemoryCheckGuard guard; - SCOPED_TIMER(&execTime_); - auto& hostLatency = resp.hostLatency(); - for (size_t i = 0; i < hostLatency.size(); ++i) { - size_t size = 0u; - auto& result = resp.responses()[i]; - if (result.dsts_ref().has_value()) { - size = (*result.dsts_ref()).size(); - } - auto info = util::collectRespProfileData(result.result, hostLatency[i], size); - otherStats_.emplace(folly::sformat("resp[{}]", i), folly::toPrettyJson(info)); - } - return handleResponse(resp, this->gd_->colNames()); - }); -} - -Status GetDstBySrcExecutor::handleResponse(RpcResponse& resps, - const std::vector& colNames) { - auto result = handleCompleteness(resps, FLAGS_accept_partial_success); - NG_RETURN_IF_ERROR(result); - ResultBuilder builder; - builder.state(result.value()); - - auto& responses = resps.responses(); - DataSet ds; - for (auto& resp : responses) { - auto* dataset = resp.get_dsts(); - if (dataset == nullptr) { - continue; - } - dataset->colNames = colNames; - ds.append(std::move(*dataset)); - } - builder.value(Value(std::move(ds))).iter(Iterator::Kind::kSequential); - return finish(builder.build()); -} - -} // namespace graph -} // namespace nebula diff --git a/src/graph/executor/query/GetDstBySrcExecutor.h b/src/graph/executor/query/GetDstBySrcExecutor.h deleted file mode 100644 index df8f046f6d6..00000000000 --- a/src/graph/executor/query/GetDstBySrcExecutor.h +++ /dev/null @@ -1,36 +0,0 @@ -// Copyright (c) 2020 vesoft inc. All rights reserved. -// -// This source code is licensed under Apache 2.0 License. - -#ifndef GRAPH_EXECUTOR_QUERY_GETDSTBYSRCEXECUTOR_H_ -#define GRAPH_EXECUTOR_QUERY_GETDSTBYSRCEXECUTOR_H_ - -#include "graph/executor/StorageAccessExecutor.h" -#include "graph/planner/plan/Query.h" - -// Get the dst id of the src id. The dst is is partially deduplicated on the storage side. -namespace nebula { -namespace graph { -class GetDstBySrcExecutor final : public StorageAccessExecutor { - public: - GetDstBySrcExecutor(const PlanNode* node, QueryContext* qctx) - : StorageAccessExecutor("GetDstBySrcExecutor", node, qctx) { - gd_ = asNode(node); - } - - folly::Future execute() override; - - StatusOr> buildRequestList(); - - private: - using RpcResponse = storage::StorageRpcResponse; - Status handleResponse(RpcResponse& resps, const std::vector& colNames); - - private: - const GetDstBySrc* gd_; -}; - -} // namespace graph -} // namespace nebula - -#endif // GRAPH_EXECUTOR_QUERY_GETDSTBYSRCEXECUTOR_H_ diff --git a/src/graph/optimizer/CMakeLists.txt b/src/graph/optimizer/CMakeLists.txt index 2c1ac1d15c1..3aa41902e12 100644 --- a/src/graph/optimizer/CMakeLists.txt +++ b/src/graph/optimizer/CMakeLists.txt @@ -20,12 +20,14 @@ nebula_add_library( rule/MergeGetNbrsAndProjectRule.cpp rule/IndexScanRule.cpp rule/PushLimitDownGetNeighborsRule.cpp + rule/PushLimitDownExpandAllRule.cpp rule/PushStepSampleDownGetNeighborsRule.cpp rule/PushStepLimitDownGetNeighborsRule.cpp rule/TopNRule.cpp rule/PushEFilterDownRule.cpp rule/PushFilterDownAggregateRule.cpp rule/PushFilterDownProjectRule.cpp + rule/PushFilterDownExpandAllRule.cpp rule/PushFilterDownHashInnerJoinRule.cpp rule/PushFilterDownHashLeftJoinRule.cpp rule/PushFilterDownInnerJoinRule.cpp @@ -52,7 +54,6 @@ nebula_add_library( rule/PushLimitDownScanEdgesAppendVerticesRule.cpp rule/PushTopNDownIndexScanRule.cpp rule/PushLimitDownScanEdgesRule.cpp - rule/RemoveProjectDedupBeforeGetDstBySrcRule.cpp rule/PushFilterDownTraverseRule.cpp rule/RemoveAppendVerticesBelowJoinRule.cpp ) diff --git a/src/graph/optimizer/Optimizer.cpp b/src/graph/optimizer/Optimizer.cpp index 58c54e30e33..138503e67b2 100644 --- a/src/graph/optimizer/Optimizer.cpp +++ b/src/graph/optimizer/Optimizer.cpp @@ -146,8 +146,12 @@ bool findArgumentRefPlanNodeInPath(const std::vector &path, Pl // Argument is in the right side dependency of binary plan node, check the left child // output columns if (argument->isColumnsIncludedIn(bpn->left())) { - argument->setInputVar(bpn->left()->outputVar()); - return true; + if (argument->inputVar().empty()) { + argument->setInputVar(bpn->left()->outputVar()); + return true; + } else { + return argument->inputVar() == bpn->left()->outputVar(); + } } } else { // Argument is in the left side dependency of binary plan node, continue to find @@ -170,9 +174,6 @@ Status Optimizer::rewriteArgumentInputVarInternal(PlanNode *root, switch (root->numDeps()) { case 0: { if (root->kind() == PlanNode::Kind::kArgument) { - DCHECK(root->inputVar().empty()) - << "Should keep the input empty for argument when plan generation"; - if (!findArgumentRefPlanNodeInPath(path, root) || root->inputVar().empty()) { return Status::Error("Could not find the right input variable for argument plan node"); } diff --git a/src/graph/optimizer/rule/PushFilterDownExpandAllRule.cpp b/src/graph/optimizer/rule/PushFilterDownExpandAllRule.cpp new file mode 100644 index 00000000000..8bc954a0327 --- /dev/null +++ b/src/graph/optimizer/rule/PushFilterDownExpandAllRule.cpp @@ -0,0 +1,144 @@ +/* Copyright (c) 2020 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "graph/optimizer/rule/PushFilterDownExpandAllRule.h" + +#include "common/expression/Expression.h" +#include "graph/optimizer/OptContext.h" +#include "graph/optimizer/OptGroup.h" +#include "graph/planner/plan/PlanNode.h" +#include "graph/planner/plan/Query.h" +#include "graph/visitor/ExtractFilterExprVisitor.h" +#include "graph/visitor/RewriteVisitor.h" + +using nebula::Expression; +using nebula::graph::ExpandAll; +using nebula::graph::Filter; +using nebula::graph::PlanNode; +using nebula::graph::QueryContext; + +namespace nebula { +namespace opt { + +std::unique_ptr PushFilterDownExpandAllRule::kInstance = + std::unique_ptr(new PushFilterDownExpandAllRule()); + +PushFilterDownExpandAllRule::PushFilterDownExpandAllRule() { + RuleSet::QueryRules().addRule(this); +} + +const Pattern &PushFilterDownExpandAllRule::pattern() const { + static Pattern pattern = + Pattern::create(PlanNode::Kind::kFilter, {Pattern::create(PlanNode::Kind::kExpandAll)}); + return pattern; +} + +bool PushFilterDownExpandAllRule::match(OptContext *ctx, const MatchedResult &matched) const { + if (!OptRule::match(ctx, matched)) { + return false; + } + auto expandAll = static_cast(matched.planNode({0, 0})); + if (expandAll->minSteps() != expandAll->maxSteps()) { + return false; + } + auto edgeProps = expandAll->edgeProps(); + // if fetching props of edge in ExpandAll, let it go and do more checks in + // transform. otherwise skip this rule. + return edgeProps != nullptr && !edgeProps->empty(); +} + +Expression *rewriteVarProp(Expression *expr, const ExpandAll *expandAll) { + auto matcher = [](const Expression *e) -> bool { + return e->kind() == Expression::Kind::kVarProperty; + }; + auto rewriter = [expandAll](const Expression *e) -> Expression * { + DCHECK_EQ(e->kind(), Expression::Kind::kVarProperty); + auto colName = static_cast(e)->prop(); + auto vertexColumns = expandAll->vertexColumns(); + if (vertexColumns) { + for (const auto &column : vertexColumns->columns()) { + if (column->name() == colName) { + return column->expr(); + } + } + } + auto edgeColumns = expandAll->edgeColumns(); + if (edgeColumns) { + for (const auto &column : edgeColumns->columns()) { + if (column->name() == colName) { + return column->expr(); + } + } + } + return const_cast(e); + }; + return graph::RewriteVisitor::transform(expr, std::move(matcher), std::move(rewriter)); +} + +StatusOr PushFilterDownExpandAllRule::transform( + OptContext *ctx, const MatchedResult &matched) const { + auto filterGroupNode = matched.node; + auto expandAllGroupNode = matched.dependencies.front().node; + auto filter = static_cast(filterGroupNode->node()); + auto expandAll = static_cast(expandAllGroupNode->node()); + auto colNames = expandAll->colNames(); + auto qctx = ctx->qctx(); + auto pool = qctx->objPool(); + auto condition = filter->condition()->clone(); + + graph::ExtractFilterExprVisitor visitor(pool, colNames); + condition->accept(&visitor); + if (!visitor.ok()) { + return TransformResult::noTransform(); + } + + auto remainedExpr = std::move(visitor).remainedExpr(); + OptGroupNode *newFilterGroupNode = nullptr; + PlanNode *newFilter = nullptr; + if (remainedExpr != nullptr) { + newFilter = Filter::make(qctx, nullptr, remainedExpr); + newFilter->setOutputVar(filter->outputVar()); + newFilterGroupNode = OptGroupNode::create(ctx, newFilter, filterGroupNode->group()); + } + + // rewrite filter to original expression (edgePropExpression、srcPropExpression) + auto newFilterExpr = rewriteVarProp(condition, expandAll); + auto newExpandAll = static_cast(expandAll->clone()); + // push filter down storage + if (expandAll->filter() != nullptr) { + auto logicExpr = LogicalExpression::makeAnd(pool, newFilterExpr, expandAll->filter()->clone()); + newFilterExpr = logicExpr; + } + newExpandAll->setFilter(newFilterExpr); + + OptGroupNode *newExpandGroupNode = nullptr; + if (newFilterGroupNode != nullptr) { + // Filter(A&&B)<-ExpandAll(C) => Filter(A)<-ExpandAll(B&&C) + auto newGroup = OptGroup::create(ctx); + newExpandGroupNode = newGroup->makeGroupNode(newExpandAll); + newFilter->setInputVar(newExpandAll->outputVar()); + newFilterGroupNode->dependsOn(newGroup); + } else { + // Filter(A)<-ExpandAll(C) => ExpandAll(A&&C) + newExpandGroupNode = OptGroupNode::create(ctx, newExpandAll, filterGroupNode->group()); + newExpandAll->setOutputVar(filter->outputVar()); + } + + for (auto dep : expandAllGroupNode->dependencies()) { + newExpandGroupNode->dependsOn(dep); + } + + TransformResult result; + result.eraseCurr = true; + result.newGroupNodes.emplace_back(newFilterGroupNode ? newFilterGroupNode : newExpandGroupNode); + return result; +} + +std::string PushFilterDownExpandAllRule::toString() const { + return "PushFilterDownExpandAllRule"; +} + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushFilterDownExpandAllRule.h b/src/graph/optimizer/rule/PushFilterDownExpandAllRule.h new file mode 100644 index 00000000000..118da96b6b0 --- /dev/null +++ b/src/graph/optimizer/rule/PushFilterDownExpandAllRule.h @@ -0,0 +1,63 @@ +/* Copyright (c) 2020 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#ifndef GRAPH_OPTIMIZER_RULE_PUSHFILTERDOWNEXPANDALLRULE_H_ +#define GRAPH_OPTIMIZER_RULE_PUSHFILTERDOWNEXPANDALLRULE_H_ + +#include "graph/optimizer/OptRule.h" + +namespace nebula { +namespace opt { + +// Embed the [[Filter]] into [[ExpandAll]] +// Required conditions: +// 1. Match the pattern +// 2. Filter contains subexpressions that meet pushdown conditions +// Benefits: +// 1. Filter data early to optimize performance +// +// Transformation: +// Before: +// +// +------------------+------------------+ +// | Filter | +// |($^.player.age>3 and $$.player.age<4)| +// +------------------+------------------+ +// | +// +------+------+ +// | ExpandAll | +// +------+------+ +// +// After: +// +// +--------+--------+ +// | Filter | +// |($$.player.age<4)| +// +--------+--------+ +// | +// +--------+--------+ +// | ExpandAll | +// |($^.player.age>3)| +// +--------+--------+ + +class PushFilterDownExpandAllRule final : public OptRule { + public: + const Pattern &pattern() const override; + + bool match(OptContext *ctx, const MatchedResult &matched) const override; + StatusOr transform(OptContext *ctx, const MatchedResult &matched) const override; + + std::string toString() const override; + + private: + PushFilterDownExpandAllRule(); + + static std::unique_ptr kInstance; +}; + +} // namespace opt +} // namespace nebula + +#endif // GRAPH_OPTIMIZER_RULE_PUSHFILTERDOWNEXPANDALLRULE_H_ diff --git a/src/graph/optimizer/rule/PushFilterDownHashInnerJoinRule.cpp b/src/graph/optimizer/rule/PushFilterDownHashInnerJoinRule.cpp index 009983c7623..4f32a724f99 100644 --- a/src/graph/optimizer/rule/PushFilterDownHashInnerJoinRule.cpp +++ b/src/graph/optimizer/rule/PushFilterDownHashInnerJoinRule.cpp @@ -102,7 +102,7 @@ OptGroup* PushFilterDownHashInnerJoinRule::pushFilterDownChild(OptContext* octx, // split the `condition` based on whether the varPropExpr comes from the left child auto picker = [&colNames](const Expression* e) -> bool { - return graph::ExpressionUtils::checkVarPropIfExist(colNames, e); + return graph::ExpressionUtils::checkColName(colNames, e); }; Expression* filterPicked = nullptr; diff --git a/src/graph/optimizer/rule/PushFilterDownHashLeftJoinRule.cpp b/src/graph/optimizer/rule/PushFilterDownHashLeftJoinRule.cpp index 8636920abe5..d68c38f6cc5 100644 --- a/src/graph/optimizer/rule/PushFilterDownHashLeftJoinRule.cpp +++ b/src/graph/optimizer/rule/PushFilterDownHashLeftJoinRule.cpp @@ -98,7 +98,7 @@ OptGroup* PushFilterDownHashLeftJoinRule::pushFilterDownChild(OptContext* octx, // split the `condition` based on whether the varPropExpr comes from the left child auto picker = [&colNames](const Expression* e) -> bool { - return graph::ExpressionUtils::checkVarPropIfExist(colNames, e); + return graph::ExpressionUtils::checkColName(colNames, e); }; Expression* filterPicked = nullptr; diff --git a/src/graph/optimizer/rule/PushFilterDownInnerJoinRule.cpp b/src/graph/optimizer/rule/PushFilterDownInnerJoinRule.cpp index dabf3105a07..acf3be67c1a 100644 --- a/src/graph/optimizer/rule/PushFilterDownInnerJoinRule.cpp +++ b/src/graph/optimizer/rule/PushFilterDownInnerJoinRule.cpp @@ -49,7 +49,7 @@ StatusOr PushFilterDownInnerJoinRule::transform( // split the `condition` based on whether the varPropExpr comes from the left child auto picker = [&leftVarColNames](const Expression* e) -> bool { - return graph::ExpressionUtils::checkVarPropIfExist(leftVarColNames, e); + return graph::ExpressionUtils::checkColName(leftVarColNames, e); }; Expression* filterPicked = nullptr; Expression* filterUnpicked = nullptr; diff --git a/src/graph/optimizer/rule/PushLimitDownExpandAllRule.cpp b/src/graph/optimizer/rule/PushLimitDownExpandAllRule.cpp new file mode 100644 index 00000000000..62a54cf6a4d --- /dev/null +++ b/src/graph/optimizer/rule/PushLimitDownExpandAllRule.cpp @@ -0,0 +1,86 @@ +/* Copyright (c) 2020 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "graph/optimizer/rule/PushLimitDownExpandAllRule.h" + +#include "graph/optimizer/OptContext.h" +#include "graph/optimizer/OptGroup.h" +#include "graph/planner/plan/PlanNode.h" +#include "graph/planner/plan/Query.h" +#include "graph/util/ExpressionUtils.h" + +using nebula::graph::ExpandAll; +using nebula::graph::Limit; +using nebula::graph::PlanNode; +using nebula::graph::QueryContext; + +namespace nebula { +namespace opt { + +std::unique_ptr PushLimitDownExpandAllRule::kInstance = + std::unique_ptr(new PushLimitDownExpandAllRule()); + +PushLimitDownExpandAllRule::PushLimitDownExpandAllRule() { + RuleSet::QueryRules().addRule(this); +} + +const Pattern &PushLimitDownExpandAllRule::pattern() const { + static Pattern pattern = Pattern::create(graph::PlanNode::Kind::kLimit, + {Pattern::create(graph::PlanNode::Kind::kExpandAll)}); + return pattern; +} + +bool PushLimitDownExpandAllRule::match(OptContext *ctx, const MatchedResult &matched) const { + if (!OptRule::match(ctx, matched)) { + return false; + } + auto expandAll = static_cast(matched.planNode({0, 0})); + return expandAll->minSteps() == expandAll->maxSteps(); +} + +StatusOr PushLimitDownExpandAllRule::transform( + OptContext *octx, const MatchedResult &matched) const { + auto *qctx = octx->qctx(); + auto limitGroupNode = matched.node; + auto expandAllGroupNode = matched.dependencies.front().node; + + const auto limit = static_cast(limitGroupNode->node()); + const auto expandAll = static_cast(expandAllGroupNode->node()); + + if (!graph::ExpressionUtils::isEvaluableExpr(limit->countExpr())) { + return TransformResult::noTransform(); + } + int64_t limitRows = limit->offset() + limit->count(qctx); + if (expandAll->limit(qctx) >= 0 && limitRows >= expandAll->limit(qctx)) { + return TransformResult::noTransform(); + } + + auto newLimit = static_cast(limit->clone()); + newLimit->setOutputVar(limit->outputVar()); + auto newLimitGroupNode = OptGroupNode::create(octx, newLimit, limitGroupNode->group()); + + auto newExpandAll = static_cast(expandAll->clone()); + newExpandAll->setLimit(limitRows); + auto newExpandAllGroup = OptGroup::create(octx); + auto newExpandAllGroupNode = newExpandAllGroup->makeGroupNode(newExpandAll); + + newLimitGroupNode->dependsOn(newExpandAllGroup); + newLimit->setInputVar(newExpandAll->outputVar()); + for (auto dep : expandAllGroupNode->dependencies()) { + newExpandAllGroupNode->dependsOn(dep); + } + + TransformResult result; + result.eraseAll = true; + result.newGroupNodes.emplace_back(newLimitGroupNode); + return result; +} + +std::string PushLimitDownExpandAllRule::toString() const { + return "PushLimitDownExpandAllRule"; +} + +} // namespace opt +} // namespace nebula diff --git a/src/graph/optimizer/rule/PushLimitDownExpandAllRule.h b/src/graph/optimizer/rule/PushLimitDownExpandAllRule.h new file mode 100644 index 00000000000..839d94aac55 --- /dev/null +++ b/src/graph/optimizer/rule/PushLimitDownExpandAllRule.h @@ -0,0 +1,62 @@ +/* Copyright (c) 2020 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#ifndef GRAPH_OPTIMIZER_RULE_PUSHLIMITDOWNEXPANDALL_H +#define GRAPH_OPTIMIZER_RULE_PUSHLIMITDOWNEXPANDALL_H + +#include "graph/optimizer/OptRule.h" + +namespace nebula { +namespace opt { + +// Embedding limit to [[ExpandAll]] +// Required conditions: +// 1. Match the pattern +// Benefits: +// 1. Limit data early to optimize performance +// +// Transformation: +// Before: +// +// +--------+--------+ +// | Limit | +// | (limit=3) | +// +--------+--------+ +// | +// +---------+---------+ +// | ExpandAll | +// +---------+---------+ +// +// After: +// +// +--------+--------+ +// | Limit | +// | (limit=3) | +// +--------+--------+ +// | +// +---------+---------+ +// | ExpandAll | +// | (limit=3) | +// +---------+---------+ + +class PushLimitDownExpandAllRule final : public OptRule { + public: + const Pattern &pattern() const override; + + bool match(OptContext *ctx, const MatchedResult &matched) const override; + StatusOr transform(OptContext *ctx, + const MatchedResult &matched) const override; + + std::string toString() const override; + + private: + PushLimitDownExpandAllRule(); + + static std::unique_ptr kInstance; +}; + +} // namespace opt +} // namespace nebula +#endif diff --git a/src/graph/optimizer/rule/RemoveProjectDedupBeforeGetDstBySrcRule.cpp b/src/graph/optimizer/rule/RemoveProjectDedupBeforeGetDstBySrcRule.cpp deleted file mode 100644 index 91e67af681b..00000000000 --- a/src/graph/optimizer/rule/RemoveProjectDedupBeforeGetDstBySrcRule.cpp +++ /dev/null @@ -1,93 +0,0 @@ -/* Copyright (c) 2021 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License. - */ - -#include "graph/optimizer/rule/RemoveProjectDedupBeforeGetDstBySrcRule.h" - -#include "common/expression/ColumnExpression.h" -#include "graph/optimizer/OptContext.h" -#include "graph/optimizer/OptGroup.h" -#include "graph/planner/plan/PlanNode.h" -#include "graph/planner/plan/Query.h" - -using nebula::graph::PlanNode; -using nebula::graph::QueryContext; - -namespace nebula { -namespace opt { - -std::unique_ptr RemoveProjectDedupBeforeGetDstBySrcRule::kInstance = - std::unique_ptr( - new RemoveProjectDedupBeforeGetDstBySrcRule()); - -RemoveProjectDedupBeforeGetDstBySrcRule::RemoveProjectDedupBeforeGetDstBySrcRule() { - RuleSet::QueryRules().addRule(this); -} - -const Pattern& RemoveProjectDedupBeforeGetDstBySrcRule::pattern() const { - static Pattern pattern = Pattern::create( - graph::PlanNode::Kind::kGetDstBySrc, - {Pattern::create(graph::PlanNode::Kind::kDedup, - {Pattern::create(graph::PlanNode::Kind::kProject, - {Pattern::create(graph::PlanNode::Kind::kDataCollect)})})}); - return pattern; -} - -StatusOr RemoveProjectDedupBeforeGetDstBySrcRule::transform( - OptContext* octx, const MatchedResult& matched) const { - auto* getDstBySrcGroupNode = matched.node; - auto* getDstBySrc = static_cast(getDstBySrcGroupNode->node()); - auto* projectGroupNode = matched.dependencies.front().dependencies.front().node; - auto* project = static_cast(projectGroupNode->node()); - - auto* newGetDstBySrc = static_cast(getDstBySrc->clone()); - // Replace `$-._vid` with `COLUMN[0]` - newGetDstBySrc->setSrc(ColumnExpression::make(getDstBySrc->src()->getObjPool(), 0)); - newGetDstBySrc->setOutputVar(getDstBySrc->outputVar()); - newGetDstBySrc->setInputVar(project->inputVar()); - newGetDstBySrc->setColNames(newGetDstBySrc->colNames()); - - auto newGetDstBySrcNode = - OptGroupNode::create(octx, newGetDstBySrc, getDstBySrcGroupNode->group()); - for (auto dep : projectGroupNode->dependencies()) { - newGetDstBySrcNode->dependsOn(dep); - } - - TransformResult result; - result.newGroupNodes.emplace_back(newGetDstBySrcNode); - result.eraseAll = true; - return result; -} - -bool RemoveProjectDedupBeforeGetDstBySrcRule::match(OptContext*, - const MatchedResult& matched) const { - auto* getDstBySrc = static_cast(matched.node->node()); - auto* project = static_cast( - matched.dependencies.front().dependencies.front().node->node()); - auto* dataCollect = static_cast( - matched.dependencies.front().dependencies.front().dependencies.front().node->node()); - - // src should be $-._vid - auto* expr = getDstBySrc->src(); - if (expr->kind() != Expression::Kind::kInputProperty || - static_cast(expr)->prop() != "_vid") { - return false; - } - if (project->columns()->size() != 1) { - return false; - } - if (dataCollect->kind() != graph::DataCollect::DCKind::kMToN || !dataCollect->distinct() || - dataCollect->colNames().size() != 1) { - return false; - } - - return true; -} - -std::string RemoveProjectDedupBeforeGetDstBySrcRule::toString() const { - return "RemoveProjectDedupBeforeGetDstBySrcRule"; -} - -} // namespace opt -} // namespace nebula diff --git a/src/graph/optimizer/rule/RemoveProjectDedupBeforeGetDstBySrcRule.h b/src/graph/optimizer/rule/RemoveProjectDedupBeforeGetDstBySrcRule.h deleted file mode 100644 index c20cfe3efb5..00000000000 --- a/src/graph/optimizer/rule/RemoveProjectDedupBeforeGetDstBySrcRule.h +++ /dev/null @@ -1,54 +0,0 @@ -/* Copyright (c) 2021 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License. - */ - -#ifndef GRAPH_OPTIMIZER_RULE_REMOVEPROJECTDEDUPBEFOREGETDSTBYSRCRULE_H_ -#define GRAPH_OPTIMIZER_RULE_REMOVEPROJECTDEDUPBEFOREGETDSTBYSRCRULE_H_ - -#include - -#include "graph/optimizer/OptRule.h" - -namespace nebula { -namespace opt { - -// DataCollect contains only one column that is a deduped vid column which GetDstBySrc needs, -// so the following Project and Dedup are useless. -// -// Transformation: -// Before: -// -// +---------+---------+ -// | GetDstBySrc | -// +---------+---------+ -// | Dedup | -// +---------+---------+ -// | Project | -// +---------+---------+ -// | DataCollect | -// +---------+---------+ -// -// After: -// // Remove Project node - -class RemoveProjectDedupBeforeGetDstBySrcRule final : public OptRule { - public: - const Pattern &pattern() const override; - - StatusOr transform(OptContext *ctx, const MatchedResult &matched) const override; - - bool match(OptContext *ctx, const MatchedResult &matched) const override; - - std::string toString() const override; - - private: - RemoveProjectDedupBeforeGetDstBySrcRule(); - - static std::unique_ptr kInstance; -}; - -} // namespace opt -} // namespace nebula - -#endif // GRAPH_OPTIMIZER_RULE_REMOVEPROJECTDEDUPBEFOREGETDSTBYSRCRULE_H_ diff --git a/src/graph/planner/SequentialPlanner.cpp b/src/graph/planner/SequentialPlanner.cpp index 08350e57d44..d05d1af4c29 100644 --- a/src/graph/planner/SequentialPlanner.cpp +++ b/src/graph/planner/SequentialPlanner.cpp @@ -25,7 +25,7 @@ StatusOr SequentialPlanner::transform(AstContext* astCtx) { // Remove left tail kStart plannode before append plan. // It allows that kUse sentence to append kMatch Sentence. // For example: Use ...; Match ... - rmLeftTailStartNode((iter + 1)->get(), iter->get()->sentence()->kind()); + rmLeftTailStartNode((iter + 1)->get()); NG_RETURN_IF_ERROR((iter + 1)->get()->appendPlan(iter->get()->root())); } if (validators.front()->tail()->isSingleInput()) { @@ -34,7 +34,6 @@ StatusOr SequentialPlanner::transform(AstContext* astCtx) { } else { subPlan.tail = validators.front()->tail(); } - VLOG(1) << subPlan; return subPlan; } @@ -42,10 +41,8 @@ StatusOr SequentialPlanner::transform(AstContext* astCtx) { // Because the left tail plannode is StartNode which needs to be removed, // and remain one size for add dependency // TODO: It's a temporary solution, remove it after Execute multiple sequences one by one. -void SequentialPlanner::rmLeftTailStartNode(Validator* validator, Sentence::Kind appendPlanKind) { - if (appendPlanKind != Sentence::Kind::kUse || - validator->tail()->kind() != PlanNode::Kind::kStart || - validator->root()->dependencies().size() == 0UL) { +void SequentialPlanner::rmLeftTailStartNode(Validator* validator) { + if (validator->tail()->kind() != PlanNode::Kind::kStart) { return; } diff --git a/src/graph/planner/SequentialPlanner.h b/src/graph/planner/SequentialPlanner.h index 769e49d8cdc..a86583966e1 100644 --- a/src/graph/planner/SequentialPlanner.h +++ b/src/graph/planner/SequentialPlanner.h @@ -26,7 +26,7 @@ class SequentialPlanner final : public Planner { */ StatusOr transform(AstContext* astCtx) override; - void rmLeftTailStartNode(Validator* validator, Sentence::Kind appendPlanKind); + void rmLeftTailStartNode(Validator* validator); private: SequentialPlanner() = default; diff --git a/src/graph/planner/ngql/GoPlanner.cpp b/src/graph/planner/ngql/GoPlanner.cpp index 76ec29965f7..5d3739ae2c7 100644 --- a/src/graph/planner/ngql/GoPlanner.cpp +++ b/src/graph/planner/ngql/GoPlanner.cpp @@ -60,6 +60,31 @@ void GoPlanner::doBuildEdgeProps(std::unique_ptr& eProps, bool onlyDs } } +std::vector GoPlanner::buildEdgeTypes() { + switch (goCtx_->over.direction) { + case storage::cpp2::EdgeDirection::IN_EDGE: { + std::vector edgeTypes; + edgeTypes.reserve(goCtx_->over.edgeTypes.size()); + for (auto edgeType : goCtx_->over.edgeTypes) { + edgeTypes.emplace_back(-edgeType); + } + return edgeTypes; + } + case storage::cpp2::EdgeDirection::OUT_EDGE: { + return goCtx_->over.edgeTypes; + } + case storage::cpp2::EdgeDirection::BOTH: { + std::vector edgeTypes; + for (auto edgeType : goCtx_->over.edgeTypes) { + edgeTypes.emplace_back(edgeType); + edgeTypes.emplace_back(-edgeType); + } + return edgeTypes; + } + } + return {}; +} + std::unique_ptr GoPlanner::buildVertexProps( const ExpressionProps::TagIDPropsMap& propsMap) { if (propsMap.empty()) { @@ -77,125 +102,6 @@ std::unique_ptr GoPlanner::buildVertexProps( return vertexProps; } -// ++loopSteps{0} <= steps && (var is Empty OR size(var) != 0) -Expression* GoPlanner::loopCondition(uint32_t steps, const std::string& var) { - auto* qctx = goCtx_->qctx; - auto* pool = qctx->objPool(); - - qctx->ectx()->setValue(loopStepVar_, 0); - auto step = ExpressionUtils::stepCondition(pool, loopStepVar_, steps); - auto empty = ExpressionUtils::equalCondition(pool, var, Value::kEmpty); - auto neZero = ExpressionUtils::neZeroCondition(pool, var); - auto* earlyEnd = LogicalExpression::makeOr(pool, empty, neZero); - return LogicalExpression::makeAnd(pool, step, earlyEnd); -} - -// Extracts vid and edge's prop from GN for joinDst & joinInput. -// The root plan node will output colNames of {srcProps, edgeProps, kVid, "JOIN_DST_VID"}. -PlanNode* GoPlanner::extractSrcEdgePropsFromGN(PlanNode* dep, const std::string& input) { - auto& srcEdgePropsExpr = goCtx_->srcEdgePropsExpr; - auto* pool = goCtx_->qctx->objPool(); - - if (goCtx_->joinInput) { - // extract vid from gn - auto* expr = new YieldColumn(ColumnExpression::make(pool, VID_INDEX), kVid); - srcEdgePropsExpr->addColumn(expr); - } - - if (goCtx_->joinDst) { - // extract dst from gn - auto* expr = new YieldColumn(EdgePropertyExpression::make(pool, "*", kDst), "JOIN_DST_VID"); - srcEdgePropsExpr->addColumn(expr); - } - - auto* project = Project::make(goCtx_->qctx, dep, srcEdgePropsExpr); - project->setInputVar(input); - return project; -} - -// Extracts vid and dst from GN for trackStartVid. -// The root plan node will output ColNames of {srcVidColName, "TRACK_DST_VID"}. -PlanNode* GoPlanner::extractSrcDstFromGN(PlanNode* dep, const std::string& input) { - auto qctx = goCtx_->qctx; - auto* pool = qctx->objPool(); - auto* columns = pool->makeAndAdd(); - - goCtx_->srcVidColName = qctx->vctx()->anonColGen()->getCol(); - auto* vidExpr = new YieldColumn(ColumnExpression::make(pool, VID_INDEX), goCtx_->srcVidColName); - columns->addColumn(vidExpr); - auto* dstExpr = new YieldColumn(EdgePropertyExpression::make(pool, "*", kDst), "TRACK_DST_VID"); - columns->addColumn(dstExpr); - - auto* project = Project::make(qctx, dep, columns); - project->setInputVar(input); - auto* dedup = Dedup::make(qctx, project); - return dedup; -} - -// Extracts vid from runTime input for joinInput. -// The root plan node will output ColNames of {runtimeVidName, dstVidColName}. -PlanNode* GoPlanner::extractVidFromRuntimeInput(PlanNode* dep) { - if (dep == nullptr) { - return dep; - } - auto qctx = goCtx_->qctx; - const auto& from = goCtx_->from; - - auto* columns = qctx->objPool()->makeAndAdd(); - auto* vidExpr = new YieldColumn(from.originalSrc->clone(), from.runtimeVidName); - columns->addColumn(vidExpr); - - goCtx_->dstVidColName = qctx->vctx()->anonColGen()->getCol(); - auto* dstExpr = new YieldColumn(from.originalSrc->clone(), goCtx_->dstVidColName); - columns->addColumn(dstExpr); - - auto* project = Project::make(qctx, dep, columns); - auto input = from.fromType == kPipe ? goCtx_->inputVarName : from.userDefinedVarName; - project->setInputVar(input); - - auto* dedup = Dedup::make(qctx, project); - return dedup; -} - -// Establishes a mapping between the original vId and the expanded destination vId -// during each step of the expansion in the n-step and mton-step scenario. -// The root plan node will output ColNames of {runtimeVidName, dstVidColName}. -// left: (n-1)th step -// right: (n)th step -PlanNode* GoPlanner::trackStartVid(PlanNode* left, PlanNode* right) { - auto qctx = goCtx_->qctx; - auto* pool = qctx->objPool(); - - auto* hashKey = VariablePropertyExpression::make(pool, left->outputVar(), goCtx_->dstVidColName); - auto* probeKey = - VariablePropertyExpression::make(pool, right->outputVar(), goCtx_->srcVidColName); - - auto* join = InnerJoin::make(qctx, - right, - {left->outputVar(), ExecutionContext::kLatestVersion}, - {right->outputVar(), ExecutionContext::kLatestVersion}, - {hashKey}, - {probeKey}); - std::vector colNames = left->colNames(); - colNames.insert(colNames.end(), right->colNames().begin(), right->colNames().end()); - join->setColNames(std::move(colNames)); - - // extract runtimeVid & dst from join result - auto* columns = pool->makeAndAdd(); - auto& vidName = goCtx_->from.runtimeVidName; - auto* vidExpr = new YieldColumn(InputPropertyExpression::make(pool, vidName), vidName); - columns->addColumn(vidExpr); - auto* dstExpr = - new YieldColumn(InputPropertyExpression::make(pool, "TRACK_DST_VID"), goCtx_->dstVidColName); - columns->addColumn(dstExpr); - - auto* project = Project::make(qctx, join, columns); - auto* dedup = Dedup::make(qctx, project); - dedup->setOutputVar(left->outputVar()); - - return dedup; -} - // The root plan node will output ColNames of // {srcProps, edgeProps, kVid, "JOIN_DST_VID", "DST_VID", dstProps} PlanNode* GoPlanner::buildJoinDstPlan(PlanNode* dep) { @@ -218,7 +124,7 @@ PlanNode* GoPlanner::buildJoinDstPlan(PlanNode* dep) { auto& dstPropsExpr = goCtx_->dstPropsExpr; // extract dst's prop - auto* vidExpr = new YieldColumn(ColumnExpression::make(pool, VID_INDEX), "DST_VID"); + auto* vidExpr = new YieldColumn(ColumnExpression::make(pool, VID_INDEX), "_getVertex_vid"); dstPropsExpr->addColumn(vidExpr); // extract dst's prop, vid is the last column @@ -229,472 +135,130 @@ PlanNode* GoPlanner::buildJoinDstPlan(PlanNode* dep) { auto* probeKey = ColumnExpression::make(pool, LAST_COL_INDEX); auto* join = HashLeftJoin::make(qctx, dep, project, {hashKey}, {probeKey}); - DLOG(INFO) << join->outputVar() << " hasKey: " << hashKey->toString() - << " probeKey: " << probeKey->toString(); - std::vector colNames = dep->colNames(); colNames.insert(colNames.end(), project->colNames().begin(), project->colNames().end()); join->setColNames(std::move(colNames)); - - return join; -} - -PlanNode* GoPlanner::buildJoinInputPlan(PlanNode* dep) { - auto qctx = goCtx_->qctx; - const auto& from = goCtx_->from; - const auto& steps = goCtx_->steps; - auto* pool = qctx->objPool(); - - const auto& vidName = (!steps.isMToN() && steps.steps() == 1) ? kVid : from.runtimeVidName; - auto* hashKey = VariablePropertyExpression::make(pool, dep->outputVar(), vidName); - auto* probeKey = from.originalSrc; - std::string probeName = from.fromType == kPipe ? goCtx_->inputVarName : from.userDefinedVarName; - - auto* join = InnerJoin::make(qctx, - dep, - {dep->outputVar(), ExecutionContext::kLatestVersion}, - {probeName, ExecutionContext::kLatestVersion}, - {hashKey}, - {probeKey}); - std::vector colNames = dep->colNames(); - auto* varPtr = qctx->symTable()->getVar(probeName); - DCHECK(varPtr != nullptr); - colNames.insert(colNames.end(), varPtr->colNames.begin(), varPtr->colNames.end()); - join->setColNames(std::move(colNames)); - return join; } -// The column named as dstVidColName of the left plan node join on the column named as kVid of the -// right one. -// left : (n-1)th step -// right : last step -PlanNode* GoPlanner::lastStepJoinInput(PlanNode* left, PlanNode* right) { +SubPlan GoPlanner::doPlan() { auto qctx = goCtx_->qctx; + auto& from = goCtx_->from; auto* pool = qctx->objPool(); - auto* hashKey = VariablePropertyExpression::make(pool, left->outputVar(), goCtx_->dstVidColName); - auto* probeKey = VariablePropertyExpression::make(pool, right->outputVar(), kVid); - - const auto& leftVersion = goCtx_->steps.isMToN() ? ExecutionContext::kPreviousOneVersion - : ExecutionContext::kLatestVersion; - - auto* join = InnerJoin::make(qctx, - right, - {left->outputVar(), leftVersion}, - {right->outputVar(), ExecutionContext::kLatestVersion}, - {hashKey}, - {probeKey}); - - std::vector colNames = left->colNames(); - colNames.insert(colNames.end(), right->colNames().begin(), right->colNames().end()); - join->setColNames(std::move(colNames)); - - return join; -} + size_t minStep = goCtx_->steps.mSteps(); + size_t maxStep = goCtx_->steps.nSteps(); -PlanNode* GoPlanner::buildLastStepJoinPlan(PlanNode* gn, PlanNode* join) { - if (!goCtx_->joinInput && !goCtx_->joinDst) { - return gn; - } - - auto* dep = extractSrcEdgePropsFromGN(gn, gn->outputVar()); - dep = goCtx_->joinDst ? buildJoinDstPlan(dep) : dep; - dep = goCtx_->joinInput ? lastStepJoinInput(join, dep) : dep; - dep = goCtx_->joinInput ? buildJoinInputPlan(dep) : dep; - - return dep; -} - -PlanNode* GoPlanner::lastStep(PlanNode* dep, PlanNode* join) { - auto qctx = goCtx_->qctx; - PlanNode* cur = nullptr; - - if (goCtx_->isSimple) { - auto* gd = GetDstBySrc::make(qctx, dep, goCtx_->space.id); - gd->setSrc(goCtx_->from.src); - gd->setEdgeTypes(buildEdgeTypes()); - gd->setInputVar(goCtx_->vidsVar); - gd->setColNames({goCtx_->dstIdColName}); - auto* dedup = Dedup::make(qctx, gd); - dedup->setColNames(gd->colNames()); - cur = dedup; - - if (goCtx_->joinDst) { - cur = buildJoinDstPlan(cur); - } - if (goCtx_->filter) { - cur = Filter::make(qctx, cur, goCtx_->filter); - } - if (goCtx_->joinDst || goCtx_->yieldExpr->columns().size() != 1) { - cur = Project::make(qctx, cur, goCtx_->yieldExpr); - } else { - gd->setColNames(goCtx_->colNames); - dedup->setColNames(goCtx_->colNames); - } - cur->setColNames(goCtx_->colNames); - return cur; + auto* expand = Expand::make(qctx, + startNode_, + goCtx_->space.id, + goCtx_->random, + minStep == 0 ? minStep : minStep - 1, + buildEdgeProps(true)); + if (goCtx_->joinInput) { + expand->setJoinInput(true); + expand->setColNames({"_expand_vid", "_expand_dst"}); } else { - auto* gn = GetNeighbors::make(qctx, dep, goCtx_->space.id); - gn->setSrc(goCtx_->from.src); - gn->setVertexProps(buildVertexProps(goCtx_->exprProps.srcTagProps())); - gn->setEdgeProps(buildEdgeProps(false)); - gn->setInputVar(goCtx_->vidsVar); - gn->setEdgeDirection(goCtx_->over.direction); - - const auto& steps = goCtx_->steps; - auto* sampleLimit = buildSampleLimit(gn, steps.isMToN() ? steps.nSteps() : steps.steps()); - - auto* root = buildLastStepJoinPlan(sampleLimit, join); - - if (goCtx_->filter != nullptr) { - root = Filter::make(qctx, root, goCtx_->filter); - } - - root = Project::make(qctx, root, goCtx_->yieldExpr); - root->setColNames(std::move(goCtx_->colNames)); - - if (goCtx_->distinct) { - root = Dedup::make(qctx, root); - } - return root; - } -} - -PlanNode* GoPlanner::buildOneStepJoinPlan(PlanNode* gn) { - if (!goCtx_->joinInput && !goCtx_->joinDst) { - return gn; + expand->setEdgeTypes(buildEdgeTypes()); + expand->setColNames({"_expand_vid"}); } + expand->setInputVar(goCtx_->vidsVar); + expand->setStepLimits(goCtx_->limits); - auto* dep = extractSrcEdgePropsFromGN(gn, gn->outputVar()); - dep = goCtx_->joinDst ? buildJoinDstPlan(dep) : dep; - dep = goCtx_->joinInput ? buildJoinInputPlan(dep) : dep; - - return dep; -} - -template -PlanNode* GoPlanner::buildSampleLimitImpl(PlanNode* input, T sampleLimit) { - DCHECK(!goCtx_->limits.empty()); - PlanNode* node = nullptr; - if (goCtx_->random) { - node = Sample::make(goCtx_->qctx, input, sampleLimit); - } else { - node = Limit::make(goCtx_->qctx, input, 0, sampleLimit); + if (goCtx_->joinDst) { + auto* dstExpr = + new YieldColumn(EdgePropertyExpression::make(pool, "*", kDst), "_expandall_dst"); + goCtx_->edgePropsExpr->addColumn(dstExpr); } - node->setInputVar(input->outputVar()); - node->setColNames(input->outputVarPtr()->colNames); - return node; -} - -// Generates $limits[$step-1] -Expression* GoPlanner::stepSampleLimit() { - auto qctx = goCtx_->qctx; - // $limits - const auto& limitsVarName = qctx->vctx()->anonVarGen()->getVar(); - List limitValues; - limitValues.reserve(goCtx_->limits.size()); - for (const auto& limit : goCtx_->limits) { - limitValues.values.emplace_back(limit); + auto* expandAll = ExpandAll::make(qctx, + expand, + goCtx_->space.id, + goCtx_->random, + minStep, + maxStep, + buildEdgeProps(false), + buildVertexProps(goCtx_->exprProps.srcTagProps()), + goCtx_->srcPropsExpr, + goCtx_->edgePropsExpr); + if (goCtx_->joinInput) { + expandAll->setJoinInput(true); + // Insert _expand_vid in the first column in colNames + auto colNames = expandAll->colNames(); + colNames.insert(colNames.begin(), "_expand_vid"); + expandAll->setColNames(colNames); } - qctx->ectx()->setValue(limitsVarName, Value(std::move(limitValues))); - auto* limitsVar = VariableExpression::make(qctx->objPool(), limitsVarName); - // $step - auto* stepVar = VariableExpression::make(qctx->objPool(), loopStepVar_); - // step inc - auto* stepInc = ArithmeticExpression::makeMinus( - qctx->objPool(), stepVar, ConstantExpression::make(qctx->objPool(), 1)); - // subscript - auto* subscript = SubscriptExpression::make(qctx->objPool(), limitsVar, stepInc); - return subscript; -} - -SubPlan GoPlanner::oneStepPlan(SubPlan& startVidPlan) { - auto qctx = goCtx_->qctx; - auto isSimple = goCtx_->isSimple; - - PlanNode* scan = nullptr; - PlanNode* cur = nullptr; - if (isSimple) { - auto* gd = GetDstBySrc::make(qctx, startVidPlan.root, goCtx_->space.id); - gd->setSrc(goCtx_->from.src); - gd->setEdgeTypes(buildEdgeTypes()); - gd->setInputVar(goCtx_->vidsVar); - gd->setColNames({goCtx_->dstIdColName}); - scan = gd; - - auto* dedup = Dedup::make(qctx, gd); - dedup->setColNames(gd->colNames()); - cur = dedup; - if (goCtx_->joinDst) { - cur = buildJoinDstPlan(cur); - } - - if (goCtx_->filter != nullptr) { - cur = Filter::make(qctx, cur, goCtx_->filter); - } - - if (goCtx_->joinDst || goCtx_->yieldExpr->columns().size() != 1) { - cur = Project::make(qctx, cur, goCtx_->yieldExpr); - } else { - gd->setColNames(goCtx_->colNames); - dedup->setColNames(goCtx_->colNames); - } - cur->setColNames(std::move(goCtx_->colNames)); - } else { - auto* gn = GetNeighbors::make(qctx, startVidPlan.root, goCtx_->space.id); - gn->setVertexProps(buildVertexProps(goCtx_->exprProps.srcTagProps())); - gn->setEdgeProps(buildEdgeProps(false)); - gn->setSrc(goCtx_->from.src); - gn->setInputVar(goCtx_->vidsVar); - gn->setEdgeDirection(goCtx_->over.direction); - scan = gn; - - auto* sampleLimit = buildSampleLimit(gn, 1 /* one step */); - cur = sampleLimit; - cur = buildOneStepJoinPlan(sampleLimit); - - if (goCtx_->filter != nullptr) { - cur = Filter::make(qctx, cur, goCtx_->filter); - } - - cur = Project::make(qctx, cur, goCtx_->yieldExpr); - cur->setColNames(std::move(goCtx_->colNames)); - if (goCtx_->distinct) { - cur = Dedup::make(qctx, cur); - } + expandAll->setStepLimits(goCtx_->limits); + PlanNode* dep = expandAll; + if (goCtx_->joinDst) { + dep = buildJoinDstPlan(expandAll); } - SubPlan subPlan; - subPlan.root = cur; - subPlan.tail = startVidPlan.tail != nullptr ? startVidPlan.tail : scan; - return subPlan; -} - -SubPlan GoPlanner::nStepsPlan(SubPlan& startVidPlan) { - auto qctx = goCtx_->qctx; - loopStepVar_ = qctx->vctx()->anonVarGen()->getVar(); - - auto* start = StartNode::make(qctx); - PlanNode* getDst = nullptr; - - PlanNode* loopBody = nullptr; - PlanNode* loopDep = startVidPlan.root; - if (!goCtx_->joinInput && goCtx_->limits.empty()) { - auto* gd = GetDstBySrc::make(qctx, start, goCtx_->space.id); - gd->setSrc(goCtx_->from.src); - gd->setEdgeTypes(buildEdgeTypes()); - gd->setInputVar(goCtx_->vidsVar); - gd->setColNames({goCtx_->dstIdColName}); - auto* dedup = Dedup::make(qctx, gd); - dedup->setOutputVar(goCtx_->vidsVar); - dedup->setColNames(gd->colNames()); - getDst = dedup; - - loopBody = getDst; - } else { - auto* gn = GetNeighbors::make(qctx, start, goCtx_->space.id); - gn->setSrc(goCtx_->from.src); - gn->setEdgeProps(buildEdgeProps(true)); - gn->setInputVar(goCtx_->vidsVar); - gn->setEdgeDirection(goCtx_->over.direction); - auto* sampleLimit = buildSampleLimit(gn); - - getDst = PlannerUtil::extractDstFromGN(qctx, sampleLimit, goCtx_->vidsVar); - - loopBody = getDst; - if (goCtx_->joinInput) { - auto* joinLeft = extractVidFromRuntimeInput(startVidPlan.root); - auto* joinRight = extractSrcDstFromGN(getDst, sampleLimit->outputVar()); - loopBody = trackStartVid(joinLeft, joinRight); - loopDep = joinLeft; - } + if (goCtx_->joinInput) { + // the result of the previous statement InnerJoin the result of the current statement + auto* hashKey = from.originalSrc; + auto* probeKey = VariablePropertyExpression::make(pool, dep->outputVar(), "_expand_vid"); + auto* join = HashInnerJoin::make(qctx, preRootNode_, dep, {hashKey}, {probeKey}); + + auto& varName = from.fromType == kPipe ? goCtx_->inputVarName : from.userDefinedVarName; + auto* varPtr = qctx->symTable()->getVar(varName); + DCHECK(varPtr != nullptr); + std::vector colNames = varPtr->colNames; + colNames.insert(colNames.end(), dep->colNames().begin(), dep->colNames().end()); + join->setColNames(std::move(colNames)); + dep = join; } - auto* condition = loopCondition(goCtx_->steps.steps() - 1, getDst->outputVar()); - auto* loop = Loop::make(qctx, loopDep, loopBody, condition); - - auto* root = lastStep(loop, loopBody == getDst ? nullptr : loopBody); - SubPlan subPlan; - subPlan.root = root; - subPlan.tail = startVidPlan.tail == nullptr ? loop : startVidPlan.tail; - - return subPlan; -} - -SubPlan GoPlanner::mToNStepsPlan(SubPlan& startVidPlan) { - auto qctx = goCtx_->qctx; - auto joinInput = goCtx_->joinInput; - auto joinDst = goCtx_->joinDst; - auto isSimple = goCtx_->isSimple; - - loopStepVar_ = qctx->vctx()->anonVarGen()->getVar(); - - auto* start = StartNode::make(qctx); - - PlanNode* getDst = nullptr; - - PlanNode* loopBody = nullptr; - PlanNode* loopDep = startVidPlan.root; - if (isSimple) { - auto* gd = GetDstBySrc::make(qctx, start, goCtx_->space.id); - gd->setSrc(goCtx_->from.src); - gd->setEdgeTypes(buildEdgeTypes()); - gd->setInputVar(goCtx_->vidsVar); - gd->setColNames({goCtx_->dstIdColName}); - auto* dedup = Dedup::make(qctx, gd); - // The outputVar of `Dedup` is the same as the inputVar of `GetDstBySrc`. - // So the output of `Dedup` of current iteration feeds into the input of `GetDstBySrc` of next - // iteration. - dedup->setOutputVar(goCtx_->vidsVar); - dedup->setColNames(gd->colNames()); - getDst = dedup; - loopBody = getDst; - - if (joinDst) { - loopBody = extractDstId(loopBody); - // Left join, join the dst id with `GetVertices` - loopBody = buildJoinDstPlan(loopBody); - } - if (goCtx_->filter) { - loopBody = Filter::make(qctx, loopBody, goCtx_->filter); - } - if (joinDst || goCtx_->yieldExpr->columns().size() != 1) { - loopBody = Project::make(qctx, loopBody, goCtx_->yieldExpr); - } else { - gd->setColNames(goCtx_->colNames); - dedup->setColNames(goCtx_->colNames); - } - loopBody->setColNames(goCtx_->colNames); - } else { - auto* gn = GetNeighbors::make(qctx, start, goCtx_->space.id); - gn->setSrc(goCtx_->from.src); - gn->setVertexProps(buildVertexProps(goCtx_->exprProps.srcTagProps())); - gn->setEdgeProps(buildEdgeProps(false)); - gn->setInputVar(goCtx_->vidsVar); - gn->setEdgeDirection(goCtx_->over.direction); - auto* sampleLimit = buildSampleLimit(gn); - - getDst = PlannerUtil::extractDstFromGN(qctx, sampleLimit, goCtx_->vidsVar); - - loopBody = getDst; - PlanNode* trackVid = nullptr; - if (joinInput) { - auto* joinLeft = extractVidFromRuntimeInput(startVidPlan.root); - auto* joinRight = extractSrcDstFromGN(getDst, sampleLimit->outputVar()); - trackVid = trackStartVid(joinLeft, joinRight); - loopBody = trackVid; - loopDep = joinLeft; - } - - if (joinInput || joinDst) { - loopBody = extractSrcEdgePropsFromGN(loopBody, sampleLimit->outputVar()); - loopBody = joinDst ? buildJoinDstPlan(loopBody) : loopBody; - loopBody = joinInput ? lastStepJoinInput(trackVid, loopBody) : loopBody; - loopBody = joinInput ? buildJoinInputPlan(loopBody) : loopBody; - } - - if (goCtx_->filter) { - const auto& filterInput = - (joinInput || joinDst) ? loopBody->outputVar() : sampleLimit->outputVar(); - loopBody = Filter::make(qctx, loopBody, goCtx_->filter); - loopBody->setInputVar(filterInput); - } - - const auto& projectInput = - (loopBody != getDst) ? loopBody->outputVar() : sampleLimit->outputVar(); - loopBody = Project::make(qctx, loopBody, goCtx_->yieldExpr); - loopBody->setInputVar(projectInput); - - if (goCtx_->distinct) { - loopBody = Dedup::make(qctx, loopBody); - } - loopBody->setColNames(goCtx_->colNames); + if (goCtx_->filter != nullptr) { + dep = Filter::make(qctx, dep, goCtx_->filter); } - auto* condition = loopCondition(goCtx_->steps.nSteps(), getDst->outputVar()); - auto* loop = Loop::make(qctx, loopDep, loopBody, condition); + dep = Project::make(qctx, dep, goCtx_->yieldExpr); + dep->setColNames(std::move(goCtx_->colNames)); - auto* dc = DataCollect::make(qctx, DataCollect::DCKind::kMToN); - dc->addDep(loop); - if (loopBody == getDst) { - StepClause newStep(goCtx_->steps.mSteps() + 1, goCtx_->steps.nSteps() + 1); - dc->setMToN(newStep); - } else { - dc->setMToN(goCtx_->steps); + if (goCtx_->distinct) { + dep = Dedup::make(qctx, dep); } - dc->setInputVars({loopBody->outputVar()}); - - dc->setDistinct(goCtx_->distinct); - dc->setColNames(goCtx_->colNames); SubPlan subPlan; - subPlan.root = dc; - subPlan.tail = startVidPlan.tail == nullptr ? loop : startVidPlan.tail; - + subPlan.root = dep; + subPlan.tail = startNode_; return subPlan; } StatusOr GoPlanner::transform(AstContext* astCtx) { goCtx_ = static_cast(astCtx); auto qctx = goCtx_->qctx; - goCtx_->joinInput = goCtx_->from.fromType != FromType::kInstantExpr && !goCtx_->isSimple; + goCtx_->joinInput = goCtx_->from.fromType != FromType::kInstantExpr; goCtx_->joinDst = !goCtx_->exprProps.dstTagProps().empty(); - SubPlan startPlan = PlannerUtil::buildStart(qctx, goCtx_->from, goCtx_->vidsVar); + startNode_ = StartNode::make(qctx); + auto& from = goCtx_->from; + if (!from.vids.empty() && from.originalSrc == nullptr) { + PlannerUtil::buildConstantInput(qctx, from, goCtx_->vidsVar); + } else { + // get root node of the previous statement + auto& varName = from.fromType == kVariable ? from.userDefinedVarName : goCtx_->inputVarName; + auto* varPtr = qctx->symTable()->getVar(varName); + DCHECK(varPtr != nullptr); + DCHECK_EQ(varPtr->writtenBy.size(), 1); + for (auto node : varPtr->writtenBy) { + preRootNode_ = node; + } - auto& steps = goCtx_->steps; - if (steps.isMToN()) { - return mToNStepsPlan(startPlan); + auto argNode = Argument::make(qctx, from.runtimeVidName); + argNode->setColNames({from.runtimeVidName}); + goCtx_->vidsVar = argNode->outputVar(); + startNode_ = argNode; } - - if (steps.steps() == 0) { + auto& steps = goCtx_->steps; + if (!steps.isMToN() && steps.steps() == 0) { auto* pt = PassThroughNode::make(qctx, nullptr); pt->setColNames(std::move(goCtx_->colNames)); SubPlan subPlan; subPlan.root = subPlan.tail = pt; return subPlan; } - - if (steps.steps() == 1) { - return oneStepPlan(startPlan); - } - return nStepsPlan(startPlan); -} - -std::vector GoPlanner::buildEdgeTypes() { - switch (goCtx_->over.direction) { - case storage::cpp2::EdgeDirection::IN_EDGE: { - std::vector edgeTypes; - edgeTypes.reserve(goCtx_->over.edgeTypes.size()); - for (auto edgeType : goCtx_->over.edgeTypes) { - edgeTypes.emplace_back(-edgeType); - } - return edgeTypes; - } - case storage::cpp2::EdgeDirection::OUT_EDGE: { - return goCtx_->over.edgeTypes; - } - case storage::cpp2::EdgeDirection::BOTH: { - std::vector edgeTypes; - for (auto edgeType : goCtx_->over.edgeTypes) { - edgeTypes.emplace_back(edgeType); - edgeTypes.emplace_back(-edgeType); - } - return edgeTypes; - } - } - return {}; -} - -PlanNode* GoPlanner::extractDstId(PlanNode* node) { - auto pool = goCtx_->qctx->objPool(); - auto* columns = pool->makeAndAdd(); - auto* column = new YieldColumn(ColumnExpression::make(pool, 0)); - columns->addColumn(column); - auto* project = Project::make(goCtx_->qctx, node, columns); - project->setColNames({goCtx_->dstIdColName}); - return project; + return doPlan(); } } // namespace graph diff --git a/src/graph/planner/ngql/GoPlanner.h b/src/graph/planner/ngql/GoPlanner.h index 563ef542514..9f620778fb1 100644 --- a/src/graph/planner/ngql/GoPlanner.h +++ b/src/graph/planner/ngql/GoPlanner.h @@ -32,11 +32,7 @@ class GoPlanner final : public Planner { StatusOr transform(AstContext* astCtx) override; private: - SubPlan oneStepPlan(SubPlan& startVidPlan); - - SubPlan nStepsPlan(SubPlan& startVidPlan); - - SubPlan mToNStepsPlan(SubPlan& startVidPlan); + SubPlan doPlan(); private: std::unique_ptr buildVertexProps(const ExpressionProps::TagIDPropsMap& propsMap); @@ -45,61 +41,20 @@ class GoPlanner final : public Planner { void doBuildEdgeProps(std::unique_ptr& edgeProps, bool onlyDst, bool isInEdge); - Expression* loopCondition(uint32_t steps, const std::string& gnVar); - - PlanNode* extractSrcEdgePropsFromGN(PlanNode* dep, const std::string& input); - - PlanNode* extractSrcDstFromGN(PlanNode* dep, const std::string& input); - - PlanNode* extractVidFromRuntimeInput(PlanNode* dep); - - PlanNode* trackStartVid(PlanNode* left, PlanNode* right); - PlanNode* buildJoinDstPlan(PlanNode* dep); - PlanNode* buildJoinInputPlan(PlanNode* dep); - - PlanNode* lastStepJoinInput(PlanNode* left, PlanNode* right); - - PlanNode* buildLastStepJoinPlan(PlanNode* gn, PlanNode* join); - - PlanNode* lastStep(PlanNode* dep, PlanNode* join); - - PlanNode* buildOneStepJoinPlan(PlanNode* gn); - - template - PlanNode* buildSampleLimitImpl(PlanNode* input, T sampleLimit); - // build step sample limit plan - PlanNode* buildSampleLimit(PlanNode* input, std::size_t currentStep) { - if (goCtx_->limits.empty()) { - // No sample/limit - return input; - } - return buildSampleLimitImpl(input, goCtx_->limits[currentStep - 1]); - } - // build step sample in loop - PlanNode* buildSampleLimit(PlanNode* input) { - if (goCtx_->limits.empty()) { - // No sample/limit - return input; - } - return buildSampleLimitImpl(input, stepSampleLimit()); - } - - // Get step sample/limit number - Expression* stepSampleLimit(); std::vector buildEdgeTypes(); - PlanNode* extractDstId(PlanNode* node); private: GoPlanner() = default; GoContext* goCtx_{nullptr}; + // runtime : argument, else : startNode + PlanNode* startNode_{nullptr}; + PlanNode* preRootNode_{nullptr}; const int16_t VID_INDEX = 0; const int16_t LAST_COL_INDEX = -1; - - std::string loopStepVar_; }; } // namespace graph } // namespace nebula diff --git a/src/graph/planner/plan/PlanNode.cpp b/src/graph/planner/plan/PlanNode.cpp index 11d4b45a03c..d73e93af514 100644 --- a/src/graph/planner/plan/PlanNode.cpp +++ b/src/graph/planner/plan/PlanNode.cpp @@ -38,6 +38,10 @@ const char* PlanNode::toString(PlanNode::Kind kind) { return "Start"; case Kind::kGetNeighbors: return "GetNeighbors"; + case Kind::kExpand: + return "Expand"; + case Kind::kExpandAll: + return "ExpandAll"; case Kind::kGetVertices: return "GetVertices"; case Kind::kGetEdges: @@ -303,8 +307,6 @@ const char* PlanNode::toString(PlanNode::Kind kind) { return "RollUpApply"; case Kind::kPatternApply: return "PatternApply"; - case Kind::kGetDstBySrc: - return "GetDstBySrc"; // no default so the compiler will warning when lack } DLOG(FATAL) << "Impossible kind plan node " << static_cast(kind); diff --git a/src/graph/planner/plan/PlanNode.h b/src/graph/planner/plan/PlanNode.h index 9cc46ba4981..cc6562964cf 100644 --- a/src/graph/planner/plan/PlanNode.h +++ b/src/graph/planner/plan/PlanNode.h @@ -27,7 +27,8 @@ class PlanNode { kGetNeighbors, kGetVertices, kGetEdges, - kGetDstBySrc, + kExpand, + kExpandAll, kTraverse, kAppendVertices, kShortestPath, diff --git a/src/graph/planner/plan/Query.cpp b/src/graph/planner/plan/Query.cpp index 63b704292ae..8d50e1f09a4 100644 --- a/src/graph/planner/plan/Query.cpp +++ b/src/graph/planner/plan/Query.cpp @@ -102,24 +102,91 @@ void GetNeighbors::cloneMembers(const GetNeighbors& g) { } } -std::unique_ptr GetDstBySrc::explain() const { +std::unique_ptr Expand::explain() const { auto desc = Explore::explain(); - addDescription("src", src_ ? src_->toString() : "", desc.get()); - addDescription("edgeTypes", folly::toJson(util::toJson(edgeTypes_)), desc.get()); + addDescription("sample", folly::toJson(util::toJson(sample_)), desc.get()); + addDescription("joinInput", folly::toJson(util::toJson(joinInput_)), desc.get()); + addDescription("maxSteps", folly::to(maxSteps_), desc.get()); + addDescription( + "edgeProps", edgeProps_ ? folly::toJson(util::toJson(*edgeProps_)) : "", desc.get()); + auto limits = folly::dynamic::array(); + for (auto i : stepLimits_) { + limits.push_back(folly::to(i)); + } + addDescription("stepLimits", folly::toJson(limits), desc.get()); return desc; } -PlanNode* GetDstBySrc::clone() const { - auto* newGV = GetDstBySrc::make(qctx_, nullptr, space_); - newGV->cloneMembers(*this); - return newGV; +PlanNode* Expand::clone() const { + auto* expand = Expand::make(qctx_, nullptr, space_); + expand->cloneMembers(*this); + return expand; } -void GetDstBySrc::cloneMembers(const GetDstBySrc& gd) { - Explore::cloneMembers(gd); +void Expand::cloneMembers(const Expand& expand) { + Explore::cloneMembers(expand); + sample_ = expand.sample(); + maxSteps_ = expand.maxSteps(); + if (expand.edgeProps()) { + auto edgeProps = *expand.edgeProps_; + auto edgePropsPtr = std::make_unique(std::move(edgeProps)); + setEdgeProps(std::move(edgePropsPtr)); + } + stepLimits_ = expand.stepLimits(); + joinInput_ = expand.joinInput(); + edgeTypes_ = expand.edgeTypes(); +} - src_ = gd.src()->clone(); - edgeTypes_ = gd.edgeTypes_; +std::unique_ptr ExpandAll::explain() const { + auto desc = Expand::explain(); + addDescription("minSteps", folly::to(minSteps_), desc.get()); + addDescription( + "vertexProps", vertexProps_ ? folly::toJson(util::toJson(*vertexProps_)) : "", desc.get()); + auto vertexColumns = folly::dynamic::array(); + if (vertexColumns_) { + for (const auto* col : vertexColumns_->columns()) { + DCHECK(col != nullptr); + vertexColumns.push_back(col->toString()); + } + addDescription("vertexColumns", folly::toJson(vertexColumns), desc.get()); + } + auto edgeColumns = folly::dynamic::array(); + if (edgeColumns_) { + for (const auto* col : edgeColumns_->columns()) { + DCHECK(col != nullptr); + edgeColumns.push_back(col->toString()); + } + addDescription("edgeColumns", folly::toJson(edgeColumns), desc.get()); + } + return desc; +} + +PlanNode* ExpandAll::clone() const { + auto* expandAll = ExpandAll::make(qctx_, nullptr, space_); + expandAll->cloneMembers(*this); + return expandAll; +} + +void ExpandAll::cloneMembers(const ExpandAll& expandAll) { + Expand::cloneMembers(expandAll); + minSteps_ = expandAll.minSteps(); + if (expandAll.vertexProps()) { + auto vertexProps = *expandAll.vertexProps_; + auto vertexPropsPtr = std::make_unique(vertexProps); + setVertexProps(std::move(vertexPropsPtr)); + } + if (expandAll.vertexColumns()) { + vertexColumns_ = qctx_->objPool()->makeAndAdd(); + for (const auto& col : expandAll.vertexColumns()->columns()) { + vertexColumns_->addColumn(col->clone().release()); + } + } + if (expandAll.edgeColumns()) { + edgeColumns_ = qctx_->objPool()->makeAndAdd(); + for (const auto& col : expandAll.edgeColumns()->columns()) { + edgeColumns_->addColumn(col->clone().release()); + } + } } std::unique_ptr GetVertices::explain() const { @@ -608,10 +675,6 @@ std::unique_ptr DataCollect::explain() const { addDescription("kind", "ROW", desc.get()); break; } - case DCKind::kMToN: { - addDescription("kind", "M TO N", desc.get()); - break; - } case DCKind::kBFSShortest: { addDescription("kind", "BFS SHORTEST", desc.get()); break; @@ -640,7 +703,6 @@ PlanNode* DataCollect::clone() const { void DataCollect::cloneMembers(const DataCollect& l) { VariableDependencyNode::cloneMembers(l); - step_ = l.step(); distinct_ = l.distinct(); } diff --git a/src/graph/planner/plan/Query.h b/src/graph/planner/plan/Query.h index df403db7f17..60f8dadf859 100644 --- a/src/graph/planner/plan/Query.h +++ b/src/graph/planner/plan/Query.h @@ -259,24 +259,36 @@ class GetNeighbors : public Explore { bool random_{false}; }; -// Get Edge dst id by src id -class GetDstBySrc : public Explore { +class Expand : public Explore { public: - static GetDstBySrc* make(QueryContext* qctx, - PlanNode* input, - GraphSpaceID space, - Expression* src = nullptr, - std::vector edgeTypes = {}) { - return qctx->objPool()->makeAndAdd( - qctx, Kind::kGetDstBySrc, input, space, src, std::move(edgeTypes)); + static Expand* make(QueryContext* qctx, + PlanNode* input, + GraphSpaceID space, + bool sample = false, + size_t maxSteps = 0, + std::unique_ptr>&& edgeProps = nullptr) { + return qctx->objPool()->makeAndAdd( + qctx, Kind::kExpand, input, space, sample, maxSteps, std::move(edgeProps)); } - Expression* src() const { - return src_; + bool sample() const { + return sample_; } - void setSrc(Expression* src) { - src_ = src; + std::vector stepLimits() const { + return stepLimits_; + } + + void setStepLimits(std::vector& limits) { + stepLimits_ = limits; + } + + const std::vector* edgeProps() const { + return edgeProps_.get(); + } + + void setEdgeProps(std::unique_ptr> edgeProps) { + edgeProps_ = std::move(edgeProps); } const std::vector& edgeTypes() const { @@ -287,25 +299,135 @@ class GetDstBySrc : public Explore { edgeTypes_ = std::move(edgeTypes); } + bool joinInput() const { + return joinInput_; + } + + void setJoinInput(bool joinInput) { + joinInput_ = joinInput; + } + + size_t maxSteps() const { + return maxSteps_; + } + + std::unique_ptr explain() const override; + PlanNode* clone() const override; + + protected: + friend ObjectPool; + Expand(QueryContext* qctx, + Kind kind, + PlanNode* input, + GraphSpaceID space, + bool sample, + size_t maxSteps, + std::unique_ptr>&& edgeProps) + : Explore(qctx, kind, input, space), + sample_(sample), + maxSteps_(maxSteps), + edgeProps_(std::move(edgeProps)) {} + + void cloneMembers(const Expand&); + + protected: + bool sample_{false}; + size_t maxSteps_{0}; + std::unique_ptr> edgeProps_{nullptr}; + std::vector stepLimits_; + bool joinInput_{false}; + std::vector edgeTypes_; +}; + +class ExpandAll : public Expand { + public: + static ExpandAll* make(QueryContext* qctx, + PlanNode* input, + GraphSpaceID space, + bool sample = false, + size_t minSteps = 0, + size_t maxSteps = 0, + std::unique_ptr>&& edgeProps = nullptr, + std::unique_ptr>&& vertexProps = nullptr, + YieldColumns* vertexColumns = nullptr, + YieldColumns* edgeColumns = nullptr) { + return qctx->objPool()->makeAndAdd(qctx, + Kind::kExpandAll, + input, + space, + sample, + minSteps, + maxSteps, + std::move(edgeProps), + std::move(vertexProps), + vertexColumns, + edgeColumns); + } + + const std::vector* vertexProps() const { + return vertexProps_.get(); + } + + size_t minSteps() const { + return minSteps_; + } + + YieldColumns* vertexColumns() const { + return vertexColumns_; + } + + YieldColumns* edgeColumns() const { + return edgeColumns_; + } + + void setVertexProps(std::unique_ptr> vertexProps) { + vertexProps_ = std::move(vertexProps); + } + std::unique_ptr explain() const override; + PlanNode* clone() const override; + protected: friend ObjectPool; - GetDstBySrc(QueryContext* qctx, - Kind kind, - PlanNode* input, - GraphSpaceID space, - Expression* src, - std::vector edgeTypes) - : Explore(qctx, kind, input, space), src_(src), edgeTypes_(std::move(edgeTypes)) {} + ExpandAll(QueryContext* qctx, + Kind kind, + PlanNode* input, + GraphSpaceID space, + bool sample, + size_t minSteps, + size_t maxSteps, + std::unique_ptr>&& edgeProps, + std::unique_ptr>&& vertexProps, + YieldColumns* vertexColumns, + YieldColumns* edgeColumns) + : Expand(qctx, kind, input, space, sample, maxSteps, std::move(edgeProps)), + minSteps_(minSteps), + vertexProps_(std::move(vertexProps)), + vertexColumns_(vertexColumns), + edgeColumns_(edgeColumns) { + std::vector colNames; + if (vertexColumns_ && edgeColumns_) { + colNames = vertexColumns_->names(); + auto edgeColNames = edgeColumns_->names(); + colNames.insert(colNames.end(), edgeColNames.begin(), edgeColNames.end()); + } else if (vertexColumns_) { + colNames = vertexColumns_->names(); + } else if (edgeColumns_) { + colNames = edgeColumns_->names(); + } + setLimit(-1); + setColNames(colNames); + } - void cloneMembers(const GetDstBySrc&); + void cloneMembers(const ExpandAll&); private: - // vertices may be parsing from runtime. - Expression* src_{nullptr}; - std::vector edgeTypes_; + size_t minSteps_{0}; + std::unique_ptr> vertexProps_{nullptr}; + YieldColumns* vertexColumns_{nullptr}; + YieldColumns* edgeColumns_{nullptr}; }; // Get property with given vertex keys. @@ -1274,7 +1396,6 @@ class DataCollect final : public VariableDependencyNode { enum class DCKind : uint8_t { kSubgraph, kRowBasedMove, - kMToN, kBFSShortest, kAllPaths, kMultiplePairShortest, @@ -1285,10 +1406,6 @@ class DataCollect final : public VariableDependencyNode { return qctx->objPool()->makeAndAdd(qctx, kind); } - void setMToN(StepClause step) { - step_ = std::move(step); - } - void setDistinct(bool distinct) { distinct_ = distinct; } @@ -1311,10 +1428,6 @@ class DataCollect final : public VariableDependencyNode { return vars; } - StepClause step() const { - return step_; - } - bool distinct() const { return distinct_; } @@ -1340,8 +1453,6 @@ class DataCollect final : public VariableDependencyNode { private: DCKind kind_; - // using for m to n steps - StepClause step_; std::vector colType_; bool distinct_{false}; }; diff --git a/src/graph/util/ExpressionUtils.cpp b/src/graph/util/ExpressionUtils.cpp index b12b1af47de..c768e21cae8 100644 --- a/src/graph/util/ExpressionUtils.cpp +++ b/src/graph/util/ExpressionUtils.cpp @@ -951,16 +951,23 @@ Expression *ExpressionUtils::flattenInnerLogicalExpr(const Expression *expr) { return allFlattenExpr; } -bool ExpressionUtils::checkVarPropIfExist(const std::vector &columns, - const Expression *e) { - auto varProps = graph::ExpressionUtils::collectAll(e, {Expression::Kind::kVarProperty}); - if (varProps.empty()) { +bool ExpressionUtils::checkColName(const std::vector &columns, const Expression *e) { + auto exprs = graph::ExpressionUtils::collectAll( + e, {Expression::Kind::kVarProperty, Expression::Kind::kVertex, Expression::Kind::kEdge}); + if (exprs.empty()) { return false; } - for (const auto *expr : varProps) { - DCHECK_EQ(expr->kind(), Expression::Kind::kVarProperty); - auto iter = std::find_if(columns.begin(), columns.end(), [expr](const std::string &item) { - return !item.compare(static_cast(expr)->prop()); + for (const auto *expr : exprs) { + std::string colName; + if (expr->kind() == Expression::Kind::kVarProperty) { + colName = static_cast(expr)->prop(); + } else if (expr->kind() == Expression::Kind::kVertex) { + colName = static_cast(expr)->name(); + } else { + colName = static_cast(expr)->toString(); + } + auto iter = std::find_if(columns.begin(), columns.end(), [&colName](const std::string &item) { + return !item.compare(colName); }); if (iter == columns.end()) { return false; diff --git a/src/graph/util/ExpressionUtils.h b/src/graph/util/ExpressionUtils.h index f5452a8be24..5139cfc22a0 100644 --- a/src/graph/util/ExpressionUtils.h +++ b/src/graph/util/ExpressionUtils.h @@ -195,8 +195,8 @@ class ExpressionUtils { // calls flattenInnerLogicalAndExpr() first then executes flattenInnerLogicalOrExpr() static Expression* flattenInnerLogicalExpr(const Expression* expr); - // Check whether there exists the property of variable expression in `columns' - static bool checkVarPropIfExist(const std::vector& columns, const Expression* e); + // Check whether there exists the colName in `columns' + static bool checkColName(const std::vector& columns, const Expression* e); // Uses the picker to split the given expression expr into two parts: filterPicked and // filterUnpicked If expr is a non-LogicalAnd expression, applies the picker to expr directly If diff --git a/src/graph/validator/GoValidator.cpp b/src/graph/validator/GoValidator.cpp index 065db1ab2f8..05e05b8d070 100644 --- a/src/graph/validator/GoValidator.cpp +++ b/src/graph/validator/GoValidator.cpp @@ -57,21 +57,7 @@ Status GoValidator::validateImpl() { return Status::SemanticError("Only support single input in a go sentence."); } - goCtx_->isSimple = isSimpleCase(); - if (goCtx_->isSimple) { - // Need to unify all EdgeDstIdExpr to *._dst. - // eg. serve._dst will be unified to *._dst - rewrite2EdgeDst(); - } NG_RETURN_IF_ERROR(buildColumns()); - if (goCtx_->isSimple) { - auto iter = propExprColMap_.find("*._dst"); - if (iter != propExprColMap_.end()) { - goCtx_->dstIdColName = iter->second->alias(); - } - // Rewrite *._dst/serve._dst to $dstIdColName - rewriteEdgeDst2VarProp(); - } return Status::OK(); } @@ -191,7 +177,8 @@ Status GoValidator::extractTagIds() { Status GoValidator::extractPropExprs(const Expression* expr, std::unordered_set& uniqueExpr) { ExtractPropExprVisitor visitor(vctx_, - goCtx_->srcEdgePropsExpr, + goCtx_->srcPropsExpr, + goCtx_->edgePropsExpr, goCtx_->dstPropsExpr, inputPropCols_, propExprColMap_, @@ -251,29 +238,11 @@ Expression* GoValidator::rewrite2VarProp(const Expression* expr) { // collect the input properties used in the query, // rewrites output expression to Input/Variable expression to get properties from previous plan node Status GoValidator::buildColumns() { - const auto& exprProps = goCtx_->exprProps; - const auto& dstTagProps = exprProps.dstTagProps(); - const auto& inputProps = exprProps.inputProps(); - const auto& varProps = exprProps.varProps(); - const auto& from = goCtx_->from; auto pool = qctx_->objPool(); - - if (dstTagProps.empty() && inputProps.empty() && varProps.empty() && - from.fromType == FromType::kInstantExpr) { - return Status::OK(); - } - - if (!exprProps.isAllPropsEmpty() || from.fromType != FromType::kInstantExpr) { - goCtx_->srcEdgePropsExpr = pool->makeAndAdd(); - } - - if (!dstTagProps.empty()) { - goCtx_->dstPropsExpr = pool->makeAndAdd(); - } - - if (!inputProps.empty() || !varProps.empty()) { - inputPropCols_ = pool->makeAndAdd(); - } + goCtx_->srcPropsExpr = pool->makeAndAdd(); + goCtx_->edgePropsExpr = pool->makeAndAdd(); + goCtx_->dstPropsExpr = pool->makeAndAdd(); + inputPropCols_ = pool->makeAndAdd(); std::unordered_set uniqueEdgeVertexExpr; auto filter = goCtx_->filter; @@ -292,88 +261,5 @@ Status GoValidator::buildColumns() { return Status::OK(); } -bool GoValidator::isSimpleCase() { - // Check limit clause - if (!goCtx_->limits.empty()) { - return false; - } - // Check if the filter or yield cluase uses: - // 1. src tag props, - // 2. or edge props, except the dst id of edge. - // 3. input or var props. - auto& exprProps = goCtx_->exprProps; - if (!exprProps.srcTagProps().empty()) return false; - if (!exprProps.edgeProps().empty()) { - for (auto& edgeProp : exprProps.edgeProps()) { - auto props = edgeProp.second; - if (props.size() != 1) return false; - if (props.find(kDst) == props.end()) return false; - } - } - if (exprProps.hasInputVarProperty()) return false; - - // Check filter clause - // Because GetDstBySrc doesn't support filter push down, - // so we don't optimize such case. - if (goCtx_->filter) { - if (ExpressionUtils::findEdgeDstExpr(goCtx_->filter)) { - return false; - } - } - - // Check yield clause - if (!goCtx_->distinct) return false; - bool atLeastOneDstId = false; - for (auto& col : goCtx_->yieldExpr->columns()) { - auto expr = col->expr(); - if (expr->kind() != Expression::Kind::kEdgeDst) continue; - atLeastOneDstId = true; - auto dstIdExpr = static_cast(expr); - if (dstIdExpr->sym() != "*" && goCtx_->over.edgeTypes.size() != 1) { - return false; - } - } - return atLeastOneDstId; -} - -void GoValidator::rewrite2EdgeDst() { - auto matcher = [](const Expression* e) -> bool { - if (e->kind() != Expression::Kind::kEdgeDst) { - return false; - } - auto* edgeDstExpr = static_cast(e); - return edgeDstExpr->sym() != "*"; - }; - auto rewriter = [this](const Expression*) -> Expression* { - return EdgeDstIdExpression::make(qctx_->objPool(), "*"); - }; - - if (goCtx_->filter != nullptr) { - goCtx_->filter = RewriteVisitor::transform(goCtx_->filter, matcher, rewriter); - } - auto* newYieldExpr = qctx_->objPool()->makeAndAdd(); - for (auto* col : goCtx_->yieldExpr->columns()) { - newYieldExpr->addColumn( - new YieldColumn(RewriteVisitor::transform(col->expr(), matcher, rewriter), col->alias())); - } - goCtx_->yieldExpr = newYieldExpr; -} - -void GoValidator::rewriteEdgeDst2VarProp() { - auto matcher = [](const Expression* expr) { return expr->kind() == Expression::Kind::kEdgeDst; }; - auto rewriter = [this](const Expression*) { - return VariablePropertyExpression::make(qctx_->objPool(), "", goCtx_->dstIdColName); - }; - if (goCtx_->filter != nullptr) { - goCtx_->filter = RewriteVisitor::transform(goCtx_->filter, matcher, rewriter); - } - auto* newYieldExpr = qctx_->objPool()->makeAndAdd(); - for (auto* col : goCtx_->yieldExpr->columns()) { - newYieldExpr->addColumn( - new YieldColumn(RewriteVisitor::transform(col->expr(), matcher, rewriter), col->alias())); - } - goCtx_->yieldExpr = newYieldExpr; -} - } // namespace graph } // namespace nebula diff --git a/src/graph/validator/GoValidator.h b/src/graph/validator/GoValidator.h index 0f5293f7021..5b9ed5b2e90 100644 --- a/src/graph/validator/GoValidator.h +++ b/src/graph/validator/GoValidator.h @@ -42,10 +42,6 @@ class GoValidator final : public Validator { Expression* rewriteVertexEdge2EdgeProp(const Expression* expr); - bool isSimpleCase(); - void rewrite2EdgeDst(); - void rewriteEdgeDst2VarProp(); - private: std::unique_ptr goCtx_; diff --git a/src/graph/validator/PipeValidator.cpp b/src/graph/validator/PipeValidator.cpp index 9b549ac08a0..ff17be194d8 100644 --- a/src/graph/validator/PipeValidator.cpp +++ b/src/graph/validator/PipeValidator.cpp @@ -31,6 +31,10 @@ Status PipeValidator::validateImpl() { Status PipeValidator::toPlan() { root_ = rValidator_->root(); tail_ = lValidator_->tail(); + PlanNode* rTail = rValidator_->tail(); + if (rTail->kind() == PlanNode::Kind::kArgument || rTail->kind() == PlanNode::Kind::kStart) { + return Status::OK(); + } NG_RETURN_IF_ERROR(rValidator_->appendPlan(lValidator_->root())); auto node = static_cast(rValidator_->tail()); if (node->inputVar().empty()) { diff --git a/src/graph/validator/Validator.cpp b/src/graph/validator/Validator.cpp index 18b4732273a..d76d350c851 100644 --- a/src/graph/validator/Validator.cpp +++ b/src/graph/validator/Validator.cpp @@ -308,6 +308,9 @@ std::vector Validator::getOutColNames() const { Status Validator::appendPlan(PlanNode* node, PlanNode* appended) { DCHECK(node != nullptr); DCHECK(appended != nullptr); + if (node->kind() == PlanNode::Kind::kArgument) { + return Status::OK(); + } if (!node->isSingleInput()) { return Status::SemanticError("PlanNode(%s) not support to append an input.", diff --git a/src/graph/validator/test/FindPathValidatorTest.cpp b/src/graph/validator/test/FindPathValidatorTest.cpp index d803c4eb8b8..734bf6edfa8 100644 --- a/src/graph/validator/test/FindPathValidatorTest.cpp +++ b/src/graph/validator/test/FindPathValidatorTest.cpp @@ -181,7 +181,8 @@ TEST_F(FindPathValidatorTest, RunTimePath) { PK::kDedup, PK::kProject, PK::kProject, - PK::kGetNeighbors, + PK::kExpandAll, + PK::kExpand, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -205,7 +206,8 @@ TEST_F(FindPathValidatorTest, RunTimePath) { PK::kDedup, PK::kProject, PK::kProject, - PK::kGetNeighbors, + PK::kExpandAll, + PK::kExpand, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -228,7 +230,8 @@ TEST_F(FindPathValidatorTest, RunTimePath) { PK::kProject, PK::kStart, PK::kProject, - PK::kGetNeighbors, + PK::kExpandAll, + PK::kExpand, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -251,7 +254,8 @@ TEST_F(FindPathValidatorTest, RunTimePath) { PK::kProject, PK::kStart, PK::kProject, - PK::kGetNeighbors, + PK::kExpandAll, + PK::kExpand, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -262,25 +266,11 @@ TEST_F(FindPathValidatorTest, RunTimePath) { "GO FROM \"2\" OVER like yield like._src AS src, like._dst AS dst " " | FIND SHORTEST PATH FROM $a.src TO $-.dst OVER like UPTO 5 STEPS YIELD path as p"; std::vector expected = { - PK::kDataCollect, - PK::kLoop, - PK::kProject, - PK::kMultiShortestPath, - PK::kProject, - PK::kGetNeighbors, - PK::kGetNeighbors, - PK::kDedup, - PK::kPassThrough, - PK::kProject, - PK::kStart, - PK::kDedup, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + PK::kDataCollect, PK::kLoop, PK::kProject, PK::kMultiShortestPath, + PK::kProject, PK::kGetNeighbors, PK::kGetNeighbors, PK::kDedup, + PK::kPassThrough, PK::kProject, PK::kStart, PK::kDedup, + PK::kProject, PK::kProject, PK::kExpandAll, PK::kExpand, + PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { diff --git a/src/graph/validator/test/GetSubgraphValidatorTest.cpp b/src/graph/validator/test/GetSubgraphValidatorTest.cpp index 26c79d9cdf7..6aeebe43ea1 100644 --- a/src/graph/validator/test/GetSubgraphValidatorTest.cpp +++ b/src/graph/validator/test/GetSubgraphValidatorTest.cpp @@ -59,7 +59,8 @@ TEST_F(GetSubgraphValidatorTest, Input) { PK::kDedup, PK::kProject, PK::kProject, - PK::kGetNeighbors, + PK::kExpandAll, + PK::kExpand, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -74,7 +75,8 @@ TEST_F(GetSubgraphValidatorTest, Input) { PK::kDedup, PK::kProject, PK::kProject, - PK::kGetNeighbors, + PK::kExpandAll, + PK::kExpand, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -108,7 +110,8 @@ TEST_F(GetSubgraphValidatorTest, Input) { PK::kDedup, PK::kProject, PK::kProject, - PK::kGetNeighbors, + PK::kExpandAll, + PK::kExpand, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -123,7 +126,8 @@ TEST_F(GetSubgraphValidatorTest, Input) { PK::kDedup, PK::kProject, PK::kProject, - PK::kGetNeighbors, + PK::kExpandAll, + PK::kExpand, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); diff --git a/src/graph/validator/test/GroupByValidatorTest.cpp b/src/graph/validator/test/GroupByValidatorTest.cpp index 89368e07828..45b82b621de 100644 --- a/src/graph/validator/test/GroupByValidatorTest.cpp +++ b/src/graph/validator/test/GroupByValidatorTest.cpp @@ -21,7 +21,7 @@ TEST_F(GroupByValidatorTest, TestGroupBy) { "GO FROM \"1\" OVER like YIELD like._dst AS id, $^.person.age AS age " "| GROUP BY $-.age YIELD $-.age, COUNT($-.id)"; std::vector expected = { - PK::kAggregate, PK::kProject, PK::kGetNeighbors, PK::kStart}; + PK::kAggregate, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -30,7 +30,7 @@ TEST_F(GroupByValidatorTest, TestGroupBy) { "age " "| GROUP BY $-.id YIELD $-.id AS id"; std::vector expected = { - PK::kAggregate, PK::kProject, PK::kGetNeighbors, PK::kStart}; + PK::kAggregate, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -39,7 +39,7 @@ TEST_F(GroupByValidatorTest, TestGroupBy) { "age " "| GROUP BY $-.id YIELD $-.id AS id, abs(avg($-.age)) AS age"; std::vector expected = { - PK::kProject, PK::kAggregate, PK::kProject, PK::kGetNeighbors, PK::kStart}; + PK::kProject, PK::kAggregate, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -61,9 +61,9 @@ TEST_F(GroupByValidatorTest, TestGroupBy) { std::vector expected = {PK::kAggregate, PK::kProject, PK::kHashLeftJoin, + PK::kExpandAll, PK::kProject, - PK::kProject, - PK::kGetNeighbors, + PK::kExpand, PK::kGetVertices, PK::kStart, PK::kArgument}; @@ -89,9 +89,9 @@ TEST_F(GroupByValidatorTest, TestGroupBy) { PK::kAggregate, PK::kProject, PK::kHashLeftJoin, + PK::kExpandAll, PK::kProject, - PK::kProject, - PK::kGetNeighbors, + PK::kExpand, PK::kGetVertices, PK::kStart, PK::kArgument}; @@ -112,9 +112,9 @@ TEST_F(GroupByValidatorTest, TestGroupBy) { std::vector expected = {PK::kAggregate, PK::kProject, PK::kHashLeftJoin, + PK::kExpandAll, PK::kProject, - PK::kProject, - PK::kGetNeighbors, + PK::kExpand, PK::kGetVertices, PK::kStart, PK::kArgument}; @@ -136,9 +136,9 @@ TEST_F(GroupByValidatorTest, TestGroupBy) { std::vector expected = {PK::kAggregate, PK::kProject, PK::kHashLeftJoin, + PK::kExpandAll, PK::kProject, - PK::kProject, - PK::kGetNeighbors, + PK::kExpand, PK::kGetVertices, PK::kStart, PK::kArgument}; @@ -160,9 +160,9 @@ TEST_F(GroupByValidatorTest, TestGroupBy) { std::vector expected = {PK::kAggregate, PK::kProject, PK::kHashLeftJoin, + PK::kExpandAll, PK::kProject, - PK::kProject, - PK::kGetNeighbors, + PK::kExpand, PK::kGetVertices, PK::kStart, PK::kArgument}; @@ -177,7 +177,7 @@ TEST_F(GroupByValidatorTest, VariableTest) { "age; " "GROUP BY $a.age YIELD $a.age, COUNT($a.id)"; std::vector expected = { - PK::kAggregate, PK::kProject, PK::kGetNeighbors, PK::kStart}; + PK::kAggregate, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -196,9 +196,9 @@ TEST_F(GroupByValidatorTest, VariableTest) { std::vector expected = {PK::kAggregate, PK::kProject, PK::kHashLeftJoin, + PK::kExpandAll, PK::kProject, - PK::kProject, - PK::kGetNeighbors, + PK::kExpand, PK::kGetVertices, PK::kStart, PK::kArgument}; @@ -220,9 +220,9 @@ TEST_F(GroupByValidatorTest, VariableTest) { std::vector expected = {PK::kAggregate, PK::kProject, PK::kHashLeftJoin, + PK::kExpandAll, PK::kProject, - PK::kProject, - PK::kGetNeighbors, + PK::kExpand, PK::kGetVertices, PK::kStart, PK::kArgument}; diff --git a/src/graph/validator/test/MutateValidatorTest.cpp b/src/graph/validator/test/MutateValidatorTest.cpp index 09f7af2ba3e..a90290bc4cc 100644 --- a/src/graph/validator/test/MutateValidatorTest.cpp +++ b/src/graph/validator/test/MutateValidatorTest.cpp @@ -85,28 +85,22 @@ TEST_F(MutateValidatorTest, DeleteVertexTest) { // pipe { auto cmd = "GO FROM \"C\" OVER like YIELD like._dst as dst | DELETE VERTEX $-.dst WITH EDGE"; - std::vector expected = { - PK::kDeleteVertices, - PK::kDeleteEdges, - PK::kDedup, - PK::kProject, - PK::kGetNeighbors, - PK::kDedup, - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kDeleteVertices, + PK::kDeleteEdges, + PK::kDedup, + PK::kProject, + PK::kGetNeighbors, + PK::kDedup, + PK::kProject, + PK::kExpandAll, + PK::kExpand, + PK::kStart}; ASSERT_TRUE(checkResult(cmd, expected)); } { auto cmd = "GO FROM \"C\" OVER like YIELD like._dst as dst | DELETE VERTEX $-.dst"; std::vector expected = { - PK::kDeleteVertices, - PK::kDedup, - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + PK::kDeleteVertices, PK::kDedup, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; ASSERT_TRUE(checkResult(cmd, expected)); } // pipe wrong input @@ -139,12 +133,7 @@ TEST_F(MutateValidatorTest, DeleteEdgeTest) { "YIELD like._src as src, like._dst as dst, like._rank as rank " "| DELETE EDGE like $-.src -> $-.dst @ $-.rank"; std::vector expected = { - PK::kDeleteEdges, - PK::kDedup, - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + PK::kDeleteEdges, PK::kDedup, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; ASSERT_TRUE(checkResult(cmd, expected)); } // var @@ -154,12 +143,7 @@ TEST_F(MutateValidatorTest, DeleteEdgeTest) { "YIELD like._src as src, like._dst as dst, like._rank as rank " "; DELETE EDGE like $var.src -> $var.dst @ $var.rank"; std::vector expected = { - PK::kDeleteEdges, - PK::kDedup, - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + PK::kDeleteEdges, PK::kDedup, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; ASSERT_TRUE(checkResult(cmd, expected)); } // pipe wrong input diff --git a/src/graph/validator/test/QueryValidatorTest.cpp b/src/graph/validator/test/QueryValidatorTest.cpp index 72f19c83b38..4412d3b3091 100644 --- a/src/graph/validator/test/QueryValidatorTest.cpp +++ b/src/graph/validator/test/QueryValidatorTest.cpp @@ -57,13 +57,12 @@ TEST_F(QueryValidatorTest, GoZeroStep) { "GO 0 STEPS FROM \"1\" OVER like YIELD like._dst as id" "| GO FROM $-.id OVER serve YIELD edge as e"; std::vector expected = {PK::kProject, - PK::kInnerJoin, - PK::kProject, - PK::kGetNeighbors, - PK::kDedup, - PK::kProject, + PK::kHashInnerJoin, PK::kPassThrough, - PK::kStart}; + PK::kExpandAll, + PK::kStart, + PK::kExpand, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -76,13 +75,7 @@ TEST_F(QueryValidatorTest, GoZeroStep) { TEST_F(QueryValidatorTest, GoNSteps) { { std::string query = "GO 2 STEPS FROM \"1\" OVER like YIELD $^ as src"; - std::vector expected = {PK::kProject, - PK::kGetNeighbors, - PK::kLoop, - PK::kStart, - PK::kDedup, - PK::kGetDstBySrc, - PK::kStart}; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -91,11 +84,8 @@ TEST_F(QueryValidatorTest, GoNSteps) { std::vector expected = { PK::kProject, PK::kFilter, - PK::kGetNeighbors, - PK::kLoop, - PK::kStart, - PK::kDedup, - PK::kGetDstBySrc, + PK::kExpandAll, + PK::kExpand, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -108,11 +98,8 @@ TEST_F(QueryValidatorTest, GoNSteps) { PK::kDedup, PK::kProject, PK::kFilter, - PK::kGetNeighbors, - PK::kLoop, - PK::kStart, - PK::kDedup, - PK::kGetDstBySrc, + PK::kExpandAll, + PK::kExpand, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -121,15 +108,8 @@ TEST_F(QueryValidatorTest, GoNSteps) { std::string query = "GO 2 STEPS FROM \"1\",\"2\",\"3\" OVER like WHERE $^.person.age > 20" "YIELD distinct $^.person.name "; - std::vector expected = {PK::kDedup, - PK::kProject, - PK::kFilter, - PK::kGetNeighbors, - PK::kLoop, - PK::kStart, - PK::kDedup, - PK::kGetDstBySrc, - PK::kStart}; + std::vector expected = { + PK::kDedup, PK::kProject, PK::kFilter, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } } @@ -139,12 +119,15 @@ TEST_F(QueryValidatorTest, GoWithPipe) { std::string query = "GO 1 STEPS FROM \"1\" OVER like YIELD like._dst AS " "id | GO 2 STEPS FROM $-.id OVER like YIELD edge as e"; - std::vector expected = { - PK::kProject, PK::kInnerJoin, PK::kInnerJoin, PK::kProject, PK::kGetNeighbors, - PK::kLoop, PK::kDedup, PK::kDedup, PK::kProject, PK::kProject, - PK::kDedup, PK::kInnerJoin, PK::kProject, PK::kDedup, PK::kProject, - PK::kProject, PK::kGetNeighbors, PK::kDedup, PK::kStart, PK::kProject, - PK::kGetNeighbors, PK::kStart}; + std::vector expected = {PK::kProject, + PK::kHashInnerJoin, + PK::kProject, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kArgument, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -152,47 +135,41 @@ TEST_F(QueryValidatorTest, GoWithPipe) { "GO 2 STEPS FROM \"1\" OVER like YIELD like._dst AS id" "| GO 1 STEPS FROM $-.id OVER like YIELD src(edge) as src"; std::vector expected = {PK::kProject, - PK::kInnerJoin, - PK::kProject, - PK::kGetNeighbors, - PK::kDedup, + PK::kHashInnerJoin, PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kLoop, - PK::kStart, - PK::kDedup, - PK::kGetDstBySrc, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kArgument, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "YIELD \"1\" AS id | GO FROM $-.id OVER like YIELD id($^) as id"; std::vector expected = {PK::kProject, - PK::kInnerJoin, + PK::kHashInnerJoin, PK::kProject, - PK::kGetNeighbors, - PK::kDedup, - PK::kProject, - PK::kProject, - PK::kStart}; + PK::kExpandAll, + PK::kStart, + PK::kExpand, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "YIELD \"1\" AS id | GO FROM $-.id OVER like YIELD id($$) as id"; std::vector expected = {PK::kProject, - PK::kInnerJoin, - PK::kProject, - PK::kGetNeighbors, - PK::kDedup, + PK::kHashInnerJoin, PK::kProject, - PK::kProject, - PK::kStart}; + PK::kExpandAll, + PK::kStart, + PK::kExpand, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM 'Tim' OVER * YIELD id($$) as id"; - std::vector expected = {PK::kProject, PK::kGetNeighbors, PK::kStart}; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -200,13 +177,13 @@ TEST_F(QueryValidatorTest, GoWithPipe) { "GO 1 STEPS FROM \"1\" OVER like YIELD like._dst AS " "id | GO 1 STEPS FROM $-.id OVER like YIELD $-.id, like._dst"; std::vector expected = {PK::kProject, - PK::kInnerJoin, + PK::kHashInnerJoin, PK::kProject, - PK::kGetNeighbors, - PK::kDedup, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kArgument, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } @@ -217,13 +194,13 @@ TEST_F(QueryValidatorTest, GoWithPipe) { "WHERE $-.id == \"2\" YIELD $-.id, like._dst"; std::vector expected = {PK::kProject, PK::kFilter, - PK::kInnerJoin, + PK::kHashInnerJoin, PK::kProject, - PK::kGetNeighbors, - PK::kDedup, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kArgument, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } @@ -235,13 +212,13 @@ TEST_F(QueryValidatorTest, GoWithPipe) { std::vector expected = {PK::kDedup, PK::kProject, PK::kFilter, - PK::kInnerJoin, + PK::kHashInnerJoin, PK::kProject, - PK::kGetNeighbors, - PK::kDedup, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kArgument, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } @@ -249,13 +226,15 @@ TEST_F(QueryValidatorTest, GoWithPipe) { std::string query = "GO 1 STEPS FROM \"1\" OVER like YIELD like._dst AS " "id | GO 2 STEPS FROM $-.id OVER like YIELD $-.id, like._dst"; - std::vector expected = { - PK::kProject, PK::kInnerJoin, PK::kInnerJoin, PK::kProject, PK::kGetNeighbors, - PK::kLoop, PK::kDedup, PK::kDedup, PK::kProject, PK::kProject, - PK::kDedup, PK::kInnerJoin, PK::kProject, PK::kDedup, PK::kProject, - PK::kProject, PK::kGetNeighbors, PK::kDedup, PK::kStart, PK::kProject, - PK::kGetNeighbors, PK::kStart, - }; + std::vector expected = {PK::kProject, + PK::kHashInnerJoin, + PK::kProject, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kArgument, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -263,13 +242,16 @@ TEST_F(QueryValidatorTest, GoWithPipe) { "GO 1 STEPS FROM \"1\" OVER like YIELD like._dst AS " "id | GO 2 STEPS FROM $-.id OVER like " "WHERE $-.id == \"2\" YIELD $-.id, like._dst"; - std::vector expected = { - PK::kProject, PK::kFilter, PK::kInnerJoin, PK::kInnerJoin, PK::kProject, - PK::kGetNeighbors, PK::kLoop, PK::kDedup, PK::kDedup, PK::kProject, - PK::kProject, PK::kDedup, PK::kInnerJoin, PK::kProject, PK::kDedup, - PK::kProject, PK::kProject, PK::kGetNeighbors, PK::kDedup, PK::kStart, - PK::kProject, PK::kGetNeighbors, PK::kStart, - }; + std::vector expected = {PK::kProject, + PK::kFilter, + PK::kHashInnerJoin, + PK::kProject, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kArgument, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -277,13 +259,17 @@ TEST_F(QueryValidatorTest, GoWithPipe) { "GO 1 STEPS FROM \"1\" OVER like YIELD like._dst AS " "id | GO 2 STEPS FROM $-.id OVER like " "WHERE $-.id == \"2\" YIELD DISTINCT $-.id, like._dst"; - std::vector expected = { - PK::kDedup, PK::kProject, PK::kFilter, PK::kInnerJoin, PK::kInnerJoin, - PK::kProject, PK::kGetNeighbors, PK::kLoop, PK::kDedup, PK::kDedup, - PK::kProject, PK::kProject, PK::kDedup, PK::kInnerJoin, PK::kProject, - PK::kDedup, PK::kProject, PK::kProject, PK::kGetNeighbors, PK::kDedup, - PK::kStart, PK::kProject, PK::kGetNeighbors, PK::kStart, - }; + std::vector expected = {PK::kDedup, + PK::kProject, + PK::kFilter, + PK::kHashInnerJoin, + PK::kProject, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kArgument, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -292,26 +278,23 @@ TEST_F(QueryValidatorTest, GoWithPipe) { "id, $$.person.name as name | GO 1 STEPS FROM $-.id OVER like " "YIELD $-.name, like.likeness + 1, $-.id, like._dst, " "$$.person.name"; - std::vector expected = { - PK::kProject, - PK::kInnerJoin, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kDedup, - PK::kArgument, - PK::kProject, - PK::kProject, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kStart, - PK::kArgument, - }; + std::vector expected = {PK::kProject, + PK::kHashInnerJoin, + PK::kProject, + PK::kHashLeftJoin, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kExpand, + PK::kGetVertices, + PK::kArgument, + PK::kArgument, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -320,12 +303,24 @@ TEST_F(QueryValidatorTest, GoWithPipe) { "id, $$.person.name as name | GO 1 STEPS FROM $-.id OVER like " "YIELD DISTINCT $-.name, like.likeness + 1, $-.id, like._dst, " "$$.person.name"; - std::vector expected = { - PK::kDedup, PK::kProject, PK::kInnerJoin, PK::kHashLeftJoin, PK::kProject, - PK::kProject, PK::kGetNeighbors, PK::kGetVertices, PK::kDedup, PK::kArgument, - PK::kProject, PK::kProject, PK::kHashLeftJoin, PK::kProject, PK::kProject, - PK::kGetNeighbors, PK::kGetVertices, PK::kStart, PK::kArgument, - }; + std::vector expected = {PK::kDedup, + PK::kProject, + PK::kHashInnerJoin, + PK::kProject, + PK::kHashLeftJoin, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kExpand, + PK::kGetVertices, + PK::kArgument, + PK::kArgument, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -334,21 +329,19 @@ TEST_F(QueryValidatorTest, GoWithPipe) { "YIELD $^.person.name AS name, like._dst AS id " "| GO FROM $-.id OVER like " "YIELD $-.name, $^.person.name, $$.person.name"; - std::vector expected = { - PK::kProject, - PK::kInnerJoin, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kDedup, - PK::kArgument, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, + PK::kHashInnerJoin, + PK::kProject, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -357,15 +350,23 @@ TEST_F(QueryValidatorTest, GoWithPipe) { "id, $$.person.name as name | GO 2 STEPS FROM $-.id OVER like " "YIELD $-.name, like.likeness + 1, $-.id, like._dst, " "$$.person.name"; - std::vector expected = { - PK::kProject, PK::kInnerJoin, PK::kInnerJoin, PK::kHashLeftJoin, PK::kProject, - PK::kProject, PK::kGetNeighbors, PK::kGetVertices, PK::kLoop, PK::kArgument, - PK::kDedup, PK::kDedup, PK::kProject, PK::kProject, PK::kDedup, - PK::kInnerJoin, PK::kProject, PK::kDedup, PK::kProject, PK::kProject, - PK::kHashLeftJoin, PK::kDedup, PK::kProject, PK::kProject, PK::kProject, - PK::kGetNeighbors, PK::kGetVertices, PK::kGetNeighbors, PK::kStart, PK::kArgument, - PK::kStart, - }; + std::vector expected = {PK::kProject, + PK::kHashInnerJoin, + PK::kProject, + PK::kHashLeftJoin, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kExpand, + PK::kGetVertices, + PK::kArgument, + PK::kArgument, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -374,23 +375,31 @@ TEST_F(QueryValidatorTest, GoWithPipe) { "id, $$.person.name as name | GO 2 STEPS FROM $-.id OVER like " "YIELD DISTINCT $-.name, like.likeness + 1, $-.id, like._dst, " "$$.person.name"; - std::vector expected = { - PK::kDedup, PK::kProject, PK::kInnerJoin, PK::kInnerJoin, PK::kHashLeftJoin, - PK::kProject, PK::kProject, PK::kGetNeighbors, PK::kGetVertices, PK::kLoop, - PK::kArgument, PK::kDedup, PK::kDedup, PK::kProject, PK::kProject, - PK::kDedup, PK::kInnerJoin, PK::kProject, PK::kDedup, PK::kProject, - PK::kProject, PK::kHashLeftJoin, PK::kDedup, PK::kProject, PK::kProject, - PK::kProject, PK::kGetNeighbors, PK::kGetVertices, PK::kGetNeighbors, PK::kStart, - PK::kArgument, PK::kStart, - }; + std::vector expected = {PK::kDedup, + PK::kProject, + PK::kHashInnerJoin, + PK::kProject, + PK::kHashLeftJoin, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kExpand, + PK::kGetVertices, + PK::kArgument, + PK::kArgument, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO 1 STEPS FROM \"1\" OVER like YIELD like._dst AS " "id | GO 1 STEPS FROM \"1\" OVER like YIELD like._dst"; - std::vector expected = { - PK::kProject, PK::kGetNeighbors, PK::kProject, PK::kGetNeighbors, PK::kStart}; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } } @@ -402,21 +411,19 @@ TEST_F(QueryValidatorTest, GoWithVariable) { "YIELD $^.person.name AS name, like._dst AS id;" "GO FROM $var.id OVER like " "YIELD $var.name, $^.person.name, $$.person.name"; - std::vector expected = { - PK::kProject, - PK::kInnerJoin, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kDedup, - PK::kArgument, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, + PK::kHashInnerJoin, + PK::kProject, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } } @@ -424,34 +431,26 @@ TEST_F(QueryValidatorTest, GoWithVariable) { TEST_F(QueryValidatorTest, GoReversely) { { std::string query = "GO FROM \"1\" OVER like REVERSELY YIELD $$.person.name"; - std::vector expected = { - PK::kProject, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kStart, - PK::kArgument, - }; + std::vector expected = {PK::kProject, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO 2 STEPS FROM \"1\" OVER like REVERSELY YIELD $$.person.name"; - std::vector expected = { - PK::kProject, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kLoop, - PK::kArgument, - PK::kStart, - PK::kDedup, - PK::kGetDstBySrc, - PK::kStart, - }; + std::vector expected = {PK::kProject, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } } @@ -459,27 +458,21 @@ TEST_F(QueryValidatorTest, GoReversely) { TEST_F(QueryValidatorTest, GoBidirectly) { { std::string query = "GO FROM \"1\" OVER like BIDIRECT YIELD edge as e"; - std::vector expected = { - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\" OVER like BIDIRECT " "YIELD $$.person.name"; - std::vector expected = { - PK::kProject, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kStart, - PK::kArgument, - }; + std::vector expected = {PK::kProject, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } } @@ -487,65 +480,43 @@ TEST_F(QueryValidatorTest, GoBidirectly) { TEST_F(QueryValidatorTest, GoOneStep) { { std::string query = "GO FROM \"1\" OVER like YIELD src(edge) as src"; - std::vector expected = { - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\" OVER like REVERSELY YIELD dst(edge) as dst"; - std::vector expected = { - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\" OVER like BIDIRECT YIELD dst(edge) as dst"; - std::vector expected = { - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\" OVER like YIELD like.start"; - std::vector expected = { - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\" OVER like " "YIELD $^.person.name,$^.person.age"; - std::vector expected = { - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\" OVER like " "YIELD $$.person.name,$$.person.age"; - std::vector expected = { - PK::kProject, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kStart, - PK::kArgument, - }; + std::vector expected = {PK::kProject, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -553,16 +524,14 @@ TEST_F(QueryValidatorTest, GoOneStep) { "GO FROM \"1\" OVER like " "YIELD $^.person.name, like._dst, " "$$.person.name, $$.person.age + 1"; - std::vector expected = { - PK::kProject, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kStart, - PK::kArgument, - }; + std::vector expected = {PK::kProject, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -571,17 +540,15 @@ TEST_F(QueryValidatorTest, GoOneStep) { "WHERE like._dst == \"2\"" "YIELD $^.person.name, like._dst, " "$$.person.name, $$.person.age + 1"; - std::vector expected = { - PK::kProject, - PK::kFilter, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kStart, - PK::kArgument, - }; + std::vector expected = {PK::kProject, + PK::kFilter, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -590,38 +557,28 @@ TEST_F(QueryValidatorTest, GoOneStep) { "WHERE like._dst == \"2\"" "YIELD DISTINCT $^.person.name, like._dst, " "$$.person.name, $$.person.age + 1"; - std::vector expected = { - PK::kDedup, - PK::kProject, - PK::kFilter, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kStart, - PK::kArgument, - }; + std::vector expected = {PK::kDedup, + PK::kProject, + PK::kFilter, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\",\"2\",\"3\" OVER like YIELD edge as e"; - std::vector expected = { - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\",\"2\",\"3\" OVER like WHERE like.likeness > 90 YIELD edge as e"; std::vector expected = { - PK::kProject, - PK::kFilter, - PK::kGetNeighbors, - PK::kStart, - }; + PK::kProject, PK::kFilter, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -629,40 +586,29 @@ TEST_F(QueryValidatorTest, GoOneStep) { "GO FROM \"1\",\"2\",\"3\" OVER like WHERE $^.person.age > 20" "YIELD distinct $^.person.name "; std::vector expected = { - PK::kDedup, - PK::kProject, - PK::kFilter, - PK::kGetNeighbors, - PK::kStart, - }; + PK::kDedup, PK::kProject, PK::kFilter, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\",\"2\",\"3\" OVER like WHERE $^.person.name == \"me\" YIELD edge as e"; std::vector expected = { - PK::kProject, - PK::kFilter, - PK::kGetNeighbors, - PK::kStart, - }; + PK::kProject, PK::kFilter, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\" OVER like YIELD like._dst AS id" "| GO FROM $-.id OVER like YIELD dst(edge) as dst"; - std::vector expected = { - PK::kProject, - PK::kInnerJoin, - PK::kProject, - PK::kGetNeighbors, - PK::kDedup, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, + PK::kHashInnerJoin, + PK::kProject, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kArgument, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } } @@ -670,43 +616,29 @@ TEST_F(QueryValidatorTest, GoOneStep) { TEST_F(QueryValidatorTest, GoOverAll) { { std::string query = "GO FROM \"1\" OVER * REVERSELY YIELD serve._src, like._src"; - std::vector expected = { - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\" OVER * REVERSELY YIELD edge as e"; - std::vector expected = { - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\" OVER * YIELD src(edge) as src"; - std::vector expected = { - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO FROM \"1\" OVER * YIELD $$.person.name"; - std::vector expected = { - PK::kProject, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetVertices, - PK::kStart, - PK::kArgument, - }; + std::vector expected = {PK::kProject, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } } @@ -717,23 +649,20 @@ TEST_F(QueryValidatorTest, OutputToAPipe) { "GO FROM '1' OVER like YIELD like._dst as id " "| ( GO FROM $-.id OVER like YIELD like._dst as id | GO FROM $-.id " "OVER serve YIELD dst(edge) as dst )"; - std::vector expected = { - PK::kProject, - PK::kInnerJoin, - PK::kProject, - PK::kGetNeighbors, - PK::kDedup, - PK::kProject, - PK::kProject, - PK::kInnerJoin, - PK::kProject, - PK::kGetNeighbors, - PK::kDedup, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, + PK::kHashInnerJoin, + PK::kProject, + PK::kExpandAll, + PK::kHashInnerJoin, + PK::kExpand, + PK::kProject, + PK::kExpandAll, + PK::kArgument, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kArgument, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } } @@ -742,85 +671,45 @@ TEST_F(QueryValidatorTest, GoMToN) { { std::string query = "GO 1 TO 2 STEPS FROM '1' OVER like YIELD DISTINCT like._dst"; std::vector expected = { - PK::kDataCollect, - PK::kLoop, - PK::kStart, - PK::kDedup, - PK::kGetDstBySrc, - PK::kStart, - }; + PK::kDedup, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO 0 TO 2 STEPS FROM '1' OVER like YIELD DISTINCT like._dst"; std::vector expected = { - PK::kDataCollect, - PK::kLoop, - PK::kStart, - PK::kDedup, - PK::kGetDstBySrc, - PK::kStart, - }; + PK::kDedup, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO 1 TO 2 STEPS FROM '1' OVER like " "YIELD DISTINCT like._dst, like.likeness, $$.person.name"; - std::vector expected = { - PK::kDataCollect, - PK::kLoop, - PK::kStart, - PK::kDedup, - PK::kProject, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kDedup, - PK::kGetVertices, - PK::kProject, - PK::kArgument, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kDedup, + PK::kProject, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO 1 TO 2 STEPS FROM '1' OVER like REVERSELY YIELD DISTINCT like._dst"; std::vector expected = { - PK::kDataCollect, - PK::kLoop, - PK::kStart, - PK::kDedup, - PK::kGetDstBySrc, - PK::kStart, - }; + PK::kDedup, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO 1 TO 2 STEPS FROM '1' OVER like BIDIRECT YIELD DISTINCT like._dst"; std::vector expected = { - PK::kDataCollect, - PK::kLoop, - PK::kStart, - PK::kDedup, - PK::kGetDstBySrc, - PK::kStart, - }; + PK::kDedup, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { std::string query = "GO 1 TO 2 STEPS FROM '1' OVER * YIELD serve._dst, like._dst"; - std::vector expected = { - PK::kDataCollect, - PK::kLoop, - PK::kStart, - PK::kProject, - PK::kDedup, - PK::kProject, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -828,21 +717,14 @@ TEST_F(QueryValidatorTest, GoMToN) { "GO 1 TO 2 STEPS FROM '1' OVER * " "YIELD serve._dst, like._dst, serve.start, like.likeness, " "$$.person.name"; - std::vector expected = { - PK::kDataCollect, - PK::kLoop, - PK::kStart, - PK::kProject, - PK::kHashLeftJoin, - PK::kProject, - PK::kProject, - PK::kDedup, - PK::kGetVertices, - PK::kProject, - PK::kArgument, - PK::kGetNeighbors, - PK::kStart, - }; + std::vector expected = {PK::kProject, + PK::kHashLeftJoin, + PK::kExpandAll, + PK::kProject, + PK::kExpand, + PK::kGetVertices, + PK::kStart, + PK::kArgument}; EXPECT_TRUE(checkResult(query, expected)); } { @@ -851,13 +733,15 @@ TEST_F(QueryValidatorTest, GoMToN) { "dst " "| GO 1 TO 2 STEPS FROM $-.src OVER like YIELD $-.src as src, " "like._dst as dst"; - std::vector expected = { - PK::kDataCollect, PK::kLoop, PK::kDedup, PK::kProject, PK::kProject, - PK::kInnerJoin, PK::kDedup, PK::kInnerJoin, PK::kProject, PK::kProject, - PK::kProject, PK::kDedup, PK::kGetNeighbors, PK::kProject, PK::kStart, - PK::kInnerJoin, PK::kDedup, PK::kProject, PK::kDedup, PK::kProject, - PK::kGetNeighbors, PK::kStart, - }; + std::vector expected = {PK::kProject, + PK::kHashInnerJoin, + PK::kProject, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kArgument, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } } @@ -989,7 +873,7 @@ TEST_F(QueryValidatorTest, Limit) { { std::string query = "GO FROM \"Ann\" OVER like YIELD like._dst AS like | LIMIT 1, 3"; std::vector expected = { - PK::kLimit, PK::kProject, PK::kGetNeighbors, PK::kStart}; + PK::kLimit, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } } @@ -999,7 +883,8 @@ TEST_F(QueryValidatorTest, OrderBy) { std::string query = "GO FROM \"Ann\" OVER like YIELD $^.person.age AS age" " | ORDER BY $-.age"; - std::vector expected = {PK::kSort, PK::kProject, PK::kGetNeighbors, PK::kStart}; + std::vector expected = { + PK::kSort, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } // not exist factor @@ -1017,7 +902,7 @@ TEST_F(QueryValidatorTest, OrderByAndLimit) { "GO FROM \"Ann\" OVER like YIELD $^.person.age AS age" " | ORDER BY $-.age | LIMIT 1"; std::vector expected = { - PK::kLimit, PK::kSort, PK::kProject, PK::kGetNeighbors, PK::kStart}; + PK::kLimit, PK::kSort, PK::kProject, PK::kExpandAll, PK::kExpand, PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } } @@ -1027,17 +912,16 @@ TEST_F(QueryValidatorTest, TestSetValidator) { { std::string query = "GO FROM \"1\" OVER like YIELD like.start AS start UNION ALL GO FROM " - "\"2\" " - "OVER like YIELD like.start AS start"; - std::vector expected = { - PK::kUnion, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetNeighbors, - PK::kPassThrough, - PK::kStart, - }; + "\"2\" OVER like YIELD like.start AS start"; + std::vector expected = {PK::kUnion, + PK::kProject, + PK::kProject, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kStart, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } // UNION DISTINCT twice @@ -1045,41 +929,40 @@ TEST_F(QueryValidatorTest, TestSetValidator) { std::string query = "GO FROM \"1\" OVER like YIELD like.start AS start UNION GO FROM \"2\" " "OVER like YIELD like.start AS start UNION GO FROM \"3\" OVER like " - "YIELD " - "like.start AS start"; - std::vector expected = { - PK::kDedup, - PK::kUnion, - PK::kDedup, - PK::kProject, - PK::kUnion, - PK::kGetNeighbors, - PK::kProject, - PK::kProject, - PK::kPassThrough, - PK::kGetNeighbors, - PK::kGetNeighbors, - PK::kStart, - PK::kPassThrough, - }; + "YIELD like.start AS start"; + std::vector expected = {PK::kDedup, + PK::kUnion, + PK::kDedup, + PK::kProject, + PK::kUnion, + PK::kExpandAll, + PK::kProject, + PK::kProject, + PK::kExpand, + PK::kExpandAll, + PK::kExpandAll, + PK::kStart, + PK::kExpand, + PK::kExpand, + PK::kStart, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } // UNION DISTINCT { std::string query = "GO FROM \"1\" OVER like YIELD like.start AS start UNION DISTINCT GO " - "FROM \"2\" " - "OVER like YIELD like.start AS start"; - std::vector expected = { - PK::kDedup, - PK::kUnion, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetNeighbors, - PK::kPassThrough, - PK::kStart, - }; + "FROM \"2\" OVER like YIELD like.start AS start"; + std::vector expected = {PK::kDedup, + PK::kUnion, + PK::kProject, + PK::kProject, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kStart, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } // INVALID UNION ALL @@ -1094,15 +977,15 @@ TEST_F(QueryValidatorTest, TestSetValidator) { std::string query = "GO FROM \"1\" OVER like YIELD like.start AS start INTERSECT GO FROM " "\"2\" OVER like YIELD like.start AS start"; - std::vector expected = { - PK::kIntersect, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetNeighbors, - PK::kPassThrough, - PK::kStart, - }; + std::vector expected = {PK::kIntersect, + PK::kProject, + PK::kProject, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kStart, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } // MINUS @@ -1110,15 +993,15 @@ TEST_F(QueryValidatorTest, TestSetValidator) { std::string query = "GO FROM \"1\" OVER like YIELD like.start AS start MINUS GO FROM " "\"2\" OVER like YIELD like.start AS start"; - std::vector expected = { - PK::kMinus, - PK::kProject, - PK::kProject, - PK::kGetNeighbors, - PK::kGetNeighbors, - PK::kPassThrough, - PK::kStart, - }; + std::vector expected = {PK::kMinus, + PK::kProject, + PK::kProject, + PK::kExpandAll, + PK::kExpandAll, + PK::kExpand, + PK::kExpand, + PK::kStart, + PK::kStart}; EXPECT_TRUE(checkResult(query, expected)); } } diff --git a/src/graph/validator/test/SymbolsTest.cpp b/src/graph/validator/test/SymbolsTest.cpp index 2bc27cb47d8..bd67d5c48e3 100644 --- a/src/graph/validator/test/SymbolsTest.cpp +++ b/src/graph/validator/test/SymbolsTest.cpp @@ -51,206 +51,89 @@ TEST_F(SymbolsTest, Variables) { auto* symTable = qctx->symTable(); { - auto varName = "__Start_0"; + auto varName = "__Start_1"; auto* variable = symTable->getVar(varName); EXPECT_NE(variable, nullptr); EXPECT_EQ(variable->name, varName); EXPECT_EQ(variable->type, Value::Type::DATASET); EXPECT_TRUE(variable->colNames.empty()); EXPECT_TRUE(checkNodes(variable->readBy, {})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {0})); - } - { - auto varName = "__GetNeighbors_1"; - auto* variable = symTable->getVar(varName); - EXPECT_NE(variable, nullptr); - EXPECT_EQ(variable->name, varName); - EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_TRUE(variable->colNames.empty()); - EXPECT_TRUE(checkNodes(variable->readBy, {2})); EXPECT_TRUE(checkNodes(variable->writtenBy, {1})); } { - auto varName = "__Project_2"; + auto varName = "__Expand_2"; auto* variable = symTable->getVar(varName); EXPECT_NE(variable, nullptr); EXPECT_EQ(variable->name, varName); EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"id"})); - EXPECT_TRUE(checkNodes(variable->readBy, {3, 9, 20})); + EXPECT_TRUE(checkNodes(variable->readBy, {3})); EXPECT_TRUE(checkNodes(variable->writtenBy, {2})); } { - auto varName = "__Project_3"; + auto varName = "__ExpandAll_3"; auto* variable = symTable->getVar(varName); EXPECT_NE(variable, nullptr); EXPECT_EQ(variable->name, varName); EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"_vid"})); EXPECT_TRUE(checkNodes(variable->readBy, {4})); EXPECT_TRUE(checkNodes(variable->writtenBy, {3})); } { - auto varName = "__Dedup_4"; - auto* variable = symTable->getVar(varName); - EXPECT_NE(variable, nullptr); - EXPECT_EQ(variable->name, varName); - EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"_vid"})); - EXPECT_TRUE(checkNodes(variable->readBy, {17, 6})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {8, 4})); - } - { - auto varName = "__Project_9"; - auto* variable = symTable->getVar(varName); - EXPECT_NE(variable, nullptr); - EXPECT_EQ(variable->name, varName); - EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"id", "__COL_1"})); - EXPECT_TRUE(checkNodes(variable->readBy, {10})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {9})); - } - { - auto varName = "__Dedup_10"; - auto* variable = symTable->getVar(varName); - EXPECT_NE(variable, nullptr); - EXPECT_EQ(variable->name, varName); - EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"id", "__COL_1"})); - EXPECT_TRUE(checkNodes(variable->readBy, {13, 16, 19})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {10, 15})); - } - { - auto varName = "__Start_5"; + auto varName = "__Project_4"; auto* variable = symTable->getVar(varName); EXPECT_NE(variable, nullptr); EXPECT_EQ(variable->name, varName); EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_TRUE(variable->colNames.empty()); - EXPECT_TRUE(checkNodes(variable->readBy, {})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {5})); + EXPECT_EQ(variable->colNames, std::vector({"id"})); + EXPECT_TRUE(checkNodes(variable->readBy, {9})); + EXPECT_TRUE(checkNodes(variable->writtenBy, {4})); } { - auto varName = "__GetNeighbors_6"; + auto varName = "__Argument_6"; auto* variable = symTable->getVar(varName); EXPECT_NE(variable, nullptr); EXPECT_EQ(variable->name, varName); EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_TRUE(variable->colNames.empty()); - EXPECT_TRUE(checkNodes(variable->readBy, {11, 7})); + EXPECT_EQ(variable->colNames, std::vector({"id"})); + EXPECT_TRUE(checkNodes(variable->readBy, {7})); EXPECT_TRUE(checkNodes(variable->writtenBy, {6})); } { - auto varName = "__Project_7"; + auto varName = "__Expand_7"; auto* variable = symTable->getVar(varName); EXPECT_NE(variable, nullptr); EXPECT_EQ(variable->name, varName); EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"_vid"})); + EXPECT_EQ(variable->colNames, std::vector({"_expand_vid", "_expand_dst"})); EXPECT_TRUE(checkNodes(variable->readBy, {8})); EXPECT_TRUE(checkNodes(variable->writtenBy, {7})); } { - auto varName = "__Project_11"; - auto* variable = symTable->getVar(varName); - EXPECT_NE(variable, nullptr); - EXPECT_EQ(variable->name, varName); - EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"__COL_2", "TRACK_DST_VID"})); - EXPECT_TRUE(checkNodes(variable->readBy, {12})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {11})); - } - { - auto varName = "__Dedup_12"; + auto varName = "__ExpandAll_8"; auto* variable = symTable->getVar(varName); EXPECT_NE(variable, nullptr); EXPECT_EQ(variable->name, varName); EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"__COL_2", "TRACK_DST_VID"})); - EXPECT_TRUE(checkNodes(variable->readBy, {13})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {12})); + EXPECT_TRUE(checkNodes(variable->readBy, {9})); + EXPECT_TRUE(checkNodes(variable->writtenBy, {8})); } { - auto varName = "__InnerJoin_13"; + auto varName = "__HashInnerJoin_9"; auto* variable = symTable->getVar(varName); EXPECT_NE(variable, nullptr); EXPECT_EQ(variable->name, varName); EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, - std::vector({"id", "__COL_1", "__COL_2", "TRACK_DST_VID"})); - EXPECT_TRUE(checkNodes(variable->readBy, {14})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {13})); - } - { - auto varName = "__Project_14"; - auto* variable = symTable->getVar(varName); - EXPECT_NE(variable, nullptr); - EXPECT_EQ(variable->name, varName); - EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"id", "__COL_1"})); - EXPECT_TRUE(checkNodes(variable->readBy, {15})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {14})); - } - { - auto varName = "__Loop_16"; - auto* variable = symTable->getVar(varName); - EXPECT_NE(variable, nullptr); - EXPECT_EQ(variable->name, varName); - EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({})); - EXPECT_TRUE(checkNodes(variable->readBy, {})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {16})); - } - { - auto varName = "__GetNeighbors_17"; - auto* variable = symTable->getVar(varName); - EXPECT_NE(variable, nullptr); - EXPECT_EQ(variable->name, varName); - EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_TRUE(variable->colNames.empty()); - EXPECT_TRUE(checkNodes(variable->readBy, {18})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {17})); - } - { - auto varName = "__Project_18"; - auto* variable = symTable->getVar(varName); - EXPECT_NE(variable, nullptr); - EXPECT_EQ(variable->name, varName); - EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"__COL_0", "_vid"})); - EXPECT_TRUE(checkNodes(variable->readBy, {19})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {18})); - } - { - auto varName = "__InnerJoin_19"; - auto* variable = symTable->getVar(varName); - EXPECT_NE(variable, nullptr); - EXPECT_EQ(variable->name, varName); - EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"id", "__COL_1", "__COL_0", "_vid"})); - EXPECT_TRUE(checkNodes(variable->readBy, {20})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {19})); - } - { - auto varName = "__InnerJoin_20"; - auto* variable = symTable->getVar(varName); - EXPECT_NE(variable, nullptr); - EXPECT_EQ(variable->name, varName); - EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, - std::vector({"id", "__COL_1", "__COL_0", "_vid", "id"})); - EXPECT_TRUE(checkNodes(variable->readBy, {21})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {20})); + EXPECT_TRUE(checkNodes(variable->readBy, {10})); + EXPECT_TRUE(checkNodes(variable->writtenBy, {9})); } { - auto varName = "__Project_21"; + auto varName = "__Project_10"; auto* variable = symTable->getVar(varName); EXPECT_NE(variable, nullptr); EXPECT_EQ(variable->name, varName); EXPECT_EQ(variable->type, Value::Type::DATASET); - EXPECT_EQ(variable->colNames, std::vector({"like._dst"})); EXPECT_TRUE(checkNodes(variable->readBy, {})); - EXPECT_TRUE(checkNodes(variable->writtenBy, {21})); + EXPECT_TRUE(checkNodes(variable->writtenBy, {10})); } } } diff --git a/src/graph/validator/test/YieldValidatorTest.cpp b/src/graph/validator/test/YieldValidatorTest.cpp index 4d4b3aa1926..21c13ced8dd 100644 --- a/src/graph/validator/test/YieldValidatorTest.cpp +++ b/src/graph/validator/test/YieldValidatorTest.cpp @@ -235,7 +235,8 @@ TEST_F(YieldValidatorTest, YieldPipe) { expected_ = { PlanNode::Kind::kProject, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_)); @@ -246,7 +247,8 @@ TEST_F(YieldValidatorTest, YieldPipe) { PlanNode::Kind::kProject, PlanNode::Kind::kFilter, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_)); @@ -257,7 +259,8 @@ TEST_F(YieldValidatorTest, YieldPipe) { PlanNode::Kind::kProject, PlanNode::Kind::kFilter, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_)); @@ -267,7 +270,8 @@ TEST_F(YieldValidatorTest, YieldPipe) { expected_ = { PlanNode::Kind::kProject, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_)); @@ -278,7 +282,8 @@ TEST_F(YieldValidatorTest, YieldPipe) { PlanNode::Kind::kProject, PlanNode::Kind::kFilter, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_, {"$-.name", "$-.start"})); @@ -290,7 +295,7 @@ TEST_F(YieldValidatorTest, YieldPipe) { PlanNode::Kind::kProject, PlanNode::Kind::kFilter, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_, {"name", "start", "hash"})); @@ -303,7 +308,7 @@ TEST_F(YieldValidatorTest, YieldPipe) { PlanNode::Kind::kProject, PlanNode::Kind::kFilter, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_, {"name", "start", "hash"})); @@ -328,7 +333,8 @@ TEST_F(YieldValidatorTest, YieldVar) { expected_ = { PlanNode::Kind::kProject, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_, {"$var.name"})); @@ -339,7 +345,8 @@ TEST_F(YieldValidatorTest, YieldVar) { PlanNode::Kind::kProject, PlanNode::Kind::kFilter, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_, {"$var.name"})); @@ -350,7 +357,8 @@ TEST_F(YieldValidatorTest, YieldVar) { PlanNode::Kind::kProject, PlanNode::Kind::kFilter, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_)); @@ -360,7 +368,8 @@ TEST_F(YieldValidatorTest, YieldVar) { expected_ = { PlanNode::Kind::kProject, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_, {"$var.name", "$var.start"})); @@ -371,7 +380,8 @@ TEST_F(YieldValidatorTest, YieldVar) { PlanNode::Kind::kProject, PlanNode::Kind::kFilter, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_, {"$var.name", "$var.start"})); @@ -383,7 +393,7 @@ TEST_F(YieldValidatorTest, YieldVar) { PlanNode::Kind::kProject, PlanNode::Kind::kFilter, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_, {"$var.name", "$var.start", "hash"})); @@ -402,7 +412,7 @@ TEST_F(YieldValidatorTest, YieldVar) { PlanNode::Kind::kProject, PlanNode::Kind::kFilter, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_, {"$var.name", "$var.start", "hash"})); @@ -491,7 +501,8 @@ TEST_F(YieldValidatorTest, AggCall) { expected_ = { PlanNode::Kind::kAggregate, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_)); @@ -505,7 +516,8 @@ TEST_F(YieldValidatorTest, AggCall) { expected_ = { PlanNode::Kind::kAggregate, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_)); @@ -520,7 +532,8 @@ TEST_F(YieldValidatorTest, AggCall) { PlanNode::Kind::kDedup, PlanNode::Kind::kAggregate, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_)); @@ -550,7 +563,8 @@ TEST_F(YieldValidatorTest, AggCall) { expected_ = { PlanNode::Kind::kAggregate, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_)); @@ -561,7 +575,8 @@ TEST_F(YieldValidatorTest, AggCall) { expected_ = { PlanNode::Kind::kProject, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_)); @@ -576,7 +591,8 @@ TEST_F(YieldValidatorTest, AggCall) { expected_ = { PlanNode::Kind::kAggregate, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query, expected_)); @@ -598,7 +614,8 @@ TEST_F(YieldValidatorTest, AggCall) { expected_ = { PlanNode::Kind::kAggregate, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query)); @@ -614,7 +631,8 @@ TEST_F(YieldValidatorTest, AggCall) { PlanNode::Kind::kDedup, PlanNode::Kind::kAggregate, PlanNode::Kind::kProject, - PlanNode::Kind::kGetNeighbors, + PlanNode::Kind::kExpandAll, + PlanNode::Kind::kExpand, PlanNode::Kind::kStart, }; EXPECT_TRUE(checkResult(query)); diff --git a/src/graph/visitor/ExtractFilterExprVisitor.cpp b/src/graph/visitor/ExtractFilterExprVisitor.cpp index 476b64d9931..64ae1d70a71 100644 --- a/src/graph/visitor/ExtractFilterExprVisitor.cpp +++ b/src/graph/visitor/ExtractFilterExprVisitor.cpp @@ -70,8 +70,10 @@ void ExtractFilterExprVisitor::visit(LabelTagPropertyExpression *) { canBePushed_ = false; } -void ExtractFilterExprVisitor::visit(VariablePropertyExpression *) { - canBePushed_ = false; +void ExtractFilterExprVisitor::visit(VariablePropertyExpression *expr) { + auto colName = expr->prop(); + auto iter = std::find(colNames_.begin(), colNames_.end(), colName); + canBePushed_ = iter != colNames_.end(); } void ExtractFilterExprVisitor::visit(DestPropertyExpression *) { diff --git a/src/graph/visitor/ExtractFilterExprVisitor.h b/src/graph/visitor/ExtractFilterExprVisitor.h index 43e2d74d004..ba04cea70ac 100644 --- a/src/graph/visitor/ExtractFilterExprVisitor.h +++ b/src/graph/visitor/ExtractFilterExprVisitor.h @@ -15,6 +15,8 @@ namespace graph { class ExtractFilterExprVisitor final : public ExprVisitorImpl { public: explicit ExtractFilterExprVisitor(ObjectPool *ObjPool) : pool_(ObjPool) {} + explicit ExtractFilterExprVisitor(ObjectPool *ObjPool, std::vector colNames) + : pool_(ObjPool), colNames_(std::move(colNames)) {} bool ok() const override { return canBePushed_; @@ -94,6 +96,7 @@ class ExtractFilterExprVisitor final : public ExprVisitorImpl { Expression *remainedExpr_{nullptr}; Expression *extractedExpr_{nullptr}; PushType pushType_{PushType::kGetNeighbors}; + std::vector colNames_; }; } // namespace graph diff --git a/src/graph/visitor/ExtractPropExprVisitor.cpp b/src/graph/visitor/ExtractPropExprVisitor.cpp index fa2dd1db8b8..c314310d372 100644 --- a/src/graph/visitor/ExtractPropExprVisitor.cpp +++ b/src/graph/visitor/ExtractPropExprVisitor.cpp @@ -10,13 +10,15 @@ namespace graph { ExtractPropExprVisitor::ExtractPropExprVisitor( ValidateContext* vctx, - YieldColumns* srcAndEdgePropCols, + YieldColumns* srcPropCols, + YieldColumns* edgePropCols, YieldColumns* dstPropCols, YieldColumns* inputPropCols, std::unordered_map& propExprColMap, std::unordered_set& uniqueEdgeVertexCol) : vctx_(DCHECK_NOTNULL(vctx)), - srcAndEdgePropCols_(srcAndEdgePropCols), + srcPropCols_(srcPropCols), + edgePropCols_(edgePropCols), dstPropCols_(dstPropCols), inputPropCols_(inputPropCols), propExprColMap_(propExprColMap), @@ -30,7 +32,7 @@ void ExtractPropExprVisitor::visit(VertexExpression* expr) { const auto& colName = expr->name(); if (uniqueEdgeVertexCol_.emplace(colName).second) { if (colName == "$^") { - srcAndEdgePropCols_->addColumn(new YieldColumn(expr->clone(), colName)); + srcPropCols_->addColumn(new YieldColumn(expr->clone(), colName)); } else { dstPropCols_->addColumn(new YieldColumn(expr->clone(), colName)); } @@ -39,7 +41,7 @@ void ExtractPropExprVisitor::visit(VertexExpression* expr) { void ExtractPropExprVisitor::visit(EdgeExpression* expr) { if (uniqueEdgeVertexCol_.emplace(expr->toString()).second) { - srcAndEdgePropCols_->addColumn(new YieldColumn(expr->clone(), expr->toString())); + edgePropCols_->addColumn(new YieldColumn(expr->clone(), expr->toString())); } } @@ -171,7 +173,11 @@ void ExtractPropExprVisitor::visitVertexEdgePropExpr(PropertyExpression* expr) { auto newExpr = propExpr->clone(); auto col = new YieldColumn(newExpr, vctx_->anonColGen()->getCol()); propExprColMap_.emplace(propExpr->toString(), col); - srcAndEdgePropCols_->addColumn(col); + if (expr->kind() == Expression::Kind::kSrcProperty) { + srcPropCols_->addColumn(col); + } else { + edgePropCols_->addColumn(col); + } } } diff --git a/src/graph/visitor/ExtractPropExprVisitor.h b/src/graph/visitor/ExtractPropExprVisitor.h index 470d403d4d0..679b8216d47 100644 --- a/src/graph/visitor/ExtractPropExprVisitor.h +++ b/src/graph/visitor/ExtractPropExprVisitor.h @@ -17,7 +17,8 @@ class ValidateContext; class ExtractPropExprVisitor final : public ExprVisitorImpl { public: ExtractPropExprVisitor(ValidateContext *vctx, - YieldColumns *srcAndEdgePropCols, + YieldColumns *srcPropCols, + YieldColumns *edgePropCols, YieldColumns *dstPropCols, YieldColumns *inputPropCols, std::unordered_map &propExprColMap, @@ -68,7 +69,8 @@ class ExtractPropExprVisitor final : public ExprVisitorImpl { private: ValidateContext *vctx_{nullptr}; - YieldColumns *srcAndEdgePropCols_{nullptr}; + YieldColumns *srcPropCols_{nullptr}; + YieldColumns *edgePropCols_{nullptr}; YieldColumns *dstPropCols_{nullptr}; YieldColumns *inputPropCols_{nullptr}; std::unordered_map &propExprColMap_; diff --git a/tests/tck/features/bugfix/PredicationPushDownInGo.feature b/tests/tck/features/bugfix/PredicationPushDownInGo.feature index fce2f1a4896..f3b4f75dba3 100644 --- a/tests/tck/features/bugfix/PredicationPushDownInGo.feature +++ b/tests/tck/features/bugfix/PredicationPushDownInGo.feature @@ -20,11 +20,13 @@ Feature: Test predication push down in go | "Tim Duncan" | | "Tim Duncan" | And the execution plan should be: - | id | name | dependencies | operator info | - | 10 | Project | 15 | | - | 15 | InnerJoin | 17 | | - | 17 | Project | 18 | | - | 18 | GetNeighbors | 3 | { "filter": "(like._dst IN [\"Tim Duncan\"])" } | - | 3 | Project | 11 | | - | 11 | GetNeighbors | 0 | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 12 | Project | 16 | | + | 16 | HashInnerJoin | 5,17 | | + | 5 | Project | 13 | | + | 13 | ExpandAll | 2 | { "filter": "(like._dst NOT IN [\"xxx\"])" } | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 17 | ExpandAll | 8 | { "filter": "(like._dst IN [\"Tim Duncan\"])" } | + | 8 | Expand | 7 | | + | 7 | Argument | | | diff --git a/tests/tck/features/explain/ExplainAndProfile.feature b/tests/tck/features/explain/ExplainAndProfile.feature index 926094493e1..f22cc232c1d 100644 --- a/tests/tck/features/explain/ExplainAndProfile.feature +++ b/tests/tck/features/explain/ExplainAndProfile.feature @@ -80,12 +80,9 @@ Feature: Explain and Profile | count(*) | | 6 | And the execution plan should be: - | id | name | dependencies | profiling data | operator info | - | 7 | Aggregate | 6 | {"version":0, "rows": 1} | | - | 6 | Project | 5 | {"version":0, "rows": 6} | | - | 5 | GetNeighbors | 4 | {"version":0, "rows": 6, "resp[0]": {"vertices": 3}} | | - | 4 | Loop | 0 | [{"version":0, "rows": 1},{"version":1, "rows": 1},{"version":2, "rows": 1},{"version":3, "rows": 1}] | {"loopBody": "3"} | - | 3 | Dedup | 2 | [{"version":0, "rows": 2},{"version":1, "rows": 3},{"version":2, "rows": 3}] | | - | 2 | GetDstBySrc | 1 | [{"version":0, "rows": 2, "resp[0]": {"vertices": 2}},{"version":1, "rows": 3, "resp[0]":{"vertices": 3}}, {"version":2, "rows": 3, "resp[0]":{"vertices": 3}}] | | - | 1 | Start | | [{"version":0, "rows": 0},{"version":1, "rows": 0},{"version":2, "rows": 0}] | | - | 0 | Start | | {"version":0, "rows": 0} | | + | id | name | dependencies | profiling data | operator info | + | 5 | Aggregate | 4 | {"version":0, "rows": 1} | | + | 4 | Project | 3 | {"version":0, "rows": 6} | | + | 3 | ExpandAll | 2 | {"version":0, "rows": 6} | | + | 2 | Expand | 1 | | | + | 1 | Start | | | | diff --git a/tests/tck/features/go/GO.IntVid.feature b/tests/tck/features/go/GO.IntVid.feature index 213f4434f70..9582d020b0e 100644 --- a/tests/tck/features/go/GO.IntVid.feature +++ b/tests/tck/features/go/GO.IntVid.feature @@ -40,7 +40,6 @@ Feature: IntegerVid Go Sentence Then the result should be, in any order, with relax comparison, and the columns 0 should be hashed: | serve._dst | | "Spurs" | - | "Spurs" | When executing query: """ YIELD hash("Tim Duncan") as vid | GO FROM $-.vid OVER serve YIELD serve._dst diff --git a/tests/tck/features/go/GO.feature b/tests/tck/features/go/GO.feature index e75d22fdea3..cb0abc1099c 100644 --- a/tests/tck/features/go/GO.feature +++ b/tests/tck/features/go/GO.feature @@ -76,7 +76,6 @@ Feature: Go Sentence Then the result should be, in any order, with relax comparison: | serve._dst | | "Spurs" | - | "Spurs" | When executing query: """ YIELD "Tim Duncan" as vid | GO FROM $-.vid OVER serve YIELD serve._dst @@ -339,7 +338,7 @@ Feature: Go Sentence | serve._dst | Scenario: multi edges over all - When profiling query: + When executing query: """ GO FROM "Russell Westbrook" OVER * REVERSELY YIELD serve._dst, like._dst """ @@ -348,12 +347,7 @@ Feature: Go Sentence | EMPTY | "James Harden" | | EMPTY | "Dejounte Murray" | | EMPTY | "Paul George" | - And the execution plan should be: - | id | name | dependencies | operator info | - | 2 | Project | 1 | | - | 1 | GetNeighbors | 0 | {"edgeDirection": "IN_EDGE"} | - | 0 | Start | | | - When profiling query: + When executing query: """ GO FROM "Russell Westbrook" OVER * BIDIRECT YIELD serve._dst, like._dst """ @@ -365,11 +359,6 @@ Feature: Go Sentence | | "James Harden" | | | "Paul George" | | "Thunders" | | - And the execution plan should be: - | id | name | dependencies | operator info | - | 2 | Project | 1 | | - | 1 | GetNeighbors | 0 | {"edgeDirection": "BOTH"} | - | 0 | Start | | | When executing query: """ GO FROM "Russell Westbrook" OVER * REVERSELY YIELD serve._src, like._src diff --git a/tests/tck/features/go/GoYieldVertexEdge.feature b/tests/tck/features/go/GoYieldVertexEdge.feature index a1b2c68fe1b..cf5f52d6e1b 100644 --- a/tests/tck/features/go/GoYieldVertexEdge.feature +++ b/tests/tck/features/go/GoYieldVertexEdge.feature @@ -54,13 +54,6 @@ Feature: Go Yield Vertex And Edge Sentence | ("LaMarcus Aldridge" :player{age: 33, name: "LaMarcus Aldridge"}) | "teammate" | | ("Manu Ginobili" :player{age: 41, name: "Manu Ginobili"}) | "teammate" | | ("Tony Parker" :player{age: 36, name: "Tony Parker"}) | "teammate" | - | ("Manu Ginobili" :player{age: 41, name: "Manu Ginobili"}) | "like" | - | ("Tony Parker" :player{age: 36, name: "Tony Parker"}) | "like" | - | ("Spurs" :team{name: "Spurs"}) | "serve" | - | ("Danny Green" :player{age: 31, name: "Danny Green"}) | "teammate" | - | ("LaMarcus Aldridge" :player{age: 33, name: "LaMarcus Aldridge"}) | "teammate" | - | ("Manu Ginobili" :player{age: 41, name: "Manu Ginobili"}) | "teammate" | - | ("Tony Parker" :player{age: 36, name: "Tony Parker"}) | "teammate" | When executing query: """ YIELD "Tim Duncan" as vid | GO FROM $-.vid OVER serve YIELD $^ as src, $$ as dst, dst(edge) as id @@ -153,6 +146,17 @@ Feature: Go Yield Vertex And Edge Sentence """ Then the result should be, in any order, with relax comparison: | dst | + When executing query: + """ + GO FROM "Tim Duncan" OVER like, serve + WHERE serve.start_year > 1970 OR properties($$).age < 50 + YIELD $$ AS dst, edge AS e + """ + Then the result should be, in any order, with relax comparison: + | dst | e | + | ("Manu Ginobili" :player{age: 41, name: "Manu Ginobili"}) | [:like "Tim Duncan"->"Manu Ginobili" @0 {likeness: 95}] | + | ("Tony Parker" :player{age: 36, name: "Tony Parker"}) | [:like "Tim Duncan"->"Tony Parker" @0 {likeness: 95}] | + | ("Spurs" :team{name: "Spurs"}) | [:serve "Tim Duncan"->"Spurs" @0 {end_year: 2016, start_year: 1997}] | Scenario: In expression When executing query: diff --git a/tests/tck/features/go/SampleLimit.feature b/tests/tck/features/go/SampleLimit.feature index 8e0d818b0ec..dc20e0f7550 100644 --- a/tests/tck/features/go/SampleLimit.feature +++ b/tests/tck/features/go/SampleLimit.feature @@ -69,6 +69,7 @@ Feature: Sample and limit GO 3 STEPS FROM 'Tim Duncan' OVER like YIELD like._dst LIMIT [1, 2, 3] """ Then the result should be, in any order: - | like._dst | - | 'Manu Ginobili' | - | 'Tony Parker' | + | like._dst | + | /[\s\w+]/ | + | /[\s\w+]/ | + | /[\s\w+]/ | diff --git a/tests/tck/features/go/SampleLimit.intVid.feature b/tests/tck/features/go/SampleLimit.intVid.feature index 2cfe3e9b2d1..95ae13c6391 100644 --- a/tests/tck/features/go/SampleLimit.intVid.feature +++ b/tests/tck/features/go/SampleLimit.intVid.feature @@ -22,16 +22,16 @@ Feature: Sample and limit GO FROM hash('Tim Duncan') OVER like YIELD like._dst LIMIT [1] """ Then the result should be, in any order: - | like._dst | - | hash('Manu Ginobili') | + | like._dst | + | /[\d\-+]/ | When executing query: """ GO FROM hash('Tim Duncan') OVER like YIELD like._dst LIMIT [3] """ Then the result should be, in any order: - | like._dst | - | hash('Manu Ginobili') | - | hash('Tony Parker') | + | like._dst | + | /[\d\-+]/ | + | /[\d\-+]/ | When executing query: """ GO FROM hash('Tim Duncan') OVER like YIELD like._dst SAMPLE [1] @@ -44,9 +44,9 @@ Feature: Sample and limit GO FROM hash('Tim Duncan') OVER like YIELD like._dst SAMPLE [3] """ Then the result should be, in any order: - | like._dst | - | hash('Manu Ginobili') | - | hash('Tony Parker') | + | like._dst | + | /[\d\-+]/ | + | /[\d\-+]/ | Scenario: Sample Limit Go in Multiple steps When executing query: @@ -59,6 +59,7 @@ Feature: Sample and limit GO 3 STEPS FROM hash('Tim Duncan') OVER like YIELD like._dst LIMIT [1, 2, 3] """ Then the result should be, in any order: - | like._dst | - | hash('Manu Ginobili') | - | hash('Tony Parker') | + | like._dst | + | /[\d\-+]/ | + | /[\d\-+]/ | + | /[\d\-+]/ | diff --git a/tests/tck/features/go/SimpleCase.feature b/tests/tck/features/go/SimpleCase.feature index 9137d11651f..8a5bf1f8ba7 100644 --- a/tests/tck/features/go/SimpleCase.feature +++ b/tests/tck/features/go/SimpleCase.feature @@ -15,11 +15,13 @@ Feature: Simple case | count(*) | | 2 | And the execution plan should be: - | id | name | dependencies | operator info | - | 3 | Aggregate | 2 | | - | 2 | Dedup | 1 | | - | 1 | GetDstBySrc | 0 | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 6 | Aggregate | 5 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO FROM "Yao Ming" OVER like YIELD DISTINCT id($$) AS dst, $$.player.age AS age | ORDER BY $-.dst @@ -30,15 +32,16 @@ Feature: Simple case | "Tracy McGrady" | 39 | And the execution plan should be: | id | name | dependencies | operator info | - | 8 | Sort | 7 | | - | 7 | Project | 6 | | - | 6 | HashLeftJoin | 2, 5 | | - | 2 | Dedup | 1 | | - | 1 | GetDstBySrc | 0 | | - | 0 | Start | | | - | 5 | Project | 4 | | - | 4 | GetVertices | 3 | | - | 3 | Argument | | | + | 10 | Sort | 9 | | + | 9 | Dedup | 8 | | + | 8 | Project | 7 | | + | 7 | HashLeftJoin | 3, 6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO FROM "Yao Ming" OVER like WHERE $$.player.age > 40 YIELD DISTINCT id($$) AS dst, $$.player.age AS age | ORDER BY $-.dst @@ -48,16 +51,17 @@ Feature: Simple case | "Shaquille O'Neal" | 47 | And the execution plan should be: | id | name | dependencies | operator info | - | 9 | Sort | 8 | | - | 8 | Project | 7 | | - | 7 | Filter | 6 | | - | 6 | HashLeftJoin | 2, 5 | | - | 2 | Dedup | 1 | | - | 1 | GetDstBySrc | 0 | | - | 0 | Start | | | - | 5 | Project | 4 | | - | 4 | GetVertices | 3 | | - | 3 | Argument | | | + | 11 | Sort | 10 | | + | 10 | Dedup | 9 | | + | 9 | Project | 8 | | + | 8 | Filter | 7 | | + | 7 | HashLeftJoin | 3, 6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO FROM "Tony Parker" OVER like YIELD DISTINCT id($$) AS a | ORDER BY $-.a @@ -68,11 +72,13 @@ Feature: Simple case | "Manu Ginobili" | | "Tim Duncan" | And the execution plan should be: - | id | name | dependencies | operator info | - | 3 | Sort | 2 | | - | 2 | Dedup | 1 | | - | 1 | GetDstBySrc | 0 | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 6 | Sort | 5 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO FROM "Tony Parker" OVER like YIELD DISTINCT 2, id($$) AS a | ORDER BY $-.a @@ -83,12 +89,13 @@ Feature: Simple case | 2 | "Manu Ginobili" | | 2 | "Tim Duncan" | And the execution plan should be: - | id | name | dependencies | operator info | - | 4 | Sort | 3 | | - | 3 | Project | 3 | | - | 2 | Dedup | 1 | | - | 1 | GetDstBySrc | 0 | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 6 | Sort | 5 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | Scenario: go m steps When profiling query: @@ -99,15 +106,13 @@ Feature: Simple case | count(*) | | 22 | And the execution plan should be: - | id | name | dependencies | operator info | - | 7 | Aggregate | 6 | | - | 6 | Dedup | 5 | | - | 5 | GetDstBySrc | 4 | | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 6 | Aggregate | 5 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO 3 STEPS FROM "Tony Parker" OVER serve BIDIRECT WHERE $$.team.name != "Lakers" YIELD DISTINCT id($$) | YIELD count(*) @@ -116,21 +121,18 @@ Feature: Simple case | count(*) | | 21 | And the execution plan should be: - | id | name | dependencies | operator info | - | 13 | Aggregate | 12 | | - | 12 | Project | 11 | | - | 11 | Filter | 10 | | - | 10 | HashLeftJoin | 6,9 | | - | 6 | Dedup | 5 | | - | 5 | GetDstBySrc | 4 | | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | - | 9 | Project | 8 | | - | 8 | GetVertices | 7 | | - | 7 | Argument | | | + | id | name | dependencies | operator info | + | 11 | Aggregate | 10 | | + | 10 | Dedup | 9 | | + | 9 | Project | 8 | | + | 8 | Filter | 7 | | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | # The last step degenerates to `GetNeighbors` when the yield clause is not `YIELD DISTINCT id($$)` When profiling query: """ @@ -140,32 +142,12 @@ Feature: Simple case | count(*) | | 65 | And the execution plan should be: - | id | name | dependencies | operator info | - | 7 | Aggregate | 6 | | - | 6 | Project | 5 | | - | 5 | GetNeighbors | 4 | | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | - When profiling query: - """ - GO 3 STEPS FROM "Tony Parker" OVER serve BIDIRECT YIELD id($$) AS dst | YIELD count(*) - """ - Then the result should be, in any order, with relax comparison: - | count(*) | - | 65 | - And the execution plan should be: - | id | name | dependencies | operator info | - | 7 | Aggregate | 6 | | - | 6 | Project | 5 | | - | 5 | GetNeighbors | 4 | | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 5 | Aggregate | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO 3 STEPS FROM "Tony Parker" OVER serve BIDIRECT YIELD DISTINCT $$.team.name, id($$) AS dst | YIELD count(*) @@ -174,20 +156,17 @@ Feature: Simple case | count(*) | | 22 | And the execution plan should be: - | id | name | dependencies | operator info | - | 12 | Aggregate | 11 | | - | 11 | Project | 10 | | - | 10 | HashLeftJoin | 6, 9 | | - | 6 | Dedup | 5 | | - | 5 | GetDstBySrc | 4 | | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | - | 9 | Project | 8 | | - | 8 | GetVertices | 7 | | - | 7 | Argument | | | + | id | name | dependencies | operator info | + | 10 | Aggregate | 9 | | + | 9 | Dedup | 8 | | + | 8 | Project | 7 | | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO 3 STEPS FROM "Tony Parker" OVER serve BIDIRECT WHERE $^.player.age > 30 YIELD DISTINCT id($$) AS dst | YIELD count(*) @@ -196,16 +175,13 @@ Feature: Simple case | count(*) | | 22 | And the execution plan should be: - | id | name | dependencies | operator info | - | 9 | Aggregate | 8 | | - | 8 | Dedup | 7 | | - | 7 | Project | 10 | | - | 10 | GetNeighbors | 4 | | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 7 | Aggregate | 6 | | + | 6 | Dedup | 5 | | + | 5 | Project | 8 | | + | 8 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO 3 STEPS FROM "Tony Parker" OVER serve BIDIRECT YIELD $$.player.age AS age | YIELD count(*) @@ -214,20 +190,16 @@ Feature: Simple case | count(*) | | 65 | And the execution plan should be: - | id | name | dependencies | operator info | - | 12 | Aggregate | 11 | | - | 11 | Project | 10 | | - | 10 | HashLeftJoin | 6,9 | | - | 6 | Project | 5 | | - | 5 | GetNeighbors | 4 | | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | - | 9 | Project | 8 | | - | 8 | GetVertices | 7 | | - | 7 | Argument | | | + | id | name | dependencies | operator info | + | 9 | Aggregate | 8 | | + | 8 | Project | 7 | | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO 3 STEPS FROM "Tony Parker" OVER * WHERE $$.player.age > 36 YIELD $$.player.age AS age | YIELD count(*) @@ -236,21 +208,17 @@ Feature: Simple case | count(*) | | 10 | And the execution plan should be: - | id | name | dependencies | operator info | - | 13 | Aggregate | 12 | | - | 12 | Project | 11 | | - | 11 | Filter | 10 | | - | 10 | HashLeftJoin | 6,9 | | - | 6 | Project | 5 | | - | 5 | GetNeighbors | 4 | | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | - | 9 | Project | 8 | | - | 8 | GetVertices | 7 | | - | 7 | Argument | | | + | id | name | dependencies | operator info | + | 10 | Aggregate | 9 | | + | 9 | Project | 8 | | + | 8 | Filter | 7 | | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ YIELD "Tony Parker" as a | GO 3 STEPS FROM $-.a OVER serve BIDIRECT YIELD DISTINCT $$.team.name, id($$) AS dst | YIELD COUNT(*) @@ -259,22 +227,20 @@ Feature: Simple case | COUNT(*) | | 22 | And the execution plan should be: - | id | name | dependencies | operator info | - | 15 | Aggregate | 14 | | - | 14 | Project | 13 | | - | 13 | HashLeftJoin | 9,12 | | - | 9 | Dedup | 8 | | - | 8 | GetDstBySrc | 7 | | - | 7 | Loop | 6 | {"loopBody": "6"} | - | 6 | Dedup | 5 | | - | 5 | GetDstBySrc | 4 | | - | 4 | Start | | | - | 3 | Dedup | 16 | | - | 16 | Project | 0 | | - | 0 | Start | | | - | 12 | Project | 11 | | - | 11 | GetVertices | 10 | | - | 10 | Argument | | | + | id | name | dependencies | operator info | + | 13 | Aggregate | 12 | | + | 12 | Dedup | 11 | | + | 11 | Project | 10 | | + | 10 | HashInnerJoin | 1,9 | | + | 1 | Project | 0 | | + | 0 | Start | | | + | 9 | HashLeftJoin | 5,8 | | + | 5 | ExpandAll | 4 | | + | 4 | Expand | 3 | | + | 3 | Argument | | | + | 8 | Project | 7 | | + | 7 | GetVertices | 6 | | + | 6 | Argument | | | # Because GetDstBySrc doesn't support limit push down, so degenerate to GetNeighbors when the query contains limit/simple clause When profiling query: """ @@ -284,19 +250,13 @@ Feature: Simple case | count(*) | | 13 | And the execution plan should be: - | id | name | dependencies | operator info | - | 11 | Aggregate | 10 | | - | 10 | Dedup | 9 | | - | 9 | Project | 12 | | - | 12 | Limit | 13 | | - | 13 | GetNeighbors | 6 | | - | 6 | Loop | 0 | {"loopBody": "5"} | - | 5 | Dedup | 4 | | - | 4 | Project | 14 | | - | 14 | Limit | 15 | | - | 15 | GetNeighbors | 1 | | - | 1 | Start | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 6 | Aggregate | 5 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | Scenario: go m to n steps When profiling query: @@ -307,14 +267,13 @@ Feature: Simple case | count(*) | | 41 | And the execution plan should be: - | id | name | dependencies | operator info | - | 6 | Aggregate | 5 | | - | 5 | DataCollect | 4 | | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 6 | Aggregate | 5 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO 1 to 3 STEP FROM "Tony Parker" OVER like WHERE $$.player.age > 40 YIELD DISTINCT id($$), $$.player.age as age, $$.player.name | ORDER BY $-.age @@ -324,21 +283,18 @@ Feature: Simple case | "Manu Ginobili" | 41 | "Manu Ginobili" | | "Tim Duncan" | 42 | "Tim Duncan" | And the execution plan should be: - | id | name | dependencies | operator info | - | 13 | Sort | 12 | | - | 12 | DataCollect | 11 | | - | 11 | Loop | 0 | {"loopBody": "10"} | - | 10 | Project | 9 | | - | 9 | Filter | 8 | | - | 8 | HashLeftJoin | 4,7 | | - | 4 | Project | 3 | | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 7 | Project | 6 | | - | 6 | GetVertices | 5 | | - | 5 | Argument | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 11 | Sort | 10 | | + | 10 | Dedup | 9 | | + | 9 | Project | 8 | | + | 8 | Filter | 7 | | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO 1 to 3 STEPS FROM "Tony Parker" OVER serve BIDIRECT YIELD DISTINCT 3, id($$) AS dst | YIELD count(*) @@ -347,15 +303,13 @@ Feature: Simple case | count(*) | | 41 | And the execution plan should be: - | id | name | dependencies | operator info | - | 7 | Aggregate | 6 | | - | 6 | DataCollect | 5 | | - | 5 | Loop | 0 | {"loopBody": "4"} | - | 4 | Project | 3 | | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 6 | Aggregate | 5 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO 1 to 3 STEPS FROM "Tony Parker" OVER serve BIDIRECT YIELD DISTINCT $$.player.age AS age, id($$) | YIELD COUNT($-.age) @@ -364,20 +318,17 @@ Feature: Simple case | COUNT($-.age) | | 19 | And the execution plan should be: - | id | name | dependencies | operator info | - | 12 | Aggregate | 11 | | - | 11 | DataCollect | 10 | | - | 10 | Loop | 9 | {"loopBody": "9"} | - | 9 | Project | 8 | | - | 8 | HashLeftJoin | 4,7 | | - | 4 | Project | 3 | | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 7 | Project | 6 | | - | 6 | GetVertices | 5 | | - | 5 | Argument | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 10 | Aggregate | 9 | | + | 9 | Dedup | 8 | | + | 8 | Project | 7 | | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO 1 to 8 steps FROM "Tony Parker" OVER serve, like YIELD distinct like._dst AS a | YIELD COUNT($-.a) @@ -386,17 +337,13 @@ Feature: Simple case | COUNT($-.a) | | 4 | And the execution plan should be: - | id | name | dependencies | operator info | - | 9 | Aggregate | 8 | | - | 8 | DataCollect | 7 | | - | 7 | Loop | 0 | {"loopBody": "6"} | - | 6 | Dedup | 5 | | - | 5 | Project | 4 | | - | 4 | Dedup | 3 | | - | 3 | Project | 2 | | - | 2 | GetNeighbors | 1 | | - | 1 | Start | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 6 | Aggregate | 5 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO 1 to 8 steps FROM "Tony Parker" OVER serve, like YIELD DISTINCT serve._dst AS a | YIELD COUNT($-.a) @@ -405,17 +352,13 @@ Feature: Simple case | COUNT($-.a) | | 3 | And the execution plan should be: - | id | name | dependencies | operator info | - | 9 | Aggregate | 8 | | - | 8 | DataCollect | 7 | | - | 7 | Loop | 0 | {"loopBody": "6"} | - | 6 | Dedup | 5 | | - | 5 | Project | 4 | | - | 4 | Dedup | 3 | | - | 3 | Project | 2 | | - | 2 | GetNeighbors | 1 | | - | 1 | Start | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 6 | Aggregate | 5 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | Scenario: k-hop neighbors When profiling query: @@ -426,20 +369,23 @@ Feature: Simple case | count(*) | | 28 | And the execution plan should be: - | id | name | dependencies | operator info | - | 14 | Aggregate | 12 | | - | 12 | Minus | 10,11 | | - | 10 | Project | 13 | | - | 13 | PassThrough | 9 | | - | 9 | Dedup | 15 | | - | 15 | GetDstBySrc | 5 | | - | 5 | DataCollect | 4 | | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | - | 11 | Project | 13 | | + | id | name | dependencies | operator info | + | 17 | Aggregate | 15 | | + | 15 | Minus | 13,14 | | + | 13 | Project | 16 | | + | 16 | PassThrough | 12 | | + | 12 | Dedup | 11 | | + | 11 | Project | 10 | | + | 10 | HashInnerJoin | 5,9 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 9 | ExpandAll | 8 | | + | 8 | Expand | 7 | | + | 7 | Argument | | | + | 14 | Project | 16 | | Scenario: other simple case When profiling query: @@ -450,15 +396,19 @@ Feature: Simple case | count(*) | | 0 | And the execution plan should be: - | id | name | dependencies | operator info | - | 7 | Aggregate | 6 | | - | 6 | Dedup | 5 | | - | 5 | GetDstBySrc | 4 | | - | 4 | Dedup | 3 | | - | 3 | Project | 2 | | - | 2 | Dedup | 1 | | - | 1 | GetDstBySrc | 0 | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 13 | Aggregate | 12 | | + | 12 | Dedup | 11 | | + | 11 | Project | 10 | | + | 10 | HashInnerJoin | 5,9 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 9 | ExpandAll | 8 | | + | 8 | Expand | 7 | | + | 7 | Argument | | | When profiling query: """ GO 1 STEP FROM "Tony Parker" OVER * YIELD distinct id($$) as id| GO 3 STEP FROM $-.id OVER * YIELD distinct id($$) | YIELD COUNT(*) @@ -467,19 +417,19 @@ Feature: Simple case | COUNT(*) | | 22 | And the execution plan should be: - | id | name | dependencies | operator info | - | 11 | Aggregate | 10 | | - | 10 | Dedup | 9 | | - | 9 | GetDstBySrc | 8 | | - | 8 | Loop | 4 | {"loopBody": "7"} | - | 7 | Dedup | 6 | | - | 6 | GetDstBySrc | 5 | | - | 5 | Start | | | - | 4 | Dedup | 3 | | - | 3 | Project | 2 | | - | 2 | Dedup | 1 | | - | 1 | GetDstBySrc | 0 | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 13 | Aggregate | 12 | | + | 12 | Dedup | 11 | | + | 11 | Project | 10 | | + | 10 | HashInnerJoin | 5,9 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 9 | ExpandAll | 8 | | + | 8 | Expand | 7 | | + | 7 | Argument | | | Scenario: could not be optimied cases When profiling query: @@ -491,17 +441,17 @@ Feature: Simple case | "Shaquille O'Neal" | "Shaquille O'Neal" | 147 | And the execution plan should be: | id | name | dependencies | operator info | - | 10 | Sort | 9 | | - | 9 | Dedup | 8 | | - | 8 | Project | 7 | | - | 7 | Filter | 6 | | - | 6 | HashLeftJoin | 2,5 | | - | 2 | Project | 1 | | - | 1 | GetNeighbors | 0 | | - | 0 | Start | | | - | 5 | Project | 4 | | - | 4 | GetVertices | 3 | | - | 3 | Argument | | | + | 11 | Sort | 10 | | + | 10 | Dedup | 9 | | + | 9 | Project | 8 | | + | 8 | Filter | 7 | | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO FROM "Tony Parker" OVER like WHERE like._dst != "Tim Duncan" YIELD DISTINCT id($$), 2, like._dst AS a | ORDER BY $-.a @@ -511,12 +461,13 @@ Feature: Simple case | "LaMarcus Aldridge" | 2 | "LaMarcus Aldridge" | | "Manu Ginobili" | 2 | "Manu Ginobili" | And the execution plan should be: - | id | name | dependencies | operator info | - | 5 | Sort | 4 | | - | 4 | Dedup | 3 | | - | 3 | Project | 6 | | - | 6 | GetNeighbors | 0 | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 7 | Sort | 6 | | + | 6 | Dedup | 5 | | + | 5 | Project | 8 | | + | 8 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO 3 STEPS FROM "Tony Parker" OVER serve BIDIRECT WHERE id($$) != "Not exists" YIELD DISTINCT id($$), $$.player.age | YIELD count(*) @@ -525,22 +476,18 @@ Feature: Simple case | count(*) | | 22 | And the execution plan should be: - | id | name | dependencies | operator info | - | 14 | Aggregate | 13 | | - | 13 | Dedup | 12 | | - | 12 | Project | 11 | | - | 11 | Filter | 10 | | - | 10 | HashLeftJoin | 6,9 | | - | 6 | Project | 5 | | - | 5 | GetNeighbors | 4 | | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | - | 1 | Start | | | - | 0 | Start | | | - | 9 | Project | 8 | | - | 8 | GetVertices | 7 | | - | 7 | Argument | | | + | id | name | dependencies | operator info | + | 11 | Aggregate | 10 | | + | 10 | Dedup | 9 | | + | 9 | Project | 8 | | + | 8 | Filter | 7 | | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO FROM "Tony Parker" OVER serve, like WHERE serve._dst !="abc" YIELD DISTINCT id($$) AS a | ORDER BY $-.a @@ -553,12 +500,13 @@ Feature: Simple case | "Spurs" | | "Tim Duncan" | And the execution plan should be: - | id | name | dependencies | operator info | - | 5 | Sort | 4 | | - | 4 | Dedup | 3 | | - | 3 | Project | 6 | | - | 6 | GetNeighbors | 0 | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 7 | Sort | 6 | | + | 6 | Dedup | 5 | | + | 5 | Project | 8 | | + | 8 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO 1 STEP FROM "Tony Parker" OVER like, serve REVERSELY WHERE id($$) != "Tim Duncan" YIELD DISTINCT id($$) | YIELD count(*) @@ -568,17 +516,17 @@ Feature: Simple case | 4 | And the execution plan should be: | id | name | dependencies | operator info | - | 10 | Aggregate | 9 | | - | 9 | Dedup | 8 | | - | 8 | Project | 7 | | - | 7 | Filter | 6 | | - | 6 | HashLeftJoin | 2,5 | | - | 2 | Project | 1 | | - | 1 | GetNeighbors | 0 | | - | 0 | Start | | | - | 5 | Project | 4 | | - | 4 | GetVertices | 3 | | - | 3 | Argument | | | + | 11 | Aggregate | 10 | | + | 10 | Dedup | 9 | | + | 9 | Project | 8 | | + | 8 | Filter | 7 | | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO 1 to 3 STEPS FROM "Tony Parker" OVER like WHERE like._dst != "Yao Ming" YIELD DISTINCT id($$) AS a | ORDER BY $-.a @@ -590,18 +538,14 @@ Feature: Simple case | "Tim Duncan" | | "Tony Parker" | And the execution plan should be: - | id | name | dependencies | operator info | - | 10 | Sort | 9 | | - | 9 | DataCollect | 8 | | - | 8 | Loop | 0 | {"loopBody": "7"} | - | 7 | Dedup | 6 | | - | 6 | Project | 5 | | - | 5 | Filter | 4 | | - | 4 | Dedup | 3 | | - | 3 | Project | 2 | | - | 2 | GetNeighbors | 1 | | - | 1 | Start | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 7 | Sort | 6 | | + | 6 | Dedup | 5 | | + | 5 | Project | 4 | | + | 4 | Filter | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO 1 to 3 STEP FROM "Tony Parker" OVER like WHERE id($$) != "Tim Duncan" YIELD DISTINCT id($$), $$.player.age as age, $$.player.name | ORDER BY $-.age @@ -612,23 +556,18 @@ Feature: Simple case | "Tony Parker" | 36 | "Tony Parker" | | "Manu Ginobili" | 41 | "Manu Ginobili" | And the execution plan should be: - | id | name | dependencies | operator info | - | 15 | Sort | 14 | | - | 14 | DataCollect | 13 | | - | 13 | Loop | 0 | {"loopBody": "12"} | - | 12 | Dedup | 11 | | - | 11 | Project | 10 | | - | 10 | Filter | 9 | | - | 9 | HashLeftJoin | 5,8 | | - | 5 | Project | 4 | | - | 4 | Dedup | 3 | | - | 3 | Project | 2 | | - | 2 | GetNeighbors | 1 | | - | 1 | Start | | | - | 8 | Project | 7 | | - | 7 | GetVertices | 6 | | - | 6 | Argument | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 11 | Sort | 10 | | + | 10 | Dedup | 9 | | + | 9 | Project | 8 | | + | 8 | Filter | 7 | | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO FROM "Yao Ming" OVER like YIELD DISTINCT id($$) AS aa | GO 1 to 3 STEP FROM $-.aa OVER like WHERE id($$) != "Tim Duncan" YIELD DISTINCT id($$), $$.player.age as age, $$.player.name | ORDER BY $-.age @@ -644,33 +583,21 @@ Feature: Simple case | "Manu Ginobili" | 41 | "Manu Ginobili" | | "Grant Hill" | 46 | "Grant Hill" | And the execution plan should be: - | id | name | dependencies | operator info | - | 28 | Sort | 27 | | - | 27 | DataCollect | 26 | | - | 26 | Loop | 10 | {"loopBody": "25"} | - | 25 | Dedup | 24 | | - | 24 | Project | 23 | | - | 23 | Filter | 22 | | - | 22 | InnerJoin | 21 | | - | 21 | InnerJoin | 20 | | - | 20 | HashLeftJoin | 16,19 | | - | 16 | Project | 15 | | - | 15 | Dedup | 14 | | - | 14 | Project | 13 | | - | 13 | InnerJoin | 12 | | - | 12 | Dedup | 11 | | - | 11 | Project | 8 | | - | 8 | Dedup | 7 | | - | 7 | Project | 6 | | - | 6 | GetNeighbors | 5 | | - | 5 | Start | | | - | 19 | Project | 18 | | - | 18 | GetVertices | 17 | | - | 17 | Argument | | | - | 10 | Dedup | 9 | | - | 9 | Project | 4 | | - | 4 | Dedup | 3 | | - | 3 | Project | 2 | | - | 2 | Dedup | 1 | | - | 1 | GetDstBySrc | 0 | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 18 | Sort | 17 | | + | 17 | Dedup | 16 | | + | 16 | Project | 21 | | + | 21 | HashInnerJoin | 5,20 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 20 | Filter | 19 | | + | 19 | HashLeftJoin | 9,12 | | + | 9 | ExpandAll | 8 | | + | 8 | Expand | 7 | | + | 7 | Argument | | | + | 12 | Project | 11 | | + | 11 | GetVertices | 10 | | + | 10 | Argument | | | diff --git a/tests/tck/features/optimizer/PushFilterDownGetNbrsRule.feature b/tests/tck/features/optimizer/PushFilterDownExpandAllRule.feature similarity index 63% rename from tests/tck/features/optimizer/PushFilterDownGetNbrsRule.feature rename to tests/tck/features/optimizer/PushFilterDownExpandAllRule.feature index 5988b56d0dd..5394cd04b68 100644 --- a/tests/tck/features/optimizer/PushFilterDownGetNbrsRule.feature +++ b/tests/tck/features/optimizer/PushFilterDownExpandAllRule.feature @@ -1,7 +1,7 @@ # Copyright (c) 2021 vesoft inc. All rights reserved. # # This source code is licensed under Apache 2.0 License. -Feature: Push Filter down GetNeighbors rule +Feature: Push Filter down ExpandAll rule Background: Given a graph with space named "nba" @@ -21,10 +21,11 @@ Feature: Push Filter down GetNeighbors rule | 2012 | | 2016 | And the execution plan should be: - | id | name | dependencies | operator info | - | 0 | Project | 1 | | - | 1 | GetNeighbors | 2 | {"filter": "($^.player.age>18)"} | - | 2 | Start | | | + | id | name | dependencies | operator info | + | 5 | Project | 6 | | + | 6 | ExpandAll | 2 | {"filter": "($^.player.age>18)"} | + | 2 | Expand | 1 | | + | 1 | Start | | | Scenario: push start vertex filter down when reversely When profiling query: @@ -39,10 +40,11 @@ Feature: Push Filter down GetNeighbors rule | 90 | | 99 | And the execution plan should be: - | id | name | dependencies | operator info | - | 0 | Project | 1 | | - | 1 | GetNeighbors | 2 | {"filter": "($^.player.age>18)"} | - | 2 | Start | | | + | id | name | dependencies | operator info | + | 5 | Project | 6 | | + | 6 | ExpandAll | 2 | {"filter": "($^.player.age>18)"} | + | 2 | Expand | 1 | | + | 1 | Start | | | Scenario: push edge props filter down When profiling query: @@ -57,10 +59,11 @@ Feature: Push Filter down GetNeighbors rule | 2012 | | 2016 | And the execution plan should be: - | id | name | dependencies | operator info | - | 0 | Project | 1 | | - | 1 | GetNeighbors | 2 | {"filter": "(serve.start_year>2005)"} | - | 2 | Start | | | + | id | name | dependencies | operator info | + | 5 | Project | 6 | | + | 6 | ExpandAll | 2 | {"filter": "(serve.start_year>2005)"} | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO FROM "Tony Parker" OVER like @@ -72,10 +75,11 @@ Feature: Push Filter down GetNeighbors rule | "Manu Ginobili" | 95 | | "Tim Duncan" | 95 | And the execution plan should be: - | id | name | dependencies | operator info | - | 0 | Project | 1 | | - | 1 | GetNeighbors | 2 | {"filter": "(like.likeness IN [__VAR_0 IN [95,99] WHERE ($__VAR_0>0)])"} | - | 2 | Start | | | + | id | name | dependencies | operator info | + | 5 | Project | 6 | | + | 6 | ExpandAll | 2 | {"filter": "(like.likeness IN [__VAR_0 IN [95,99] WHERE ($__VAR_0>0)])"} | + | 2 | Expand | 1 | | + | 1 | Start | | | When profiling query: """ GO FROM "Tony Parker" OVER like @@ -86,10 +90,11 @@ Feature: Push Filter down GetNeighbors rule | like._dst | like.likeness | | "LaMarcus Aldridge" | 90 | And the execution plan should be: - | id | name | dependencies | operator info | - | 0 | Project | 1 | | - | 1 | GetNeighbors | 2 | {"filter": "any(__VAR_0 IN [5,6] WHERE ((like.likeness+$__VAR_0)<100))"} | - | 2 | Start | | | + | id | name | dependencies | operator info | + | 5 | Project | 6 | | + | 6 | ExpandAll | 2 | {"filter": "any(__VAR_0 IN [5,6] WHERE ((like.likeness+$__VAR_0)<100))"} | + | 2 | Expand | 1 | | + | 1 | Start | | | Scenario: push edge props filter down when reversely When profiling query: @@ -106,10 +111,11 @@ Feature: Push Filter down GetNeighbors rule | 2008 | | 2012 | And the execution plan should be: - | id | name | dependencies | operator info | - | 0 | Project | 1 | | - | 1 | GetNeighbors | 2 | {"filter": "(serve.start_year<2017)"} | - | 2 | Start | | | + | id | name | dependencies | operator info | + | 5 | Project | 6 | | + | 6 | ExpandAll | 2 | {"filter": "(serve.start_year<2017)"} | + | 2 | Expand | 1 | | + | 1 | Start | | | @skip Scenario: Only push source vertex filter down diff --git a/tests/tck/features/optimizer/PushFilterDownHashInnerJoinRule.feature b/tests/tck/features/optimizer/PushFilterDownHashInnerJoinRule.feature index 4e0b3c36732..b4916b86299 100644 --- a/tests/tck/features/optimizer/PushFilterDownHashInnerJoinRule.feature +++ b/tests/tck/features/optimizer/PushFilterDownHashInnerJoinRule.feature @@ -221,6 +221,29 @@ Feature: Push Filter down HashInnerJoin rule | 9 | AppendVertices | 8 | | | 8 | Traverse | 7 | | | 7 | Argument | | | + When profiling query: + """ + LOOKUP ON player WHERE player.name == "Tony Parker" + YIELD id(vertex) as id | + GO FROM $-.id OVER like + WHERE (like.likeness - 1) >= 0 + YIELD like._src AS src_id, like._dst AS dst_id, like.likeness AS likeness + """ + Then the result should be, in any order: + | src_id | dst_id | likeness | + | "Tony Parker" | "LaMarcus Aldridge" | 90 | + | "Tony Parker" | "Manu Ginobili" | 95 | + | "Tony Parker" | "Tim Duncan" | 95 | + And the execution plan should be: + | id | name | dependencies | operator info | + | 10 | Project | 14 | | + | 14 | HashInnerJoin | 3,15 | | + | 3 | Project | 11 | | + | 11 | TagIndexPrefixScan | 0 | | + | 0 | Start | | | + | 15 | ExpandAll | 6 | {"filter": "((like.likeness-1)>=0)"} | + | 6 | Expand | 5 | | + | 5 | Argument | | | Scenario: NOT push filter down HashInnerJoin When profiling query: diff --git a/tests/tck/features/optimizer/PushFilterDownHashLeftJoinRule.feature b/tests/tck/features/optimizer/PushFilterDownHashLeftJoinRule.feature index 9ddb989f02d..c5a7ef3995d 100644 --- a/tests/tck/features/optimizer/PushFilterDownHashLeftJoinRule.feature +++ b/tests/tck/features/optimizer/PushFilterDownHashLeftJoinRule.feature @@ -19,14 +19,14 @@ Feature: Push Filter down HashLeftJoin rule | ("Tim Duncan" :bachelor{name: "Tim Duncan", speciality: "psychology"} :player{age: 42, name: "Tim Duncan"}) | [:like "Tony Parker"->"Tim Duncan" @0 {likeness: 95}] | And the execution plan should be: | id | name | dependencies | operator info | - | 8 | Project | 11 | | - | 11 | HashLeftJoin | 13,5 | | - | 13 | Project | 14 | | - | 14 | GetNeighbors | 0 | {"filter": "(like.likeness>90)"} | - | 0 | Start | | | - | 5 | Project | 4 | | - | 4 | GetVertices | 3 | | - | 3 | Argument | | | + | 9 | Project | 12 | | + | 12 | HashLeftJoin | 13,6 | | + | 13 | ExpandAll | 2 | {"filter": "(like.likeness>90)"} | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO 2 STEPS FROM 'Tony Parker' OVER like @@ -39,18 +39,14 @@ Feature: Push Filter down HashLeftJoin rule | ("Tony Parker" :player{age: 36, name: "Tony Parker"}) | [:like "Tim Duncan"->"Tony Parker" @0 {likeness: 95}] | And the execution plan should be: | id | name | dependencies | operator info | - | 12 | Project | 15 | | - | 15 | HashLeftJoin | 17,9 | | - | 17 | Project | 18 | | - | 18 | GetNeighbors | 4 | {"filter": "(like.likeness>90)"} | - | 4 | Loop | 0 | {"loopBody": "3"} | - | 3 | Dedup | 2 | | - | 2 | GetDstBySrc | 1 | | + | 9 | Project | 12 | | + | 12 | HashLeftJoin | 13,6 | | + | 13 | ExpandAll | 2 | {"filter": "(like.likeness>90)"} | + | 2 | Expand | 1 | | | 1 | Start | | | - | 0 | Start | | | - | 9 | Project | 8 | | - | 8 | GetVertices | 7 | | - | 7 | Argument | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ $hop1 = GO FROM 'Tony Parker' OVER * BIDIRECT YIELD DISTINCT id($$) as dst; @@ -66,20 +62,23 @@ Feature: Push Filter down HashLeftJoin rule | "Boris Diaw" | "Spurs" | ["team"] | | "Boris Diaw" | "Suns" | ["team"] | And the execution plan should be: - | id | name | dependencies | operator info | - | 19 | TopN | 14 | | - | 14 | Dedup | 13 | | - | 13 | Project | 21 | | - | 21 | InnerJoin | 24 | | - | 24 | HashLeftJoin | 26,9 | | - | 26 | Project | 27 | | - | 27 | GetNeighbors | 2 | {"filter": "((like.likeness>80) OR like.likeness IS EMPTY)"} | - | 2 | Dedup | 1 | | - | 1 | GetDstBySrc | 0 | | - | 0 | Start | | | - | 9 | Project | 8 | | - | 8 | GetVertices | 7 | | - | 7 | Argument | | | + | id | name | dependencies | operator info | + | 20 | TopN | 17 | | + | 17 | Dedup | 16 | | + | 16 | Project | 23 | | + | 23 | HashInnerJoin | 5,26 | | + | 5 | Dedup | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 26 | HashLeftJoin | 27,12 | | + | 27 | ExpandAll | 8 | {"filter": "((like.likeness>80) OR like.likeness IS EMPTY)"} | + | 8 | Expand | 7 | | + | 7 | Argument | | | + | 12 | Project | 11 | | + | 11 | GetVertices | 10 | | + | 10 | Argument | | | Scenario: NOT push filter down HashLeftJoin When profiling query: @@ -94,16 +93,16 @@ Feature: Push Filter down HashLeftJoin rule | ("Manu Ginobili" :player{age: 41, name: "Manu Ginobili"}) | [:like "Tony Parker"->"Manu Ginobili" @0 {likeness: 95}] | | ("Tim Duncan" :bachelor{name: "Tim Duncan", speciality: "psychology"} :player{age: 42, name: "Tim Duncan"}) | [:like "Tony Parker"->"Tim Duncan" @0 {likeness: 95}] | And the execution plan should be: - | id | name | dependencies | operator info | - | 8 | Project | 7 | | - | 7 | Filter | 6 | | - | 6 | HashLeftJoin | 2,5 | | - | 2 | Project | 1 | | - | 1 | GetNeighbors | 0 | | - | 0 | Start | | | - | 5 | Project | 4 | | - | 4 | GetVertices | 3 | | - | 3 | Argument | | | + | id | name | dependencies | operator info | + | 9 | Project | 8 | | + | 8 | Filter | 7 | {"condition": "(($__COL_0>90) OR ($__COL_1>30))"} | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ GO FROM "Tony Parker" OVER like @@ -117,15 +116,15 @@ Feature: Push Filter down HashLeftJoin rule | "Tim Duncan" | 95 | 42 | And the execution plan should be: | id | name | dependencies | operator info | - | 8 | Project | 12 | | - | 12 | Filter | 11 | {"condition" : "($__COL_0>=32)"} | - | 11 | HashLeftJoin | 14,5 | | - | 14 | Project | 15 | | - | 15 | GetNeighbors | 0 | | - | 0 | Start | | | - | 5 | Project | 4 | | - | 4 | GetVertices | 3 | | - | 3 | Argument | | | + | 9 | Project | 8 | | + | 8 | Filter | 7 | {"condition": "($__COL_0>=32)"} | + | 7 | HashLeftJoin | 3,6 | | + | 3 | ExpandAll | 2 | {"filter": "(like.likeness>85)"} | + | 2 | Expand | 1 | | + | 1 | Start | | | + | 6 | Project | 5 | | + | 5 | GetVertices | 4 | | + | 4 | Argument | | | When profiling query: """ LOOKUP ON player WHERE player.name=='Tim Duncan' YIELD id(vertex) as id @@ -141,24 +140,26 @@ Feature: Push Filter down HashLeftJoin rule | "Tim Duncan" | And the execution plan should be: | id | name | dependencies | operator info | - | 26 | Project | 36 | | - | 36 | InnerJoin | 35 | | - | 35 | Filter | 23 | | - | 23 | HashLeftJoin | 19,20 | | - | 19 | Project | 32 | | - | 32 | GetNeighbors | 15 | | - | 15 | Project | 34 | | - | 34 | InnerJoin | 33 | | - | 33 | Filter | 12 | | - | 12 | HashLeftJoin | 8, 11 | | - | 8 | Project | 31 | | - | 31 | GetNeighbors | 28 | | + | 26 | Project | 34 | | + | 34 | HashInnerJoin | 15,33 | | + | 15 | Project | 31 | | + | 31 | HashInnerJoin | 28,30 | | | 28 | Project | 27 | | | 27 | TagIndexPrefixScan | 0 | | | 0 | Start | | | + | 30 | Filter | 29 | | + | 29 | HashLeftJoin | 8,11 | | + | 8 | ExpandAll | 7 | | + | 7 | Expand | 6 | | + | 6 | Argument | | | | 11 | Project | 10 | | | 10 | GetVertices | 9 | | | 9 | Argument | | | + | 33 | Filter | 32 | | + | 32 | HashLeftJoin | 19,22 | | + | 19 | ExpandAll | 18 | | + | 18 | Expand | 17 | | + | 17 | Argument | | | | 22 | Project | 21 | | | 21 | GetVertices | 20 | | | 20 | Argument | | | diff --git a/tests/tck/features/optimizer/PushFilterDownInnerJoinRule.feature b/tests/tck/features/optimizer/PushFilterDownInnerJoinRule.feature deleted file mode 100644 index c74487c6741..00000000000 --- a/tests/tck/features/optimizer/PushFilterDownInnerJoinRule.feature +++ /dev/null @@ -1,31 +0,0 @@ -# Copyright (c) 2021 vesoft inc. All rights reserved. -# -# This source code is licensed under Apache 2.0 License. -Feature: Push Filter down InnerJoin rule - - Background: - Given a graph with space named "nba" - - Scenario: push filter down InnerJoin - When profiling query: - """ - LOOKUP ON player WHERE player.name == "Tony Parker" - YIELD id(vertex) as id | - GO FROM $-.id OVER like - WHERE (like.likeness - 1) >= 0 - YIELD like._src AS src_id, like._dst AS dst_id, like.likeness AS likeness - """ - Then the result should be, in any order: - | src_id | dst_id | likeness | - | "Tony Parker" | "LaMarcus Aldridge" | 90 | - | "Tony Parker" | "Manu Ginobili" | 95 | - | "Tony Parker" | "Tim Duncan" | 95 | - And the execution plan should be: - | id | name | dependencies | operator info | - | 10 | Project | 15 | | - | 15 | InnerJoin | 17 | | - | 17 | Project | 18 | | - | 18 | GetNeighbors | 3 | | - | 3 | Project | 11 | | - | 11 | TagIndexPrefixScan | 0 | | - | 0 | Start | | | diff --git a/tests/tck/features/optimizer/PushLimitDownExpandAllRule.feature b/tests/tck/features/optimizer/PushLimitDownExpandAllRule.feature new file mode 100644 index 00000000000..991d2c24a92 --- /dev/null +++ b/tests/tck/features/optimizer/PushLimitDownExpandAllRule.feature @@ -0,0 +1,75 @@ +# Copyright (c) 2021 vesoft inc. All rights reserved. +# +# This source code is licensed under Apache 2.0 License. +Feature: Push Limit down rule + + Background: + Given a graph with space named "nba" + + Scenario: push limit down to ExpandAll + When profiling query: + """ + GO 1 STEPS FROM "James Harden" OVER like REVERSELY YIELD like._dst | Limit 2 + """ + Then the result should be, in any order: + | like._dst | + | "Dejounte Murray" | + | "Luka Doncic" | + And the execution plan should be: + | id | name | dependencies | operator info | + | 7 | Project | 8 | | + | 8 | Limit | 9 | | + | 9 | ExpandAll | 2 | {"limit": "2"} | + | 2 | Expand | 1 | | + | 1 | Start | | | + When profiling query: + """ + GO FROM "James Harden" OVER * YIELD id($$) as dst | Limit 2 + """ + Then the result should be, in any order: + | dst | + | "Russell Westbrook" | + | "Rockets" | + And the execution plan should be: + | id | name | dependencies | operator info | + | 7 | Project | 8 | | + | 8 | Limit | 9 | | + | 9 | ExpandAll | 2 | {"limit": "2"} | + | 2 | Expand | 1 | | + | 1 | Start | | | + When profiling query: + """ + GO FROM "James Harden" OVER * YIELD id($^) as src | Limit 2 + """ + Then the result should be, in any order: + | src | + | "James Harden" | + | "James Harden" | + And the execution plan should be: + | id | name | dependencies | operator info | + | 7 | Project | 8 | | + | 8 | Limit | 9 | | + | 9 | ExpandAll | 2 | {"limit": "2"} | + | 2 | Expand | 1 | | + | 1 | Start | | | + + Scenario: push limit down to expand with offset + When profiling query: + """ + GO 1 STEPS FROM "Vince Carter" OVER serve + YIELD serve.start_year as start_year | + Limit 3, 4 + """ + Then the result should be, in any order: + | start_year | + | 2009 | + | 2011 | + | 2004 | + | 1998 | + And the execution plan should be: + | id | name | dependencies | operator info | + | 7 | Project | 8 | | + | 8 | Limit | 9 | | + | 9 | ExpandAll | 2 | {"limit": "7"} | + | 2 | Expand | 1 | | + | 1 | Start | | | diff --git a/tests/tck/features/optimizer/PushLimitDownGetNeighborsRule.feature b/tests/tck/features/optimizer/PushLimitDownGetNeighborsRule.feature deleted file mode 100644 index e60f0168e8e..00000000000 --- a/tests/tck/features/optimizer/PushLimitDownGetNeighborsRule.feature +++ /dev/null @@ -1,72 +0,0 @@ -# Copyright (c) 2021 vesoft inc. All rights reserved. -# -# This source code is licensed under Apache 2.0 License. -Feature: Push Limit down rule - - Background: - Given a graph with space named "nba" - - Scenario: push limit down to GetNeighbors - When profiling query: - """ - GO 1 STEPS FROM "James Harden" OVER like REVERSELY YIELD like._dst | - Limit 2 - """ - Then the result should be, in any order: - | like._dst | - | "Dejounte Murray" | - | "Luka Doncic" | - And the execution plan should be: - | id | name | dependencies | operator info | - | 5 | Project | 6 | | - | 6 | Limit | 7 | | - | 7 | GetNeighbors | 0 | {"limit": "2"} | - | 0 | Start | | | - When profiling query: - """ - GO FROM "James Harden" OVER * YIELD id($$) as dst | Limit 2 - """ - Then the result should be, in any order: - | dst | - | "Russell Westbrook" | - | "Rockets" | - And the execution plan should be: - | id | name | dependencies | operator info | - | 5 | Project | 6 | | - | 6 | Limit | 7 | | - | 7 | GetNeighbors | 0 | {"limit": "2"} | - | 0 | Start | | | - When profiling query: - """ - GO FROM "James Harden" OVER * YIELD id($^) as src | Limit 2 - """ - Then the result should be, in any order: - | src | - | "James Harden" | - | "James Harden" | - And the execution plan should be: - | id | name | dependencies | operator info | - | 5 | Project | 6 | | - | 6 | Limit | 7 | | - | 7 | GetNeighbors | 0 | {"limit": "2"} | - | 0 | Start | | | - - Scenario: push limit down to GetNeighbors with offset - When profiling query: - """ - GO 1 STEPS FROM "Vince Carter" OVER serve - YIELD serve.start_year as start_year | - Limit 3, 4 - """ - Then the result should be, in any order: - | start_year | - | 2009 | - | 2011 | - | 2004 | - | 1998 | - And the execution plan should be: - | id | name | dependencies | operator info | - | 1 | Project | 2 | | - | 2 | Limit | 3 | | - | 3 | GetNeighbors | 4 | {"limit": "7"} | - | 4 | Start | | | diff --git a/tests/tck/features/optimizer/PushSampleDownRule.feature b/tests/tck/features/optimizer/PushSampleDownRule.feature index d04021b3fab..15ce745cdf4 100644 --- a/tests/tck/features/optimizer/PushSampleDownRule.feature +++ b/tests/tck/features/optimizer/PushSampleDownRule.feature @@ -6,64 +6,53 @@ Feature: Push Limit down rule Background: Given a graph with space named "nba" - Scenario: push limit down to GetNeighbors + Scenario: push limit down to Expand When profiling query: """ GO 1 STEPS FROM "James Harden" OVER like REVERSELY YIELD like._dst LIMIT [2] """ Then the result should be, in any order: - | like._dst | - | "Dejounte Murray" | - | "Luka Doncic" | + | like._dst | + | /[\w\s]+/ | + | /[\w\s]+/ | And the execution plan should be: - | id | name | dependencies | operator info | - | 5 | Project | 6 | | - | 6 | Limit | 7 | | - | 7 | GetNeighbors | 0 | {"limit": "2"} | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | {"stepLimits": ["2"]} | + | 2 | Expand | 1 | {"stepLimits": ["2"]} | + | 1 | Start | | | When profiling query: """ GO 2 STEPS FROM "James Harden" OVER like REVERSELY YIELD like._dst LIMIT [2, 2] """ Then the result should be, in any order: - | like._dst | - | "Kristaps Porzingis" | + | like._dst | + | /[\w\s]+/ | + | /[\w\s]+/ | And the execution plan should be: - | id | name | dependencies | operator info | - | 9 | Project | 12 | | - | 12 | Limit | 13 | | - | 13 | GetNeighbors | 6 | {"limit": "2", "random": "false"} | - | 6 | Loop | 0 | {"loopBody": "5"} | - | 5 | Dedup | 4 | | - | 4 | Project | 20 | | - | 20 | Limit | 21 | | - | 21 | GetNeighbors | 1 | {"limit": "$__VAR_2[($__VAR_1-1)]", "random": "false"} | - | 1 | Start | | | - | 0 | Start | | | + | id | name | dependencies | operator info | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | {"stepLimits": ["2", "2"]} | + | 2 | Expand | 1 | {"stepLimits": ["2", "2"]} | + | 1 | Start | | | When profiling query: """ GO 2 STEPS FROM "James Harden" OVER like REVERSELY YIELD $^.player.name AS src, like.likeness AS likeness, $$.player.name AS dst LIMIT [2, 2] """ Then the result should be, in any order: - | src | likeness | dst | - | "Luka Doncic" | 90 | "Kristaps Porzingis" | + | src | likeness | dst | + | /[\w\s]+/ | /\d\d/ | /[\w\s]+/ | + | /[\w\s]+/ | /\d\d/ | /[\w\s]+/ | And the execution plan should be: - | id | name | dependencies | profiling data | operator info | - | 14 | Project | 13 | | | - | 13 | HashLeftJoin | 9, 12 | | | - | 9 | Project | 15 | | | - | 15 | Limit | 16 | | | - | 16 | GetNeighbors | 6 | | {"limit": "2", "random": "false"} | - | 6 | Loop | 0 | | {"loopBody": "5"} | - | 5 | Dedup | 4 | | | - | 4 | Project | 17 | | | - | 17 | Limit | 18 | | | - | 18 | GetNeighbors | 1 | | {"limit": "$__VAR_2[($__VAR_1-1)]", "random": "false"} | - | 1 | Start | | | | - | 0 | Start | | | | - | 12 | Project | 11 | | | - | 11 | GetVertices | 10 | | | - | 10 | Argument | | | | + | id | name | dependencies | profiling data | operator info | + | 8 | Project | 7 | | | + | 7 | HashLeftJoin | 3,6 | | | + | 3 | ExpandAll | 2 | | {"stepLimits": ["2", "2"], "sample": "false"} | + | 2 | Expand | 1 | | {"stepLimits": ["2", "2"], "sample": "false"} | + | 1 | Start | | | | + | 6 | Project | 5 | | | + | 5 | GetVertices | 4 | | | + | 4 | Argument | | | | When profiling query: """ $var=GO FROM "Tim Duncan" OVER like YIELD like._dst AS dst; @@ -73,34 +62,19 @@ Feature: Push Limit down rule | count(*) | | 3 | And the execution plan should be: - | id | name | dependencies | profiling data | operator info | - | 24 | Aggregate | 23 | | | - | 23 | Project | 22 | | | - | 22 | InnerJoin | 21 | | | - | 21 | InnerJoin | 20 | | | - | 20 | Project | 26 | | | - | 26 | Limit | 27 | | { "count": "3" } | - | 27 | GetNeighbors | 17 | | {"limit": "3", "random": "false"} | - | 17 | Loop | 11 | | {"loopBody": "16"} | - | 16 | Dedup | 15 | | | - | 15 | Project | 14 | | | - | 14 | InnerJoin | 13 | | | - | 13 | Dedup | 12 | | | - | 12 | Project | 9 | | | - | 9 | Dedup | 8 | | | - | 8 | Project | 34 | | | - | 34 | Limit | 35 | | {"count": "$__VAR_2[($__VAR_1-1)]"} | - | 35 | GetNeighbors | 5 | | {"limit": "$__VAR_2[($__VAR_1-1)]", "random": "false"} | - | 5 | Start | | | | - | 11 | Dedup | 10 | | | - | 10 | Project | 4 | | | - | 4 | Dedup | 3 | | | - | 3 | Project | 2 | | | - | 2 | Project | 1 | | | - | 1 | GetNeighbors | 0 | | | - | 0 | Start | | | | + | id | name | dependencies | profiling data | operator info | + | 11 | Aggregate | 10 | | | + | 10 | Project | 9 | | | + | 9 | HashInnerJoin | 4,8 | | | + | 4 | Project | 3 | | | + | 3 | ExpandAll | 2 | | | + | 2 | Expand | 1 | | | + | 1 | Start | | | | + | 8 | ExpandAll | 7 | | {"stepLimits": ["2", "3"], "sample": "false"} | + | 7 | Expand | 6 | | {"stepLimits": ["2", "3"], "sample": "false"} | + | 6 | Argument | | | | - Scenario: push sample down to GetNeighbors + Scenario: push sample down to Expand When profiling query: """ GO 1 STEPS FROM "James Harden" OVER like REVERSELY YIELD like._dst SAMPLE [2] @@ -110,84 +84,48 @@ Feature: Push Limit down rule | /[\w\s]+/ | | /[\w\s]+/ | And the execution plan should be: - | id | name | dependencies | operator info | - | 1 | Project | 2 | | - | 2 | Sample | 3 | | - | 3 | GetNeighbors | 4 | {"limit": "2", "random": "true"} | - | 4 | Start | | | + | id | name | dependencies | operator info | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | {"stepLimits": ["2"], "sample": "true"} | + | 2 | Expand | 1 | {"stepLimits": ["2"], "sample": "true"} | + | 1 | Start | | | When profiling query: """ GO 2 STEPS FROM "James Harden" OVER like REVERSELY YIELD like._dst SAMPLE [2, 2] """ - Then the result should be, in any order: - | like._dst | - | /[\w\s]+/ | - | /[\w\s]+/ | - And the execution plan should be: - | id | name | dependencies | operator info | - | 9 | Project | 12 | | - | 12 | Sample | 13 | | - | 13 | GetNeighbors | 6 | {"limit": "2", "random": "true"} | - | 6 | Loop | 0 | {"loopBody": "5"} | - | 5 | Dedup | 4 | | - | 4 | Project | 20 | | - | 20 | Sample | 21 | | - | 21 | GetNeighbors | 1 | {"limit": "$__VAR_2[($__VAR_1-1)]", "random": "true"} | - | 1 | Start | | | - | 0 | Start | | | + Then the execution plan should be: + | id | name | dependencies | operator info | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | {"stepLimits": ["2", "2"], "sample": "true"} | + | 2 | Expand | 1 | {"stepLimits": ["2", "2"], "sample": "true"} | + | 1 | Start | | | When profiling query: """ GO 2 STEPS FROM "James Harden" OVER like REVERSELY YIELD $^.player.name AS src, like.likeness AS likeness, $$.player.name AS dst SAMPLE [2, 2] """ - Then the result should be, in any order: - | src | likeness | dst | - | /[\w\s]+/ | /\d\d/ | /[\w\s]+/ | - | /[\w\s]+/ | /\d\d/ | /[\w\s]+/ | - And the execution plan should be: - | id | name | dependencies | profiling data | operator info | - | 14 | Project | 13 | | | - | 13 | HashLeftJoin | 9, 12 | | | - | 9 | Project | 17 | | | - | 17 | Sample | 18 | | | - | 18 | GetNeighbors | 6 | | {"limit": "2", "random": "true"} | - | 6 | Loop | 0 | | {"loopBody": "5"} | - | 5 | Dedup | 4 | | | - | 4 | Project | 25 | | | - | 25 | Sample | 26 | | | - | 26 | GetNeighbors | 1 | | {"limit": "$__VAR_2[($__VAR_1-1)]", "random": "true"} | - | 1 | Start | | | | - | 0 | Start | | | | - | 12 | Project | 11 | | | - | 11 | GetVertices | 10 | | | - | 10 | Argument | | | | + Then the execution plan should be: + | id | name | dependencies | profiling data | operator info | + | 8 | Project | 7 | | | + | 7 | HashLeftJoin | 3,6 | | | + | 3 | ExpandAll | 2 | | {"stepLimits": ["2", "2"], "sample": "true"} | + | 2 | Expand | 1 | | {"stepLimits": ["2", "2"], "sample": "true"} | + | 1 | Start | | | | + | 6 | Project | 5 | | | + | 5 | GetVertices | 4 | | | + | 4 | Argument | | | | When profiling query: """ $var=GO FROM "Tim Duncan" OVER like YIELD like._dst AS dst; GO 2 steps FROM $var.dst OVER like YIELD $var.dst AS dst1,like._dst AS dst2 SAMPLE [2,3] """ Then the execution plan should be: - | id | name | dependencies | profiling data | operator info | - | 23 | Project | 22 | | | - | 22 | InnerJoin | 21 | | | - | 21 | InnerJoin | 20 | | | - | 20 | Project | 26 | | | - | 26 | Sample | 27 | | { "count": "3" } | - | 27 | GetNeighbors | 17 | | {"limit": "3", "random": "true"} | - | 17 | Loop | 11 | | {"loopBody": "16"} | - | 16 | Dedup | 15 | | | - | 15 | Project | 14 | | | - | 14 | InnerJoin | 13 | | | - | 13 | Dedup | 12 | | | - | 12 | Project | 9 | | | - | 9 | Dedup | 8 | | | - | 8 | Project | 34 | | | - | 34 | Sample | 35 | | {"count": "$__VAR_2[($__VAR_1-1)]"} | - | 35 | GetNeighbors | 5 | | {"limit": "$__VAR_2[($__VAR_1-1)]", "random": "true"} | - | 5 | Start | | | | - | 11 | Dedup | 10 | | | - | 10 | Project | 4 | | | - | 4 | Dedup | 3 | | | - | 3 | Project | 2 | | | - | 2 | Project | 1 | | | - | 1 | GetNeighbors | 0 | | | - | 0 | Start | | | | + | id | name | dependencies | profiling data | operator info | + | 10 | Project | 9 | | | + | 9 | HashInnerJoin | 4,8 | | | + | 4 | Project | 3 | | | + | 3 | ExpandAll | 2 | | | + | 2 | Expand | 1 | | | + | 1 | Start | | | | + | 8 | ExpandAll | 7 | | {"stepLimits": ["2", "3"], "sample": "true"} | + | 7 | Expand | 6 | | {"stepLimits": ["2", "3"], "sample": "true"} | + | 6 | Argument | | | | diff --git a/tests/tck/features/optimizer/TopNRule.feature b/tests/tck/features/optimizer/TopNRule.feature index 547a3ae637f..94272971c52 100644 --- a/tests/tck/features/optimizer/TopNRule.feature +++ b/tests/tck/features/optimizer/TopNRule.feature @@ -19,11 +19,12 @@ Feature: TopN rule | 50 | | 55 | And the execution plan should be: - | id | name | dependencies | operator info | - | 1 | TopN | 2 | | - | 2 | Project | 3 | | - | 3 | GetNeighbors | 4 | | - | 4 | Start | | | + | id | name | dependencies | operator info | + | 7 | TopN | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | Scenario: apply topn opt rule with reverse traversal When profiling query: @@ -37,11 +38,12 @@ Feature: TopN rule | likeness | | 83 | And the execution plan should be: - | id | name | dependencies | operator info | - | 1 | TopN | 2 | | - | 2 | Project | 3 | | - | 3 | GetNeighbors | 4 | | - | 4 | Start | | | + | id | name | dependencies | operator info | + | 7 | TopN | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | Scenario: [1] fail to apply topn rule When profiling query: @@ -55,12 +57,13 @@ Feature: TopN rule | likeness | | 60 | And the execution plan should be: - | id | name | dependencies | operator info | - | 1 | Limit | 2 | | - | 2 | Sort | 3 | | - | 3 | Project | 4 | | - | 4 | GetNeighbors | 5 | | - | 5 | Start | | | + | id | name | dependencies | operator info | + | 6 | Limit | 5 | | + | 5 | Sort | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | | Scenario: [2] fail to apply topn rule When profiling query: @@ -75,8 +78,9 @@ Feature: TopN rule | 55 | | 60 | And the execution plan should be: - | id | name | dependencies | operator info | - | 1 | Sort | 2 | | - | 2 | Project | 3 | | - | 3 | GetNeighbors | 4 | | - | 4 | Start | | | + | id | name | dependencies | operator info | + | 5 | Sort | 4 | | + | 4 | Project | 3 | | + | 3 | ExpandAll | 2 | | + | 2 | Expand | 1 | | + | 1 | Start | | |