From e4d0ebdad0319c6a1953a0d035ff58df591a973a Mon Sep 17 00:00:00 2001 From: DongZe Li <9546726@qq.com> Date: Thu, 10 Jun 2021 15:18:58 +0800 Subject: [PATCH] Add Lazy mode support to GraphScope (#340) * Add Lazy mode support to GraphScope As an important performance optimization technique, lazy evaluation has been widely applied by many big data processing systems like TensorFlow, which provides three-fold benefits compared with eager evaluation. Eager VS Lazy: Eager execution is a flexible platform for research and experimentation, it provides: - An intuitive interface: Quickly test on small data. - Easier debugging: Call ops directly to inspect running models and test changes. Lazy execution means GraphScope does not process the data till it has to. It just gathers all the information to a DAG that we feed into it, and processes only when we execute code `sess.run(fetches)` --- .github/workflows/ci.yml | 5 +- .github/workflows/local.yml | 2 +- analytical_engine/core/grape_instance.cc | 4 + .../core/server/graphscope_service.cc | 159 +-- analytical_engine/frame/project_frame.cc | 6 +- coordinator/gscoordinator/coordinator.py | 259 +++-- coordinator/gscoordinator/dag_manager.py | 71 ++ .../template/CMakeLists.template | 2 +- coordinator/gscoordinator/utils.py | 355 +++++++ k8s/ubuntu/gsvineyard.Dockerfile | 2 +- proto/attr_value.proto | 4 +- proto/error_codes.proto | 7 + proto/message.proto | 13 +- proto/op_def.proto | 24 + proto/types.proto | 30 +- python/graphscope/client/session.py | 211 ++-- python/graphscope/config.py | 3 + python/graphscope/dataset/ldbc.py | 363 ++++++- .../graphscope/deploy/kubernetes/cluster.py | 5 +- python/graphscope/framework/app.py | 153 ++- python/graphscope/framework/context.py | 306 +++--- python/graphscope/framework/dag.py | 164 +++ python/graphscope/framework/dag_utils.py | 241 +++-- python/graphscope/framework/errors.py | 10 + python/graphscope/framework/graph.py | 939 +++++++++--------- python/graphscope/framework/operation.py | 88 +- python/graphscope/framework/utils.py | 11 +- python/graphscope/interactive/query.py | 13 +- python/graphscope/nx/classes/graph.py | 13 +- python/graphscope/nx/utils/other.py | 2 +- python/tests/conftest.py | 238 ++++- python/tests/test_create_graph.py | 8 +- python/tests/test_graph.py | 23 +- python/tests/test_lazy.py | 158 +++ python/tests/test_session.py | 7 +- 35 files changed, 2676 insertions(+), 1223 deletions(-) create mode 100644 coordinator/gscoordinator/dag_manager.py create mode 100644 python/graphscope/framework/dag.py create mode 100644 python/tests/test_lazy.py diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 7ba72c5093d8..e2f4f053ab8b 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -340,8 +340,9 @@ jobs: export GS_TEST_DIR='/root/gstest' python3 -m pytest --exitfirst -s -v python/graphscope/nx/tests/classes python3 -m pytest --exitfirst -s -v python/graphscope/nx/tests/test_nx.py - python3 -m pytest --exitfirst -s -v python/graphscope/nx/tests/algorithms/builtin \ - python/graphscope/nx/tests/test_utils.py + python3 -m pytest --exitfirst -s -v python/graphscope/nx/tests/test_ctx_builtin.py + python3 -m pytest --exitfirst -s -v python/graphscope/nx/tests/algorithms/builtin + python3 -m pytest --exitfirst -s -v python/graphscope/nx/tests/test_utils.py pkill -TERM etcd || true diff --git a/.github/workflows/local.yml b/.github/workflows/local.yml index cdd9bbd58b55..369f2ccf25bc 100644 --- a/.github/workflows/local.yml +++ b/.github/workflows/local.yml @@ -85,7 +85,7 @@ jobs: - name: Install libvineyard run: | - git clone -b v0.2.3 --single-branch --depth=1 https://github.com/alibaba/libvineyard.git + git clone -b main-v0.2.3 --single-branch --depth=1 https://github.com/alibaba/libvineyard.git cd libvineyard git submodule update --init mkdir build && pushd build diff --git a/analytical_engine/core/grape_instance.cc b/analytical_engine/core/grape_instance.cc index dba8803c3c67..22ebe22e143a 100644 --- a/analytical_engine/core/grape_instance.cc +++ b/analytical_engine/core/grape_instance.cc @@ -897,6 +897,10 @@ bl::result> GrapeInstance::OnReceive( break; } case rpc::CREATE_APP: { + // do nothing + break; + } + case rpc::BIND_APP: { BOOST_LEAF_AUTO(app_name, loadApp(params)); r->set_data(app_name); break; diff --git a/analytical_engine/core/server/graphscope_service.cc b/analytical_engine/core/server/graphscope_service.cc index 42821d9c2849..04f8bae5c8b9 100644 --- a/analytical_engine/core/server/graphscope_service.cc +++ b/analytical_engine/core/server/graphscope_service.cc @@ -13,6 +13,8 @@ * limitations under the License. */ +#include + #include "core/server/graphscope_service.h" #include "core/server/rpc_utils.h" @@ -34,91 +36,102 @@ ::grpc::Status GraphScopeService::RunStep(::grpc::ServerContext* context, RunStepResponse* response) { CHECK(request->has_dag_def()); const DagDef& dag_def = request->dag_def(); - CHECK_EQ(dag_def.op().size(), 1); - const auto& op = dag_def.op(0); - - CommandDetail cmd = OpToCmd(op); - auto result = dispatcher_->Dispatch(cmd); - auto policy = result[0].aggregate_policy(); - bool success = true; - std::string error_msgs; - - // First pass: make sure all result are valid and check the consistency - for (auto& e : result) { - auto ok = (e.error_code() == rpc::Code::OK); - - if (ok) { - CHECK_EQ(e.aggregate_policy(), policy); - auto& graph_def = e.graph_def(); - - if (!graph_def.key().empty()) { - if (response->graph_def().key().empty()) { - response->mutable_graph_def()->CopyFrom(graph_def); - } else if (graph_def.SerializeAsString() != - response->graph_def().SerializeAsString()) { - LOG(FATAL) << "BUG: Multiple workers return different graph def."; - } - } - } else { - error_msgs += e.message() + "\n"; - } - - success &= ok; - } - auto* res_status = response->mutable_status(); + std::unordered_map op_key_to_result; - if (!success) { - res_status->set_code(rpc::Code::ANALYTICAL_ENGINE_INTERNAL_ERROR); - res_status->set_error_msg(error_msgs); - OpDef* opdef = res_status->mutable_op(); - opdef->CopyFrom(op); - } + for (const auto& op : dag_def.op()) { + OpResult* op_result = response->add_results(); + op_result->set_key(op.key()); + op_key_to_result.emplace(op.key(), op_result); + CommandDetail cmd = OpToCmd(op); - // Second pass: aggregate result according to the policy - switch (policy) { - case DispatchResult::AggregatePolicy::kPickFirst: { - response->mutable_result()->assign(result[0].data()); - break; - } - case DispatchResult::AggregatePolicy::kPickFirstNonEmpty: { - for (auto& e : result) { - auto& data = e.data(); + bool success = true; + std::string error_msgs; + auto result = dispatcher_->Dispatch(cmd); + auto policy = result[0].aggregate_policy(); - if (!data.empty()) { - response->mutable_result()->assign(data.begin(), data.end()); - break; + // First pass: make sure all result are valid and check the consistency + for (auto& e : result) { + auto ok = (e.error_code() == rpc::Code::OK); + if (ok) { + CHECK_EQ(e.aggregate_policy(), policy); + auto& graph_def = e.graph_def(); + + if (!graph_def.key().empty()) { + if (op_result->graph_def().key().empty()) { + op_result->mutable_graph_def()->CopyFrom(graph_def); + } else if (graph_def.SerializeAsString() != + op_result->graph_def().SerializeAsString()) { + LOG(FATAL) << "BUG: Multiple workers return different graph def."; + } + } + } else { + error_msgs += e.message() + "\n"; + op_result->set_code(e.error_code()); } + success &= ok; } - break; - } - case DispatchResult::AggregatePolicy::kRequireConsistent: { - for (auto& e : result) { - auto& data = e.data(); - if (response->result().empty()) { - response->mutable_result()->assign(data.begin(), data.end()); - } else if (response->result() != data) { - std::stringstream ss; + if (!success) { + res_status->set_code(rpc::Code::ANALYTICAL_ENGINE_INTERNAL_ERROR); + res_status->set_error_msg(error_msgs); + op_result->set_error_msg(error_msgs); + // break dag exection flow + break; + } - ss << "Error: Multiple workers return different data." - << " Current worker id: " << e.worker_id() << " " << data - << " vs the previous: " << response->result(); + // Second pass: aggregate result according to the policy + switch (policy) { + case DispatchResult::AggregatePolicy::kPickFirst: { + op_result->mutable_result()->assign(result[0].data()); + break; + } + case DispatchResult::AggregatePolicy::kPickFirstNonEmpty: { + for (auto& e : result) { + auto& data = e.data(); - res_status->set_code(rpc::Code::ANALYTICAL_ENGINE_INTERNAL_ERROR); - res_status->set_error_msg(ss.str()); - LOG(ERROR) << ss.str(); - break; + if (!data.empty()) { + op_result->mutable_result()->assign(data.begin(), data.end()); + break; + } } + break; } - break; - } - case DispatchResult::AggregatePolicy::kConcat: { - for (auto& e : result) { - response->mutable_result()->append(e.data()); + case DispatchResult::AggregatePolicy::kRequireConsistent: { + for (auto& e : result) { + auto& data = e.data(); + + if (op_result->result().empty()) { + op_result->mutable_result()->assign(data.begin(), data.end()); + } else if (op_result->result() != data) { + std::stringstream ss; + + ss << "Error: Multiple workers return different data." + << " Current worker id: " << e.worker_id() << " " << data + << " vs the previous: " << op_result->result(); + + op_result->set_code(rpc::Code::WORKER_RESULTS_INCONSISTENT_ERROR); + res_status->set_error_msg(ss.str()); + LOG(ERROR) << ss.str(); + success &= false; + break; + } + } + break; + } + case DispatchResult::AggregatePolicy::kConcat: { + for (auto& e : result) { + op_result->mutable_result()->append(e.data()); + } + break; + } + } + + if (!success) { + res_status->set_code(rpc::Code::ANALYTICAL_ENGINE_INTERNAL_ERROR); + // break dag exection flow + break; } - break; - } } return ::grpc::Status::OK; diff --git a/analytical_engine/frame/project_frame.cc b/analytical_engine/frame/project_frame.cc index 699b8c1c04ef..b3e55506d3f2 100644 --- a/analytical_engine/frame/project_frame.cc +++ b/analytical_engine/frame/project_frame.cc @@ -99,8 +99,10 @@ class ProjectSimpleFrame< if (graph_def.has_extension()) { graph_def.extension().UnpackTo(&vy_info); } - vy_info.set_oid_type(PropertyTypeToPb(parent_meta.GetKeyValue("oid_type"))); - vy_info.set_vid_type(PropertyTypeToPb(parent_meta.GetKeyValue("vid_type"))); + vy_info.set_oid_type(PropertyTypeToPb( + vineyard::normalize_datatype(parent_meta.GetKeyValue("oid_type")))); + vy_info.set_vid_type(PropertyTypeToPb( + vineyard::normalize_datatype(parent_meta.GetKeyValue("vid_type")))); std::string vdata_type, edata_type; if (v_prop != "-1") { diff --git a/coordinator/gscoordinator/coordinator.py b/coordinator/gscoordinator/coordinator.py index 449756373442..210f2eebf598 100644 --- a/coordinator/gscoordinator/coordinator.py +++ b/coordinator/gscoordinator/coordinator.py @@ -24,6 +24,7 @@ import json import logging import os +import pickle import queue import random import signal @@ -53,6 +54,8 @@ from graphscope.proto import types_pb2 from gscoordinator.cluster import KubernetesClusterLauncher +from gscoordinator.dag_manager import DAGManager +from gscoordinator.dag_manager import GSEngine from gscoordinator.launcher import LocalLauncher from gscoordinator.object_manager import GraphMeta from gscoordinator.object_manager import LibMeta @@ -64,6 +67,7 @@ from gscoordinator.utils import get_app_sha256 from gscoordinator.utils import get_graph_sha256 from gscoordinator.utils import get_lib_path +from gscoordinator.utils import op_pre_process from gscoordinator.utils import str2bool from gscoordinator.utils import to_maxgraph_schema from gscoordinator.version import __version__ @@ -201,6 +205,9 @@ def ConnectSession(self, request, context): # Generate session id self._session_id = self._generate_session_id() + self._key_to_op = dict() + # dict of op_def_pb2.OpResult + self._op_result_pool = dict() self._udf_app_workspace = os.path.join(WORKSPACE, self._session_id) # Session connected, fetch logs via gRPC. @@ -249,70 +256,84 @@ def HeartBeat(self, request, context): message_pb2.HeartBeatResponse, error_codes_pb2.OK ) - def RunStep(self, request, context): # noqa: C901 - # only one op in one step is allowed. - if len(request.dag_def.op) != 1: - return self._make_response( - message_pb2.RunStepResponse, - error_codes_pb2.INVALID_ARGUMENT_ERROR, - "Request's op size is not equal to 1.", - ) - - op = request.dag_def.op[0] - - # Compile app or not. - if op.op == types_pb2.CREATE_APP: + def run_on_analytical_engine( # noqa: C901 + self, session_id, dag_def: op_def_pb2.DagDef, op_results: list + ): + for op in dag_def.op: + self._key_to_op[op.key] = op try: - op, app_sig, app_lib_path = self._maybe_compile_app(op) + op_pre_process(op, self._op_result_pool, self._key_to_op) except Exception as e: - error_msg = "Failed to compile app: {}".format(str(e)) + error_msg = ( + "Failed to pre process op {0} with error message {1}".format( + op, str(e) + ) + ) logger.error(error_msg) return self._make_response( message_pb2.RunStepResponse, - error_codes_pb2.COMPILATION_ERROR, + error_codes_pb2.COORDINATOR_INTERNAL_ERROR, error_msg, - op, + full_exception=pickle.dumps(e), + results=op_results, ) - # If engine crashed, we will get a SocketClosed grpc Exception. - # In that case, we should notify client the engine is dead. - - # Compile graph or not - # arrow property graph and project graph need to compile - if ( - ( - op.op == types_pb2.CREATE_GRAPH - and op.attr[types_pb2.GRAPH_TYPE].graph_type - == graph_def_pb2.ARROW_PROPERTY - ) - or op.op == types_pb2.TRANSFORM_GRAPH - or op.op == types_pb2.PROJECT_TO_SIMPLE - or op.op == types_pb2.ADD_LABELS - ): - try: - op = self._maybe_register_graph(op, request.session_id) - except grpc.RpcError as e: - logger.error("self._launcher.poll() = %s", self._launcher.poll()) - if self._launcher.poll() is not None: - message = "Analytical engine exited with %s" % self._launcher.poll() - else: - message = str(e) - return self._make_response( - message_pb2.RunStepResponse, - error_codes_pb2.FATAL_ERROR, - message, - op, - ) - except Exception as e: - error_msg = "Graph compile error: {}".format(str(e)) - logger.error(error_msg) - return self._make_response( - message_pb2.RunStepResponse, - error_codes_pb2.COMPILATION_ERROR, - error_msg, - op, + # Compile app or not. + if op.op == types_pb2.BIND_APP: + try: + op, app_sig, app_lib_path = self._maybe_compile_app(op) + except Exception as e: + error_msg = "Failed to compile app: {0}".format(str(e)) + logger.error(error_msg) + return self._make_response( + message_pb2.RunStepResponse, + error_codes_pb2.COMPILATION_ERROR, + error_msg, + results=op_results, + ) + # Compile graph or not + # arrow property graph and project graph need to compile + # If engine crashed, we will get a SocketClosed grpc Exception. + # In that case, we should notify client the engine is dead. + if ( + ( + op.op == types_pb2.CREATE_GRAPH + and op.attr[types_pb2.GRAPH_TYPE].graph_type + == graph_def_pb2.ARROW_PROPERTY ) + or op.op == types_pb2.TRANSFORM_GRAPH + or op.op == types_pb2.PROJECT_TO_SIMPLE + or op.op == types_pb2.ADD_LABELS + ): + try: + op = self._maybe_register_graph(op, session_id) + except grpc.RpcError as e: + logger.error("self._launcher.poll() = %s", self._launcher.poll()) + if self._launcher.poll() is not None: + message = ( + "Analytical engine exited with %s" % self._launcher.poll() + ) + else: + message = str(e) + return self._make_response( + message_pb2.RunStepResponse, + error_codes_pb2.COMPILATION_ERROR, + message, + results=op_results, + ) + except Exception as e: + error_msg = "Graph compile error: {}".format(str(e)) + logger.error(error_msg) + return self._make_response( + message_pb2.RunStepResponse, + error_codes_pb2.COMPILATION_ERROR, + error_msg, + results=op_results, + ) + request = message_pb2.RunStepRequest( + session_id=self._session_id, dag_def=dag_def + ) try: response = self._analytical_engine_stub.RunStep(request) except grpc.RpcError as e: @@ -321,53 +342,93 @@ def RunStep(self, request, context): # noqa: C901 message = "Analytical engine exited with %s" % self._launcher.poll() else: message = str(e) + op_results.extend(response.results) return self._make_response( - message_pb2.RunStepResponse, error_codes_pb2.FATAL_ERROR, message, op + message_pb2.RunStepResponse, + error_codes_pb2.FATAL_ERROR, + message, + results=op_results, ) except Exception as e: + op_results.extend(response.results) return self._make_response( - message_pb2.RunStepResponse, error_codes_pb2.UNKNOWN, str(e), op + message_pb2.RunStepResponse, + error_codes_pb2.UNKNOWN, + str(e), + results=op_results, ) - if response.status.code == error_codes_pb2.OK: - if op.op in ( - types_pb2.CREATE_GRAPH, - types_pb2.PROJECT_GRAPH, - types_pb2.ADD_LABELS, - types_pb2.ADD_COLUMN, + op_results.extend(response.results) + for r in response.results: + op = self._key_to_op[r.key] + if op.op not in ( + types_pb2.CONTEXT_TO_NUMPY, + types_pb2.CONTEXT_TO_DATAFRAME, + types_pb2.TO_VINEYARD_TENSOR, + types_pb2.TO_VINEYARD_DATAFRAME, + types_pb2.REPORT_GRAPH, ): - schema_path = os.path.join("/tmp", response.graph_def.key + ".json") - vy_info = graph_def_pb2.VineyardInfoPb() - response.graph_def.extension.Unpack(vy_info) + self._op_result_pool[r.key] = r - self._object_manager.put( - response.graph_def.key, - GraphMeta( - response.graph_def.key, - vy_info.vineyard_id, - response.graph_def, - schema_path, - ), - ) - if response.graph_def.graph_type == graph_def_pb2.ARROW_PROPERTY: - dump_string( - to_maxgraph_schema(vy_info.property_schema_json), - schema_path, + if response.status.code == error_codes_pb2.OK: + for op_result in response.results: + key = op_result.key + op = self._key_to_op[key] + if op.op in ( + types_pb2.CREATE_GRAPH, + types_pb2.PROJECT_GRAPH, + types_pb2.ADD_LABELS, + types_pb2.ADD_COLUMN, + ): + schema_path = os.path.join( + "/tmp", op_result.graph_def.key + ".json" ) - vy_info.schema_path = schema_path - response.graph_def.extension.Pack(vy_info) - elif op.op == types_pb2.CREATE_APP: - self._object_manager.put( - app_sig, - LibMeta(response.result.decode("utf-8"), "app", app_lib_path), - ) - elif op.op == types_pb2.UNLOAD_GRAPH: - self._object_manager.pop(op.attr[types_pb2.GRAPH_NAME].s.decode()) - elif op.op == types_pb2.UNLOAD_APP: - self._object_manager.pop(op.attr[types_pb2.APP_NAME].s.decode()) - + vy_info = graph_def_pb2.VineyardInfoPb() + op_result.graph_def.extension.Unpack(vy_info) + self._object_manager.put( + op_result.graph_def.key, + GraphMeta( + op_result.graph_def.key, + vy_info.vineyard_id, + op_result.graph_def, + schema_path, + ), + ) + if op_result.graph_def.graph_type == graph_def_pb2.ARROW_PROPERTY: + dump_string( + to_maxgraph_schema(vy_info.property_schema_json), + schema_path, + ) + vy_info.schema_path = schema_path + op_result.graph_def.extension.Pack(vy_info) + elif op.op == types_pb2.BIND_APP: + self._object_manager.put( + app_sig, + LibMeta(op_result.result.decode("utf-8"), "app", app_lib_path), + ) + elif op.op == types_pb2.UNLOAD_GRAPH: + self._object_manager.pop(op.attr[types_pb2.GRAPH_NAME].s.decode()) + elif op.op == types_pb2.UNLOAD_APP: + self._object_manager.pop(op.attr[types_pb2.APP_NAME].s.decode()) return response + def RunStep(self, request, context): + op_results = list() + # split dag + dag_manager = DAGManager(request.dag_def) + while not dag_manager.empty(): + next_dag = dag_manager.get_next_dag() + run_dag_on, dag_def = next_dag + if run_dag_on == GSEngine.analytical_engine: + ret = self.run_on_analytical_engine( + request.session_id, dag_def, op_results + ) + if ret.status.code != error_codes_pb2.OK: + return ret + return self._make_response( + message_pb2.RunStepResponse, error_codes_pb2.OK, results=op_results + ) + def _maybe_compile_app(self, op): app_sig = get_app_sha256(op.attr) space = self._builtin_workspace @@ -418,7 +479,11 @@ def _maybe_register_graph(self, op, session_id): if register_response.status.code == error_codes_pb2.OK: self._object_manager.put( graph_sig, - LibMeta(register_response.result, "graph_frame", graph_lib_path), + LibMeta( + register_response.results[0].result, + "graph_frame", + graph_lib_path, + ), ) else: raise RuntimeError("Error occur when register graph") @@ -612,12 +677,17 @@ def CloseLearningInstance(self, request, context): ) @staticmethod - def _make_response(resp_cls, code, error_msg="", op=None, **args): + def _make_response( + resp_cls, code, error_msg="", op=None, full_exception=None, **kwargs + ): resp = resp_cls( - status=message_pb2.ResponseStatus(code=code, error_msg=error_msg), **args + status=message_pb2.ResponseStatus(code=code, error_msg=error_msg), **kwargs ) if op: resp.status.op.CopyFrom(op) + elif full_exception: + # bytes + resp.status.full_exception = full_exception return resp def _cleanup(self, cleanup_instance=True, is_dangling=False): @@ -689,7 +759,8 @@ def _get_engine_config(self): session_id=self._session_id, dag_def=dag_def ) fetch_response = self._analytical_engine_stub.RunStep(fetch_request) - config = json.loads(fetch_response.result.decode("utf-8")) + + config = json.loads(fetch_response.results[0].result.decode("utf-8")) if self._launcher_type == types_pb2.K8S: config["vineyard_service_name"] = self._launcher.get_vineyard_service_name() config["vineyard_rpc_endpoint"] = self._launcher.get_vineyard_rpc_endpoint() diff --git a/coordinator/gscoordinator/dag_manager.py b/coordinator/gscoordinator/dag_manager.py new file mode 100644 index 000000000000..4a86432b0150 --- /dev/null +++ b/coordinator/gscoordinator/dag_manager.py @@ -0,0 +1,71 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2020 Alibaba Group Holding Limited. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import copy +import queue +from enum import Enum + +from graphscope.proto import op_def_pb2 +from graphscope.proto import types_pb2 + + +class GSEngine(Enum): + analytical_engine = 0 + interactive_engine = 1 + learning_engine = 2 + + +class DAGManager(object): + _analytical_engine_split_op = [ + types_pb2.BIND_APP, # need loaded graph to compile + types_pb2.ADD_LABELS, # need loaded graph + types_pb2.RUN_APP, # need loaded app + types_pb2.CONTEXT_TO_NUMPY, # need loaded graph to transform selector + types_pb2.CONTEXT_TO_DATAFRAME, # need loaded graph to transform selector + types_pb2.TO_VINEYARD_TENSOR, # need loaded graph to transform selector + types_pb2.TO_VINEYARD_DATAFRAME, # need loaded graph to transform selector + types_pb2.PROJECT_GRAPH, # need loaded graph to transform selector + types_pb2.PROJECT_TO_SIMPLE, # need loaded graph schema information + types_pb2.ADD_COLUMN, # need ctx result + types_pb2.UNLOAD_GRAPH, # need loaded graph information + ] + + def __init__(self, dag_def: op_def_pb2.DagDef): + self._dag_def = dag_def + self._split_dag_def_queue = queue.Queue() + + # split dag + split_dag_def = op_def_pb2.DagDef() + split_dag_def_for = GSEngine.analytical_engine + for op in self._dag_def.op: + if op.op in self._analytical_engine_split_op: + if len(split_dag_def.op) > 0: + self._split_dag_def_queue.put((split_dag_def_for, split_dag_def)) + split_dag_def = op_def_pb2.DagDef() + split_dag_def_for = GSEngine.analytical_engine + split_dag_def.op.extend([copy.deepcopy(op)]) + if len(split_dag_def.op) > 0: + self._split_dag_def_queue.put((split_dag_def_for, split_dag_def)) + + def empty(self): + return self._split_dag_def_queue.empty() + + def get_next_dag(self): + if not self._split_dag_def_queue.empty(): + return self._split_dag_def_queue.get() + return None diff --git a/coordinator/gscoordinator/template/CMakeLists.template b/coordinator/gscoordinator/template/CMakeLists.template index 5ed1de640667..5fd2fd7708f6 100644 --- a/coordinator/gscoordinator/template/CMakeLists.template +++ b/coordinator/gscoordinator/template/CMakeLists.template @@ -174,6 +174,6 @@ else () target_compile_definitions(${FRAME_NAME} PRIVATE _GRAPH_TYPE=$_graph_type _GRAPH_HEADER=$_graph_header _APP_TYPE=$_app_type _APP_HEADER=$_app_header) target_include_directories(${FRAME_NAME} PRIVATE utils apps) - target_link_libraries(${FRAME_NAME} ${LIBGRAPELITE_LIBRARIES} ${PROTO}) + target_link_libraries(${FRAME_NAME} ${LIBGRAPELITE_LIBRARIES} ${VINEYARD_LIBRARIES} ${PROTO}) set_target_properties(${FRAME_NAME} PROPERTIES COMPILE_FLAGS "-fPIC") endif () diff --git a/coordinator/gscoordinator/utils.py b/coordinator/gscoordinator/utils.py index bc2fc49e93fa..2b5078a1be07 100644 --- a/coordinator/gscoordinator/utils.py +++ b/coordinator/gscoordinator/utils.py @@ -21,6 +21,7 @@ import datetime import glob import hashlib +import inspect import json import logging import numbers @@ -40,6 +41,13 @@ from string import Template import yaml +from graphscope.framework import utils +from graphscope.framework.graph_schema import GraphSchema +from graphscope.framework.utils import transform_labeled_vertex_data_selector +from graphscope.framework.utils import transform_labeled_vertex_property_data_selector +from graphscope.framework.utils import transform_vertex_data_selector +from graphscope.framework.utils import transform_vertex_property_data_selector +from graphscope.proto import attr_value_pb2 from graphscope.proto import graph_def_pb2 from graphscope.proto import op_def_pb2 from graphscope.proto import types_pb2 @@ -330,6 +338,346 @@ def compile_graph_frame(workspace: str, library_name, attr: dict, engine_config: return lib_path +def op_pre_process(op, op_result_pool, key_to_op): + if op.op == types_pb2.REPORT_GRAPH: + # do nothing for nx report graph + return + if op.op == types_pb2.ADD_LABELS: + _pre_process_for_add_labels_op(op, op_result_pool, key_to_op) + if op.op == types_pb2.RUN_APP: + _pre_process_for_run_app_op(op, op_result_pool, key_to_op) + if op.op == types_pb2.BIND_APP: + _pre_process_for_bind_app_op(op, op_result_pool, key_to_op) + if op.op == types_pb2.PROJECT_GRAPH: + _pre_process_for_project_op(op, op_result_pool, key_to_op) + if op.op == types_pb2.PROJECT_TO_SIMPLE: + _pre_process_for_project_to_simple_op(op, op_result_pool, key_to_op) + if op.op == types_pb2.ADD_COLUMN: + _pre_process_for_add_column_op(op, op_result_pool, key_to_op) + if op.op == types_pb2.UNLOAD_GRAPH: + _pre_process_for_unload_graph_op(op, op_result_pool, key_to_op) + if op.op in ( + types_pb2.CONTEXT_TO_NUMPY, + types_pb2.CONTEXT_TO_DATAFRAME, + types_pb2.TO_VINEYARD_TENSOR, + types_pb2.TO_VINEYARD_DATAFRAME, + ): + _pre_process_for_context_op(op, op_result_pool, key_to_op) + + +def _pre_process_for_add_labels_op(op, op_result_pool, key_to_op): + assert len(op.parents) == 1 + key_of_parent_op = op.parents[0] + result = op_result_pool[key_of_parent_op] + op.attr[types_pb2.GRAPH_NAME].CopyFrom(utils.s_to_attr(result.graph_def.key)) + + +# get `bind_app` runtime informarion in lazy mode +def _pre_process_for_bind_app_op(op, op_result_pool, key_to_op): + for key_of_parent_op in op.parents: + parent_op = key_to_op[key_of_parent_op] + if parent_op.op == types_pb2.CREATE_APP: + # app assets + op.attr[types_pb2.APP_ALGO].CopyFrom(parent_op.attr[types_pb2.APP_ALGO]) + if types_pb2.GAR in parent_op.attr: + op.attr[types_pb2.GAR].CopyFrom(parent_op.attr[types_pb2.GAR]) + else: + # get graph runtime information from results + result = op_result_pool[key_of_parent_op] + assert result.graph_def.extension.Is( + graph_def_pb2.VineyardInfoPb.DESCRIPTOR + ) + vy_info = graph_def_pb2.VineyardInfoPb() + result.graph_def.extension.Unpack(vy_info) + op.attr[types_pb2.GRAPH_NAME].CopyFrom( + attr_value_pb2.AttrValue(s=result.graph_def.key.encode("utf-8")) + ) + op.attr[types_pb2.GRAPH_TYPE].CopyFrom( + attr_value_pb2.AttrValue(graph_type=result.graph_def.graph_type) + ) + op.attr[types_pb2.OID_TYPE].CopyFrom( + utils.s_to_attr( + utils.normalize_data_type_str( + utils.data_type_to_cpp(vy_info.oid_type) + ) + ) + ) + op.attr[types_pb2.VID_TYPE].CopyFrom( + utils.s_to_attr(utils.data_type_to_cpp(vy_info.vid_type)) + ) + op.attr[types_pb2.V_DATA_TYPE].CopyFrom( + utils.s_to_attr(utils.data_type_to_cpp(vy_info.vdata_type)) + ) + op.attr[types_pb2.E_DATA_TYPE].CopyFrom( + utils.s_to_attr(utils.data_type_to_cpp(vy_info.edata_type)) + ) + + +# get `run_app` runtime informarion in lazy mode +def _pre_process_for_run_app_op(op, op_result_pool, key_to_op): + # run_app op has only one parent + assert len(op.parents) == 1 + key_of_parent_op = op.parents[0] + parent_op = key_to_op[key_of_parent_op] + assert parent_op.op == types_pb2.BIND_APP + # set graph key + op.attr[types_pb2.GRAPH_NAME].CopyFrom(parent_op.attr[types_pb2.GRAPH_NAME]) + result = op_result_pool[key_of_parent_op] + # set app key + op.attr[types_pb2.APP_NAME].CopyFrom( + attr_value_pb2.AttrValue(s=result.result.decode("utf-8").encode("utf-8")) + ) + + +def _pre_process_for_unload_graph_op(op, op_result_pool, key_to_op): + assert len(op.parents) == 1 + key_of_parent_op = op.parents[0] + result = op_result_pool[key_of_parent_op] + assert result.graph_def.extension.Is(graph_def_pb2.VineyardInfoPb.DESCRIPTOR) + vy_info = graph_def_pb2.VineyardInfoPb() + result.graph_def.extension.Unpack(vy_info) + op.attr[types_pb2.GRAPH_NAME].CopyFrom(utils.s_to_attr(result.graph_def.key)) + op.attr[types_pb2.VINEYARD_ID].CopyFrom(utils.i_to_attr(vy_info.vineyard_id)) + + +def _pre_process_for_add_column_op(op, op_result_pool, key_to_op): + for key_of_parent_op in op.parents: + parent_op = key_to_op[key_of_parent_op] + if parent_op.op != types_pb2.RUN_APP: + # get graph information + r = op_result_pool[key_of_parent_op] + graph_name = r.graph_def.key + graph_type = r.graph_def.graph_type + schema = GraphSchema() + schema.from_graph_def(r.graph_def) + for key_of_parent_op in op.parents: + parent_op = key_to_op[key_of_parent_op] + if parent_op.op == types_pb2.RUN_APP: + selector = op.attr[types_pb2.SELECTOR].s.decode("utf-8") + r = op_result_pool[key_of_parent_op] + parent_op_result = json.loads(r.result.decode("utf-8")) + context_key = parent_op_result["context_key"] + context_type = parent_op_result["context_type"] + selector = _tranform_dataframe_selector(context_type, schema, selector) + op.attr[types_pb2.GRAPH_NAME].CopyFrom(utils.s_to_attr(graph_name)) + op.attr[types_pb2.GRAPH_TYPE].CopyFrom(utils.graph_type_to_attr(graph_type)) + op.attr[types_pb2.CTX_NAME].CopyFrom(utils.s_to_attr(context_key)) + op.attr[types_pb2.SELECTOR].CopyFrom(utils.s_to_attr(selector)) + + +def _pre_process_for_context_op(op, op_result_pool, key_to_op): + def __backtrack_key_of_graph_op(key): + bfs_queue = Queue() + bfs_queue.put(key) + while not bfs_queue.empty(): + next_op_key = bfs_queue.get() + if next_op_key in key_to_op: + next_op = key_to_op[next_op_key] + if next_op.op in ( + types_pb2.CREATE_GRAPH, + types_pb2.ADD_LABELS, + types_pb2.TRANSFORM_GRAPH, + types_pb2.PROJECT_GRAPH, + types_pb2.PROJECT_TO_SIMPLE, + ): + return next_op + for parent_key in next_op.parents: + bfs_queue.put(parent_key) + return None + + assert len(op.parents) == 1 + schema = None + key_of_parent_op = op.parents[0] + graph_op = __backtrack_key_of_graph_op(key_of_parent_op) + r = op_result_pool[key_of_parent_op] + # set context key + parent_op_result = json.loads(r.result.decode("utf-8")) + context_key = parent_op_result["context_key"] + context_type = parent_op_result["context_type"] + op.attr[types_pb2.CTX_NAME].CopyFrom( + attr_value_pb2.AttrValue(s=context_key.encode("utf-8")) + ) + r = op_result_pool[graph_op.key] + # transform selector + schema = GraphSchema() + schema.from_graph_def(r.graph_def) + selector = op.attr[types_pb2.SELECTOR].s.decode("utf-8") + if op.op in (types_pb2.CONTEXT_TO_DATAFRAME, types_pb2.TO_VINEYARD_DATAFRAME): + selector = _tranform_dataframe_selector(context_type, schema, selector) + else: + # to numpy + selector = _tranform_numpy_selector(context_type, schema, selector) + if selector is not None: + op.attr[types_pb2.SELECTOR].CopyFrom( + attr_value_pb2.AttrValue(s=selector.encode("utf-8")) + ) + + +def _pre_process_for_project_to_simple_op(op, op_result_pool, key_to_op): + # for nx graph + if op.attr[types_pb2.GRAPH_TYPE].graph_type == graph_def_pb2.DYNAMIC_PROJECTED: + return + assert len(op.parents) == 1 + # get parent graph schema + key_of_parent_op = op.parents[0] + r = op_result_pool[key_of_parent_op] + schema = GraphSchema() + schema.from_graph_def(r.graph_def) + graph_name = r.graph_def.key + check_argument( + schema.vertex_label_num == 1, + "Cannot project to simple, vertex label number is not one.", + ) + check_argument( + schema.edge_label_num == 1, + "Cannot project to simple, edge label number is not one.", + ) + v_label = schema.vertex_labels[0] + e_label = schema.edge_labels[0] + relation = (v_label, v_label) + check_argument( + relation in schema.get_relationships(e_label), + f"Cannot project to simple, Graph doesn't contain such relationship: {v_label} -> {e_label} <- {v_label}.", + ) + v_props = schema.get_vertex_properties(v_label) + e_props = schema.get_edge_properties(e_label) + check_argument(len(v_props) <= 1) + check_argument(len(e_props) <= 1) + v_label_id = schema.get_vertex_label_id(v_label) + e_label_id = schema.get_edge_label_id(e_label) + v_prop_id, vdata_type = (v_props[0].id, v_props[0].type) if v_props else (-1, None) + e_prop_id, edata_type = (e_props[0].id, e_props[0].type) if e_props else (-1, None) + oid_type = schema.oid_type + vid_type = schema.vid_type + op.attr[types_pb2.GRAPH_NAME].CopyFrom( + attr_value_pb2.AttrValue(s=graph_name.encode("utf-8")) + ) + op.attr[types_pb2.GRAPH_TYPE].CopyFrom( + utils.graph_type_to_attr(graph_def_pb2.ARROW_PROJECTED) + ) + op.attr[types_pb2.V_LABEL_ID].CopyFrom(utils.i_to_attr(v_label_id)) + op.attr[types_pb2.V_PROP_ID].CopyFrom(utils.i_to_attr(v_prop_id)) + op.attr[types_pb2.E_LABEL_ID].CopyFrom(utils.i_to_attr(e_label_id)) + op.attr[types_pb2.E_PROP_ID].CopyFrom(utils.i_to_attr(e_prop_id)) + op.attr[types_pb2.OID_TYPE].CopyFrom( + utils.s_to_attr(utils.data_type_to_cpp(oid_type)) + ) + op.attr[types_pb2.VID_TYPE].CopyFrom( + utils.s_to_attr(utils.data_type_to_cpp(vid_type)) + ) + op.attr[types_pb2.V_DATA_TYPE].CopyFrom( + utils.s_to_attr(utils.data_type_to_cpp(vdata_type)) + ) + op.attr[types_pb2.E_DATA_TYPE].CopyFrom( + utils.s_to_attr(utils.data_type_to_cpp(edata_type)) + ) + + +def _pre_process_for_project_op(op, op_result_pool, key_to_op): + def _get_all_v_props_id(schema, label): + props = schema.get_vertex_properties(label) + return [schema.get_vertex_property_id(label, prop.name) for prop in props] + + def _get_all_e_props_id(schema, label): + props = schema.get_edge_properties(label) + return [schema.get_edge_property_id(label, prop.name) for prop in props] + + assert len(op.parents) == 1 + # get parent graph schema + key_of_parent_op = op.parents[0] + r = op_result_pool[key_of_parent_op] + schema = GraphSchema() + schema.from_graph_def(r.graph_def) + graph_name = r.graph_def.key + vertices = json.loads(op.attr[types_pb2.VERTEX_COLLECTIONS].s.decode("utf-8")) + edges = json.loads(op.attr[types_pb2.EDGE_COLLECTIONS].s.decode("utf-8")) + vertex_collections = {} + edge_collections = {} + for label, props in vertices.items(): + label_id = schema.get_vertex_label_id(label) + if props is None: + vertex_collections[label_id] = _get_all_v_props_id(schema, label) + else: + vertex_collections[label_id] = sorted( + [schema.get_vertex_property_id(label, prop) for prop in props] + ) + for label, props in edges.items(): + relations = schema.get_relationships(label) + valid = False + for src, dst in relations: + if src in vertices and dst in vertices: + valid = True + break + if not valid: + raise ValueError("Cannot find a valid relation in given vertices and edges") + label_id = schema.get_edge_label_id(label) + if props is None: + edge_collections[label_id] = _get_all_e_props_id(schema, label) + else: + edge_collections[label_id] = sorted( + [schema.get_edge_property_id(label, prop) for prop in props] + ) + vertex_collections = dict(sorted(vertex_collections.items())) + edge_collections = dict(sorted(edge_collections.items())) + + # construct op attr + attr = attr_value_pb2.AttrValue() + v_attr = attr_value_pb2.NameAttrList() + e_attr = attr_value_pb2.NameAttrList() + for label, props in vertex_collections.items(): + v_attr.attr[label].CopyFrom(utils.list_i_to_attr(props)) + for label, props in edge_collections.items(): + e_attr.attr[label].CopyFrom(utils.list_i_to_attr(props)) + attr.list.func.extend([v_attr, e_attr]) + op.attr[types_pb2.GRAPH_NAME].CopyFrom( + attr_value_pb2.AttrValue(s=graph_name.encode("utf-8")) + ) + op.attr[types_pb2.ARROW_PROPERTY_DEFINITION].CopyFrom(attr) + del op.attr[types_pb2.VERTEX_COLLECTIONS] + del op.attr[types_pb2.EDGE_COLLECTIONS] + + +def _tranform_numpy_selector(context_type, schema, selector): + if context_type == "tensor": + selector = None + if context_type == "vertex_data": + selector = transform_vertex_data_selector(selector) + if context_type == "labeled_vertex_data": + selector = transform_labeled_vertex_data_selector(schema, selector) + if context_type == "vertex_property": + selector = transform_vertex_property_data_selector(selector) + if context_type == "labeled_vertex_property": + selector = transform_labeled_vertex_property_data_selector(schema, selector) + return selector + + +def _tranform_dataframe_selector(context_type, schema, selector): + selector = json.loads(selector) + if context_type == "tensor": + selector = {key: None for key, value in selector.items()} + if context_type == "vertex_data": + selector = { + key: transform_vertex_data_selector(value) + for key, value in selector.items() + } + if context_type == "labeled_vertex_data": + selector = { + key: transform_labeled_vertex_data_selector(schema, value) + for key, value in selector.items() + } + if context_type == "vertex_property": + selector = { + key: transform_vertex_property_data_selector(value) + for key, value in selector.items() + } + if context_type == "labeled_vertex_property": + selector = { + key: transform_labeled_vertex_property_data_selector(schema, value) + for key, value in selector.items() + } + return json.dumps(selector) + + def _extract_gar(app_dir: str, attr): """Extract gar to workspace Args: @@ -676,3 +1024,10 @@ def to_maxgraph_schema(gsa_schema_json): for prop in item["propertyDefList"]: prop["id"] = 1 + prop_list.index(prop["name"]) return json.dumps(mg_schema) + + +def check_argument(condition, message=None): + if not condition: + if message is None: + message = "in '%s'" % inspect.stack()[1].code_context[0] + raise ValueError("Check failed: %s" % message) diff --git a/k8s/ubuntu/gsvineyard.Dockerfile b/k8s/ubuntu/gsvineyard.Dockerfile index d21d42e4f41a..bee04ee42c4b 100644 --- a/k8s/ubuntu/gsvineyard.Dockerfile +++ b/k8s/ubuntu/gsvineyard.Dockerfile @@ -14,7 +14,7 @@ RUN cd /tmp && \ make -j`nproc` && \ make install && \ cd /tmp && \ - git clone -b v0.1.15 https://github.com/alibaba/libvineyard.git --depth=1 && \ + git clone -b v0.2.3 https://github.com/alibaba/libvineyard.git --depth=1 && \ cd libvineyard && \ git submodule update --init && \ mkdir -p /tmp/libvineyard/build && \ diff --git a/proto/attr_value.proto b/proto/attr_value.proto index 90e26ebc8557..f53918628904 100644 --- a/proto/attr_value.proto +++ b/proto/attr_value.proto @@ -35,7 +35,6 @@ import "proto/types.proto"; message AttrValue { enum NullValue { NULL_VALUE = 0; } - message ListValue { repeated bytes s = 2; // "list(string)" repeated int64 i = 3 [ packed = true ]; // "list(int)" @@ -59,6 +58,9 @@ message AttrValue { ListValue list = 10; // any "list(...)" NameAttrList func = 11; + + // for lazy mode, inferred from the context of dag + PlaceHolder place_holder = 21; } } diff --git a/proto/error_codes.proto b/proto/error_codes.proto index 8cfccdcf138c..24bc5f4427fa 100644 --- a/proto/error_codes.proto +++ b/proto/error_codes.proto @@ -64,6 +64,13 @@ enum Code { // Errors raised by learning engine LEARNING_ENGINE_INTERNAL_ERROR = 32; + // Errors raised by coordinator + COORDINATOR_INTERNAL_ERROR = 33; + + // Errors raised by analytical engine + // Results of workers not consistent + WORKER_RESULTS_INCONSISTENT_ERROR = 41; + // Unknown error. UNKNOWN_ERROR = 101; diff --git a/proto/message.proto b/proto/message.proto index cd992732cb31..9e4f03cc66d0 100644 --- a/proto/message.proto +++ b/proto/message.proto @@ -18,7 +18,6 @@ syntax = "proto3"; package gs.rpc; import "proto/error_codes.proto"; -import "proto/graph_def.proto"; import "proto/op_def.proto"; import "proto/types.proto"; @@ -30,6 +29,8 @@ message ResponseStatus { oneof detail { NullDetail null = 3; OpDef op = 4; + // pickle coordinator python exception + bytes full_exception = 5; } } @@ -93,15 +94,9 @@ message RunStepRequest { message RunStepResponse { ResponseStatus status = 1; - // If success, store the metrics. (e.g. how many seconds used, memory bytes...) - string metrics = 2; - // result represents app_name or ctx_name or raw bytes If the op returns a NDArray or DataFrame - bytes result = 3; - - // If the op create a graph or modify a graph, return the meta data of the - // graph. - gs.rpc.graph.GraphDefPb graph_def = 31; + // list of result of ops in dag + repeated OpResult results = 2; } //////////////////////////////////////////////////////////////////////////////// diff --git a/proto/op_def.proto b/proto/op_def.proto index 59c72bb43d50..c8963fe88bb5 100644 --- a/proto/op_def.proto +++ b/proto/op_def.proto @@ -18,6 +18,8 @@ syntax = "proto3"; package gs.rpc; import "proto/attr_value.proto"; +import "proto/error_codes.proto"; +import "proto/graph_def.proto"; import "proto/types.proto"; import "proto/query_args.proto"; @@ -43,6 +45,28 @@ message OpDef { // arguments that served as application quering parameters. // Such as source vertex id for SSSP. QueryArgs query_args = 7; + + // mark fetch op or not + bool fetch = 8; +}; + +// Result of Op +message OpResult { + Code code = 1; + + // unique key for every op + string key = 2; + + // if success, store the metrics. (e.g. how many seconds used, memory bytes...) + string metrics = 3; + + string error_msg = 4; + + // result represents app_name or ctx_name or raw bytes If the op returns a NDArray or DataFrame + bytes result = 5; + + // if the op create a graph or modify a graph, return the meta data of the graph. + gs.rpc.graph.GraphDefPb graph_def = 31; }; // Consist by list of ops. diff --git a/proto/types.proto b/proto/types.proto index b38c8cca0ffa..6390c36cdc53 100644 --- a/proto/types.proto +++ b/proto/types.proto @@ -62,17 +62,20 @@ enum Direction { enum OutputType { GRAPH = 0; APP = 1; - RESULTS = 2; - TENSOR = 3; - DATAFRAME = 4; - VINEYARD_TENSOR = 5; - VINEYARD_DATAFRAME = 6; + BOUND_APP = 2; + RESULTS = 3; + TENSOR = 4; + DATAFRAME = 5; + VINEYARD_TENSOR = 6; + VINEYARD_DATAFRAME = 7; + NULL_OUTPUT = 101; }; enum OperationType { // command CREATE_GRAPH = 0; // return output_type = graph - CREATE_APP = 2; // return app + BIND_APP = 1; // return app + CREATE_APP = 2; // do nothing MODIFY_VERTICES = 3; // return graph MODIFY_EDGES = 4; // return graph RUN_APP = 5; // return result @@ -94,6 +97,11 @@ enum OperationType { VIEW_GRAPH = 21; // create graph view INDUCE_SUBGRAPH = 22; // induce subgraph + CREATE_INTERACTIVE_QUERY = 31; // interactive query + SUBGRAPH = 32; // subgraph in interactive query + + CREATE_LEARNING = 41; // learning graph + // data CONTEXT_TO_NUMPY = 50; CONTEXT_TO_DATAFRAME = 51; @@ -151,6 +159,10 @@ enum ParamKey { PARAM = 26; DISTRIBUTED = 27; + SCHEMA_PATH = 31; + GIE_GREMLIN_SERVER_CPU = 32; + GIE_GREMLIN_SERVER_MEM = 33; + GIE_GREMLIN_ENGINE_PARAMS = 34; APP_SIGNATURE = 40; GRAPH_SIGNATURE = 41; @@ -158,6 +170,10 @@ enum ParamKey { VINEYARD_ID = 43; VINEYARD_NAME = 44; + // project + VERTEX_COLLECTIONS = 51; + EDGE_COLLECTIONS = 52; + APP_NAME = 100; APP_ALGO = 101; APP_LIBRARY_PATH = 102; @@ -231,3 +247,5 @@ enum ReportType { NODES_BY_LOC = 18; SELFLOOPS_NUM = 19; } + +message PlaceHolder {} diff --git a/python/graphscope/client/session.py b/python/graphscope/client/session.py index 490c6718eb99..9830869e08ee 100755 --- a/python/graphscope/client/session.py +++ b/python/graphscope/client/session.py @@ -48,6 +48,7 @@ from graphscope.config import GSConfig as gs_config from graphscope.deploy.hosts.cluster import HostsClusterLauncher from graphscope.deploy.kubernetes.cluster import KubernetesClusterLauncher +from graphscope.framework.dag import Dag from graphscope.framework.errors import ConnectionError from graphscope.framework.errors import FatalError from graphscope.framework.errors import GRPCError @@ -57,7 +58,10 @@ from graphscope.framework.errors import LearningEngineInternalError from graphscope.framework.errors import check_argument from graphscope.framework.graph import Graph +from graphscope.framework.graph import GraphDAGNode from graphscope.framework.operation import Operation +from graphscope.framework.utils import decode_dataframe +from graphscope.framework.utils import decode_numpy from graphscope.interactive.query import InteractiveQuery from graphscope.interactive.query import InteractiveQueryStatus from graphscope.proto import graph_def_pb2 @@ -74,6 +78,115 @@ logger = logging.getLogger("graphscope") +class _FetchHandler(object): + """Handler for structured fetches. + This class takes care of extracting a sub-DAG as targets for a user-provided structure for fetches, + which can be used for a low level `run` call of grpc_client. + + Given the results of the low level run call, this class can also rebuild a result structure + matching the user-provided structure for fetches, but containing the corresponding results. + """ + + def __init__(self, dag, fetches): + self._fetches = fetches + self._ops = list() + self._unpack = False + if not isinstance(self._fetches, (list, tuple)): + self._fetches = [self._fetches] + self._unpack = True + for fetch in self._fetches: + if hasattr(fetch, "op"): + fetch = fetch.op + if not isinstance(fetch, Operation): + raise ValueError("Expect a `Operation` in sess run method.") + self._ops.append(fetch) + # extract sub dag + self._sub_dag = dag.extract_subdag_for(self._ops) + if "debug" in os.environ: + logger.info("sub_dag: %s", self._sub_dag) + + @property + def targets(self): + return self._sub_dag + + def _rebuild_graph(self, seq, op: Operation, op_result: op_def_pb2.OpResult): + if isinstance(self._fetches[seq], Operation): + # for nx Graph + return op_result.graph_def + # get graph dag node as base + graph_dag_node = self._fetches[seq] + # construct graph + g = Graph(graph_dag_node) + # update graph flied from graph_def + g.update_from_graph_def(op_result.graph_def) + return g + + def _rebuild_app(self, seq, op: Operation, op_result: op_def_pb2.OpResult): + from graphscope.framework.app import App + + # get app dag node as base + app_dag_node = self._fetches[seq] + # construct app + app = App(app_dag_node, op_result.result.decode("utf-8")) + return app + + def _rebuild_context(self, seq, op: Operation, op_result: op_def_pb2.OpResult): + from graphscope.framework.context import Context + from graphscope.framework.context import DynamicVertexDataContext + + # get context dag node as base + context_dag_node = self._fetches[seq] + ret = json.loads(op_result.result.decode("utf-8")) + context_type = ret["context_type"] + if context_type == "dynamic_vertex_data": + # for nx + return DynamicVertexDataContext( + context_dag_node, ret["context_key"], context_type + ) + else: + return Context(context_dag_node, ret["context_key"], context_type) + + def wrapper_results(self, response: message_pb2.RunStepResponse): + rets = list() + for seq, op in enumerate(self._ops): + for op_result in response.results: + if op.key == op_result.key: + if op.output_types == types_pb2.RESULTS: + if op.type == types_pb2.RUN_APP: + rets.append(self._rebuild_context(seq, op, op_result)) + else: + # for nx Graph + rets.append(op_result.result.decode("utf-8")) + if op.output_types == types_pb2.GRAPH: + rets.append(self._rebuild_graph(seq, op, op_result)) + if op.output_types == types_pb2.APP: + rets.append(None) + if op.output_types == types_pb2.BOUND_APP: + rets.append(self._rebuild_app(seq, op, op_result)) + if op.output_types in ( + types_pb2.VINEYARD_TENSOR, + types_pb2.VINEYARD_DATAFRAME, + ): + rets.append( + json.loads(op_result.result.decode("utf-8"))["object_id"] + ) + if op.output_types in (types_pb2.TENSOR, types_pb2.DATAFRAME): + if ( + op.type == types_pb2.CONTEXT_TO_DATAFRAME + or op.type == types_pb2.GRAPH_TO_DATAFRAME + ): + rets.append(decode_dataframe(op_result.result)) + if ( + op.type == types_pb2.CONTEXT_TO_NUMPY + or op.type == types_pb2.GRAPH_TO_NUMPY + ): + rets.append(decode_numpy(op_result.result)) + if op.output_types == types_pb2.NULL_OUTPUT: + rets.append(None) + break + return rets[0] if self._unpack else rets + + class Session(object): """A class for interacting with GraphScope graph computation service cluster. @@ -98,15 +211,15 @@ class Session(object): >>> sess = gs.session() >>> g = sess.g() >>> pg = g.project(vertices={'v': []}, edges={'e': ['dist']}) - >>> r = s.sssp(g, 4) - >>> s.close() + >>> r = gs.sssp(g, 4) + >>> sess.close() >>> # or use a session as default - >>> s = gs.session().as_default() - >>> g = g() + >>> sess = gs.session().as_default() + >>> g = gs.g() >>> pg = g.project(vertices={'v': []}, edges={'e': ['dist']}) >>> r = gs.sssp(pg, 4) - >>> s.close() + >>> sess.close() We support setup a service cluster and create a RPC session in following ways: @@ -138,8 +251,9 @@ class Session(object): def __init__( self, config=None, - cluster_type=gs_config.cluster_type, addr=gs_config.addr, + mode=gs_config.mode, + cluster_type=gs_config.cluster_type, num_workers=gs_config.num_workers, preemptive=gs_config.preemptive, k8s_namespace=gs_config.k8s_namespace, @@ -188,6 +302,13 @@ def __init__( addr (str, optional): The endpoint of a pre-launched GraphScope instance with ':' format. A new session id will be generated for each session connection. + mode (str, optional): optional values are eager and lazy. Defaults to eager. + Eager execution is a flexible platform for research and experimentation, it provides: + An intuitive interface: Quickly test on small data. + Easier debugging: Call ops directly to inspect running models and test changes. + Lazy execution means GraphScope does not precess the data till it has to. It just gathers all the + information to a DAG that we feed into it, and processes only when we execute :code:`sess.run(fetches)` + cluster_type (str, optional): Deploy GraphScope instance on hosts or k8s cluster. Defaults to k8s. Available options: "k8s" and "hosts". Note that only support deployed on localhost with hosts mode. @@ -354,10 +475,10 @@ def __init__( TypeError: If the given argument combination is invalid and cannot be used to create a GraphScope session. """ - num_workers = int(num_workers) self._config_params = {} self._accessable_params = ( "addr", + "mode", "cluster_type", "num_workers", "preemptive", @@ -403,7 +524,7 @@ def __init__( if isinstance(config, dict): self._config_params.update(config) elif isinstance(config, str): - self._load_config(config, False) + self._load_config(config, slient=False) elif DEFAULT_CONFIG_FILE: self._load_config(DEFAULT_CONFIG_FILE) @@ -413,6 +534,9 @@ def __init__( # initial setting of cluster_type self._cluster_type = self._parse_cluster_type() + # initial dag + self._dag = Dag() + # mars cannot work with run-on-local mode if self._cluster_type == types_pb2.HOSTS and self._config_params["with_mars"]: raise NotImplementedError( @@ -436,7 +560,7 @@ def __init__( ) if "k8s_vineyard_shared_mem" in kw: warnings.warn( - "The `k8s_vineyard_shared_mem` has benn deprecated and has no effect, " + "The `k8s_vineyard_shared_mem` has been deprecated and has no effect, " "please use `vineyard_shared_mem` instead." % kw.pop("k8s_vineyard_shared_mem", None), category=DeprecationWarning, @@ -511,6 +635,10 @@ def __str__(self): def session_id(self): return self._session_id + @property + def dag(self): + return self._dag + def _load_config(self, path, slient=True): config_path = os.path.expandvars(os.path.expanduser(path)) try: @@ -565,6 +693,9 @@ def info(self): info["engine_config"] = self._engine_config return info + def eager(self): + return self._config_params["mode"] == "eager" + def _send_heartbeat(self): while not self._closed: if self._grpc_client: @@ -671,9 +802,14 @@ def _deregister_default(self): self._default_session.__exit__(None, None, None) self._default_session = None - def run(self, fetch): - """Run operations of `fetch`. + def _wrapper(self, dag_node): + if self.eager(): + return self.run(dag_node) + else: + return dag_node + def run(self, fetches, debug=False): + """Run operations of `fetch`. Args: fetch: :class:`Operation` @@ -691,60 +827,17 @@ def run(self, fetch): Returns: Different values for different output types of :class:`Operation` """ - - # prepare names to run and fetch - if hasattr(fetch, "op"): - fetch = fetch.op - if not isinstance(fetch, Operation): - raise ValueError("Expect a `Operation`") - if fetch.output is not None: - raise ValueError("The op <%s> are evaluated duplicated." % fetch.key) - - # convert to list to be compatible with rpc client method signature - fetch_ops = [fetch] - - dag = op_def_pb2.DagDef() - for op in fetch_ops: - dag.op.extend([copy.deepcopy(op.as_op_def())]) - if self._closed: raise RuntimeError("Attempted to use a closed Session.") - if not self._grpc_client: raise RuntimeError("Session disconnected.") - - # execute the query + fetch_handler = _FetchHandler(self.dag, fetches) try: - response = self._grpc_client.run(dag) + response = self._grpc_client.run(fetch_handler.targets) except FatalError: self.close() raise - check_argument( - len(fetch_ops) == 1, "Cannot execute multiple ops at the same time" - ) - return self._parse_value(fetch_ops[0], response) - - def _parse_value(self, op, response: message_pb2.RunStepResponse): - # attach an output to op, indicating the op is already run. - op.set_output(response.metrics) - - # if loads a arrow property graph, will return {'object_id': xxxx} - if op.output_types == types_pb2.GRAPH: - return response.graph_def - if op.output_types == types_pb2.APP: - return response.result.decode("utf-8") - if op.output_types in ( - types_pb2.RESULTS, - types_pb2.VINEYARD_TENSOR, - types_pb2.VINEYARD_DATAFRAME, - ): - return response.result.decode("utf-8") - if op.output_types in (types_pb2.TENSOR, types_pb2.DATAFRAME): - return response.result - else: - raise InvalidArgumentError( - "Not recognized output type: %s" % op.output_types - ) + return fetch_handler.wrapper_results(response) def _connect(self): if self._config_params["addr"] is not None: @@ -821,7 +914,9 @@ def get_config(self): return self._config_params def g(self, incoming_data=None, oid_type="int64", directed=True, generate_eid=True): - return Graph(self, incoming_data, oid_type, directed, generate_eid) + return self._wrapper( + GraphDAGNode(self, incoming_data, oid_type, directed, generate_eid) + ) def load_from(self, *args, **kwargs): """Load a graph within the session. diff --git a/python/graphscope/config.py b/python/graphscope/config.py index 7cfec1f0fb4e..2e3aedcab03e 100644 --- a/python/graphscope/config.py +++ b/python/graphscope/config.py @@ -26,6 +26,9 @@ class GSConfig(object): # the endpoint of a pre-launched GraphScope instance. addr = None + # "lazy" or "eager", defaults to "eager" + mode = "eager" + # "k8s" or "hosts" cluster_type = "k8s" diff --git a/python/graphscope/dataset/ldbc.py b/python/graphscope/dataset/ldbc.py index 686a92cf47c0..3d9fc8149db7 100644 --- a/python/graphscope/dataset/ldbc.py +++ b/python/graphscope/dataset/ldbc.py @@ -21,19 +21,8 @@ from graphscope.framework.graph import Graph from graphscope.framework.loader import Loader - +""" def load_ldbc(sess, prefix, directed=True): - """Load ldbc dataset as a ArrowProperty Graph. - - Args: - sess (:class:`graphscope.Session`): Load graph within the session. - prefix (str): Data directory. - directed (bool, optional): Determine to load a directed or undirected graph. - Defaults to True. - - Returns: - :class:`graphscope.Graph`: A Graph object which graph type is ArrowProperty - """ graph = sess.g(directed=directed) graph = ( graph.add_vertices( @@ -278,3 +267,353 @@ def load_ldbc(sess, prefix, directed=True): ) ) return graph +""" + + +def load_ldbc(sess, prefix, directed=True): + """Load ldbc dataset as a ArrowProperty Graph. + Args: + sess (:class:`graphscope.Session`): Load graph within the session. + prefix (str): Data directory. + directed (bool, optional): Determine to load a directed or undirected graph. + Defaults to True. + Returns: + :class:`graphscope.Graph`: A Graph object which graph type is ArrowProperty + """ + vertices = { + "comment": ( + Loader( + os.path.join(prefix, "comment_0_0.csv"), header_row=True, delimiter="|" + ), + ["creationDate", "locationIP", "browserUsed", "content", "length"], + "id", + ), + "organisation": ( + Loader( + os.path.join(prefix, "organisation_0_0.csv"), + header_row=True, + delimiter="|", + ), + ["type", "name", "url"], + "id", + ), + "tagclass": ( + Loader( + os.path.join(prefix, "tagclass_0_0.csv"), header_row=True, delimiter="|" + ), + ["name", "url"], + "id", + ), + "person": ( + Loader( + os.path.join(prefix, "person_0_0.csv"), header_row=True, delimiter="|" + ), + [ + "firstName", + "lastName", + "gender", + "birthday", + "creationDate", + "locationIP", + "browserUsed", + ], + "id", + ), + "forum": ( + Loader( + os.path.join(prefix, "forum_0_0.csv"), header_row=True, delimiter="|" + ), + ["title", "creationDate"], + "id", + ), + "place": ( + Loader( + os.path.join(prefix, "place_0_0.csv"), header_row=True, delimiter="|" + ), + ["name", "url", "type"], + "id", + ), + "post": ( + Loader( + os.path.join(prefix, "post_0_0.csv"), header_row=True, delimiter="|" + ), + [ + "imageFile", + "creationDate", + "locationIP", + "browserUsed", + "language", + "content", + "length", + ], + "id", + ), + "tag": ( + Loader(os.path.join(prefix, "tag_0_0.csv"), header_row=True, delimiter="|"), + ["name", "url"], + "id", + ), + } + edges = { + "replyOf": [ + ( + Loader( + os.path.join(prefix, "comment_replyOf_comment_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Comment.id", "comment"), + ("Comment.id.1", "comment"), + ), + ( + Loader( + os.path.join(prefix, "comment_replyOf_post_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Comment.id", "comment"), + ("Post.id", "post"), + ), + ], + "isPartOf": [ + ( + Loader( + os.path.join(prefix, "place_isPartOf_place_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Place.id", "place"), + ("Place.id.1", "place"), + ) + ], + "isSubclassOf": [ + ( + Loader( + os.path.join(prefix, "tagclass_isSubclassOf_tagclass_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("TagClass.id", "tagclass"), + ("TagClass.id.1", "tagclass"), + ) + ], + "hasTag": [ + ( + Loader( + os.path.join(prefix, "forum_hasTag_tag_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Forum.id", "forum"), + ("Tag.id", "tag"), + ), + ( + Loader( + os.path.join(prefix, "comment_hasTag_tag_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Comment.id", "comment"), + ("Tag.id", "tag"), + ), + ( + Loader( + os.path.join(prefix, "post_hasTag_tag_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Post.id", "post"), + ("Tag.id", "tag"), + ), + ], + "knows": [ + ( + Loader( + os.path.join(prefix, "person_knows_person_0_0.csv"), + header_row=True, + delimiter="|", + ), + ["creationDate"], + ("Person.id", "person"), + ("Person.id.1", "person"), + ) + ], + "hasModerator": [ + ( + Loader( + os.path.join(prefix, "forum_hasModerator_person_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Forum.id", "forum"), + ("Person.id", "person"), + ) + ], + "hasInterest": [ + ( + Loader( + os.path.join(prefix, "person_hasInterest_tag_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Person.id", "person"), + ("Tag.id", "tag"), + ) + ], + "isLocatedIn": [ + ( + Loader( + os.path.join(prefix, "post_isLocatedIn_place_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Post.id", "post"), + ("Place.id", "place"), + ), + ( + Loader( + os.path.join(prefix, "comment_isLocatedIn_place_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Comment.id", "comment"), + ("Place.id", "place"), + ), + ( + Loader( + os.path.join(prefix, "organisation_isLocatedIn_place_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Organisation.id", "organisation"), + ("Place.id", "place"), + ), + ( + Loader( + os.path.join(prefix, "person_isLocatedIn_place_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Person.id", "person"), + ("Place.id", "place"), + ), + ], + "hasType": [ + ( + Loader( + os.path.join(prefix, "tag_hasType_tagclass_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Tag.id", "tag"), + ("TagClass.id", "tagclass"), + ) + ], + "hasCreator": [ + ( + Loader( + os.path.join(prefix, "post_hasCreator_person_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Post.id", "post"), + ("Person.id", "person"), + ), + ( + Loader( + os.path.join(prefix, "comment_hasCreator_person_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Comment.id", "comment"), + ("Person.id", "person"), + ), + ], + "containerOf": [ + ( + Loader( + os.path.join(prefix, "forum_containerOf_post_0_0.csv"), + header_row=True, + delimiter="|", + ), + [], + ("Forum.id", "forum"), + ("Post.id", "post"), + ) + ], + "hasMember": [ + ( + Loader( + os.path.join(prefix, "forum_hasMember_person_0_0.csv"), + header_row=True, + delimiter="|", + ), + ["joinDate"], + ("Forum.id", "forum"), + ("Person.id", "person"), + ) + ], + "workAt": [ + ( + Loader( + os.path.join(prefix, "person_workAt_organisation_0_0.csv"), + header_row=True, + delimiter="|", + ), + ["workFrom"], + ("Person.id", "person"), + ("Organisation.id", "organisation"), + ) + ], + "likes": [ + ( + Loader( + os.path.join(prefix, "person_likes_comment_0_0.csv"), + header_row=True, + delimiter="|", + ), + ["creationDate"], + ("Person.id", "person"), + ("Comment.id", "comment"), + ), + ( + Loader( + os.path.join(prefix, "person_likes_post_0_0.csv"), + header_row=True, + delimiter="|", + ), + ["creationDate"], + ("Person.id", "person"), + ("Post.id", "post"), + ), + ], + "studyAt": [ + ( + Loader( + os.path.join(prefix, "person_studyAt_organisation_0_0.csv"), + header_row=True, + delimiter="|", + ), + ["classYear"], + ("Person.id", "person"), + ("Organisation.id", "organisation"), + ) + ], + } + return sess.load_from(edges, vertices, directed, generate_eid=True) diff --git a/python/graphscope/deploy/kubernetes/cluster.py b/python/graphscope/deploy/kubernetes/cluster.py index 012ea170dda2..c49c8041e61e 100644 --- a/python/graphscope/deploy/kubernetes/cluster.py +++ b/python/graphscope/deploy/kubernetes/cluster.py @@ -561,8 +561,11 @@ def start(self): self._create_services() time.sleep(1) self._waiting_for_services_ready() - logger.info("Coordinator pod start successful, connecting to service...") self._coordinator_endpoint = self._get_coordinator_endpoint() + logger.info( + "Coordinator pod start successful with address %s, connecting to service ...", + self._coordinator_endpoint, + ) except Exception as e: time.sleep(1) self._dump_coordinator_failed_status() diff --git a/python/graphscope/framework/app.py b/python/graphscope/framework/app.py index 9077d46b2841..3e06632fc10d 100644 --- a/python/graphscope/framework/app.py +++ b/python/graphscope/framework/app.py @@ -20,12 +20,15 @@ import json import os import zipfile +from copy import deepcopy from io import BytesIO import yaml import graphscope -from graphscope.framework.context import create_context +from graphscope.framework.context import ContextDAGNode +from graphscope.framework.dag import DAGNode +from graphscope.framework.dag_utils import bind_app from graphscope.framework.dag_utils import create_app from graphscope.framework.dag_utils import run_app from graphscope.framework.dag_utils import unload_app @@ -109,7 +112,7 @@ def wrapper(*args, **kwargs): return _not_compatible_for -class AppAssets(object): +class AppAssets(DAGNode): """A class holds the bytes of the gar resource. Assets includes name (for builtin algorithm), and gar (for user defined algorithm), @@ -142,6 +145,7 @@ def __init__(self, algo, gar=None, **kwargs): else: # built_in apps has no gar resource. self._gar = None + self._op = create_app(self) def __repr__(self) -> str: return f"graphscope.AppAssets " @@ -202,7 +206,7 @@ def is_compatible(self, graph): """Determine if this algorithm can run on this type of graph. Args: - graph (:class:`Graph`): A graph instance. + graph (:class:`GraphDAGNode`): A graph instance. Raises: InvalidArgumentError: @@ -213,15 +217,6 @@ def is_compatible(self, graph): ScannerError: - Yaml file format is incorrect. """ - if not isinstance( - graph, - ( - graphscope.framework.graph.Graph, - graphscope.nx.classes.graph.Graph, - graphscope.nx.classes.digraph.DiGraph, - ), - ): - raise InvalidArgumentError("Wrong type of graph.") # builtin app if self._gar is None: self._type = "cpp_pie" @@ -245,12 +240,14 @@ def is_compatible(self, graph): def __call__(self, graph, *args, **kwargs): """Instantiate an App and do queries over it.""" - app_ = App(graph, self) + app_ = graph.session._wrapper(AppDAGNode(graph, self)) return app_(*args, **kwargs) -class App(object): - """An application that can run on graphs and produce results. +class AppDAGNode(DAGNode): + """App node in a DAG. + + An application that can run on graphs and produce results. Analytical engine will build the app dynamic library when instantiate a app instance. The dynamic library will be reused if subsequent app's signature matches one of previous ones. @@ -270,53 +267,25 @@ def __init__(self, graph, app_assets: AppAssets): class name. Args: - graph (:class:`Graph`): A :class:`Graph` instance. + graph (:class:`GraphDAGNode`): A :class:`GraphDAGNode` instance. app_assets: A :class:`AppAssets` instance. - - Raise: - TypeError: The type of app_assets incorrect. """ - if not graph.loaded(): - raise RuntimeError("The graph is not loaded.") - app_assets.is_compatible(graph) - - self._key = None self._graph = graph + self._app_assets = app_assets - self._session_id = graph.session_id + self._session = graph.session + self._app_assets.is_compatible(self._graph) - opr = create_app(graph, self) - self._key = opr.eval() - self._saved_signature = self.signature + self._op = bind_app(graph, self._app_assets) + # add op to dag + self._session.dag.add_op(self._app_assets.op) + self._session.dag.add_op(self._op) def __repr__(self): s = f"graphscope.App " return s - @property - def key(self): - """A unique identifier of App.""" - return self._key - - @property - def signature(self): - """Signature is computed by all critical components of the App.""" - return hashlib.sha256( - "{}.{}".format(self._app_assets.signature, self._graph.template_str).encode( - "utf-8" - ) - ).hexdigest() - - @property - def session_id(self): - """Return the session_id, which is copied from the graph. - - Returns: - str: Id of the session which loaded the app. - """ - return self._session_id - @property def algo(self): """Algorithm name, e.g. sssp, pagerank. @@ -348,51 +317,63 @@ def __call__(self, *args, **kwargs): app_type = self._app_assets.type check_argument(app_type is not None) + if not isinstance(self._graph, DAGNode) and not self._graph.loaded(): + raise RuntimeError("The graph is not loaded") + if self._app_assets.type in ["cython_pie", "cython_pregel"]: # cython app support kwargs only check_argument( not args, "Only support using keyword arguments in cython app." ) - return self._query(json.dumps(kwargs)) + return ContextDAGNode(self, self._graph, json.dumps(kwargs)) - return self._query(*args, **kwargs) + return ContextDAGNode(self, self._graph, *args, **kwargs) - def _query(self, *args, **kwargs): - """Create a `RUN_APP` Operation, and send it to analytical engine to do actual query. - Then retrieve query contexts and return. - """ - self._check_unmodified() - op = run_app(self._graph, self, *args, **kwargs) - ret = op.eval() - ret = json.loads(ret) - context_key, context_type = ret["context_key"], ret["context_type"] - results = create_context( - context_type, self._session_id, context_key, self._graph - ) - return results - - def _check_unmodified(self): - """Ensure app is not modified, cause it may need to recompile the dynamic library.""" - check_argument(self.signature == self._saved_signature) - - def loaded(self): - """Since key is only set by engine after it load the app, and unset to None when unload, - we can use the key to detect whether the app is loaded. + def unload(self): + # do nothing for dag node + pass - Returns: - bool: The app is loaded or not. - """ - return self._key is not None + +class App(object): + def __init__(self, app_node, key): + self._app_node = app_node + self._session = self._app_node.session + self._key = key + # copy and set op evaluated + self._app_node.op = deepcopy(self._app_node.op) + self._app_node.evaluated = True + self._session.dag.add_op(self._app_node.op) + self._saved_signature = self.signature + + def __getattr__(self, name): + if hasattr(self._app_node, name): + return getattr(self._app_node, name) + else: + raise AttributeError("{0} not found.".format(name)) + + @property + def key(self): + """A unique identifier of App.""" + return self._key + + @property + def signature(self): + """Signature is computed by all critical components of the App.""" + return hashlib.sha256( + "{}.{}".format(self._app_assets.signature, self._graph.template_str).encode( + "utf-8" + ) + ).hexdigest() def unload(self): """Unload app. Both on engine side and python side. Set the key to None.""" - if self._key: - op = unload_app(self) - op.eval() - self._key = None - self._graph = None - self._app_assets = None - self._session_id = None + op = unload_app(self) + op.eval() + self._key = None + self._session = None + + def __call__(self, *args, **kwargs): + return self._session._wrapper(self._app_node(*args, **kwargs)) def load_app(algo, gar=None, **kwargs): @@ -431,10 +412,8 @@ def load_app(algo, gar=None, **kwargs): elif isinstance(gar, str): with open(gar, "rb") as f: content = f.read() - if not zipfile.is_zipfile(gar): raise InvalidArgumentError("{} is not a zip file.".format(gar)) - return AppAssets(str(algo), content, **kwargs) else: raise InvalidArgumentError("Wrong type with {}".format(gar)) diff --git a/python/graphscope/framework/context.py b/python/graphscope/framework/context.py index 6a156454566b..accfcb50619b 100644 --- a/python/graphscope/framework/context.py +++ b/python/graphscope/framework/context.py @@ -19,18 +19,28 @@ import collections import hashlib import json +from copy import deepcopy from typing import Mapping from graphscope.client.session import get_session_by_id from graphscope.framework import dag_utils from graphscope.framework import utils +from graphscope.framework.dag import DAGNode +from graphscope.framework.dag_utils import run_app from graphscope.framework.errors import InvalidArgumentError from graphscope.framework.errors import check_argument -from graphscope.framework.utils import decode_dataframe -from graphscope.framework.utils import decode_numpy -class BaseContext(object): +class ResultDAGNode(DAGNode): + def __init__(self, context, op): + self._context = context + self._session = self._context.session + self._op = op + # add op to dag + self._session.dag.add_op(self._op) + + +class ContextDAGNode(DAGNode): """Base class of concrete contexts. Hold a handle of app querying context. @@ -61,41 +71,17 @@ class BaseContext(object): >>> out = ret.to_vineyard_dataframe() # return an object id """ - def __init__(self, session_id, context_key, graph): - self._key = context_key + def __init__(self, bound_app, graph, *args, **kwargs): + self._bound_app = bound_app self._graph = graph - self._session_id = session_id - self._saved_signature = self.signature + self._session = self._bound_app.session + # add op to dag + self._op = run_app(self._bound_app, *args, **kwargs) + self._session.dag.add_op(self._op) def __repr__(self): return f"graphscope.{self.__class__.__name__} from graph {str(self._graph)}" - @property - def key(self): - """Unique identifier of a context.""" - return self._key - - @property - def signature(self): - """Compute digest by key and graph signatures. - Used to ensure the critical information of context is untouched. - """ - check_argument( - self._key is not None, - "Context key error, maybe it is not connected to engine.", - ) - return hashlib.sha256( - "{}.{}".format(self._key, self._graph.signature).encode("utf-8") - ).hexdigest() - - def _check_unmodified(self): - check_argument(self._saved_signature == self.signature) - - @property - def session_id(self): - """Return the session id associated with the context.""" - return self._session_id - def _transform_selector(self, selector): raise NotImplementedError() @@ -117,13 +103,12 @@ def to_numpy(self, selector, vertex_range=None, axis=0): Returns: numpy.ndarray. """ - self._check_unmodified() - selector = self._transform_selector(selector) + if selector is None: + raise RuntimeError("selector cannot be None") vertex_range = utils.transform_vertex_range(vertex_range) op = dag_utils.context_to_numpy(self, selector, vertex_range, axis) - raw_values = op.eval() - return decode_numpy(raw_values) + return ResultDAGNode(self, op) def to_dataframe(self, selector, vertex_range=None): """Return results as a pandas DataFrame @@ -142,19 +127,13 @@ def to_dataframe(self, selector, vertex_range=None): Returns: pandas.DataFrame """ - self._check_unmodified() - check_argument( isinstance(selector, Mapping), "selector of to_dataframe must be a dict" ) - selector = { - key: self._transform_selector(value) for key, value in selector.items() - } selector = json.dumps(selector) vertex_range = utils.transform_vertex_range(vertex_range) op = dag_utils.context_to_dataframe(self, selector, vertex_range) - raw_values = op.eval() - return decode_dataframe(raw_values) + return ResultDAGNode(self, op) def to_vineyard_tensor(self, selector=None, vertex_range=None, axis=0): """Return results as a vineyard tensor. @@ -163,14 +142,10 @@ def to_vineyard_tensor(self, selector=None, vertex_range=None, axis=0): Returns: str: object id of vineyard tensor """ - self._check_unmodified() - selector = self._transform_selector(selector) vertex_range = utils.transform_vertex_range(vertex_range) op = dag_utils.to_vineyard_tensor(self, selector, vertex_range, axis) - ret = op.eval() - object_id = json.loads(ret)["object_id"] - return object_id + return ResultDAGNode(self, op) def to_vineyard_dataframe(self, selector=None, vertex_range=None): """Return results as a vineyard dataframe. @@ -190,21 +165,76 @@ def to_vineyard_dataframe(self, selector=None, vertex_range=None): Returns: str: object id of vineyard tensor """ - self._check_unmodified() if selector is not None: - check_argument( - isinstance(selector, Mapping), - "selector of to_vineyard_dataframe must be a dict", - ) - selector = { - key: self._transform_selector(value) for key, value in selector.items() - } selector = json.dumps(selector) vertex_range = utils.transform_vertex_range(vertex_range) op = dag_utils.to_vineyard_dataframe(self, selector, vertex_range) - ret = op.eval() - object_id = json.loads(ret)["object_id"] - return object_id + return ResultDAGNode(self, op) + + +class Context(object): + def __init__(self, context_node, key, type): + self._context_node = context_node + self._session = context_node.session + self._graph = self._context_node._graph + self._key = key + self._type = type + # copy and set op evaluated + self._context_node.op = deepcopy(self._context_node.op) + self._context_node.evaluated = True + self._saved_signature = self.signature + + @property + def op(self): + return self._context_node.op + + @property + def key(self): + """Unique identifier of a context.""" + return self._key + + @property + def signature(self): + """Compute digest by key and graph signatures. + Used to ensure the critical information of context is untouched. + """ + check_argument( + self._key is not None, + "Context key error, maybe it is not connected to engine.", + ) + return hashlib.sha256( + "{}.{}".format(self._key, self._graph.signature).encode("utf-8") + ).hexdigest() + + def __repr__(self): + return f"graphscope.{self.__class__.__name__} from graph {str(self._graph)}" + + def _check_unmodified(self): + check_argument(self._saved_signature == self.signature) + + def to_numpy(self, selector, vertex_range=None, axis=0): + self._check_unmodified() + return self._session._wrapper( + self._context_node.to_numpy(selector, vertex_range, axis) + ) + + def to_dataframe(self, selector, vertex_range=None): + self._check_unmodified() + return self._session._wrapper( + self._context_node.to_dataframe(selector, vertex_range) + ) + + def to_vineyard_tensor(self, selector=None, vertex_range=None, axis=0): + self._check_unmodified() + return self._session._wrapper( + self._context_node.to_vineyard_tensor(selector, vertex_range, axis) + ) + + def to_vineyard_dataframe(self, selector=None, vertex_range=None): + self._check_unmodified() + return self._session._wrapper( + self._context_node.to_vineyard_dataframe(selector, vertex_range) + ) def output(self, fd, selector, vertex_range=None, **kwargs): """Dump results to `fd`. @@ -273,52 +303,37 @@ def output_to_client(self, fd, selector, vertex_range=None): df.to_csv(fd, header=True, index=False) -class TensorContext(BaseContext): - """Tensor context holds a tensor. - Only axis is meaningful when considering a TensorContext. - """ - - def _transform_selector(self, selector): - return None - - -class VertexDataContext(BaseContext): - """The most simple kind of context. - A vertex has a single value as results. - - - The syntax of selector on vertex is: - - `v.id`: Get the Id of vertices - - `v.data`: Get the data of vertices (If there is any, means origin data on the graph, not results) - - - The syntax of selector of edge is: - - `e.src`: Get the source Id of edges - - `e.dst`: Get the destination Id of edges - - `e.data`: Get the edge data on the edges (If there is any, means origin data on the graph) - - - The syntax of selector of results is: - - `r`: Get quering results of algorithms. e.g. Rankings of vertices after doing PageRank. - """ - - def _transform_selector(self, selector): - return utils.transform_vertex_data_selector(selector) - - class DynamicVertexDataContext(collections.abc.Mapping): """Vertex data context for complicated result store. A vertex has a single value as results. """ - def __init__(self, session_id, context_key, graph): - self._key = context_key - self._graph = graph - self._session_id = session_id + def __init__(self, context_node, key, type): + self._key = key + self._graph = context_node._graph + self._session_id = context_node.session_id self._saved_signature = self.signature - # partial inherit the BaseContext methods - session_id = BaseContext.__dict__["session_id"] - key = BaseContext.__dict__["key"] - signature = BaseContext.__dict__["signature"] - __repr__ = BaseContext.__dict__["__repr__"] + @property + def session_id(self): + return self._session_id + + @property + def key(self): + return self._key + + @property + def signature(self): + check_argument( + self._key is not None, + "Context key error, maybe it is not connected to engine.", + ) + return hashlib.sha256( + "{}.{}".format(self._key, self._graph.signature).encode("utf-8") + ) + + def __repr__(self): + return f"graphscope.{self.__class__.__name__} from graph {str(self._graph)}" def __len__(self): return self._graph._graph.number_of_nodes() @@ -331,96 +346,3 @@ def __getitem__(self, key): def __iter__(self): return iter(self._graph._graph) - - -class LabeledVertexDataContext(BaseContext): - """The labeld kind of context. - This context has several vertex labels and edge labels, - and each label has several properties. - Selection are performed on labels first, then on properties. - - We use `:` to filter labels, and `.` to select properties. - And the results has no property, only have labels. - - - The syntax of selector of vertex is: - - `v:label_name.id`: Get Id that belongs to a specific vertex label. - - `v:label_name.property_name`: Get data that on a specific property of a specific vertex label. - - - The syntax of selector of edge is: - - `e:label_name.src`: Get source Id of a specific edge label. - - `e:label_name.dst`: Get destination Id of a specific edge label. - - `e:label_name.property_name`: Get data on a specific property of a specific edge label. - - - The syntax of selector of results is: - - `r:label_name`: Get results data of a vertex label. - """ - - def _transform_selector(self, selector): - return utils.transform_labeled_vertex_data_selector(self._graph, selector) - - -class VertexPropertyContext(BaseContext): - """The simple kind of context with property. - A vertex can have multiple values (a.k.a. properties) as results. - - - The syntax of selector on vertex is: - - `v.id`: Get the Id of vertices - - `v.data`: Get the data of vertices (If there is any, means origin data on the graph, not results) - - - The syntax of selector of edge is: - - `e.src`: Get the source Id of edges - - `e.dst`: Get the destination Id of edges - - `e.data`: Get the edge data on the edges (If there is any, means origin data on the graph) - - - The syntax of selector of results is: - - `r.column_name`: Get the property named `column_name` in results. e.g. `r.hub` in :func:`graphscope.hits`. - """ - - def _transform_selector(self, selector): - return utils.transform_vertex_property_data_selector(selector) - - -class LabeledVertexPropertyContext(BaseContext): - """The labeld kind of context with properties. - This context has several vertex labels and edge labels, - And each label has several properties. - Selection are performed on labels first, then on properties. - - We use `:` to filter labels, and `.` to select properties. - And the results can have several properties. - - The syntax of selector of vertex is: - - `v:label_name.id`: Get Id that belongs to a specific vertex label. - - `v:label_name.property_name`: Get data that on a specific property of a specific vertex label. - - - The syntax of selector of edge is: - - `e:label_name.src`: Get source Id of a specific edge label. - - `e:label_name.dst`: Get destination Id of a specific edge label. - - `e:label_name.property_name`: Get data on a specific property of a specific edge label. - - - The syntax of selector of results is: - - `r:label_name.column_name`: Get the property named `column_name` of `label_name`. - - """ - - def _transform_selector(self, selector): - return utils.transform_labeled_vertex_property_data_selector( - self._graph, selector - ) - - -def create_context(context_type, session_id, context_key, graph): - """A context factory, create concrete context class by context_type.""" - if context_type == "tensor": - return TensorContext(session_id, context_key, graph) - if context_type == "vertex_data": - return VertexDataContext(session_id, context_key, graph) - elif context_type == "labeled_vertex_data": - return LabeledVertexDataContext(session_id, context_key, graph) - elif context_type == "dynamic_vertex_data": - return DynamicVertexDataContext(session_id, context_key, graph) - elif context_type == "vertex_property": - return VertexPropertyContext(session_id, context_key, graph) - elif context_type == "labeled_vertex_property": - return LabeledVertexPropertyContext(session_id, context_key, graph) - else: - raise InvalidArgumentError("Not supported context type: " + context_type) diff --git a/python/graphscope/framework/dag.py b/python/graphscope/framework/dag.py new file mode 100644 index 000000000000..d4c1f65d730f --- /dev/null +++ b/python/graphscope/framework/dag.py @@ -0,0 +1,164 @@ +#!/usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2020 Alibaba Group Holding Limited. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" Classes and functions used to manage dags. +""" + + +import queue + +from graphscope.framework.operation import Operation +from graphscope.proto import op_def_pb2 + + +class Dag(object): + """Class represented as a GraphScope dataflow dag. + + A :class:`Dag` is always belongs to a session and containes a set of + :class:`Operation` object, which performs computations on tensors. + """ + + def __init__(self): + self._seq = 1 + self._ops_by_key = dict() + self._ops_seq_by_key = dict() + + def __str__(self): + return str(self.as_dag_def()) + + def __repr__(self): + return self.__str__() + + def add_op(self, op): + if not isinstance(op, Operation): + raise TypeError("op must be an Operation: {0}".format(op)) + if not op.evaluated and op.key in self._ops_by_key: + raise ValueError("op named {0} already exist in dag".format(op.key)) + self._ops_by_key[op.key] = op + self._ops_seq_by_key[op.key] = self._seq + self._seq += 1 + + def as_dag_def(self): + """Return :class:`Dag` as a :class:`DagDef` proto buffer.""" + dag_def = op_def_pb2.DagDef() + for _, op in self._ops_by_key.items(): + dag_def.op.extend([op.as_op_def()]) + return dag_def + + def to_json(self): + return dict({k: op.to_json() for k, op in self._ops_by_key}) + + def extract_subdag_for(self, ops): + """Extract all nodes included the path that can reach the target ops.""" + out = op_def_pb2.DagDef() + # leaf op handle + # there are two kinds of leaf op: + # 1) unload graph / app + # 2) networkx releated op + if len(ops) == 1 and ops[0].is_leaf_op(): + out.op.extend([ops[0].as_op_def()]) + return out + op_keys = list() + # assert op is not present in current dag + for op in ops: + assert op.key in self._ops_by_key, "%s is not in the dag" % op.key + assert not self._ops_by_key[op.key].evaluated, "%is is evaluated" % op.key + op_keys.append(op.key) + op_keys_to_keep = self._bfs_for_reachable_ops(op_keys) + op_keys_to_keep = sorted(op_keys_to_keep, key=lambda n: self._ops_seq_by_key[n]) + for key in op_keys_to_keep: + op_def = self._ops_by_key[key].as_op_def() + # mark op fetch or not + if key in op_keys: + op_def.fetch = True + out.op.extend([op_def]) + return out + + def clear(self): + self._ops_by_key.clear() + self._ops_seq_by_key.clear() + self._seq = 1 + + def _bfs_for_reachable_ops(self, op_keys): + """Breadth first search for reachable ops from target ops. + + Why we need bfs: + We need to build a dependency order of ops in a DAG + Why we need record a sequence number: + We need to ensure the dependency order is correct when: + - an op is depended by multiple ops + - an op occurs multiple times in target_keys + """ + op_keys_to_keep = set() + next_to_visit = queue.Queue() + for key in op_keys: + next_to_visit.put(key) + while not next_to_visit.empty(): + next_op = next_to_visit.get() + if next_op in op_keys_to_keep: + continue + op_keys_to_keep.add(next_op) + for parent_op in self._ops_by_key[next_op].parents: + if not parent_op.evaluated: + parent_key = parent_op.key + next_to_visit.put(parent_key) + return list(op_keys_to_keep) + + +class DAGNode(object): + """Base class to own :class:`Operation` information which as a node in a DAG.""" + + def __init__(self): + self._op = None + self._session = None + + @property + def op(self): + if self._op is None: + raise ValueError("None value of op in dag node.") + if not isinstance(self._op, Operation): + raise ValueError("Type of op in dag node must be Operation") + return self._op + + @op.setter + def op(self, value): + self._op = value + + @property + def evaluated(self): + return self._op.evaluated + + @evaluated.setter + def evaluated(self, value): + self._op.evaluated = bool(value) + + @property + def session(self): + """Get the session that the graph belogs to.""" + assert self._session is not None + return self._session + + @session.setter + def session(self, value): + self._session = value + + @property + def session_id(self): + """Get the session id that the graph belogs to.""" + assert self._session is not None + return self._session.session_id diff --git a/python/graphscope/framework/dag_utils.py b/python/graphscope/framework/dag_utils.py index 2d4fafde3a18..c9d03a61ba92 100644 --- a/python/graphscope/framework/dag_utils.py +++ b/python/graphscope/framework/dag_utils.py @@ -19,6 +19,7 @@ import numpy as np from graphscope.framework import utils +from graphscope.framework.dag import DAGNode from graphscope.framework.errors import check_argument from graphscope.framework.operation import Operation from graphscope.proto import attr_value_pb2 @@ -27,47 +28,75 @@ from graphscope.proto import types_pb2 -def create_app(graph, app): +def create_app(app_assets): """Wrapper for create an `CREATE_APP` Operation with configuration. + + This op will do nothing but provide required information for `BOUND_APP` + """ + config = {types_pb2.APP_ALGO: utils.s_to_attr(app_assets.algo)} + if app_assets.gar is not None: + config[types_pb2.GAR] = utils.bytes_to_attr(app_assets.gar) + op = Operation( + None, types_pb2.CREATE_APP, config=config, output_types=types_pb2.APP + ) + return op + + +def bind_app(graph, app_assets): + """Wrapper for create an `BIND_APP` Operation with configuration. Compile and load an application after evaluated. Args: - graph (:class:`Graph`): A :class:`Graph` instance - app (:class:`App`): A :class:`App` instance. + graph (:class:`GraphDAGNode`): A :class:`GraphDAGNode` instance + app (:class:`AppAssets`): A :class:`AppAssets` instance. Returns: An :class:`Operation` with configuration that instruct analytical engine how to build the app. """ - config = { - types_pb2.APP_ALGO: utils.s_to_attr(app.algo), - types_pb2.GRAPH_TYPE: utils.graph_type_to_attr(graph.graph_type), - types_pb2.OID_TYPE: utils.s_to_attr( - utils.normalize_data_type_str(utils.data_type_to_cpp(graph.schema.oid_type)) - ), - types_pb2.VID_TYPE: utils.s_to_attr( - utils.data_type_to_cpp(graph.schema.vid_type) - ), - types_pb2.V_DATA_TYPE: utils.s_to_attr( - utils.data_type_to_cpp(graph.schema.vdata_type) - ), - types_pb2.E_DATA_TYPE: utils.s_to_attr( - utils.data_type_to_cpp(graph.schema.edata_type) - ), - } - if app.gar is not None: - config[types_pb2.GAR] = utils.bytes_to_attr(app.gar) - - opr = Operation( + inputs = [graph.op, app_assets.op] + op = Operation( graph.session_id, - types_pb2.CREATE_APP, + types_pb2.BIND_APP, + inputs=inputs, + config={}, + output_types=types_pb2.BOUND_APP, + ) + return op + + +def run_app(app, *args, **kwargs): + """Run `bound app` on the `graph`. + + Args: + app (:class:`AppDAGNode`): A :class:`AppDAGNode` instance which represent a bound app. + key (str): Key of query results, can be used to retrieve results. + *args: Additional query params that will be used in evaluation. + **kwargs: Key-value formated query params that mostly used in Cython apps. + + Returns: + An op to run app on the specified graph, with optional query parameters. + """ + inputs = [app.op] + config = {} + output_prefix = kwargs.pop("output_prefix", ".") + config[types_pb2.OUTPUT_PREFIX] = utils.s_to_attr(output_prefix) + # optional query arguments. + params = utils.pack_query_params(*args, **kwargs) + query_args = query_args_pb2.QueryArgs() + query_args.args.extend(params) + op = Operation( + app.session_id, + types_pb2.RUN_APP, + inputs=inputs, config=config, - output_types=types_pb2.APP, + output_types=types_pb2.RESULTS, + query_args=query_args, ) - return opr + return op -def create_graph(session_id, graph_type, **kwargs): +def create_graph(session_id, graph_type, inputs=None, **kwargs): """Create an `CREATE_GRAPH` op, add op to default dag. Args: @@ -97,7 +126,11 @@ def create_graph(session_id, graph_type, **kwargs): raise RuntimeError("Not supported graph type {}".format(graph_type)) op = Operation( - session_id, types_pb2.CREATE_GRAPH, config=config, output_types=types_pb2.GRAPH + session_id, + types_pb2.CREATE_GRAPH, + inputs=inputs, + config=config, + output_types=types_pb2.GRAPH, ) return op @@ -120,11 +153,15 @@ def add_labels_to_graph(graph, **kwargs): Since we don't want to trigger the loading, we must not use any api that can trigger the loading process implicitly. """ + from graphscope.framework.graph import GraphDAGNode + + assert isinstance(graph, GraphDAGNode) + inputs = [graph.op] config = { - types_pb2.GRAPH_NAME: utils.s_to_attr(graph._key), types_pb2.GRAPH_TYPE: utils.graph_type_to_attr(graph._graph_type), } - + # inferred from the context of the dag. + config.update({types_pb2.GRAPH_NAME: utils.place_holder_to_attr()}) if graph._graph_type == graph_def_pb2.ARROW_PROPERTY: attrs = kwargs.pop("attrs", None) if attrs: @@ -138,6 +175,7 @@ def add_labels_to_graph(graph, **kwargs): op = Operation( graph._session.session_id, types_pb2.ADD_LABELS, + inputs=inputs, config=config, output_types=types_pb2.GRAPH, ) @@ -271,40 +309,6 @@ def modify_vertices(graph, modify_type, vertices): return op -def run_app(graph, app, *args, **kwargs): - """Run `app` on the `graph`. - - Args: - graph (:class:`Graph`): A loaded graph. - app (:class:`App`): A loaded app that will be queried. - key (str): Key of query results, can be used to retrieve results. - *args: Additional query params that will be used in evaluation. - **kwargs: Key-value formated query params that mostly used in Cython apps. - - Returns: - An op to run app on the specified graph, with optional query parameters. - """ - config = { - types_pb2.GRAPH_NAME: utils.s_to_attr(graph.key), - types_pb2.APP_NAME: utils.s_to_attr(app.key), - } - output_prefix = kwargs.pop("output_prefix", ".") - config[types_pb2.OUTPUT_PREFIX] = utils.s_to_attr(output_prefix) - # optional query arguments. - params = utils.pack_query_params(*args, **kwargs) - query_args = query_args_pb2.QueryArgs() - query_args.args.extend(params) - - op = Operation( - graph.session_id, - types_pb2.RUN_APP, - config=config, - output_types=types_pb2.RESULTS, - query_args=query_args, - ) - return op - - def report_graph( graph, report_type, node=None, edge=None, fid=None, lid=None, key=None ): @@ -369,24 +373,20 @@ def report_graph( def project_arrow_property_graph(graph, vertex_collections, edge_collections): check_argument(graph.graph_type == graph_def_pb2.ARROW_PROPERTY) - attr = attr_value_pb2.AttrValue() - v_attr = attr_value_pb2.NameAttrList() - e_attr = attr_value_pb2.NameAttrList() - for label, props in vertex_collections.items(): - v_attr.attr[label].CopyFrom(utils.list_i_to_attr(props)) - for label, props in edge_collections.items(): - e_attr.attr[label].CopyFrom(utils.list_i_to_attr(props)) - attr.list.func.extend([v_attr, e_attr]) - config = { - types_pb2.GRAPH_NAME: utils.s_to_attr(graph.key), types_pb2.GRAPH_TYPE: utils.graph_type_to_attr(graph.graph_type), - types_pb2.ARROW_PROPERTY_DEFINITION: attr, } + config.update( + { + types_pb2.VERTEX_COLLECTIONS: utils.s_to_attr(vertex_collections), + types_pb2.EDGE_COLLECTIONS: utils.s_to_attr(edge_collections), + } + ) op = Operation( graph.session_id, types_pb2.PROJECT_GRAPH, config=config, + inputs=[graph.op], output_types=types_pb2.GRAPH, ) return op @@ -394,12 +394,12 @@ def project_arrow_property_graph(graph, vertex_collections, edge_collections): def project_arrow_property_graph_to_simple( graph, - v_label_id, - v_prop_id, - e_label_id, - e_prop_id, - v_data_type, - e_data_type, + v_label_id=None, + v_prop_id=None, + e_label_id=None, + e_prop_id=None, + v_data_type=None, + e_data_type=None, oid_type=None, vid_type=None, ): @@ -417,22 +417,12 @@ def project_arrow_property_graph_to_simple( An op to project `graph`, results in a simple ARROW_PROJECTED graph. """ check_argument(graph.graph_type == graph_def_pb2.ARROW_PROPERTY) - config = { - types_pb2.GRAPH_NAME: utils.s_to_attr(graph.key), - types_pb2.GRAPH_TYPE: utils.graph_type_to_attr(graph_def_pb2.ARROW_PROJECTED), - types_pb2.V_LABEL_ID: utils.i_to_attr(v_label_id), - types_pb2.V_PROP_ID: utils.i_to_attr(v_prop_id), - types_pb2.E_LABEL_ID: utils.i_to_attr(e_label_id), - types_pb2.E_PROP_ID: utils.i_to_attr(e_prop_id), - types_pb2.OID_TYPE: utils.s_to_attr(utils.data_type_to_cpp(oid_type)), - types_pb2.VID_TYPE: utils.s_to_attr(utils.data_type_to_cpp(vid_type)), - types_pb2.V_DATA_TYPE: utils.s_to_attr(utils.data_type_to_cpp(v_data_type)), - types_pb2.E_DATA_TYPE: utils.s_to_attr(utils.data_type_to_cpp(e_data_type)), - } + config = {} op = Operation( graph.session_id, types_pb2.PROJECT_TO_SIMPLE, config=config, + inputs=[graph.op], output_types=types_pb2.GRAPH, ) return op @@ -654,9 +644,13 @@ def unload_app(app): Returns: An op to unload the `app`. """ - config = {types_pb2.APP_NAME: utils.s_to_attr(app.key)} + config = {} op = Operation( - app._session_id, types_pb2.UNLOAD_APP, config=config, output_types=types_pb2.APP + app._session_id, + types_pb2.UNLOAD_APP, + config=config, + inputs=[app.op], + output_types=types_pb2.NULL_OUTPUT, ) return op @@ -670,20 +664,23 @@ def unload_graph(graph): Returns: An op to unload the `graph`. """ - config = {types_pb2.GRAPH_NAME: utils.s_to_attr(graph.key)} - # Dynamic graph doesn't have a vineyard id - if hasattr(graph, "vineyard_id"): - config[types_pb2.VINEYARD_ID] = utils.i_to_attr(graph.vineyard_id) + config = {} + if not isinstance(graph, DAGNode): + config.update({types_pb2.GRAPH_NAME: utils.s_to_attr(graph.key)}) + # Dynamic graph doesn't have a vineyard id + if hasattr(graph, "vineyard_id"): + config[types_pb2.VINEYARD_ID] = utils.i_to_attr(graph.vineyard_id) op = Operation( graph.session_id, types_pb2.UNLOAD_GRAPH, config=config, - output_types=types_pb2.GRAPH, + inputs=[graph.op], + output_types=types_pb2.NULL_OUTPUT, ) return op -def context_to_numpy(results, selector=None, vertex_range=None, axis=0): +def context_to_numpy(context, selector=None, vertex_range=None, axis=0): """Retrieve results as a numpy ndarray. Args: @@ -694,9 +691,7 @@ def context_to_numpy(results, selector=None, vertex_range=None, axis=0): Returns: An op to retrieve query results and convert to numpy ndarray. """ - config = { - types_pb2.CTX_NAME: utils.s_to_attr(results.key), - } + config = {} if selector is not None: config[types_pb2.SELECTOR] = utils.s_to_attr(selector) if vertex_range is not None: @@ -704,15 +699,16 @@ def context_to_numpy(results, selector=None, vertex_range=None, axis=0): if axis is not None: config[types_pb2.AXIS] = utils.i_to_attr(axis) op = Operation( - results._session_id, + context.session_id, types_pb2.CONTEXT_TO_NUMPY, config=config, + inputs=[context.op], output_types=types_pb2.TENSOR, ) return op -def context_to_dataframe(results, selector=None, vertex_range=None): +def context_to_dataframe(context, selector=None, vertex_range=None): """Retrieve results as a pandas DataFrame. Args: @@ -723,23 +719,22 @@ def context_to_dataframe(results, selector=None, vertex_range=None): Returns: An op to retrieve query results and convert to pandas DataFrame. """ - config = { - types_pb2.CTX_NAME: utils.s_to_attr(results.key), - } + config = {} if selector is not None: config[types_pb2.SELECTOR] = utils.s_to_attr(selector) if vertex_range is not None: config[types_pb2.VERTEX_RANGE] = utils.s_to_attr(vertex_range) op = Operation( - results._session_id, + context.session_id, types_pb2.CONTEXT_TO_DATAFRAME, config=config, + inputs=[context.op], output_types=types_pb2.DATAFRAME, ) return op -def to_vineyard_tensor(results, selector=None, vertex_range=None, axis=None): +def to_vineyard_tensor(context, selector=None, vertex_range=None, axis=None): """Retrieve results as vineyard tensor. Parameters: @@ -749,9 +744,7 @@ def to_vineyard_tensor(results, selector=None, vertex_range=None, axis=None): Returns: An op to convert query results into a vineyard tensor. """ - config = { - types_pb2.CTX_NAME: utils.s_to_attr(results.key), - } + config = {} if selector is not None: config[types_pb2.SELECTOR] = utils.s_to_attr(selector) if vertex_range is not None: @@ -759,15 +752,16 @@ def to_vineyard_tensor(results, selector=None, vertex_range=None, axis=None): if axis is not None: config[types_pb2.AXIS] = utils.i_to_attr(axis) op = Operation( - results._session_id, + context.session_id, types_pb2.TO_VINEYARD_TENSOR, config=config, + inputs=[context.op], output_types=types_pb2.VINEYARD_TENSOR, ) return op -def to_vineyard_dataframe(results, selector=None, vertex_range=None): +def to_vineyard_dataframe(context, selector=None, vertex_range=None): """Retrieve results as vineyard dataframe. Parameters: @@ -777,17 +771,16 @@ def to_vineyard_dataframe(results, selector=None, vertex_range=None): Returns: An op to convert query results into a vineyard dataframe. """ - config = { - types_pb2.CTX_NAME: utils.s_to_attr(results.key), - } + config = {} if selector is not None: config[types_pb2.SELECTOR] = utils.s_to_attr(selector) if vertex_range is not None: config[types_pb2.VERTEX_RANGE] = utils.s_to_attr(vertex_range) op = Operation( - results._session_id, + context.session_id, types_pb2.TO_VINEYARD_DATAFRAME, config=config, + inputs=[context.op], output_types=types_pb2.VINEYARD_DATAFRAME, ) return op @@ -819,16 +812,12 @@ def add_column(graph, results, selector): Returns: A new graph with new columns added. """ - config = { - types_pb2.GRAPH_NAME: utils.s_to_attr(graph.key), - types_pb2.GRAPH_TYPE: utils.graph_type_to_attr(graph.graph_type), - types_pb2.CTX_NAME: utils.s_to_attr(results.key), - types_pb2.SELECTOR: utils.s_to_attr(selector), - } + config = {types_pb2.SELECTOR: utils.s_to_attr(selector)} op = Operation( graph.session_id, types_pb2.ADD_COLUMN, config=config, + inputs=[graph.op, results.op], output_types=types_pb2.GRAPH, ) return op diff --git a/python/graphscope/framework/errors.py b/python/graphscope/framework/errors.py index 411d9ea17487..80a53492bbbc 100644 --- a/python/graphscope/framework/errors.py +++ b/python/graphscope/framework/errors.py @@ -17,6 +17,7 @@ # import inspect +import pickle from graphscope.proto import error_codes_pb2 from graphscope.proto import op_def_pb2 @@ -36,6 +37,7 @@ "AnalyticalEngineInternalError", "InteractiveEngineInternalError", "LearningEngineInternalError", + "CoordinatorInternalError", "NetworkError", "K8sError", "UnknownError", @@ -112,6 +114,10 @@ class LearningEngineInternalError(GSError): pass +class CoordinatorInternalError(GSError): + pass + + class NetworkError(GSError): pass @@ -151,6 +157,7 @@ def __init__(self, message): error_codes_pb2.ANALYTICAL_ENGINE_INTERNAL_ERROR: AnalyticalEngineInternalError, error_codes_pb2.INTERACTIVE_ENGINE_INTERNAL_ERROR: InteractiveEngineInternalError, error_codes_pb2.LEARNING_ENGINE_INTERNAL_ERROR: LearningEngineInternalError, + error_codes_pb2.COORDINATOR_INTERNAL_ERROR: CoordinatorInternalError, error_codes_pb2.UNKNOWN_ERROR: UnknownError, error_codes_pb2.FATAL_ERROR: FatalError, } @@ -168,6 +175,9 @@ def check_grpc_response(response): error_type = _gs_error_types.get(status.code) if error_type: + if error_type == CoordinatorInternalError: + e = pickle.loads(detail) + raise (e) raise error_type(status.error_msg, detail) else: raise RuntimeError( diff --git a/python/graphscope/framework/graph.py b/python/graphscope/framework/graph.py index 08a3cde72b97..465d13d9d851 100644 --- a/python/graphscope/framework/graph.py +++ b/python/graphscope/framework/graph.py @@ -20,6 +20,8 @@ import json import logging import threading +from abc import ABCMeta +from abc import abstractmethod from copy import deepcopy from typing import List from typing import Mapping @@ -31,6 +33,7 @@ from graphscope.framework import dag_utils from graphscope.framework import graph_utils from graphscope.framework import utils +from graphscope.framework.dag import DAGNode from graphscope.framework.errors import check_argument from graphscope.framework.graph_schema import GraphSchema from graphscope.framework.graph_utils import EdgeLabel @@ -38,13 +41,439 @@ from graphscope.framework.graph_utils import VertexLabel from graphscope.framework.operation import Operation from graphscope.framework.utils import data_type_to_cpp +from graphscope.proto import attr_value_pb2 from graphscope.proto import graph_def_pb2 from graphscope.proto import types_pb2 logger = logging.getLogger("graphscope") -class Graph(object): +class GraphInterface(metaclass=ABCMeta): + """Base Class to derive GraphDAGNode and Graph""" + + def __init__(self): + self._session = None + + @abstractmethod + def add_column(self, results, selector): + raise NotImplementedError + + @abstractmethod + def add_vertices(self, vertices, label="_", properties=None, vid_field=0): + raise NotImplementedError + + @abstractmethod + def add_edges( + self, + edges, + label="_", + properties=None, + src_label=None, + dst_label=None, + src_field=0, + dst_field=1, + ): + raise NotImplementedError + + @abstractmethod + def unload(self): + raise NotImplementedError + + def to_numpy(self, selector, vertex_range=None): + raise NotImplementedError + + def to_dataframe(self, selector, vertex_range=None): + raise NotImplementedError + + def save_to(self, path, **kwargs): + raise NotImplementedError + + def load_from(cls, path, sess, **kwargs): + raise NotImplementedError + + @abstractmethod + def project(self, vertices, edges): + raise NotImplementedError + + def _from_vineyard(self, vineyard_object): + """Load a graph from a already existed vineyard graph. + + Args: + vineyard_object (:class:`vineyard.Object`, :class:`vineyard.ObjectID` + or :class:`vineyard.ObjectName`): vineyard object, + which represents a graph. + + Returns: :class:`Operation` + """ + if isinstance(vineyard_object, vineyard.Object): + return self._construct_op_from_vineyard_id(vineyard_object.id) + if isinstance(vineyard_object, vineyard.ObjectID): + return self._construct_op_from_vineyard_id(vineyard_object) + if isinstance(vineyard_object, vineyard.ObjectName): + return self._construct_op_from_vineyard_name(vineyard_object) + + def _construct_op_from_vineyard_id(self, vineyard_id): + assert self._session is not None + config = {} + config[types_pb2.IS_FROM_VINEYARD_ID] = utils.b_to_attr(True) + config[types_pb2.VINEYARD_ID] = utils.i_to_attr(int(vineyard_id)) + # FIXME(hetao) hardcode oid/vid type for codegen, when loading from vineyard + # + # the metadata should be retrived from vineyard + config[types_pb2.OID_TYPE] = utils.s_to_attr("int64_t") + config[types_pb2.VID_TYPE] = utils.s_to_attr("uint64_t") + return dag_utils.create_graph( + self.session_id, graph_def_pb2.ARROW_PROPERTY, attrs=config + ) + + def _construct_op_from_vineyard_name(self, vineyard_name): + assert self._session is not None + config = {} + config[types_pb2.IS_FROM_VINEYARD_ID] = utils.b_to_attr(True) + config[types_pb2.VINEYARD_NAME] = utils.s_to_attr(str(vineyard_name)) + # FIXME(hetao) hardcode oid/vid type for codegen, when loading from vineyard + # + # the metadata should be retrived from vineyard + config[types_pb2.OID_TYPE] = utils.s_to_attr("int64_t") + config[types_pb2.VID_TYPE] = utils.s_to_attr("uint64_t") + return dag_utils.create_graph( + self.session_id, graph_def_pb2.ARROW_PROPERTY, attrs=config + ) + + def _construct_op_of_empty_graph(self): + config = {} + config[types_pb2.ARROW_PROPERTY_DEFINITION] = attr_value_pb2.AttrValue() + config[types_pb2.DIRECTED] = utils.b_to_attr(self._directed) + config[types_pb2.GENERATE_EID] = utils.b_to_attr(self._generate_eid) + config[types_pb2.OID_TYPE] = utils.s_to_attr(self._oid_type) + config[types_pb2.VID_TYPE] = utils.s_to_attr("uint64_t") + config[types_pb2.IS_FROM_VINEYARD_ID] = utils.b_to_attr(False) + return dag_utils.create_graph( + self.session_id, graph_def_pb2.ARROW_PROPERTY, inputs=None, attrs=config + ) + + +class GraphDAGNode(DAGNode, GraphInterface): + """Graph node in a DAG.""" + + def __init__( + self, + session, + incoming_data=None, + oid_type="int64", + directed=True, + generate_eid=True, + ): + """Construct a :class:`GraphDAGNode` object. + + Args: + session (:class:`Session`): A graphscope session instance. + incoming_data: Graph can be initialized through various type of sources, + which can be one of: + + - :class:`Operation` + - :class:`nx.Graph` + - :class:`Graph` + - :class:`vineyard.Object`, :class:`vineyard.ObjectId` or :class:`vineyard.ObjectName` + oid_type: (str, optional): Type of vertex original id. Defaults to "int64". + directed: (bool, optional): Directed graph or not. Defaults to True. + generate_eid: (bool, optional): Generate id for each edge when setted True. Defaults to True. + """ + + super().__init__() + self._session = session + oid_type = utils.normalize_data_type_str(oid_type) + if oid_type not in ("int64_t", "std::string"): + raise ValueError("oid_type can only be int64_t or string.") + self._oid_type = oid_type + self._directed = directed + self._generate_eid = generate_eid + self._graph_type = graph_def_pb2.ARROW_PROPERTY + # list of pair + self._unsealed_vertices_and_edges = list() + # check for newly added vertices and edges. + self._v_labels = list() + self._e_labels = list() + self._e_relationships = list() + self._base_graph = None + # add op to dag + self._resolve_op(incoming_data) + self._session.dag.add_op(self._op) + + def __del__(self): + # TODO think about the DAGNode has been run. + pass + + @property + def v_labels(self): + return self._v_labels + + @v_labels.setter + def v_labels(self, value): + self._v_labels = value + + @property + def e_labels(self): + return self._e_labels + + @e_labels.setter + def e_labels(self, value): + self._e_labels = value + + @property + def e_relationships(self): + return self._e_relationships + + @e_relationships.setter + def e_relationships(self, value): + self._e_relationships = value + + @property + def graph_type(self): + """The type of the graph object. + + Returns: + type (`types_pb2.GraphType`): the type of the graph. + """ + return self._graph_type + + def _resolve_op(self, incoming_data): + # Don't import the :code:`NXGraph` in top-level statements to improve the + # performance of :code:`import graphscope`. + from graphscope import nx + + if incoming_data is None: + # create dag node of empty graph + self._op = self._construct_op_of_empty_graph() + elif isinstance(incoming_data, Operation): + self._op = incoming_data + if self._op.type == types_pb2.PROJECT_TO_SIMPLE: + self._graph_type = graph_def_pb2.ARROW_PROJECTED + elif isinstance(incoming_data, nx.Graph): + self._op = self._from_nx_graph(incoming_data) + elif isinstance(incoming_data, Graph): + self._op = dag_utils.copy_graph(incoming_data) + self._graph_type = incoming_data.graph_type + elif isinstance(incoming_data, GraphDAGNode): + if incoming_data.session_id != self.session_id: + raise RuntimeError("{0} not in the same session.".formar(incoming_data)) + raise NotImplementedError + elif isinstance( + incoming_data, (vineyard.Object, vineyard.ObjectID, vineyard.ObjectName) + ): + self._op = self._from_vineyard(incoming_data) + else: + raise RuntimeError("Not supported incoming data.") + + def add_vertices(self, vertices, label="_", properties=None, vid_field=0): + if label in self._v_labels: + raise ValueError(f"Label {label} already existed in graph.") + if not self._v_labels and self._e_labels: + raise ValueError("Cannot manually add vertices after inferred vertices.") + unsealed_vertices_and_edges = deepcopy(self._unsealed_vertices_and_edges) + vertex_label = VertexLabel( + label=label, + loader=vertices, + properties=properties, + vid_field=vid_field, + session_id=self._session.session_id, + ) + unsealed_vertices_and_edges.append((self.op.key, vertex_label)) + v_labels = deepcopy(self._v_labels) + v_labels.append(label) + # construct op + config = graph_utils.assemble_op_config( + [vertex_label], [], self._oid_type, self._directed, self._generate_eid + ) + op = dag_utils.add_labels_to_graph(self, attrs=config) + # construct dag node + graph_dag_node = GraphDAGNode( + self._session, op, self._oid_type, self._directed, self._generate_eid + ) + graph_dag_node._v_labels = v_labels + graph_dag_node._e_labels = self._e_labels + graph_dag_node._e_relationships = self._e_relationships + graph_dag_node._unsealed_vertices_and_edges = unsealed_vertices_and_edges + graph_dag_node._base_graph = self + return graph_dag_node + + def add_edges( + self, + edges, + label="_", + properties=None, + src_label=None, + dst_label=None, + src_field=0, + dst_field=1, + ): + if src_label is None and dst_label is None: + check_argument( + len(self._v_labels) <= 1, + "Ambiguous vertex label, please specify the src_label and dst_label.", + ) + if len(self._v_labels) == 1: + src_label = dst_label = self._v_labels[0] + else: + src_label = dst_label = "_" + + if src_label is None or dst_label is None: + raise ValueError( + "src and dst label must be both specified or either unspecified." + ) + + if self._v_labels: + if src_label not in self._v_labels or dst_label not in self._v_labels: + raise ValueError("src label or dst_label not existed in graph.") + else: + # We can infer all vertices label in the graph constructing stage. + pass + + check_argument( + src_field != dst_field, "src and dst field cannot refer to the same field" + ) + + if self.evaluated: + if label in self._e_labels: + raise ValueError(f"Label {label} already existed in graph") + + unsealed_vertices = list() + unsealed_edges = list() + + e_labels = deepcopy(self._e_labels) + relations = deepcopy(self._e_relationships) + + parent = self + if label in self.e_labels: + # aggregate op with the same edge label + fork = False + unsealed_vertices_and_edges = list() + for parent_op_key, unsealed_v_or_e in self._unsealed_vertices_and_edges: + if ( + isinstance(unsealed_v_or_e, EdgeLabel) + and unsealed_v_or_e.label == label + ): + parent = self._backtrack_graph_dag_node_by_op_key(parent_op_key) + cur_label = unsealed_v_or_e + cur_label.add_sub_label( + EdgeSubLabel( + edges, + properties, + src_label, + dst_label, + src_field, + dst_field, + ) + ) + fork = True + else: + unsealed_vertices_and_edges.append((parent_op_key, unsealed_v_or_e)) + if fork: + if isinstance(unsealed_v_or_e, VertexLabel): + unsealed_vertices.append(unsealed_v_or_e) + else: + unsealed_edges.append(unsealed_v_or_e) + unsealed_edges.append(cur_label) + unsealed_vertices_and_edges.append((parent.op.key, cur_label)) + else: + unsealed_vertices_and_edges = deepcopy(self._unsealed_vertices_and_edges) + e_labels.append(label) + relations.append([(src_label, dst_label)]) + cur_label = EdgeLabel(label, self._session.session_id) + cur_label.add_sub_label( + EdgeSubLabel( + edges, properties, src_label, dst_label, src_field, dst_field + ) + ) + unsealed_edges.append(cur_label) + unsealed_vertices_and_edges.append((parent.op.key, cur_label)) + # construct op + config = graph_utils.assemble_op_config( + unsealed_vertices, + unsealed_edges, + self._oid_type, + self._directed, + self._generate_eid, + ) + op = dag_utils.add_labels_to_graph(parent, attrs=config) + # construct dag node + graph_dag_node = GraphDAGNode( + self._session, op, self._oid_type, self._directed, self._generate_eid + ) + graph_dag_node._v_labels = self._v_labels + graph_dag_node._e_labels = e_labels + graph_dag_node._e_relationships = relations + graph_dag_node._unsealed_vertices_and_edges = unsealed_vertices_and_edges + graph_dag_node._base_graph = parent + return graph_dag_node + + def _from_nx_graph(self, incoming_graph): + """Create a gs graph from a nx graph. + Args: + incoming_graph (:class:`nx.graph`): A nx graph that contains graph data. + + Returns: + that will be used to construct a gs.Graph + + Raises: + TypeError: Raise Error if graph type not match. + + Examples: + >>> nx_g = nx.path_graph(10) + >>> gs_g = gs.Graph(nx_g) + """ + if hasattr(incoming_graph, "_graph"): + msg = "graph view can not convert to gs graph" + raise TypeError(msg) + return dag_utils.dynamic_to_arrow(incoming_graph) + + def _backtrack_graph_dag_node_by_op_key(self, key): + if self.op.key == key: + return self + graph_dag_node = self._base_graph + while graph_dag_node is not None: + if graph_dag_node.op.key == key: + return graph_dag_node + graph_dag_node = graph_dag_node._base_graph + + def add_column(self, results, selector): + check_argument( + isinstance(selector, Mapping), "selector of add column must be a dict" + ) + selector = json.dumps(selector) + op = dag_utils.add_column(self, results, selector) + graph_dag_node = GraphDAGNode(self._session, op) + graph_dag_node._base_graph = self + return graph_dag_node + + def unload(self): + # do nothing for dag node + pass + + def project( + self, + vertices: Mapping[str, Union[List[str], None]], + edges: Mapping[str, Union[List[str], None]], + ): + check_argument(self.graph_type == graph_def_pb2.ARROW_PROPERTY) + op = dag_utils.project_arrow_property_graph( + self, json.dumps(vertices), json.dumps(edges) + ) + # construct dag node + graph_dag_node = GraphDAGNode(self._session, op) + graph_dag_node._base_graph = self + return graph_dag_node + + def _project_to_simple(self): + check_argument(self.graph_type == graph_def_pb2.ARROW_PROPERTY) + op = dag_utils.project_arrow_property_graph_to_simple(self) + # construct dag node + graph_dag_node = GraphDAGNode(self._session, op) + graph_dag_node._base_graph = self + return graph_dag_node + + +class Graph(GraphInterface): """A class for representing metadata of a graph in the GraphScope. A :class:`Graph` object holds the metadata of a graph, such as key, schema, and the graph is directed or not. @@ -70,77 +499,26 @@ class Graph(object): def __init__( self, - session, - incoming_data=None, - oid_type="int64", - directed=True, - generate_eid=True, + graph_node, ): - """Construct a :class:`Graph` object. + """Construct a :class:`Graph` object.""" - Args: - session_id (str): Session id of the session the graph is created in. - incoming_data: Graph can be initialized through various type of sources, - which can be one of: - - - :class:`Operation` - - :class:`nx.Graph` - - :class:`Graph` - - :class:`vineyard.Object`, :class:`vineyard.ObjectId` or :class:`vineyard.ObjectName` - """ + self._graph_node = graph_node + self._session = self._graph_node.session + # copy and set op evaluated + self._graph_node.op = deepcopy(self._graph_node.op) + self._graph_node.evaluated = True + self._session.dag.add_op(self._graph_node.op) self._key = None - self._graph_type = graph_def_pb2.ARROW_PROPERTY self._vineyard_id = 0 self._schema = GraphSchema() - self._session = session self._detached = False self._interactive_instance_launching_thread = None self._interactive_instance_list = [] self._learning_instance_list = [] - # Hold uncompleted operation for lazy evaluation - self._pending_op = None - # Hold a reference to base graph of modify operation, - # to avoid being garbage collected - self._base_graph = None - - oid_type = utils.normalize_data_type_str(oid_type) - if oid_type not in ("int64_t", "std::string"): - raise ValueError("oid_type can only be int64_t or string.") - self._oid_type = oid_type - self._directed = directed - self._generate_eid = generate_eid - - self._unsealed_vertices = {} - self._unsealed_edges = {} - # Used to isplay schema without load into vineyard, - # and do sanity checking for newly added vertices and edges. - self._v_labels = [] - self._e_labels = [] - self._e_relationships = [] - - if incoming_data is not None: - # Don't import the :code:`NXGraph` in top-level statements to improve the - # performance of :code:`import graphscope`. - from graphscope import nx - - if isinstance(incoming_data, Operation): - self._pending_op = incoming_data - if self._pending_op.type == types_pb2.PROJECT_TO_SIMPLE: - self._graph_type = graph_def_pb2.ARROW_PROJECTED - elif isinstance(incoming_data, nx.Graph): - self._pending_op = self._from_nx_graph(incoming_data) - elif isinstance(incoming_data, Graph): - self._pending_op = self._copy_from(incoming_data) - elif isinstance( - incoming_data, (vineyard.Object, vineyard.ObjectID, vineyard.ObjectName) - ): - self._pending_op = self._from_vineyard(incoming_data) - else: - raise RuntimeError("Not supported incoming data.") - def __del__(self): # cleanly ignore all exceptions, cause session may already closed / destroyed. try: @@ -168,11 +546,13 @@ def _launch_interactive_instance_impl(self): # Unexpect and suppress all exceptions here. pass - def _from_graph_def(self, graph_def): + def update_from_graph_def(self, graph_def): check_argument( - self._graph_type == graph_def.graph_type, "Graph type doesn't match." + self._graph_node.graph_type == graph_def.graph_type, + "Graph type doesn't match {} versus {}".format( + self._graph_node.graph_type, graph_def.graph_type + ), ) - self._key = graph_def.key self._directed = graph_def.directed vy_info = graph_def_pb2.VineyardInfoPb() @@ -186,49 +566,26 @@ def _from_graph_def(self, graph_def): self._v_labels = self._schema.vertex_labels self._e_labels = self._schema.edge_labels self._e_relationships = self._schema.edge_relationships + # init saved_signature (must be after init schema) + self._saved_signature = self.signature + # create gremlin server pod asynchronously + if self._session.eager() and gs_config.initializing_interactive_engine: + self._interactive_instance_launching_thread = threading.Thread( + target=self._launch_interactive_instance_impl, args=() + ) + self._interactive_instance_launching_thread.start() - def _ensure_loaded(self): - if self._key is not None and self._pending_op is None: - return - # Unloaded - if self._session is None: - raise RuntimeError("The graph is not loaded") - # Empty graph - if self._key is None and self._pending_op is None: - self._pending_op = self._construct_op({}, {}, False) - # Try to load - if self._pending_op is not None: - # Create a graph from scratch. - graph_def = self._pending_op.eval() - self._from_graph_def(graph_def) - self._pending_op = None - self._base_graph = None - self._unsealed_vertices.clear() - self._unsealed_edges.clear() - # init saved_signature (must be after init schema) - self._saved_signature = self.signature - # create gremlin server pod asynchronously - if gs_config.initializing_interactive_engine: - self._interactive_instance_launching_thread = threading.Thread( - target=self._launch_interactive_instance_impl, args=() - ) - self._interactive_instance_launching_thread.start() + def __getattr__(self, name): + if hasattr(self._graph_node, name): + return getattr(self._graph_node, name) + else: + raise AttributeError("{0} not found.".format(name)) @property def key(self): """The key of the corresponding graph in engine.""" - self._ensure_loaded() return self._key - @property - def graph_type(self): - """The type of the graph object. - - Returns: - type (`types_pb2.GraphType`): the type of the graph. - """ - return self._graph_type - @property def schema(self): """Schema of the graph. @@ -236,7 +593,6 @@ def schema(self): Returns: :class:`GraphSchema`: the schema of the graph """ - self._ensure_loaded() return self._schema @property @@ -246,20 +602,16 @@ def schema_path(self): Returns: str: The path contains the schema. for interactive engine. """ - self._ensure_loaded() return self._schema_path @property def signature(self): - self._ensure_loaded() return hashlib.sha256( "{}.{}".format(self._schema.signature(), self._key).encode("utf-8") ).hexdigest() @property def template_str(self): - self._ensure_loaded() - # transform str/string to std::string oid_type = utils.normalize_data_type_str(self._oid_type) vid_type = self._schema.vid_type @@ -282,7 +634,6 @@ def vineyard_id(self): Returns: str: return vineyard id of this graph """ - self._ensure_loaded() return self._vineyard_id @property @@ -303,10 +654,6 @@ def detach(self): self._detached = True def loaded(self): - try: - self._ensure_loaded() - except RuntimeError: - return False return self._key is not None def __str__(self): @@ -332,14 +679,8 @@ def unload(self): if self._key is None: self._session = None - self._pending_op = None return - if self._pending_op is not None: - self._key = None - self._session = None - self._pending_op = None - return # close interactive instances first try: if ( @@ -365,57 +706,9 @@ def unload(self): op.eval() self._key = None self._session = None - self._pending_op = None def _project_to_simple(self): - self._ensure_loaded() - check_argument(self.graph_type == graph_def_pb2.ARROW_PROPERTY) - check_argument( - self.schema.vertex_label_num == 1, - "Cannot project to simple, vertex label number is not one.", - ) - check_argument( - self.schema.edge_label_num == 1, - "Cannot project to simple, edge label number is not one.", - ) - # Check relation v_label -> e_label <- v_label exists. - v_label = self.schema.vertex_labels[0] - e_label = self.schema.edge_labels[0] - relation = (v_label, v_label) - check_argument( - relation in self._schema.get_relationships(e_label), - f"Cannot project to simple, Graph doesn't contain such relationship: {v_label} -> {e_label} <- {v_label}.", - ) - v_props = self.schema.get_vertex_properties(v_label) - e_props = self.schema.get_edge_properties(e_label) - check_argument(len(v_props) <= 1) - check_argument(len(e_props) <= 1) - - v_label_id = self.schema.get_vertex_label_id(v_label) - e_label_id = self.schema.get_edge_label_id(e_label) - v_prop_id, vdata_type = ( - (v_props[0].id, v_props[0].type) if v_props else (-1, None) - ) - e_prop_id, edata_type = ( - (e_props[0].id, e_props[0].type) if e_props else (-1, None) - ) - oid_type = self._schema.oid_type - vid_type = self._schema.vid_type - - op = dag_utils.project_arrow_property_graph_to_simple( - self, - v_label_id, - v_prop_id, - e_label_id, - e_prop_id, - vdata_type, - edata_type, - oid_type, - vid_type, - ) - graph = Graph(self._session, op) - graph._base_graph = self - return graph + return self._session._wrapper(self._graph_node._project_to_simple()) def add_column(self, results, selector): """Add the results as a column to the graph. Modification rules are given by the selector. @@ -427,23 +720,7 @@ def add_column(self, results, selector): Returns: :class:`Graph`: A new `Graph` with new columns. """ - self._ensure_loaded() - check_argument( - isinstance(selector, Mapping), "selector of add column must be a dict" - ) - check_argument(self.graph_type == graph_def_pb2.ARROW_PROPERTY) - self._check_unmodified() - selector = { - key: results._transform_selector(value) for key, value in selector.items() - } - selector = json.dumps(selector) - op = dag_utils.add_column(self, results, selector) - graph = Graph(self._session, op) - graph._base_graph = self - # We trigger the load manually here to let the error raises in - # this specific method (if any) - graph._ensure_loaded() - return graph + return self._session._wrapper(self._graph_node.add_column(results, selector)) def to_numpy(self, selector, vertex_range=None): """Select some elements of the graph and output to numpy. @@ -455,13 +732,14 @@ def to_numpy(self, selector, vertex_range=None): `numpy.ndarray` """ check_argument(self.graph_type == graph_def_pb2.ARROW_PROPERTY) - self._ensure_loaded() self._check_unmodified() - selector = utils.transform_labeled_vertex_property_data_selector(self, selector) + selector = utils.transform_labeled_vertex_property_data_selector( + self.schema, selector + ) vertex_range = utils.transform_vertex_range(vertex_range) op = dag_utils.graph_to_numpy(self, selector, vertex_range) ret = op.eval() - return utils.decode_numpy(ret) + return ret def to_dataframe(self, selector, vertex_range=None): """Select some elements of the graph and output as a pandas.DataFrame @@ -474,14 +752,15 @@ def to_dataframe(self, selector, vertex_range=None): `pandas.DataFrame` """ check_argument(self.graph_type == graph_def_pb2.ARROW_PROPERTY) - self._ensure_loaded() self._check_unmodified() check_argument( isinstance(selector, Mapping), "selector of to_vineyard_dataframe must be a dict", ) selector = { - key: utils.transform_labeled_vertex_property_data_selector(self, value) + key: utils.transform_labeled_vertex_property_data_selector( + self.schema, value + ) for key, value in selector.items() } selector = json.dumps(selector) @@ -489,95 +768,16 @@ def to_dataframe(self, selector, vertex_range=None): op = dag_utils.graph_to_dataframe(self, selector, vertex_range) ret = op.eval() - return utils.decode_dataframe(ret) + return ret def is_directed(self): - self._ensure_loaded() return self._directed def _check_unmodified(self): - self._ensure_loaded() check_argument( self.signature == self._saved_signature, "Graph has been modified!" ) - def _from_nx_graph(self, incoming_graph): - """Create a gs graph from a nx graph. - Args: - incoming_graph (:class:`nx.graph`): A nx graph that contains graph data. - - Returns: - that will be used to construct a gs.Graph - - Raises: - TypeError: Raise Error if graph type not match. - - Examples: - >>> nx_g = nx.path_graph(10) - >>> gs_g = gs.Graph(nx_g) - """ - if hasattr(incoming_graph, "_graph"): - msg = "graph view can not convert to gs graph" - raise TypeError(msg) - return dag_utils.dynamic_to_arrow(incoming_graph) - - def _copy_from(self, incoming_graph): - """Copy a graph. - - Args: - incoming_graph (:class:`Graph`): Source graph to be copied from - - Returns: - :class:`Graph`: An identical graph, but with a new vineyard id. - """ - check_argument(incoming_graph.graph_type == graph_def_pb2.ARROW_PROPERTY) - check_argument(incoming_graph.loaded()) - return dag_utils.copy_graph(incoming_graph) - - def _from_vineyard(self, vineyard_object): - """Load a graph from a already existed vineyard graph. - - Args: - vineyard_object (:class:`vineyard.Object`, :class:`vineyard.ObjectID` - or :class:`vineyard.ObjectName`): vineyard object, - which represents a graph. - - Returns: - A graph_def. - """ - if isinstance(vineyard_object, vineyard.Object): - return self._from_vineyard_id(vineyard_object.id) - if isinstance(vineyard_object, vineyard.ObjectID): - return self._from_vineyard_id(vineyard_object) - if isinstance(vineyard_object, vineyard.ObjectName): - return self._from_vineyard_name(vineyard_object) - - def _from_vineyard_id(self, vineyard_id): - config = {} - config[types_pb2.IS_FROM_VINEYARD_ID] = utils.b_to_attr(True) - config[types_pb2.VINEYARD_ID] = utils.i_to_attr(int(vineyard_id)) - # FIXME(hetao) hardcode oid/vid type for codegen, when loading from vineyard - # - # the metadata should be retrived from vineyard - config[types_pb2.OID_TYPE] = utils.s_to_attr("int64_t") - config[types_pb2.VID_TYPE] = utils.s_to_attr("uint64_t") - return dag_utils.create_graph( - self.session_id, graph_def_pb2.ARROW_PROPERTY, attrs=config - ) - - def _from_vineyard_name(self, vineyard_name): - config = {} - config[types_pb2.IS_FROM_VINEYARD_ID] = utils.b_to_attr(True) - config[types_pb2.VINEYARD_NAME] = utils.s_to_attr(str(vineyard_name)) - # FIXME(hetao) hardcode oid/vid type for codegen, when loading from vineyard - # - # the metadata should be retrived from vineyard - config[types_pb2.OID_TYPE] = utils.s_to_attr("int64_t") - config[types_pb2.VID_TYPE] = utils.s_to_attr("uint64_t") - return dag_utils.create_graph( - self.session_id, graph_def_pb2.ARROW_PROPERTY, attrs=config - ) - def _attach_interactive_instance(self, instance): """Store the instance when a new interactive instance is started. @@ -607,7 +807,6 @@ def save_to(self, path, **kwargs): import vineyard import vineyard.io - self._ensure_loaded() sess = self._session deployment = "kubernetes" if sess.info["type"] == "k8s" else "ssh" conf = sess.info["engine_config"] @@ -670,95 +869,13 @@ def load_from(cls, path, sess, **kwargs): deployment=deployment, hosts=hosts, ) - return cls(sess, vineyard.ObjectID(graph_id)) - - def _construct_op(self, vertices, edges, is_from_existed_graph): - config = graph_utils.assemble_op_config( - vertices.values(), - edges.values(), - self._oid_type, - self._directed, - self._generate_eid, - ) - if is_from_existed_graph: - op = dag_utils.add_labels_to_graph(self, attrs=config) - else: - op = dag_utils.create_graph( - self.session_id, graph_def_pb2.ARROW_PROPERTY, attrs=config - ) - return op - - def _construct_graph(self, vertices, edges, v_labels, e_labels, e_relations): - """Construct graph. - 1. Construct a graph from scratch. - If the vertices and edges is empty, return a empty graph. - 2. Construct a graph from existed builded graph. - If the vertices and edges is empty, return a copied graph. - - Args: - vertices (dict[str, VertexLabel]): VertexLabels - edges (dict[str, EdgeLabel]): EdgeLabels - v_labels (list[str]): Names of vertex labels - e_labels (list[str]): Names of edge labels - e_relations (list[list[str]]): Relationships of edges - - Returns: - Graph: Constructed new graph. not yet evaluated. - """ - - is_from_existed_graph = self._key is not None - # edge case. - if not vertices and not edges: - if is_from_existed_graph: - # Rely on `self._key` - return Graph(self._session, self) - else: - return Graph( - self._session, - None, - self._oid_type, - self._directed, - self._generate_eid, - ) - - op = self._construct_op(vertices, edges, is_from_existed_graph) - - graph = Graph( - self._session, op, self._oid_type, self._directed, self._generate_eid - ) - graph._unsealed_vertices = vertices - graph._unsealed_edges = edges - graph._v_labels = v_labels - graph._e_labels = e_labels - graph._e_relationships = e_relations - # propage info about whether is a loaded graph. - graph._key = self._key - if is_from_existed_graph: - graph._base_graph = self._base_graph or self - return graph + return sess._wrapper(GraphDAGNode(sess, vineyard.ObjectID(graph_id))) def add_vertices(self, vertices, label="_", properties=None, vid_field=0): - if label in self._v_labels: - raise ValueError(f"Label {label} already existed in graph.") - if not self._v_labels and self._e_labels: - raise ValueError("Cannot manually add vertices after inferred vertices.") - unsealed_vertices = deepcopy(self._unsealed_vertices) - unsealed_vertices[label] = VertexLabel( - label=label, - loader=vertices, - properties=properties, - vid_field=vid_field, - session_id=self._session.session_id, - ) - v_labels = deepcopy(self._v_labels) - v_labels.append(label) - - return self._construct_graph( - unsealed_vertices, - self._unsealed_edges, - v_labels, - self._e_labels, - self._e_relationships, + if not self.loaded(): + raise RuntimeError("The graph is not loaded") + return self._session._wrapper( + self._graph_node.add_vertices(vertices, label, properties, vid_field) ) def add_edges( @@ -803,58 +920,12 @@ def add_edges( Returns: Graph: A new graph with edge added, not yet evaluated. """ - if src_label is None and dst_label is None: - check_argument( - len(self._v_labels) <= 1, - "Ambiguous vertex label, please specify the src_label and dst_label.", - ) - if len(self._v_labels) == 1: - src_label = dst_label = self._v_labels[0] - else: - src_label = dst_label = "_" - - if src_label is None or dst_label is None: - raise ValueError( - "src and dst label must be both specified or either unspecified." + if not self.loaded(): + raise RuntimeError("The graph is not loaded") + return self._session._wrapper( + self._graph_node.add_edges( + edges, label, properties, src_label, dst_label, src_field, dst_field ) - - if self._v_labels: - if src_label not in self._v_labels or dst_label not in self._v_labels: - raise ValueError("src label or dst_label not existed in graph.") - else: - # We can infer all vertices label in the graph constructing stage. - pass - - check_argument( - src_field != dst_field, "src and dst field cannot refer to the same field" - ) - - is_from_existed_graph = self._key is not None - - if is_from_existed_graph: - if label in self._e_labels and label not in self._unsealed_edges: - raise ValueError("Cannot add new relation to existed graph.") - - unsealed_edges = deepcopy(self._unsealed_edges) - e_labels = deepcopy(self._e_labels) - relations = deepcopy(self._e_relationships) - if label in unsealed_edges: - assert label in self._e_labels - label_idx = self._e_labels.index(label) - # Will check conflict in `add_sub_label` - relations[label_idx].append((src_label, dst_label)) - cur_label = unsealed_edges[label] - else: - e_labels.append(label) - relations.append([(src_label, dst_label)]) - cur_label = EdgeLabel(label, self._session.session_id) - cur_label.add_sub_label( - EdgeSubLabel(edges, properties, src_label, dst_label, src_field, dst_field) - ) - unsealed_edges[label] = cur_label - - return self._construct_graph( - self._unsealed_vertices, unsealed_edges, self._v_labels, e_labels, relations ) def project( @@ -862,58 +933,6 @@ def project( vertices: Mapping[str, Union[List[str], None]], edges: Mapping[str, Union[List[str], None]], ): - check_argument(self.graph_type == graph_def_pb2.ARROW_PROPERTY) - - def get_all_v_props_id(label) -> List[int]: - props = self.schema.get_vertex_properties(label) - return [ - self.schema.get_vertex_property_id(label, prop.name) for prop in props - ] - - def get_all_e_props_id(label) -> List[int]: - props = self.schema.get_edge_properties(label) - return [ - self.schema.get_edge_property_id(label, prop.name) for prop in props - ] - - vertex_collections = {} - edge_collections = {} - - for label, props in vertices.items(): - label_id = self.schema.get_vertex_label_id(label) - if props is None: - vertex_collections[label_id] = get_all_v_props_id(label) - else: - vertex_collections[label_id] = sorted( - [self.schema.get_vertex_property_id(label, prop) for prop in props] - ) - for label, props in edges.items(): - # find whether exist a valid relation - relations = self.schema.get_relationships(label) - valid = False - for src, dst in relations: - if src in vertices and dst in vertices: - valid = True - break - if not valid: - raise ValueError( - "Cannot find a valid relation in given vertices and edges" - ) - - label_id = self.schema.get_edge_label_id(label) - if props is None: - edge_collections[label_id] = get_all_e_props_id(label) - else: - edge_collections[label_id] = sorted( - [self.schema.get_edge_property_id(label, prop) for prop in props] - ) - - vertex_collections = dict(sorted(vertex_collections.items())) - edge_collections = dict(sorted(edge_collections.items())) - - op = dag_utils.project_arrow_property_graph( - self, vertex_collections, edge_collections - ) - graph = Graph(self._session, op) - graph._base_graph = self - return graph + if not self.loaded(): + raise RuntimeError("The graph is not loaded") + return self._session._wrapper(self._graph_node.project(vertices, edges)) diff --git a/python/graphscope/framework/operation.py b/python/graphscope/framework/operation.py index 0f97566d9490..81221db42964 100644 --- a/python/graphscope/framework/operation.py +++ b/python/graphscope/framework/operation.py @@ -64,78 +64,86 @@ def __init__( Values that used as query parameters when evaluating app. Raises: - TypeError: if `op_def` is not a `OpDef`, or if `g` is not a `Dag`. - ValueError: if the `op_def` name is not valid. + TypeError: value in inputs is not a :class:`Operation` """ self._session_id = session_id self._op_def = op_def_pb2.OpDef(op=op_type, key=uuid.uuid4().hex) + self._parents = list() if config: for k, v in config.items(): self._op_def.attr[k].CopyFrom(v) - if query_args is not None: self._op_def.query_args.CopyFrom(query_args) - + if inputs: + for op in inputs: + if not isinstance(op, Operation): + raise TypeError("Input op must be an Operation: {0}".format(op)) + self.add_parent(op) self._output_types = output_types - - self._signature = None - self._output = None # hold the executed result of the DAG. - - def as_op_def(self): - return self._op_def + self._evaluated = False + self._leaf = False @property def key(self): + """Unique key for each :code:`types_pb2.OpDef`""" return self._op_def.key + @property + def parents(self): + return self._parents + + @property + def evaluated(self): + return self._evaluated + + @evaluated.setter + def evaluated(self, value): + self._evaluated = bool(value) + + @property + def type(self): + return self._op_def.op + + @property + def output_types(self): + return self._output_types + @property def signature(self): """Signature of its parents' signatures and its own parameters. Used to unique identify one `Operation` with fixed configuration, if the configuration changed, the signature will be changed accordingly. """ - content = str(self.as_op_def()) + content = "" + for op in self._parents: + content += str(op.as_op_def) + content += str(self.as_op_def()) return hashlib.sha224(content.encode()).hexdigest() - def eval(self): + def is_leaf_op(self): + return self._leaf + + def eval(self, leaf=True): # NB: to void cycle import # pylint: disable=import-outside-toplevel, cyclic-import from graphscope.client.session import get_session_by_id + self._leaf = leaf sess = get_session_by_id(self._session_id) + if not self._leaf: + sess.dag.add_op(self) res = sess.run(self) return res - def set_output(self, output): - """Set Operation's output value. - Args: - output: The output after evaluated the op - - Raises: - RuntimeError: If the output is already be set before, since one op can only be evaluated once. - """ - if self._output is not None: - raise RuntimeError("The executed value of a DAG node is already set") - self._output = output - - @property - def evaluated(self): - return self._output is not None + def generate_new_key(self): + self._op_def.key = uuid.uuid4().hex - @property - def output(self): - """Executed result of the DAG node. - Returns None if the DAG node hasn't been evaluated yet, otherwise a `dict` from `JSON` object. - """ - return self._output + def add_parent(self, op): + self._parents.append(op) + self._op_def.parents.extend([op.key]) - @property - def type(self): - return self._op_def.op - - @property - def output_types(self): - return self._output_types + def as_op_def(self): + return self._op_def def __str__(self): return str(self.as_op_def()) diff --git a/python/graphscope/framework/utils.py b/python/graphscope/framework/utils.py index 46caec2248e3..83c9ea930c3a 100644 --- a/python/graphscope/framework/utils.py +++ b/python/graphscope/framework/utils.py @@ -89,6 +89,10 @@ def report_type_to_attr(t): return attr_value_pb2.AttrValue(report_type=t) +def place_holder_to_attr(): + return attr_value_pb2.AttrValue(place_holder=types_pb2.PlaceHolder()) + + def list_str_to_attr(list_of_str): attr = attr_value_pb2.AttrValue() attr.list.s[:] = [ @@ -413,7 +417,7 @@ def transform_vertex_property_data_selector(selector): return selector -def transform_labeled_vertex_data_selector(graph, selector): +def transform_labeled_vertex_data_selector(schema, selector): """Formats: 'v:x.y/id', 'e:x.y/src/dst', 'r:label', x denotes label name, y denotes property name. Returned selector will change label name to 'label{id}', where id is x's id in labels. @@ -421,7 +425,7 @@ def transform_labeled_vertex_data_selector(graph, selector): """ if selector is None: raise RuntimeError("selector cannot be None") - schema = graph.schema + ret_type, segments = selector.split(":") if ret_type not in ("v", "e", "r"): raise SyntaxError("Invalid selector: " + selector) @@ -436,7 +440,7 @@ def transform_labeled_vertex_data_selector(graph, selector): return "{}:{}".format(ret_type, ret) -def transform_labeled_vertex_property_data_selector(graph, selector): +def transform_labeled_vertex_property_data_selector(schema, selector): """Formats: 'v:x.y/id', 'e:x.y/src/dst', 'r:x.y', x denotes label name, y denotes property name. Returned selector will change label name to 'label{id}', where id is x's id in labels. @@ -444,7 +448,6 @@ def transform_labeled_vertex_property_data_selector(graph, selector): """ if selector is None: raise RuntimeError("selector cannot be None") - schema = graph.schema ret_type, segments = selector.split(":") if ret_type not in ("v", "e", "r"): raise SyntaxError("Invalid selector: " + selector) diff --git a/python/graphscope/interactive/query.py b/python/graphscope/interactive/query.py index c9bb5f6c3fd7..af08c80caf95 100644 --- a/python/graphscope/interactive/query.py +++ b/python/graphscope/interactive/query.py @@ -139,16 +139,11 @@ def subgraph(self, gremlin_script): def load_subgraph(name): import vineyard - import graphscope - - graph = self._session.g(generate_eid=False) - graph = graph.add_vertices( - Loader(vineyard.ObjectName("__%s_vertex_stream" % name)) - ) - graph = graph.add_edges( - Loader(vineyard.ObjectName("__%s_edge_stream" % name)) + graph = self._session.load_from( + vertices=[Loader(vineyard.ObjectName("__%s_vertex_stream" % name))], + edges=[Loader(vineyard.ObjectName("__%s_edge_stream" % name))], + generate_eid=False, ) - graph._ensure_loaded() logger.info("subgraph has been loaded") return graph diff --git a/python/graphscope/nx/classes/graph.py b/python/graphscope/nx/classes/graph.py index f92fbe8296a2..edd99d504816 100644 --- a/python/graphscope/nx/classes/graph.py +++ b/python/graphscope/nx/classes/graph.py @@ -315,6 +315,12 @@ def op(self): """The DAG op of this graph.""" return self._op + @property + def session(self): + from graphscope.client.session import get_session_by_id + + return get_session_by_id(self._session_id) + @property def session_id(self): """Get the currrent session_id. @@ -1718,6 +1724,7 @@ def edge_subgraph(self, edges): g._key = graph_def.key g._session_id = self._session_id g._schema = copy.deepcopy(self._schema) + g._op = op return g def _is_view(self): @@ -1785,7 +1792,8 @@ def _get_nbrs(self, n, report_type=types_pb2.SUCCS_BY_NODE): if n not in self: raise NetworkXError("The node %s is not in the graph." % (n,)) op = dag_utils.report_graph(self, report_type, node=json.dumps([n])) - return op.eval() + ret = op.eval() + return ret def _batch_get_nbrs(self, location, report_type=types_pb2.SUCCS_BY_LOC): """Get neighbors of nodes by location in batch. @@ -1947,7 +1955,7 @@ def _project_to_simple(self, v_prop=None, e_prop=None): op = dag_utils.project_dynamic_property_graph( self, v_prop, e_prop, v_prop_type, e_prop_type ) - graph_def = op.eval() + graph_def = op.eval(leaf=False) graph = self.__class__(create_empty_in_engine=False) graph = nx.freeze(graph) graph._graph_type = graph_def_pb2.DYNAMIC_PROJECTED @@ -1956,5 +1964,6 @@ def _project_to_simple(self, v_prop=None, e_prop=None): graph.schema.from_graph_def(graph_def) graph._saved_signature = self._saved_signature graph._graph = self # projected graph also can report nodes. + graph._op = op graph._is_client_view = False return graph diff --git a/python/graphscope/nx/utils/other.py b/python/graphscope/nx/utils/other.py index e78af76e6322..2c2a78a7ee99 100644 --- a/python/graphscope/nx/utils/other.py +++ b/python/graphscope/nx/utils/other.py @@ -41,7 +41,7 @@ def empty_graph_in_engine(graph, directed, distributed): efile="", vfile="", ) - graph_def = sess.run(op) + graph_def = op.eval() return graph_def diff --git a/python/tests/conftest.py b/python/tests/conftest.py index c2c4814f1cf0..ee7ed42d9176 100644 --- a/python/tests/conftest.py +++ b/python/tests/conftest.py @@ -137,38 +137,232 @@ def twitter_e_1_1_1(): @pytest.fixture(scope="module") def arrow_property_graph(graphscope_session): - g = graphscope_session.g(generate_eid=False) - g = g.add_vertices(f"{new_property_dir}/twitter_v_0", "v0") - g = g.add_vertices(f"{new_property_dir}/twitter_v_1", "v1") - g = g.add_edges(f"{new_property_dir}/twitter_e_0_0_0", "e0", ["weight"], "v0", "v0") - g = g.add_edges(f"{new_property_dir}/twitter_e_0_1_0", "e0", ["weight"], "v0", "v1") - g = g.add_edges(f"{new_property_dir}/twitter_e_1_0_0", "e0", ["weight"], "v1", "v0") - g = g.add_edges(f"{new_property_dir}/twitter_e_1_1_0", "e0", ["weight"], "v1", "v1") - g = g.add_edges(f"{new_property_dir}/twitter_e_0_0_1", "e1", ["weight"], "v0", "v0") - g = g.add_edges(f"{new_property_dir}/twitter_e_0_1_1", "e1", ["weight"], "v0", "v1") - g = g.add_edges(f"{new_property_dir}/twitter_e_1_0_1", "e1", ["weight"], "v1", "v0") - g = g.add_edges(f"{new_property_dir}/twitter_e_1_1_1", "e1", ["weight"], "v1", "v1") - + g = graphscope_session.load_from( + edges={ + "e0": [ + ( + Loader( + "{}/twitter_e_0_0_0".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v0"), + ("dst", "v0"), + ), + ( + Loader( + "{}/twitter_e_0_1_0".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v0"), + ("dst", "v1"), + ), + ( + Loader( + "{}/twitter_e_1_0_0".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v1"), + ("dst", "v0"), + ), + ( + Loader( + "{}/twitter_e_1_1_0".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v1"), + ("dst", "v1"), + ), + ], + "e1": [ + ( + Loader( + "{}/twitter_e_0_0_1".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v0"), + ("dst", "v0"), + ), + ( + Loader( + "{}/twitter_e_0_1_1".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v0"), + ("dst", "v1"), + ), + ( + Loader( + "{}/twitter_e_1_0_1".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v1"), + ("dst", "v0"), + ), + ( + Loader( + "{}/twitter_e_1_1_1".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v1"), + ("dst", "v1"), + ), + ], + }, + vertices={ + "v0": Loader("{}/twitter_v_0".format(new_property_dir), header_row=True), + "v1": Loader("{}/twitter_v_1".format(new_property_dir), header_row=True), + }, + generate_eid=False, + ) yield g g.unload() @pytest.fixture(scope="module") def arrow_property_graph_only_from_efile(graphscope_session): - g = graphscope_session.g(generate_eid=False) - g = g.add_edges(f"{new_property_dir}/twitter_e_0_0_0", "e0", ["weight"], "v0", "v0") - g = g.add_edges(f"{new_property_dir}/twitter_e_0_1_0", "e0", ["weight"], "v0", "v1") - g = g.add_edges(f"{new_property_dir}/twitter_e_1_0_0", "e0", ["weight"], "v1", "v0") - g = g.add_edges(f"{new_property_dir}/twitter_e_1_1_0", "e0", ["weight"], "v1", "v1") - g = g.add_edges(f"{new_property_dir}/twitter_e_0_0_1", "e1", ["weight"], "v0", "v0") - g = g.add_edges(f"{new_property_dir}/twitter_e_0_1_1", "e1", ["weight"], "v0", "v1") - g = g.add_edges(f"{new_property_dir}/twitter_e_1_0_1", "e1", ["weight"], "v1", "v0") - g = g.add_edges(f"{new_property_dir}/twitter_e_1_1_1", "e1", ["weight"], "v1", "v1") - + g = graphscope_session.load_from( + edges={ + "e0": [ + ( + Loader( + "{}/twitter_e_0_0_0".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v0"), + ("dst", "v0"), + ), + ( + Loader( + "{}/twitter_e_0_1_0".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v0"), + ("dst", "v1"), + ), + ( + Loader( + "{}/twitter_e_1_0_0".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v1"), + ("dst", "v0"), + ), + ( + Loader( + "{}/twitter_e_1_1_0".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v1"), + ("dst", "v1"), + ), + ], + "e1": [ + ( + Loader( + "{}/twitter_e_0_0_1".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v0"), + ("dst", "v0"), + ), + ( + Loader( + "{}/twitter_e_0_1_1".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v0"), + ("dst", "v1"), + ), + ( + Loader( + "{}/twitter_e_1_0_1".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v1"), + ("dst", "v0"), + ), + ( + Loader( + "{}/twitter_e_1_1_1".format(new_property_dir), + header_row=True, + delimiter=",", + ), + ["weight"], + ("src", "v1"), + ("dst", "v1"), + ), + ], + }, + generate_eid=False, + ) yield g g.unload() +# @pytest.fixture(scope="module") +# def arrow_property_graph(graphscope_session): +# g = graphscope_session.g(generate_eid=False) +# g = g.add_vertices(f"{new_property_dir}/twitter_v_0", "v0") +# g = g.add_vertices(f"{new_property_dir}/twitter_v_1", "v1") +# g = g.add_edges(f"{new_property_dir}/twitter_e_0_0_0", "e0", ["weight"], "v0", "v0") +# g = g.add_edges(f"{new_property_dir}/twitter_e_0_1_0", "e0", ["weight"], "v0", "v1") +# g = g.add_edges(f"{new_property_dir}/twitter_e_1_0_0", "e0", ["weight"], "v1", "v0") +# g = g.add_edges(f"{new_property_dir}/twitter_e_1_1_0", "e0", ["weight"], "v1", "v1") +# g = g.add_edges(f"{new_property_dir}/twitter_e_0_0_1", "e1", ["weight"], "v0", "v0") +# g = g.add_edges(f"{new_property_dir}/twitter_e_0_1_1", "e1", ["weight"], "v0", "v1") +# g = g.add_edges(f"{new_property_dir}/twitter_e_1_0_1", "e1", ["weight"], "v1", "v0") +# g = g.add_edges(f"{new_property_dir}/twitter_e_1_1_1", "e1", ["weight"], "v1", "v1") + +# yield g +# g.unload() + + +# @pytest.fixture(scope="module") +# def arrow_property_graph_only_from_efile(graphscope_session): +# g = graphscope_session.g(generate_eid=False) +# g = g.add_edges(f"{new_property_dir}/twitter_e_0_0_0", "e0", ["weight"], "v0", "v0") +# g = g.add_edges(f"{new_property_dir}/twitter_e_0_1_0", "e0", ["weight"], "v0", "v1") +# g = g.add_edges(f"{new_property_dir}/twitter_e_1_0_0", "e0", ["weight"], "v1", "v0") +# g = g.add_edges(f"{new_property_dir}/twitter_e_1_1_0", "e0", ["weight"], "v1", "v1") +# g = g.add_edges(f"{new_property_dir}/twitter_e_0_0_1", "e1", ["weight"], "v0", "v0") +# g = g.add_edges(f"{new_property_dir}/twitter_e_0_1_1", "e1", ["weight"], "v0", "v1") +# g = g.add_edges(f"{new_property_dir}/twitter_e_1_0_1", "e1", ["weight"], "v1", "v0") +# g = g.add_edges(f"{new_property_dir}/twitter_e_1_1_1", "e1", ["weight"], "v1", "v1") + +# yield g +# g.unload() + + @pytest.fixture(scope="module") def arrow_property_graph_undirected(graphscope_session): g = graphscope_session.g(directed=False, generate_eid=False) diff --git a/python/tests/test_create_graph.py b/python/tests/test_create_graph.py index 8f4b84d3b7e3..85fd30f77e3b 100644 --- a/python/tests/test_create_graph.py +++ b/python/tests/test_create_graph.py @@ -257,6 +257,7 @@ def test_loader_with_specified_data_type( ] +@pytest.mark.skip(reason="waiting for vineyard support") def test_multi_src_dst_edge_loader( graphscope_session, student_group_e, teacher_group_e, student_v, teacher_v ): @@ -321,6 +322,7 @@ def test_v_property_omitted_form_loader(graphscope_session, student_group_e, stu assert graph.loaded() +@pytest.mark.skip(reason="vertex must be exist before add edges in eager mode.") def test_vertices_omitted_form_loader(graphscope_session, student_group_e): # vertices can be omit. graph = graphscope_session.g() @@ -328,12 +330,14 @@ def test_vertices_omitted_form_loader(graphscope_session, student_group_e): assert graph.loaded() +@pytest.mark.skip(reason="vertex must be exist before add edges in eager mode.") def test_all_omitted_form_loader(graphscope_session, student_group_e): graph = graphscope_session.g() graph = graph.add_edges(student_group_e, "group") assert graph.loaded() +@pytest.mark.skip(reason="vertex must be exist before add edges in eager mode.") def test_multiple_e_all_omitted_form_loader( graphscope_session, student_group_e, friend_e ): @@ -362,9 +366,7 @@ def test_errors_on_files( graphscope_session, one_column_file, double_type_id_file, empty_file ): with pytest.raises(AnalyticalEngineInternalError, match="Object not exists"): - Graph( - graphscope_session, vineyard.ObjectName("non_exist_vy_name") - )._ensure_loaded() + graphscope_session.g(vineyard.ObjectName("non_exist_vy_name")) return graph = graphscope_session.g() with pytest.raises(AnalyticalEngineInternalError, match="IOError"): diff --git a/python/tests/test_graph.py b/python/tests/test_graph.py index f11e4ca90fc9..a50786dd1b37 100644 --- a/python/tests/test_graph.py +++ b/python/tests/test_graph.py @@ -28,6 +28,7 @@ from graphscope import sssp from graphscope.dataset.ldbc import load_ldbc from graphscope.framework.errors import AnalyticalEngineInternalError +from graphscope.framework.errors import GRPCError from graphscope.framework.errors import InvalidArgumentError from graphscope.framework.graph import Graph from graphscope.framework.loader import Loader @@ -153,9 +154,9 @@ def test_error_on_project_to_simple_wrong_graph_type(arrow_property_graph): sg = arrow_property_graph.project(vertices={"v0": []}, edges={"e0": []}) pg = sg._project_to_simple() assert pg._graph_type == graph_def_pb2.ARROW_PROJECTED - with pytest.raises(AssertionError): + with pytest.raises(InvalidArgumentError): pg._project_to_simple() - with pytest.raises(AssertionError): + with pytest.raises(InvalidArgumentError): pg.project(vertices={"v0": []}, edges={"e0": []}) @@ -323,7 +324,7 @@ def test_add_vertices_edges(graphscope_session): with pytest.raises(ValueError, match="Ambiguous vertex label"): graph = graph.add_edges(Loader(f"{prefix}/knows.csv", delimiter="|"), "created") - with pytest.raises(ValueError, match="Cannot add new relation to existed graph"): + with pytest.raises(ValueError, match="already existed in graph"): graph = graph.add_edges( Loader(f"{prefix}/knows.csv", delimiter="|"), "knows", @@ -471,7 +472,7 @@ def test_project_subgraph(arrow_modern_graph): assert sub_graph.schema.vertex_labels == [] assert sub_graph.schema.edge_labels == [] with pytest.raises( - InvalidArgumentError, + ValueError, match="Check failed: Cannot project to simple, vertex label number is not one.", ): graphscope.wcc(sub_graph) @@ -481,7 +482,7 @@ def test_project_subgraph(arrow_modern_graph): assert sub_graph.schema.vertex_labels == ["person"] assert sub_graph.schema.edge_labels == [] with pytest.raises( - InvalidArgumentError, + ValueError, match="Check failed: Cannot project to simple, edge label number is not one.", ): graphscope.wcc(sub_graph) @@ -519,7 +520,7 @@ def test_project_subgraph(arrow_modern_graph): def test_error_on_project(arrow_property_graph, ldbc_graph): graph = arrow_property_graph - with pytest.raises(AssertionError, match="Cannot project to simple"): + with pytest.raises(ValueError, match="Cannot project to simple"): graphscope.sssp(graph, 4) g2 = graph.project(vertices={"v0": []}, edges={"e0": []}) assert g2.schema.edge_relationships == [[("v0", "v0")]] @@ -527,13 +528,15 @@ def test_error_on_project(arrow_property_graph, ldbc_graph): ldbc = ldbc_graph # vertices empty with pytest.raises( - ValueError, match="Cannot find a valid relation in given vertices and edges" + ValueError, + match="Cannot find a valid relation in given vertices and edges", ): sub_graph = ldbc.project(vertices={}, edges={"knows": None}) # project not related vertex and edge with pytest.raises( - ValueError, match="Cannot find a valid relation in given vertices and edges" + ValueError, + match="Cannot find a valid relation in given vertices and edges", ): sub_graph = ldbc.project(vertices={"person": None}, edges={"hasInterest": None}) @@ -554,14 +557,14 @@ def test_error_on_project(arrow_property_graph, ldbc_graph): sub_graph = ldbc.project( vertices={"person": ["id", "firstName"]}, edges={"knows": ["eid"]} ) - with pytest.raises(InvalidArgumentError): + with pytest.raises(ValueError): sub_graph._project_to_simple() # more than one property on edge can not project to simple sub_graph = ldbc.project( vertices={"person": ["id"]}, edges={"knows": ["eid", "creationDate"]} ) - with pytest.raises(InvalidArgumentError): + with pytest.raises(ValueError): sub_graph._project_to_simple() diff --git a/python/tests/test_lazy.py b/python/tests/test_lazy.py new file mode 100644 index 000000000000..5cfe26385a53 --- /dev/null +++ b/python/tests/test_lazy.py @@ -0,0 +1,158 @@ +#!/usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2020 Alibaba Group Holding Limited. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import importlib +import logging +import os +import random +import string +import sys +import time + +import numpy as np +import pytest + +import graphscope + +graphscope.set_option(show_log=True) + +from graphscope import property_sssp +from graphscope import sssp +from graphscope.framework.loader import Loader + +test_repo_dir = os.path.expandvars("${GS_TEST_DIR}") +prefix = os.path.join(test_repo_dir, "ogbn_mag_small") + +new_property_dir = os.path.join(test_repo_dir, "new_property", "v2_e2") + + +@pytest.fixture(scope="module") +def sess(): + session = graphscope.session(cluster_type="hosts", num_workers=2, mode="lazy") + session.as_default() + yield session + session.close() + + +@pytest.fixture +def student_v(data_dir=os.path.expandvars("${GS_TEST_DIR}/property_graph")): + return Loader("%s/student.v" % data_dir, header_row=True, delimiter=",") + + +@pytest.fixture +def teacher_v(data_dir=os.path.expandvars("${GS_TEST_DIR}/property_graph")): + return Loader("%s/teacher.v" % data_dir, header_row=True, delimiter=",") + + +@pytest.fixture +def student_group_e(data_dir=os.path.expandvars("${GS_TEST_DIR}/property_graph")): + return Loader("%s/group.e" % data_dir, header_row=True, delimiter=",") + + +@pytest.fixture +def teacher_group_e(data_dir=os.path.expandvars("${GS_TEST_DIR}/property_graph")): + return Loader("%s/teacher_group.e" % data_dir, header_row=True, delimiter=",") + + +def arrow_property_graph(graphscope_session): + g = graphscope_session.g(generate_eid=False) + g = g.add_vertices(f"{new_property_dir}/twitter_v_0", "v0") + g = g.add_vertices(f"{new_property_dir}/twitter_v_1", "v1") + g = g.add_edges(f"{new_property_dir}/twitter_e_0_0_0", "e0", ["weight"], "v0", "v0") + g = g.add_edges(f"{new_property_dir}/twitter_e_0_1_0", "e0", ["weight"], "v0", "v1") + g = g.add_edges(f"{new_property_dir}/twitter_e_1_0_0", "e0", ["weight"], "v1", "v0") + g = g.add_edges(f"{new_property_dir}/twitter_e_1_1_0", "e0", ["weight"], "v1", "v1") + g = g.add_edges(f"{new_property_dir}/twitter_e_0_0_1", "e1", ["weight"], "v0", "v0") + g = g.add_edges(f"{new_property_dir}/twitter_e_0_1_1", "e1", ["weight"], "v0", "v1") + g = g.add_edges(f"{new_property_dir}/twitter_e_1_0_1", "e1", ["weight"], "v1", "v0") + g = g.add_edges(f"{new_property_dir}/twitter_e_1_1_1", "e1", ["weight"], "v1", "v1") + return g + + +@pytest.mark.skip(reason="waiting for optimization of dag") +def test_vertices_omitted_form_loader(sess, student_group_e): + g = sess.g() + g1 = g.add_edges(student_group_e) + g2 = sess.run(g1) # g2 is a Graph instance + assert g2.loaded() + + +def test_context(sess): + g = arrow_property_graph(sess) + c = property_sssp(g, 20) + r1 = c.to_numpy("r:v0.dist_0") + r2 = c.to_dataframe({"id": "v:v0.id", "result": "r:v0.dist_0"}) + r3 = c.to_vineyard_tensor("v:v0.id") + r4 = c.to_vineyard_dataframe( + {"id": "v:v0.id", "data": "v:v0.dist", "result": "r:v0.dist_0"} + ) + r = sess.run([r1, r2, r3, r4]) + assert r[0].shape == (40521,) + assert r[1].shape == (40521, 2) + assert r[2] is not None + assert r[3] is not None + + +def test_query_after_project(sess): + g = arrow_property_graph(sess) + pg = g.project(vertices={"v0": ["id"]}, edges={"e0": ["weight"]}) + # property sssp on property graph + # sssp on simple graph + c = sssp(pg, 20) + r1 = c.to_dataframe({"node": "v.id", "r": "r"}) + r = sess.run(r1) + assert r.shape == (40521, 2) + + +def test_add_column(sess): + g = arrow_property_graph(sess) + pg = g.project(vertices={"v0": ["id"]}, edges={"e0": ["weight"]}) + c = sssp(pg, 20) + g1 = g.add_column(c, {"id_col": "v.id", "data_col": "v.data", "result_col": "r"}) + sess.run(g1) + + +def test_multi_src_dst_edge_loader( + sess, student_group_e, teacher_group_e, student_v, teacher_v +): + graph = sess.g() + graph = graph.add_vertices( + student_v, "student", ["name", "lesson_nums", "avg_score"], "student_id" + ) + graph = graph.add_vertices( + teacher_v, "teacher", ["student_num", "score", "email", "tel"], "teacher_id" + ) + graph = graph.add_edges( + student_group_e, + "group", + ["group_id", "member_size"], + src_label="student", + dst_label="student", + src_field="leader_student_id", + dst_field="member_student_id", + ) + graph = graph.add_edges( + teacher_group_e, + "group", + ["group_id", "member_size"], + src_label="teacher", + dst_label="teacher", + src_field="leader_teacher_id", + dst_field="member_teacher_id", + ) + g = sess.run(graph) diff --git a/python/tests/test_session.py b/python/tests/test_session.py index 58bd89ad1959..da29419d9292 100644 --- a/python/tests/test_session.py +++ b/python/tests/test_session.py @@ -187,9 +187,8 @@ def test_error_on_used_after_close(): s1 = graphscope.session(cluster_type="hosts") s1.close() - with pytest.raises(ValueError, match="Session not exists."): + with pytest.raises(RuntimeError, match="Attempted to use a closed Session."): g = load_graph(s1) - g._ensure_loaded() with pytest.raises(RuntimeError, match="No default session found."): g = graphscope.load_from( @@ -197,7 +196,6 @@ def test_error_on_used_after_close(): "e0": "twitter_property_e_0#header_row=true", } ) - g._ensure_loaded() # close after close s2 = graphscope.session(cluster_type="hosts") @@ -229,12 +227,10 @@ def test_border_cases(): "e0": "twitter_property_e_0#header_row=true", } ) - g._ensure_loaded() s1.as_default() assert graphscope.get_default_session() == s1 g3 = load_graph(s3) - g3._ensure_loaded() # g3 is op of s3 with pytest.raises( ValueError, @@ -259,4 +255,3 @@ def test_border_cases(): "e0": "twitter_property_e_0#header_row=true", } ) - g._ensure_loaded()