From a20a3a3e05b8b126417855130bfd1bcddf98d0b6 Mon Sep 17 00:00:00 2001 From: Dongze Li Date: Fri, 27 Oct 2023 17:51:24 +0800 Subject: [PATCH 1/4] FLEX: Refactor code both in gsctl and coordinator side --- coordinator/gscoordinator/coordinator.py | 2 + coordinator/gscoordinator/scheduler.py | 260 ++++++++ .../gscoordinator/servicer/__init__.py | 1 + .../gscoordinator/servicer/base_service.py | 45 ++ .../servicer/interactive/__init__.py | 26 + .../gscoordinator/servicer/interactive/job.py | 125 ++++ .../servicer/interactive/service.py | 449 +++++++++++++ coordinator/gscoordinator/stoppable_thread.py | 39 ++ coordinator/gscoordinator/utils.py | 20 + coordinator/requirements.txt | 2 + proto/coordinator_service.proto | 29 + proto/error_codes.proto | 3 + proto/interactive.proto | 283 ++++++++ python/graphscope/gsctl/client/rpc.py | 169 +++++ python/graphscope/gsctl/commands/__init__.py | 25 +- .../commands/{common_command.py => common.py} | 18 +- .../gsctl/commands/{dev_command.py => dev.py} | 0 .../graphscope/gsctl/commands/interactive.py | 625 ++++++++++++++++++ python/graphscope/gsctl/commands/utils.py | 40 -- python/graphscope/gsctl/config.py | 34 +- python/graphscope/gsctl/gsctl.py | 2 +- python/graphscope/gsctl/rpc.py | 87 --- python/graphscope/gsctl/utils.py | 87 +++ 23 files changed, 2216 insertions(+), 155 deletions(-) create mode 100644 coordinator/gscoordinator/scheduler.py create mode 100644 coordinator/gscoordinator/servicer/base_service.py create mode 100644 coordinator/gscoordinator/servicer/interactive/__init__.py create mode 100644 coordinator/gscoordinator/servicer/interactive/job.py create mode 100644 coordinator/gscoordinator/servicer/interactive/service.py create mode 100644 coordinator/gscoordinator/stoppable_thread.py create mode 100644 proto/interactive.proto create mode 100644 python/graphscope/gsctl/client/rpc.py rename python/graphscope/gsctl/commands/{common_command.py => common.py} (86%) rename python/graphscope/gsctl/commands/{dev_command.py => dev.py} (100%) create mode 100644 python/graphscope/gsctl/commands/interactive.py delete mode 100644 python/graphscope/gsctl/commands/utils.py delete mode 100644 python/graphscope/gsctl/rpc.py create mode 100644 python/graphscope/gsctl/utils.py diff --git a/coordinator/gscoordinator/coordinator.py b/coordinator/gscoordinator/coordinator.py index 90e855a11808..adf154c9a8e6 100644 --- a/coordinator/gscoordinator/coordinator.py +++ b/coordinator/gscoordinator/coordinator.py @@ -31,6 +31,7 @@ from graphscope.proto import coordinator_service_pb2_grpc from gscoordinator.monitor import Monitor +from gscoordinator.servicer import init_interactive_service_servicer from gscoordinator.servicer import init_graphscope_one_service_servicer from gscoordinator.utils import GS_GRPC_MAX_MESSAGE_LENGTH @@ -110,6 +111,7 @@ def get_servicer(config: Config): """Get servicer of specified solution under FLEX architecture""" service_initializers = { "GraphScope One": init_graphscope_one_service_servicer, + "Interactive": init_interactive_service_servicer, } initializer = service_initializers.get(config.solution) diff --git a/coordinator/gscoordinator/scheduler.py b/coordinator/gscoordinator/scheduler.py new file mode 100644 index 000000000000..86328c95f9bf --- /dev/null +++ b/coordinator/gscoordinator/scheduler.py @@ -0,0 +1,260 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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 datetime +import json +import time +from abc import ABCMeta +from abc import abstractmethod + +import schedule +from schedule import CancelJob + +from gscoordinator.stoppable_thread import StoppableThread +from gscoordinator.utils import decode_datetimestr + + +class Schedule(object): + """Schedule class that wrapper dbader schedule + + Repo: https://github.com/dbader/schedule. + """ + + def __init__(self): + self._schedule = schedule.Scheduler() + self._run_pending_thread = StoppableThread(target=self.run_pending, args=()) + self._run_pending_thread.daemon = True + self._run_pending_thread.start() + + @property + def schedule(self): + return self._schedule + + def run_pending(self): + """Run all jobs that are scheduled to run.""" + while True: + try: + self._schedule.run_pending() + time.sleep(1) + except: # noqa: E722 + pass + + +schedule = Schedule().schedule # noqa: F811 + + +class Scheduler(metaclass=ABCMeta): + """ + Objects instantiated by the :class:`Scheduler ` are + factories to create jobs, keep record of scheduled jobs and + handle their execution in the :method:`run` method. + """ + + def __init__(self, at_time, repeat): + # scheduler id + self._scheduler_id = "Job-scheduler-{0}".format( + datetime.datetime.now().strftime("%Y-%m-%d-%H-%M-%S") + ) + # periodic job as used + self._job = None + # true will be run immediately + self._run_now = False + # time at which this job to schedule + self._at_time = self._decode_datetimestr(at_time) + # repeat every day or week, or run job once(no repeat) + # optional value "day", "week", "null" + self._repeat = repeat + # job running thread, note that: + # the last job should be end of execution at the beginning of the next job + self._running_thread = None + # tags + self._tags = [] + + # when the job actually scheduled, the following variables will be generated and overridden. + self._jobid = None + self._last_run = None + + def _decode_datetimestr(self, datetime_str): + if datetime_str == "now": + self._run_now = True + return datetime.datetime.now() + return decode_datetimestr(datetime_str) + + def __str__(self): + return "Scheduler(at_time={}, repeat={})".format(self._at_time, self._repeat) + + @property + def monday(self): + return self._at_time.weekday() == 0 + + @property + def tuesday(self): + return self._at_time.weekday() == 1 + + @property + def wednesday(self): + return self._at_time.weekday() == 2 + + @property + def thursday(self): + return self._at_time.weekday() == 3 + + @property + def friday(self): + return self._at_time.weekday() == 4 + + @property + def saturday(self): + return self._at_time.weekday() == 5 + + @property + def sunday(self): + return self._at_time.weekday() == 6 + + @property + def timestr(self): + """return str of the time object. + time([hour[, minute[, second[, microsecond[, tzinfo]]]]]) --> a time object + """ + return str(self._at_time.time()) + + @property + def job(self): + """A periodic job managed by the dbader scheduler. + https://github.com/dbader/schedule. + """ + return self._job + + @property + def jobid(self): + """id for the last scheduled job""" + return self._jobid + + @property + def schedulerid(self): + """id for the scheduler""" + return self._scheduler_id + + @property + def last_run(self): + """datetime of the last run""" + return self._last_run + + @property + def tags(self): + return self._tags + + @property + def running_thread(self): + return self._running_thread + + def run_once(self): + """Run the job immediately.""" + self.do_run() + return CancelJob + + def waiting_until_to_run(self): + """Run the job once at a specific time.""" + if datetime.datetime.now() >= self._at_time: + return self.run_once() + + def do_run(self): + """Start a thread for the job.""" + # overwrite for each scheduled job + self._jobid = "job-{0}".format( + datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%S") + ) + self._last_run = datetime.datetime.now() + # schedule in a thread + self._running_thread = StoppableThread(target=self.run, args=()) + self._running_thread.daemon = True + self._running_thread.start() + + def submit(self): + if not self._run_now and self._repeat not in ["week", "day", "null", None]: + raise RuntimeError( + "Submit schedule job failed: at_time is '{0}', repeat is '{1}'".format( + self._at_time, self._repeat + ) + ) + + if self._run_now: + self._job = schedule.every().seconds.do(self.run_once) + + if not self._run_now and self._repeat == "week": + if self.monday: + self._job = schedule.every().monday.at(self.timestr).do(self.do_run) + elif self.tuesday: + self._job = schedule.every().tuesday.at(self.timestr).do(self.do_run) + elif self.wednesday: + self._job = schedule.every().wednesday.at(self.timestr).do(self.do_run) + elif self.thursday: + self._job = schedule.every().thursday.at(self.timestr).do(self.do_run) + elif self.friday: + self._job = schedule.every().friday.at(self.timestr).do(self.do_run) + elif self.saturday: + self._job = schedule.every().saturday.at(self.timestr).do(self.do_run) + elif self.sunday: + self._job = schedule.every().sunday.at(self.timestr).do(self.do_run) + + if not self._run_now and self._repeat == "day": + self._job = schedule.every().day.at(self.timestr).do(self.do_run) + + if not self._run_now and self._repeat in ["null", None]: + self._job = ( + schedule.every().day.at(self.timestr).do(self.waiting_until_to_run) + ) + + # tag + self._job.tag(self._scheduler_id, *self._tags) + + def start(self): + """Submit and schedule the job.""" + self.submit() + + def cancel(self): + """ + Set the running job thread stoppable and wait for the + thread to exit properly by using join() method. + """ + if self._running_thread is not None and self._running_thread.is_alive(): + self._running_thread.stop() + self._running_thread.join() + + @abstractmethod + def run(self): + """ + Methods that all subclasses need to implement, note that + subclass needs to handle exception by itself. + """ + raise NotImplementedError + + +def cancel_job(job, delete_scheduler=True): + """ + Cancel the job which going to scheduled or cancel the whole scheduler. + + Args: + job: Periodic job as used by :class:`Scheduler`. + delete_scheduler: True will can the whole scheduler, otherwise, + delay the next-run time by on period. + """ + if delete_scheduler: + schedule.cancel_job(job) + else: + job.next_run += job.period diff --git a/coordinator/gscoordinator/servicer/__init__.py b/coordinator/gscoordinator/servicer/__init__.py index 9cc0c51d02ed..9b169d1e23d0 100644 --- a/coordinator/gscoordinator/servicer/__init__.py +++ b/coordinator/gscoordinator/servicer/__init__.py @@ -16,4 +16,5 @@ # limitations under the License. # +from gscoordinator.servicer.interactive.service import * from gscoordinator.servicer.graphscope_one.service import * diff --git a/coordinator/gscoordinator/servicer/base_service.py b/coordinator/gscoordinator/servicer/base_service.py new file mode 100644 index 000000000000..895ea53fb3dd --- /dev/null +++ b/coordinator/gscoordinator/servicer/base_service.py @@ -0,0 +1,45 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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 atexit +import logging + +from graphscope.config import Config +from graphscope.proto import coordinator_service_pb2_grpc +from graphscope.proto import message_pb2 + + +class BaseServiceServicer(coordinator_service_pb2_grpc.CoordinatorServiceServicer): + """Base class of coordinator service""" + + def __init__(self, config: Config): + self._config = config + atexit.register(self.cleanup) + + def __del__(self): + self.cleanup() + + def Connect(self, request, context): + return message_pb2.ConnectResponse(solution=self._config.solution) + + @property + def launcher_type(self): + return self._config.launcher_type + + def cleanup(self): + pass diff --git a/coordinator/gscoordinator/servicer/interactive/__init__.py b/coordinator/gscoordinator/servicer/interactive/__init__.py new file mode 100644 index 000000000000..3a8204719d16 --- /dev/null +++ b/coordinator/gscoordinator/servicer/interactive/__init__.py @@ -0,0 +1,26 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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 os +import sys + +try: + sys.path.insert(0, os.path.dirname(__file__)) + import interactive_client +except ImportError: + raise diff --git a/coordinator/gscoordinator/servicer/interactive/job.py b/coordinator/gscoordinator/servicer/interactive/job.py new file mode 100644 index 000000000000..0281f5f7840f --- /dev/null +++ b/coordinator/gscoordinator/servicer/interactive/job.py @@ -0,0 +1,125 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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 datetime +import logging +from enum import Enum + +import interactive_client +from graphscope.proto import interactive_pb2 + +from gscoordinator.scheduler import Scheduler +from gscoordinator.utils import encode_datetime + +logger = logging.getLogger("graphscope") + + +class JobType(Enum): + + GRAPH_IMPORT = 0 + + +class Status(Enum): + + RUNNING = 0 + CANCELLED = 1 + SUCCESS = 2 + FAILED = 3 + WAITING = 4 + + +class JobStatus(object): + """Base class of job status for GraphScope FLEX runnable tasks""" + + def __init__( + self, jobid, status, start_time, end_time=None, detail=dict(), message="" + ): + self.jobid = jobid + self.status = status + self.start_time = start_time + self.end_time = end_time + # detail for specific job + self.detail = detail + self.message = message + + def to_dict(self): + return { + "jobid": self.jobid, + "status": self.status.name, + "start_time": encode_datetime(self.start_time), + "end_time": encode_datetime(self.end_time), + "detail": self.detail, + "message": self.message, + } + + def set_success(self, message=""): + self.status = Status.SUCCESS + self.message = message + self.end_time = datetime.datetime.now() + + def set_failed(self, message=""): + self.status = Status.FAILED + self.message = message + self.end_time = datetime.datetime.now() + + def set_canncelled(self): + self.status = Status.CANCELLED + + +class GraphImportScheduler(Scheduler): + """This class responsible for scheduling and managing the import of interactive graphs.""" + + def __init__(self, at_time, repeat, schema_mapping, servicer): + super().__init__(at_time, repeat) + + self._schema_mapping = schema_mapping + # we use interactive servicer to get the latest in runtime + self._servicer = servicer + + self._tags = [JobType.GRAPH_IMPORT] + + def run(self): + """This function needs to handle exception by itself""" + graph_name = self._schema_mapping["graph"] + + detail = {"graph name": graph_name, "type": JobType.GRAPH_IMPORT.name} + status = JobStatus(self.jobid, Status.RUNNING, self.last_run, detail=detail) + + # register status to servicer + self._servicer.register_job_status(self.jobid, status) + + with interactive_client.ApiClient( + interactive_client.Configuration(self._servicer.interactive_host) + ) as api_client: + # create an instance of the API class + api_instance = interactive_client.DataloadingApi(api_client) + + try: + api_response = api_instance.create_dataloading_job( + graph_name, + interactive_client.SchemaMapping.from_dict(self._schema_mapping), + ) + except Exception as e: + logger.warning( + "Failed to create dataloading job on graph %s: %s", + graph_name, + str(e), + ) + status.set_failed(message=str(e)) + else: + status.set_success(message=api_response.message) diff --git a/coordinator/gscoordinator/servicer/interactive/service.py b/coordinator/gscoordinator/servicer/interactive/service.py new file mode 100644 index 000000000000..2e883e1119e7 --- /dev/null +++ b/coordinator/gscoordinator/servicer/interactive/service.py @@ -0,0 +1,449 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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. +# + +"""Interactive Service under FLEX Architecture""" + +import itertools +import logging +import os +import threading + +import interactive_client +from google.protobuf.json_format import MessageToDict +from graphscope.config import Config +from graphscope.gsctl.utils import dict_to_proto_message +from graphscope.proto import error_codes_pb2 +from graphscope.proto import interactive_pb2 + +from gscoordinator.scheduler import schedule +from gscoordinator.servicer.base_service import BaseServiceServicer +from gscoordinator.servicer.interactive.job import GraphImportScheduler +from gscoordinator.utils import WORKSPACE +from gscoordinator.utils import delegate_command_to_pod +from gscoordinator.utils import run_kube_cp_command + +__all__ = ["InteractiveServiceServicer", "init_interactive_service_servicer"] + +logger = logging.getLogger("graphscope") + + +# There are two workspaces for FLEX Interactive, one residing on "coordinator" node +# and the other on the "interactive" node. These two workspaces are equivalent when +# running mode is "hosts". Hence, INTERACTIVE_WORKSPACE is only effective in +# Kubernetes (k8s) environment. +INTERACTIVE_WORKSPACE = ( + os.environ["INTERACTIVE_WORKSPACE"] + if "INTERACTIVE_WORKSPACE" in os.environ + else "/tmp" +) + + +INTERACTIVE_CONTAINER_NAME = "interactive" + + +class InteractiveServiceServicer(BaseServiceServicer): + """Interactive service under flex architecture.""" + + def __init__(self, config: Config): + super().__init__(config) + + # lock to protect the service + self._lock = threading.RLock() + + # interactive host + self._interactive_host = None + self._fetch_interactive_host_impl() + self._fetch_interactive_host_job = schedule.every(60).seconds.do( + self._fetch_interactive_host_impl + ) + + # interactive pod list + self._interactive_pod_list = [] + + # job status + self._job_status = {} + + def cleanup(self): + pass + + def _fetch_interactive_host_impl(self): + # TODO: get interactive service endpoint by instance id + self._interactive_host = "http://47.242.172.5:7777" + + @property + def interactive_host(self): + return self._interactive_host + + def register_job_status(self, jobid, status): + self._job_status[jobid] = status + + def write_and_distribute_file(self, graph, location, raw_data): + """ + Args: + graph(str): graph name. + location(str): file location over coordinator node. + raw_data(bytes): binary of file data. + """ + # format: _ + filename = "{0}_{1}".format(graph, os.path.basename(location)) + + # write temp file + tmp_file = os.path.join(WORKSPACE, filename) + with open(tmp_file, "wb") as f: + f.write(raw_data) + + # distribute + target_file = tmp_file + if self.launcher_type == "k8s": + # update file path in interactive pod + target_file = os.path.join(INTERACTIVE_WORKSPACE, filename) + + for pod in self._interactive_pod_list: + container = INTERACTIVE_CONTAINER_NAME + cmd = f"mkdir -p {os.path.dirname(target_file)}" + logger.debug(delegate_command_to_pod(cmd, pod, container)) + logger.debug( + run_kube_cp_command(tmp_file, target_file, pod, container, True) + ) + + return target_file + + def CreateInteractiveGraph(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + # create an instance of the API class + api_instance = interactive_client.GraphApi(api_client) + + graph_def_dict = MessageToDict( + request.graph_def, preserving_proto_field_name=True + ) + graph = interactive_client.Graph.from_dict(graph_def_dict) + + try: + api_response = api_instance.create_graph(graph) + except Exception as e: + logger.warning("Failed to create interactive graph. %s", str(e)) + return interactive_pb2.ApiResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + return interactive_pb2.ApiResponse( + code=error_codes_pb2.OK, error_msg=api_response.message + ) + + def RemoveInteractiveGraph(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + api_instance = interactive_client.GraphApi(api_client) + + try: + api_response = api_instance.delete_graph(request.graph_name) + except Exception as e: + logger.warning("Failed to remove interactive graph. %s", str(e)) + return interactive_pb2.ApiResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + return interactive_pb2.ApiResponse( + code=error_codes_pb2.OK, error_msg=api_response.message + ) + + def ListInteractiveGraph(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + api_instance = interactive_client.GraphApi(api_client) + + try: + api_response = api_instance.list_graphs() + except Exception as e: + logger.warning("Failed to list interactive graph. %s", str(e)) + return interactive_pb2.ListInteractiveGraphResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + return interactive_pb2.ListInteractiveGraphResponse( + code=error_codes_pb2.OK, + graphs=[ + dict_to_proto_message(g.to_dict(), interactive_pb2.GraphProto()) + for g in api_response + ], + ) + + def ImportInteractiveGraph(self, request, context): + # write raw data to file and copy to interactive workspace + try: + schema_mapping_proto = request.schema_mapping + for mapping in itertools.chain( + schema_mapping_proto.vertex_mappings, schema_mapping_proto.edge_mappings + ): + raw_data_index = 0 + for index, location in enumerate(mapping.inputs): + if location.startswith("@"): + # write raw data and distribute file to interactive workspace + new_location = self.write_and_distribute_file( + schema_mapping_proto.graph, + location, + mapping.raw_data[raw_data_index], + ) + raw_data_index += 1 + # update the location + mapping.inputs[index] = new_location + except Exception as e: + logger.warning("Failed to distribute file. %s", str(e)) + return interactive_pb2.ApiResponse( + code=error_codes_pb2.NETWORK_ERROR, error_msg=str(e) + ) + + # transform proto to dict + schema_mapping = MessageToDict( + schema_mapping_proto, preserving_proto_field_name=True + ) + + # create job scheduler + scheduler = GraphImportScheduler( + at_time="now", + repeat=None, + schema_mapping=schema_mapping, + servicer=self, + ) + scheduler.start() + + return interactive_pb2.ApiResponse(code=error_codes_pb2.OK) + + def ListInteractiveJob(self, request, context): + return interactive_pb2.ListInteractiveJobResponse( + code=error_codes_pb2.OK, + job_status=[ + dict_to_proto_message(s.to_dict(), interactive_pb2.JobStatus()) + for _, s in self._job_status.items() + ], + ) + + def CreateInteractiveProcedure(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + # create an instance of the API class + api_instance = interactive_client.ProcedureApi(api_client) + + # transform proto to dict + procedure_def_dict = MessageToDict( + request.procedure_def, preserving_proto_field_name=True + ) + + graph_name = procedure_def_dict["bound_graph"] + procedure = interactive_client.Procedure.from_dict(procedure_def_dict) + + try: + api_response = api_instance.create_procedure(graph_name, procedure) + except Exception as e: + logger.warning("Failed to create procedure. %s", str(e)) + return interactive_pb2.ApiResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + return interactive_pb2.ApiResponse( + code=error_codes_pb2.OK, error_msg=api_response.message + ) + + def ListInteractiveProcedure(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + api_instance = interactive_client.ProcedureApi(api_client) + + try: + api_response = api_instance.list_procedures(request.graph_name) + except Exception as e: + logger.warning("Failed to list procedure. %s", str(e)) + return interactive_pb2.ListInteractiveProcedureResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + # change "returns" in the openapi definition to "rlts_meta", + # as "returns" is a reserved keyword in proto + procedures = [] + for p in api_response: + procedure_dict = p.to_dict() + # "returns" -> "rlts_meta" + procedure_dict["rlts_meta"] = procedure_dict.pop("returns") + # push + procedures.append(procedure_dict) + + return interactive_pb2.ListInteractiveProcedureResponse( + code=error_codes_pb2.OK, + procedures=[ + dict_to_proto_message(p, interactive_pb2.Procedure()) + for p in procedures + ], + ) + + def UpdateInteractiveProcedure(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + api_instance = interactive_client.ProcedureApi(api_client) + + for p in request.procedures: + # transform proto to dict + procedure_def_dict = MessageToDict(p, preserving_proto_field_name=True) + + graph_name = procedure_def_dict["bound_graph"] + procedure_name = procedure_def_dict["name"] + procedure = interactive_client.Procedure.from_dict(procedure_def_dict) + + try: + api_response = api_instance.update_procedure( + graph_name, procedure_name, procedure + ) + except Exception as e: + logger.warning("Failed to update procedure. %s", str(e)) + return interactive_pb2.ApiResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + + return interactive_pb2.ApiResponse(code=error_codes_pb2.OK) + + def RemoveInteractiveProcedure(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + api_instance = interactive_client.ProcedureApi(api_client) + + try: + api_response = api_instance.delete_procedure( + request.graph_name, request.procedure_name + ) + except Exception as e: + logger.warning("Failed to remove procedure. %s", str(e)) + return interactive_pb2.ApiResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + return interactive_pb2.ApiResponse( + code=error_codes_pb2.OK, error_msg=api_response.message + ) + + def GetInteractiveServiceStatus(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + api_instance = interactive_client.ServiceApi(api_client) + + try: + api_response = api_instance.get_service_status() + except Exception as e: + logger.warning("Failed to get service status. %s", str(e)) + return interactive_pb2.GetInteractiveServiceStatusResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + return interactive_pb2.GetInteractiveServiceStatusResponse( + code=error_codes_pb2.OK, + service_status=dict_to_proto_message( + api_response.to_dict(), interactive_pb2.ServiceStatus() + ), + ) + + def StartInteractiveService(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + api_instance = interactive_client.ServiceApi(api_client) + + service_def_dict = MessageToDict( + request.service_def, preserving_proto_field_name=True + ) + service = interactive_client.Service.from_dict(service_def_dict) + + try: + api_response = api_instance.start_service(service) + except Exception as e: + logger.warning("Failed to start service. %s", str(e)) + return interactive_pb2.ApiResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + return interactive_pb2.ApiResponse( + code=error_codes_pb2.OK, error_msg=api_response.message + ) + + def StopInteractiveService(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + api_instance = interactive_client.ServiceApi(api_client) + + try: + api_response = api_instance.stop_service() + except Exception as e: + logger.warning("Failed to stop service. %s", str(e)) + return interactive_pb2.ApiResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + return interactive_pb2.ApiResponse( + code=error_codes_pb2.OK, error_msg=api_response.message + ) + + def RestartInteractiveService(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + api_instance = interactive_client.ServiceApi(api_client) + + try: + api_response = api_instance.restart_service() + except Exception as e: + logger.warning("Failed to restart service. %s", str(e)) + return interactive_pb2.ApiResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + return interactive_pb2.ApiResponse( + code=error_codes_pb2.OK, error_msg=api_response.message + ) + + def GetNodeStatus(self, request, context): + with interactive_client.ApiClient( + interactive_client.Configuration(self._interactive_host) + ) as api_client: + api_instance = interactive_client.NodeApi(api_client) + + try: + api_response = api_instance.get_node_status() + except Exception as e: + logger.warning("Failed to get node status. %s", str(e)) + return interactive_pb2.GetNodeStatusResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + return interactive_pb2.GetNodeStatusResponse( + code=error_codes_pb2.OK, + nodes_status=[ + dict_to_proto_message(s.to_dict(), interactive_pb2.NodeStatus()) + for s in api_response + ], + ) + + +def init_interactive_service_servicer(config: Config): + return InteractiveServiceServicer(config) diff --git a/coordinator/gscoordinator/stoppable_thread.py b/coordinator/gscoordinator/stoppable_thread.py new file mode 100644 index 000000000000..96a072e9f5a7 --- /dev/null +++ b/coordinator/gscoordinator/stoppable_thread.py @@ -0,0 +1,39 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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 threading + + +class StoppableThread(threading.Thread): + """ + This is one of the simplest mechaisms for a stoppable thread to + hold an 'exit_request' flag that each thread checks on a regular + interval to see if it is time for it to exit. + """ + + def __init__(self, *args, **kwargs): + super(StoppableThread, self).__init__(*args, **kwargs) + self._stop_event = threading.Event() + + def stop(self): + self._stop_event.set() + + def stopped(self): + """The thread itself should check regularly for the stopped() condition. + """ + return self._stop_event.is_set() diff --git a/coordinator/gscoordinator/utils.py b/coordinator/gscoordinator/utils.py index f39d99f2b036..61095bb739c4 100644 --- a/coordinator/gscoordinator/utils.py +++ b/coordinator/gscoordinator/utils.py @@ -219,6 +219,26 @@ def get_timestamp() -> float: return datetime.datetime.timestamp(datetime.datetime.now()) +def decode_datetimestr(datetime_str): + formats = ["%Y-%m-%d %H:%M:%S", "%Y-%m-%d %H:%M:%S.%f", "%Y-%m-%d"] + for f in formats: + try: + return datetime.datetime.strptime(datetime_str, f) + except ValueError: + pass + raise RuntimeError( + "Decode '{0}' failed: format should be one of '{1}'".format( + datetime_str, str(formats) + ) + ) + + +def encode_datetime(dt): + if isinstance(dt, datetime.datetime): + return dt.strftime("%Y-%m-%d %H:%M:%S") + return str(dt) + + def get_lib_path(app_dir: str, app_name: str) -> str: if sys.platform == "linux" or sys.platform == "linux2": return os.path.join(app_dir, "lib%s.so" % app_name) diff --git a/coordinator/requirements.txt b/coordinator/requirements.txt index 72e88e88fbe1..a8618f86746c 100644 --- a/coordinator/requirements.txt +++ b/coordinator/requirements.txt @@ -9,3 +9,5 @@ vineyard-io>=0.16.3;sys_platform!="win32" prometheus-client>=0.14.1 packaging tqdm +pydantic >= 2 +schedule diff --git a/proto/coordinator_service.proto b/proto/coordinator_service.proto index 1d5ed680f8d5..d721bcd056c8 100644 --- a/proto/coordinator_service.proto +++ b/proto/coordinator_service.proto @@ -17,6 +17,7 @@ syntax = "proto3"; package gs.rpc; import "message.proto"; +import "interactive.proto"; service CoordinatorService { // Connect a session. @@ -51,4 +52,32 @@ service CoordinatorService { // service functions under FLEX architecture rpc Connect(ConnectRequest) returns (ConnectResponse); + + rpc CreateInteractiveGraph(CreateInteractiveGraphRequest) returns (ApiResponse); + + rpc RemoveInteractiveGraph(RemoveInteractiveGraphRequest) returns (ApiResponse); + + rpc ListInteractiveGraph(ListInteractiveGraphRequest) returns (ListInteractiveGraphResponse); + + rpc ImportInteractiveGraph(ImportInteractiveGraphRequest) returns (ApiResponse); + + rpc ListInteractiveJob(ListInteractiveJobRequest) returns (ListInteractiveJobResponse); + + rpc CreateInteractiveProcedure(CreateInteractiveProcedureRequest) returns (ApiResponse); + + rpc ListInteractiveProcedure(ListInteractiveProcedureRequest) returns (ListInteractiveProcedureResponse); + + rpc UpdateInteractiveProcedure(UpdateInteractiveProcedureRequest) returns (ApiResponse); + + rpc RemoveInteractiveProcedure(RemoveInteractiveProcedureRequest) returns (ApiResponse); + + rpc GetInteractiveServiceStatus(GetInteractiveServiceStatusRequest) returns (GetInteractiveServiceStatusResponse); + + rpc StartInteractiveService(StartInteractiveServiceRequest) returns (ApiResponse); + + rpc StopInteractiveService(StopInteractiveServiceRequest) returns (ApiResponse); + + rpc RestartInteractiveService(RestartInteractiveServiceRequest) returns (ApiResponse); + + rpc GetNodeStatus(GetNodeStatusRequest) returns (GetNodeStatusResponse); } diff --git a/proto/error_codes.proto b/proto/error_codes.proto index 7b9ca43e8838..1107e3bb55e4 100644 --- a/proto/error_codes.proto +++ b/proto/error_codes.proto @@ -84,6 +84,9 @@ enum Code { // Results of workers not consistent WORKER_RESULTS_INCONSISTENT_ERROR = 41; + // Api exception during communication between coordinator and engines + API_EXCEPTION_ERROR = 51; + // Unknown error. UNKNOWN_ERROR = 101; diff --git a/proto/interactive.proto b/proto/interactive.proto new file mode 100644 index 000000000000..6c5fcea45c83 --- /dev/null +++ b/proto/interactive.proto @@ -0,0 +1,283 @@ +// 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. +// + +syntax = "proto3"; + +package gs.rpc; + +import "error_codes.proto"; + +// property type +message PropertyTypeProto { + string primitive_type = 1; // [ DT_DOUBLE, DT_STRING, DT_SIGNED_INT32, DT_SIGNED_INT64 ] +} + +// property +message PropertyProto { + int32 property_id = 1; + string property_name = 2; + PropertyTypeProto property_type = 3; +} + +// csr params definition, used for storage optimization +message CSRParamsProto { + optional string incoming_edge_strategy = 1; // one of [Single, Multple] + optional string outgoing_edge_strategy = 2; // one of [Single, Multple] +} + +// vertex pair +message VertexPairProto { + string source_vertex = 1; // label + string destination_vertex = 2; // label + string relation = 3; // one of [ MANY_TO_MANY, ONE_TO_MANY, MANY_TO_ONE, ONE_TO_ONE, ] + CSRParamsProto x_csr_params = 4; +} + +// vertex +message VertexProto { + int32 type_id = 1; // index + string type_name = 2; // label + repeated PropertyProto properties = 3; + repeated string primary_keys = 4; +} + +// edge +message EdgeProto { + int32 type_id = 1; // index + string type_name = 2; // label + repeated VertexPairProto vertex_type_pair_relations = 3; + repeated PropertyProto properties = 4; +} + +// schema +message Schema { + repeated VertexProto vertex_types = 1; + repeated EdgeProto edge_types = 2; +} + +// stored procedure +message StoredProcedureProto { + string directory = 1; // one of [ plugins ] +} + +// graph +message GraphProto { + string name = 1; + string store_type = 2; // one of [ mutable_csr, ] + StoredProcedureProto stored_procedures = 3; + Schema schema = 4; +} + +// column +message Column { + int32 index = 1; + string name = 2; +} + +// column mapping +// column index(name) of datasource -> property +message ColumnMapping { + Column column = 1; + string property = 2; +} + +// vertex mapping +message VertexMapping { + string type_name = 1; // vertex label + repeated string inputs = 2; // protocol:///path/to/data + repeated bytes raw_data = 3; // raw data + repeated ColumnMapping column_mappings = 4; +} + +// type triplet for edge +message TypeTriplet { + string edge = 1; // edge label + string source_vertex = 2; // src vertex label + string destination_vertex = 3; // dst vertex label +} + +// edge mapping +message EdgeMapping { + TypeTriplet type_triplet = 1; + repeated string inputs = 2; + repeated bytes raw_data = 3; + repeated ColumnMapping source_vertex_mappings = 4; + repeated ColumnMapping destination_vertex_mappings = 5; + repeated ColumnMapping column_mappings = 6; +} + +// data source config +message DataSource { + string scheme = 1; // file, oss, s3, hdfs; only file is supported now + string location = 2; // unified prefix +} + +// format +message DataSourceFormat { + string type = 1; // csv + map metadata = 2; +} + +// loading config +message LoadingConfig { + DataSource data_source = 1; + string import_option = 2; // init, append, overwrite, only init is supported now + DataSourceFormat format = 3; +} + +// schema mapping +message SchemaMapping { + string graph = 1; + LoadingConfig loading_config = 2; + repeated VertexMapping vertex_mappings = 3; + repeated EdgeMapping edge_mappings = 4; +} + +// job status +message JobStatus { + string jobid = 1; + string status = 2; + string start_time = 3; + string end_time = 4; + map detail = 5; + string message = 6; +} + +// procedure runtime params, used both in property and return type +message ProcedureRuntimeParam { + string name = 1; + string type = 2; +} + +// procedure +message Procedure { + string name = 1; + string bound_graph = 2; + string description = 3; + optional string type = 4; // [ cpp, cypher ] + string query = 5; // content + bool enable = 6; + + // runtime info, assign by Interactive server side. + repeated ProcedureRuntimeParam params = 7; // parameters in call + repeated ProcedureRuntimeParam rlts_meta = 8; +} + +// service +message Service { + string graph_name = 1; +} + +// service status +message ServiceStatus { + string status = 1; + string graph_name = 2; + int32 bolt_port = 3; + int32 hqps_port = 4; +} + +// node status +message NodeStatus { + string node = 1; // node name + int32 cpu_usage = 2; + int32 memory_usage = 3; + int32 disk_usage = 4; +} + +// message +message ApiResponse { + Code code = 1; + optional string error_msg = 2; +} + +message CreateInteractiveGraphRequest { + GraphProto graph_def = 1; +} + +message RemoveInteractiveGraphRequest { + string graph_name = 1; +} + +message ListInteractiveGraphRequest {} + +message ListInteractiveGraphResponse { + Code code = 1; + optional string error_msg = 2; + + repeated GraphProto graphs = 3; +} + +message ImportInteractiveGraphRequest { + SchemaMapping schema_mapping = 1; +} + +message ListInteractiveJobRequest {} + +message ListInteractiveJobResponse { + Code code = 1; + optional string error_msg = 2; + + repeated JobStatus job_status = 3; +} + +message CreateInteractiveProcedureRequest { + Procedure procedure_def = 1; +} + +message ListInteractiveProcedureRequest { + string graph_name = 1; +} + +message ListInteractiveProcedureResponse { + Code code = 1; + optional string error_msg = 2; + + repeated Procedure procedures = 3; +} + +message UpdateInteractiveProcedureRequest { + repeated Procedure procedures = 1; +} + +message RemoveInteractiveProcedureRequest { + string graph_name = 1; + string procedure_name = 2; +} + +message GetInteractiveServiceStatusRequest {} + +message GetInteractiveServiceStatusResponse { + Code code = 1; + optional string error_msg = 2; + + ServiceStatus service_status = 3; +} + +message StartInteractiveServiceRequest { + Service service_def = 1; +} + +message StopInteractiveServiceRequest {} + +message RestartInteractiveServiceRequest {} + +message GetNodeStatusRequest {} + +message GetNodeStatusResponse { + Code code = 1; + optional string error_msg = 2; + + repeated NodeStatus nodes_status = 3; +} diff --git a/python/graphscope/gsctl/client/rpc.py b/python/graphscope/gsctl/client/rpc.py new file mode 100644 index 000000000000..0b448cf06900 --- /dev/null +++ b/python/graphscope/gsctl/client/rpc.py @@ -0,0 +1,169 @@ +#!/usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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 atexit +import time +from typing import List + +import click +import grpc +from graphscope.client.utils import GS_GRPC_MAX_MESSAGE_LENGTH +from graphscope.client.utils import handle_grpc_error +from graphscope.gsctl.config import GS_CONFIG_DEFAULT_LOCATION +from graphscope.gsctl.config import get_current_context +from graphscope.proto import coordinator_service_pb2_grpc +from graphscope.proto import interactive_pb2 +from graphscope.proto import message_pb2 +from graphscope.version import __version__ + + +class GRPCClient(object): + def __init__(self, endpoint): + # create the grpc stub + options = [ + ("grpc.max_send_message_length", GS_GRPC_MAX_MESSAGE_LENGTH), + ("grpc.max_receive_message_length", GS_GRPC_MAX_MESSAGE_LENGTH), + ("grpc.max_metadata_size", GS_GRPC_MAX_MESSAGE_LENGTH), + ] + self._channel = grpc.insecure_channel(endpoint, options=options) + self._stub = coordinator_service_pb2_grpc.CoordinatorServiceStub(self._channel) + + atexit.register(self.close) + + @handle_grpc_error + def _connect_impl(self, timeout_seconds): + begin_time = time.time() + + request = message_pb2.ConnectRequest(version=__version__) + while True: + try: + response = self._stub.Connect(request) + break + except Exception as e: + msg = f"code: {e.code().name}, details: {e.details()}" + click.secho( + f"Failed to connect coordinator: {e}, try after second...", + fg="yellow", + ) + if time.time() - begin_time >= timeout_seconds: + raise ConnectionError(f"Connect coordinator timeout, {msg}") + time.sleep(1) + + return response.solution + + def connect(self, timeout_seconds=10): + return self._connect_impl(timeout_seconds) + + def close(self): + try: + self._channel.close() + except: # noqa: E722 + pass + + @handle_grpc_error + def create_interactive_graph(self, graph_def: interactive_pb2.GraphProto): + request = interactive_pb2.CreateInteractiveGraphRequest(graph_def=graph_def) + return self._stub.CreateInteractiveGraph(request) + + @handle_grpc_error + def remove_interactive_graph(self, graph: str): + request = interactive_pb2.RemoveInteractiveGraphRequest(graph_name=graph) + return self._stub.RemoveInteractiveGraph(request) + + @handle_grpc_error + def list_interactive_graph(self): + request = interactive_pb2.ListInteractiveGraphRequest() + return self._stub.ListInteractiveGraph(request) + + @handle_grpc_error + def import_interactive_graph(self, schema_mapping: interactive_pb2.SchemaMapping): + request = interactive_pb2.ImportInteractiveGraphRequest( + schema_mapping=schema_mapping + ) + return self._stub.ImportInteractiveGraph(request) + + @handle_grpc_error + def list_interactive_job(self): + request = interactive_pb2.ListInteractiveJobRequest() + return self._stub.ListInteractiveJob(request) + + @handle_grpc_error + def create_interactive_procedure(self, procedure: interactive_pb2.Procedure): + request = interactive_pb2.CreateInteractiveProcedureRequest( + procedure_def=procedure + ) + return self._stub.CreateInteractiveProcedure(request) + + @handle_grpc_error + def list_interactive_procedure(self, graph: str): + request = interactive_pb2.ListInteractiveProcedureRequest(graph_name=graph) + return self._stub.ListInteractiveProcedure(request) + + @handle_grpc_error + def update_interactive_procedure(self, procedures: List[interactive_pb2.Procedure]): + request = interactive_pb2.UpdateInteractiveProcedureRequest( + procedures=procedures + ) + return self._stub.UpdateInteractiveProcedure(request) + + @handle_grpc_error + def remove_interactive_procedure(self, graph: str, procedure: str): + request = interactive_pb2.RemoveInteractiveProcedureRequest( + graph_name=graph, procedure_name=procedure + ) + return self._stub.RemoveInteractiveProcedure(request) + + @handle_grpc_error + def get_interactive_service_status(self): + request = interactive_pb2.GetInteractiveServiceStatusRequest() + return self._stub.GetInteractiveServiceStatus(request) + + @handle_grpc_error + def start_interactive_service(self, service: interactive_pb2.Service): + request = interactive_pb2.StartInteractiveServiceRequest(service_def=service) + return self._stub.StartInteractiveService(request) + + @handle_grpc_error + def stop_interactive_service(self): + request = interactive_pb2.StopInteractiveServiceRequest() + return self._stub.StopInteractiveService(request) + + @handle_grpc_error + def restart_interactive_service(self): + request = interactive_pb2.RestartInteractiveServiceRequest() + return self._stub.RestartInteractiveService(request) + + @handle_grpc_error + def get_node_status(self): + request = interactive_pb2.GetNodeStatusRequest() + return self._stub.GetNodeStatus(request) + + +def get_grpc_client(coordinator_endpoint=None): + if coordinator_endpoint is not None: + return GRPCClient(coordinator_endpoint) + + # use the latest context in config file + current_context = get_current_context() + if current_context is None: + raise RuntimeError( + "No available context found in {0}, please connect to a launched coordinator first.".format( + GS_CONFIG_DEFAULT_LOCATION + ) + ) + return GRPCClient(current_context.coordinator_endpoint) diff --git a/python/graphscope/gsctl/commands/__init__.py b/python/graphscope/gsctl/commands/__init__.py index 29ab31f2d6ab..8bdcbed90f5f 100644 --- a/python/graphscope/gsctl/commands/__init__.py +++ b/python/graphscope/gsctl/commands/__init__.py @@ -16,4 +16,27 @@ # limitations under the License. # -from graphscope.gsctl.commands.utils import * +import click + +from graphscope.gsctl.commands.common import cli as common_cli +from graphscope.gsctl.commands.dev import cli as dev_cli +from graphscope.gsctl.commands.interactive import cli as interactive_cli +from graphscope.gsctl.config import Context +from graphscope.gsctl.config import FLEX_INTERACTIVE + + +def get_command_collection(context: Context): + if context is None: + # treat gsctl as an utility script, providing hepler functions or utilities. e.g. + # initialize and manage cluster, install the dependencies required to build graphscope locally + commands = click.CommandCollection(sources=[common_cli, dev_cli]) + + elif context.solution == FLEX_INTERACTIVE: + commands = click.CommandCollection(sources=[common_cli, interactive_cli]) + + else: + raise RuntimeError( + f"Failed to get command collection with context {context.name}" + ) + + return commands diff --git a/python/graphscope/gsctl/commands/common_command.py b/python/graphscope/gsctl/commands/common.py similarity index 86% rename from python/graphscope/gsctl/commands/common_command.py rename to python/graphscope/gsctl/commands/common.py index ff3a02a4a3e2..c01a2986def4 100644 --- a/python/graphscope/gsctl/commands/common_command.py +++ b/python/graphscope/gsctl/commands/common.py @@ -22,7 +22,7 @@ from graphscope.gsctl.config import Context from graphscope.gsctl.config import load_gs_config -from graphscope.gsctl.rpc import get_grpc_client +from graphscope.gsctl.client.rpc import get_grpc_client @click.group() @@ -31,14 +31,16 @@ def cli(): pass -@click.command() +@cli.command() @click.option( "--coordinator-endpoint", help="Coordinator endpoint which gsctl connect to, e.g. http://127.0.0.1:9527", ) def connect(coordinator_endpoint): - """Connect to the launched coordinator by ~/.graphscope/config. If '--coordinator-endpoint' is specified, - use it as the current context and override the config file. + """Connect to the launched coordinator by ~/.graphscope/config + + If '--coordinator-endpoint' is specified, use it as the current context + and override the config file. """ if coordinator_endpoint is not None: click.secho( @@ -56,9 +58,9 @@ def connect(coordinator_endpoint): click.secho("Coordinator service connected.", fg="green") -@click.command() +@cli.command() def close(): - """Close the connection from the coordinator.""" + """Close the connection from the coordinator""" config = load_gs_config() current_context = config.current_context() @@ -69,9 +71,5 @@ def close(): click.secho(f"Disconnect from the {current_context.to_dict()}.", fg="green") -cli.add_command(connect) -cli.add_command(close) - - if __name__ == "__main__": cli() diff --git a/python/graphscope/gsctl/commands/dev_command.py b/python/graphscope/gsctl/commands/dev.py similarity index 100% rename from python/graphscope/gsctl/commands/dev_command.py rename to python/graphscope/gsctl/commands/dev.py diff --git a/python/graphscope/gsctl/commands/interactive.py b/python/graphscope/gsctl/commands/interactive.py new file mode 100644 index 000000000000..7381453a4802 --- /dev/null +++ b/python/graphscope/gsctl/commands/interactive.py @@ -0,0 +1,625 @@ +#!/usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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. +# + +"""Group of interactive commands under the FLEX architecture""" + +import itertools +import json + +import click +import yaml +from google.protobuf.json_format import MessageToDict + +from graphscope.gsctl.client.rpc import get_grpc_client +from graphscope.gsctl.utils import dict_to_proto_message +from graphscope.gsctl.utils import is_valid_file_path +from graphscope.gsctl.utils import read_yaml_file +from graphscope.gsctl.utils import terminal_display +from graphscope.proto import error_codes_pb2 +from graphscope.proto import interactive_pb2 + + +@click.group() +def cli(): + pass + + +@cli.group() +def get(): + """Display one or many resources""" + pass + + +@cli.group() +def describe(): + """Show details of a specific resource or group of resources""" + pass + + +@cli.group() +def create(): + """Create a resource from a file or from stdin""" + pass + + +@cli.group() +def delete(): + """Delete resource by name""" + pass + + +@cli.group() +def enable(): + """Enable stored procedures over a given graph""" + pass + + +@cli.group() +def disable(): + """Disable stored procedures over a given graph""" + pass + + +@cli.group() +def service(): + """Start, restart or stop the database service""" + pass + + +def is_success(response): + if response.code != error_codes_pb2.OK: + click.secho( + "{0}: {1}".format( + error_codes_pb2.Code.Name(response.code), response.error_msg + ), + fg="red", + ) + return False + return True + + +@cli.command(name="import") +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph.", +) +@click.option( + "-c", + "--config", + required=True, + help="Yaml path or raw data for loading graph.", +) +def data_import(graph, config): + """Load the raw data specified in bulk load file""" + + def _read_and_fill_raw_data(config): + for mapping in itertools.chain( + config["vertex_mappings"], config["edge_mappings"] + ): + for index, location in enumerate(mapping["inputs"]): + # location is one of: + # 1) protocol:///path/to/the/file + # 2) @/path/to/the/file, which represents the local file + if location.startswith("@"): + if "raw_data" not in mapping: + mapping["raw_data"] = [] + + # read file and set raw data + with open(location[1:], "rb") as f: + content = f.read() + mapping["raw_data"].append(content) + + schema_mapping_dict = config + if is_valid_file_path(config): + schema_mapping_dict = read_yaml_file(config) + + if graph is not None: + schema_mapping_dict["graph"] = graph + + _read_and_fill_raw_data(schema_mapping_dict) + + # transfiorm dict to proto message + schema_mapping = interactive_pb2.SchemaMapping() + dict_to_proto_message(schema_mapping_dict, schema_mapping) + + grpc_client = get_grpc_client() + response = grpc_client.import_interactive_graph(schema_mapping) + + if is_success(response): + click.secho("Create dataloading job successfully.", fg="green") + + +@create.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +@click.option( + "-c", + "--config", + required=True, + help="Yaml path or json string of schema for the graph", +) +def graph(graph, config): + """Create a graph in database, with the provided schema file""" + + graph_def_dict = config + if is_valid_file_path(config): + graph_def_dict = read_yaml_file(config) + + # override graph name + if graph is not None: + graph_def_dict["name"] = graph + + # transform graph dict to proto message + graph_def = interactive_pb2.GraphProto() + dict_to_proto_message(graph_def_dict, graph_def) + + grpc_client = get_grpc_client() + response = grpc_client.create_interactive_graph(graph_def) + + if is_success(response): + click.secho( + "Create interactive graph {0} successfully.".format(graph_def_dict["name"]), + fg="green", + ) + + +@create.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +@click.option( + "-n", + "--name", + required=True, + help="The name of the procedure", +) +@click.option( + "-i", + "--sourcefile", + required=True, + help="Path of [ .cc, .cypher ] file", +) +@click.option( + "-d", + "--description", + required=False, + help="Description for the specific procedure", +) +def procedure(graph, name, sourcefile, description): + """Create and compile procedure over a specific graph""" + + # read source + with open(sourcefile, "r") as f: + query = f.read() + + # construct procedure proto + procedure = interactive_pb2.Procedure( + name=name, bound_graph=graph, description=description, query=query, enable=True + ) + + grpc_client = get_grpc_client() + response = grpc_client.create_interactive_procedure(procedure) + + if is_success(response): + click.secho("Create procedure {0} successfully.".format(name), fg="green") + + +@delete.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +def graph(graph): + """Delete a graph, as well as the loaded data by name""" + + grpc_client = get_grpc_client() + response = grpc_client.remove_interactive_graph(graph) + + if is_success(response): + click.secho( + "Delete interactive graph {0} successfully.".format(graph), + fg="green", + ) + + +@delete.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +@click.option( + "-n", + "--name", + required=True, + help="The name of the procedure to be deleted", +) +def procedure(graph, name): + """Delete a procedure over a specific graph in database""" + + grpc_client = get_grpc_client() + response = grpc_client.remove_interactive_procedure(graph, name) + + if is_success(response): + click.secho( + "Remove procedure {0} on graph {1} successfully.".format(name, graph), + fg="green", + ) + + +@enable.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +@click.option( + "-n", + "--name", + required=True, + help="List of procedure's name to enable, seprated by comma", +) +def procedure(graph, name): + """Enable stored procedures in a given graph""" + + # remove the last "," if exists + if name.endswith(","): + name = name[:-1] + procedure_list = name.split(",") + + # list current procedures + grpc_client = get_grpc_client() + response = grpc_client.list_interactive_procedure(graph) + + if is_success(response): + # enable procedures locally + procedures = response.procedures + for p in procedures: + if p.name in procedure_list: + p.enable = True + procedure_list.remove(p.name) + + # check + if procedure_list: + click.secho( + "Procedure {0} not found.".format(str(procedure_list)), fg="red" + ) + return + + # update procedures + response = grpc_client.update_interactive_procedure(procedures) + if is_success(response): + click.secho("Update procedures successfully.", fg="green") + + +@disable.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +@click.option( + "-n", + "--name", + required=True, + help="List of procedure's name to enable, seprated by comma", +) +def proceduree(graph, name): + """Disable stored procedures in a given graph""" + + # remove the last "," if exists + if name.endswith(","): + name = name[:-1] + procedure_list = name.split(",") + + # list current procedures + grpc_client = get_grpc_client() + response = grpc_client.list_interactive_procedure(graph) + + if is_success(response): + # disable procedures locally + procedures = response.procedures + for p in procedures: + if p.name in procedure_list: + p.enable = False + procedure_list.remove(p.name) + + # check + if procedure_list: + click.secho( + "Procedure {0} not found.".format(str(procedure_list)), fg="red" + ) + return + + # update procedures + response = grpc_client.update_interactive_procedure(procedures) + if is_success(response): + click.secho("Update procedures successfully.", fg="green") + + +@service.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +def start(graph): + """Stop current service and start a new one on a specified graph""" + + service_def = interactive_pb2.Service(graph_name=graph) + + grpc_client = get_grpc_client() + response = grpc_client.start_interactive_service(service_def) + + if is_success(response): + click.secho( + "Start service on graph {0} successfully.".format(graph), fg="green" + ) + + +@service.command() +def stop(): + """Stop the current database service""" + + grpc_client = get_grpc_client() + response = grpc_client.stop_interactive_service() + + if is_success(response): + click.secho("Service has stopped.", fg="green") + + +@service.command() +def restart(): + """Restart database service on current graph""" + + grpc_client = get_grpc_client() + response = grpc_client.restart_interactive_service() + + if is_success(response): + click.secho("Restart service successfully.", fg="green") + + +@get.command() +def graph(): + """Display graphs in database""" + + def _construct_and_display_data(graphs): + if not graphs: + click.secho("no graph found in database.", fg="blue") + return + + head = ["", "NAME", "STORE TYPE", "VERTEX LABEL SIZE", "EDGE LABEL SIZE"] + data = [head] + + for g in graphs: + # vertex_schema = [] + # for v in g.schema.vertex_types: + # vertex_schema.append(v.type_name) + # edge_schema = [] + # for e in g.schema.edge_types: + # edge_schema.append(e.type_name) + data.append( + [ + "*", # service graph + g.name, + g.store_type, + str(len(g.schema.vertex_types)), + str(len(g.schema.edge_types)), + # "({0}) {1}".format(len(vertex_schema), ",".join(vertex_schema)), + # "({0}) {1}".format(len(edge_schema), ",".join(edge_schema)), + ] + ) + + terminal_display(data) + + grpc_client = get_grpc_client() + response = grpc_client.list_interactive_graph() + + if is_success(response): + _construct_and_display_data(response.graphs) + return response.graphs + + +@get.command() +def job(): + """Display jobs in database""" + + def _construct_and_display_data(job_status): + if not job_status: + click.secho("no job found in database.", fg="blue") + return + + head = ["JOBID", "STATUS", "START TIME", "END TIME", "DETAIL", "MESSAGE"] + data = [head] + + for s in job_status: + detail = {} + for k, v in s.detail.items(): + detail[k] = v + + message = s.message.replace("\n", "") + if len(message) > 63: + message = message[:63] + "..." + + data.append( + [ + s.jobid, + s.status, + s.start_time, + s.end_time, + json.dumps(detail), + message, + ] + ) + + terminal_display(data) + + grpc_client = get_grpc_client() + response = grpc_client.list_interactive_job() + + if is_success(response): + _construct_and_display_data(response.job_status) + return response.job_status + + +@get.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +def procedure(graph): + """Display procedures in database""" + + def _construct_and_display_data(procedures): + if not procedures: + click.secho("no procedure found in database.", fg="blue") + return + + head = ["NAME", "TYPE", "ENABLE", "DESCRIPTION"] + data = [head] + + for procedure in procedures: + data.append( + [ + procedure.name, + procedure.type, + str(procedure.enable), + procedure.description, + ] + ) + + terminal_display(data) + + grpc_client = get_grpc_client() + response = grpc_client.list_interactive_procedure(graph) + + if is_success(response): + _construct_and_display_data(response.procedures) + return response.procedures + + +@get.command() +def service(): + """Display service status in database""" + + def _construct_and_display_data(service_status): + head = [ + "STATUS", + "SERVICE GRAPH", + "BOLT SERVICE ENDPOINT", + "HQPS SERVICE ENDPOINT", + ] + data = [head] + + data.append( + [ + service_status.status, + service_status.graph_name, + service_status.bolt_port, + service_status.hqps_port, + ] + ) + + terminal_display(data) + + grpc_client = get_grpc_client() + response = grpc_client.get_interactive_service_status() + + if is_success(response): + _construct_and_display_data(response.service_status) + return response.service_status + + +@get.command() +def node(): + """Display resource (CPU/memory) usage of nodes""" + + def _construct_and_display_data(nodes_status): + head = ["NAME", "CPU USAGE(%)", "MEMORY USAGE(%)", "DISK USAGE(%)"] + data = [head] + + for s in nodes_status: + data.append( + [ + s.node, + s.cpu_usage, + s.memory_usage, + s.disk_usage, + ] + ) + + terminal_display(data) + + grpc_client = get_grpc_client() + response = grpc_client.get_node_status() + + if is_success(response): + _construct_and_display_data(response.nodes_status) + return response.nodes_status + + +@describe.command() +@click.argument("graph", required=False) +def graph(graph): + """Show details of graph""" + + grpc_client = get_grpc_client() + response = grpc_client.list_interactive_graph() + + if is_success(response): + graphs = response.graphs + + if not graphs: + click.secho("no graph found in database.", fg="blue") + return + + specific_graph_exist = False + for g in graphs: + if graph is not None and g.name != graph: + continue + + # display + click.secho(yaml.dump(MessageToDict(g, preserving_proto_field_name=True))) + + if graph is not None and g.name == graph: + specific_graph_exist = True + break + + if graph is not None and not specific_graph_exist: + click.secho("graphs \"{0}\" not found.".format(graph), fg="blue") + + +if __name__ == "__main__": + cli() diff --git a/python/graphscope/gsctl/commands/utils.py b/python/graphscope/gsctl/commands/utils.py deleted file mode 100644 index 711a7b32e7f5..000000000000 --- a/python/graphscope/gsctl/commands/utils.py +++ /dev/null @@ -1,40 +0,0 @@ -#!/usr/bin/env python3 -# -*- coding: utf-8 -*- -# -# Copyright 2023 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 click - -from graphscope.gsctl.commands.common_command import cli as common_cli -from graphscope.gsctl.commands.dev_command import cli as dev_cli -from graphscope.gsctl.config import Context - - -def get_command_collection(context: Context): - if context is None: - # treat gsctl as an utility script, providing hepler functions or utilities. e.g. - # initialize and manage cluster, install the dependencies required to build graphscope locally - commands = click.CommandCollection(sources=[common_cli, dev_cli]) - - elif context.solution == "interactive": - commands = click.CommandCollection(sources=[common_cli]) - - else: - raise RuntimeError( - f"Failed to get command collection with context {context.name}" - ) - - return commands diff --git a/python/graphscope/gsctl/config.py b/python/graphscope/gsctl/config.py index e9357c0b4919..f38cbb903157 100644 --- a/python/graphscope/gsctl/config.py +++ b/python/graphscope/gsctl/config.py @@ -22,16 +22,19 @@ import random from string import ascii_letters -import yaml +from graphscope.gsctl.utils import read_yaml_file +from graphscope.gsctl.utils import write_yaml_file GS_CONFIG_DEFAULT_LOCATION = os.environ.get( "GSCONFIG", os.path.expanduser("~/.graphscope/config") ) +FLEX_INTERACTIVE = "Interactive" + class Context(object): def __init__(self, solution, coordinator_endpoint, name=None): - self.supported_solutions = ["interactive"] + self.supported_solutions = [FLEX_INTERACTIVE] if solution not in self.supported_solutions: raise RuntimeError( "The solution {0} in context {1} is not supported yet.".format( @@ -83,10 +86,10 @@ def set_and_write(self, context: Context): # write contexts = [v.to_dict() for _, v in self._contexts.items()] - with open(GS_CONFIG_DEFAULT_LOCATION, "w") as file: - yaml.dump( - {"contexts": contexts, "current-context": self._current_context}, file - ) + write_yaml_file( + {"contexts": contexts, "current-context": self._current_context}, + GS_CONFIG_DEFAULT_LOCATION, + ) def remove_and_write(self, current_context: Context): # remove @@ -95,10 +98,10 @@ def remove_and_write(self, current_context: Context): # write contexts = [v.to_dict() for _, v in self._contexts.items()] - with open(GS_CONFIG_DEFAULT_LOCATION, "w") as file: - yaml.dump( - {"contexts": contexts, "current-context": self._current_context}, file - ) + write_yaml_file( + {"contexts": contexts, "current-context": self._current_context}, + GS_CONFIG_DEFAULT_LOCATION, + ) class GSConfigLoader(object): @@ -126,15 +129,15 @@ def _parse_config(self, config_dict): if not current_context_exists: raise RuntimeError( - f"Current context {current_context} is not exists in config file {GS_CONFIG_DEFAULT_LOCATION}" + "Current context {0} is not exists in config file {1}".format( + current_context, GS_CONFIG_DEFAULT_LOCATION + ) ) return contexts, current_context def load_config(self): - config_dict = None - with open(self._config_file, "r") as file: - config_dict = yaml.safe_load(file) + config_dict = read_yaml_file(self._config_file) contexts, current_context = self._parse_config(config_dict) return GSConfig(contexts, current_context) @@ -149,8 +152,7 @@ def load_gs_config(): if not os.path.exists(config_file): workdir = os.path.dirname(config_file) os.makedirs(workdir, exist_ok=True) - with open(config_file, "w") as file: - yaml.safe_dump({}, file) + write_yaml_file({}, config_file) loader = GSConfigLoader(config_file) return loader.load_config() diff --git a/python/graphscope/gsctl/gsctl.py b/python/graphscope/gsctl/gsctl.py index 21d9dc7a69a7..a37b1b450ce9 100644 --- a/python/graphscope/gsctl/gsctl.py +++ b/python/graphscope/gsctl/gsctl.py @@ -25,7 +25,7 @@ import graphscope except ModuleNotFoundError: # if graphscope is not installed, only basic functions or utilities - # can be used, e.g. install dependencies + # can be used, e.g. install dependencies. graphscope = None diff --git a/python/graphscope/gsctl/rpc.py b/python/graphscope/gsctl/rpc.py deleted file mode 100644 index f77d0fe58bae..000000000000 --- a/python/graphscope/gsctl/rpc.py +++ /dev/null @@ -1,87 +0,0 @@ -#!/usr/bin/env python3 -# -*- coding: utf-8 -*- -# -# Copyright 2023 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 atexit -import time - -import click -import grpc - -from graphscope.client.utils import GS_GRPC_MAX_MESSAGE_LENGTH -from graphscope.client.utils import handle_grpc_error -from graphscope.gsctl.config import get_current_context -from graphscope.proto import coordinator_service_pb2_grpc -from graphscope.proto import message_pb2 -from graphscope.version import __version__ - - -class GRPCClient(object): - def __init__(self, endpoint): - # create the grpc stub - options = [ - ("grpc.max_send_message_length", GS_GRPC_MAX_MESSAGE_LENGTH), - ("grpc.max_receive_message_length", GS_GRPC_MAX_MESSAGE_LENGTH), - ("grpc.max_metadata_size", GS_GRPC_MAX_MESSAGE_LENGTH), - ] - self._channel = grpc.insecure_channel(endpoint, options=options) - self._stub = coordinator_service_pb2_grpc.CoordinatorServiceStub(self._channel) - - atexit.register(self.close) - - @handle_grpc_error - def _connect_impl(self, timeout_seconds): - begin_time = time.time() - - request = message_pb2.ConnectRequest(version=__version__) - while True: - try: - response = self._stub.Connect(request) - break - except Exception as e: - msg = f"code: {e.code().name}, details: {e.details()}" - click.secho( - f"Failed to connect coordinator: {e}, try after second...", - fg="yellow", - ) - if time.time() - begin_time >= timeout_seconds: - raise ConnectionError(f"Connect coordinator timeout, {msg}") - time.sleep(1) - - return response.solution - - def connect(self, timeout_seconds=10): - return self._connect_impl(timeout_seconds) - - def close(self): - try: - self._channel.close() - except: # noqa: E722 - pass - - -def get_grpc_client(coordinator_endpoint=None): - if coordinator_endpoint is not None: - return GRPCClient(coordinator_endpoint) - - # use the latest context in config file - current_context = get_current_context() - if current_context is None: - raise RuntimeError( - "No available context found, please connect to a launched coordinator first." - ) - return GRPCClient(current_context.coordinator_endpoint) diff --git a/python/graphscope/gsctl/utils.py b/python/graphscope/gsctl/utils.py new file mode 100644 index 000000000000..e0861acda132 --- /dev/null +++ b/python/graphscope/gsctl/utils.py @@ -0,0 +1,87 @@ +#!/usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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 os +from typing import List + +import yaml + + +def read_yaml_file(path): + """Reads YAML file and returns as a python object.""" + with open(path, "r") as file: + return yaml.safe_load(file) + + +def write_yaml_file(data, path): + """Writes python object to the YAML file.""" + with open(path, "w") as file: + yaml.dump(data, file) + + +def is_valid_file_path(path): + """Check if the path exists and corresponds to a regular file.""" + return os.path.exists(path) and os.path.isfile(path) + + +def dict_to_proto_message(values, message): + """Transform pyhon dict object to protobuf message + + Args: + values (dict): values to be transformed. + message (proto): protobuf message, such as graph_def_pb2.GraphMessage() + """ + + def _parse_list(values, message): + if isinstance(values[0], dict): # value needs to be further parsed + for v in values: + cmd = message.add() + _parse_dict(v, cmd) + else: # value can be set + message.extend(values) + + def _parse_dict(values, message): + for k, v in values.items(): + if isinstance(v, dict): # value needs to be further parsed + _parse_dict(v, getattr(message, k)) + elif isinstance(v, list): + _parse_list(v, getattr(message, k)) + else: # value can be set + try: + setattr(message, k, v) + except AttributeError: + # treat as map + message[k] = v + + _parse_dict(values, message) + return message + + +def terminal_display(data: List[list]): + """Display tablular data in terminal""" + + # Compute the maximum width for each column + column_widths = [max(len(str(item)) for item in column) for column in zip(*data)] + + # Display the data with aligned columns + for row in data: + print( + " ".join( + "{:<{}}".format(item, width) for item, width in zip(row, column_widths) + ) + ) From c561ae0b79b540481ac86ea2e9f7217a58892a00 Mon Sep 17 00:00:00 2001 From: Dongze Li Date: Thu, 21 Dec 2023 20:35:48 +0800 Subject: [PATCH 2/4] Refactor --- coordinator/gscoordinator/coordinator.py | 5 +- .../gscoordinator/servicer/__init__.py | 2 +- .../gscoordinator/servicer/base_service.py | 45 -- .../gscoordinator/servicer/flex/__init__.py | 17 + .../{ => flex}/interactive/__init__.py | 4 +- .../servicer/flex/interactive/hiactor.py | 57 ++ .../servicer/{interactive => flex}/job.py | 0 .../gscoordinator/servicer/flex/service.py | 104 +++ .../servicer/interactive/service.py | 449 ------------- proto/coordinator_service.proto | 8 +- proto/{interactive.proto => flex.proto} | 5 +- proto/message.proto | 1 - python/graphscope/gsctl/client/rpc.py | 85 +-- python/graphscope/gsctl/commands/__init__.py | 24 +- python/graphscope/gsctl/commands/common.py | 21 +- .../graphscope/gsctl/commands/interactive.py | 625 ------------------ python/graphscope/gsctl/config.py | 16 +- python/graphscope/gsctl/gsctl.py | 3 +- 18 files changed, 278 insertions(+), 1193 deletions(-) delete mode 100644 coordinator/gscoordinator/servicer/base_service.py create mode 100644 coordinator/gscoordinator/servicer/flex/__init__.py rename coordinator/gscoordinator/servicer/{ => flex}/interactive/__init__.py (89%) create mode 100644 coordinator/gscoordinator/servicer/flex/interactive/hiactor.py rename coordinator/gscoordinator/servicer/{interactive => flex}/job.py (100%) create mode 100644 coordinator/gscoordinator/servicer/flex/service.py delete mode 100644 coordinator/gscoordinator/servicer/interactive/service.py rename proto/{interactive.proto => flex.proto} (98%) delete mode 100644 python/graphscope/gsctl/commands/interactive.py diff --git a/coordinator/gscoordinator/coordinator.py b/coordinator/gscoordinator/coordinator.py index adf154c9a8e6..0679f694f5cb 100644 --- a/coordinator/gscoordinator/coordinator.py +++ b/coordinator/gscoordinator/coordinator.py @@ -31,7 +31,7 @@ from graphscope.proto import coordinator_service_pb2_grpc from gscoordinator.monitor import Monitor -from gscoordinator.servicer import init_interactive_service_servicer +from gscoordinator.servicer import init_flex_service_servicer from gscoordinator.servicer import init_graphscope_one_service_servicer from gscoordinator.utils import GS_GRPC_MAX_MESSAGE_LENGTH @@ -111,10 +111,9 @@ def get_servicer(config: Config): """Get servicer of specified solution under FLEX architecture""" service_initializers = { "GraphScope One": init_graphscope_one_service_servicer, - "Interactive": init_interactive_service_servicer, } - initializer = service_initializers.get(config.solution) + initializer = service_initializers.get(config.solution, init_flex_service_servicer) if initializer is None: raise RuntimeError( f"Expect {service_initializers.keys()} of solution parameter" diff --git a/coordinator/gscoordinator/servicer/__init__.py b/coordinator/gscoordinator/servicer/__init__.py index 9b169d1e23d0..46ecd123672b 100644 --- a/coordinator/gscoordinator/servicer/__init__.py +++ b/coordinator/gscoordinator/servicer/__init__.py @@ -16,5 +16,5 @@ # limitations under the License. # -from gscoordinator.servicer.interactive.service import * +from gscoordinator.servicer.flex.service import * from gscoordinator.servicer.graphscope_one.service import * diff --git a/coordinator/gscoordinator/servicer/base_service.py b/coordinator/gscoordinator/servicer/base_service.py deleted file mode 100644 index 895ea53fb3dd..000000000000 --- a/coordinator/gscoordinator/servicer/base_service.py +++ /dev/null @@ -1,45 +0,0 @@ -#! /usr/bin/env python3 -# -*- coding: utf-8 -*- -# -# Copyright 2023 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 atexit -import logging - -from graphscope.config import Config -from graphscope.proto import coordinator_service_pb2_grpc -from graphscope.proto import message_pb2 - - -class BaseServiceServicer(coordinator_service_pb2_grpc.CoordinatorServiceServicer): - """Base class of coordinator service""" - - def __init__(self, config: Config): - self._config = config - atexit.register(self.cleanup) - - def __del__(self): - self.cleanup() - - def Connect(self, request, context): - return message_pb2.ConnectResponse(solution=self._config.solution) - - @property - def launcher_type(self): - return self._config.launcher_type - - def cleanup(self): - pass diff --git a/coordinator/gscoordinator/servicer/flex/__init__.py b/coordinator/gscoordinator/servicer/flex/__init__.py new file mode 100644 index 000000000000..7c0d26b525f8 --- /dev/null +++ b/coordinator/gscoordinator/servicer/flex/__init__.py @@ -0,0 +1,17 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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. +# diff --git a/coordinator/gscoordinator/servicer/interactive/__init__.py b/coordinator/gscoordinator/servicer/flex/interactive/__init__.py similarity index 89% rename from coordinator/gscoordinator/servicer/interactive/__init__.py rename to coordinator/gscoordinator/servicer/flex/interactive/__init__.py index 3a8204719d16..7a63c67de21c 100644 --- a/coordinator/gscoordinator/servicer/interactive/__init__.py +++ b/coordinator/gscoordinator/servicer/flex/interactive/__init__.py @@ -21,6 +21,8 @@ try: sys.path.insert(0, os.path.dirname(__file__)) - import interactive_client + import hiactor_client except ImportError: raise + +from gscoordinator.servicer.flex.interactive.hiactor import * diff --git a/coordinator/gscoordinator/servicer/flex/interactive/hiactor.py b/coordinator/gscoordinator/servicer/flex/interactive/hiactor.py new file mode 100644 index 000000000000..b6a6699cf1ed --- /dev/null +++ b/coordinator/gscoordinator/servicer/flex/interactive/hiactor.py @@ -0,0 +1,57 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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 os +import logging + +import hiactor_client + +from graphscope.config import Config + +__all__ = ["init_interactive_service"] + +logger = logging.getLogger("graphscope") + + +class Hiactor(object): + """Hiactor module used to interact with hiactor engine""" + + def __init__(self, config: Config): + self._config = config + # hiactor admin service host + self._hiactor_host = self._get_hiactor_service_endpoints() + logger.info("Connect to hiactor service at %s", self._hiactor_host) + + def _get_hiactor_service_endpoints(self): + if self._config.launcher_type == "hosts": + # TODO change to 127.0.0.1 + endpoint = "http://192.168.0.9:{0}".format( + os.environ.get("HIACTOR_ADMIN_SERVICE_PORT", 7777) + ) + return endpoint + + def list_graph(self): + with hiactor_client.ApiClient( + hiactor_client.Configuration(self._hiactor_host) + ) as api_client: + api_instance = hiactor_client.GraphApi(api_client) + return api_instance.list_graphs() + + +def init_interactive_service(config: Config): + return Hiactor(config) diff --git a/coordinator/gscoordinator/servicer/interactive/job.py b/coordinator/gscoordinator/servicer/flex/job.py similarity index 100% rename from coordinator/gscoordinator/servicer/interactive/job.py rename to coordinator/gscoordinator/servicer/flex/job.py diff --git a/coordinator/gscoordinator/servicer/flex/service.py b/coordinator/gscoordinator/servicer/flex/service.py new file mode 100644 index 000000000000..f8d9cfff2b64 --- /dev/null +++ b/coordinator/gscoordinator/servicer/flex/service.py @@ -0,0 +1,104 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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. +# + +"""Service under FLEX Architecture""" + +import atexit + +# import itertools +import logging +import os +import threading + +# from google.protobuf.json_format import MessageToDict +from graphscope.config import Config +from graphscope.gsctl.utils import dict_to_proto_message +from graphscope.proto import coordinator_service_pb2_grpc +from graphscope.proto import error_codes_pb2 +from graphscope.proto import flex_pb2 +from graphscope.proto import message_pb2 + +# from gscoordinator.scheduler import schedule +# from gscoordinator.utils import WORKSPACE +# from gscoordinator.utils import delegate_command_to_pod +# from gscoordinator.utils import run_kube_cp_command + +from gscoordinator.servicer.flex.interactive import * + +__all__ = ["FlexServiceServicer", "init_flex_service_servicer"] + +logger = logging.getLogger("graphscope") + + +class FlexServiceServicer(coordinator_service_pb2_grpc.CoordinatorServiceServicer): + """Service under flex architecture.""" + + services_initializer = { + "interactive": init_interactive_service, + } + + def __init__(self, config: Config): + self._config = config + # We use the solution encompasses the various applications and use cases of + # the product across different industries and business scenarios, + # e.g. interactive, analytical + self._solution = self._config.solution.lower() + + atexit.register(self.cleanup) + + # lock to protect the service + self._lock = threading.RLock() + + # initialize specific service client + self._service_client = self._initialize_service_client() + + def __del__(self): + self.cleanup() + + def _initialize_service_client(self): + initializer = self.services_initializer.get(self._solution) + if initializer is None: + raise RuntimeError("Failed to launch {0} service".format(self._solution)) + return initializer(self._config) + + def cleanup(self): + pass + + def Connect(self, request, context): + return message_pb2.ConnectResponse(solution=self._solution) + + def ListGraph(self, request, context): + try: + graphs = self._service_client.list_graph() + except Exception as e: + logger.warning("Failed to list graph: %s", str(e)) + return flex_pb2.ListGraphResponse( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + else: + return flex_pb2.ListGraphResponse( + code=error_codes_pb2.OK, + graphs=[ + dict_to_proto_message(g.to_dict(), flex_pb2.GraphProto()) + for g in graphs + ], + ) + + +def init_flex_service_servicer(config: Config): + return FlexServiceServicer(config) diff --git a/coordinator/gscoordinator/servicer/interactive/service.py b/coordinator/gscoordinator/servicer/interactive/service.py deleted file mode 100644 index 2e883e1119e7..000000000000 --- a/coordinator/gscoordinator/servicer/interactive/service.py +++ /dev/null @@ -1,449 +0,0 @@ -#! /usr/bin/env python3 -# -*- coding: utf-8 -*- -# -# Copyright 2023 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. -# - -"""Interactive Service under FLEX Architecture""" - -import itertools -import logging -import os -import threading - -import interactive_client -from google.protobuf.json_format import MessageToDict -from graphscope.config import Config -from graphscope.gsctl.utils import dict_to_proto_message -from graphscope.proto import error_codes_pb2 -from graphscope.proto import interactive_pb2 - -from gscoordinator.scheduler import schedule -from gscoordinator.servicer.base_service import BaseServiceServicer -from gscoordinator.servicer.interactive.job import GraphImportScheduler -from gscoordinator.utils import WORKSPACE -from gscoordinator.utils import delegate_command_to_pod -from gscoordinator.utils import run_kube_cp_command - -__all__ = ["InteractiveServiceServicer", "init_interactive_service_servicer"] - -logger = logging.getLogger("graphscope") - - -# There are two workspaces for FLEX Interactive, one residing on "coordinator" node -# and the other on the "interactive" node. These two workspaces are equivalent when -# running mode is "hosts". Hence, INTERACTIVE_WORKSPACE is only effective in -# Kubernetes (k8s) environment. -INTERACTIVE_WORKSPACE = ( - os.environ["INTERACTIVE_WORKSPACE"] - if "INTERACTIVE_WORKSPACE" in os.environ - else "/tmp" -) - - -INTERACTIVE_CONTAINER_NAME = "interactive" - - -class InteractiveServiceServicer(BaseServiceServicer): - """Interactive service under flex architecture.""" - - def __init__(self, config: Config): - super().__init__(config) - - # lock to protect the service - self._lock = threading.RLock() - - # interactive host - self._interactive_host = None - self._fetch_interactive_host_impl() - self._fetch_interactive_host_job = schedule.every(60).seconds.do( - self._fetch_interactive_host_impl - ) - - # interactive pod list - self._interactive_pod_list = [] - - # job status - self._job_status = {} - - def cleanup(self): - pass - - def _fetch_interactive_host_impl(self): - # TODO: get interactive service endpoint by instance id - self._interactive_host = "http://47.242.172.5:7777" - - @property - def interactive_host(self): - return self._interactive_host - - def register_job_status(self, jobid, status): - self._job_status[jobid] = status - - def write_and_distribute_file(self, graph, location, raw_data): - """ - Args: - graph(str): graph name. - location(str): file location over coordinator node. - raw_data(bytes): binary of file data. - """ - # format: _ - filename = "{0}_{1}".format(graph, os.path.basename(location)) - - # write temp file - tmp_file = os.path.join(WORKSPACE, filename) - with open(tmp_file, "wb") as f: - f.write(raw_data) - - # distribute - target_file = tmp_file - if self.launcher_type == "k8s": - # update file path in interactive pod - target_file = os.path.join(INTERACTIVE_WORKSPACE, filename) - - for pod in self._interactive_pod_list: - container = INTERACTIVE_CONTAINER_NAME - cmd = f"mkdir -p {os.path.dirname(target_file)}" - logger.debug(delegate_command_to_pod(cmd, pod, container)) - logger.debug( - run_kube_cp_command(tmp_file, target_file, pod, container, True) - ) - - return target_file - - def CreateInteractiveGraph(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - # create an instance of the API class - api_instance = interactive_client.GraphApi(api_client) - - graph_def_dict = MessageToDict( - request.graph_def, preserving_proto_field_name=True - ) - graph = interactive_client.Graph.from_dict(graph_def_dict) - - try: - api_response = api_instance.create_graph(graph) - except Exception as e: - logger.warning("Failed to create interactive graph. %s", str(e)) - return interactive_pb2.ApiResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - return interactive_pb2.ApiResponse( - code=error_codes_pb2.OK, error_msg=api_response.message - ) - - def RemoveInteractiveGraph(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - api_instance = interactive_client.GraphApi(api_client) - - try: - api_response = api_instance.delete_graph(request.graph_name) - except Exception as e: - logger.warning("Failed to remove interactive graph. %s", str(e)) - return interactive_pb2.ApiResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - return interactive_pb2.ApiResponse( - code=error_codes_pb2.OK, error_msg=api_response.message - ) - - def ListInteractiveGraph(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - api_instance = interactive_client.GraphApi(api_client) - - try: - api_response = api_instance.list_graphs() - except Exception as e: - logger.warning("Failed to list interactive graph. %s", str(e)) - return interactive_pb2.ListInteractiveGraphResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - return interactive_pb2.ListInteractiveGraphResponse( - code=error_codes_pb2.OK, - graphs=[ - dict_to_proto_message(g.to_dict(), interactive_pb2.GraphProto()) - for g in api_response - ], - ) - - def ImportInteractiveGraph(self, request, context): - # write raw data to file and copy to interactive workspace - try: - schema_mapping_proto = request.schema_mapping - for mapping in itertools.chain( - schema_mapping_proto.vertex_mappings, schema_mapping_proto.edge_mappings - ): - raw_data_index = 0 - for index, location in enumerate(mapping.inputs): - if location.startswith("@"): - # write raw data and distribute file to interactive workspace - new_location = self.write_and_distribute_file( - schema_mapping_proto.graph, - location, - mapping.raw_data[raw_data_index], - ) - raw_data_index += 1 - # update the location - mapping.inputs[index] = new_location - except Exception as e: - logger.warning("Failed to distribute file. %s", str(e)) - return interactive_pb2.ApiResponse( - code=error_codes_pb2.NETWORK_ERROR, error_msg=str(e) - ) - - # transform proto to dict - schema_mapping = MessageToDict( - schema_mapping_proto, preserving_proto_field_name=True - ) - - # create job scheduler - scheduler = GraphImportScheduler( - at_time="now", - repeat=None, - schema_mapping=schema_mapping, - servicer=self, - ) - scheduler.start() - - return interactive_pb2.ApiResponse(code=error_codes_pb2.OK) - - def ListInteractiveJob(self, request, context): - return interactive_pb2.ListInteractiveJobResponse( - code=error_codes_pb2.OK, - job_status=[ - dict_to_proto_message(s.to_dict(), interactive_pb2.JobStatus()) - for _, s in self._job_status.items() - ], - ) - - def CreateInteractiveProcedure(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - # create an instance of the API class - api_instance = interactive_client.ProcedureApi(api_client) - - # transform proto to dict - procedure_def_dict = MessageToDict( - request.procedure_def, preserving_proto_field_name=True - ) - - graph_name = procedure_def_dict["bound_graph"] - procedure = interactive_client.Procedure.from_dict(procedure_def_dict) - - try: - api_response = api_instance.create_procedure(graph_name, procedure) - except Exception as e: - logger.warning("Failed to create procedure. %s", str(e)) - return interactive_pb2.ApiResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - return interactive_pb2.ApiResponse( - code=error_codes_pb2.OK, error_msg=api_response.message - ) - - def ListInteractiveProcedure(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - api_instance = interactive_client.ProcedureApi(api_client) - - try: - api_response = api_instance.list_procedures(request.graph_name) - except Exception as e: - logger.warning("Failed to list procedure. %s", str(e)) - return interactive_pb2.ListInteractiveProcedureResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - # change "returns" in the openapi definition to "rlts_meta", - # as "returns" is a reserved keyword in proto - procedures = [] - for p in api_response: - procedure_dict = p.to_dict() - # "returns" -> "rlts_meta" - procedure_dict["rlts_meta"] = procedure_dict.pop("returns") - # push - procedures.append(procedure_dict) - - return interactive_pb2.ListInteractiveProcedureResponse( - code=error_codes_pb2.OK, - procedures=[ - dict_to_proto_message(p, interactive_pb2.Procedure()) - for p in procedures - ], - ) - - def UpdateInteractiveProcedure(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - api_instance = interactive_client.ProcedureApi(api_client) - - for p in request.procedures: - # transform proto to dict - procedure_def_dict = MessageToDict(p, preserving_proto_field_name=True) - - graph_name = procedure_def_dict["bound_graph"] - procedure_name = procedure_def_dict["name"] - procedure = interactive_client.Procedure.from_dict(procedure_def_dict) - - try: - api_response = api_instance.update_procedure( - graph_name, procedure_name, procedure - ) - except Exception as e: - logger.warning("Failed to update procedure. %s", str(e)) - return interactive_pb2.ApiResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - - return interactive_pb2.ApiResponse(code=error_codes_pb2.OK) - - def RemoveInteractiveProcedure(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - api_instance = interactive_client.ProcedureApi(api_client) - - try: - api_response = api_instance.delete_procedure( - request.graph_name, request.procedure_name - ) - except Exception as e: - logger.warning("Failed to remove procedure. %s", str(e)) - return interactive_pb2.ApiResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - return interactive_pb2.ApiResponse( - code=error_codes_pb2.OK, error_msg=api_response.message - ) - - def GetInteractiveServiceStatus(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - api_instance = interactive_client.ServiceApi(api_client) - - try: - api_response = api_instance.get_service_status() - except Exception as e: - logger.warning("Failed to get service status. %s", str(e)) - return interactive_pb2.GetInteractiveServiceStatusResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - return interactive_pb2.GetInteractiveServiceStatusResponse( - code=error_codes_pb2.OK, - service_status=dict_to_proto_message( - api_response.to_dict(), interactive_pb2.ServiceStatus() - ), - ) - - def StartInteractiveService(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - api_instance = interactive_client.ServiceApi(api_client) - - service_def_dict = MessageToDict( - request.service_def, preserving_proto_field_name=True - ) - service = interactive_client.Service.from_dict(service_def_dict) - - try: - api_response = api_instance.start_service(service) - except Exception as e: - logger.warning("Failed to start service. %s", str(e)) - return interactive_pb2.ApiResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - return interactive_pb2.ApiResponse( - code=error_codes_pb2.OK, error_msg=api_response.message - ) - - def StopInteractiveService(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - api_instance = interactive_client.ServiceApi(api_client) - - try: - api_response = api_instance.stop_service() - except Exception as e: - logger.warning("Failed to stop service. %s", str(e)) - return interactive_pb2.ApiResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - return interactive_pb2.ApiResponse( - code=error_codes_pb2.OK, error_msg=api_response.message - ) - - def RestartInteractiveService(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - api_instance = interactive_client.ServiceApi(api_client) - - try: - api_response = api_instance.restart_service() - except Exception as e: - logger.warning("Failed to restart service. %s", str(e)) - return interactive_pb2.ApiResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - return interactive_pb2.ApiResponse( - code=error_codes_pb2.OK, error_msg=api_response.message - ) - - def GetNodeStatus(self, request, context): - with interactive_client.ApiClient( - interactive_client.Configuration(self._interactive_host) - ) as api_client: - api_instance = interactive_client.NodeApi(api_client) - - try: - api_response = api_instance.get_node_status() - except Exception as e: - logger.warning("Failed to get node status. %s", str(e)) - return interactive_pb2.GetNodeStatusResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - return interactive_pb2.GetNodeStatusResponse( - code=error_codes_pb2.OK, - nodes_status=[ - dict_to_proto_message(s.to_dict(), interactive_pb2.NodeStatus()) - for s in api_response - ], - ) - - -def init_interactive_service_servicer(config: Config): - return InteractiveServiceServicer(config) diff --git a/proto/coordinator_service.proto b/proto/coordinator_service.proto index d721bcd056c8..6102cb5ba7a2 100644 --- a/proto/coordinator_service.proto +++ b/proto/coordinator_service.proto @@ -16,8 +16,8 @@ syntax = "proto3"; package gs.rpc; +import "flex.proto"; import "message.proto"; -import "interactive.proto"; service CoordinatorService { // Connect a session. @@ -53,12 +53,14 @@ service CoordinatorService { // service functions under FLEX architecture rpc Connect(ConnectRequest) returns (ConnectResponse); + rpc ListGraph(ListGraphRequest) returns (ListGraphResponse); + + // end + rpc CreateInteractiveGraph(CreateInteractiveGraphRequest) returns (ApiResponse); rpc RemoveInteractiveGraph(RemoveInteractiveGraphRequest) returns (ApiResponse); - rpc ListInteractiveGraph(ListInteractiveGraphRequest) returns (ListInteractiveGraphResponse); - rpc ImportInteractiveGraph(ImportInteractiveGraphRequest) returns (ApiResponse); rpc ListInteractiveJob(ListInteractiveJobRequest) returns (ListInteractiveJobResponse); diff --git a/proto/interactive.proto b/proto/flex.proto similarity index 98% rename from proto/interactive.proto rename to proto/flex.proto index 6c5fcea45c83..d90afd374f22 100644 --- a/proto/interactive.proto +++ b/proto/flex.proto @@ -19,6 +19,7 @@ package gs.rpc; import "error_codes.proto"; + // property type message PropertyTypeProto { string primitive_type = 1; // [ DT_DOUBLE, DT_STRING, DT_SIGNED_INT32, DT_SIGNED_INT64 ] @@ -210,9 +211,9 @@ message RemoveInteractiveGraphRequest { string graph_name = 1; } -message ListInteractiveGraphRequest {} +message ListGraphRequest {} -message ListInteractiveGraphResponse { +message ListGraphResponse { Code code = 1; optional string error_msg = 2; diff --git a/proto/message.proto b/proto/message.proto index caf6859709d7..5b839c01e1b9 100644 --- a/proto/message.proto +++ b/proto/message.proto @@ -247,7 +247,6 @@ message CloseLearningInstanceResponse { }; - //////////////////////////////////////////////////////////////////////////////// // // Protos under the FLEX architecture diff --git a/python/graphscope/gsctl/client/rpc.py b/python/graphscope/gsctl/client/rpc.py index 0b448cf06900..94ea255d263b 100644 --- a/python/graphscope/gsctl/client/rpc.py +++ b/python/graphscope/gsctl/client/rpc.py @@ -27,13 +27,14 @@ from graphscope.gsctl.config import GS_CONFIG_DEFAULT_LOCATION from graphscope.gsctl.config import get_current_context from graphscope.proto import coordinator_service_pb2_grpc -from graphscope.proto import interactive_pb2 +from graphscope.proto import flex_pb2 from graphscope.proto import message_pb2 from graphscope.version import __version__ class GRPCClient(object): def __init__(self, endpoint): + self._endpoint = endpoint # create the grpc stub options = [ ("grpc.max_send_message_length", GS_GRPC_MAX_MESSAGE_LENGTH), @@ -57,18 +58,30 @@ def _connect_impl(self, timeout_seconds): except Exception as e: msg = f"code: {e.code().name}, details: {e.details()}" click.secho( - f"Failed to connect coordinator: {e}, try after second...", + f"Couldn't connect to current server: {self._endpoint}: i/o timeout", fg="yellow", ) if time.time() - begin_time >= timeout_seconds: - raise ConnectionError(f"Connect coordinator timeout, {msg}") - time.sleep(1) - + click.secho(f"Unable to connect to server: {msg}", fg="yellow") + return None + time.sleep(8) return response.solution - def connect(self, timeout_seconds=10): + @property + def coordinator_endpoint(self): + return self._endpoint + + def connect(self, timeout_seconds=24): return self._connect_impl(timeout_seconds) + def connection_available(self): + try: + request = message_pb2.ConnectRequest(version=__version__) + response = self._stub.Connect(request) + return response.solution + except: # noqa: E722 + return None + def close(self): try: self._channel.close() @@ -76,81 +89,75 @@ def close(self): pass @handle_grpc_error - def create_interactive_graph(self, graph_def: interactive_pb2.GraphProto): - request = interactive_pb2.CreateInteractiveGraphRequest(graph_def=graph_def) + def create_interactive_graph(self, graph_def: flex_pb2.GraphProto): + request = flex_pb2.CreateInteractiveGraphRequest(graph_def=graph_def) return self._stub.CreateInteractiveGraph(request) @handle_grpc_error def remove_interactive_graph(self, graph: str): - request = interactive_pb2.RemoveInteractiveGraphRequest(graph_name=graph) + request = flex_pb2.RemoveInteractiveGraphRequest(graph_name=graph) return self._stub.RemoveInteractiveGraph(request) @handle_grpc_error - def list_interactive_graph(self): - request = interactive_pb2.ListInteractiveGraphRequest() - return self._stub.ListInteractiveGraph(request) + def list_graph(self): + request = flex_pb2.ListGraphRequest() + return self._stub.ListGraph(request) @handle_grpc_error - def import_interactive_graph(self, schema_mapping: interactive_pb2.SchemaMapping): - request = interactive_pb2.ImportInteractiveGraphRequest( - schema_mapping=schema_mapping - ) + def import_interactive_graph(self, schema_mapping: flex_pb2.SchemaMapping): + request = flex_pb2.ImportInteractiveGraphRequest(schema_mapping=schema_mapping) return self._stub.ImportInteractiveGraph(request) @handle_grpc_error def list_interactive_job(self): - request = interactive_pb2.ListInteractiveJobRequest() + request = flex_pb2.ListInteractiveJobRequest() return self._stub.ListInteractiveJob(request) @handle_grpc_error - def create_interactive_procedure(self, procedure: interactive_pb2.Procedure): - request = interactive_pb2.CreateInteractiveProcedureRequest( - procedure_def=procedure - ) + def create_interactive_procedure(self, procedure: flex_pb2.Procedure): + request = flex_pb2.CreateInteractiveProcedureRequest(procedure_def=procedure) return self._stub.CreateInteractiveProcedure(request) @handle_grpc_error def list_interactive_procedure(self, graph: str): - request = interactive_pb2.ListInteractiveProcedureRequest(graph_name=graph) + request = flex_pb2.ListInteractiveProcedureRequest(graph_name=graph) return self._stub.ListInteractiveProcedure(request) @handle_grpc_error - def update_interactive_procedure(self, procedures: List[interactive_pb2.Procedure]): - request = interactive_pb2.UpdateInteractiveProcedureRequest( - procedures=procedures - ) + def update_interactive_procedure(self, procedures: List[flex_pb2.Procedure]): + request = flex_pb2.UpdateInteractiveProcedureRequest(procedures=procedures) return self._stub.UpdateInteractiveProcedure(request) @handle_grpc_error def remove_interactive_procedure(self, graph: str, procedure: str): - request = interactive_pb2.RemoveInteractiveProcedureRequest( + request = flex_pb2.RemoveInteractiveProcedureRequest( graph_name=graph, procedure_name=procedure ) return self._stub.RemoveInteractiveProcedure(request) @handle_grpc_error def get_interactive_service_status(self): - request = interactive_pb2.GetInteractiveServiceStatusRequest() + request = flex_pb2.GetInteractiveServiceStatusRequest() return self._stub.GetInteractiveServiceStatus(request) @handle_grpc_error - def start_interactive_service(self, service: interactive_pb2.Service): - request = interactive_pb2.StartInteractiveServiceRequest(service_def=service) + def start_interactive_service(self, service: flex_pb2.Service): + request = flex_pb2.StartInteractiveServiceRequest(service_def=service) return self._stub.StartInteractiveService(request) @handle_grpc_error def stop_interactive_service(self): - request = interactive_pb2.StopInteractiveServiceRequest() + request = flex_pb2.StopInteractiveServiceRequest() return self._stub.StopInteractiveService(request) @handle_grpc_error def restart_interactive_service(self): - request = interactive_pb2.RestartInteractiveServiceRequest() + request = flex_pb2.RestartInteractiveServiceRequest() return self._stub.RestartInteractiveService(request) @handle_grpc_error def get_node_status(self): - request = interactive_pb2.GetNodeStatusRequest() + request = flex_pb2.GetNodeStatusRequest() return self._stub.GetNodeStatus(request) @@ -161,9 +168,13 @@ def get_grpc_client(coordinator_endpoint=None): # use the latest context in config file current_context = get_current_context() if current_context is None: - raise RuntimeError( - "No available context found in {0}, please connect to a launched coordinator first.".format( - GS_CONFIG_DEFAULT_LOCATION - ) + command = "gsctl connect --coordinator-endpoint " + click.secho( + "No available context found, you may want to connect to a coordinator by: {0}".format( + command + ), + fg="blue", ) + return None + return GRPCClient(current_context.coordinator_endpoint) diff --git a/python/graphscope/gsctl/commands/__init__.py b/python/graphscope/gsctl/commands/__init__.py index 8bdcbed90f5f..ca5ab0dc8a5f 100644 --- a/python/graphscope/gsctl/commands/__init__.py +++ b/python/graphscope/gsctl/commands/__init__.py @@ -17,23 +17,33 @@ # import click +import sys -from graphscope.gsctl.commands.common import cli as common_cli -from graphscope.gsctl.commands.dev import cli as dev_cli -from graphscope.gsctl.commands.interactive import cli as interactive_cli from graphscope.gsctl.config import Context -from graphscope.gsctl.config import FLEX_INTERACTIVE + +from graphscope.gsctl.commands.common import cli as common +from graphscope.gsctl.commands.dev import cli as dev +from graphscope.gsctl.commands.interactive import cli as interactive +from graphscope.gsctl.client.rpc import get_grpc_client def get_command_collection(context: Context): if context is None: # treat gsctl as an utility script, providing hepler functions or utilities. e.g. # initialize and manage cluster, install the dependencies required to build graphscope locally - commands = click.CommandCollection(sources=[common_cli, dev_cli]) + return click.CommandCollection(sources=[common, dev]) - elif context.solution == FLEX_INTERACTIVE: - commands = click.CommandCollection(sources=[common_cli, interactive_cli]) + grpc_client = get_grpc_client(context.coordinator_endpoint) + solution = grpc_client.connection_available() + # in general, we should use 'solution' returned from the coordinator + # to determine the behavior in gsctl, but sometimes coordinator may crash + # or be closed manually, thus, we use the 'solution' exists in the client + # as default. + if solution is None: + solution = context.solution + if solution == "interactive": + commands = click.CommandCollection(sources=[common, interactive]) else: raise RuntimeError( f"Failed to get command collection with context {context.name}" diff --git a/python/graphscope/gsctl/commands/common.py b/python/graphscope/gsctl/commands/common.py index c01a2986def4..fe9e4488cf40 100644 --- a/python/graphscope/gsctl/commands/common.py +++ b/python/graphscope/gsctl/commands/common.py @@ -22,6 +22,7 @@ from graphscope.gsctl.config import Context from graphscope.gsctl.config import load_gs_config +from graphscope.gsctl.config import GS_CONFIG_DEFAULT_LOCATION from graphscope.gsctl.client.rpc import get_grpc_client @@ -33,23 +34,33 @@ def cli(): @cli.command() @click.option( + "-c", "--coordinator-endpoint", - help="Coordinator endpoint which gsctl connect to, e.g. http://127.0.0.1:9527", + help="Coordinator endpoint which gsctl connect to, e.g. 127.0.0.1:9527", ) def connect(coordinator_endpoint): - """Connect to the launched coordinator by ~/.graphscope/config + """Connect to a launched coordinator If '--coordinator-endpoint' is specified, use it as the current context and override the config file. """ - if coordinator_endpoint is not None: + grpc_client = get_grpc_client(coordinator_endpoint) + if grpc_client is None: + return + + if coordinator_endpoint is None: click.secho( - f"Connect to the coordinator at {coordinator_endpoint}.", fg="green" + "Find context from {0}, connect to coordinator at {1}".format( + GS_CONFIG_DEFAULT_LOCATION, grpc_client.coordinator_endpoint + ), + fg="blue", ) - grpc_client = get_grpc_client(coordinator_endpoint) solution = grpc_client.connect() + if solution is None: + return + # coordinator connected, set the context if coordinator_endpoint is not None: context = Context(solution=solution, coordinator_endpoint=coordinator_endpoint) config = load_gs_config() diff --git a/python/graphscope/gsctl/commands/interactive.py b/python/graphscope/gsctl/commands/interactive.py deleted file mode 100644 index 7381453a4802..000000000000 --- a/python/graphscope/gsctl/commands/interactive.py +++ /dev/null @@ -1,625 +0,0 @@ -#!/usr/bin/env python3 -# -*- coding: utf-8 -*- -# -# Copyright 2023 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. -# - -"""Group of interactive commands under the FLEX architecture""" - -import itertools -import json - -import click -import yaml -from google.protobuf.json_format import MessageToDict - -from graphscope.gsctl.client.rpc import get_grpc_client -from graphscope.gsctl.utils import dict_to_proto_message -from graphscope.gsctl.utils import is_valid_file_path -from graphscope.gsctl.utils import read_yaml_file -from graphscope.gsctl.utils import terminal_display -from graphscope.proto import error_codes_pb2 -from graphscope.proto import interactive_pb2 - - -@click.group() -def cli(): - pass - - -@cli.group() -def get(): - """Display one or many resources""" - pass - - -@cli.group() -def describe(): - """Show details of a specific resource or group of resources""" - pass - - -@cli.group() -def create(): - """Create a resource from a file or from stdin""" - pass - - -@cli.group() -def delete(): - """Delete resource by name""" - pass - - -@cli.group() -def enable(): - """Enable stored procedures over a given graph""" - pass - - -@cli.group() -def disable(): - """Disable stored procedures over a given graph""" - pass - - -@cli.group() -def service(): - """Start, restart or stop the database service""" - pass - - -def is_success(response): - if response.code != error_codes_pb2.OK: - click.secho( - "{0}: {1}".format( - error_codes_pb2.Code.Name(response.code), response.error_msg - ), - fg="red", - ) - return False - return True - - -@cli.command(name="import") -@click.option( - "-g", - "--graph", - required=True, - help="The name of the graph.", -) -@click.option( - "-c", - "--config", - required=True, - help="Yaml path or raw data for loading graph.", -) -def data_import(graph, config): - """Load the raw data specified in bulk load file""" - - def _read_and_fill_raw_data(config): - for mapping in itertools.chain( - config["vertex_mappings"], config["edge_mappings"] - ): - for index, location in enumerate(mapping["inputs"]): - # location is one of: - # 1) protocol:///path/to/the/file - # 2) @/path/to/the/file, which represents the local file - if location.startswith("@"): - if "raw_data" not in mapping: - mapping["raw_data"] = [] - - # read file and set raw data - with open(location[1:], "rb") as f: - content = f.read() - mapping["raw_data"].append(content) - - schema_mapping_dict = config - if is_valid_file_path(config): - schema_mapping_dict = read_yaml_file(config) - - if graph is not None: - schema_mapping_dict["graph"] = graph - - _read_and_fill_raw_data(schema_mapping_dict) - - # transfiorm dict to proto message - schema_mapping = interactive_pb2.SchemaMapping() - dict_to_proto_message(schema_mapping_dict, schema_mapping) - - grpc_client = get_grpc_client() - response = grpc_client.import_interactive_graph(schema_mapping) - - if is_success(response): - click.secho("Create dataloading job successfully.", fg="green") - - -@create.command() -@click.option( - "-g", - "--graph", - required=True, - help="The name of the graph", -) -@click.option( - "-c", - "--config", - required=True, - help="Yaml path or json string of schema for the graph", -) -def graph(graph, config): - """Create a graph in database, with the provided schema file""" - - graph_def_dict = config - if is_valid_file_path(config): - graph_def_dict = read_yaml_file(config) - - # override graph name - if graph is not None: - graph_def_dict["name"] = graph - - # transform graph dict to proto message - graph_def = interactive_pb2.GraphProto() - dict_to_proto_message(graph_def_dict, graph_def) - - grpc_client = get_grpc_client() - response = grpc_client.create_interactive_graph(graph_def) - - if is_success(response): - click.secho( - "Create interactive graph {0} successfully.".format(graph_def_dict["name"]), - fg="green", - ) - - -@create.command() -@click.option( - "-g", - "--graph", - required=True, - help="The name of the graph", -) -@click.option( - "-n", - "--name", - required=True, - help="The name of the procedure", -) -@click.option( - "-i", - "--sourcefile", - required=True, - help="Path of [ .cc, .cypher ] file", -) -@click.option( - "-d", - "--description", - required=False, - help="Description for the specific procedure", -) -def procedure(graph, name, sourcefile, description): - """Create and compile procedure over a specific graph""" - - # read source - with open(sourcefile, "r") as f: - query = f.read() - - # construct procedure proto - procedure = interactive_pb2.Procedure( - name=name, bound_graph=graph, description=description, query=query, enable=True - ) - - grpc_client = get_grpc_client() - response = grpc_client.create_interactive_procedure(procedure) - - if is_success(response): - click.secho("Create procedure {0} successfully.".format(name), fg="green") - - -@delete.command() -@click.option( - "-g", - "--graph", - required=True, - help="The name of the graph", -) -def graph(graph): - """Delete a graph, as well as the loaded data by name""" - - grpc_client = get_grpc_client() - response = grpc_client.remove_interactive_graph(graph) - - if is_success(response): - click.secho( - "Delete interactive graph {0} successfully.".format(graph), - fg="green", - ) - - -@delete.command() -@click.option( - "-g", - "--graph", - required=True, - help="The name of the graph", -) -@click.option( - "-n", - "--name", - required=True, - help="The name of the procedure to be deleted", -) -def procedure(graph, name): - """Delete a procedure over a specific graph in database""" - - grpc_client = get_grpc_client() - response = grpc_client.remove_interactive_procedure(graph, name) - - if is_success(response): - click.secho( - "Remove procedure {0} on graph {1} successfully.".format(name, graph), - fg="green", - ) - - -@enable.command() -@click.option( - "-g", - "--graph", - required=True, - help="The name of the graph", -) -@click.option( - "-n", - "--name", - required=True, - help="List of procedure's name to enable, seprated by comma", -) -def procedure(graph, name): - """Enable stored procedures in a given graph""" - - # remove the last "," if exists - if name.endswith(","): - name = name[:-1] - procedure_list = name.split(",") - - # list current procedures - grpc_client = get_grpc_client() - response = grpc_client.list_interactive_procedure(graph) - - if is_success(response): - # enable procedures locally - procedures = response.procedures - for p in procedures: - if p.name in procedure_list: - p.enable = True - procedure_list.remove(p.name) - - # check - if procedure_list: - click.secho( - "Procedure {0} not found.".format(str(procedure_list)), fg="red" - ) - return - - # update procedures - response = grpc_client.update_interactive_procedure(procedures) - if is_success(response): - click.secho("Update procedures successfully.", fg="green") - - -@disable.command() -@click.option( - "-g", - "--graph", - required=True, - help="The name of the graph", -) -@click.option( - "-n", - "--name", - required=True, - help="List of procedure's name to enable, seprated by comma", -) -def proceduree(graph, name): - """Disable stored procedures in a given graph""" - - # remove the last "," if exists - if name.endswith(","): - name = name[:-1] - procedure_list = name.split(",") - - # list current procedures - grpc_client = get_grpc_client() - response = grpc_client.list_interactive_procedure(graph) - - if is_success(response): - # disable procedures locally - procedures = response.procedures - for p in procedures: - if p.name in procedure_list: - p.enable = False - procedure_list.remove(p.name) - - # check - if procedure_list: - click.secho( - "Procedure {0} not found.".format(str(procedure_list)), fg="red" - ) - return - - # update procedures - response = grpc_client.update_interactive_procedure(procedures) - if is_success(response): - click.secho("Update procedures successfully.", fg="green") - - -@service.command() -@click.option( - "-g", - "--graph", - required=True, - help="The name of the graph", -) -def start(graph): - """Stop current service and start a new one on a specified graph""" - - service_def = interactive_pb2.Service(graph_name=graph) - - grpc_client = get_grpc_client() - response = grpc_client.start_interactive_service(service_def) - - if is_success(response): - click.secho( - "Start service on graph {0} successfully.".format(graph), fg="green" - ) - - -@service.command() -def stop(): - """Stop the current database service""" - - grpc_client = get_grpc_client() - response = grpc_client.stop_interactive_service() - - if is_success(response): - click.secho("Service has stopped.", fg="green") - - -@service.command() -def restart(): - """Restart database service on current graph""" - - grpc_client = get_grpc_client() - response = grpc_client.restart_interactive_service() - - if is_success(response): - click.secho("Restart service successfully.", fg="green") - - -@get.command() -def graph(): - """Display graphs in database""" - - def _construct_and_display_data(graphs): - if not graphs: - click.secho("no graph found in database.", fg="blue") - return - - head = ["", "NAME", "STORE TYPE", "VERTEX LABEL SIZE", "EDGE LABEL SIZE"] - data = [head] - - for g in graphs: - # vertex_schema = [] - # for v in g.schema.vertex_types: - # vertex_schema.append(v.type_name) - # edge_schema = [] - # for e in g.schema.edge_types: - # edge_schema.append(e.type_name) - data.append( - [ - "*", # service graph - g.name, - g.store_type, - str(len(g.schema.vertex_types)), - str(len(g.schema.edge_types)), - # "({0}) {1}".format(len(vertex_schema), ",".join(vertex_schema)), - # "({0}) {1}".format(len(edge_schema), ",".join(edge_schema)), - ] - ) - - terminal_display(data) - - grpc_client = get_grpc_client() - response = grpc_client.list_interactive_graph() - - if is_success(response): - _construct_and_display_data(response.graphs) - return response.graphs - - -@get.command() -def job(): - """Display jobs in database""" - - def _construct_and_display_data(job_status): - if not job_status: - click.secho("no job found in database.", fg="blue") - return - - head = ["JOBID", "STATUS", "START TIME", "END TIME", "DETAIL", "MESSAGE"] - data = [head] - - for s in job_status: - detail = {} - for k, v in s.detail.items(): - detail[k] = v - - message = s.message.replace("\n", "") - if len(message) > 63: - message = message[:63] + "..." - - data.append( - [ - s.jobid, - s.status, - s.start_time, - s.end_time, - json.dumps(detail), - message, - ] - ) - - terminal_display(data) - - grpc_client = get_grpc_client() - response = grpc_client.list_interactive_job() - - if is_success(response): - _construct_and_display_data(response.job_status) - return response.job_status - - -@get.command() -@click.option( - "-g", - "--graph", - required=True, - help="The name of the graph", -) -def procedure(graph): - """Display procedures in database""" - - def _construct_and_display_data(procedures): - if not procedures: - click.secho("no procedure found in database.", fg="blue") - return - - head = ["NAME", "TYPE", "ENABLE", "DESCRIPTION"] - data = [head] - - for procedure in procedures: - data.append( - [ - procedure.name, - procedure.type, - str(procedure.enable), - procedure.description, - ] - ) - - terminal_display(data) - - grpc_client = get_grpc_client() - response = grpc_client.list_interactive_procedure(graph) - - if is_success(response): - _construct_and_display_data(response.procedures) - return response.procedures - - -@get.command() -def service(): - """Display service status in database""" - - def _construct_and_display_data(service_status): - head = [ - "STATUS", - "SERVICE GRAPH", - "BOLT SERVICE ENDPOINT", - "HQPS SERVICE ENDPOINT", - ] - data = [head] - - data.append( - [ - service_status.status, - service_status.graph_name, - service_status.bolt_port, - service_status.hqps_port, - ] - ) - - terminal_display(data) - - grpc_client = get_grpc_client() - response = grpc_client.get_interactive_service_status() - - if is_success(response): - _construct_and_display_data(response.service_status) - return response.service_status - - -@get.command() -def node(): - """Display resource (CPU/memory) usage of nodes""" - - def _construct_and_display_data(nodes_status): - head = ["NAME", "CPU USAGE(%)", "MEMORY USAGE(%)", "DISK USAGE(%)"] - data = [head] - - for s in nodes_status: - data.append( - [ - s.node, - s.cpu_usage, - s.memory_usage, - s.disk_usage, - ] - ) - - terminal_display(data) - - grpc_client = get_grpc_client() - response = grpc_client.get_node_status() - - if is_success(response): - _construct_and_display_data(response.nodes_status) - return response.nodes_status - - -@describe.command() -@click.argument("graph", required=False) -def graph(graph): - """Show details of graph""" - - grpc_client = get_grpc_client() - response = grpc_client.list_interactive_graph() - - if is_success(response): - graphs = response.graphs - - if not graphs: - click.secho("no graph found in database.", fg="blue") - return - - specific_graph_exist = False - for g in graphs: - if graph is not None and g.name != graph: - continue - - # display - click.secho(yaml.dump(MessageToDict(g, preserving_proto_field_name=True))) - - if graph is not None and g.name == graph: - specific_graph_exist = True - break - - if graph is not None and not specific_graph_exist: - click.secho("graphs \"{0}\" not found.".format(graph), fg="blue") - - -if __name__ == "__main__": - cli() diff --git a/python/graphscope/gsctl/config.py b/python/graphscope/gsctl/config.py index f38cbb903157..a781cac21c39 100644 --- a/python/graphscope/gsctl/config.py +++ b/python/graphscope/gsctl/config.py @@ -29,21 +29,11 @@ "GSCONFIG", os.path.expanduser("~/.graphscope/config") ) -FLEX_INTERACTIVE = "Interactive" - class Context(object): - def __init__(self, solution, coordinator_endpoint, name=None): - self.supported_solutions = [FLEX_INTERACTIVE] - if solution not in self.supported_solutions: - raise RuntimeError( - "The solution {0} in context {1} is not supported yet.".format( - solution, name - ) - ) - + def __init__(self, coordinator_endpoint, solution, name=None): if name is None: - name = "context_" + "".join(random.choices(ascii_letters, k=6)) + name = "context_" + "".join(random.choices(ascii_letters, k=8)) self.name = name self.solution = solution @@ -72,7 +62,7 @@ def current_context(self) -> Context: return self._contexts[self._current_context] def set_and_write(self, context: Context): - # treat the same endpoint as the same coordinator + # treat the same endpoint with same services as the same coordinator for _, v in self._contexts.items(): if ( context.coordinator_endpoint == v.coordinator_endpoint diff --git a/python/graphscope/gsctl/gsctl.py b/python/graphscope/gsctl/gsctl.py index a37b1b450ce9..ea315a6bb3ad 100644 --- a/python/graphscope/gsctl/gsctl.py +++ b/python/graphscope/gsctl/gsctl.py @@ -45,7 +45,8 @@ def cli(): # get the specified commands under the FLEX architecture commands = get_command_collection(context) # serve the command - commands() + if commands is not None: + commands() if __name__ == "__main__": From f6abea2975d69e74fc32c808a50245a68fb56e5d Mon Sep 17 00:00:00 2001 From: Dongze Li Date: Wed, 27 Dec 2023 18:05:43 +0800 Subject: [PATCH 3/4] update --- .../servicer/flex/interactive/hiactor.py | 142 ++++ .../flex/interactive/job_scheduler.py | 77 ++ .../gscoordinator/servicer/flex/job.py | 66 +- .../{ => servicer/flex}/scheduler.py | 4 +- .../gscoordinator/servicer/flex/service.py | 81 ++- .../{ => servicer/flex}/stoppable_thread.py | 0 proto/coordinator_service.proto | 10 +- proto/flex.proto | 58 +- python/graphscope/gsctl/client/rpc.py | 31 +- .../graphscope/gsctl/commands/interactive.py | 665 ++++++++++++++++++ python/graphscope/gsctl/utils.py | 28 +- 11 files changed, 1051 insertions(+), 111 deletions(-) create mode 100644 coordinator/gscoordinator/servicer/flex/interactive/job_scheduler.py rename coordinator/gscoordinator/{ => servicer/flex}/scheduler.py (98%) rename coordinator/gscoordinator/{ => servicer/flex}/stoppable_thread.py (100%) create mode 100644 python/graphscope/gsctl/commands/interactive.py diff --git a/coordinator/gscoordinator/servicer/flex/interactive/hiactor.py b/coordinator/gscoordinator/servicer/flex/interactive/hiactor.py index b6a6699cf1ed..29f8e417168b 100644 --- a/coordinator/gscoordinator/servicer/flex/interactive/hiactor.py +++ b/coordinator/gscoordinator/servicer/flex/interactive/hiactor.py @@ -17,17 +17,44 @@ # import os +import itertools import logging import hiactor_client +from google.protobuf.json_format import MessageToDict + from graphscope.config import Config +from graphscope.proto import flex_pb2 + +from gscoordinator.servicer.flex.interactive.job_scheduler import ( + DataloadingJobScheduler, +) +from gscoordinator.servicer.flex.job import JobStatus + +from gscoordinator.utils import WORKSPACE +from gscoordinator.utils import delegate_command_to_pod +from gscoordinator.utils import run_kube_cp_command __all__ = ["init_interactive_service"] logger = logging.getLogger("graphscope") +# There are two workspaces for FLEX Interactive, one residing on "coordinator" node +# and the other on the "interactive" node. These two workspaces are equivalent when +# running mode is "hosts". Hence, INTERACTIVE_WORKSPACE is only effective in +# Kubernetes (k8s) environment. +INTERACTIVE_WORKSPACE = ( + os.environ["INTERACTIVE_WORKSPACE"] + if "INTERACTIVE_WORKSPACE" in os.environ + else "/tmp" +) + + +INTERACTIVE_CONTAINER_NAME = "interactive" + + class Hiactor(object): """Hiactor module used to interact with hiactor engine""" @@ -35,6 +62,12 @@ def __init__(self, config: Config): self._config = config # hiactor admin service host self._hiactor_host = self._get_hiactor_service_endpoints() + # workspace + self._workspace = os.path.join(WORKSPACE, "interactive") + os.makedirs(self._workspace, exist_ok=True) + # job status + self._job_status = {} + # check heartbeat to interactive engine logger.info("Connect to hiactor service at %s", self._hiactor_host) def _get_hiactor_service_endpoints(self): @@ -45,6 +78,52 @@ def _get_hiactor_service_endpoints(self): ) return endpoint + @property + def hiactor_host(self): + return self._hiactor_host + + @property + def job_status(self): + return self._job_status + + def register_job_status(self, jobid: str, status: JobStatus): + self._job_status[jobid] = status + + def write_and_distribute_file( + self, graph_name: str, basename: str, raw_data: bytes + ): + # //raw_data/ + filedir = os.path.join(self._workspace, "raw_data", graph_name) + os.makedirs(filedir, exist_ok=True) + # //basename + filepath = os.path.join(filedir, basename) + # dump raw data to file + with open(filepath, "wb") as f: + f.write(raw_data) + # distribute + target_file = filepath + if self._config.launcher_type == "k8s": + # filepath is different in interactive pod + target_file = os.path.join( + INTERACTIVE_WORKSPACE, "raw_data", graph_name, basename + ) + for pod in []: + container = INTERACTIVE_CONTAINER_NAME + cmd = f"mkdir -p {os.path.dirname(target_file)}" + logger.debug(delegate_command_to_pod(cmd, pod, container)) + logger.debug( + run_kube_cp_command(tmp_file, target_file, pod, container, True) + ) + return target_file + + def create_graph(self, graph_def_dict): + with hiactor_client.ApiClient( + hiactor_client.Configuration(self._hiactor_host) + ) as api_client: + api_instance = hiactor_client.GraphApi(api_client) + graph = hiactor_client.Graph.from_dict(graph_def_dict) + return api_instance.create_graph(graph) + def list_graph(self): with hiactor_client.ApiClient( hiactor_client.Configuration(self._hiactor_host) @@ -52,6 +131,69 @@ def list_graph(self): api_instance = hiactor_client.GraphApi(api_client) return api_instance.list_graphs() + def delete_graph(self, graph_name: str): + with hiactor_client.ApiClient( + hiactor_client.Configuration(self._hiactor_host) + ) as api_client: + api_instance = hiactor_client.GraphApi(api_client) + return api_instance.delete_graph(graph_name) + + def create_job( + self, + job_type: str, + schedule_proto: flex_pb2.Schedule, + description_proto: flex_pb2.JobDescription, + ): + if job_type != "dataloading": + raise RuntimeError( + "Job type {0} is not supported in interacive.".format(job_type) + ) + + # write raw data to file and distribute to interactive workspace + schema_mapping = description_proto.schema_mapping + for mapping in itertools.chain( + schema_mapping.vertex_mappings, schema_mapping.edge_mappings + ): + for index, location in enumerate(mapping.inputs): + if location.startswith("@"): + # write raw data and distribute file to interactive workspace + new_location = self.write_and_distribute_file( + schema_mapping.graph, + os.path.basename(location), + mapping.raw_data[index], + ) + # clear the raw_data + mapping.raw_data[index] = bytes() + # update the location + mapping.inputs[index] = new_location + # schedule + schedule = MessageToDict( + schedule_proto, + preserving_proto_field_name=True, + including_default_value_fields=True, + ) + # job description + description = MessageToDict( + description_proto, + preserving_proto_field_name=True, + including_default_value_fields=True, + ) + # submit + if schedule["run_now"]: + at_time = "now" + repeat = "null" + else: + at_time = schedule["at_time"] + repeat = schedule["repeat"] + + scheduler = DataloadingJobScheduler( + at_time=at_time, + repeat=repeat, + description=description, + servicer=self, + ) + scheduler.start() + def init_interactive_service(config: Config): return Hiactor(config) diff --git a/coordinator/gscoordinator/servicer/flex/interactive/job_scheduler.py b/coordinator/gscoordinator/servicer/flex/interactive/job_scheduler.py new file mode 100644 index 000000000000..07fa7baa2d9f --- /dev/null +++ b/coordinator/gscoordinator/servicer/flex/interactive/job_scheduler.py @@ -0,0 +1,77 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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 logging + +import hiactor_client + +from gscoordinator.servicer.flex.job import JobType +from gscoordinator.servicer.flex.job import JobStatus +from gscoordinator.servicer.flex.scheduler import Scheduler + +logger = logging.getLogger("graphscope") + + +class DataloadingJobScheduler(Scheduler): + """Class for scheduling the dataloading process of interactive graphs""" + + def __init__(self, at_time, repeat, description, servicer): + super().__init__(at_time, repeat) + + self._description = description + self._servicer = servicer + + self._tags = [JobType.GRAPH_IMPORT] + + def run(self): + """This function needs to handle exception by itself""" + + schema_mapping = self._description["schema_mapping"] + graph_name = schema_mapping["graph"] + + detail = {"graph name": graph_name} + status = JobStatus( + jobid=self.jobid, + type=JobType.GRAPH_IMPORT, + start_time=self.last_run, + detail=detail, + ) + + # register status to servicer + self._servicer.register_job_status(self.jobid, status) + + with hiactor_client.ApiClient( + hiactor_client.Configuration(self._servicer.hiactor_host) + ) as api_client: + # create an instance of the API class + api_instance = hiactor_client.DataloadingApi(api_client) + + try: + api_response = api_instance.create_dataloading_job( + graph_name, + hiactor_client.SchemaMapping.from_dict(schema_mapping), + ) + except Exception as e: + logger.warning( + "Failed to create dataloading job on graph %s: %s", + graph_name, + str(e), + ) + status.set_failed(message=str(e)) + else: + status.set_success(message=api_response.message) diff --git a/coordinator/gscoordinator/servicer/flex/job.py b/coordinator/gscoordinator/servicer/flex/job.py index 0281f5f7840f..20f5fc7d5055 100644 --- a/coordinator/gscoordinator/servicer/flex/job.py +++ b/coordinator/gscoordinator/servicer/flex/job.py @@ -17,25 +17,16 @@ # import datetime -import logging from enum import Enum -import interactive_client -from graphscope.proto import interactive_pb2 - -from gscoordinator.scheduler import Scheduler from gscoordinator.utils import encode_datetime -logger = logging.getLogger("graphscope") - class JobType(Enum): - GRAPH_IMPORT = 0 class Status(Enum): - RUNNING = 0 CANCELLED = 1 SUCCESS = 2 @@ -44,12 +35,18 @@ class Status(Enum): class JobStatus(object): - """Base class of job status for GraphScope FLEX runnable tasks""" - def __init__( - self, jobid, status, start_time, end_time=None, detail=dict(), message="" + self, + jobid, + type, + start_time, + status=Status.RUNNING, + end_time=None, + detail=dict(), + message="", ): self.jobid = jobid + self.type = type self.status = status self.start_time = start_time self.end_time = end_time @@ -60,6 +57,7 @@ def __init__( def to_dict(self): return { "jobid": self.jobid, + "type": self.type.name, "status": self.status.name, "start_time": encode_datetime(self.start_time), "end_time": encode_datetime(self.end_time), @@ -79,47 +77,3 @@ def set_failed(self, message=""): def set_canncelled(self): self.status = Status.CANCELLED - - -class GraphImportScheduler(Scheduler): - """This class responsible for scheduling and managing the import of interactive graphs.""" - - def __init__(self, at_time, repeat, schema_mapping, servicer): - super().__init__(at_time, repeat) - - self._schema_mapping = schema_mapping - # we use interactive servicer to get the latest in runtime - self._servicer = servicer - - self._tags = [JobType.GRAPH_IMPORT] - - def run(self): - """This function needs to handle exception by itself""" - graph_name = self._schema_mapping["graph"] - - detail = {"graph name": graph_name, "type": JobType.GRAPH_IMPORT.name} - status = JobStatus(self.jobid, Status.RUNNING, self.last_run, detail=detail) - - # register status to servicer - self._servicer.register_job_status(self.jobid, status) - - with interactive_client.ApiClient( - interactive_client.Configuration(self._servicer.interactive_host) - ) as api_client: - # create an instance of the API class - api_instance = interactive_client.DataloadingApi(api_client) - - try: - api_response = api_instance.create_dataloading_job( - graph_name, - interactive_client.SchemaMapping.from_dict(self._schema_mapping), - ) - except Exception as e: - logger.warning( - "Failed to create dataloading job on graph %s: %s", - graph_name, - str(e), - ) - status.set_failed(message=str(e)) - else: - status.set_success(message=api_response.message) diff --git a/coordinator/gscoordinator/scheduler.py b/coordinator/gscoordinator/servicer/flex/scheduler.py similarity index 98% rename from coordinator/gscoordinator/scheduler.py rename to coordinator/gscoordinator/servicer/flex/scheduler.py index 86328c95f9bf..5a79dde9cb9b 100644 --- a/coordinator/gscoordinator/scheduler.py +++ b/coordinator/gscoordinator/servicer/flex/scheduler.py @@ -25,7 +25,7 @@ import schedule from schedule import CancelJob -from gscoordinator.stoppable_thread import StoppableThread +from gscoordinator.servicer.flex.stoppable_thread import StoppableThread from gscoordinator.utils import decode_datetimestr @@ -177,7 +177,7 @@ def do_run(self): """Start a thread for the job.""" # overwrite for each scheduled job self._jobid = "job-{0}".format( - datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%S") + datetime.datetime.now().strftime("%Y-%m-%d-%H-%M-%S") ) self._last_run = datetime.datetime.now() # schedule in a thread diff --git a/coordinator/gscoordinator/servicer/flex/service.py b/coordinator/gscoordinator/servicer/flex/service.py index f8d9cfff2b64..2219f9ca267e 100644 --- a/coordinator/gscoordinator/servicer/flex/service.py +++ b/coordinator/gscoordinator/servicer/flex/service.py @@ -19,13 +19,15 @@ """Service under FLEX Architecture""" import atexit +import traceback +import functools # import itertools import logging import os import threading -# from google.protobuf.json_format import MessageToDict +from google.protobuf.json_format import MessageToDict from graphscope.config import Config from graphscope.gsctl.utils import dict_to_proto_message from graphscope.proto import coordinator_service_pb2_grpc @@ -45,6 +47,26 @@ logger = logging.getLogger("graphscope") +def handle_api_exception(proto_message_response): + """Decorator to handle api exception occurs during request engine service.""" + + def _handle_api_exception(fn): + @functools.wraps(fn) + def wrapper(*args, **kwargs): + try: + return fn(*args, **kwargs) + except Exception as e: + logger.warning("Failed to execute %s: %s", str(fn.__name__), str(e)) + traceback.print_exc() + return proto_message_response( + code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) + ) + + return wrapper + + return _handle_api_exception + + class FlexServiceServicer(coordinator_service_pb2_grpc.CoordinatorServiceServicer): """Service under flex architecture.""" @@ -63,7 +85,6 @@ def __init__(self, config: Config): # lock to protect the service self._lock = threading.RLock() - # initialize specific service client self._service_client = self._initialize_service_client() @@ -82,22 +103,48 @@ def cleanup(self): def Connect(self, request, context): return message_pb2.ConnectResponse(solution=self._solution) + @handle_api_exception(flex_pb2.ApiResponse) + def CreateGraph(self, request, context): + graph_def_dict = MessageToDict( + request.graph_def, + preserving_proto_field_name=True, + including_default_value_fields=True, + ) + api_response = self._service_client.create_graph(graph_def_dict) + return flex_pb2.ApiResponse(code=error_codes_pb2.OK, error_msg=api_response) + + @handle_api_exception(flex_pb2.ListGraphResponse) def ListGraph(self, request, context): - try: - graphs = self._service_client.list_graph() - except Exception as e: - logger.warning("Failed to list graph: %s", str(e)) - return flex_pb2.ListGraphResponse( - code=error_codes_pb2.API_EXCEPTION_ERROR, error_msg=str(e) - ) - else: - return flex_pb2.ListGraphResponse( - code=error_codes_pb2.OK, - graphs=[ - dict_to_proto_message(g.to_dict(), flex_pb2.GraphProto()) - for g in graphs - ], - ) + graphs = self._service_client.list_graph() + return flex_pb2.ListGraphResponse( + code=error_codes_pb2.OK, + graphs=[ + dict_to_proto_message(g.to_dict(), flex_pb2.GraphProto()) + for g in graphs + ], + ) + + @handle_api_exception(flex_pb2.ApiResponse) + def DeleteGraph(self, request, context): + api_response = self._service_client.delete_graph(request.graph_name) + return flex_pb2.ApiResponse(code=error_codes_pb2.OK, error_msg=api_response) + + @handle_api_exception(flex_pb2.ApiResponse) + def CreateJob(self, request, context): + self._service_client.create_job( + request.type, request.schedule, request.description + ) + return flex_pb2.ApiResponse(code=error_codes_pb2.OK) + + @handle_api_exception(flex_pb2.ListJobResponse) + def ListJob(self, request, context): + return flex_pb2.ListJobResponse( + code=error_codes_pb2.OK, + job_status=[ + dict_to_proto_message(s.to_dict(), flex_pb2.JobStatus()) + for _, s in self._service_client.job_status.items() + ], + ) def init_flex_service_servicer(config: Config): diff --git a/coordinator/gscoordinator/stoppable_thread.py b/coordinator/gscoordinator/servicer/flex/stoppable_thread.py similarity index 100% rename from coordinator/gscoordinator/stoppable_thread.py rename to coordinator/gscoordinator/servicer/flex/stoppable_thread.py diff --git a/proto/coordinator_service.proto b/proto/coordinator_service.proto index 6102cb5ba7a2..6d7fddff3e21 100644 --- a/proto/coordinator_service.proto +++ b/proto/coordinator_service.proto @@ -55,15 +55,15 @@ service CoordinatorService { rpc ListGraph(ListGraphRequest) returns (ListGraphResponse); - // end + rpc CreateGraph(CreateGraphRequest) returns (ApiResponse); - rpc CreateInteractiveGraph(CreateInteractiveGraphRequest) returns (ApiResponse); + rpc DeleteGraph(DeleteGraphRequest) returns (ApiResponse); - rpc RemoveInteractiveGraph(RemoveInteractiveGraphRequest) returns (ApiResponse); + rpc CreateJob(CreateJobRequest) returns (ApiResponse); - rpc ImportInteractiveGraph(ImportInteractiveGraphRequest) returns (ApiResponse); + rpc ListJob(ListJobRequest) returns (ListJobResponse); - rpc ListInteractiveJob(ListInteractiveJobRequest) returns (ListInteractiveJobResponse); + // end rpc CreateInteractiveProcedure(CreateInteractiveProcedureRequest) returns (ApiResponse); diff --git a/proto/flex.proto b/proto/flex.proto index d90afd374f22..86229e9aefed 100644 --- a/proto/flex.proto +++ b/proto/flex.proto @@ -19,10 +19,18 @@ package gs.rpc; import "error_codes.proto"; +// attributes value +message MapValue { + oneof value { + string s = 1; + int64 i = 2; + bool b = 3; + } +} // property type message PropertyTypeProto { - string primitive_type = 1; // [ DT_DOUBLE, DT_STRING, DT_SIGNED_INT32, DT_SIGNED_INT64 ] + string primitive_type = 1; // [ DT_DOUBLE, DT_STRING, DT_SIGNED_INT32, DT_SIGNED_INT64 ] } // property @@ -128,7 +136,7 @@ message DataSource { // format message DataSourceFormat { string type = 1; // csv - map metadata = 2; + map metadata = 2; } // loading config @@ -149,11 +157,12 @@ message SchemaMapping { // job status message JobStatus { string jobid = 1; - string status = 2; - string start_time = 3; - string end_time = 4; - map detail = 5; - string message = 6; + string type = 2; + string status = 3; + string start_time = 4; + string end_time = 5; + map detail = 6; + string message = 7; } // procedure runtime params, used both in property and return type @@ -197,17 +206,38 @@ message NodeStatus { int32 disk_usage = 4; } +// job schedule +message Schedule { + bool run_now = 1; + string time = 2; // formate with "2023-02-21 11:56:30" + string repeate = 3; // day, week, or null +} + +// AnalyticalJobPlaceHolder +message AnalyticalJobPlaceHolder { + string a = 1; + int32 b = 2; +} + +// job description +message JobDescription { + oneof value { + SchemaMapping schema_mapping = 1; + AnalyticalJobPlaceHolder ajph = 2; + } +} + // message message ApiResponse { Code code = 1; optional string error_msg = 2; } -message CreateInteractiveGraphRequest { +message CreateGraphRequest { GraphProto graph_def = 1; } -message RemoveInteractiveGraphRequest { +message DeleteGraphRequest { string graph_name = 1; } @@ -220,13 +250,15 @@ message ListGraphResponse { repeated GraphProto graphs = 3; } -message ImportInteractiveGraphRequest { - SchemaMapping schema_mapping = 1; +message CreateJobRequest { + string type = 1; // optional value: dataloading + Schedule schedule = 2; + JobDescription description = 3; } -message ListInteractiveJobRequest {} +message ListJobRequest {} -message ListInteractiveJobResponse { +message ListJobResponse { Code code = 1; optional string error_msg = 2; diff --git a/python/graphscope/gsctl/client/rpc.py b/python/graphscope/gsctl/client/rpc.py index 94ea255d263b..fb0ba99cf776 100644 --- a/python/graphscope/gsctl/client/rpc.py +++ b/python/graphscope/gsctl/client/rpc.py @@ -89,14 +89,14 @@ def close(self): pass @handle_grpc_error - def create_interactive_graph(self, graph_def: flex_pb2.GraphProto): - request = flex_pb2.CreateInteractiveGraphRequest(graph_def=graph_def) - return self._stub.CreateInteractiveGraph(request) + def create_graph(self, graph_def: flex_pb2.GraphProto): + request = flex_pb2.CreateGraphRequest(graph_def=graph_def) + return self._stub.CreateGraph(request) @handle_grpc_error - def remove_interactive_graph(self, graph: str): - request = flex_pb2.RemoveInteractiveGraphRequest(graph_name=graph) - return self._stub.RemoveInteractiveGraph(request) + def delete_graph(self, graph_name: str): + request = flex_pb2.DeleteGraphRequest(graph_name=graph_name) + return self._stub.DeleteGraph(request) @handle_grpc_error def list_graph(self): @@ -104,14 +104,21 @@ def list_graph(self): return self._stub.ListGraph(request) @handle_grpc_error - def import_interactive_graph(self, schema_mapping: flex_pb2.SchemaMapping): - request = flex_pb2.ImportInteractiveGraphRequest(schema_mapping=schema_mapping) - return self._stub.ImportInteractiveGraph(request) + def create_job( + self, + type: str, + schedule: flex_pb2.Schedule, + description: flex_pb2.JobDescription, + ): + request = flex_pb2.CreateJobRequest( + type=type, schedule=schedule, description=description + ) + return self._stub.CreateJob(request) @handle_grpc_error - def list_interactive_job(self): - request = flex_pb2.ListInteractiveJobRequest() - return self._stub.ListInteractiveJob(request) + def list_job(self): + request = flex_pb2.ListJobRequest() + return self._stub.ListJob(request) @handle_grpc_error def create_interactive_procedure(self, procedure: flex_pb2.Procedure): diff --git a/python/graphscope/gsctl/commands/interactive.py b/python/graphscope/gsctl/commands/interactive.py new file mode 100644 index 000000000000..9b9766ddd0c4 --- /dev/null +++ b/python/graphscope/gsctl/commands/interactive.py @@ -0,0 +1,665 @@ +#!/usr/bin/env python3 +# -*- coding: utf-8 -*- +# +# Copyright 2023 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. +# + +"""Group of interactive commands under the FLEX architecture""" + +import itertools +import json + +import click +import yaml +from google.protobuf.json_format import MessageToDict + +from graphscope.gsctl.client.rpc import get_grpc_client +from graphscope.gsctl.utils import dict_to_proto_message +from graphscope.gsctl.utils import is_valid_file_path +from graphscope.gsctl.utils import read_yaml_file +from graphscope.gsctl.utils import terminal_display +from graphscope.proto import error_codes_pb2 +from graphscope.proto import flex_pb2 + + +@click.group() +def cli(): + pass + + +@cli.group() +def get(): + """Display a specific resource or group of resources""" + pass + + +@cli.group() +def describe(): + """Show details of a specific resource or group of resources""" + pass + + +@cli.group() +def create(): + """Create a resource(graph, procedure) from a file""" + pass + + +@cli.group() +def delete(): + """Delete a resource(graph, procedure) by name""" + pass + + +@cli.group() +def enable(): + """Enable a stored procedure by name""" + pass + + +@cli.group() +def disable(): + """Disable a stored procedure by name""" + pass + + +@cli.group() +def service(): + """Start, restart or stop the database service""" + pass + + +def is_success(response): + if response.code != error_codes_pb2.OK: + click.secho( + "{0}: {1}".format( + error_codes_pb2.Code.Name(response.code), response.error_msg + ), + fg="red", + ) + return False + return True + + +@create.command() +@click.option( + "-n", + "--name", + required=False, + help="The name of the graph", +) +@click.option( + "-f", + "--filename", + required=True, + help="Path of yaml file to use to create the graph", +) +def graph(name, filename): + """Create a new graph in database, with the provided schema file""" + + if not is_valid_file_path(filename): + click.secho("Invalid file: {0}".format(filename), fg="blue") + return + + graph_def_dict = read_yaml_file(filename) + + # override graph name + if name is not None: + graph_def_dict["name"] = name + + # transform graph dict to proto message + graph_def = flex_pb2.GraphProto() + dict_to_proto_message(graph_def_dict, graph_def) + + grpc_client = get_grpc_client() + if not grpc_client.connect(): + return + + response = grpc_client.create_graph(graph_def) + + if is_success(response): + click.secho( + "Create graph {0} successfully.".format(graph_def_dict["name"]), + fg="green", + ) + + +@create.command() +@click.option( + "-f", + "--filename", + required=True, + help="Path of yaml file to use to create the job", +) +def job(filename): + """Create or schedule a job in database""" + + def _read_and_fill_raw_data(config): + for mapping in itertools.chain( + config["vertex_mappings"], config["edge_mappings"] + ): + for index, location in enumerate(mapping["inputs"]): + # path begin with "@" represents the local file + if location.startswith("@"): + if "raw_data" not in mapping: + mapping["raw_data"] = [] + + # read file and set raw data + with open(location[1:], "rb") as f: + content = f.read() + mapping["raw_data"].append(content) + + if not is_valid_file_path(filename): + click.secho("Invalid file: {0}".format(filename), fg="blue") + return + + job = read_yaml_file(filename) + + try: + _read_and_fill_raw_data(job["description"]) + + # job schedule + schedule = flex_pb2.Schedule() + dict_to_proto_message(job["schedule"], schedule) + + # job description + schema_mapping = flex_pb2.SchemaMapping() + dict_to_proto_message(job["description"], schema_mapping) + description = flex_pb2.JobDescription(schema_mapping=schema_mapping) + + grpc_client = get_grpc_client() + if not grpc_client.connect(): + return + + response = grpc_client.create_job( + type="dataloading", + schedule=schedule, + description=description, + ) + + if is_success(response): + click.secho( + "Job has been submitted", + fg="green", + ) + except Exception as e: + click.secho(str(e), fg="red") + + +@create.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +@click.option( + "-n", + "--name", + required=True, + help="The name of the procedure", +) +@click.option( + "-i", + "--sourcefile", + required=True, + help="Path of [ .cc, .cypher ] file", +) +@click.option( + "-d", + "--description", + required=False, + help="Description for the specific procedure", +) +def procedure(graph, name, sourcefile, description): + """Create and compile procedure over a specific graph""" + + # read source + with open(sourcefile, "r") as f: + query = f.read() + + # construct procedure proto + procedure = flex_pb2.Procedure( + name=name, bound_graph=graph, description=description, query=query, enable=True + ) + + grpc_client = get_grpc_client() + response = grpc_client.create_interactive_procedure(procedure) + + if is_success(response): + click.secho("Create procedure {0} successfully.".format(name), fg="green") + + +@delete.command() +@click.argument("graph", required=True) +def graph(graph): + """Delete a graph by name""" + + grpc_client = get_grpc_client() + response = grpc_client.delete_graph(graph) + + if is_success(response): + click.secho( + "Delete graph {0} successfully.".format(graph), + fg="green", + ) + + +@delete.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +@click.option( + "-n", + "--name", + required=True, + help="The name of the procedure to be deleted", +) +def procedure(graph, name): + """Delete a procedure over a specific graph in database""" + + grpc_client = get_grpc_client() + response = grpc_client.remove_interactive_procedure(graph, name) + + if is_success(response): + click.secho( + "Remove procedure {0} on graph {1} successfully.".format(name, graph), + fg="green", + ) + + +@enable.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +@click.option( + "-n", + "--name", + required=True, + help="List of procedure's name to enable, seprated by comma", +) +def procedure(graph, name): + """Enable stored procedures in a given graph""" + + # remove the last "," if exists + if name.endswith(","): + name = name[:-1] + procedure_list = name.split(",") + + # list current procedures + grpc_client = get_grpc_client() + response = grpc_client.list_interactive_procedure(graph) + + if is_success(response): + # enable procedures locally + procedures = response.procedures + for p in procedures: + if p.name in procedure_list: + p.enable = True + procedure_list.remove(p.name) + + # check + if procedure_list: + click.secho( + "Procedure {0} not found.".format(str(procedure_list)), fg="red" + ) + return + + # update procedures + response = grpc_client.update_interactive_procedure(procedures) + if is_success(response): + click.secho("Update procedures successfully.", fg="green") + + +@disable.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +@click.option( + "-n", + "--name", + required=True, + help="List of procedure's name to enable, seprated by comma", +) +def proceduree(graph, name): + """Disable stored procedures in a given graph""" + + # remove the last "," if exists + if name.endswith(","): + name = name[:-1] + procedure_list = name.split(",") + + # list current procedures + grpc_client = get_grpc_client() + response = grpc_client.list_interactive_procedure(graph) + + if is_success(response): + # disable procedures locally + procedures = response.procedures + for p in procedures: + if p.name in procedure_list: + p.enable = False + procedure_list.remove(p.name) + + # check + if procedure_list: + click.secho( + "Procedure {0} not found.".format(str(procedure_list)), fg="red" + ) + return + + # update procedures + response = grpc_client.update_interactive_procedure(procedures) + if is_success(response): + click.secho("Update procedures successfully.", fg="green") + + +@service.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +def start(graph): + """Stop current service and start a new one on a specified graph""" + + service_def = flex_pb2.Service(graph_name=graph) + + grpc_client = get_grpc_client() + response = grpc_client.start_interactive_service(service_def) + + if is_success(response): + click.secho( + "Start service on graph {0} successfully.".format(graph), fg="green" + ) + + +@service.command() +def stop(): + """Stop the current database service""" + + grpc_client = get_grpc_client() + response = grpc_client.stop_interactive_service() + + if is_success(response): + click.secho("Service has stopped.", fg="green") + + +@service.command() +def restart(): + """Restart database service on current graph""" + + grpc_client = get_grpc_client() + response = grpc_client.restart_interactive_service() + + if is_success(response): + click.secho("Restart service successfully.", fg="green") + + +@get.command() +def graph(): + """Display graphs in database""" + + def _construct_and_display_data(graphs): + if not graphs: + click.secho("no graph found in database.", fg="blue") + return + + head = ["NAME", "STORE TYPE", "VERTEX LABEL SIZE", "EDGE LABEL SIZE"] + data = [head] + + for g in graphs: + data.append( + [ + g.name, + g.store_type, + str(len(g.schema.vertex_types)), + str(len(g.schema.edge_types)), + ] + ) + + terminal_display(data) + + grpc_client = get_grpc_client() + if not grpc_client.connect(): + return + + response = grpc_client.list_graph() + + if is_success(response): + _construct_and_display_data(response.graphs) + + +@get.command() +def job(): + """Display jobs in database""" + + def _construct_and_display_data(job_status): + if not job_status: + click.secho("no job found in database", fg="blue") + return + + head = ["JOBID", "TYPE", "STATUS", "START TIME", "END TIME"] + data = [head] + + for s in job_status: + # detail = {} + # for k, v in s.detail.items(): + # detail[k] = v + + # message = s.message.replace("\n", "") + # if len(message) > 63: + # message = message[:63] + "..." + + data.append( + [ + s.jobid, + s.type, + s.status, + s.start_time, + s.end_time, + ] + ) + + terminal_display(data) + + grpc_client = get_grpc_client() + if not grpc_client.connect(): + return + + response = grpc_client.list_job() + + if is_success(response): + _construct_and_display_data(response.job_status) + + +@get.command() +@click.option( + "-g", + "--graph", + required=True, + help="The name of the graph", +) +def procedure(graph): + """Display procedures in database""" + + def _construct_and_display_data(procedures): + if not procedures: + click.secho("no procedure found in database.", fg="blue") + return + + head = ["NAME", "TYPE", "ENABLE", "DESCRIPTION"] + data = [head] + + for procedure in procedures: + data.append( + [ + procedure.name, + procedure.type, + str(procedure.enable), + procedure.description, + ] + ) + + terminal_display(data) + + grpc_client = get_grpc_client() + response = grpc_client.list_interactive_procedure(graph) + + if is_success(response): + _construct_and_display_data(response.procedures) + return response.procedures + + +@get.command() +def service(): + """Display service status in database""" + + def _construct_and_display_data(service_status): + head = [ + "STATUS", + "SERVICE GRAPH", + "BOLT SERVICE ENDPOINT", + "HQPS SERVICE ENDPOINT", + ] + data = [head] + + data.append( + [ + service_status.status, + service_status.graph_name, + service_status.bolt_port, + service_status.hqps_port, + ] + ) + + terminal_display(data) + + grpc_client = get_grpc_client() + response = grpc_client.get_interactive_service_status() + + if is_success(response): + _construct_and_display_data(response.service_status) + return response.service_status + + +@get.command() +def node(): + """Display resource (CPU/memory) usage of nodes""" + + def _construct_and_display_data(nodes_status): + head = ["NAME", "CPU USAGE(%)", "MEMORY USAGE(%)", "DISK USAGE(%)"] + data = [head] + + for s in nodes_status: + data.append( + [ + s.node, + s.cpu_usage, + s.memory_usage, + s.disk_usage, + ] + ) + + terminal_display(data) + + grpc_client = get_grpc_client() + response = grpc_client.get_node_status() + + if is_success(response): + _construct_and_display_data(response.nodes_status) + return response.nodes_status + + +@describe.command() +@click.argument("graph", required=False) +def graph(graph): + """Show details of graph""" + + grpc_client = get_grpc_client() + if not grpc_client.connect(): + return + + response = grpc_client.list_graph() + + if is_success(response): + graphs = response.graphs + + if not graphs: + click.secho("no graph found in database.", fg="blue") + return + + specific_graph_exist = False + for g in graphs: + if graph is not None and g.name != graph: + continue + + # display + click.secho(yaml.dump(MessageToDict(g, preserving_proto_field_name=True))) + + if graph is not None and g.name == graph: + specific_graph_exist = True + break + + if graph is not None and not specific_graph_exist: + click.secho('graph "{0}" not found.'.format(graph), fg="blue") + + +@describe.command() +@click.argument("job", required=False) +def job(job): + """Show details of job""" + + grpc_client = get_grpc_client() + if not grpc_client.connect(): + return + + response = grpc_client.list_job() + + if is_success(response): + job_status = response.job_status + + if not job_status: + click.secho("no job found in database", fg="blue") + return + + specific_job_exist = False + for s in job_status: + if job is not None and s.jobid != job: + continue + + # display + click.secho(yaml.dump(MessageToDict(s, preserving_proto_field_name=True))) + + if job is not None and s.jobid == job: + specific_job_exist = True + break + + if job is not None and not specific_job_exist: + click.secho('job "{0}" not found.'.format(job), fg="blue") + + +if __name__ == "__main__": + cli() diff --git a/python/graphscope/gsctl/utils.py b/python/graphscope/gsctl/utils.py index e0861acda132..30c5d62d9d43 100644 --- a/python/graphscope/gsctl/utils.py +++ b/python/graphscope/gsctl/utils.py @@ -19,13 +19,21 @@ import os from typing import List +import click import yaml +from google.protobuf.internal.containers import ScalarMap + +from graphscope.proto import flex_pb2 + def read_yaml_file(path): """Reads YAML file and returns as a python object.""" - with open(path, "r") as file: - return yaml.safe_load(file) + try: + with open(path, "r") as file: + return yaml.safe_load(file) + except Exception as e: + click.secho(str(e), fg="red") def write_yaml_file(data, path): @@ -62,11 +70,19 @@ def _parse_dict(values, message): elif isinstance(v, list): _parse_list(v, getattr(message, k)) else: # value can be set - try: + if hasattr(message, k): setattr(message, k, v) - except AttributeError: - # treat as map - message[k] = v + else: + try: + # treat as a map + if isinstance(v, bool): + message[k].CopyFrom(flex_pb2.MapValue(b=v)) + elif isinstance(v, int): + message[k].CopyFrom(flex_pb2.MapValue(i=v)) + elif isinstance(v, str): + message[k].CopyFrom(flex_pb2.MapValue(s=v)) + except AttributeError as e: + click.secho(str(e), fg="red") _parse_dict(values, message) return message From 5ec4456b23c173dcd2d91c792184234b31157098 Mon Sep 17 00:00:00 2001 From: Dongze Li Date: Mon, 8 Jan 2024 13:14:50 +0800 Subject: [PATCH 4/4] update --- .../servicer/flex/interactive/hiactor.py | 11 +- .../flex/interactive/job_scheduler.py | 6 +- .../gscoordinator/servicer/flex/job.py | 19 +++- .../gscoordinator/servicer/flex/scheduler.py | 10 +- .../gscoordinator/servicer/flex/service.py | 62 ++++++++++- proto/coordinator_service.proto | 8 +- proto/flex.proto | 26 +++-- python/graphscope/gsctl/client/rpc.py | 17 ++- .../graphscope/gsctl/commands/interactive.py | 105 +++++++++--------- python/graphscope/gsctl/utils.py | 17 +-- 10 files changed, 189 insertions(+), 92 deletions(-) diff --git a/coordinator/gscoordinator/servicer/flex/interactive/hiactor.py b/coordinator/gscoordinator/servicer/flex/interactive/hiactor.py index 29f8e417168b..e57eb453d10f 100644 --- a/coordinator/gscoordinator/servicer/flex/interactive/hiactor.py +++ b/coordinator/gscoordinator/servicer/flex/interactive/hiactor.py @@ -144,7 +144,7 @@ def create_job( schedule_proto: flex_pb2.Schedule, description_proto: flex_pb2.JobDescription, ): - if job_type != "dataloading": + if job_type != "DATALOADING": raise RuntimeError( "Job type {0} is not supported in interacive.".format(job_type) ) @@ -194,6 +194,15 @@ def create_job( ) scheduler.start() + def create_procedure(self, procedure_def_dict): + with hiactor_client.ApiClient( + hiactor_client.Configuration(self._hiactor_host) + ) as api_client: + api_instance = hiactor_client.ProcedureApi(api_client) + graph_name = procedure_def_dict["bound_graph"] + procedure = hiactor_client.Procedure.from_dict(procedure_def_dict) + return api_instance.create_procedure(graph_name, procedure) + def init_interactive_service(config: Config): return Hiactor(config) diff --git a/coordinator/gscoordinator/servicer/flex/interactive/job_scheduler.py b/coordinator/gscoordinator/servicer/flex/interactive/job_scheduler.py index 07fa7baa2d9f..7d2182776387 100644 --- a/coordinator/gscoordinator/servicer/flex/interactive/job_scheduler.py +++ b/coordinator/gscoordinator/servicer/flex/interactive/job_scheduler.py @@ -36,7 +36,7 @@ def __init__(self, at_time, repeat, description, servicer): self._description = description self._servicer = servicer - self._tags = [JobType.GRAPH_IMPORT] + self._tags = [JobType.DATALOADING.name] def run(self): """This function needs to handle exception by itself""" @@ -47,7 +47,7 @@ def run(self): detail = {"graph name": graph_name} status = JobStatus( jobid=self.jobid, - type=JobType.GRAPH_IMPORT, + type=JobType.DATALOADING, start_time=self.last_run, detail=detail, ) @@ -74,4 +74,4 @@ def run(self): ) status.set_failed(message=str(e)) else: - status.set_success(message=api_response.message) + status.set_success(message=api_response) diff --git a/coordinator/gscoordinator/servicer/flex/job.py b/coordinator/gscoordinator/servicer/flex/job.py index 20f5fc7d5055..8d34775fa5ca 100644 --- a/coordinator/gscoordinator/servicer/flex/job.py +++ b/coordinator/gscoordinator/servicer/flex/job.py @@ -23,7 +23,8 @@ class JobType(Enum): - GRAPH_IMPORT = 0 + SCHEDULER = 0 + DATALOADING = 1 class Status(Enum): @@ -42,6 +43,7 @@ def __init__( start_time, status=Status.RUNNING, end_time=None, + log="", detail=dict(), message="", ): @@ -50,10 +52,24 @@ def __init__( self.status = status self.start_time = start_time self.end_time = end_time + self.log = log # detail for specific job self.detail = detail self.message = message + @staticmethod + def from_dict(data): + return JobStatus( + jobid=data["jobid"], + type=data["type"], + status=data["status"], + start_time=data["start_time"], + end_time=data["end_time"], + log=data["log"], + detail=data["detail"], + message=data["message"], + ) + def to_dict(self): return { "jobid": self.jobid, @@ -61,6 +77,7 @@ def to_dict(self): "status": self.status.name, "start_time": encode_datetime(self.start_time), "end_time": encode_datetime(self.end_time), + "log": self.log, "detail": self.detail, "message": self.message, } diff --git a/coordinator/gscoordinator/servicer/flex/scheduler.py b/coordinator/gscoordinator/servicer/flex/scheduler.py index 5a79dde9cb9b..9318b63da3c5 100644 --- a/coordinator/gscoordinator/servicer/flex/scheduler.py +++ b/coordinator/gscoordinator/servicer/flex/scheduler.py @@ -18,9 +18,11 @@ import datetime import json +import random import time from abc import ABCMeta from abc import abstractmethod +from string import ascii_uppercase import schedule from schedule import CancelJob @@ -67,8 +69,8 @@ class Scheduler(metaclass=ABCMeta): def __init__(self, at_time, repeat): # scheduler id - self._scheduler_id = "Job-scheduler-{0}".format( - datetime.datetime.now().strftime("%Y-%m-%d-%H-%M-%S") + self._scheduler_id = "SCHEDULER-{0}".format( + "".join(random.choices(ascii_uppercase, k=16)) ) # periodic job as used self._job = None @@ -176,8 +178,8 @@ def waiting_until_to_run(self): def do_run(self): """Start a thread for the job.""" # overwrite for each scheduled job - self._jobid = "job-{0}".format( - datetime.datetime.now().strftime("%Y-%m-%d-%H-%M-%S") + self._jobid = "JOB-{0}".format( + "".join(random.choices(ascii_uppercase, k=16)) ) self._last_run = datetime.datetime.now() # schedule in a thread diff --git a/coordinator/gscoordinator/servicer/flex/service.py b/coordinator/gscoordinator/servicer/flex/service.py index 2219f9ca267e..7640b79db6b0 100644 --- a/coordinator/gscoordinator/servicer/flex/service.py +++ b/coordinator/gscoordinator/servicer/flex/service.py @@ -19,6 +19,7 @@ """Service under FLEX Architecture""" import atexit +import datetime import traceback import functools @@ -35,7 +36,12 @@ from graphscope.proto import flex_pb2 from graphscope.proto import message_pb2 -# from gscoordinator.scheduler import schedule +from gscoordinator.servicer.flex.job import JobStatus +from gscoordinator.servicer.flex.job import JobType +from gscoordinator.servicer.flex.job import Status +from gscoordinator.servicer.flex.scheduler import cancel_job +from gscoordinator.servicer.flex.scheduler import schedule + # from gscoordinator.utils import WORKSPACE # from gscoordinator.utils import delegate_command_to_pod # from gscoordinator.utils import run_kube_cp_command @@ -136,16 +142,68 @@ def CreateJob(self, request, context): ) return flex_pb2.ApiResponse(code=error_codes_pb2.OK) + @handle_api_exception(flex_pb2.ListProcedureResponse) + def ListProcedure(self, request, context): + pass + @handle_api_exception(flex_pb2.ListJobResponse) def ListJob(self, request, context): + # job to be scheduled in the future + jobs = {} + for job in schedule.get_jobs(): + if datetime.datetime.now() >= job.next_run: + continue + for tag in job.tags: + if tag.startswith("SCHEDULER"): + jobid = tag + jobs[jobid] = JobStatus.from_dict( + { + "jobid": jobid, + "type": JobType.SCHEDULER, + "status": Status.WAITING, + "start_time": str(job.next_run), + "end_time": None, + "log": "", + "detail": {"tag": str(job.tags)}, + "message": "", + } + ) + jobs.update(self._service_client.job_status) return flex_pb2.ListJobResponse( code=error_codes_pb2.OK, job_status=[ dict_to_proto_message(s.to_dict(), flex_pb2.JobStatus()) - for _, s in self._service_client.job_status.items() + for _, s in jobs.items() ], ) + @handle_api_exception(flex_pb2.ApiResponse) + def CancelJob(self, request, context): + jobid = request.jobid + delete_scheduler = request.delete_scheduler + if jobid in self._service_client.job_status: + raise RuntimeError("Job cancellation is not supported yet!") + # cancel job scheduler + scheduler = None + for job in schedule.get_jobs(): + if jobid in job.tags: + scheduler = job + break + if scheduler is None: + raise RuntimeError("Job {0} not found".format(jobid)) + cancel_job(scheduler, delete_scheduler) + return flex_pb2.ApiResponse(code=error_codes_pb2.OK) + + @handle_api_exception(flex_pb2.ApiResponse) + def CreateProcedure(self, request, context): + procedure_def_dict = MessageToDict( + request.procedure_def, + preserving_proto_field_name=True, + including_default_value_fields=True, + ) + api_response = self._service_client.create_procedure(procedure_def_dict) + return flex_pb2.ApiResponse(code=error_codes_pb2.OK, error_msg=api_response) + def init_flex_service_servicer(config: Config): return FlexServiceServicer(config) diff --git a/proto/coordinator_service.proto b/proto/coordinator_service.proto index 6d7fddff3e21..042f43f17c0b 100644 --- a/proto/coordinator_service.proto +++ b/proto/coordinator_service.proto @@ -63,11 +63,13 @@ service CoordinatorService { rpc ListJob(ListJobRequest) returns (ListJobResponse); - // end + rpc CancelJob(CancelJobRequest) returns (ApiResponse); + + rpc CreateProcedure(CreateProcedureRequest) returns (ApiResponse); - rpc CreateInteractiveProcedure(CreateInteractiveProcedureRequest) returns (ApiResponse); + rpc ListProcedure(ListProcedureRequest) returns (ListProcedureResponse); - rpc ListInteractiveProcedure(ListInteractiveProcedureRequest) returns (ListInteractiveProcedureResponse); + // end rpc UpdateInteractiveProcedure(UpdateInteractiveProcedureRequest) returns (ApiResponse); diff --git a/proto/flex.proto b/proto/flex.proto index 86229e9aefed..97bf67e06b93 100644 --- a/proto/flex.proto +++ b/proto/flex.proto @@ -136,7 +136,7 @@ message DataSource { // format message DataSourceFormat { string type = 1; // csv - map metadata = 2; + map metadata = 2; } // loading config @@ -161,8 +161,9 @@ message JobStatus { string status = 3; string start_time = 4; string end_time = 5; - map detail = 6; - string message = 7; + string log = 6; + map detail = 7; + string message = 8; } // procedure runtime params, used both in property and return type @@ -176,7 +177,7 @@ message Procedure { string name = 1; string bound_graph = 2; string description = 3; - optional string type = 4; // [ cpp, cypher ] + string type = 4; // [ cpp, cypher ] string query = 5; // content bool enable = 6; @@ -209,8 +210,8 @@ message NodeStatus { // job schedule message Schedule { bool run_now = 1; - string time = 2; // formate with "2023-02-21 11:56:30" - string repeate = 3; // day, week, or null + string at_time = 2; // formate with "2023-02-21 11:56:30" + string repeat = 3; // day, week, or null } // AnalyticalJobPlaceHolder @@ -265,15 +266,18 @@ message ListJobResponse { repeated JobStatus job_status = 3; } -message CreateInteractiveProcedureRequest { - Procedure procedure_def = 1; +message CancelJobRequest { + string jobid = 1; + bool delete_scheduler = 2; } -message ListInteractiveProcedureRequest { - string graph_name = 1; +message CreateProcedureRequest { + Procedure procedure_def = 1; } -message ListInteractiveProcedureResponse { +message ListProcedureRequest {} + +message ListProcedureResponse { Code code = 1; optional string error_msg = 2; diff --git a/python/graphscope/gsctl/client/rpc.py b/python/graphscope/gsctl/client/rpc.py index fb0ba99cf776..53b28cf34969 100644 --- a/python/graphscope/gsctl/client/rpc.py +++ b/python/graphscope/gsctl/client/rpc.py @@ -121,14 +121,19 @@ def list_job(self): return self._stub.ListJob(request) @handle_grpc_error - def create_interactive_procedure(self, procedure: flex_pb2.Procedure): - request = flex_pb2.CreateInteractiveProcedureRequest(procedure_def=procedure) - return self._stub.CreateInteractiveProcedure(request) + def cancel_job(self, jobid: str, delete_scheduler: bool): + request = flex_pb2.CancelJobRequest(jobid=jobid, delete_scheduler=delete_scheduler) + return self._stub.CancelJob(request) @handle_grpc_error - def list_interactive_procedure(self, graph: str): - request = flex_pb2.ListInteractiveProcedureRequest(graph_name=graph) - return self._stub.ListInteractiveProcedure(request) + def create_procedure(self, procedure_def: flex_pb2.Procedure): + request = flex_pb2.CreateProcedureRequest(procedure_def=procedure_def) + return self._stub.CreateProcedure(request) + + @handle_grpc_error + def list_procedure(self): + request = flex_pb2.ListProcedureRequest() + return self._stub.ListProcedure(request) @handle_grpc_error def update_interactive_procedure(self, procedures: List[flex_pb2.Procedure]): diff --git a/python/graphscope/gsctl/commands/interactive.py b/python/graphscope/gsctl/commands/interactive.py index 9b9766ddd0c4..fd7d75d9b51d 100644 --- a/python/graphscope/gsctl/commands/interactive.py +++ b/python/graphscope/gsctl/commands/interactive.py @@ -136,6 +136,37 @@ def graph(name, filename): ) +@create.command() +@click.option( + "-f", + "--filename", + required=True, + help="Path of yaml file to use to create a procedure", +) +def procedure(filename): + """Create a stored procedure in database""" + + if not is_valid_file_path(filename): + click.secho("Invalid file: {0}".format(filename), fg="blue") + return + + procedure_dict = read_yaml_file(filename) + # transform procedure dict to proto message + procedure_def = flex_pb2.Procedure() + dict_to_proto_message(procedure_dict, procedure_def) + + grpc_client = get_grpc_client() + if not grpc_client.connect(): + return + + response = grpc_client.create_procedure(procedure_def) + + if is_success(response): + click.secho( + "Create stored procedure successfully", fg="green", + ) + + @create.command() @click.option( "-f", @@ -184,7 +215,7 @@ def _read_and_fill_raw_data(config): return response = grpc_client.create_job( - type="dataloading", + type=job["type"], schedule=schedule, description=description, ) @@ -198,50 +229,6 @@ def _read_and_fill_raw_data(config): click.secho(str(e), fg="red") -@create.command() -@click.option( - "-g", - "--graph", - required=True, - help="The name of the graph", -) -@click.option( - "-n", - "--name", - required=True, - help="The name of the procedure", -) -@click.option( - "-i", - "--sourcefile", - required=True, - help="Path of [ .cc, .cypher ] file", -) -@click.option( - "-d", - "--description", - required=False, - help="Description for the specific procedure", -) -def procedure(graph, name, sourcefile, description): - """Create and compile procedure over a specific graph""" - - # read source - with open(sourcefile, "r") as f: - query = f.read() - - # construct procedure proto - procedure = flex_pb2.Procedure( - name=name, bound_graph=graph, description=description, query=query, enable=True - ) - - grpc_client = get_grpc_client() - response = grpc_client.create_interactive_procedure(procedure) - - if is_success(response): - click.secho("Create procedure {0} successfully.".format(name), fg="green") - - @delete.command() @click.argument("graph", required=True) def graph(graph): @@ -256,6 +243,25 @@ def graph(graph): fg="green", ) +@delete.command() +@click.argument("job", required=True) +@click.option( + '--delete-scheduler/--no-delete-scheduler', + default=False, + help="True will delete the job scheduler, otherwise just cancel the next schedule", +) +def job(job, delete_scheduler): + """Cancel a job by jobid""" + + grpc_client = get_grpc_client() + response = grpc_client.cancel_job(job, delete_scheduler) + + if is_success(response): + click.secho( + "Cancel job {0} successfully.".format(job), + fg="green", + ) + @delete.command() @click.option( @@ -496,12 +502,6 @@ def _construct_and_display_data(job_status): @get.command() -@click.option( - "-g", - "--graph", - required=True, - help="The name of the graph", -) def procedure(graph): """Display procedures in database""" @@ -526,11 +526,10 @@ def _construct_and_display_data(procedures): terminal_display(data) grpc_client = get_grpc_client() - response = grpc_client.list_interactive_procedure(graph) + response = grpc_client.list_procedure() if is_success(response): _construct_and_display_data(response.procedures) - return response.procedures @get.command() diff --git a/python/graphscope/gsctl/utils.py b/python/graphscope/gsctl/utils.py index 30c5d62d9d43..4610b5392de3 100644 --- a/python/graphscope/gsctl/utils.py +++ b/python/graphscope/gsctl/utils.py @@ -74,14 +74,15 @@ def _parse_dict(values, message): setattr(message, k, v) else: try: - # treat as a map - if isinstance(v, bool): - message[k].CopyFrom(flex_pb2.MapValue(b=v)) - elif isinstance(v, int): - message[k].CopyFrom(flex_pb2.MapValue(i=v)) - elif isinstance(v, str): - message[k].CopyFrom(flex_pb2.MapValue(s=v)) - except AttributeError as e: + message[k] = str(v) + # # treat as a map + # if isinstance(v, bool): + # message[k].CopyFrom(flex_pb2.MapValue(b=v)) + # elif isinstance(v, int): + # message[k].CopyFrom(flex_pb2.MapValue(i=v)) + # elif isinstance(v, str): + # message[k].CopyFrom(flex_pb2.MapValue(s=v)) + except Exception as e: click.secho(str(e), fg="red") _parse_dict(values, message)