diff --git a/src/graph/context/ExecutionContext.cpp b/src/graph/context/ExecutionContext.cpp index 282bb24a8e7..ecbdcb54544 100644 --- a/src/graph/context/ExecutionContext.cpp +++ b/src/graph/context/ExecutionContext.cpp @@ -75,6 +75,20 @@ const Result& ExecutionContext::getResult(const std::string& name) const { } } +void ExecutionContext::setVersionedResult(const std::string& name, + Result&& result, + int64_t version) { + auto it = valueMap_.find(name); + if (it != valueMap_.end()) { + auto& hist = it->second; + auto size = hist.size(); + if (static_cast(std::abs(version)) >= size) { + return; + } + hist[(size + version - 1) % size] = std::move(result); + } +} + const Result& ExecutionContext::getVersionedResult(const std::string& name, int64_t version) const { auto& result = getHistory(name); auto size = result.size(); diff --git a/src/graph/context/ExecutionContext.h b/src/graph/context/ExecutionContext.h index 28aa31d7be5..d34acddc6a9 100644 --- a/src/graph/context/ExecutionContext.h +++ b/src/graph/context/ExecutionContext.h @@ -51,6 +51,8 @@ class ExecutionContext { const Result& getVersionedResult(const std::string& name, int64_t version) const; + void setVersionedResult(const std::string& name, Result&& result, int64_t version); + size_t numVersions(const std::string& name) const; // Return all existing history of the value. The front is the latest value diff --git a/src/graph/context/Iterator.cpp b/src/graph/context/Iterator.cpp index fc369192a69..95df48d66fc 100644 --- a/src/graph/context/Iterator.cpp +++ b/src/graph/context/Iterator.cpp @@ -5,6 +5,8 @@ #include "graph/context/Iterator.h" +#include + #include "common/datatypes/Edge.h" #include "common/datatypes/Vertex.h" #include "common/memory/MemoryUtils.h" @@ -417,13 +419,13 @@ const Value& GetNeighborsIter::getEdgeProp(const std::string& edge, const std::s auto& currentEdge = currentEdgeName(); if (edge != "*" && (currentEdge.compare(1, std::string::npos, edge) != 0)) { - VLOG(1) << "Current edge: " << currentEdgeName() << " Wanted: " << edge; + DLOG(INFO) << "Current edge: " << currentEdgeName() << " Wanted: " << edge; return Value::kEmpty; } auto index = currentDs_->edgePropsMap.find(currentEdge); if (index == currentDs_->edgePropsMap.end()) { - VLOG(1) << "No edge found: " << edge; - VLOG(1) << "Current edge: " << currentEdge; + DLOG(INFO) << "No edge found: " << edge; + DLOG(INFO) << "Current edge: " << currentEdge; return Value::kEmpty; } auto propIndex = index->second.propIndices.find(prop); @@ -468,6 +470,22 @@ Value GetNeighborsIter::getVertex(const std::string& name) const { return Value(std::move(vertex)); } +std::vector GetNeighborsIter::vids() { + std::vector vids; + vids.reserve(numRows()); + valid_ = true; + colIdx_ = -2; + for (currentDs_ = dsIndices_.begin(); currentDs_ < dsIndices_.end(); ++currentDs_) { + rowsUpperBound_ = currentDs_->ds->rows.end(); + for (currentRow_ = currentDs_->ds->rows.begin(); currentRow_ < currentDs_->ds->rows.end(); + ++currentRow_) { + vids.emplace_back(getColumn(0)); + } + } + reset(); + return vids; +} + List GetNeighborsIter::getVertices() { List vertices; vertices.reserve(numRows()); @@ -478,7 +496,6 @@ List GetNeighborsIter::getVertices() { for (currentRow_ = currentDs_->ds->rows.begin(); currentRow_ < currentDs_->ds->rows.end(); ++currentRow_) { vertices.values.emplace_back(getVertex()); - VLOG(1) << "vertex: " << getVertex() << " size: " << vertices.size(); } } reset(); @@ -549,8 +566,8 @@ List GetNeighborsIter::getEdges() { auto edge = getEdge(); if (edge.isEdge()) { const_cast(edge.getEdge()).format(); + edges.values.emplace_back(std::move(edge)); } - edges.values.emplace_back(std::move(edge)); } reset(); return edges; diff --git a/src/graph/context/Iterator.h b/src/graph/context/Iterator.h index d8e2985dde9..9727680336c 100644 --- a/src/graph/context/Iterator.h +++ b/src/graph/context/Iterator.h @@ -327,6 +327,9 @@ class GetNeighborsIter final : public Iterator { // Its unique based on the plan List getVertices(); + // return start vids + std::vector vids(); + // Its unique based on the GN interface dedup List getEdges(); diff --git a/src/graph/context/ast/QueryAstContext.h b/src/graph/context/ast/QueryAstContext.h index dc918076a95..3e5146a318c 100644 --- a/src/graph/context/ast/QueryAstContext.h +++ b/src/graph/context/ast/QueryAstContext.h @@ -127,10 +127,14 @@ struct SubgraphContext final : public AstContext { Starts from; StepClause steps; std::string loopSteps; + Expression* filter{nullptr}; + Expression* tagFilter{nullptr}; + Expression* edgeFilter{nullptr}; std::vector colNames; std::unordered_set edgeTypes; std::unordered_set biDirectEdgeTypes; std::vector colType; + ExpressionProps exprProps; bool withProp{false}; bool getVertexProp{false}; bool getEdgeProp{false}; diff --git a/src/graph/executor/algo/SubgraphExecutor.cpp b/src/graph/executor/algo/SubgraphExecutor.cpp index e98dacc8b43..826b5c1d86c 100644 --- a/src/graph/executor/algo/SubgraphExecutor.cpp +++ b/src/graph/executor/algo/SubgraphExecutor.cpp @@ -4,42 +4,153 @@ #include "graph/executor/algo/SubgraphExecutor.h" -#include "graph/planner/plan/Algo.h" +#include "graph/service/GraphFlags.h" +using nebula::storage::StorageClient; namespace nebula { namespace graph { folly::Future SubgraphExecutor::execute() { SCOPED_TIMER(&execTime_); - auto* subgraph = asNode(node()); - DataSet ds; - ds.colNames = subgraph->colNames(); + totalSteps_ = subgraph_->steps(); + auto iter = ectx_->getResult(subgraph_->inputVar()).iter(); + auto res = buildRequestListByVidType(iter.get(), subgraph_->src(), true); + NG_RETURN_IF_ERROR(res); + vids_ = res.value(); + if (vids_.empty()) { + DataSet emptyResult; + return finish(ResultBuilder().value(Value(std::move(emptyResult))).build()); + } + return getNeighbors(); +} + +folly::Future SubgraphExecutor::getNeighbors() { + time::Duration getNbrTime; + StorageClient* storageClient = qctx_->getStorageClient(); + StorageClient::CommonRequestParam param(subgraph_->space(), + qctx_->rctx()->session()->id(), + qctx_->plan()->id(), + qctx_->plan()->isProfileEnabled()); + + storage::cpp2::EdgeDirection edgeDirection{Direction::OUT_EDGE}; + return storageClient + ->getNeighbors(param, + {nebula::kVid}, + std::move(vids_), + {}, + edgeDirection, + nullptr, + subgraph_->vertexProps(), + subgraph_->edgeProps(), + nullptr, + false, + false, + {}, + -1, + currentStep_ == 1 ? subgraph_->edgeFilter() : subgraph_->filter(), + currentStep_ == 1 ? nullptr : subgraph_->tagFilter()) + .via(runner()) + .thenValue([this, getNbrTime](RpcResponse&& resp) mutable { + otherStats_.emplace("total_rpc_time", folly::sformat("{}(us)", getNbrTime.elapsedInUSec())); + auto& hostLatency = resp.hostLatency(); + for (size_t i = 0; i < hostLatency.size(); ++i) { + size_t size = 0u; + auto& result = resp.responses()[i]; + if (result.vertices_ref().has_value()) { + size = (*result.vertices_ref()).size(); + } + auto& info = hostLatency[i]; + otherStats_.emplace( + folly::sformat("{} exec/total/vertices", std::get<0>(info).toString()), + folly::sformat("{}(us)/{}(us)/{},", std::get<1>(info), std::get<2>(info), size)); + auto detail = getStorageDetail(result.result.latency_detail_us_ref()); + if (!detail.empty()) { + otherStats_.emplace("storage_detail", detail); + } + } + vids_.clear(); + return handleResponse(std::move(resp)); + }); +} + +folly::Future SubgraphExecutor::handleResponse(RpcResponse&& resps) { + auto result = handleCompleteness(resps, FLAGS_accept_partial_success); + if (!result.ok()) { + return folly::makeFuture(std::move(result).status()); + } + + auto& responses = resps.responses(); + List list; + for (auto& resp : responses) { + auto dataset = resp.get_vertices(); + if (dataset == nullptr) { + continue; + } + list.values.emplace_back(std::move(*dataset)); + } + + auto listVal = std::make_shared(std::move(list)); + auto iter = std::make_unique(listVal); + + auto steps = totalSteps_; + if (!subgraph_->oneMoreStep()) { + --steps; + } - uint32_t steps = subgraph->steps(); - const auto& currentStepVal = ectx_->getValue(subgraph->currentStepVar()); - DCHECK(currentStepVal.isInt()); - auto currentStep = currentStepVal.getInt(); - auto resultVar = subgraph->resultVar(); + if (!process(std::move(iter)) || ++currentStep_ > steps) { + filterEdges(0); + return folly::makeFuture(Status::OK()); + } else { + return getNeighbors(); + } +} - auto iter = ectx_->getResult(subgraph->inputVar()).iter(); - auto gnSize = iter->size(); +void SubgraphExecutor::filterEdges(int version) { + auto iter = ectx_->getVersionedResult(subgraph_->outputVar(), version).iter(); + auto* gnIter = static_cast(iter.get()); + while (gnIter->valid()) { + const auto& dst = gnIter->getEdgeProp("*", nebula::kDst); + if (validVids_.find(dst) == validVids_.end()) { + auto edge = gnIter->getEdge(); + gnIter->erase(); + } else { + gnIter->next(); + } + } + gnIter->reset(); + ResultBuilder builder; + builder.iter(std::move(iter)); + ectx_->setVersionedResult(subgraph_->outputVar(), builder.build(), version); +} + +bool SubgraphExecutor::process(std::unique_ptr iter) { + auto gnSize = iter->numRows(); + if (gnSize == 0) { + return false; + } ResultBuilder builder; builder.value(iter->valuePtr()); - robin_hood::unordered_flat_map> currentVids; + HashMap currentVids; currentVids.reserve(gnSize); historyVids_.reserve(historyVids_.size() + gnSize); - if (currentStep == 1) { - for (; iter->valid(); iter->next()) { - const auto& src = iter->getColumn(nebula::kVid); - currentVids.emplace(src, 0); + auto startVids = iter->vids(); + if (currentStep_ == 1) { + for (auto& startVid : startVids) { + currentVids.emplace(startVid, 0); } - iter->reset(); } - auto& biDirectEdgeTypes = subgraph->biDirectEdgeTypes(); + validVids_.insert(std::make_move_iterator(startVids.begin()), + std::make_move_iterator(startVids.end())); + auto& biDirectEdgeTypes = subgraph_->biDirectEdgeTypes(); while (iter->valid()) { const auto& dst = iter->getEdgeProp("*", nebula::kDst); + if (dst.empty()) { + // no edge, dst is empty + iter->next(); + continue; + } auto findIter = historyVids_.find(dst); if (findIter != historyVids_.end()) { if (biDirectEdgeTypes.empty()) { @@ -52,7 +163,7 @@ folly::Future SubgraphExecutor::execute() { } auto type = typeVal.getInt(); if (biDirectEdgeTypes.find(type) != biDirectEdgeTypes.end()) { - if (type < 0 || findIter->second + 2 == currentStep) { + if (type < 0 || findIter->second + 2 == currentStep_) { iter->erase(); } else { iter->next(); @@ -62,25 +173,30 @@ folly::Future SubgraphExecutor::execute() { } } } else { - if (currentStep == steps) { + if (currentStep_ == totalSteps_) { iter->erase(); continue; } - if (currentVids.emplace(dst, currentStep).second) { - Row row; - row.values.emplace_back(std::move(dst)); - ds.rows.emplace_back(std::move(row)); + if (currentVids.emplace(dst, currentStep_).second) { + // next vids for getNeighbor + vids_.emplace_back(std::move(dst)); } iter->next(); } } iter->reset(); builder.iter(std::move(iter)); - ectx_->setResult(resultVar, builder.build()); + finish(builder.build()); // update historyVids historyVids_.insert(std::make_move_iterator(currentVids.begin()), std::make_move_iterator(currentVids.end())); - return finish(ResultBuilder().value(Value(std::move(ds))).build()); + if (currentStep_ != 1 && subgraph_->tagFilter()) { + filterEdges(-1); + } + if (vids_.empty()) { + return false; + } + return true; } } // namespace graph diff --git a/src/graph/executor/algo/SubgraphExecutor.h b/src/graph/executor/algo/SubgraphExecutor.h index 4b3c3f7f493..6344660c166 100644 --- a/src/graph/executor/algo/SubgraphExecutor.h +++ b/src/graph/executor/algo/SubgraphExecutor.h @@ -7,7 +7,8 @@ #include -#include "graph/executor/Executor.h" +#include "graph/executor/StorageAccessExecutor.h" +#include "graph/planner/plan/Algo.h" // Subgraph receive result from GetNeighbors // There are two Main functions @@ -39,15 +40,37 @@ namespace nebula { namespace graph { -class SubgraphExecutor : public Executor { +using RpcResponse = storage::StorageRpcResponse; + +class SubgraphExecutor : public StorageAccessExecutor { public: + using HashMap = robin_hood::unordered_flat_map>; + using HashSet = robin_hood::unordered_flat_set>; + SubgraphExecutor(const PlanNode* node, QueryContext* qctx) - : Executor("SubgraphExecutor", node, qctx) {} + : StorageAccessExecutor("SubgraphExecutor", node, qctx) { + subgraph_ = asNode(node); + } folly::Future execute() override; + folly::Future getNeighbors(); + + bool process(std::unique_ptr iter); + + // filter out edges that do not meet the conditions in the previous step + void filterEdges(int version); + + folly::Future handleResponse(RpcResponse&& resps); + private: - robin_hood::unordered_flat_map> historyVids_; + HashMap historyVids_; + const Subgraph* subgraph_{nullptr}; + size_t currentStep_{1}; + size_t totalSteps_{1}; + std::vector vids_; + // save vids already visited + HashSet validVids_; }; } // namespace graph diff --git a/src/graph/executor/query/DataCollectExecutor.cpp b/src/graph/executor/query/DataCollectExecutor.cpp index b00efd2c0d1..c4440741860 100644 --- a/src/graph/executor/query/DataCollectExecutor.cpp +++ b/src/graph/executor/query/DataCollectExecutor.cpp @@ -72,27 +72,13 @@ Status DataCollectExecutor::collectSubgraph(const std::vector& vars bool notEmpty = false; for (const auto& type : colType) { if (type == Value::Type::VERTEX) { - auto originVertices = gnIter->getVertices(); - vertices.reserve(originVertices.size()); - for (auto& v : originVertices.values) { - if (UNLIKELY(!v.isVertex())) { - continue; - } - vertices.emplace_back(std::move(v)); - } + vertices = gnIter->getVertices(); if (!vertices.empty()) { notEmpty = true; row.emplace_back(std::move(vertices)); } } else { - auto originEdges = gnIter->getEdges(); - edges.reserve(originEdges.size()); - for (auto& edge : originEdges.values) { - if (UNLIKELY(!edge.isEdge())) { - continue; - } - edges.emplace_back(std::move(edge)); - } + edges = gnIter->getEdges(); if (!edges.empty()) { notEmpty = true; } diff --git a/src/graph/planner/ngql/SubgraphPlanner.cpp b/src/graph/planner/ngql/SubgraphPlanner.cpp index 0474f1a0824..c7b480f7136 100644 --- a/src/graph/planner/ngql/SubgraphPlanner.cpp +++ b/src/graph/planner/ngql/SubgraphPlanner.cpp @@ -13,12 +13,19 @@ namespace nebula { namespace graph { +StatusOr>> SubgraphPlanner::buildVertexProps() { + auto* qctx = subgraphCtx_->qctx; + const auto& space = subgraphCtx_->space; + bool getVertexProp = subgraphCtx_->withProp && subgraphCtx_->getVertexProp; + auto vertexProps = SchemaUtil::getAllVertexProp(qctx, space.id, getVertexProp); + return vertexProps; +} StatusOr>> SubgraphPlanner::buildEdgeProps() { auto* qctx = subgraphCtx_->qctx; - bool getEdgeProp = subgraphCtx_->withProp && subgraphCtx_->getEdgeProp; const auto& space = subgraphCtx_->space; auto& edgeTypes = subgraphCtx_->edgeTypes; + auto& exprProps = subgraphCtx_->exprProps; auto& biDirectEdgeTypes = subgraphCtx_->biDirectEdgeTypes; if (edgeTypes.empty()) { @@ -32,65 +39,69 @@ StatusOr>> SubgraphPlanner::buildEdgeProps biDirectEdgeTypes.emplace(-edge.first); } } - std::vector vEdgeTypes(edgeTypes.begin(), edgeTypes.end()); - auto edgeProps = SchemaUtil::getEdgeProps(qctx, space, std::move(vEdgeTypes), getEdgeProp); - NG_RETURN_IF_ERROR(edgeProps); - return edgeProps; -} - -// ++loopSteps{0} < steps && (var is Empty OR size(var) != 0) -Expression* SubgraphPlanner::loopCondition(uint32_t steps, const std::string& var) { - auto* qctx = subgraphCtx_->qctx; - auto* pool = qctx->objPool(); - auto& loopSteps = subgraphCtx_->loopSteps; - qctx->ectx()->setValue(loopSteps, 0); - auto step = ExpressionUtils::stepCondition(pool, loopSteps, 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); + const auto& edgeProps = exprProps.edgeProps(); + if (!subgraphCtx_->withProp && !edgeProps.empty()) { + auto edgePropsPtr = std::make_unique>(); + edgePropsPtr->reserve(edgeTypes.size()); + for (const auto& edgeType : edgeTypes) { + EdgeProp ep; + ep.type_ref() = edgeType; + const auto& found = edgeProps.find(std::abs(edgeType)); + if (found != edgeProps.end()) { + std::set props(found->second.begin(), found->second.end()); + props.emplace(kType); + props.emplace(kRank); + props.emplace(kDst); + ep.props_ref() = std::vector(props.begin(), props.end()); + } else { + ep.props_ref() = std::vector({kType, kRank, kDst}); + } + edgePropsPtr->emplace_back(std::move(ep)); + } + return edgePropsPtr; + } else { + bool getEdgeProp = subgraphCtx_->withProp && subgraphCtx_->getEdgeProp; + std::vector vEdgeTypes(edgeTypes.begin(), edgeTypes.end()); + return SchemaUtil::getEdgeProps(qctx, space, std::move(vEdgeTypes), getEdgeProp); + } } StatusOr SubgraphPlanner::nSteps(SubPlan& startVidPlan, const std::string& input) { auto* qctx = subgraphCtx_->qctx; const auto& space = subgraphCtx_->space; + const auto& dstTagProps = subgraphCtx_->exprProps.dstTagProps(); const auto& steps = subgraphCtx_->steps; - auto* startNode = StartNode::make(qctx); - bool getVertexProp = subgraphCtx_->withProp && subgraphCtx_->getVertexProp; - auto vertexProps = SchemaUtil::getAllVertexProp(qctx, space.id, getVertexProp); + auto vertexProps = buildVertexProps(); NG_RETURN_IF_ERROR(vertexProps); auto edgeProps = buildEdgeProps(); NG_RETURN_IF_ERROR(edgeProps); - auto* gn = GetNeighbors::make(qctx, startNode, space.id); - gn->setSrc(subgraphCtx_->from.src); - gn->setVertexProps(std::move(vertexProps).value()); - gn->setEdgeProps(std::move(edgeProps).value()); - gn->setInputVar(input); - - auto resultVar = qctx->vctx()->anonVarGen()->getVar(); - auto loopSteps = qctx->vctx()->anonVarGen()->getVar(); - subgraphCtx_->loopSteps = loopSteps; - auto* subgraph = Subgraph::make(qctx, gn, resultVar, loopSteps, steps.steps() + 1); - subgraph->setOutputVar(input); + + auto* subgraph = Subgraph::make(qctx, + startVidPlan.root, + space.id, + subgraphCtx_->from.src, + subgraphCtx_->tagFilter, + subgraphCtx_->edgeFilter, + subgraphCtx_->filter, + steps.steps() + 1); + subgraph->setVertexProps(std::move(vertexProps).value()); + subgraph->setEdgeProps(std::move(edgeProps).value()); + subgraph->setInputVar(input); subgraph->setBiDirectEdgeTypes(subgraphCtx_->biDirectEdgeTypes); - subgraph->setColNames({nebula::kVid}); - uint32_t maxSteps = steps.steps(); - if (subgraphCtx_->getEdgeProp || subgraphCtx_->withProp) { - ++maxSteps; + if (subgraphCtx_->getEdgeProp || subgraphCtx_->withProp || !dstTagProps.empty()) { + subgraph->setOneMoreStep(); } - auto* condition = loopCondition(maxSteps, gn->outputVar()); - auto* loop = Loop::make(qctx, startVidPlan.root, subgraph, condition); auto* dc = DataCollect::make(qctx, DataCollect::DCKind::kSubgraph); - dc->addDep(loop); - dc->setInputVars({resultVar}); + dc->addDep(subgraph); + dc->setInputVars({subgraph->outputVar()}); dc->setColType(std::move(subgraphCtx_->colType)); dc->setColNames(subgraphCtx_->colNames); SubPlan subPlan; subPlan.root = dc; - subPlan.tail = startVidPlan.tail != nullptr ? startVidPlan.tail : loop; + subPlan.tail = startVidPlan.tail == nullptr ? subgraph : startVidPlan.tail; return subPlan; } diff --git a/src/graph/planner/ngql/SubgraphPlanner.h b/src/graph/planner/ngql/SubgraphPlanner.h index 7dfe7cec5bc..b2ce204e481 100644 --- a/src/graph/planner/ngql/SubgraphPlanner.h +++ b/src/graph/planner/ngql/SubgraphPlanner.h @@ -17,6 +17,7 @@ namespace graph { class SubgraphPlanner final : public Planner { public: using EdgeProp = nebula::storage::cpp2::EdgeProp; + using Vertexprop = nebula::storage::cpp2::VertexProp; static std::unique_ptr make() { return std::unique_ptr(new SubgraphPlanner()); @@ -32,9 +33,9 @@ class SubgraphPlanner final : public Planner { StatusOr nSteps(SubPlan& startVidPlan, const std::string& input); - StatusOr>> buildEdgeProps(); + StatusOr>> buildVertexProps(); - Expression* loopCondition(uint32_t steps, const std::string& var); + StatusOr>> buildEdgeProps(); private: SubgraphPlanner() = default; diff --git a/src/graph/planner/plan/Algo.cpp b/src/graph/planner/plan/Algo.cpp index 1f42394f49a..e3c4e09fa3e 100644 --- a/src/graph/planner/plan/Algo.cpp +++ b/src/graph/planner/plan/Algo.cpp @@ -133,5 +133,20 @@ void BiCartesianProduct::accept(PlanNodeVisitor* visitor) { BiCartesianProduct::BiCartesianProduct(QueryContext* qctx) : BinaryInputNode(qctx, Kind::kBiCartesianProduct) {} +std::unique_ptr Subgraph::explain() const { + auto desc = SingleDependencyNode::explain(); + addDescription("src", src_ ? src_->toString() : "", desc.get()); + addDescription("tag_filter", tagFilter_ ? tagFilter_->toString() : "", desc.get()); + addDescription("edge_filter", edgeFilter_ ? edgeFilter_->toString() : "", desc.get()); + addDescription("filter", filter_ ? filter_->toString() : "", desc.get()); + addDescription( + "vertexProps", vertexProps_ ? folly::toJson(util::toJson(*vertexProps_)) : "", desc.get()); + addDescription( + "edgeProps", edgeProps_ ? folly::toJson(util::toJson(*edgeProps_)) : "", desc.get()); + addDescription( + "steps", folly::toJson(util::toJson(oneMoreStep_ ? steps_ : steps_ - 1)), desc.get()); + return desc; +} + } // namespace graph } // namespace nebula diff --git a/src/graph/planner/plan/Algo.h b/src/graph/planner/plan/Algo.h index bff214caee2..c6b6dac74fa 100644 --- a/src/graph/planner/plan/Algo.h +++ b/src/graph/planner/plan/Algo.h @@ -268,48 +268,103 @@ class Subgraph final : public SingleInputNode { public: static Subgraph* make(QueryContext* qctx, PlanNode* input, - const std::string& resultVar, - const std::string& currentStepVar, - uint32_t steps) { - return qctx->objPool()->makeAndAdd(qctx, input, resultVar, currentStepVar, steps); + GraphSpaceID space, + Expression* src, + const Expression* tagFilter, + const Expression* edgeFilter, + const Expression* filter, + size_t steps) { + return qctx->objPool()->makeAndAdd( + qctx, input, space, DCHECK_NOTNULL(src), tagFilter, edgeFilter, filter, steps); } - const std::string& resultVar() const { - return resultVar_; + GraphSpaceID space() const { + return space_; } - const std::string& currentStepVar() const { - return currentStepVar_; + Expression* src() const { + return src_; } - uint32_t steps() const { + const Expression* tagFilter() const { + return tagFilter_; + } + + const Expression* edgeFilter() const { + return edgeFilter_; + } + + const Expression* filter() const { + return filter_; + } + + size_t steps() const { return steps_; } + bool oneMoreStep() const { + return oneMoreStep_; + } + const std::unordered_set biDirectEdgeTypes() const { return biDirectEdgeTypes_; } + const std::vector* edgeProps() const { + return edgeProps_.get(); + } + + const std::vector* vertexProps() const { + return vertexProps_.get(); + } + + void setOneMoreStep() { + oneMoreStep_ = true; + } + void setBiDirectEdgeTypes(std::unordered_set edgeTypes) { biDirectEdgeTypes_ = std::move(edgeTypes); } + void setVertexProps(std::unique_ptr> vertexProps) { + vertexProps_ = std::move(vertexProps); + } + + void setEdgeProps(std::unique_ptr> edgeProps) { + edgeProps_ = std::move(edgeProps); + } + + std::unique_ptr explain() const override; + private: friend ObjectPool; Subgraph(QueryContext* qctx, PlanNode* input, - const std::string& resultVar, - const std::string& currentStepVar, - uint32_t steps) + GraphSpaceID space, + Expression* src, + const Expression* tagFilter, + const Expression* edgeFilter, + const Expression* filter, + size_t steps) : SingleInputNode(qctx, Kind::kSubgraph, input), - resultVar_(resultVar), - currentStepVar_(currentStepVar), + space_(space), + src_(src), + tagFilter_(tagFilter), + edgeFilter_(edgeFilter), + filter_(filter), steps_(steps) {} - std::string resultVar_; - std::string currentStepVar_; - uint32_t steps_; + GraphSpaceID space_; + // vertices may be parsing from runtime. + Expression* src_{nullptr}; + const Expression* tagFilter_{nullptr}; + const Expression* edgeFilter_{nullptr}; + const Expression* filter_{nullptr}; + size_t steps_{1}; + bool oneMoreStep_{false}; std::unordered_set biDirectEdgeTypes_; + std::unique_ptr> vertexProps_; + std::unique_ptr> edgeProps_; }; class BiCartesianProduct final : public BinaryInputNode { diff --git a/src/graph/validator/GetSubgraphValidator.cpp b/src/graph/validator/GetSubgraphValidator.cpp index d4e31ff645f..e2350c1d326 100644 --- a/src/graph/validator/GetSubgraphValidator.cpp +++ b/src/graph/validator/GetSubgraphValidator.cpp @@ -7,7 +7,9 @@ #include "graph/planner/plan/Logic.h" #include "graph/planner/plan/Query.h" +#include "graph/util/ExpressionUtils.h" #include "graph/util/ValidateUtil.h" +#include "graph/visitor/ExtractFilterExprVisitor.h" #include "parser/TraverseSentences.h" namespace nebula { @@ -23,6 +25,7 @@ Status GetSubgraphValidator::validateImpl() { NG_RETURN_IF_ERROR(validateInBound(gsSentence->in())); NG_RETURN_IF_ERROR(validateOutBound(gsSentence->out())); NG_RETURN_IF_ERROR(validateBothInOutBound(gsSentence->both())); + NG_RETURN_IF_ERROR(validateWhere(gsSentence->where())); NG_RETURN_IF_ERROR(validateYield(gsSentence->yield())); return Status::OK(); } @@ -99,6 +102,67 @@ Status GetSubgraphValidator::validateBothInOutBound(BothInOutClause* out) { return Status::OK(); } +static Expression* rewriteDstProp2SrcProp(const Expression* expr) { + ObjectPool* pool = expr->getObjPool(); + auto matcher = [](const Expression* e) -> bool { + return e->kind() == Expression::Kind::kDstProperty; + }; + auto rewriter = [pool](const Expression* e) -> Expression* { + auto dstExpr = static_cast(e); + return SourcePropertyExpression::make(pool, dstExpr->sym(), dstExpr->prop()); + }; + + return RewriteVisitor::transform(expr, std::move(matcher), std::move(rewriter)); +} + +// Check validity of filter expression, rewrites expression to fit its sementic, +// disable some invalid expression types, collect properties used in filter. +Status GetSubgraphValidator::validateWhere(WhereClause* where) { + if (where == nullptr) { + return Status::OK(); + } + auto* expr = where->filter(); + if (ExpressionUtils::findAny(expr, + {Expression::Kind::kAggregate, + Expression::Kind::kSrcProperty, + Expression::Kind::kVarProperty, + Expression::Kind::kInputProperty, + Expression::Kind::kLogicalOr})) { + return Status::SemanticError("Not support `%s' in where sentence.", expr->toString().c_str()); + } + + where->setFilter(ExpressionUtils::rewriteLabelAttr2EdgeProp(expr)); + auto filter = where->filter(); + auto typeStatus = deduceExprType(filter); + NG_RETURN_IF_ERROR(typeStatus); + auto type = typeStatus.value(); + if (type != Value::Type::BOOL && type != Value::Type::NULLVALUE && + type != Value::Type::__EMPTY__) { + std::stringstream ss; + ss << "`" << filter->toString() << "', expected Boolean, " + << "but was `" << type << "'"; + return Status::SemanticError(ss.str()); + } + + NG_RETURN_IF_ERROR(deduceProps(filter, subgraphCtx_->exprProps)); + + auto condition = filter->clone(); + if (ExpressionUtils::findAny(expr, {Expression::Kind::kDstProperty})) { + auto visitor = ExtractFilterExprVisitor::makePushGetVertices(qctx_->objPool()); + filter->accept(&visitor); + if (!visitor.ok()) { + return Status::SemanticError("filter error"); + } + subgraphCtx_->edgeFilter = visitor.remainedExpr(); + auto tagFilter = visitor.extractedExpr() ? visitor.extractedExpr() : filter; + subgraphCtx_->tagFilter = rewriteDstProp2SrcProp(tagFilter); + } else { + subgraphCtx_->edgeFilter = condition; + } + subgraphCtx_->filter = rewriteDstProp2SrcProp(condition); + return Status::OK(); +} + // Validate yield clause, which only supports YIELD vertices or edges Status GetSubgraphValidator::validateYield(YieldClause* yield) { if (yield == nullptr) { diff --git a/src/graph/validator/GetSubgraphValidator.h b/src/graph/validator/GetSubgraphValidator.h index ae8caa5dfba..39929fdfe5b 100644 --- a/src/graph/validator/GetSubgraphValidator.h +++ b/src/graph/validator/GetSubgraphValidator.h @@ -25,6 +25,8 @@ class GetSubgraphValidator final : public Validator { Status validateBothInOutBound(BothInOutClause* out); + Status validateWhere(WhereClause* where); + Status validateYield(YieldClause* yield); AstContext* getAstContext() override { diff --git a/src/graph/validator/test/GetSubgraphValidatorTest.cpp b/src/graph/validator/test/GetSubgraphValidatorTest.cpp index 57b4ea6324a..26c79d9cdf7 100644 --- a/src/graph/validator/test/GetSubgraphValidatorTest.cpp +++ b/src/graph/validator/test/GetSubgraphValidatorTest.cpp @@ -22,10 +22,7 @@ TEST_F(GetSubgraphValidatorTest, Base) { std::string query = "GET SUBGRAPH FROM \"1\" YIELD vertices as nodes"; std::vector expected = { PK::kDataCollect, - PK::kLoop, - PK::kStart, PK::kSubgraph, - PK::kGetNeighbors, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -34,22 +31,7 @@ TEST_F(GetSubgraphValidatorTest, Base) { std::string query = "GET SUBGRAPH WITH PROP 3 STEPS FROM \"1\" YIELD edges as relationships"; std::vector expected = { PK::kDataCollect, - PK::kLoop, - PK::kStart, - PK::kSubgraph, - PK::kGetNeighbors, - PK::kStart, - }; - EXPECT_TRUE(checkResult(query, expected)); - } - { - std::string query = "GET SUBGRAPH WITH PROP FROM \"1\" BOTH like YIELD vertices AS a"; - std::vector expected = { - PK::kDataCollect, - PK::kLoop, - PK::kStart, PK::kSubgraph, - PK::kGetNeighbors, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -59,10 +41,7 @@ TEST_F(GetSubgraphValidatorTest, Base) { "GET SUBGRAPH WITH PROP FROM \"1\", \"2\" IN like YIELD vertices as a, edges as b"; std::vector expected = { PK::kDataCollect, - PK::kLoop, - PK::kStart, PK::kSubgraph, - PK::kGetNeighbors, PK::kStart, }; EXPECT_TRUE(checkResult(query, expected)); @@ -76,13 +55,10 @@ TEST_F(GetSubgraphValidatorTest, Input) { "PROP FROM $-.src YIELD vertices as a, edges as b"; std::vector expected = { PK::kDataCollect, - PK::kLoop, - PK::kDedup, PK::kSubgraph, + PK::kDedup, PK::kProject, - PK::kGetNeighbors, PK::kProject, - PK::kStart, PK::kGetNeighbors, PK::kStart, }; @@ -94,13 +70,10 @@ TEST_F(GetSubgraphValidatorTest, Input) { "FROM $a.src YIELD vertices as a, edges as b"; std::vector expected = { PK::kDataCollect, - PK::kLoop, - PK::kDedup, PK::kSubgraph, + PK::kDedup, PK::kProject, - PK::kGetNeighbors, PK::kProject, - PK::kStart, PK::kGetNeighbors, PK::kStart, }; diff --git a/src/graph/visitor/ExtractFilterExprVisitor.cpp b/src/graph/visitor/ExtractFilterExprVisitor.cpp index 39bf6414854..06751fe98d1 100644 --- a/src/graph/visitor/ExtractFilterExprVisitor.cpp +++ b/src/graph/visitor/ExtractFilterExprVisitor.cpp @@ -75,7 +75,15 @@ void ExtractFilterExprVisitor::visit(VariablePropertyExpression *) { } void ExtractFilterExprVisitor::visit(DestPropertyExpression *) { - canBePushed_ = false; + switch (pushType_) { + case PushType::kGetNeighbors: + case PushType::kGetEdges: + canBePushed_ = false; + break; + case PushType::kGetVertices: + canBePushed_ = true; + break; + } } void ExtractFilterExprVisitor::visit(SourcePropertyExpression *) { @@ -222,6 +230,14 @@ void ExtractFilterExprVisitor::ExtractRemainExpr(LogicalExpression *expr, } operands.resize(lastUsedExprInd); + if (lastUsedExprInd > 1) { + auto extractedExpr = LogicalExpression::makeAnd(pool_); + extractedExpr->setOperands(operands); + extractedExpr_ = std::move(extractedExpr); + } else { + extractedExpr_ = operands[0]; + } + if (remainedOperands.size() > 1) { auto remainedExpr = LogicalExpression::makeAnd(pool_); remainedExpr->setOperands(std::move(remainedOperands)); diff --git a/src/graph/visitor/ExtractFilterExprVisitor.h b/src/graph/visitor/ExtractFilterExprVisitor.h index f4c6fd98c43..43e2d74d004 100644 --- a/src/graph/visitor/ExtractFilterExprVisitor.h +++ b/src/graph/visitor/ExtractFilterExprVisitor.h @@ -20,6 +20,10 @@ class ExtractFilterExprVisitor final : public ExprVisitorImpl { return canBePushed_; } + Expression *extractedExpr() { + return extractedExpr_; + } + Expression *remainedExpr() { return remainedExpr_; } @@ -88,6 +92,7 @@ class ExtractFilterExprVisitor final : public ExprVisitorImpl { bool hasSplit{false}; bool splitForbidden{false}; Expression *remainedExpr_{nullptr}; + Expression *extractedExpr_{nullptr}; PushType pushType_{PushType::kGetNeighbors}; }; diff --git a/src/interface/storage.thrift b/src/interface/storage.thrift index 42779735e32..2a95ba3ee12 100644 --- a/src/interface/storage.thrift +++ b/src/interface/storage.thrift @@ -159,6 +159,9 @@ struct TraverseSpec { // If provided, only the rows satisfied the given expression will be returned 11: optional binary filter, // only contain filter expression for tag, tag_filter is a subset of filter + // Attention: tag_filter cannot extract expression from filter + // when filter contains logicalOR expression + // bcz $^.player.age > 30 OR like.likeness > 80 can't filter data only by tag_Filter 12: optional binary tag_filter, } diff --git a/src/parser/TraverseSentences.cpp b/src/parser/TraverseSentences.cpp index 5e21a6b3c44..af1fd83ec47 100644 --- a/src/parser/TraverseSentences.cpp +++ b/src/parser/TraverseSentences.cpp @@ -277,6 +277,10 @@ std::string GetSubgraphSentence::toString() const { buf += " "; buf += both_->toString(); } + if (where_ != nullptr) { + buf += " "; + buf += where_->toString(); + } if (yield_ != nullptr) { buf += " "; buf += yield_->toString(); diff --git a/src/parser/TraverseSentences.h b/src/parser/TraverseSentences.h index 54d14713aa4..b4ef8bb11f0 100644 --- a/src/parser/TraverseSentences.h +++ b/src/parser/TraverseSentences.h @@ -606,6 +606,7 @@ class GetSubgraphSentence final : public Sentence { InBoundClause* in, OutBoundClause* out, BothInOutClause* both, + WhereClause* where, YieldClause* yield) { kind_ = Kind::kGetSubgraph; withProp_ = withProp; @@ -614,6 +615,7 @@ class GetSubgraphSentence final : public Sentence { in_.reset(in); out_.reset(out); both_.reset(both); + where_.reset(where); yield_.reset(yield); } @@ -641,6 +643,10 @@ class GetSubgraphSentence final : public Sentence { return both_.get(); } + WhereClause* where() const { + return where_.get(); + } + YieldClause* yield() const { return yield_.get(); } @@ -654,6 +660,7 @@ class GetSubgraphSentence final : public Sentence { std::unique_ptr in_; std::unique_ptr out_; std::unique_ptr both_; + std::unique_ptr where_; std::unique_ptr yield_; }; } // namespace nebula diff --git a/src/parser/parser.yy b/src/parser/parser.yy index 6fa80066c75..bfd06536385 100644 --- a/src/parser/parser.yy +++ b/src/parser/parser.yy @@ -2387,8 +2387,8 @@ both_in_out_clause | KW_BOTH over_edges { $$ = new BothInOutClause($2, BoundClause::BOTH); } get_subgraph_sentence - : KW_GET KW_SUBGRAPH opt_with_properties step_clause from_clause in_bound_clause out_bound_clause both_in_out_clause yield_clause { - $$ = new GetSubgraphSentence($3, $4, $5, $6, $7, $8, $9); + : KW_GET KW_SUBGRAPH opt_with_properties step_clause from_clause in_bound_clause out_bound_clause both_in_out_clause where_clause yield_clause { + $$ = new GetSubgraphSentence($3, $4, $5, $6, $7, $8, $9, $10); } use_sentence diff --git a/src/storage/exec/GetNeighborsNode.h b/src/storage/exec/GetNeighborsNode.h index 1125c53ea11..ee9719b7458 100644 --- a/src/storage/exec/GetNeighborsNode.h +++ b/src/storage/exec/GetNeighborsNode.h @@ -53,6 +53,8 @@ class GetNeighborsNode : public QueryNode { if (context_->resultStat_ == ResultStatus::TAG_FILTER_OUT) { // if the filter condition of the tag is not satisfied // do not return the data for this vertex and corresponding edge + // todo (need lock when multi-thread?) + context_->resultStat_ = ResultStatus::NORMAL; return nebula::cpp2::ErrorCode::SUCCEEDED; } diff --git a/tests/tck/features/subgraph/subgraphWithFilter.feature b/tests/tck/features/subgraph/subgraphWithFilter.feature new file mode 100644 index 00000000000..c5793a5ced5 --- /dev/null +++ b/tests/tck/features/subgraph/subgraphWithFilter.feature @@ -0,0 +1,130 @@ +# Copyright (c) 2022 vesoft inc. All rights reserved. +# +# This source code is licensed under Apache 2.0 License. +Feature: subgraph with fitler + + Background: + Given a graph with space named "nba" + + Scenario: subgraph with edge filter + When executing query: + """ + GET SUBGRAPH FROM 'Tim Duncan' OUT like WHERE like.likeness > 90 YIELD vertices as v, edges as e + """ + Then the result should be, in any order, with relax comparison: + | v | e | + | [("Tim Duncan")] | [[:like "Tim Duncan"->"Manu Ginobili" @0 {likeness: 95}], [:like "Tim Duncan"->"Tony Parker" @0 {likeness: 95}]] | + | [("Manu Ginobili"), ("Tony Parker")] | [[:like "Tony Parker"->"Manu Ginobili" @0 {likeness: 95}], [:like "Tony Parker"->"Tim Duncan" @0 {likeness: 95}]] | + When executing query: + """ + GET SUBGRAPH FROM 'Tim Duncan' BOTH like WHERE like.likeness > 90 YIELD vertices as v, edges as e + """ + Then the result should be, in any order, with relax comparison: + | v | e | + | [("Tim Duncan")] | [[:like "Dejounte Murray"->"Tim Duncan" @0 {likeness: 99}],[:like "Tony Parker"->"Tim Duncan" @0 {likeness: 95}], [:like "Tim Duncan"->"Manu Ginobili" @0 {likeness: 95}], [:like "Tim Duncan"->"Tony Parker" @0 {likeness: 95}]] | + | [("Dejounte Murray"), ("Manu Ginobili"), ("Tony Parker")] | [[:like "Dejounte Murray"->"Manu Ginobili" @0 {likeness: 99}], [:like "Dejounte Murray"->"Tony Parker" @0 {likeness: 99}],[:like "Tony Parker"->"Manu Ginobili" @0 {likeness: 95}]] | + When executing query: + """ + GET SUBGRAPH WITH PROP 2 STEPS FROM 'Tim Duncan' + WHERE like.likeness > 95 AND like.likeness < 100 + YIELD vertices as nodes, edges as relationships + """ + Then define some list variables: + | edge2 | vertex3 | + | [:like "Dejounte Murray"->"Chris Paul"@0] | ("Kyle Anderson") | + | [:like "Dejounte Murray"->"Danny Green"@0] | ("Kevin Durant") | + | [:like "Dejounte Murray"->"James Harden"@0] | ("James Harden") | + | [:like "Dejounte Murray"->"Kevin Durant"@0] | ("Chris Paul") | + | [:like "Dejounte Murray"->"Kyle Anderson"@0] | ("Danny Green") | + | [:like "Dejounte Murray"->"LeBron James"@0] | ("Marco Belinelli") | + | [:like "Dejounte Murray"->"Manu Ginobili"@0] | ("LeBron James") | + | [:like "Dejounte Murray"->"Marco Belinelli"@0] | ("Manu Ginobili") | + | [:like "Dejounte Murray"->"Russell Westbrook"@0] | ("Russell Westbrook") | + | [:like "Dejounte Murray"->"Tony Parker"@0] | ("Tony Parker") | + Then the result should be, in any order, with relax comparison: + | nodes | relationships | + | [("Tim Duncan")] | [[:like "Dejounte Murray"->"Tim Duncan"@0]] | + | [("Dejounte Murray")] | <[edge2]> | + | <[vertex3]> | [] | + When executing query: + """ + GET SUBGRAPH WITH PROP 2 STEPS FROM 'Tim Duncan' IN like, serve + WHERE like.likeness > 80 + YIELD vertices as nodes, edges as relationships + """ + Then define some list variables: + | edge1 | vertex2 | edge2 | vertex3 | + | [:like "Dejounte Murray"->"Tim Duncan"@0] | ("Manu Ginobili") | [:like "Dejounte Murray"->"Manu Ginobili"@0] | ("Tiago Splitter") | + | [:like "Manu Ginobili"->"Tim Duncan"@0] | ("Dejounte Murray") | [:like "Tiago Splitter"->"Manu Ginobili"@0] | | + | [:like "Tony Parker"->"Tim Duncan"@0] | ("Tony Parker") | [:like "Tim Duncan"->"Manu Ginobili"@0] | | + | | | [:like "Tony Parker"->"Manu Ginobili"@0] | | + | | | [:like "Dejounte Murray"->"Tony Parker"@0] | | + | | | [:like "Tim Duncan"->"Tony Parker"@0] | | + Then the result should be, in any order, with relax comparison: + | nodes | relationships | + | [("Tim Duncan")] | <[edge1]> | + | <[vertex2]> | <[edge2]> | + | <[vertex3]> | [] | + When executing query: + """ + GET SUBGRAPH WITH PROP 2 STEPS FROM 'Tim Duncan', 'James Harden' OUT serve + WHERE serve.start_year > 2012 + YIELD vertices as nodes, edges as relationships + """ + Then the result should be, in any order, with relax comparison: + | nodes | relationships | + | [("Tim Duncan" :bachelor{name: "Tim Duncan", speciality: "psychology"} :player{age: 42, name: "Tim Duncan"}), ("James Harden" :player{age: 29, name: "James Harden"})] | [] | + When executing query: + """ + GET SUBGRAPH WITH PROP 2 STEPS FROM 'Tim Duncan', 'James Harden' OUT serve + WHERE serve.start_year > 1996 + YIELD vertices as nodes, edges as relationships + """ + Then the result should be, in any order, with relax comparison: + | nodes | relationships | + | [("Tim Duncan" :bachelor{name: "Tim Duncan", speciality: "psychology"} :player{age: 42, name: "Tim Duncan"}), ("James Harden" :player{age: 29, name: "James Harden"})] | [[:serve "Tim Duncan"->"Spurs" @0 {end_year: 2016, start_year: 1997}], [:serve "James Harden"->"Rockets" @0 {end_year: 2019, start_year: 2012}], [:serve "James Harden"->"Thunders" @0 {end_year: 2012, start_year: 2009}]] | + | [("Spurs" :team{name: "Spurs"}), ("Rockets" :team{name: "Rockets"}), ("Thunders" :team{name: "Thunders"})] | [] | + + Scenario: subgraph with tag filter + When executing query: + """ + GET SUBGRAPH WITH PROP FROM 'Tim Duncan' OUT like WHERE $$.player.age > 36 YIELD vertices as v, edges as e + """ + Then the result should be, in any order, with relax comparison: + | v | e | + | [("Tim Duncan" :bachelor{name: "Tim Duncan", speciality: "psychology"} :player{age: 42, name: "Tim Duncan"})] | [[:like "Tim Duncan"->"Manu Ginobili" @0 {likeness: 95}]] | + | [("Manu Ginobili" :player{age: 41, name: "Manu Ginobili"})] | [[:like "Manu Ginobili"->"Tim Duncan" @0 {likeness: 90}]] | + When executing query: + """ + GO FROM 'Tim Duncan' over serve YIELD serve._src AS id + | GET SUBGRAPH WITH PROP FROM $-.id + WHERE $$.player.age > 36 + YIELD VERTICES as a, EDGES as b + """ + Then the result should be, in any order, with relax comparison: + | a | b | + | [("Tim Duncan" :player{age: 42, name: "Tim Duncan"} :bachelor{name: "Tim Duncan", speciality: "psychology"})] | [[:teammate "Manu Ginobili"->"Tim Duncan" @0 {end_year: 2016, start_year: 2002}], [:like "Tim Duncan"->"Manu Ginobili" @0 {likeness: 95}], [:like "Manu Ginobili"->"Tim Duncan" @0 {likeness: 90}], [:like "Shaquille O'Neal"->"Tim Duncan" @0 {likeness: 80}], [:teammate "Tim Duncan"->"Manu Ginobili" @0 {end_year: 2016, start_year: 2002}]] | + | [("Manu Ginobili" :player{age: 41, name: "Manu Ginobili"}), ("Shaquille O'Neal" :player{age: 47, name: "Shaquille O'Neal"})] | [] | + + Scenario: subgraph with tag and edge filter + When executing query: + """ + GO FROM 'Tim Duncan' over serve YIELD serve._src AS id + | GET SUBGRAPH WITH PROP FROM $-.id + WHERE $$.player.age > 36 AND like.likeness > 80 + YIELD VERTICES as a, EDGES as b + """ + Then the result should be, in any order, with relax comparison: + | a | b | + | [("Tim Duncan" :bachelor{name: "Tim Duncan", speciality: "psychology"} :player{age: 42, name: "Tim Duncan"})] | [[:like "Tim Duncan"->"Manu Ginobili" @0 {likeness: 95}], [:like "Manu Ginobili"->"Tim Duncan" @0 {likeness: 90}]] | + | [("Manu Ginobili" :player{age: 41, name: "Manu Ginobili"})] | [] | + When executing query: + """ + GET SUBGRAPH WITH PROP 5 steps from 'Tony Parker' BOTH like + WHERE $$.player.age > 36 AND like.likeness > 80 + YIELD VERTICES as nodes, EDGES as relationships + """ + Then the result should be, in any order, with relax comparison: + | nodes | relationships | + | [("Tony Parker" :player{age: 36, name: "Tony Parker"})] | [[:like "Tim Duncan"->"Tony Parker" @0 {likeness: 95}], [:like "Tony Parker"->"Manu Ginobili" @0 {likeness: 95}], [:like "Tony Parker"->"Tim Duncan" @0 {likeness: 95}]] | + | [("Tim Duncan" :bachelor{name: "Tim Duncan", speciality: "psychology"} :player{age: 42, name: "Tim Duncan"}), ("Manu Ginobili" :player{age: 41, name: "Manu Ginobili"})] | [[:like "Tim Duncan"->"Manu Ginobili" @0 {likeness: 95}], [:like "Manu Ginobili"->"Tim Duncan" @0 {likeness: 90}]] |