From 86d51300cf1f1f9a4a2dcf9451e59f47fb7e4958 Mon Sep 17 00:00:00 2001 From: Alex Barcelo Date: Fri, 6 Sep 2024 12:55:32 +0200 Subject: [PATCH 01/24] some problem due to concurrency or race condition --- src/dataclay/contrib/modeltest/concurrency.py | 24 ++++++ tests/functional/test_concurrency.py | 77 +++++++++++++++++++ 2 files changed, 101 insertions(+) create mode 100644 src/dataclay/contrib/modeltest/concurrency.py create mode 100644 tests/functional/test_concurrency.py diff --git a/src/dataclay/contrib/modeltest/concurrency.py b/src/dataclay/contrib/modeltest/concurrency.py new file mode 100644 index 0000000..8dd03b0 --- /dev/null +++ b/src/dataclay/contrib/modeltest/concurrency.py @@ -0,0 +1,24 @@ +import time + +from dataclay import DataClayObject, activemethod + + +class PingPong(DataClayObject): + _event: bool + pong_obj: 'PingPong' + + def __init__(self): + self._event = False + self.pong_obj = None + + @activemethod + def event_set(self): + self._event = True + + @activemethod + def ping(self, chain=1, wait_event=True): + if chain > 0 and self.pong_obj is not None: + self.pong_obj.ping(chain - 1, wait_event) + if wait_event: + while not self._event: + time.sleep(1) diff --git a/tests/functional/test_concurrency.py b/tests/functional/test_concurrency.py new file mode 100644 index 0000000..68c379d --- /dev/null +++ b/tests/functional/test_concurrency.py @@ -0,0 +1,77 @@ +import asyncio +import pytest + +import concurrent.futures +from threading import Thread + +from dataclay.event_loop import get_dc_event_loop +from dataclay.config import get_runtime + +from dataclay.contrib.modeltest.concurrency import PingPong + +def test_sync_basic_concurrency(client): + """ + Test the synchronous execution of nested activemethods + """ + backends = list(client.get_backends().keys()) + pp1 = PingPong() + pp1.make_persistent(backend_id=backends[0]) + pp2 = PingPong() + pp2.make_persistent(backend_id=backends[1]) + pp1.pong_obj = pp2 + pp2.pong_obj = pp1 + + # Some low-level asyncio stuff here, because we cannot use Threads and + # starting a new client instance seems overkill (but would be best, would it?) + future = asyncio.run_coroutine_threadsafe( + get_runtime().call_remote_method(pp1, "ping", tuple(), dict()), + get_dc_event_loop(), + ) + + with pytest.raises(concurrent.futures.TimeoutError): + future.result(timeout=1) + + pp2.ping(wait_event=False) + + pp1.event_set() + pp2.event_set() + + future.result(timeout=1) + assert future.done() + +def test_sync_high_concurrency(client): + """ + Test the synchronous execution of nested activemethods + """ + backends = list(client.get_backends().keys()) + pp1 = PingPong() + pp1.make_persistent(backend_id=backends[0]) + pp2 = PingPong() + pp2.make_persistent(backend_id=backends[1]) + pp1.pong_obj = pp2 + pp2.pong_obj = pp1 + + future1 = asyncio.run_coroutine_threadsafe( + get_runtime().call_remote_method(pp1, "ping", (10, True), {}), + get_dc_event_loop(), + ) + + future2 = asyncio.run_coroutine_threadsafe( + get_runtime().call_remote_method(pp2, "ping", (10, True), {}), + get_dc_event_loop(), + ) + + with pytest.raises(concurrent.futures.TimeoutError): + future1.result(timeout=1) + + with pytest.raises(concurrent.futures.TimeoutError): + future2.result(timeout=1) + + pp2.ping(wait_event=False) + pp1.event_set() + pp2.event_set() + + future1.result(timeout=1) + future2.result(timeout=1) + assert future1.done() + assert future2.done() From 80d5e7a4af2435acbc6b56919274a408beee9bec Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Mon, 30 Sep 2024 23:29:15 +0800 Subject: [PATCH 02/24] Updating logs and comments --- src/dataclay/backend/api.py | 36 +++++++-------- src/dataclay/backend/client.py | 6 --- src/dataclay/backend/servicer.py | 3 ++ src/dataclay/client/api.py | 7 +-- src/dataclay/dataclay_object.py | 12 ++--- src/dataclay/metadata/servicer.py | 1 + src/dataclay/runtime.py | 75 ++++++++++++++++++++++--------- 7 files changed, 83 insertions(+), 57 deletions(-) diff --git a/src/dataclay/backend/api.py b/src/dataclay/backend/api.py index fe2cd91..d4c2549 100644 --- a/src/dataclay/backend/api.py +++ b/src/dataclay/backend/api.py @@ -133,7 +133,7 @@ async def call_active_method( ) -> tuple[bytes, bool]: """Entry point for calling an active method of a DataClayObject""" - logger.debug("(%s) Receiving call to activemethod '%s'", object_id, method_name) + logger.debug("(%s) Receiving remote call to activemethod '%s'", object_id, method_name) instance = await self.runtime.get_object_by_id(object_id) @@ -146,11 +146,10 @@ async def call_active_method( # by passing the backend_id of the new object to the proxy, but this can create # problems with race conditions (e.g. a move before the consolidation). Therefore, # we check to the metadata which is more reliable. + logger.warning("(%s) Wrong backend", object_id) await self.runtime.sync_object_metadata(instance) logger.warning( - "(%s) Wrong backend. Update to %s", - object_id, - instance._dc_meta.master_backend_id, + "(%s) Update backend to %s", object_id, instance._dc_meta.master_backend_id ) return ( pickle.dumps( @@ -169,18 +168,19 @@ async def call_active_method( max_threads = ( None if exec_constraints.get("max_threads", 0) == 0 else exec_constraints["max_threads"] ) - logger.info("(%s) Max threads: %s", object_id, max_threads) + logger.info("(%s) Max threads for activemethod: %s", object_id, max_threads) with threadpool_limits(limits=max_threads): try: func = getattr(instance, method_name) if asyncio.iscoroutinefunction(func): - logger.debug("(%s) Activemethod '%s' is a coroutine", object_id, method_name) + logger.debug("(%s) Awaiting activemethod coroutine", object_id) result = await func(*args, **kwargs) else: + logger.debug("(%s) Running activemethod in new thread", object_id) result = await dc_to_thread(func, *args, **kwargs) except Exception as e: # If an exception was raised, serialize it and return it to be raised by the client - logger.debug("(%s) *** Exception in activemethod '%s'", object_id, method_name) + logger.info("(%s) *** Exception in activemethod '%s'", object_id, method_name) return pickle.dumps(e), True logger.info("(%s) *** Finished activemethod '%s' in executor", object_id, method_name) @@ -192,7 +192,6 @@ async def call_active_method( # Store Methods - # TODO: Rename to get_object_property @tracer.start_as_current_span("get_object_attribute") async def get_object_attribute(self, object_id: UUID, attribute: str) -> tuple[bytes, bool]: """Returns value of the object attibute with ID provided @@ -203,14 +202,13 @@ async def get_object_attribute(self, object_id: UUID, attribute: str) -> tuple[b The pickled value of the object attibute. If it's an exception or not """ - logger.debug("(%s) Receiving get attribute '%s'", object_id, attribute) + logger.debug("(%s) Receiving remote call to __getattribute__ '%s'", object_id, attribute) instance = await self.runtime.get_object_by_id(object_id) if not instance._dc_is_local: + logger.warning("(%s) Wrong backend", object_id) await self.runtime.sync_object_metadata(instance) logger.warning( - "(%s) Wrong backend. Update to %s", - object_id, - instance._dc_meta.master_backend_id, + "(%s) Update backend to %s", object_id, instance._dc_meta.master_backend_id ) return ( pickle.dumps( @@ -232,16 +230,14 @@ async def set_object_attribute( self, object_id: UUID, attribute: str, serialized_attribute: bytes ) -> tuple[bytes, bool]: """Updates an object attibute with ID provided""" - logger.debug("(%s) Receiving set attribute '%s'", object_id, attribute) + logger.debug("(%s) Receiving remote call to __setattr__ '%s'", object_id, attribute) instance = await self.runtime.get_object_by_id(object_id) if not instance._dc_is_local: + logger.warning("(%s) Wrong backend", object_id) await self.runtime.sync_object_metadata(instance) logger.warning( - "(%s) Wrong backend. Update to %s", - object_id, - instance._dc_meta.master_backend_id, + "(%s) Update backend to %s", object_id, instance._dc_meta.master_backend_id ) - return ( pickle.dumps( ObjectWithWrongBackendIdError( @@ -260,13 +256,13 @@ async def set_object_attribute( @tracer.start_as_current_span("del_object_attribute") async def del_object_attribute(self, object_id: UUID, attribute: str) -> tuple[bytes, bool]: """Deletes an object attibute with ID provided""" + logger.debug("(%s) Receiving remote call to __delattr__'%s'", object_id, attribute) instance = await self.runtime.get_object_by_id(object_id) if not instance._dc_is_local: + logger.warning("(%s) Wrong backend", object_id) await self.runtime.sync_object_metadata(instance) logger.warning( - "(%s) Wrong backend. Update to %s", - object_id, - instance._dc_meta.master_backend_id, + "(%s) Update backend to %s", object_id, instance._dc_meta.master_backend_id ) return ( pickle.dumps( diff --git a/src/dataclay/backend/client.py b/src/dataclay/backend/client.py index 963652f..906d9c7 100644 --- a/src/dataclay/backend/client.py +++ b/src/dataclay/backend/client.py @@ -199,12 +199,6 @@ async def get_object_attribute(self, object_id: UUID, attribute: str) -> tuple[b ("authorization", current_context["token"]), ] response = await self.stub.GetObjectAttribute(request, metadata=metadata) - current_context = session_var.get() - metadata = self.metadata_call + [ - ("username", current_context["username"]), - ("authorization", current_context["token"]), - ] - response = await self.stub.GetObjectAttribute(request, metadata=metadata) return response.value, response.is_exception @grpc_aio_error_handler diff --git a/src/dataclay/backend/servicer.py b/src/dataclay/backend/servicer.py index f00f921..bedef71 100644 --- a/src/dataclay/backend/servicer.py +++ b/src/dataclay/backend/servicer.py @@ -72,6 +72,8 @@ def _get_or_generate_backend_id() -> UUID: async def serve(): + + # Set the event loop created by `asyncio.run` as the dataclay event loop set_dc_event_loop(asyncio.get_running_loop()) backend_id = _get_or_generate_backend_id() @@ -135,6 +137,7 @@ async def serve(): ) # Wait for the server to stop + logger.info("Backend service started") await server.wait_for_termination() logger.info("Backend service stopped") await backend.stop() diff --git a/src/dataclay/client/api.py b/src/dataclay/client/api.py index ff21246..df93643 100644 --- a/src/dataclay/client/api.py +++ b/src/dataclay/client/api.py @@ -177,15 +177,14 @@ def start(self): loop = get_dc_event_loop() if loop == None: - logger.info("Create_new_loop") - # Create new event loop in a new thread + logger.info("Creating event loop in new thread") loop = asyncio.new_event_loop() set_dc_event_loop(loop) event_loop_thread = EventLoopThread(loop) event_loop_thread.start() event_loop_thread.ready.wait() else: - logger.info("Already_had_a_loop") + logger.info("Using existing event loop") # Replace settings self.previous_settings = settings.client @@ -210,6 +209,7 @@ def start(self): settings.client.dataclay_host, settings.client.dataclay_port ) + logger.info("Starting client runtime coroutine in event loop") future = asyncio.run_coroutine_threadsafe(self.runtime.start(), loop) future.result() @@ -227,6 +227,7 @@ def start(self): # self.runtime.dataclay_id self.is_active = True + logger.info("Client runtime started") @tracer.start_as_current_span("stop") def stop(self): diff --git a/src/dataclay/dataclay_object.py b/src/dataclay/dataclay_object.py index 713ff66..940ef1b 100644 --- a/src/dataclay/dataclay_object.py +++ b/src/dataclay/dataclay_object.py @@ -138,7 +138,7 @@ def __get__(self, instance: DataClayObject, owner): ) if instance._dc_is_local: - logger.debug("local get") + logger.debug("(%s) Calling local __getattribute__", instance._dc_meta.id) # If the object is local and loaded, we can access the attribute directly if not instance._dc_is_loaded: # NOTE: Should be called from another thread. @@ -159,7 +159,7 @@ def __get__(self, instance: DataClayObject, owner): else: return self.transformer.getter(attr) else: - logger.debug("remote get") + logger.debug("(%s) Calling remote __getattribute__", instance._dc_meta.id) return asyncio.run_coroutine_threadsafe( get_runtime().call_remote_method(instance, "__getattribute__", (self.name,), {}), get_dc_event_loop(), @@ -178,7 +178,7 @@ def __set__(self, instance: DataClayObject, value): ) if instance._dc_is_local: - logger.debug("local set") + logger.debug("(%s) Calling local __setattr__", instance._dc_meta.id) if not instance._dc_is_loaded: assert get_dc_event_loop()._thread_id != threading.get_ident() asyncio.run_coroutine_threadsafe( @@ -188,7 +188,7 @@ def __set__(self, instance: DataClayObject, value): value = self.transformer.setter(value) setattr(instance, self.dc_property_name, value) else: - logger.debug("remote set") + logger.debug("(%s) Calling remote __setattr__", instance._dc_meta.id) return asyncio.run_coroutine_threadsafe( get_runtime().call_remote_method(instance, "__setattr__", (self.name, value), {}), get_dc_event_loop(), @@ -204,7 +204,7 @@ def __delete__(self, instance: DataClayObject): ) if instance._dc_is_local: - logger.debug("local delete") + logger.debug("(%s) Calling local __delattr__", instance._dc_meta.id) if not instance._dc_is_loaded: assert get_dc_event_loop()._thread_id != threading.get_ident() asyncio.run_coroutine_threadsafe( @@ -213,7 +213,7 @@ def __delete__(self, instance: DataClayObject): delattr(instance, self.dc_property_name) else: - logger.debug("remote delete") + logger.debug("(%s) Calling remote __delattr__", instance._dc_meta.id) return asyncio.run_coroutine_threadsafe( get_runtime().call_remote_method(instance, "__delattr__", (self.name,), {}), get_dc_event_loop(), diff --git a/src/dataclay/metadata/servicer.py b/src/dataclay/metadata/servicer.py index cc58d07..626694e 100644 --- a/src/dataclay/metadata/servicer.py +++ b/src/dataclay/metadata/servicer.py @@ -89,6 +89,7 @@ async def serve(): ) # Wait for the server to stop + logger.info("MetadataService started") await server.wait_for_termination() logger.info("MetadataService stopped") await metadata_servicer.backend_clients.stop() diff --git a/src/dataclay/runtime.py b/src/dataclay/runtime.py index 3893dae..d415ede 100644 --- a/src/dataclay/runtime.py +++ b/src/dataclay/runtime.py @@ -186,11 +186,12 @@ async def get_object_by_id( """Get dataclay object from inmemory_objects. If not present, get object metadata and create new proxy object. """ - logger.debug("(%s) Getting object by id", object_id) + logger.debug("(%s) Getting dataclay object by id", object_id) try: dc_object = self.inmemory_objects[object_id] self.dataclay_inmemory_hits_total.inc() + logger.debug("(%s) Object found in inmemory_objects", object_id) return dc_object except KeyError: async with lock_manager.get_lock(object_id).writer_lock: @@ -203,12 +204,11 @@ async def get_object_by_id( # we get the object metadata from kvstore, and create a new proxy # object from it. - logger.debug( - "(%s) Object not in inmemory_objects, creating new proxy", object_id - ) + logger.debug("(%s) Object not found in inmemory_objects", object_id) # If object metadata is not provided, get it from the metadata service if object_md is None: + logger.debug("(%s) Getting object metadata from MDS", object_id) object_md = await self.metadata_service.get_object_md_by_id(object_id) # Create a new proxy object @@ -237,6 +237,9 @@ async def get_object_by_id( # if many calls are made to the same object, and this is deleted every time. # TODO: Check if this is really the case. If so, gc should act in a LIFO way. self.inmemory_objects[proxy_object._dc_meta.id] = proxy_object + logger.debug( + "(%s) Proxy object created and added to inmemory_objects", object_id + ) return proxy_object async def get_object_by_alias(self, alias: str, dataset_name: str = None) -> DataClayObject: @@ -359,14 +362,17 @@ async def call_remote_method( ) # Fault tolerance loop + num_retries = 0 while True: + num_retries += 1 + logger.debug("(%s) Attempt %s", instance._dc_meta.id, num_retries) # Get the intersection between backend clients and object backends avail_backends = instance._dc_all_backend_ids.intersection( self.backend_clients.keys() ) # If the intersection is empty (no backends available), update the list of backend - # clients and the object backend locations, and try again + # clients and the object backend locations, and try again... if not avail_backends: logger.warning("(%s) No backends available. Syncing...", instance._dc_meta.id) await asyncio.gather(self.backend_clients.update(), instance.a_sync()) @@ -382,10 +388,14 @@ async def call_remote_method( # Choose a random backend from the available ones backend_id = random.choice(tuple(avail_backends)) backend_client = await self.backend_clients.get(backend_id) + logger.debug("(%s) Backend %s chosen", instance._dc_meta.id, backend_id) # If the connection fails, update the list of backend clients, and try again try: if method_name == "__getattribute__": + logger.debug( + "(%s) Getting remote attribute %s", instance._dc_meta.id, args[0] + ) ( serialized_response, is_exception, @@ -394,6 +404,9 @@ async def call_remote_method( args[0], # attribute name ) elif method_name == "__setattr__": + logger.debug( + "(%s) Setting remote attribute %s", instance._dc_meta.id, args[0] + ) ( serialized_response, is_exception, @@ -403,6 +416,9 @@ async def call_remote_method( await dcdumps(args[1]), # attribute value ) elif method_name == "__delattr__": + logger.debug( + "(%s) Deleting remote attribute %s", instance._dc_meta.id, args[0] + ) ( serialized_response, is_exception, @@ -411,8 +427,12 @@ async def call_remote_method( args[0], # attribute name ) else: - # Print the constraings - logger.debug("Method Constraints: %s", exec_constraints_var.get()) + logger.debug( + "(%s) Executing remote method %s with constraints %s", + instance._dc_meta.id, + method_name, + exec_constraints_var.get(), + ) serialized_response, is_exception = await backend_client.call_active_method( object_id=instance._dc_meta.id, method_name=method_name, @@ -422,31 +442,42 @@ async def call_remote_method( ) except DataClayException as e: if "failed to connect" in str(e): - logger.warning("(%s) Connection failed. Syncing...", instance._dc_meta.id) + logger.warning("(%s) Connection failed. Retrying...", instance._dc_meta.id) await self.backend_clients.update() continue else: raise e + # Deserialize the response if not None if serialized_response: + logger.debug("(%s) Deserializing response", instance._dc_meta.id) response = await dcloads(serialized_response) + else: + logger.debug("(%s) Response is None", instance._dc_meta.id) + response = None - # If response is ObjectWithWrongBackendIdError, update object metadata and retry - if isinstance(response, ObjectWithWrongBackendIdError): - instance._dc_meta.master_backend_id = response.backend_id - instance._dc_meta.replica_backend_ids = response.replica_backend_ids - continue - - # If the response is and exception, raise it. Correct workflow. - # NOTE: The exception was raised inside the active method - if is_exception: - raise response + # If response is ObjectWithWrongBackendIdError, update object metadata and retry + if isinstance(response, ObjectWithWrongBackendIdError): + logger.warning( + "(%s) Object with wrong backend id. Retrying...", instance._dc_meta.id + ) + instance._dc_meta.master_backend_id = response.backend_id + instance._dc_meta.replica_backend_ids = response.replica_backend_ids + continue - return response + # If the response is an exception, it is raised + if is_exception: + logger.debug( + "(%s) Remote method %s raised an exception", + instance._dc_meta.id, + method_name, + ) + raise response - else: - # Void active method returns None - return None + logger.info( + "(%s) Remote method %s executed successfully", instance._dc_meta.id, method_name + ) + return response ######### # Alias # From 100ee65c2e1551d0f2fdd384b5d1ffc6d05b859a Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Tue, 8 Oct 2024 01:38:51 +0800 Subject: [PATCH 03/24] Add ThreadPoolExecutor's for CPU and IO bound tasks. Check the number of CPUs after numactl restrictions to decide the max_workers of the executors --- orchestration/mn5/config/external_vars.yml | 1 + src/dataclay/backend/api.py | 13 +++++---- src/dataclay/client/api.py | 3 ++ src/dataclay/config.py | 9 ++++-- src/dataclay/data_manager.py | 8 +++--- src/dataclay/dataclay_object.py | 5 ++++ src/dataclay/event_loop.py | 32 ++++++++++++++++++++-- src/dataclay/runtime.py | 14 ++++++---- src/dataclay/utils/serialization.py | 31 +++++++++------------ 9 files changed, 77 insertions(+), 39 deletions(-) diff --git a/orchestration/mn5/config/external_vars.yml b/orchestration/mn5/config/external_vars.yml index 3f729c6..51fcda7 100644 --- a/orchestration/mn5/config/external_vars.yml +++ b/orchestration/mn5/config/external_vars.yml @@ -14,6 +14,7 @@ global_env: DATACLAY_TRACING: "{{ lookup('env', 'DATACLAY_TRACING') | default('false', true) }}" DATACLAY_TRACING_HOST: "{{ lookup('env', 'DATACLAY_TRACING_HOST') }}" DATACLAY_JOB_PATH: "{{ job_path }}" # Used in otel-json-exporter.yaml + DATACLAY_IO_BOUND_MULTIPLIER: "{{ lookup('env', 'DATACLAY_IO_BOUND_MULTIPLIER') | default(2, true) }}" backend_properties: DATACLAY_BACKEND_PORT: "{{ lookup('env', 'DATACLAY_BACKEND_PORT') | default(6867, true) }}" diff --git a/src/dataclay/backend/api.py b/src/dataclay/backend/api.py index d4c2549..fa80174 100644 --- a/src/dataclay/backend/api.py +++ b/src/dataclay/backend/api.py @@ -15,7 +15,7 @@ from dataclay import utils from dataclay.config import set_runtime, settings -from dataclay.event_loop import dc_to_thread +from dataclay.event_loop import dc_to_thread_io from dataclay.exceptions import ( DataClayException, DoesNotExistError, @@ -169,6 +169,8 @@ async def call_active_method( None if exec_constraints.get("max_threads", 0) == 0 else exec_constraints["max_threads"] ) logger.info("(%s) Max threads for activemethod: %s", object_id, max_threads) + # TODO: Check that the threadpool_limit is not limiting our internal pool of threads. + # like when we are serializing dataclay objects. with threadpool_limits(limits=max_threads): try: func = getattr(instance, method_name) @@ -177,7 +179,7 @@ async def call_active_method( result = await func(*args, **kwargs) else: logger.debug("(%s) Running activemethod in new thread", object_id) - result = await dc_to_thread(func, *args, **kwargs) + result = await dc_to_thread_io(func, *args, **kwargs) except Exception as e: # If an exception was raised, serialize it and return it to be raised by the client logger.info("(%s) *** Exception in activemethod '%s'", object_id, method_name) @@ -219,12 +221,11 @@ async def get_object_attribute(self, object_id: UUID, attribute: str) -> tuple[b False, ) try: - value = await dc_to_thread(getattr, instance, attribute) + value = await dc_to_thread_io(getattr, instance, attribute) return await dcdumps(value), False except Exception as e: return pickle.dumps(e), True - # TODO: Rename to set_object_property @tracer.start_as_current_span("set_object_attribute") async def set_object_attribute( self, object_id: UUID, attribute: str, serialized_attribute: bytes @@ -248,7 +249,7 @@ async def set_object_attribute( ) try: object_attribute = await dcloads(serialized_attribute) - await dc_to_thread(setattr, instance, attribute, object_attribute) + await dc_to_thread_io(setattr, instance, attribute, object_attribute) return None, False except Exception as e: return pickle.dumps(e), True @@ -273,7 +274,7 @@ async def del_object_attribute(self, object_id: UUID, attribute: str) -> tuple[b False, ) try: - await dc_to_thread(delattr, instance, attribute) + await dc_to_thread_io(delattr, instance, attribute) return None, False except Exception as e: return pickle.dumps(e), True diff --git a/src/dataclay/client/api.py b/src/dataclay/client/api.py index df93643..232ab08 100644 --- a/src/dataclay/client/api.py +++ b/src/dataclay/client/api.py @@ -7,6 +7,8 @@ from __future__ import annotations +import threading + __all__ = ["init", "finish", "DataClayObject"] import asyncio @@ -210,6 +212,7 @@ def start(self): ) logger.info("Starting client runtime coroutine in event loop") + assert loop._thread_id != threading.get_ident() # Redundancy check future = asyncio.run_coroutine_threadsafe(self.runtime.start(), loop) future.result() diff --git a/src/dataclay/config.py b/src/dataclay/config.py index e08badd..8158a58 100644 --- a/src/dataclay/config.py +++ b/src/dataclay/config.py @@ -146,11 +146,16 @@ class Settings(BaseSettings): # Other dataclay_id: Optional[uuid.UUID] = Field(default=None, alias="dataclay_id") storage_path: str = "/data/storage/" - thread_pool_max_workers: Optional[int] = None - healthcheck_max_workers: Optional[int] = None loglevel: Annotated[str, StringConstraints(strip_whitespace=True, to_upper=True)] = "INFO" ephemeral: bool = False + # Threads + #: Multiplier for I/O-bound tasks + io_bound_multiplier: int = 2 + # TODO: Rename it with proxy... + thread_pool_max_workers: Optional[int] = None + healthcheck_max_workers: Optional[int] = None + # Timeouts grpc_check_alive_timeout: int = 60 unload_timeout: int = 5 diff --git a/src/dataclay/data_manager.py b/src/dataclay/data_manager.py index 2316eee..9c3533e 100644 --- a/src/dataclay/data_manager.py +++ b/src/dataclay/data_manager.py @@ -9,7 +9,7 @@ import psutil from dataclay.config import settings -from dataclay.event_loop import get_dc_event_loop +from dataclay.event_loop import dc_to_thread_cpu, get_dc_event_loop from dataclay.exceptions import DataClayException from dataclay.lock_manager import lock_manager from dataclay.utils.serialization import DataClayPickler @@ -138,9 +138,9 @@ async def load_object(self, instance: DataClayObject): # Load object from disk try: path = f"{settings.storage_path}/{object_id}" - state, getstate = await get_dc_event_loop().run_in_executor( - None, pickle.load, open(path, "rb") - ) + # TODO: Is it necessary dc_to_thread_cpu? Should be blocking + # to avoid bugs with parallel loads? + state, getstate = await dc_to_thread_cpu(pickle.load, open(path, "rb")) self.dataclay_stored_objects.dec() except Exception as e: raise DataClayException("Object not found.") from e diff --git a/src/dataclay/dataclay_object.py b/src/dataclay/dataclay_object.py index 940ef1b..5a06d2c 100644 --- a/src/dataclay/dataclay_object.py +++ b/src/dataclay/dataclay_object.py @@ -160,6 +160,7 @@ def __get__(self, instance: DataClayObject, owner): return self.transformer.getter(attr) else: logger.debug("(%s) Calling remote __getattribute__", instance._dc_meta.id) + assert get_dc_event_loop()._thread_id != threading.get_ident() return asyncio.run_coroutine_threadsafe( get_runtime().call_remote_method(instance, "__getattribute__", (self.name,), {}), get_dc_event_loop(), @@ -189,6 +190,7 @@ def __set__(self, instance: DataClayObject, value): setattr(instance, self.dc_property_name, value) else: logger.debug("(%s) Calling remote __setattr__", instance._dc_meta.id) + assert get_dc_event_loop()._thread_id != threading.get_ident() return asyncio.run_coroutine_threadsafe( get_runtime().call_remote_method(instance, "__setattr__", (self.name, value), {}), get_dc_event_loop(), @@ -214,6 +216,7 @@ def __delete__(self, instance: DataClayObject): delattr(instance, self.dc_property_name) else: logger.debug("(%s) Calling remote __delattr__", instance._dc_meta.id) + assert get_dc_event_loop()._thread_id != threading.get_ident() return asyncio.run_coroutine_threadsafe( get_runtime().call_remote_method(instance, "__delattr__", (self.name,), {}), get_dc_event_loop(), @@ -460,6 +463,8 @@ def get_by_id(cls, object_id: UUID) -> DataClayObject: # "pickle.loads" of dataClay objects is calling this method behind. With `dcloads` this method # will be called in another thread, so it will not block the event loop. + logger.debug("(%s) Calling get_by_id", object_id) + assert get_dc_event_loop()._thread_id != threading.get_ident() future = asyncio.run_coroutine_threadsafe(cls._get_by_id(object_id), get_dc_event_loop()) return future.result() diff --git a/src/dataclay/event_loop.py b/src/dataclay/event_loop.py index d549213..a8a48e9 100644 --- a/src/dataclay/event_loop.py +++ b/src/dataclay/event_loop.py @@ -1,15 +1,29 @@ import asyncio +import concurrent.futures import contextvars import functools import threading from asyncio import AbstractEventLoop from typing import Awaitable, Union +import psutil + +from dataclay.config import settings + # NOTE: This global event loop is necessary (even if not recommended by asyncio) because # dataClay methods can be called from different threads (when running activemethods in backend) # and we need to access the single event loop from the main thread. dc_event_loop: AbstractEventLoop = None +# Get available CPUs after numactl restriction +cpu_count = len(psutil.Process().cpu_affinity()) +# For CPU-bound tasks, use the number of CPUs available +cpu_bound_executor = concurrent.futures.ThreadPoolExecutor(max_workers=cpu_count) +# For I/O-bound tasks, use a higher multiplier +io_bound_executor = concurrent.futures.ThreadPoolExecutor( + max_workers=cpu_count * settings.io_bound_multiplier +) + def set_dc_event_loop(loop): global dc_event_loop @@ -39,8 +53,8 @@ def run_dc_coroutine(func: Awaitable, *args, **kwargs): return future.result() -# Based on asyncio.to_thread -async def dc_to_thread(func, /, *args, **kwargs): +# Shared helper to run the function in the executor. Based on asyncio.to_thread +async def _dc_to_thread(func, executor, *args, **kwargs): """Asynchronously run function *func* in a separate thread. Any *args and **kwargs supplied for this function are directly passed to *func*. Also, the current :class:`contextvars.Context` is propogated, @@ -51,4 +65,16 @@ async def dc_to_thread(func, /, *args, **kwargs): loop = get_dc_event_loop() ctx = contextvars.copy_context() func_call = functools.partial(ctx.run, func, *args, **kwargs) - return await loop.run_in_executor(None, func_call) + return await loop.run_in_executor(executor, func_call) + + +# For CPU-bound tasks +async def dc_to_thread_cpu(func, /, *args, **kwargs): + """Asynchronously run function *func* in a separate thread using the CPU-bound executor.""" + return await _dc_to_thread(func, cpu_bound_executor, *args, **kwargs) + + +# For I/O-bound tasks +async def dc_to_thread_io(func, /, *args, **kwargs): + """Asynchronously run function *func* in a separate thread using the I/O-bound executor.""" + return await _dc_to_thread(func, io_bound_executor, *args, **kwargs) diff --git a/src/dataclay/runtime.py b/src/dataclay/runtime.py index d415ede..a35b0db 100644 --- a/src/dataclay/runtime.py +++ b/src/dataclay/runtime.py @@ -394,7 +394,7 @@ async def call_remote_method( try: if method_name == "__getattribute__": logger.debug( - "(%s) Getting remote attribute %s", instance._dc_meta.id, args[0] + "(%s) Getting remote attribute '%s'", instance._dc_meta.id, args[0] ) ( serialized_response, @@ -405,7 +405,7 @@ async def call_remote_method( ) elif method_name == "__setattr__": logger.debug( - "(%s) Setting remote attribute %s", instance._dc_meta.id, args[0] + "(%s) Setting remote attribute '%s'", instance._dc_meta.id, args[0] ) ( serialized_response, @@ -417,7 +417,7 @@ async def call_remote_method( ) elif method_name == "__delattr__": logger.debug( - "(%s) Deleting remote attribute %s", instance._dc_meta.id, args[0] + "(%s) Deleting remote attribute '%s'", instance._dc_meta.id, args[0] ) ( serialized_response, @@ -428,7 +428,7 @@ async def call_remote_method( ) else: logger.debug( - "(%s) Executing remote method %s with constraints %s", + "(%s) Executing remote method '%s' with constraints %s", instance._dc_meta.id, method_name, exec_constraints_var.get(), @@ -468,14 +468,16 @@ async def call_remote_method( # If the response is an exception, it is raised if is_exception: logger.debug( - "(%s) Remote method %s raised an exception", + "(%s) Remote method '%s' raised an exception", instance._dc_meta.id, method_name, ) raise response logger.info( - "(%s) Remote method %s executed successfully", instance._dc_meta.id, method_name + "(%s) Remote method '%s' executed successfully", + instance._dc_meta.id, + method_name, ) return response diff --git a/src/dataclay/utils/serialization.py b/src/dataclay/utils/serialization.py index a778bbf..964262a 100644 --- a/src/dataclay/utils/serialization.py +++ b/src/dataclay/utils/serialization.py @@ -4,13 +4,14 @@ import io import logging import pickle +import threading from typing import TYPE_CHECKING, Optional from uuid import UUID from dataclay import utils from dataclay.config import get_runtime from dataclay.dataclay_object import DataClayObject -from dataclay.event_loop import dc_to_thread, get_dc_event_loop, run_dc_coroutine +from dataclay.event_loop import dc_to_thread_cpu, get_dc_event_loop, run_dc_coroutine if TYPE_CHECKING: from dataclay.dataclay_object import DataClayObject @@ -67,6 +68,8 @@ def persistent_id(self, obj): if obj._dc_meta.id not in self.visited_local_objects: self.visited_local_objects[obj._dc_meta.id] = obj if not obj._dc_is_loaded: + # TODO: Check that assert don't create overhead + assert get_dc_event_loop()._thread_id != threading.get_ident() asyncio.run_coroutine_threadsafe( get_runtime().data_manager.load_object(obj), get_dc_event_loop() ).result() @@ -115,7 +118,7 @@ async def recursive_dcdumps( # NOTE: Executor needed to allow loading objects in parallel (async call inside non-async) file = io.BytesIO() - await dc_to_thread( + await dc_to_thread_cpu( RecursiveDataClayPickler( file, local_objects, remote_objects, serialized_local_objects, make_persistent ).dump, @@ -148,11 +151,9 @@ async def recursive_dcloads(object_binary, unserialized_objects: dict[UUID, Data if unserialized_objects is None: unserialized_objects = {} - # Run in executor to avoid blocking the event loop in `get_by_id_sync` - loop = get_dc_event_loop() - object_dict, state = await loop.run_in_executor( - None, - RecursiveDataClayObjectUnpickler(io.BytesIO(object_binary), unserialized_objects).load, + # Use dc_to_thread_cpu to avoid blocking the event loop in `get_by_id_sync` + object_dict, state = await dc_to_thread_cpu( + RecursiveDataClayObjectUnpickler(io.BytesIO(object_binary), unserialized_objects).load ) object_id = object_dict["_dc_meta"].id @@ -177,27 +178,21 @@ async def dcdumps(obj): obj: The object to serialize. Should never be a DataClayObject, but the _dc_state attribute of it. """ logger.debug("Serializing object in executor") - - # TODO: Avoid calling run_in_executor if not needed. Dunnot how, but optimize! + # TODO: Avoid calling dc_to_thread_cpu if not needed. Dunnot how, but optimize! # If object is None, return None - - # NOTE: Executor needed to avoid blocking the event loop in `make_persistent` - # The context needs to be propagated to access the session_var + # Use dc_to_thread_cpu to avoid blocking the event loop in `get_by_id_sync` file = io.BytesIO() - await dc_to_thread(DataClayPickler(file).dump, obj) + await dc_to_thread_cpu(DataClayPickler(file).dump, obj) return file.getvalue() async def dcloads(binary): """Deserialize the object using pickle.loads. It will manage the deserialization of DataClayObjects. - Necessary to use run_in_executor to avoid blocking the event loop. - Args: binary: The binary to deserialize. Should be the result of dcdumps. """ logger.debug("Deserializing binary in executor") - # NOTE: session_var is not used in deserialization. No need to propagate context - loop = get_dc_event_loop() - result = await loop.run_in_executor(None, pickle.loads, binary) + # Use dc_to_thread_cpu to avoid blocking the event loop in `get_by_id_sync` + result = await dc_to_thread_cpu(pickle.loads, binary) return result From 7756dd5b9a00aaa96337b6b0654bf0cfae3ccfeb Mon Sep 17 00:00:00 2001 From: Alex Barcelo Date: Tue, 8 Oct 2024 15:36:34 +0200 Subject: [PATCH 04/24] Revert concurrency test because it does not suit GitHub actions and misbehaves. This reverts commit 86d51300cf1f1f9a4a2dcf9451e59f47fb7e4958. --- src/dataclay/contrib/modeltest/concurrency.py | 24 ------ tests/functional/test_concurrency.py | 77 ------------------- 2 files changed, 101 deletions(-) delete mode 100644 src/dataclay/contrib/modeltest/concurrency.py delete mode 100644 tests/functional/test_concurrency.py diff --git a/src/dataclay/contrib/modeltest/concurrency.py b/src/dataclay/contrib/modeltest/concurrency.py deleted file mode 100644 index 8dd03b0..0000000 --- a/src/dataclay/contrib/modeltest/concurrency.py +++ /dev/null @@ -1,24 +0,0 @@ -import time - -from dataclay import DataClayObject, activemethod - - -class PingPong(DataClayObject): - _event: bool - pong_obj: 'PingPong' - - def __init__(self): - self._event = False - self.pong_obj = None - - @activemethod - def event_set(self): - self._event = True - - @activemethod - def ping(self, chain=1, wait_event=True): - if chain > 0 and self.pong_obj is not None: - self.pong_obj.ping(chain - 1, wait_event) - if wait_event: - while not self._event: - time.sleep(1) diff --git a/tests/functional/test_concurrency.py b/tests/functional/test_concurrency.py deleted file mode 100644 index 68c379d..0000000 --- a/tests/functional/test_concurrency.py +++ /dev/null @@ -1,77 +0,0 @@ -import asyncio -import pytest - -import concurrent.futures -from threading import Thread - -from dataclay.event_loop import get_dc_event_loop -from dataclay.config import get_runtime - -from dataclay.contrib.modeltest.concurrency import PingPong - -def test_sync_basic_concurrency(client): - """ - Test the synchronous execution of nested activemethods - """ - backends = list(client.get_backends().keys()) - pp1 = PingPong() - pp1.make_persistent(backend_id=backends[0]) - pp2 = PingPong() - pp2.make_persistent(backend_id=backends[1]) - pp1.pong_obj = pp2 - pp2.pong_obj = pp1 - - # Some low-level asyncio stuff here, because we cannot use Threads and - # starting a new client instance seems overkill (but would be best, would it?) - future = asyncio.run_coroutine_threadsafe( - get_runtime().call_remote_method(pp1, "ping", tuple(), dict()), - get_dc_event_loop(), - ) - - with pytest.raises(concurrent.futures.TimeoutError): - future.result(timeout=1) - - pp2.ping(wait_event=False) - - pp1.event_set() - pp2.event_set() - - future.result(timeout=1) - assert future.done() - -def test_sync_high_concurrency(client): - """ - Test the synchronous execution of nested activemethods - """ - backends = list(client.get_backends().keys()) - pp1 = PingPong() - pp1.make_persistent(backend_id=backends[0]) - pp2 = PingPong() - pp2.make_persistent(backend_id=backends[1]) - pp1.pong_obj = pp2 - pp2.pong_obj = pp1 - - future1 = asyncio.run_coroutine_threadsafe( - get_runtime().call_remote_method(pp1, "ping", (10, True), {}), - get_dc_event_loop(), - ) - - future2 = asyncio.run_coroutine_threadsafe( - get_runtime().call_remote_method(pp2, "ping", (10, True), {}), - get_dc_event_loop(), - ) - - with pytest.raises(concurrent.futures.TimeoutError): - future1.result(timeout=1) - - with pytest.raises(concurrent.futures.TimeoutError): - future2.result(timeout=1) - - pp2.ping(wait_event=False) - pp1.event_set() - pp2.event_set() - - future1.result(timeout=1) - future2.result(timeout=1) - assert future1.done() - assert future2.done() From 2db8fff33b749cdbe6ce84926e57633311ba75dd Mon Sep 17 00:00:00 2001 From: Alex Barcelo Date: Tue, 8 Oct 2024 15:54:38 +0200 Subject: [PATCH 05/24] upgrading grpc and adding py313 to the gh actions matrix --- .github/workflows/docker-publish.yml | 2 +- .github/workflows/tests.yml | 2 +- pyproject.toml | 6 +++--- tox.ini | 2 ++ 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/.github/workflows/docker-publish.yml b/.github/workflows/docker-publish.yml index 8158a7b..83aad13 100644 --- a/.github/workflows/docker-publish.yml +++ b/.github/workflows/docker-publish.yml @@ -20,7 +20,7 @@ jobs: runs-on: ubuntu-latest strategy: matrix: - python-version: ['3.9', '3.10', '3.11', '3.12'] + python-version: ['3.9', '3.10', '3.11', '3.12', '3.13'] permissions: contents: read diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index 49ecc37..92dcfe6 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -9,7 +9,7 @@ jobs: runs-on: ubuntu-latest strategy: matrix: - python-version: ["3.9", "3.10", "3.11", "3.12"] + python-version: ["3.9", "3.10", "3.11", "3.12", "3.13"] steps: - uses: actions/checkout@v4 diff --git a/pyproject.toml b/pyproject.toml index d637b65..5ad9514 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -28,8 +28,8 @@ requires-python = ">=3.9" dependencies = [ "aiorwlock>=1.4.0", "bcrypt>=4.0.1", - "grpcio>=1.59.2", - "grpcio-health-checking>=1.59.2", + "grpcio>=1.66.2", + "grpcio-health-checking>=1.66.2", "hiredis>=2.2.3", "opentelemetry-api>=1.20.0", "protobuf>=4.25.0", @@ -103,4 +103,4 @@ omit = [ filterwarnings = [ "error", "ignore::UserWarning", -] \ No newline at end of file +] diff --git a/tox.ini b/tox.ini index eb42a90..722565a 100644 --- a/tox.ini +++ b/tox.ini @@ -6,6 +6,7 @@ envlist = py310 py311 py312 + py313 coverage_report @@ -48,3 +49,4 @@ python = 3.10: py310 3.11: py311 3.12: py312 + 3.13: py313 From e18c1707bca33db0b1e05cd6489490e7641d77ea Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Sat, 19 Oct 2024 15:38:18 +0800 Subject: [PATCH 06/24] Replace tox for nox --- .flake8 | 3 ++ .github/workflows/tests.yml | 35 +++++++++++---------- noxfile.py | 63 +++++++++++++++++++++++++++++++++++++ pyproject.toml | 2 +- 4 files changed, 85 insertions(+), 18 deletions(-) create mode 100644 .flake8 create mode 100644 noxfile.py diff --git a/.flake8 b/.flake8 new file mode 100644 index 0000000..179ca30 --- /dev/null +++ b/.flake8 @@ -0,0 +1,3 @@ +[flake8] +max-line-length = 100 +extend-ignore = E203,E701 \ No newline at end of file diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index 92dcfe6..2113ed7 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -5,24 +5,25 @@ on: - pull_request jobs: - build: + generate-jobs: + runs-on: ubuntu-latest + outputs: + session: ${{ steps.set-matrix.outputs.session }} + steps: + - uses: actions/checkout@v4 + - uses: wntrblm/nox@main + - id: set-matrix + shell: bash + run: echo session=$(nox --json -l | jq -c '[.[].session]') | tee --append $GITHUB_OUTPUT + checks: + name: Session ${{ matrix.session }} + needs: [generate-jobs] runs-on: ubuntu-latest strategy: + fail-fast: false matrix: - python-version: ["3.9", "3.10", "3.11", "3.12", "3.13"] - + session: ${{ fromJson(needs.generate-jobs.outputs.session) }} steps: - - uses: actions/checkout@v4 - - name: Set up Python ${{ matrix.python-version }} - uses: actions/setup-python@v5 - with: - python-version: ${{ matrix.python-version }} - - name: Install dependencies - run: | - python -m pip install --upgrade pip - python -m pip install tox tox-gh-actions - # TODO: when calling pytest, it build a new dev image, but it should build the image - # using the specific python version. Before it was working cause the compose was not - # building the image, rather the image was built here with the {{ matrix.python-version}} - - name: Test with tox - run: tox + - uses: actions/checkout@v4 + - uses: wntrblm/nox@main + - run: nox -s "${{ matrix.session }}" \ No newline at end of file diff --git a/noxfile.py b/noxfile.py new file mode 100644 index 0000000..c67141c --- /dev/null +++ b/noxfile.py @@ -0,0 +1,63 @@ +import nox + +# Define which Python versions to test with +PYTHON_VERSIONS = ["3.9", "3.10", "3.11", "3.12", "3.13"] +LATEST_PYTHON_VERSION = PYTHON_VERSIONS[-1] + +# Default sessions (these will be executed in Github Actions) +# Maintain a clear separation between code checking and code altering tasks (don't add format) +nox.options.sessions = ["lint", "tests"] +# nox.options.reuse_existing_virtualenvs = True # TODO: Check if necessary + + +@nox.session(python=PYTHON_VERSIONS) +def tests(session): + """Run the test suite.""" + session.install("pytest", "pytest-asyncio", "pytest-docker", "pytest-cov") + session.install(".") + session.run("pytest", "--cov", "--cov-report=term-missing") + + +@nox.session(python=LATEST_PYTHON_VERSION) +def lint(session): + """Lint the codebase using flake8.""" + session.install("flake8") + session.run("flake8", "dataclay", "tests") + + +@nox.session(python=LATEST_PYTHON_VERSION) +def format(session): + """Automatically format code with black and isort.""" + session.install("black", "isort") + # Run isort before black + session.run("isort", "--gitignore", ".") + session.run("black", ".") + + +@nox.session(python=LATEST_PYTHON_VERSION) +def mypy(session): + """Run type checks using mypy.""" + session.install(".") + session.install("mypy") + session.run("mypy", "src/dataclay") + + +@nox.session(python=LATEST_PYTHON_VERSION) +def safety(session): + """Check for security vulnerabilities.""" + session.install(".") + session.install("safety") + session.run("safety", "check") + + +@nox.session(default=False) +def dev(session: nox.Session) -> None: + """ + Set up a python development environment for the project at ".venv". + """ + session.install("virtualenv") + session.run("virtualenv", ".venv", silent=True) + session.run(".venv/bin/pip", "install", "-e", ".[dev]", external=True) + + +# TODO: Check https://nox.thea.codes/en/stable/cookbook.html#the-auto-release diff --git a/pyproject.toml b/pyproject.toml index 5ad9514..44155eb 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -49,7 +49,7 @@ dynamic = ["version"] [project.optional-dependencies] dev = [ - "tox", + "nox", "pytest", "pytest-asyncio", "pytest-docker", From 930e37c67959db59e0d5a1351949d0cb9703bc6d Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Sat, 19 Oct 2024 16:28:12 +0800 Subject: [PATCH 07/24] Update .flake8, .gitignore and gh actions --- .flake8 | 3 ++- .github/workflows/tests.yml | 16 ++++++++-------- .gitignore | 2 ++ noxfile.py | 2 +- 4 files changed, 13 insertions(+), 10 deletions(-) diff --git a/.flake8 b/.flake8 index 179ca30..7e253d1 100644 --- a/.flake8 +++ b/.flake8 @@ -1,3 +1,4 @@ [flake8] max-line-length = 100 -extend-ignore = E203,E701 \ No newline at end of file +extend-ignore = E203,E701 +exclude = */proto/*_pb2*.py \ No newline at end of file diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index 2113ed7..7172cbc 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -10,11 +10,11 @@ jobs: outputs: session: ${{ steps.set-matrix.outputs.session }} steps: - - uses: actions/checkout@v4 - - uses: wntrblm/nox@main - - id: set-matrix - shell: bash - run: echo session=$(nox --json -l | jq -c '[.[].session]') | tee --append $GITHUB_OUTPUT + - uses: actions/checkout@v4 + - uses: wntrblm/nox@main + - id: set-matrix + shell: bash + run: echo session=$(nox --json -l -s tests | jq -c '[.[].session]') | tee --append $GITHUB_OUTPUT checks: name: Session ${{ matrix.session }} needs: [generate-jobs] @@ -24,6 +24,6 @@ jobs: matrix: session: ${{ fromJson(needs.generate-jobs.outputs.session) }} steps: - - uses: actions/checkout@v4 - - uses: wntrblm/nox@main - - run: nox -s "${{ matrix.session }}" \ No newline at end of file + - uses: actions/checkout@v4 + - uses: wntrblm/nox@main + - run: nox -s "${{ matrix.session }}" diff --git a/.gitignore b/.gitignore index 3188c57..73990b9 100644 --- a/.gitignore +++ b/.gitignore @@ -15,6 +15,8 @@ tests/mock/client.properties /setup.py.orig /.project .tox +.nox +.mypy_cache *__pycache__* /.idea/ /venv*/ diff --git a/noxfile.py b/noxfile.py index c67141c..601ff4f 100644 --- a/noxfile.py +++ b/noxfile.py @@ -22,7 +22,7 @@ def tests(session): def lint(session): """Lint the codebase using flake8.""" session.install("flake8") - session.run("flake8", "dataclay", "tests") + session.run("flake8", "src/dataclay", "tests") @nox.session(python=LATEST_PYTHON_VERSION) From 1d4812fe3ceb195d1ff3c16d7fe60d35978a2b2f Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Sat, 19 Oct 2024 17:42:15 +0800 Subject: [PATCH 08/24] Remove old methods, format code and correct lint errors --- src/dataclay/alien.py | 6 +- src/dataclay/backend/api.py | 225 ----------------------- src/dataclay/backend/client.py | 190 ------------------- src/dataclay/backend/servicer.py | 187 ------------------- src/dataclay/client/api.py | 54 +----- src/dataclay/contrib/modeltest/remote.py | 16 +- src/dataclay/contrib/nvm.py | 1 + src/dataclay/contrib/zenoh_module.py | 1 + src/dataclay/data_manager.py | 15 +- src/dataclay/dataclay_object.py | 87 ++++----- src/dataclay/exceptions.py | 62 +------ src/dataclay/metadata/client.py | 2 +- src/dataclay/metadata/kvdata.py | 2 +- src/dataclay/metadata/redismanager.py | 3 +- src/dataclay/metadata/servicer.py | 4 +- src/dataclay/paraver/__init__.py | 1 + src/dataclay/runtime.py | 16 +- src/dataclay/utils/__init__.py | 2 + src/dataclay/utils/backend_clients.py | 3 +- src/dataclay/utils/metrics.py | 1 + src/dataclay/utils/serialization.py | 41 +++-- tests/functional/conftest.py | 5 +- tests/functional/test_activemethod.py | 14 +- tests/functional/test_alias.py | 2 +- tests/functional/test_alien.py | 6 +- tests/functional/test_client.py | 3 - tests/functional/test_getstate.py | 2 - tests/functional/test_heapmanager.py | 2 +- tests/functional/test_make_persistent.py | 48 ++--- tests/functional/test_move_object.py | 3 +- tests/functional/test_new_replica.py | 2 +- tests/functional/test_versioning.py | 2 +- 32 files changed, 142 insertions(+), 866 deletions(-) diff --git a/src/dataclay/alien.py b/src/dataclay/alien.py index 50a9823..f5d85eb 100644 --- a/src/dataclay/alien.py +++ b/src/dataclay/alien.py @@ -127,9 +127,9 @@ def __new__(cls, obj: T, *args, **kwargs): proxy_class = cls._create_class_proxy(obj.__class__) instance = DataClayObject.__new__(proxy_class) - object.__getattribute__( - instance, "_dc_meta" - ).class_name = f"AlienDataClayObject[{type(obj).__module__}.{type(obj).__name__}]" + object.__getattribute__(instance, "_dc_meta").class_name = ( + f"AlienDataClayObject[{type(obj).__module__}.{type(obj).__name__}]" + ) object.__setattr__(instance, "_dc_base_object", obj) return instance diff --git a/src/dataclay/backend/api.py b/src/dataclay/backend/api.py index fa80174..85ac925 100644 --- a/src/dataclay/backend/api.py +++ b/src/dataclay/backend/api.py @@ -3,11 +3,9 @@ from __future__ import annotations import asyncio -import inspect import logging import pickle import time -import traceback from collections.abc import Iterable from typing import TYPE_CHECKING, Any, Optional @@ -405,226 +403,3 @@ async def new_object_replica( ): instance = await self.runtime.get_object_by_id(object_id) await self.runtime.new_object_replica(instance, backend_id, recursive, remotes) - - def synchronize( - self, session_id, object_id, implementation_id, serialized_value, calling_backend_id=None - ): - raise Exception("To refactor") - # set field - logger.debug( - "----> Starting synchronization of %s from calling backend %s", - object_id, - calling_backend_id, - ) - - self.ds_exec_impl(object_id, implementation_id, serialized_value, session_id) - instance = self.get_local_instance(object_id, True) - src_exec_env_id = instance._dc_meta.master_backend_id() - if src_exec_env_id is not None: - logger.debug("Found origin location %s", src_exec_env_id) - if calling_backend_id is None or src_exec_env_id != calling_backend_id: - # do not synchronize to calling source (avoid infinite loops) - dest_backend = self.runtime.get_backend_client(src_exec_env_id) - logger.debug( - "----> Propagating synchronization of %s to origin location %s", - object_id, - src_exec_env_id, - ) - - dest_backend.synchronize( - session_id, - object_id, - implementation_id, - serialized_value, - calling_backend_id=self.backend_id, - ) - - replica_locations = instance._dc_meta.replica_backend_ids - if replica_locations is not None: - logger.debug("Found replica locations %s", replica_locations) - for replica_location in replica_locations: - if calling_backend_id is None or replica_location != calling_backend_id: - # do not synchronize to calling source (avoid infinite loops) - dest_backend = self.runtime.get_backend_client(replica_location) - logger.debug( - "----> Propagating synchronization of %s to replica location %s", - object_id, - replica_location, - ) - dest_backend.synchronize( - session_id, - object_id, - implementation_id, - serialized_value, - calling_backend_id=self.backend_id, - ) - logger.debug("----> Finished synchronization of %s", object_id) - - # Federation - - def federate(self, session_id, object_id, external_execution_env_id, recursive): - """Federate object with id provided to external execution env id specified - - Args: - session_id: id of the session federating objects - object_id: id of object to federate - external_execution_id: id of dest external execution environment - recursive: indicates if federation is recursive - """ - raise Exception("To refactor") - logger.debug("----> Starting federation of %s", object_id) - - object_ids = set() - object_ids.add(object_id) - # TODO: check that current dataClay/EE has permission to federate the object (refederation use-case) - serialized_objs = self.get_objects( - session_id, object_ids, set(), recursive, external_execution_env_id, 1 - ) - client_backend = self.runtime.get_backend_client(external_execution_env_id) - client_backend.notify_federation(session_id, serialized_objs) - # TODO: add federation reference to object send ?? how is it working with replicas? - logger.debug("<---- Finished federation of %s", object_id) - - def notify_federation(self, session_id, objects_to_persist): - """This function will deserialize object "parameters" (i.e. object to persist - and subobjects if needed) into dataClay memory heap using the same design as for - volatile parameters. This function processes objects recieved from federation calls. - - Args: - session_id: ID of session of federation call - objects_to_persist: [num_params, imm_objs, lang_objs, vol_params, pers_params] - """ - - raise Exception("To refactor") - self.set_local_session(session_id) - - try: - logger.debug("----> Notified federation") - - # No need to provide params specs or param order since objects are not language types - federated_objs = self.store_in_memory(objects_to_persist) - - # Register objects with alias (should we?) - for object in federated_objs: - if object._dc_alias: - self.runtime.metadata_service.upsert_object(object._dc_meta) - - for federated_obj in federated_objs: - try: - federated_obj.when_federated() - except Exception: - # ignore if method is not implemented - pass - - except Exception as e: - traceback.print_exc() - raise e - logger.debug("<---- Finished notification of federation") - - def unfederate(self, session_id, object_id, external_execution_env_id, recursive): - """Unfederate object in external execution environment specified - - Args: - session_id: id of session - object_id: id of the object - external_execution_env_id: external ee - recursive: also unfederates sub-objects - """ - # TODO: redirect unfederation to owner if current dataClay is not the owner, check origLoc belongs to current dataClay - raise Exception("To refactor") - try: - logger.debug("----> Starting unfederation of %s", object_id) - object_ids = set() - object_ids.add(object_id) - serialized_objs = self.get_objects( - session_id, object_ids, set(), recursive, external_execution_env_id, 2 - ) - - unfederate_per_backend = {} - - for serialized_obj in serialized_objs: - replica_locs = serialized_obj.metadata.replica_locations - for replica_loc in replica_locs: - exec_env = self.runtime.get_execution_environment_info(replica_loc) - if exec_env.dataclay_instance_id != self.runtime.dataclay_id: - if ( - external_execution_env_id is not None - and replica_loc != external_execution_env_id - ): - continue - objs_in_backend = None - if replica_loc not in unfederate_per_backend: - objs_in_backend = set() - unfederate_per_backend[replica_loc] = objs_in_backend - else: - objs_in_backend = unfederate_per_backend[replica_loc] - objs_in_backend.add(serialized_obj.object_id) - - for external_ee_id, objs_in_backend in unfederate_per_backend.items(): - client_backend = self.runtime.get_backend_client(external_ee_id) - client_backend.notify_unfederation(session_id, objs_in_backend) - - logger.debug("<---- Finished unfederation of %s", object_ids) - - except Exception as e: - traceback.print_exc() - raise e - - def notify_unfederation(self, session_id, object_ids): - """This function is called when objects are unfederated. - - Args: - session_id: ID of session of federation call - object_ids: List of IDs of the objects to unfederate - """ - raise Exception("To refactor") - self.set_local_session(session_id) - logger.debug("---> Notified unfederation: running when_unfederated") - try: - for object_id in object_ids: - instance = self.get_local_instance(object_id, True) - - try: - instance.when_unfederated() - except: - # ignore if method is not implemented - pass - instance.set_origin_location(None) - try: - if instance._dc_alias is not None and instance._dc_alias != "": - logger.debug("Removing alias %s", instance._dc_alias) - self.self.runtime.delete_alias(instance) - - except Exception as ex: - traceback.print_exc() - logger.debug( - "Caught exception %s, Ignoring if object was not registered yet", - type(ex).__name__, - ) - # ignore if object was not registered yet - pass - except DataClayException as e: - # TODO: better algorithm to avoid unfederation in wrong backend - logger.debug( - "Caught exception %s, Ignoring if object is not in current backend", - type(e).__name__, - ) - except Exception as e: - logger.debug("Caught exception %s", type(e).__name__) - raise e - logger.debug("<--- Finished notification of unfederation") - - # Tracing - - def activate_tracing(self, task_id): - if not extrae_tracing_is_enabled(): - set_current_available_task_id(task_id) - initialize_extrae(True) - - def deactivate_tracing(self): - if extrae_tracing_is_enabled(): - finish_tracing(True) - - def get_traces(self): - logger.debug("Merging...") - return get_traces() diff --git a/src/dataclay/backend/client.py b/src/dataclay/backend/client.py index 906d9c7..399bea5 100644 --- a/src/dataclay/backend/client.py +++ b/src/dataclay/backend/client.py @@ -1,5 +1,4 @@ import logging -import traceback from typing import Any, Iterable, Optional from uuid import UUID @@ -10,9 +9,7 @@ import dataclay from dataclay.config import session_var, settings -from dataclay.exceptions import DataClayException from dataclay.proto.backend import backend_pb2, backend_pb2_grpc -from dataclay.proto.common import common_pb2 from dataclay.utils.decorators import grpc_aio_error_handler logger = logging.getLogger(__name__) @@ -309,190 +306,3 @@ async def stop(self): @grpc_aio_error_handler async def drain(self): await self.stub.Drain(empty_pb2.Empty()) - - # @grpc_aio_error_handler - # def new_object_replica(self, object_id: UUID, backend_id: UUID, recursive: bool, remotes: bool): - # request = backend_pb2.NewObjectReplicaRequest( - # object_id=str(object_id), - # backend_id=str(backend_id), - # recursive=recursive, - # remotes=remotes, - # ) - # self.stub.NewObjectReplica(request, metadata=self.metadata_call) - - ########### - # END NEW # - ########### - - def federate(self, session_id, object_id, external_execution_env_id, recursive): - raise Exception("To refactor") - try: - request = dataservice_messages_pb2.FederateRequest( - sessionID=str(session_id), - objectID=str(object_id), - externalExecutionEnvironmentID=str(external_execution_env_id), - recursive=recursive, - ) - response = self.stub.federate(request, metadata=self.metadata_call) - except RuntimeError as e: - traceback.print_exc() - logger.error("Failed to federate", exc_info=True) - raise e - if response.isException: - raise DataClayException(response.exceptionMessage) - - def unfederate(self, session_id, object_id, external_execution_env_id, recursive): - raise Exception("To refactor") - request = dataservice_messages_pb2.UnfederateRequest( - sessionID=str(session_id), - objectID=str(object_id), - externalExecutionEnvironmentID=str(external_execution_env_id), - recursive=recursive, - ) - try: - response = self.stub.unfederate(request, metadata=self.metadata_call) - except RuntimeError as e: - logger.error("Failed to unfederate", exc_info=True) - raise e - if response.isException: - raise DataClayException(response.exceptionMessage) - - def notify_federation(self, session_id, params): - raise Exception("To refactor") - obj_list = [] - for entry in params: - obj_list.append(Utils.get_obj_with_data_param_or_return(entry)) - - request = dataservice_messages_pb2.NotifyFederationRequest( - sessionID=str(session_id), - objects=obj_list, - ) - - try: - response = self.stub.notifyFederation(request, metadata=self.metadata_call) - - except RuntimeError as e: - logger.error("Failed to federate", exc_info=True) - raise e - - if response.isException: - raise DataClayException(response.exceptionMessage) - - def notify_unfederation(self, session_id, object_ids): - raise Exception("To refactor") - obj_list = [] - for entry in object_ids: - obj_list.append(str(entry)) - - request = dataservice_messages_pb2.NotifyUnfederationRequest( - sessionID=str(session_id), - objectIDs=obj_list, - ) - - try: - response = self.stub.notifyUnfederation(request, metadata=self.metadata_call) - - except RuntimeError as e: - logger.error("Failed to federate", exc_info=True) - raise e - - if response.isException: - raise DataClayException(response.exceptionMessage) - - def synchronize(self, session_id, object_id, implementation_id, params, calling_backend_id): - raise Exception("To refactor") - request = dataservice_messages_pb2.SynchronizeRequest( - sessionID=str(session_id), - objectID=str(object_id), - implementationID=str(implementation_id), - params=Utils.get_param_or_return(params), - callingBackendID=str(calling_backend_id), - ) - try: - response = self.stub.synchronize(request, metadata=self.metadata_call) - except RuntimeError as e: - raise e - if response.isException: - raise DataClayException(response.exceptionMessage) - - def ds_migrate_objects_to_backends(self, back_ends): - raise ("To refactor") - - back_ends_dict = {} - - for k, v in back_ends.items(): - back_ends_dict[k] = Utils.get_storage_location(v) - - request = dataservice_messages_pb2.MigrateObjectsRequest(destStorageLocs=back_ends_dict) - - try: - response = self.stub.migrateObjectsToBackends(request, metadata=self.metadata_call) - - except RuntimeError as e: - raise e - - if response.excInfo.isException: - raise DataClayException(response.excInfo.exceptionMessage) - - result = {} - - for k, v in response.migratedObjs.items(): - m_objs = v - oids = set() - - for oid in m_objs.getObjsList(): - oids.add(UUID(oid)) - - result[UUID(k)] = oids - - non_migrated = set() - - for oid in response.nonMigratedObjs.getObjsList(): - non_migrated.add(UUID(oid)) - - t = (result, non_migrated) - - return t - - def activate_tracing(self, task_id): - raise Exception("To refactor") - request = dataservice_messages_pb2.ActivateTracingRequest(taskid=task_id) - - try: - response = self.stub.activateTracing(request, metadata=self.metadata_call) - - except RuntimeError as e: - raise e - - if response.isException: - raise DataClayException(response.exceptionMessage) - - def deactivate_tracing(self): - raise Exception("To refactor") - try: - response = self.stub.deactivateTracing( - common_pb2.EmptyMessage(), metadata=self.metadata_call - ) - - except RuntimeError as e: - raise e - - if response.isException: - raise DataClayException(response.exceptionMessage) - - def get_traces(self): - raise Exception("To refactor") - try: - response = self.stub.getTraces(common_pb2.EmptyMessage(), metadata=self.metadata_call) - except RuntimeError as e: - raise e - - result = {} - for k, v in response.stubs.items(): - result[k] = v - - return result - - ################## EXTRAE IGNORED FUNCTIONS ########################### - # deactivate_tracing.do_not_trace = True - # activate_tracing.do_not_trace = True diff --git a/src/dataclay/backend/servicer.py b/src/dataclay/backend/servicer.py index bedef71..66fecb9 100644 --- a/src/dataclay/backend/servicer.py +++ b/src/dataclay/backend/servicer.py @@ -4,7 +4,6 @@ import logging import os.path import signal -import traceback from concurrent import futures from functools import wraps from typing import Optional @@ -25,7 +24,6 @@ from dataclay.config import session_var, settings from dataclay.event_loop import get_dc_event_loop, set_dc_event_loop from dataclay.proto.backend import backend_pb2, backend_pb2_grpc -from dataclay.proto.common import common_pb2 logger = logging.getLogger(__name__) @@ -343,188 +341,3 @@ async def NewObjectReplica(self, request, context): request.remotes, ) return Empty() - - ########### - # END NEW # - ########### - - def synchronize(self, request, context): - raise Exception("To refactor") - try: - object_id = UUID(request.objectID) - implementation_id = UUID(request.implementationID) - serialized_params = Utils.get_param_or_return(request.params) - session_id = UUID(request.sessionID) - calling_backend_id = UUID(request.callingBackendID) - self.backend.synchronize( - session_id, object_id, implementation_id, serialized_params, calling_backend_id - ) - return common_pb2.ExceptionInfo() - except DataClayException as ex: - return self.get_exception_info(ex) - - def removeObjects(self, request, context): - raise Exception("To refactor") - try: - object_ids = set() - - for oid in request.getObjectsIDSList(): - object_ids.add(UUID(oid)) - - result = self.client.ds_remove_objects( - UUID(request.sessionID), - object_ids, - request.recursive, - request.moving, - UUID(request.newHint), - ) - - rem_obj = {} - - for k, v in result.items(): - rem_obj[str(k)] = str(v) - - return dataservice_messages_pb2.RemoveObjectsResponse(removedObjects=rem_obj) - - except Exception as ex: - return dataservice_messages_pb2.RemoveObjectsResponse( - excInfo=self.get_exception_info(ex) - ) - - def migrateObjectsToBackends(self, request, context): - raise Exception("To refactor") - try: - backends = {} - - for k, v in request.destStorageLocs.items(): - backends[UUID(k)] = Utils.get_storage_location(v) - - result = self.client.ds_migrate_objects_to_backends(backends) - - migr_obj_res = {} - - for k, v in result[0].items(): - migrated_obj_list = list() - - for oid in v: - migrated_obj_list.append(Utils.get_msg_id(oid)) - - migrated_obj_builder = dataservice_messages_pb2.MigratedObjects( - objs=migrated_obj_list - ) - migr_obj_res[str(k)] = migrated_obj_builder - - non_migrated_objs_list = list() - - for oid in result[1]: - non_migrated_objs_list.append(Utils.get_msg_id(oid)) - - non_migrated_objs_builder = dataservice_messages_pb2.MigratedObjects( - objs=non_migrated_objs_list - ) - - return dataservice_messages_pb2.MigrateObjectsResponse( - migratedObjs=migr_obj_res, nonMigratedObjs=non_migrated_objs_builder - ) - - except Exception as ex: - return dataservice_messages_pb2.MigrateObjectsResponse( - excInfo=self.get_exception_info(ex) - ) - - ############## - # Federation # - ############## - - def federate(self, request, context): - raise Exception("To refactor") - try: - logger.debug("Federation started") - self.backend.federate( - UUID(request.sessionID), - UUID(request.objectID), - UUID(request.externalExecutionEnvironmentID), - request.recursive, - ) - logger.debug("Federation finished, sending response") - return common_pb2.ExceptionInfo() - except Exception as ex: - traceback.print_exc() - return self.get_exception_info(ex) - - def unfederate(self, request, context): - raise Exception("To refactor") - try: - logger.debug("Unfederation started") - self.backend.unfederate( - UUID(request.sessionID), - UUID(request.objectID), - UUID(request.externalExecutionEnvironmentID), - request.recursive, - ) - logger.debug("Unfederation finished, sending response") - return common_pb2.ExceptionInfo() - except Exception as ex: - return self.get_exception_info(ex) - - def notifyFederation(self, request, context): - raise Exception("To refactor") - try: - logger.debug("Notify Federation started") - objects_to_persist = [] - for vol_param in request.objects: - param = Utils.get_obj_with_data_param_or_return(vol_param) - objects_to_persist.append(param) - session_id = UUID(request.sessionID) - self.backend.notify_federation(session_id, objects_to_persist) - logger.debug("Notify Federation finished, sending response") - return common_pb2.ExceptionInfo() - - except Exception as ex: - return self.get_exception_info(ex) - - def notifyUnfederation(self, request, context): - raise Exception("To refactor") - try: - logger.debug("Notify Unfederation started") - session_id = UUID(request.sessionID) - object_ids = set() - for oid in request.objectIDs: - object_ids.add(UUID(oid)) - self.backend.notify_unfederation(session_id, object_ids) - logger.debug("Notify Unfederation finished, sending response") - return common_pb2.ExceptionInfo() - - except Exception as ex: - traceback.print_exc() - return self.get_exception_info(ex) - - ########### - # Tracing # - ########### - - def activateTracing(self, request, context): - raise Exception("To refactor") - try: - self.backend.activate_tracing(request.taskid) - return common_pb2.ExceptionInfo() - - except Exception as ex: - return self.get_exception_info(ex) - - def getTraces(self, request, context): - raise Exception("To refactor") - try: - result = self.backend.get_traces() - return common_pb2.GetTracesResponse(traces=result) - except Exception as ex: - return self.get_exception_info(ex) - - def deactivateTracing(self, request, context): - raise Exception("To refactor") - try: - self.backend.deactivate_tracing() - return common_pb2.ExceptionInfo() - - except Exception as ex: - return self.get_exception_info(ex) diff --git a/src/dataclay/client/api.py b/src/dataclay/client/api.py index 232ab08..d15b9ec 100644 --- a/src/dataclay/client/api.py +++ b/src/dataclay/client/api.py @@ -15,7 +15,6 @@ import logging import logging.config from typing import TYPE_CHECKING, Optional -from uuid import UUID from dataclay.config import ( ClientSettings, @@ -165,7 +164,9 @@ def __init__( # LOCAL = ee_id # break # else: - # logger.warning("Backend with name '%s' not found, ignoring", settings.LOCAL_BACKEND) + # logger.warning( + # "Backend with name '%s' not found, ignoring", settings.LOCAL_BACKEND + # ) @tracer.start_as_current_span("start") def start(self): @@ -178,7 +179,7 @@ def start(self): logger.info("Starting client runtime") loop = get_dc_event_loop() - if loop == None: + if loop is None: logger.info("Creating event loop in new thread") loop = asyncio.new_event_loop() set_dc_event_loop(loop) @@ -291,50 +292,3 @@ async def a_get_backends(self) -> dict[UUID, BackendClient]: ) await asyncio.wrap_future(future) return self.runtime.backend_clients - - -############### -# To Refactor # -############### - - -# def register_dataclay(id, host, port): -# """Register external dataClay for federation -# Args: -# host: external dataClay host name -# port: external dataClay port -# """ -# return get_runtime().register_external_dataclay(id, host, port) - - -# def unfederate(ext_dataclay_id=None): -# """Unfederate all objects belonging to/federated with external data clay with id provided -# or with all any external dataclay if no argument provided. -# :param ext_dataclay_id: external dataClay id -# :return: None -# :type ext_dataclay_id: uuid -# :rtype: None -# """ -# if ext_dataclay_id is not None: -# return get_runtime().unfederate_all_objects(ext_dataclay_id) -# else: -# return get_runtime().unfederate_all_objects_with_all_dcs() - - -# def migrate_federated_objects(origin_dataclay_id, dest_dataclay_id): -# """Migrate federated objects from origin dataclay to destination dataclay -# :param origin_dataclay_id: origin dataclay id -# :param dest_dataclay_id destination dataclay id -# :return: None -# :rtype: None -# """ -# return get_runtime().migrate_federated_objects(origin_dataclay_id, dest_dataclay_id) - - -# def federate_all_objects(dest_dataclay_id): -# """Federate all objects from current dataclay to destination dataclay -# :param dest_dataclay_id destination dataclay id -# :return: None -# :rtype: None -# """ -# return get_runtime().federate_all_objects(dest_dataclay_id) diff --git a/src/dataclay/contrib/modeltest/remote.py b/src/dataclay/contrib/modeltest/remote.py index 8d9e0d1..d15fe93 100644 --- a/src/dataclay/contrib/modeltest/remote.py +++ b/src/dataclay/contrib/modeltest/remote.py @@ -7,7 +7,7 @@ from dataclay import DataClayObject, activemethod from dataclay.config import get_runtime -from dataclay.contrib.modeltest.family import Dog, Family, Person +from dataclay.contrib.modeltest.family import Person class MakePersistentTestClass(DataClayObject): @@ -15,7 +15,7 @@ class MakePersistentTestClass(DataClayObject): def test_remote_automatic_register(self): """DataClay objects are persistent automatically in the backend""" person = Person("Marc", 24) - assert person.is_persistent == True + assert person.is_persistent is True assert person.name == "Marc" assert person.age == 24 assert person._dc_meta.master_backend_id == get_runtime().backend_id @@ -27,9 +27,9 @@ def test_remote_automatic_register(self): def test_remote_make_persistent(self): """A call to make_persistent won't have any effect""" person = Person("Marc", 24) - assert person.is_persistent == True + assert person.is_persistent is True person.make_persistent() - assert person.is_persistent == True + assert person.is_persistent is True assert person.name == "Marc" assert person.age == 24 assert person._dc_meta.master_backend_id == get_runtime().backend_id @@ -38,9 +38,9 @@ def test_remote_make_persistent(self): def test_remote_make_persistent_alias(self): """A call to make_persistent with an alias will add the alias to the object""" person = Person("Marc", 24) - assert person.is_persistent == True + assert person.is_persistent is True person.make_persistent() - assert person.is_persistent == True + assert person.is_persistent is True person.make_persistent(alias="test_remote_make_persistent_alias") # person.sync() # Sync to update object metadata @@ -50,9 +50,9 @@ def test_remote_make_persistent_alias(self): def test_remote_make_persistent_backend(self): """A call to make_persistent with a backend_id will move the object to the specified backend""" person = Person("Marc", 24) - assert person.is_persistent == True + assert person.is_persistent is True person.make_persistent() - assert person.is_persistent == True + assert person.is_persistent is True get_runtime().backend_clients.update() backend_ids = list(get_runtime().backend_clients) diff --git a/src/dataclay/contrib/nvm.py b/src/dataclay/contrib/nvm.py index 8e0267b..a63ed33 100644 --- a/src/dataclay/contrib/nvm.py +++ b/src/dataclay/contrib/nvm.py @@ -6,6 +6,7 @@ ships the :class:`InNVM` annotation; this offers a transparent and automatic in-NVM placement for class attributes. """ + from dataclay.annotated import PropertyTransformer try: diff --git a/src/dataclay/contrib/zenoh_module.py b/src/dataclay/contrib/zenoh_module.py index 461d914..a066a2a 100644 --- a/src/dataclay/contrib/zenoh_module.py +++ b/src/dataclay/contrib/zenoh_module.py @@ -10,6 +10,7 @@ The publisher can send messages to a specific topic. """ + try: import zenoh except ImportError: diff --git a/src/dataclay/data_manager.py b/src/dataclay/data_manager.py index 9c3533e..13752aa 100644 --- a/src/dataclay/data_manager.py +++ b/src/dataclay/data_manager.py @@ -38,10 +38,11 @@ class DataManager: def __init__(self): # Loaded objects so they cannot be GC by PythonGC. - # It is very important to be a sorted dict (guaranteed in py3.7), so first elements to arrive are cleaned before, - # n any deserialization from DB or parameter, objects deserialized first are referrers to - # objects deserialized later. Second ones cannot be GC if first ones are not cleaned. - # During GC,we should know that somehow. It's a hint but improves GC a lot. + # It is very important to be a sorted dict (guaranteed in py3.7), so first elements + # to arrive are cleaned before, n any deserialization from DB or parameter, objects + # deserialized first are referrers to objects deserialized later. Second ones cannot + # be GC if first ones are not cleaned. During GC,we should know that somehow. It's a + # hint but improves GC a lot. self.loaded_objects: dict[UUID, DataClayObject] = {} self.memory_lock = asyncio.Lock() self.memory_task = None @@ -149,7 +150,8 @@ async def load_object(self, instance: DataClayObject): del state["_dc_meta"] vars(instance).update(state) - # NOTE: We need to set _dc_is_loaded before calling __setstate__ to avoid infinite recursion + # NOTE: We need to set _dc_is_loaded before calling __setstate__ + # to avoid infinite recursion instance._dc_is_loaded = True if getstate is not None: instance.__setstate__(getstate) @@ -227,7 +229,8 @@ async def flush_all(self, unload_timeout: Optional[str] = None, force_unload: bo Args: unload_timeout (Optional[str]): The timeout to acquire the lock. - force_unload (bool): If True, the objects will be unloaded even if the lock cannot be acquired. + force_unload (bool): If True, the objects will be unloaded + even if the lock cannot be acquired. """ if unload_timeout is None: diff --git a/src/dataclay/dataclay_object.py b/src/dataclay/dataclay_object.py index 5a06d2c..6f9c4e0 100644 --- a/src/dataclay/dataclay_object.py +++ b/src/dataclay/dataclay_object.py @@ -19,7 +19,7 @@ from dataclay.annotated import LocalOnly, PropertyTransformer from dataclay.config import get_runtime -from dataclay.event_loop import get_dc_event_loop, run_dc_coroutine +from dataclay.event_loop import get_dc_event_loop from dataclay.exceptions import ObjectIsMasterError, ObjectNotRegisteredError from dataclay.metadata.kvdata import ObjectMetadata from dataclay.utils.telemetry import trace @@ -268,25 +268,26 @@ def __new__(cls, *args, **kwargs): obj._dc_meta = ObjectMetadata(class_name=cls.__module__ + "." + cls.__name__) logger.debug( - "(%s) Creating new dc_object '%s' args=%s, kwargs=%s", + "(%s) Creating new DataClayObject '%s' with args=%s, kwargs=%s", obj._dc_meta.id, cls.__name__, args, kwargs, ) - # If the object is being created in a backend, it should be made persistent immediately - # This should only happen when instantiating a dataClay object from an activemethod, - # The activemethod must had been called in another thread using an executor. Therefore, - # there is not running event loop in the current thread, and we can use run_coroutine_threadsafe - # to the main dc_running_loop. - # TODO: This same logic applies to all DataClayObject methods that are called withing an activemethod. + # If the object is created on a backend, it should be made persistent immediately. + # This happens when a DataClay object is instantiated from an activemethod. + # Since activemethods are executed in another thread (using an executor), + # there is no active event loop in the current thread. Therefore, we can safely use + # run_coroutine_threadsafe to interact with the main event loop (dc_running_loop). + # TODO: Apply this logic to all DataClayObject methods invoked within activemethods. if get_runtime() and get_runtime().is_backend: logger.debug("(%s) Calling implicit make_persistent", obj._dc_meta.id) - # TODO: Option to make an eventual call to make_persistent async - # loop.create_task(obj.a_make_persistent()) + # TODO: Consider making make_persistent an asynchronous call + # Example: loop.create_task(obj.a_make_persistent()) obj.make_persistent() + # Alternatively, use the event loop for async behavior: # loop = get_dc_event_loop() # t = asyncio.run_coroutine_threadsafe(obj.make_persistent(), loop) # t.result() @@ -460,8 +461,8 @@ def get_by_id(cls, object_id: UUID) -> DataClayObject: # WARNING: This method must not be called from the same thread as the running event loop # or it will block the event loop. When unserializing dataClay objects, use "await dcloads" # if possible. Only use "pickle.loads" if you are sure that the event loop is not running. - # "pickle.loads" of dataClay objects is calling this method behind. With `dcloads` this method - # will be called in another thread, so it will not block the event loop. + # "pickle.loads" of dataClay objects is calling this method behind. With `dcloads` this + # method will be called in another thread, so it will not block the event loop. logger.debug("(%s) Calling get_by_id", object_id) assert get_dc_event_loop()._thread_id != threading.get_ident() @@ -483,18 +484,20 @@ async def a_get_by_alias(cls: Type[T], alias: str, dataset_name: str = None) -> @classmethod def get_by_alias(cls: Type[T], alias: str, dataset_name: str = None) -> T: - """Returns the object with the given alias. + """ + Retrieve an object by its alias. Args: - alias: Alias of the object. - dataset_name: Name of the dataset where the alias is stored. If None, the active dataset is used. + alias: The alias of the object to retrieve. + dataset_name: Optional. The name of the dataset where the alias is stored. + If not provided, the active dataset is used. Returns: - The object with the given alias. + The object associated with the given alias. Raises: - DoesNotExistError: If the alias does not exist. - DatasetIsNotAccessibleError: If the dataset is not accessible. + DoesNotExistError: If no object with the given alias exists. + DatasetIsNotAccessibleError: If the specified dataset is not accessible. """ future = asyncio.run_coroutine_threadsafe( cls._get_by_alias(alias, dataset_name), get_dc_event_loop() @@ -561,7 +564,8 @@ def delete_alias(cls, alias: str, dataset_name: str = None): Args: alias: Alias to be removed. - dataset_name: Name of the dataset where the alias is stored. If None, the active dataset is used. + dataset_name: Name of the dataset where the alias is stored. + If None, the active dataset is used. Raises: DoesNotExistError: If the alias does not exist. @@ -751,22 +755,23 @@ async def dc_update_by_alias(cls, alias: str, from_object: DataClayObject): @tracer.start_as_current_span("dc_update") async def dc_update(self, from_object: DataClayObject): - """Updates current object with contents of from_object. + """Updates the current object with the properties of from_object. Args: - from_object: object with the new values to update current object. + from_object: The object with the new values to update current object. Raises: TypeError: If the objects are not of the same type. """ - if type(self) != type(from_object): + if not isinstance(from_object, type(self)): raise TypeError("Objects must be of the same type") await get_runtime().replace_object_properties(self, from_object) @tracer.start_as_current_span("dc_update_properties") async def _dc_update_properties(self, new_properties: dict[str, Any]): - # TODO: Check that the new properties are the same and of the same type as the current object + # TODO: Check that the new properties are the same and + # of the same type as the current object await get_runtime().update_object_properties(self, new_properties) async def a_dc_update_properties(self, new_properties: dict[str, Any]): @@ -810,39 +815,13 @@ async def dc_put(self, alias: str, backend_id: UUID = None): raise AttributeError("Alias cannot be null or empty") await self.a_make_persistent(alias=alias, backend_id=backend_id) - ############## - # Federation # - ############## - - def federate_to_backend(self, ext_execution_env_id, recursive=True): - get_runtime().federate_to_backend(self, ext_execution_env_id, recursive) - - def federate(self, ext_dataclay_id, recursive=True): - get_runtime().federate_object(self, ext_dataclay_id, recursive) - - def unfederate_from_backend(self, ext_execution_env_id, recursive=True): - get_runtime().unfederate_from_backend(self, ext_execution_env_id, recursive) - - def unfederate(self, ext_dataclay_id=None, recursive=True): - # FIXME: unfederate only from specific ext dataClay - get_runtime().unfederate_object(self, ext_dataclay_id, recursive) - - def synchronize(self, field_name, value): - # from dataclay.DataClayObjProperties import DCLAY_SETTER_PREFIX - raise Exception("Synchronize need refactor") - return get_runtime().synchronize(self, DCLAY_SETTER_PREFIX + field_name, value) + ################# + # Magic Methods # + ################# def __repr__(self): - if self._dc_is_registered: - return "<%s instance with ObjectID=%s>" % ( - self._dc_meta.class_name, - self._dc_meta.id, - ) - else: - return "<%s volatile instance with ObjectID=%s>" % ( - self._dc_meta.class_name, - self._dc_meta.id, - ) + status = "instance" if self._dc_is_registered else "volatile instance" + return f"<{self._dc_meta.class_name} {status} with ObjectID={self._dc_meta.id}>" def __eq__(self, other): if not isinstance(other, DataClayObject): diff --git a/src/dataclay/exceptions.py b/src/dataclay/exceptions.py index a5b624a..dfc30ad 100644 --- a/src/dataclay/exceptions.py +++ b/src/dataclay/exceptions.py @@ -208,67 +208,9 @@ class DataclayIdError(DataClayException): class DataclayIdDoesNotExistError(DataclayIdError): def __str__(self): - return f"Dataclay ID does not exist!" + return "Dataclay ID does not exist!" class DataclayIdAlreadyExistError(DataclayIdError): def __str__(self): - return f"Dataclay ID already exist!" - - -# TODO: Check if old excepions are used -################## -# OLD EXCEPTIONS # -################## - - -class ImproperlyConfigured(DataClayException): - """Raised when the settings are not well-formed.""" - - # def __init__(self, msg): - # self.msg = msg - pass - - -class IdentifierNotFound(DataClayException): - """Raised when a certain identifier (UUID, name...) has not been found.""" - - pass - - -class InvalidPythonSignature(DataClayException): - """Raised when trying to use a not recognizable Python-signature.""" - - pass - - -class RemoteException(RuntimeError): - """Exception thrown in client code after a RPC call return with an exception.""" - - def __init__(self, error_code, error_string): - self.error_code = error_code - self.error_string = error_string - try: - self.error_name = ErrorCodes.error_codes[error_code] - except KeyError: - self.error_name = "UNDEFINED".format(error_code) - super(RuntimeError, self).__init__( - "Error [{}: {}]. Server response: {}".format( - self.error_code, self.error_name, self.error_string - ) - ) - - -class NetworkError(RuntimeError): - """Exception when some socket input/output recv or similar operation - does not behave as expected.""" - - def __init__(self, *args): - super(RuntimeError, self).__init__(*args) - - -class ClientError(RuntimeError): - """Exception when a client has sent some invalid request.""" - - def __init__(self, *args): - super(RuntimeError, self).__init__(*args) + return "Dataclay ID already exist!" diff --git a/src/dataclay/metadata/client.py b/src/dataclay/metadata/client.py index 29de9af..6123bc8 100644 --- a/src/dataclay/metadata/client.py +++ b/src/dataclay/metadata/client.py @@ -8,7 +8,7 @@ from dataclay.metadata.kvdata import Alias, Backend, Dataclay, ObjectMetadata from dataclay.proto.metadata import metadata_pb2, metadata_pb2_grpc from dataclay.utils.decorators import grpc_aio_error_handler -from dataclay.utils.uuid import str_to_uuid, uuid_to_str +from dataclay.utils.uuid import uuid_to_str logger = logging.getLogger(__name__) diff --git a/src/dataclay/metadata/kvdata.py b/src/dataclay/metadata/kvdata.py index f159a35..65ba8a3 100644 --- a/src/dataclay/metadata/kvdata.py +++ b/src/dataclay/metadata/kvdata.py @@ -4,7 +4,7 @@ from uuid import UUID, uuid4 import bcrypt -from google.protobuf.json_format import MessageToDict, ParseDict +from google.protobuf.json_format import MessageToDict from pydantic import BaseModel, BeforeValidator, Field from dataclay.proto.common import common_pb2 diff --git a/src/dataclay/metadata/redismanager.py b/src/dataclay/metadata/redismanager.py index f1d05f3..41a7c71 100644 --- a/src/dataclay/metadata/redismanager.py +++ b/src/dataclay/metadata/redismanager.py @@ -72,7 +72,8 @@ async def is_ready(self, timeout: Optional[float] = None, pause: float = 0.5): async def set_new(self, kv_object: KeyValue): """Sets a new key, failing if already exists. - Use "set" if the key is using a UUID (should avoid conflict), in order to optimize for etcd (if used) + Use "set" if the key is using a UUID (should avoid conflict), + in order to optimize for etcd (if used) """ if not await self._client.set(kv_object.key, kv_object.value, nx=True): raise AlreadyExistError(kv_object.key) diff --git a/src/dataclay/metadata/servicer.py b/src/dataclay/metadata/servicer.py index 626694e..5d8a3f7 100644 --- a/src/dataclay/metadata/servicer.py +++ b/src/dataclay/metadata/servicer.py @@ -3,7 +3,6 @@ import asyncio import logging import signal -import traceback from concurrent import futures from functools import wraps from uuid import UUID, uuid4 @@ -151,7 +150,8 @@ async def GetAllBackends(self, request, context): for id, backend in backends.items(): response[str(id)] = backend.get_proto() else: - # Using a cached version of the backends to avoid querying the KV store for each client request + # Using a cached version of the backends to avoid querying the KV store for + # each client request for id, backend_client in self.backend_clients.items(): backend = Backend( id=id, diff --git a/src/dataclay/paraver/__init__.py b/src/dataclay/paraver/__init__.py index 087a07f..314c35a 100644 --- a/src/dataclay/paraver/__init__.py +++ b/src/dataclay/paraver/__init__.py @@ -7,6 +7,7 @@ Additionally, this module defines an application capable of performing several Paraver-related routines, like "merge". """ + from __future__ import print_function import ctypes diff --git a/src/dataclay/runtime.py b/src/dataclay/runtime.py index a35b0db..0277b3f 100644 --- a/src/dataclay/runtime.py +++ b/src/dataclay/runtime.py @@ -6,7 +6,6 @@ import collections import copy import logging -import pickle import random from abc import ABC, abstractmethod from typing import TYPE_CHECKING, Any, Optional @@ -53,7 +52,7 @@ def __init__(self, backend_id: UUID = None): self.backend_id = backend_id self.is_backend = bool(backend_id) - # Memory objects. This dictionary must contain all objects in runtime memory (client or server), as weakrefs. + # Dictionary of all runtime memory objects stored as weakrefs. self.inmemory_objects: WeakValueDictionary[UUID, DataClayObject] = WeakValueDictionary() if settings.metrics: @@ -89,17 +88,16 @@ async def make_persistent( alias: Optional[str] = None, backend_id: Optional[str] = None, ): - """This method creates a new Persistent Object using the provided stub - instance and, if indicated, all its associated objects also Logic module API used for communication - This function is called from a stub/execution class + """ + Persist an object and optionally its associated objects. Args: - instance: Instance to make persistent - backend_id: Indicates which is the destination backend - alias: Alias for the object + instance: The object to persist. + alias: Optional alias for the object. + backend_id: Optional ID of the destination backend. Returns: - ID of the backend in which the object was persisted. + The ID of the backend where the object was persisted. """ logger.debug( "(%s) Starting make_persistent. Alias=%s, backend_id=%s", diff --git a/src/dataclay/utils/__init__.py b/src/dataclay/utils/__init__.py index 17aaca3..c02e256 100644 --- a/src/dataclay/utils/__init__.py +++ b/src/dataclay/utils/__init__.py @@ -4,6 +4,8 @@ from dataclay.alien import AlienDataClayObject from dataclay.utils.telemetry import LoggerEvent +__all__ = ["LoggerEvent"] + ALIENDCO_BASE_CLASS = re.compile(r"AlienDataClayObject\[([a-zA-Z0-9_.]+)\]") diff --git a/src/dataclay/utils/backend_clients.py b/src/dataclay/utils/backend_clients.py index ec567dd..d4acfd6 100644 --- a/src/dataclay/utils/backend_clients.py +++ b/src/dataclay/utils/backend_clients.py @@ -13,7 +13,6 @@ from dataclay.metadata.kvdata import Backend if TYPE_CHECKING: - from dataclay.metadata.api import MetadataAPI from dataclay.metadata.client import MetadataClient logger = logging.getLogger(__name__) @@ -116,7 +115,7 @@ async def add_backend_client(self, backend_info, check_ready=False): del self._backend_clients[backend_info.id] def start_subscribe(self): - """Subscribe to the new-backend-client and del-backend-client pub/sub topics. Only for backends""" + """Subscribe to the new-backend-client and del-backend-client pub/sub topics""" if not isinstance(self.metadata_api, MetadataAPI): logger.warning("Pub/sub not available. Access to kv data is not allowed for clients.") return diff --git a/src/dataclay/utils/metrics.py b/src/dataclay/utils/metrics.py index 367df0d..e70629d 100644 --- a/src/dataclay/utils/metrics.py +++ b/src/dataclay/utils/metrics.py @@ -3,6 +3,7 @@ Importing this class happens inside conditionals that check for the value of settings.metrics. If metrics are not enabled, this class will not be imported and the code will not be executed. """ + import os import threading import time diff --git a/src/dataclay/utils/serialization.py b/src/dataclay/utils/serialization.py index 964262a..48ae736 100644 --- a/src/dataclay/utils/serialization.py +++ b/src/dataclay/utils/serialization.py @@ -5,16 +5,13 @@ import logging import pickle import threading -from typing import TYPE_CHECKING, Optional +from typing import Optional from uuid import UUID from dataclay import utils from dataclay.config import get_runtime from dataclay.dataclay_object import DataClayObject -from dataclay.event_loop import dc_to_thread_cpu, get_dc_event_loop, run_dc_coroutine - -if TYPE_CHECKING: - from dataclay.dataclay_object import DataClayObject +from dataclay.event_loop import dc_to_thread_cpu, get_dc_event_loop logger = logging.getLogger(__name__) @@ -50,18 +47,19 @@ def __init__( def persistent_id(self, obj): """ - This method is called to get the persistent id of an object. - If the object is a non registered DataClayObject, then it returns a tuple with the following elements: - - "unregistered" tag - - object id - - object class - - If the object is a persistent DataClayObject, then it returns None. And the reducer_override method is called. - The reducer_override will serialize the object as a tuple with the following elements: - - get_by_id_sync method - - object id - - If the object is not a DataClayObject, then it returns None. + Retrieves the persistent ID for an object. + + If the object is an unregistered DataClayObject, returns a tuple containing: + - A tag "unregistered" + - The object's ID + - The object's class + + If the object is a registered persistent DataClayObject, returns None and triggers + the 'reducer_override' method. This method serializes the object as a tuple with: + - The get_by_id_sync method + - The object's ID + + If the object is not a DataClayObject, returns None. """ if isinstance(obj, DataClayObject): if obj._dc_is_local and not obj._dc_is_replica: @@ -172,10 +170,12 @@ async def recursive_dcloads(object_binary, unserialized_objects: dict[UUID, Data async def dcdumps(obj): - """Serialize the object using DataClayPickler. It will manage the serialization of DataClayObjects. + """Serialize the object using DataClayPickler. + It will manage the serialization of DataClayObjects. Args: - obj: The object to serialize. Should never be a DataClayObject, but the _dc_state attribute of it. + obj: The object to serialize. Should never be a DataClayObject, + but the _dc_state attribute of it. """ logger.debug("Serializing object in executor") # TODO: Avoid calling dc_to_thread_cpu if not needed. Dunnot how, but optimize! @@ -187,7 +187,8 @@ async def dcdumps(obj): async def dcloads(binary): - """Deserialize the object using pickle.loads. It will manage the deserialization of DataClayObjects. + """Deserialize the object using pickle.loads. + It will manage the deserialization of DataClayObjects. Args: binary: The binary to deserialize. Should be the result of dcdumps. diff --git a/tests/functional/conftest.py b/tests/functional/conftest.py index c01de41..0dd1ee0 100644 --- a/tests/functional/conftest.py +++ b/tests/functional/conftest.py @@ -1,6 +1,4 @@ import os -import subprocess -import time import grpc import pytest @@ -25,6 +23,9 @@ def deploy_dataclay(docker_ip, docker_services): mds_port = docker_services.port_for("metadata-service", 16587) grpc.channel_ready_future(grpc.insecure_channel(f"127.0.0.1:{mds_port}")).result(timeout=10) + # TODO: Wait for the backend to be ready before starting the tests + # NOTE: Below code is not working since it is not the correct ip + # The ip is masked by the docker-compose network # backend_port = docker_services.port_for("backend", 6867) # grpc.channel_ready_future(grpc.insecure_channel(f"127.0.0.1:{backend_port}")).result(timeout=10) diff --git a/tests/functional/test_activemethod.py b/tests/functional/test_activemethod.py index 74bd670..32dafbd 100644 --- a/tests/functional/test_activemethod.py +++ b/tests/functional/test_activemethod.py @@ -1,5 +1,3 @@ -import pytest - from dataclay.contrib.modeltest.family import Dog, Family, Person from dataclay.contrib.modeltest.remote import ActivemethodTestClass @@ -12,10 +10,10 @@ def test_activemethod_argument_make_persistent(client): family = Family() family.make_persistent() person = Person("Marc", 24) - assert person._dc_is_registered == False + assert person._dc_is_registered is False family.add(person) - assert person._dc_is_registered == True + assert person._dc_is_registered is True assert person == family.members[0] @@ -33,7 +31,8 @@ def test_activemethod_persistent_argument(client): def test_activemethod_defined_properties(client): """ - Object properties defined in class annotations are sychronized between the client and backend + Object properties defined in class annotations are sychronized + between the client and backend """ person = Person("Marc", 24) assert person.age == 24 @@ -45,7 +44,8 @@ def test_activemethod_defined_properties(client): def test_activemethod_non_defined_properties(client): """ - Object properties not defined in class annotations are not synchronized between the client and backend + Object properties not defined in class annotations are not synchronized + between the client and backend """ dog = Dog("Duna", 6) assert dog.dog_age == 6 * 7 @@ -63,7 +63,7 @@ def test_activemethod_inner_make_persistent(client): dog = Dog("Duna", 6) dog.make_persistent() puppy = dog.new_puppy("Rio") - assert puppy._dc_is_registered == True + assert puppy._dc_is_registered is True assert puppy == dog.puppies[0] assert puppy.name == "Rio" assert puppy.age == 0 diff --git a/tests/functional/test_alias.py b/tests/functional/test_alias.py index 5066351..64f2d88 100644 --- a/tests/functional/test_alias.py +++ b/tests/functional/test_alias.py @@ -1,6 +1,6 @@ import pytest -from dataclay.contrib.modeltest.family import Family, Person +from dataclay.contrib.modeltest.family import Person from dataclay.exceptions import DataClayException diff --git a/tests/functional/test_alien.py b/tests/functional/test_alien.py index c7fe839..f184028 100644 --- a/tests/functional/test_alien.py +++ b/tests/functional/test_alien.py @@ -15,7 +15,7 @@ def test_alien_builtin(client): l.make_persistent() - assert l._dc_is_registered == True + assert l._dc_is_registered is True assert len(l) == 3 assert l.__len__() == 3 @@ -42,14 +42,14 @@ def test_alien_python_class(client): assert persistent_now == now assert now == persistent_now - assert persistent_now._dc_is_registered == True + assert persistent_now._dc_is_registered is True def test_alien_pydantic_model(client): p = AlienDataClayObject(Person(name="Alice", age=30)) p.make_persistent() - assert p._dc_is_registered == True + assert p._dc_is_registered is True assert p.name == "Alice" assert p.age == 30 diff --git a/tests/functional/test_client.py b/tests/functional/test_client.py index bccf4e3..765eeea 100644 --- a/tests/functional/test_client.py +++ b/tests/functional/test_client.py @@ -1,6 +1,3 @@ -import pytest - -import dataclay from dataclay import Client diff --git a/tests/functional/test_getstate.py b/tests/functional/test_getstate.py index 6adafd1..f2b8759 100644 --- a/tests/functional/test_getstate.py +++ b/tests/functional/test_getstate.py @@ -1,5 +1,3 @@ -import pytest - from dataclay.contrib.modeltest.classes import Box, TextReader from dataclay.event_loop import run_dc_coroutine diff --git a/tests/functional/test_heapmanager.py b/tests/functional/test_heapmanager.py index 4e7c41c..b10b468 100644 --- a/tests/functional/test_heapmanager.py +++ b/tests/functional/test_heapmanager.py @@ -1,4 +1,4 @@ -from dataclay.contrib.modeltest.family import Dog, Family, Person +from dataclay.contrib.modeltest.family import Family, Person from dataclay.event_loop import run_dc_coroutine diff --git a/tests/functional/test_make_persistent.py b/tests/functional/test_make_persistent.py index f008293..d9c0b07 100644 --- a/tests/functional/test_make_persistent.py +++ b/tests/functional/test_make_persistent.py @@ -7,10 +7,10 @@ def test_make_persistent_basic(client): """Test a simple make_persistent call""" person = Person("Marc", 24) - assert person._dc_is_registered == False + assert person._dc_is_registered is False person.make_persistent() - assert person._dc_is_registered == True + assert person._dc_is_registered is True assert person.name == "Marc" assert person.age == 24 @@ -22,10 +22,10 @@ def test_make_persistent_basic(client): async def test_make_persistent_async(client): """Test a simple make_persistent call""" person = Person("Marc", 24) - assert person._dc_is_registered == False + assert person._dc_is_registered is False await person.a_make_persistent() - assert person._dc_is_registered == True + assert person._dc_is_registered is True assert person.name == "Marc" assert person.age == 24 @@ -35,10 +35,10 @@ async def test_make_persistent_async(client): def test_make_persistent_alias(client): person = Person("Marc", 24) - assert person._dc_is_registered == False + assert person._dc_is_registered is False person.make_persistent(alias="test_make_persistent_alias") - assert person._dc_is_registered == True + assert person._dc_is_registered is True persistent_person = person.get_by_alias("test_make_persistent_alias") assert persistent_person.name == person.name @@ -51,10 +51,10 @@ def test_make_persistent_alias(client): @pytest.mark.asyncio async def test_make_persistent_alias_async(client): person = Person("Marc", 24) - assert person._dc_is_registered == False + assert person._dc_is_registered is False await person.a_make_persistent(alias="test_make_persistent_alias_async") - assert person._dc_is_registered == True + assert person._dc_is_registered is True persistent_person = await person.a_get_by_alias("test_make_persistent_alias_async") assert persistent_person.name == person.name @@ -74,13 +74,13 @@ def test_make_persistent_recursive(client): dog = Dog("Rio", 5) family.add(person) person.dog = dog - assert person._dc_is_registered == False - assert dog._dc_is_registered == False + assert person._dc_is_registered is False + assert dog._dc_is_registered is False family.make_persistent() - assert person._dc_is_registered == True + assert person._dc_is_registered is True assert person == family.members[0] - assert dog._dc_is_registered == True + assert dog._dc_is_registered is True assert dog == person.dog @@ -95,13 +95,13 @@ async def test_make_persistent_recursive_async(client): dog = Dog("Rio", 5) family.add(person) person.dog = dog - assert person._dc_is_registered == False - assert dog._dc_is_registered == False + assert person._dc_is_registered is False + assert dog._dc_is_registered is False await family.a_make_persistent() - assert person._dc_is_registered == True + assert person._dc_is_registered is True assert person == family.members[0] - assert dog._dc_is_registered == True + assert dog._dc_is_registered is True assert dog == person.dog @@ -116,8 +116,8 @@ def test_make_persistent_cycle(client): person_2.spouse = person_1 person_1.make_persistent() - assert person_1._dc_is_registered == True - assert person_2._dc_is_registered == True + assert person_1._dc_is_registered is True + assert person_2._dc_is_registered is True assert person_1 == person_2.spouse assert person_2 == person_1.spouse @@ -134,8 +134,8 @@ async def test_make_persistent_cycle_async(client): person_2.spouse = person_1 await person_1.a_make_persistent() - assert person_1._dc_is_registered == True - assert person_2._dc_is_registered == True + assert person_1._dc_is_registered is True + assert person_2._dc_is_registered is True assert person_1 == person_2.spouse assert person_2 == person_1.spouse @@ -219,10 +219,10 @@ def test_persistent_references(client): person = Person("Marc", 24) person.make_persistent() family = Family(person) - assert family._dc_is_registered == False + assert family._dc_is_registered is False family.make_persistent() - assert family._dc_is_registered == True + assert family._dc_is_registered is True assert person == family.members[0] @@ -234,10 +234,10 @@ async def test_persistent_references_async(client): person = Person("Marc", 24) await person.a_make_persistent() family = Family(person) - assert family._dc_is_registered == False + assert family._dc_is_registered is False await family.a_make_persistent() - assert family._dc_is_registered == True + assert family._dc_is_registered is True assert person == family.members[0] diff --git a/tests/functional/test_move_object.py b/tests/functional/test_move_object.py index a82f154..b84f6b7 100644 --- a/tests/functional/test_move_object.py +++ b/tests/functional/test_move_object.py @@ -1,7 +1,6 @@ import pytest -from dataclay.contrib.modeltest.family import Dog, Family, Person -from dataclay.contrib.modeltest.remote import MoveObjectTestClass +from dataclay.contrib.modeltest.family import Family, Person from dataclay.event_loop import run_dc_coroutine diff --git a/tests/functional/test_new_replica.py b/tests/functional/test_new_replica.py index 7818173..448e630 100644 --- a/tests/functional/test_new_replica.py +++ b/tests/functional/test_new_replica.py @@ -1,6 +1,6 @@ import pytest -from dataclay.contrib.modeltest.family import Dog, Family, Person +from dataclay.contrib.modeltest.family import Person def test_new_replica(client): diff --git a/tests/functional/test_versioning.py b/tests/functional/test_versioning.py index 557acb8..0323a83 100644 --- a/tests/functional/test_versioning.py +++ b/tests/functional/test_versioning.py @@ -1,7 +1,7 @@ import pytest import storage.api -from dataclay.contrib.modeltest.family import Dog, Family, Person +from dataclay.contrib.modeltest.family import Family, Person from dataclay.event_loop import run_dc_coroutine From e4fe78af916ef3266ba96e081b2c3e3557e3344b Mon Sep 17 00:00:00 2001 From: Alex Barcelo Date: Mon, 21 Oct 2024 11:32:02 +0200 Subject: [PATCH 09/24] relaxing grpcio version requirement for python_version<3.13 --- pyproject.toml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pyproject.toml b/pyproject.toml index 44155eb..4963ef8 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -28,8 +28,10 @@ requires-python = ">=3.9" dependencies = [ "aiorwlock>=1.4.0", "bcrypt>=4.0.1", - "grpcio>=1.66.2", - "grpcio-health-checking>=1.66.2", + "grpcio>=1.59.2;python_version<\"3.13\"", + "grpcio>=1.66.2;python_version>=\"3.13\"", + "grpcio-health-checking>=1.59.2;python_version<\"3.13\"", + "grpcio-health-checking>=1.66.2;python_version>=\"3.13\"", "hiredis>=2.2.3", "opentelemetry-api>=1.20.0", "protobuf>=4.25.0", From f4efeeac907d29067c34b7382ebdd0fe1e64a2a1 Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Tue, 22 Oct 2024 00:03:21 +0800 Subject: [PATCH 10/24] Updated dependency versions --- pyproject.toml | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/pyproject.toml b/pyproject.toml index 4963ef8..3cfe3aa 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -27,19 +27,19 @@ classifiers = [ requires-python = ">=3.9" dependencies = [ "aiorwlock>=1.4.0", - "bcrypt>=4.0.1", + "bcrypt>=4.1.1", "grpcio>=1.59.2;python_version<\"3.13\"", "grpcio>=1.66.2;python_version>=\"3.13\"", "grpcio-health-checking>=1.59.2;python_version<\"3.13\"", "grpcio-health-checking>=1.66.2;python_version>=\"3.13\"", - "hiredis>=2.2.3", - "opentelemetry-api>=1.20.0", + "hiredis>=3.0.0", + "opentelemetry-api>=1.27.0", "protobuf>=4.25.0", - "psutil>=5.9.6", - "pydantic-settings>=2.0.3", - "redis>=5.0.1", + "psutil>=6.1.0", + "pydantic-settings>=2.6.0", + "redis>=5.1.1", "get-annotations;python_version<\"3.10\"", - "PyJWT>=2.8.0", + "PyJWT>=2.9.0", "threadpoolctl>=3.5.0", ] dynamic = ["version"] @@ -68,6 +68,7 @@ telemetry = [ "opentelemetry-exporter-otlp-proto-grpc", "opentelemetry-instrumentation-grpc", "opentelemetry-instrumentation-redis", + "protobuf<5", # Remove when protobuf 5 is supported by opentelemetry ] [project.scripts] From 76700e6b850f793902b80f9352742b0889db4496 Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Tue, 22 Oct 2024 00:10:22 +0800 Subject: [PATCH 11/24] Install telemetry dependencies in nox dev session --- noxfile.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/noxfile.py b/noxfile.py index 601ff4f..5fd7f48 100644 --- a/noxfile.py +++ b/noxfile.py @@ -57,7 +57,7 @@ def dev(session: nox.Session) -> None: """ session.install("virtualenv") session.run("virtualenv", ".venv", silent=True) - session.run(".venv/bin/pip", "install", "-e", ".[dev]", external=True) + session.run(".venv/bin/pip", "install", "-e", ".[telemetry,dev]", external=True) # TODO: Check https://nox.thea.codes/en/stable/cookbook.html#the-auto-release From 4f105469ec7c6153ea99674f98ef9e3cbb012dfe Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Tue, 22 Oct 2024 01:30:08 +0800 Subject: [PATCH 12/24] Updated tracing/json-exporter example --- examples/tracing/json-exporter/README.md | 36 +++++++++++++++++++ .../tracing/json-exporter/docker-compose.yaml | 15 ++++++-- ...porter.yaml => otel-collector-config.yaml} | 4 +++ src/dataclay/config.py | 2 +- src/dataclay/utils/telemetry.py | 4 +++ 5 files changed, 57 insertions(+), 4 deletions(-) create mode 100644 examples/tracing/json-exporter/README.md rename examples/tracing/json-exporter/{otel-json-exporter.yaml => otel-collector-config.yaml} (86%) diff --git a/examples/tracing/json-exporter/README.md b/examples/tracing/json-exporter/README.md new file mode 100644 index 0000000..33377bb --- /dev/null +++ b/examples/tracing/json-exporter/README.md @@ -0,0 +1,36 @@ + +# JSON Exporter + +This example demonstrates how to export opentelemetry traces to a JSON file. The JSON file is created in the `traces` folder. + +To activate traces in dataClay, you need to set the following environment variables: + +- `DATACLAY_TRACING`: Set to `true` to enable tracing. +- `DATACLAY_TRACING_EXPORTER`: Set to `otlp` to export traces to the OpenTelemetry Collector, or `console` to print traces to the console (default is `otlp`). +- `DATACLAY_TRACING_HOST`: The host of the OpenTelemetry Collector (default is `localhost`). +- `DATACLAY_TRACING_PORT`: The port of the OpenTelemetry Collector (default is 4317). +- `DATACLAY_SERVICE_NAME`: The service name. + +## Running the Example + +Start the dataClay services: + +```bash +docker compose up +``` + +Run the example: + +```bash +python3 client.py +``` + +The traces are exported to the `traces` folder. You can view the traces in the JSON file. + +## Troubleshooting + +If you get this error `open /traces/otel-traces.json: permission denied`, then you need to give the permission to the `traces` folder. + +```bash +sudo chmod -R 777 traces +``` diff --git a/examples/tracing/json-exporter/docker-compose.yaml b/examples/tracing/json-exporter/docker-compose.yaml index 2d72ff1..57934f2 100644 --- a/examples/tracing/json-exporter/docker-compose.yaml +++ b/examples/tracing/json-exporter/docker-compose.yaml @@ -21,6 +21,7 @@ services: - DATACLAY_TRACING_EXPORTER=otlp - DATACLAY_SERVICE_NAME=metadata - DATACLAY_TRACING_HOST=otel-collector + - DATACLAY_LOGLEVEL=DEBUG command: python -m dataclay.metadata volumes: - ../../../:/app:ro @@ -36,14 +37,22 @@ services: - DATACLAY_TRACING_EXPORTER=otlp - DATACLAY_SERVICE_NAME=backend - DATACLAY_TRACING_HOST=otel-collector + - DATACLAY_LOGLEVEL=DEBUG command: python -m dataclay.backend volumes: - ../../../:/app:ro otel-collector: - image: otel/opentelemetry-collector:latest - command: [ "--config=/etc/otel-collector.yaml" ] + image: otel/opentelemetry-collector-contrib:latest volumes: - - ./otel-json-exporter.yaml:/etc/otel-collector.yaml + - ./otel-collector-config.yaml:/etc/otelcol-contrib/config.yaml - ./traces/:/traces/ + ports: + - 1888:1888 # pprof extension + - 8888:8888 # Prometheus metrics exposed by the Collector + - 8889:8889 # Prometheus exporter metrics + - 13133:13133 # health_check extension + - 4317:4317 # OTLP gRPC receiver + - 4318:4318 # OTLP http receiver + - 55679:55679 # zpages extension diff --git a/examples/tracing/json-exporter/otel-json-exporter.yaml b/examples/tracing/json-exporter/otel-collector-config.yaml similarity index 86% rename from examples/tracing/json-exporter/otel-json-exporter.yaml rename to examples/tracing/json-exporter/otel-collector-config.yaml index 8227fdc..0b6fbcc 100644 --- a/examples/tracing/json-exporter/otel-json-exporter.yaml +++ b/examples/tracing/json-exporter/otel-collector-config.yaml @@ -2,11 +2,15 @@ receivers: otlp: protocols: grpc: + endpoint: "0.0.0.0:4317" + exporters: file: path: /traces/otel-traces.json + processors: batch: + service: pipelines: traces: diff --git a/src/dataclay/config.py b/src/dataclay/config.py index 8158a58..e3e4b4d 100644 --- a/src/dataclay/config.py +++ b/src/dataclay/config.py @@ -183,7 +183,7 @@ class Settings(BaseSettings): # Tracing service_name: Optional[str] = None tracing: bool = False - tracing_exporter: Literal["otlp", "jaeger", "zipkin", "none"] = "otlp" + tracing_exporter: Literal["otlp", "console", "none"] = "otlp" tracing_host: str = "localhost" tracing_port: int = 4317 diff --git a/src/dataclay/utils/telemetry.py b/src/dataclay/utils/telemetry.py index 55ed067..168047b 100644 --- a/src/dataclay/utils/telemetry.py +++ b/src/dataclay/utils/telemetry.py @@ -56,12 +56,14 @@ def set_tracing(service_name, host, port, exporter="otlp"): OTLPSpanExporter, ) + logger.debug("Setting OTLP exporter") otlp_exporter = OTLPSpanExporter(endpoint=f"{host}:{port}", insecure=True) processor = BatchSpanProcessor(otlp_exporter) elif exporter == "console": from opentelemetry.sdk.trace.export import ConsoleSpanExporter + logger.debug("Setting Console exporter") processor = BatchSpanProcessor(ConsoleSpanExporter()) trace.get_tracer_provider().add_span_processor(processor) @@ -76,3 +78,5 @@ def set_tracing(service_name, host, port, exporter="otlp"): GrpcInstrumentorServer().instrument() RedisInstrumentor().instrument() + + logger.info("Tracer %s set for service %s", exporter, service_name) From e432ac4df74074faf4be8576d2d3358ebe184091 Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Wed, 23 Oct 2024 01:41:19 +0800 Subject: [PATCH 13/24] Replace tox to nox in contributin.rst docs --- docs/contributing.rst | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/docs/contributing.rst b/docs/contributing.rst index 0a28b95..d7e948b 100644 --- a/docs/contributing.rst +++ b/docs/contributing.rst @@ -1,7 +1,8 @@ Contributing ============ -dataClay is a `BSC `_ +The dataClay distributed data store is a +`BSC `_ project under the `BSD License `_ and we happily accept contributions. @@ -16,7 +17,7 @@ If you wish to add a new feature or fix a bug: #. Write a test which shows that the bug was fixed or that the feature works as expected. #. Format your changes with `Black `_ using the - command `tox -e format` and lint your changes using the command `tox -e lint`. + command `nox -s format` and lint your changes using the command `nox -s lint`. #. Send a pull request and follow up with the maintainer until it gets merged and published. .. #. Add a `changelog entry @@ -25,14 +26,14 @@ If you wish to add a new feature or fix a bug: Setting up your development environment --------------------------------------- -To set up your development environment, you will need `tox`_ installed on your machine: +To set up your development environment, you will need `nox`_ installed on your machine: .. code-block:: console - $ python -m pip install --user --upgrade tox + $ python -m pip install --user --upgrade nox You wll also need to have `docker engine `_ installed -for `tox`_ to use `pytest-docker `_. +for `nox`_ to use `pytest-docker `_. Install dataClay in editable mode with the ``dev`` extra requirement: @@ -44,11 +45,11 @@ Running the tests ----------------- When running the test suite, we use external dependencies, multiple interpreters, and code coverage analysis. -Our `tox.ini `_ file handles much of this for you: +Our `noxfile.py `_ file handles much of this for you: .. code-block:: console - $ tox + $ nox -.. _tox: https://tox.wiki/en/stable/ \ No newline at end of file +.. _nox: https://nox.thea.codes/en/stable/ \ No newline at end of file From ce49c8e4c0381a928134aca0b1173f4236e0459f Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Thu, 24 Oct 2024 01:39:10 +0800 Subject: [PATCH 14/24] Update and move tracing examples to a common telemetry folder, and update documentation --- examples/telemetry/offline/README.md | 71 +++++++++++++++++++ .../offline}/json-exporter/.env | 0 .../offline}/json-exporter/client.py | 0 .../json-exporter/docker-compose.yaml | 8 +-- .../offline}/json-exporter/job.sh | 0 .../json-exporter/otel-collector-config.yaml | 0 .../json-post-mortem/docker-compose.yaml | 10 +-- .../grafana-datasources.yaml | 0 .../json-post-mortem}/otel-collector.yaml | 0 .../offline/json-post-mortem}/tempo.yaml | 23 ++++-- .../offline}/traces/.gitkeep | 0 .../telemetry/offline/traces/otel-traces.json | 7 ++ examples/tracing/README.md | 2 - examples/tracing/json-exporter/README.md | 36 ---------- .../json-exporter/traces/otel-traces.json | 2 - examples/tracing/json-post-mortem/README.md | 12 ---- .../tracing/json-post-mortem/traces/.gitkeep | 0 .../traces/otel-traces-old.json | 2 - .../json-post-mortem/traces/otel-traces.json | 2 - 19 files changed, 107 insertions(+), 68 deletions(-) create mode 100644 examples/telemetry/offline/README.md rename examples/{tracing => telemetry/offline}/json-exporter/.env (100%) rename examples/{tracing => telemetry/offline}/json-exporter/client.py (100%) rename examples/{tracing => telemetry/offline}/json-exporter/docker-compose.yaml (92%) rename examples/{tracing => telemetry/offline}/json-exporter/job.sh (100%) rename examples/{tracing => telemetry/offline}/json-exporter/otel-collector-config.yaml (100%) rename examples/{tracing => telemetry/offline}/json-post-mortem/docker-compose.yaml (72%) rename examples/{tracing/json-post-mortem/config => telemetry/offline/json-post-mortem}/grafana-datasources.yaml (100%) rename examples/{tracing/json-post-mortem/config => telemetry/offline/json-post-mortem}/otel-collector.yaml (100%) rename examples/{tracing/json-post-mortem/config => telemetry/offline/json-post-mortem}/tempo.yaml (59%) rename examples/{tracing/json-exporter => telemetry/offline}/traces/.gitkeep (100%) create mode 100644 examples/telemetry/offline/traces/otel-traces.json delete mode 100644 examples/tracing/README.md delete mode 100644 examples/tracing/json-exporter/README.md delete mode 100755 examples/tracing/json-exporter/traces/otel-traces.json delete mode 100644 examples/tracing/json-post-mortem/README.md delete mode 100644 examples/tracing/json-post-mortem/traces/.gitkeep delete mode 100755 examples/tracing/json-post-mortem/traces/otel-traces-old.json delete mode 100755 examples/tracing/json-post-mortem/traces/otel-traces.json diff --git a/examples/telemetry/offline/README.md b/examples/telemetry/offline/README.md new file mode 100644 index 0000000..d0be9ba --- /dev/null +++ b/examples/telemetry/offline/README.md @@ -0,0 +1,71 @@ +# Offline Telemetry Example + +This example demonstrates how to export opentelemetry traces to a JSON file and then +visualize in Grafana. + +To activate traces in dataClay, you need to set the following environment variables: + +- `DATACLAY_TRACING`: Set to `true` to enable tracing. +- `DATACLAY_TRACING_EXPORTER`: Set to `otlp` to export traces to the OpenTelemetry Collector, or `console` to print traces to the console (default is `otlp`). +- `DATACLAY_TRACING_HOST`: The host of the OpenTelemetry Collector (default is `localhost`). +- `DATACLAY_TRACING_PORT`: The port of the OpenTelemetry Collector (default is 4317). +- `DATACLAY_SERVICE_NAME`: The service name. + +## Generating Traces and Exporting to JSON file + +Go into the `json-exporter` folder: + +```bash +cd json-exporter +``` + +Start the dataClay and OpenTelemetry Collector services: + +```bash +docker compose up +``` + +Run the dataClay client to generate some traces: + +```bash +python3 client.py +``` + +The traces are exported to the `traces` folder. You can view the traces in the JSON file. + +## Visualizing the Traces in Grafana + +Go into the `json-post-mortem` folder: + +```bash +cd json-post-mortem +``` + +Start the OpenTelemetry Collector, Tempo, and Grafana services: + +```bash +docker compose up +``` + +Open the Grafana UI in your browser at . The default username and password are both `admin`. + +Navigate to the `Explore` section and select the `Tempo` data source. You can query the traces using the `Trace ID` field. + +You could also run manually the OpenTelemetry Collector with the following command: + +```bash +docker run \ +-v ./config/otel-collector.yaml:/etc/otel-collector.yaml \ +otel/opentelemetry-collector-contrib \ +"--config=/etc/otel-collector.yaml" +``` + +## Troubleshooting + +If you get a `permission denied` error for the `/traces` folder, then you need to give the permission to the `traces` folder. + +```bash +sudo chmod -R 777 traces +``` + + diff --git a/examples/tracing/json-exporter/.env b/examples/telemetry/offline/json-exporter/.env similarity index 100% rename from examples/tracing/json-exporter/.env rename to examples/telemetry/offline/json-exporter/.env diff --git a/examples/tracing/json-exporter/client.py b/examples/telemetry/offline/json-exporter/client.py similarity index 100% rename from examples/tracing/json-exporter/client.py rename to examples/telemetry/offline/json-exporter/client.py diff --git a/examples/tracing/json-exporter/docker-compose.yaml b/examples/telemetry/offline/json-exporter/docker-compose.yaml similarity index 92% rename from examples/tracing/json-exporter/docker-compose.yaml rename to examples/telemetry/offline/json-exporter/docker-compose.yaml index 57934f2..a7aa214 100644 --- a/examples/tracing/json-exporter/docker-compose.yaml +++ b/examples/telemetry/offline/json-exporter/docker-compose.yaml @@ -8,7 +8,7 @@ services: metadata-service: image: "ghcr.io/bsc-dom/dataclay:dev" build: - context: ../../../ + context: ../../../../ dockerfile: Dockerfile.dev depends_on: - redis @@ -24,7 +24,7 @@ services: - DATACLAY_LOGLEVEL=DEBUG command: python -m dataclay.metadata volumes: - - ../../../:/app:ro + - ../../../../:/app:ro backend: image: "ghcr.io/bsc-dom/dataclay:dev" @@ -40,13 +40,13 @@ services: - DATACLAY_LOGLEVEL=DEBUG command: python -m dataclay.backend volumes: - - ../../../:/app:ro + - ../../../../:/app:ro otel-collector: image: otel/opentelemetry-collector-contrib:latest volumes: - ./otel-collector-config.yaml:/etc/otelcol-contrib/config.yaml - - ./traces/:/traces/ + - ../traces/:/traces/ ports: - 1888:1888 # pprof extension - 8888:8888 # Prometheus metrics exposed by the Collector diff --git a/examples/tracing/json-exporter/job.sh b/examples/telemetry/offline/json-exporter/job.sh similarity index 100% rename from examples/tracing/json-exporter/job.sh rename to examples/telemetry/offline/json-exporter/job.sh diff --git a/examples/tracing/json-exporter/otel-collector-config.yaml b/examples/telemetry/offline/json-exporter/otel-collector-config.yaml similarity index 100% rename from examples/tracing/json-exporter/otel-collector-config.yaml rename to examples/telemetry/offline/json-exporter/otel-collector-config.yaml diff --git a/examples/tracing/json-post-mortem/docker-compose.yaml b/examples/telemetry/offline/json-post-mortem/docker-compose.yaml similarity index 72% rename from examples/tracing/json-post-mortem/docker-compose.yaml rename to examples/telemetry/offline/json-post-mortem/docker-compose.yaml index 91f1c83..b772b92 100644 --- a/examples/tracing/json-post-mortem/docker-compose.yaml +++ b/examples/telemetry/offline/json-post-mortem/docker-compose.yaml @@ -4,11 +4,13 @@ services: image: grafana/tempo:latest command: [ "-config.file=/etc/tempo.yaml" ] volumes: - - ./config/tempo.yaml:/etc/tempo.yaml + - ./tempo.yaml:/etc/tempo.yaml ports: + - "14268" # jaeger ingest - "3200" # tempo - "4317" # otlp grpc - "4318" # otlp http + - "9411" # zipkin grafana: image: grafana/grafana:latest @@ -23,7 +25,7 @@ services: ports: - "3000:3000" volumes: - - ./config/grafana-datasources.yaml:/etc/grafana/provisioning/datasources/datasources.yaml + - ./grafana-datasources.yaml:/etc/grafana/provisioning/datasources/datasources.yaml otel-collector: image: otel/opentelemetry-collector-contrib:latest @@ -32,6 +34,6 @@ services: - grafana command: [ "--config=/etc/otel-collector.yaml" ] volumes: - - ./config/otel-collector.yaml:/etc/otel-collector.yaml - - ./traces:/tmp/traces + - ./otel-collector.yaml:/etc/otel-collector.yaml + - ../traces:/tmp/traces diff --git a/examples/tracing/json-post-mortem/config/grafana-datasources.yaml b/examples/telemetry/offline/json-post-mortem/grafana-datasources.yaml similarity index 100% rename from examples/tracing/json-post-mortem/config/grafana-datasources.yaml rename to examples/telemetry/offline/json-post-mortem/grafana-datasources.yaml diff --git a/examples/tracing/json-post-mortem/config/otel-collector.yaml b/examples/telemetry/offline/json-post-mortem/otel-collector.yaml similarity index 100% rename from examples/tracing/json-post-mortem/config/otel-collector.yaml rename to examples/telemetry/offline/json-post-mortem/otel-collector.yaml diff --git a/examples/tracing/json-post-mortem/config/tempo.yaml b/examples/telemetry/offline/json-post-mortem/tempo.yaml similarity index 59% rename from examples/tracing/json-post-mortem/config/tempo.yaml rename to examples/telemetry/offline/json-post-mortem/tempo.yaml index 4735596..e2699ed 100644 --- a/examples/tracing/json-post-mortem/config/tempo.yaml +++ b/examples/telemetry/offline/json-post-mortem/tempo.yaml @@ -1,5 +1,17 @@ +stream_over_http_enabled: true server: http_listen_port: 3200 + log_level: info + +# query_frontend: +# search: +# duration_slo: 5s +# throughput_bytes_slo: 1.073741824e+09 +# metadata_slo: +# duration_slo: 5s +# throughput_bytes_slo: 1.073741824e+09 +# trace_by_id: +# duration_slo: 5s distributor: receivers: # this configuration will listen on all ports and protocols that tempo is capable of. @@ -21,15 +33,18 @@ ingester: compactor: compaction: - block_retention: 1h # overall Tempo trace retention. set for demo purposes + block_retention: 100000h # Retain traces for a very long period (about 416 days). Use only for post-mortem analysis. storage: trace: backend: local # backend configuration to use wal: - path: /tmp/tempo/wal # where to store the the wal locally + path: /var/tempo/wal # where to store the wal locally local: - path: /tmp/tempo/blocks + path: /var/tempo/blocks overrides: - metrics_generator_processors: [service-graphs, span-metrics] # enables metrics generator + defaults: + metrics_generator: + processors: [service-graphs, span-metrics, local-blocks] # enables metrics generator + generate_native_histograms: both diff --git a/examples/tracing/json-exporter/traces/.gitkeep b/examples/telemetry/offline/traces/.gitkeep similarity index 100% rename from examples/tracing/json-exporter/traces/.gitkeep rename to examples/telemetry/offline/traces/.gitkeep diff --git a/examples/telemetry/offline/traces/otel-traces.json b/examples/telemetry/offline/traces/otel-traces.json new file mode 100644 index 0000000..0624721 --- /dev/null +++ b/examples/telemetry/offline/traces/otel-traces.json @@ -0,0 +1,7 @@ +{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"backend"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.48b0"},"spans":[{"traceId":"98acf5406dd9aa5c00f04a3c1c281787","spanId":"f6960f3ce71f1fe7","parentSpanId":"","flags":256,"name":"PING","kind":3,"startTimeUnixNano":"1729703299029276888","endTimeUnixNano":"1729703299037560309","attributes":[{"key":"db.statement","value":{"stringValue":"PING"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"c4d0e7d2e4c7a320bc8b57c6cd0aaf7f","spanId":"ed7e43310164fa34","parentSpanId":"7e515c677f971109","flags":256,"name":"SCAN","kind":3,"startTimeUnixNano":"1729703299031308838","endTimeUnixNano":"1729703299038665557","attributes":[{"key":"db.statement","value":{"stringValue":"SCAN ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"6be81f8fa54cbb840834789f2d55152b","spanId":"dfa41a7ee2b80cc8","parentSpanId":"ec8826ccebbe38a7","flags":256,"name":"GET","kind":3,"startTimeUnixNano":"1729703299037981341","endTimeUnixNano":"1729703299038960551","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}},{"traceId":"2b179b73e966deef38f6e264c8009301","spanId":"7d1d096ddeb85344","parentSpanId":"ada567b48b27b766","flags":256,"name":"GET","kind":3,"startTimeUnixNano":"1729703299541903938","endTimeUnixNano":"1729703299542938372","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}},{"traceId":"0cf9821c8f0431a69ba7752ebd90047a","spanId":"24b447f0eab0d76a","parentSpanId":"fcb61adcab6fe208","flags":256,"name":"SET","kind":3,"startTimeUnixNano":"1729703299546421299","endTimeUnixNano":"1729703299547012801","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"0cf9821c8f0431a69ba7752ebd90047a","spanId":"69659ef0758186a7","parentSpanId":"fcb61adcab6fe208","flags":256,"name":"PUBLISH","kind":3,"startTimeUnixNano":"1729703299547230711","endTimeUnixNano":"1729703299547620524","attributes":[{"key":"db.statement","value":{"stringValue":"PUBLISH ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"3"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"c4d0e7d2e4c7a320bc8b57c6cd0aaf7f","spanId":"7e515c677f971109","parentSpanId":"","flags":256,"name":"get_all_backends","kind":1,"startTimeUnixNano":"1729703299030967256","endTimeUnixNano":"1729703299038724908","status":{}},{"traceId":"6be81f8fa54cbb840834789f2d55152b","spanId":"ec8826ccebbe38a7","parentSpanId":"","flags":256,"name":"get_dataclay","kind":1,"startTimeUnixNano":"1729703299037732052","endTimeUnixNano":"1729703299040572271","events":[{"timeUnixNano":"1729703299040538437","name":"exception","attributes":[{"key":"exception.type","value":{"stringValue":"dataclay.exceptions.DoesNotExistError"}},{"key":"exception.message","value":{"stringValue":"/dataclay/this does not exist"}},{"key":"exception.stacktrace","value":{"stringValue":"Traceback (most recent call last):\n File \"/usr/local/lib/python3.10/site-packages/opentelemetry/trace/__init__.py\", line 590, in use_span\n yield span\n File \"/usr/local/lib/python3.10/site-packages/opentelemetry/sdk/trace/__init__.py\", line 1108, in start_as_current_span\n yield span\n File \"/usr/local/lib/python3.10/site-packages/opentelemetry/trace/__init__.py\", line 453, in start_as_current_span\n yield span\n File \"/usr/local/lib/python3.10/site-packages/opentelemetry/util/_decorator.py\", line 68, in async_wrapper\n return await func(*args, **kwargs) # type: ignore\n File \"/app/src/dataclay/metadata/api.py\", line 153, in get_dataclay\n return await self.kv_manager.get_kv(Dataclay, dataclay_id)\n File \"/app/src/dataclay/metadata/redismanager.py\", line 99, in get_kv\n raise DoesNotExistError(name)\ndataclay.exceptions.DoesNotExistError: /dataclay/this does not exist\n"}},{"key":"exception.escaped","value":{"stringValue":"False"}}]}],"status":{"message":"DoesNotExistError: /dataclay/this does not exist","code":2}},{"traceId":"2b179b73e966deef38f6e264c8009301","spanId":"ada567b48b27b766","parentSpanId":"","flags":256,"name":"get_dataclay","kind":1,"startTimeUnixNano":"1729703299541446358","endTimeUnixNano":"1729703299543105607","status":{}},{"traceId":"0cf9821c8f0431a69ba7752ebd90047a","spanId":"fcb61adcab6fe208","parentSpanId":"","flags":256,"name":"register_backend","kind":1,"startTimeUnixNano":"1729703299545944954","endTimeUnixNano":"1729703299547644058","status":{}}]}]},{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"metadata"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.48b0"},"spans":[{"traceId":"2eaa0a5b4e9ea6941dedcceb67a8a5a8","spanId":"c3eab6f2c48cbc1f","parentSpanId":"","flags":256,"name":"PING","kind":3,"startTimeUnixNano":"1729703299056084861","endTimeUnixNano":"1729703299060497957","attributes":[{"key":"db.statement","value":{"stringValue":"PING"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"2007fc10f8b1f362f88f0f78c6c125f6","spanId":"5895ad67d47a87e5","parentSpanId":"00a658e5bb6677d8","flags":256,"name":"SET","kind":3,"startTimeUnixNano":"1729703299061259538","endTimeUnixNano":"1729703299061951810","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"33dab5c9719b69a1e1f2d92660b2b406","spanId":"6d35df0136eca756","parentSpanId":"79c1792cecb4399b","flags":256,"name":"SET","kind":3,"startTimeUnixNano":"1729703299679522218","endTimeUnixNano":"1729703299680579315","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"33dab5c9719b69a1e1f2d92660b2b406","spanId":"e2141607c1654242","parentSpanId":"79c1792cecb4399b","flags":256,"name":"SET","kind":3,"startTimeUnixNano":"1729703299680796283","endTimeUnixNano":"1729703299681380511","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"9f5c97ad097e07c1c2bae08ebcc35031","spanId":"311757c6c23f3301","parentSpanId":"75b68f94296463a3","flags":256,"name":"SCAN","kind":3,"startTimeUnixNano":"1729703299684636572","endTimeUnixNano":"1729703299686465941","attributes":[{"key":"db.statement","value":{"stringValue":"SCAN ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"9f5c97ad097e07c1c2bae08ebcc35031","spanId":"ad05b4c4a43c5ff5","parentSpanId":"75b68f94296463a3","flags":256,"name":"GET","kind":3,"startTimeUnixNano":"1729703299686711673","endTimeUnixNano":"1729703299687728083","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"2007fc10f8b1f362f88f0f78c6c125f6","spanId":"00a658e5bb6677d8","parentSpanId":"","flags":256,"name":"new_dataclay","kind":1,"startTimeUnixNano":"1729703299060824070","endTimeUnixNano":"1729703299062084068","status":{}},{"traceId":"33dab5c9719b69a1e1f2d92660b2b406","spanId":"79c1792cecb4399b","parentSpanId":"","flags":256,"name":"new_superuser","kind":1,"startTimeUnixNano":"1729703299062217439","endTimeUnixNano":"1729703299681643155","status":{}},{"traceId":"9f5c97ad097e07c1c2bae08ebcc35031","spanId":"75b68f94296463a3","parentSpanId":"","flags":256,"name":"get_all_backends","kind":1,"startTimeUnixNano":"1729703299684345745","endTimeUnixNano":"1729703299687961322","status":{}}]}]}]} +{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"backend"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.48b0"},"spans":[{"traceId":"e90461971549304013c30551bae474de","spanId":"ac4e4311c506dadc","parentSpanId":"927597c9f3de0b4c","flags":256,"name":"SCAN","kind":3,"startTimeUnixNano":"1729703309046442470","endTimeUnixNano":"1729703309047439494","attributes":[{"key":"db.statement","value":{"stringValue":"SCAN ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"e90461971549304013c30551bae474de","spanId":"14d3ae225a3104e7","parentSpanId":"927597c9f3de0b4c","flags":256,"name":"GET","kind":3,"startTimeUnixNano":"1729703309047603573","endTimeUnixNano":"1729703309048146543","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"e90461971549304013c30551bae474de","spanId":"927597c9f3de0b4c","parentSpanId":"","flags":256,"name":"get_all_backends","kind":1,"startTimeUnixNano":"1729703309046101960","endTimeUnixNano":"1729703309048292909","status":{}}]}]},{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"metadata"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.48b0"},"spans":[{"traceId":"a5c978a102e6da9d52d98a525c0af4ca","spanId":"7833fc57877f1b9f","parentSpanId":"4bd4df4a1f6ac3c2","flags":256,"name":"SCAN","kind":3,"startTimeUnixNano":"1729703309699530683","endTimeUnixNano":"1729703309700537295","attributes":[{"key":"db.statement","value":{"stringValue":"SCAN ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"a5c978a102e6da9d52d98a525c0af4ca","spanId":"d1156a184256f255","parentSpanId":"4bd4df4a1f6ac3c2","flags":256,"name":"GET","kind":3,"startTimeUnixNano":"1729703309700726361","endTimeUnixNano":"1729703309701335606","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"a5c978a102e6da9d52d98a525c0af4ca","spanId":"4bd4df4a1f6ac3c2","parentSpanId":"","flags":256,"name":"get_all_backends","kind":1,"startTimeUnixNano":"1729703309699207115","endTimeUnixNano":"1729703309701502369","status":{}}]}]}]} +{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"client"}}]},"scopeSpans":[{"scope":{"name":"dataclay.client.api"},"spans":[{"traceId":"6c5a91ee33b65d8fadbba0a12cec4d06","spanId":"87895257267b207d","parentSpanId":"","flags":256,"name":"start","kind":1,"startTimeUnixNano":"1729703314775081009","endTimeUnixNano":"1729703314780618929","status":{}}]},{"scope":{"name":"dataclay.dataclay_object"},"spans":[{"traceId":"0033a3f6b4319c2311fdfcbba8fa30ae","spanId":"6bd3fa3318c01f54","parentSpanId":"","flags":256,"name":"get_by_alias","kind":1,"startTimeUnixNano":"1729703314781311270","endTimeUnixNano":"1729703314796797990","events":[{"timeUnixNano":"1729703314796732927","name":"exception","attributes":[{"key":"exception.type","value":{"stringValue":"dataclay.exceptions.DataClayException"}},{"key":"exception.message","value":{"stringValue":"/alias/admin/myfamily does not exist"}},{"key":"exception.stacktrace","value":{"stringValue":"Traceback (most recent call last):\n File \"/home/marc/dev/dataclay/venv/lib/python3.12/site-packages/opentelemetry/trace/__init__.py\", line 590, in use_span\n yield span\n File \"/home/marc/dev/dataclay/venv/lib/python3.12/site-packages/opentelemetry/sdk/trace/__init__.py\", line 1108, in start_as_current_span\n yield span\n File \"/home/marc/dev/dataclay/venv/lib/python3.12/site-packages/opentelemetry/trace/__init__.py\", line 453, in start_as_current_span\n yield span\n File \"/home/marc/dev/dataclay/venv/lib/python3.12/site-packages/opentelemetry/util/_decorator.py\", line 68, in async_wrapper\n return await func(*args, **kwargs) # type: ignore\n ^^^^^^^^^^^^^^^^^^^^^^^^^^^\n File \"/home/marc/dev/dataclay/src/dataclay/dataclay_object.py\", line 475, in _get_by_alias\n return await get_runtime().get_object_by_alias(alias, dataset_name)\n ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^\n File \"/home/marc/dev/dataclay/src/dataclay/runtime.py\", line 248, in get_object_by_alias\n object_md = await self.metadata_service.get_object_md_by_alias(alias, dataset_name)\n ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^\n File \"/home/marc/dev/dataclay/src/dataclay/utils/decorators.py\", line 25, in wrapper\n raise DataClayException(rpc_error.details()) from None\ndataclay.exceptions.DataClayException: /alias/admin/myfamily does not exist\n"}},{"key":"exception.escaped","value":{"stringValue":"False"}}]}],"status":{"message":"DataClayException: /alias/admin/myfamily does not exist","code":2}},{"traceId":"0928345ec4b6484dbf9ba0cdb309e1ff","spanId":"ff57e76f1a6cb64a","parentSpanId":"","flags":256,"name":"make_persistent","kind":1,"startTimeUnixNano":"1729703314797780567","endTimeUnixNano":"1729703314816107856","status":{}},{"traceId":"3f2e105c6188e389ef6f37f6ac82256c","spanId":"919f25e8b469fe20","parentSpanId":"0be31e01ca114a47","flags":256,"name":"make_persistent","kind":1,"startTimeUnixNano":"1729703314818724594","endTimeUnixNano":"1729703314824076053","status":{}}]},{"scope":{"name":"dataclay.runtime"},"spans":[{"traceId":"3f2e105c6188e389ef6f37f6ac82256c","spanId":"0be31e01ca114a47","parentSpanId":"","flags":256,"name":"call_remote_method","kind":1,"startTimeUnixNano":"1729703314816986507","endTimeUnixNano":"1729703314843452023","attributes":[{"key":"class","value":{"stringValue":"dataclay.contrib.modeltest.family.Family"}},{"key":"method","value":{"stringValue":"add"}},{"key":"args","value":{"stringValue":"(\u003cdataclay.contrib.modeltest.family.Person volatile instance with ObjectID=94bce497-d159-44ee-89f4-3a79b630d2c6\u003e,)"}},{"key":"kwargs","value":{"stringValue":"{}"}}],"status":{}},{"traceId":"9713adf7486923c6fb8393b7ef14c8fa","spanId":"720acad52e6a9dfb","parentSpanId":"","flags":256,"name":"call_remote_method","kind":1,"startTimeUnixNano":"1729703314844114779","endTimeUnixNano":"1729703314851282321","attributes":[{"key":"class","value":{"stringValue":"dataclay.contrib.modeltest.family.Family"}},{"key":"method","value":{"stringValue":"__str__"}},{"key":"args","value":{"stringValue":"()"}},{"key":"kwargs","value":{"stringValue":"{}"}}],"status":{}}]}]}]} +{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"backend"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.48b0"},"spans":[{"traceId":"eb79820c484a7e5ebe1cffecd7f22793","spanId":"c2ce8ab63556b4d2","parentSpanId":"a260196896f32903","flags":256,"name":"SET","kind":3,"startTimeUnixNano":"1729703314813881541","endTimeUnixNano":"1729703314814928088","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"3"}}],"status":{}},{"traceId":"b624487ec6452dae8499a2f247c6588e","spanId":"65341bb1c0928b9e","parentSpanId":"347c0c053e407234","flags":256,"name":"SET","kind":3,"startTimeUnixNano":"1729703314822658068","endTimeUnixNano":"1729703314823339218","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"3"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"eb79820c484a7e5ebe1cffecd7f22793","spanId":"a260196896f32903","parentSpanId":"fd6bb6879d80c0e8","flags":256,"name":"upsert_object","kind":1,"startTimeUnixNano":"1729703314813444850","endTimeUnixNano":"1729703314814975207","status":{}},{"traceId":"b624487ec6452dae8499a2f247c6588e","spanId":"347c0c053e407234","parentSpanId":"548d2ee3758e6ef2","flags":256,"name":"upsert_object","kind":1,"startTimeUnixNano":"1729703314822254279","endTimeUnixNano":"1729703314823378442","status":{}}]},{"scope":{"name":"dataclay.backend.api"},"spans":[{"traceId":"eb79820c484a7e5ebe1cffecd7f22793","spanId":"fd6bb6879d80c0e8","parentSpanId":"","flags":256,"name":"make_persistent","kind":1,"startTimeUnixNano":"1729703314808564176","endTimeUnixNano":"1729703314815019830","events":[{"timeUnixNano":"1729703314808700803","name":"Receiving (1) objects to make persistent"},{"timeUnixNano":"1729703314813062732","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Registering Family"}],"status":{}},{"traceId":"b624487ec6452dae8499a2f247c6588e","spanId":"548d2ee3758e6ef2","parentSpanId":"","flags":256,"name":"make_persistent","kind":1,"startTimeUnixNano":"1729703314820820604","endTimeUnixNano":"1729703314823403018","events":[{"timeUnixNano":"1729703314820870177","name":"Receiving (1) objects to make persistent"},{"timeUnixNano":"1729703314821891687","name":"(94bce497-d159-44ee-89f4-3a79b630d2c6) Registering Person"}],"status":{}},{"traceId":"d56a60c6714d742006c5a4a35211fa31","spanId":"8d482d43a57860fe","parentSpanId":"","flags":256,"name":"call_active_method","kind":1,"startTimeUnixNano":"1729703314825940968","endTimeUnixNano":"1729703314842535069","events":[{"timeUnixNano":"1729703314825993076","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Receiving remote call to activemethod 'add'"},{"timeUnixNano":"1729703314829392566","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) *** Starting activemethod 'add' in executor"},{"timeUnixNano":"1729703314829539122","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Max threads for activemethod: None"},{"timeUnixNano":"1729703314840958897","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Running activemethod in new thread"},{"timeUnixNano":"1729703314842371021","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) *** Finished activemethod 'add' in executor"}],"status":{}},{"traceId":"e919a10b7bb7f319694de160cfcde3fa","spanId":"0482ccf066a81b2d","parentSpanId":"","flags":256,"name":"call_active_method","kind":1,"startTimeUnixNano":"1729703314846302321","endTimeUnixNano":"1729703314850523355","events":[{"timeUnixNano":"1729703314846355020","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Receiving remote call to activemethod '__str__'"},{"timeUnixNano":"1729703314847907027","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) *** Starting activemethod '__str__' in executor"},{"timeUnixNano":"1729703314848023716","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Max threads for activemethod: None"},{"timeUnixNano":"1729703314848954786","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Running activemethod in new thread"},{"timeUnixNano":"1729703314850019948","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) *** Finished activemethod '__str__' in executor"}],"status":{}}]},{"scope":{"name":"dataclay.dataclay_object"},"spans":[{"traceId":"d56a60c6714d742006c5a4a35211fa31","spanId":"06d98683a939f905","parentSpanId":"8d482d43a57860fe","flags":256,"name":"get_by_id","kind":1,"startTimeUnixNano":"1729703314828500900","endTimeUnixNano":"1729703314828791837","status":{}}]}]},{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"metadata"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.48b0"},"spans":[{"traceId":"072e983ab2bde29cd3d4591e539c485f","spanId":"7a9d511cc65cacc0","parentSpanId":"6e95d9b388fd0b31","flags":256,"name":"GET","kind":3,"startTimeUnixNano":"1729703314785494643","endTimeUnixNano":"1729703314787154443","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}},{"traceId":"07ae1f7ae4d3e52066be188a12fbd4f1","spanId":"79fd793d96542494","parentSpanId":"bbc73e2bce0f87b9","flags":256,"name":"SET","kind":3,"startTimeUnixNano":"1729703314800399069","endTimeUnixNano":"1729703314801870454","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"072e983ab2bde29cd3d4591e539c485f","spanId":"6e95d9b388fd0b31","parentSpanId":"","flags":256,"name":"get_object_md_by_alias","kind":1,"startTimeUnixNano":"1729703314785055759","endTimeUnixNano":"1729703314790057500","events":[{"timeUnixNano":"1729703314789958233","name":"exception","attributes":[{"key":"exception.type","value":{"stringValue":"dataclay.exceptions.DoesNotExistError"}},{"key":"exception.message","value":{"stringValue":"/alias/admin/myfamily does not exist"}},{"key":"exception.stacktrace","value":{"stringValue":"Traceback (most recent call last):\n File \"/usr/local/lib/python3.10/site-packages/opentelemetry/trace/__init__.py\", line 590, in use_span\n yield span\n File \"/usr/local/lib/python3.10/site-packages/opentelemetry/sdk/trace/__init__.py\", line 1108, in start_as_current_span\n yield span\n File \"/usr/local/lib/python3.10/site-packages/opentelemetry/trace/__init__.py\", line 453, in start_as_current_span\n yield span\n File \"/usr/local/lib/python3.10/site-packages/opentelemetry/util/_decorator.py\", line 68, in async_wrapper\n return await func(*args, **kwargs) # type: ignore\n File \"/app/src/dataclay/metadata/api.py\", line 232, in get_object_md_by_alias\n alias = await self.kv_manager.get_kv(Alias, f\"{dataset_name}/{alias_name}\")\n File \"/app/src/dataclay/metadata/redismanager.py\", line 99, in get_kv\n raise DoesNotExistError(name)\ndataclay.exceptions.DoesNotExistError: /alias/admin/myfamily does not exist\n"}},{"key":"exception.escaped","value":{"stringValue":"False"}}]}],"status":{"message":"DoesNotExistError: /alias/admin/myfamily does not exist","code":2}},{"traceId":"07ae1f7ae4d3e52066be188a12fbd4f1","spanId":"bbc73e2bce0f87b9","parentSpanId":"","flags":256,"name":"new_alias","kind":1,"startTimeUnixNano":"1729703314799870485","endTimeUnixNano":"1729703314801942209","status":{}}]}]}]} +{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"client"}}]},"scopeSpans":[{"scope":{"name":"dataclay.client.api"},"spans":[{"traceId":"786160991829efc49564028b1cd04926","spanId":"561b14a5663f2597","parentSpanId":"","flags":256,"name":"start","kind":1,"startTimeUnixNano":"1729703319269841486","endTimeUnixNano":"1729703319274614056","status":{}}]},{"scope":{"name":"dataclay.dataclay_object"},"spans":[{"traceId":"1eb45111bae8f6c4ba82943d531ab84b","spanId":"032956dc292634c1","parentSpanId":"","flags":256,"name":"get_by_alias","kind":1,"startTimeUnixNano":"1729703319275284857","endTimeUnixNano":"1729703319282569840","status":{}},{"traceId":"b7a2b42e9cd119427e44e6c7c3c769d3","spanId":"4fba5d2497850682","parentSpanId":"2e9cc5359e9674e5","flags":256,"name":"make_persistent","kind":1,"startTimeUnixNano":"1729703319285667693","endTimeUnixNano":"1729703319294233463","status":{}}]},{"scope":{"name":"dataclay.runtime"},"spans":[{"traceId":"b7a2b42e9cd119427e44e6c7c3c769d3","spanId":"2e9cc5359e9674e5","parentSpanId":"","flags":256,"name":"call_remote_method","kind":1,"startTimeUnixNano":"1729703319283591310","endTimeUnixNano":"1729703319302249871","attributes":[{"key":"class","value":{"stringValue":"dataclay.contrib.modeltest.family.Family"}},{"key":"method","value":{"stringValue":"add"}},{"key":"args","value":{"stringValue":"(\u003cdataclay.contrib.modeltest.family.Person volatile instance with ObjectID=8d097e19-8825-4503-ae84-a8c8733c5b41\u003e,)"}},{"key":"kwargs","value":{"stringValue":"{}"}}],"status":{}},{"traceId":"2638bf6083c7c38a7facbab6140b46ac","spanId":"54c3d6cf8e18751e","parentSpanId":"","flags":256,"name":"call_remote_method","kind":1,"startTimeUnixNano":"1729703319302843637","endTimeUnixNano":"1729703319310883379","attributes":[{"key":"class","value":{"stringValue":"dataclay.contrib.modeltest.family.Family"}},{"key":"method","value":{"stringValue":"__str__"}},{"key":"args","value":{"stringValue":"()"}},{"key":"kwargs","value":{"stringValue":"{}"}}],"status":{}}]}]}]} +{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"backend"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.48b0"},"spans":[{"traceId":"91240396353ccc05ffc355cec45737ee","spanId":"6af52a00f435676a","parentSpanId":"89ff73591e7bcf02","flags":256,"name":"SCAN","kind":3,"startTimeUnixNano":"1729703319052991442","endTimeUnixNano":"1729703319054016158","attributes":[{"key":"db.statement","value":{"stringValue":"SCAN ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"91240396353ccc05ffc355cec45737ee","spanId":"370852c0013176e9","parentSpanId":"89ff73591e7bcf02","flags":256,"name":"GET","kind":3,"startTimeUnixNano":"1729703319054171099","endTimeUnixNano":"1729703319054717647","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}},{"traceId":"ad8e03196bbb87500011782cb5b10f89","spanId":"92f6254e1e46cdf2","parentSpanId":"5bd2a74409887b4c","flags":256,"name":"SET","kind":3,"startTimeUnixNano":"1729703319292233865","endTimeUnixNano":"1729703319293199620","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"3"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"91240396353ccc05ffc355cec45737ee","spanId":"89ff73591e7bcf02","parentSpanId":"","flags":256,"name":"get_all_backends","kind":1,"startTimeUnixNano":"1729703319052676801","endTimeUnixNano":"1729703319054878840","status":{}},{"traceId":"ad8e03196bbb87500011782cb5b10f89","spanId":"5bd2a74409887b4c","parentSpanId":"b6e5225bc2f59f1b","flags":256,"name":"upsert_object","kind":1,"startTimeUnixNano":"1729703319291674553","endTimeUnixNano":"1729703319293246408","status":{}}]},{"scope":{"name":"dataclay.backend.api"},"spans":[{"traceId":"ad8e03196bbb87500011782cb5b10f89","spanId":"b6e5225bc2f59f1b","parentSpanId":"","flags":256,"name":"make_persistent","kind":1,"startTimeUnixNano":"1729703319289391562","endTimeUnixNano":"1729703319293271796","events":[{"timeUnixNano":"1729703319289464469","name":"Receiving (1) objects to make persistent"},{"timeUnixNano":"1729703319291201825","name":"(8d097e19-8825-4503-ae84-a8c8733c5b41) Registering Person"}],"status":{}},{"traceId":"62e42757154c8c9f35e9072282dacf34","spanId":"f1c322fa4c1aaaf0","parentSpanId":"","flags":256,"name":"call_active_method","kind":1,"startTimeUnixNano":"1729703319296157219","endTimeUnixNano":"1729703319301390516","events":[{"timeUnixNano":"1729703319296208756","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Receiving remote call to activemethod 'add'"},{"timeUnixNano":"1729703319298767686","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) *** Starting activemethod 'add' in executor"},{"timeUnixNano":"1729703319298947674","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Max threads for activemethod: None"},{"timeUnixNano":"1729703319300257336","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Running activemethod in new thread"},{"timeUnixNano":"1729703319301252196","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) *** Finished activemethod 'add' in executor"}],"status":{}},{"traceId":"c3dbc407e26e568ccc8c26872d75958c","spanId":"b4f90546d76c7a08","parentSpanId":"","flags":256,"name":"call_active_method","kind":1,"startTimeUnixNano":"1729703319305100920","endTimeUnixNano":"1729703319310101078","events":[{"timeUnixNano":"1729703319305186120","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Receiving remote call to activemethod '__str__'"},{"timeUnixNano":"1729703319306909118","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) *** Starting activemethod '__str__' in executor"},{"timeUnixNano":"1729703319307029415","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Max threads for activemethod: None"},{"timeUnixNano":"1729703319308013084","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) Running activemethod in new thread"},{"timeUnixNano":"1729703319309545524","name":"(e1a380d4-c421-46e4-a175-e5a968de541a) *** Finished activemethod '__str__' in executor"}],"status":{}}]},{"scope":{"name":"dataclay.dataclay_object"},"spans":[{"traceId":"62e42757154c8c9f35e9072282dacf34","spanId":"122b4b57181e6396","parentSpanId":"f1c322fa4c1aaaf0","flags":256,"name":"get_by_id","kind":1,"startTimeUnixNano":"1729703319297994623","endTimeUnixNano":"1729703319298194478","status":{}}]}]},{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"metadata"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.48b0"},"spans":[{"traceId":"8af6d4a0861f9bd5d8659e579bbff159","spanId":"e9223014740c30fc","parentSpanId":"ab33c0b1ad5a66e6","flags":256,"name":"GET","kind":3,"startTimeUnixNano":"1729703319277959044","endTimeUnixNano":"1729703319279861409","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}},{"traceId":"8af6d4a0861f9bd5d8659e579bbff159","spanId":"76c984916e03324a","parentSpanId":"ab33c0b1ad5a66e6","flags":256,"name":"GET","kind":3,"startTimeUnixNano":"1729703319280229842","endTimeUnixNano":"1729703319281062697","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}},{"traceId":"74fffb21ca1e7c339872f73551bdae20","spanId":"cbe14012c5e949f0","parentSpanId":"1c51791fe4d48755","flags":256,"name":"SCAN","kind":3,"startTimeUnixNano":"1729703319704346766","endTimeUnixNano":"1729703319705414653","attributes":[{"key":"db.statement","value":{"stringValue":"SCAN ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"74fffb21ca1e7c339872f73551bdae20","spanId":"fa8e397804f03fa4","parentSpanId":"1c51791fe4d48755","flags":256,"name":"GET","kind":3,"startTimeUnixNano":"1729703319705616833","endTimeUnixNano":"1729703319706242079","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"8af6d4a0861f9bd5d8659e579bbff159","spanId":"ab33c0b1ad5a66e6","parentSpanId":"","flags":256,"name":"get_object_md_by_alias","kind":1,"startTimeUnixNano":"1729703319277430670","endTimeUnixNano":"1729703319281208682","status":{}},{"traceId":"74fffb21ca1e7c339872f73551bdae20","spanId":"1c51791fe4d48755","parentSpanId":"","flags":256,"name":"get_all_backends","kind":1,"startTimeUnixNano":"1729703319703993112","endTimeUnixNano":"1729703319706418300","status":{}}]}]}]} +{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"backend"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.48b0"},"spans":[{"traceId":"3bdf7e49fbe56be80d056b39efd5e18d","spanId":"6992104b31715490","parentSpanId":"d4858ed2de900d86","flags":256,"name":"DEL","kind":3,"startTimeUnixNano":"1729703324766352673","endTimeUnixNano":"1729703324769984129","attributes":[{"key":"db.statement","value":{"stringValue":"DEL ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}},{"traceId":"3bdf7e49fbe56be80d056b39efd5e18d","spanId":"2a6cc2c14914bb78","parentSpanId":"d4858ed2de900d86","flags":256,"name":"PUBLISH","kind":3,"startTimeUnixNano":"1729703324770207078","endTimeUnixNano":"1729703324770947860","attributes":[{"key":"db.statement","value":{"stringValue":"PUBLISH ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"3"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"3bdf7e49fbe56be80d056b39efd5e18d","spanId":"d4858ed2de900d86","parentSpanId":"63db1b6bdb62135a","flags":256,"name":"delete_backend","kind":1,"startTimeUnixNano":"1729703324765975514","endTimeUnixNano":"1729703324770981824","status":{}}]},{"scope":{"name":"dataclay.backend.api"},"spans":[{"traceId":"3bdf7e49fbe56be80d056b39efd5e18d","spanId":"63db1b6bdb62135a","parentSpanId":"","flags":256,"name":"stop","kind":1,"startTimeUnixNano":"1729703324764528855","endTimeUnixNano":"1729703324774618519","status":{}}]}]}]} diff --git a/examples/tracing/README.md b/examples/tracing/README.md deleted file mode 100644 index 99b5fbd..0000000 --- a/examples/tracing/README.md +++ /dev/null @@ -1,2 +0,0 @@ - -http://localhost:3000 \ No newline at end of file diff --git a/examples/tracing/json-exporter/README.md b/examples/tracing/json-exporter/README.md deleted file mode 100644 index 33377bb..0000000 --- a/examples/tracing/json-exporter/README.md +++ /dev/null @@ -1,36 +0,0 @@ - -# JSON Exporter - -This example demonstrates how to export opentelemetry traces to a JSON file. The JSON file is created in the `traces` folder. - -To activate traces in dataClay, you need to set the following environment variables: - -- `DATACLAY_TRACING`: Set to `true` to enable tracing. -- `DATACLAY_TRACING_EXPORTER`: Set to `otlp` to export traces to the OpenTelemetry Collector, or `console` to print traces to the console (default is `otlp`). -- `DATACLAY_TRACING_HOST`: The host of the OpenTelemetry Collector (default is `localhost`). -- `DATACLAY_TRACING_PORT`: The port of the OpenTelemetry Collector (default is 4317). -- `DATACLAY_SERVICE_NAME`: The service name. - -## Running the Example - -Start the dataClay services: - -```bash -docker compose up -``` - -Run the example: - -```bash -python3 client.py -``` - -The traces are exported to the `traces` folder. You can view the traces in the JSON file. - -## Troubleshooting - -If you get this error `open /traces/otel-traces.json: permission denied`, then you need to give the permission to the `traces` folder. - -```bash -sudo chmod -R 777 traces -``` diff --git a/examples/tracing/json-exporter/traces/otel-traces.json b/examples/tracing/json-exporter/traces/otel-traces.json deleted file mode 100755 index d82fc7e..0000000 --- a/examples/tracing/json-exporter/traces/otel-traces.json +++ /dev/null @@ -1,2 +0,0 @@ -{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"metadata"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.42b0"},"spans":[{"traceId":"d6715bdef218fdec436b1489bacbdcf7","spanId":"a867ad93a8f964e5","parentSpanId":"","name":"INFO","kind":3,"startTimeUnixNano":"1701705230134918507","endTimeUnixNano":"1701705230137576707","attributes":[{"key":"db.statement","value":{"stringValue":"INFO"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"528e9fae31980eede35b6bd93fb50d5f","spanId":"fc99d298c2d5f6ff","parentSpanId":"","name":"PING","kind":3,"startTimeUnixNano":"1701705230138161136","endTimeUnixNano":"1701705230138958869","attributes":[{"key":"db.statement","value":{"stringValue":"PING"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"fa61440cec3f5c5ff52343bee7e4bb36","spanId":"3dbd44b356a477df","parentSpanId":"73cefd78ac0eda25","name":"SET","kind":3,"startTimeUnixNano":"1701705230139336268","endTimeUnixNano":"1701705230139593014","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"f633e3c92c4c21e34d99735d74092b4d","spanId":"e02c6edccfd2de13","parentSpanId":"79ba9fa5f87432bc","name":"SET","kind":3,"startTimeUnixNano":"1701705230361428916","endTimeUnixNano":"1701705230361688669","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"f633e3c92c4c21e34d99735d74092b4d","spanId":"5e859d81cba3506b","parentSpanId":"79ba9fa5f87432bc","name":"SET","kind":3,"startTimeUnixNano":"1701705230361793358","endTimeUnixNano":"1701705230361937425","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"fa61440cec3f5c5ff52343bee7e4bb36","spanId":"73cefd78ac0eda25","parentSpanId":"","name":"new_dataclay","kind":1,"startTimeUnixNano":"1701705230139181747","endTimeUnixNano":"1701705230139713062","status":{}},{"traceId":"f633e3c92c4c21e34d99735d74092b4d","spanId":"79ba9fa5f87432bc","parentSpanId":"","name":"new_superuser","kind":1,"startTimeUnixNano":"1701705230139818981","endTimeUnixNano":"1701705230362038240","status":{}}]}]},{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"backend"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.42b0"},"spans":[{"traceId":"ef3002c053e40efa28c670b37d99455d","spanId":"467dabe5b38eff20","parentSpanId":"","name":"INFO","kind":3,"startTimeUnixNano":"1701705230159321426","endTimeUnixNano":"1701705230161034869","attributes":[{"key":"db.statement","value":{"stringValue":"INFO"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"71c9585dd43b6600ed9ee1a70f954f19","spanId":"c03ef11edd8923c8","parentSpanId":"","name":"PING","kind":3,"startTimeUnixNano":"1701705230162612930","endTimeUnixNano":"1701705230163856352","attributes":[{"key":"db.statement","value":{"stringValue":"PING"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"4562d1b35c51807943c1f0014df73b92","spanId":"e2522f18da7e4efa","parentSpanId":"3d2488f8de2781b2","name":"GET","kind":3,"startTimeUnixNano":"1701705230164088832","endTimeUnixNano":"1701705230164234165","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}},{"traceId":"50610c8deb1d3c9f5c4b40e2032746ed","spanId":"1170a8073b8a01bb","parentSpanId":"da8b811748309a86","name":"SET","kind":3,"startTimeUnixNano":"1701705230171940023","endTimeUnixNano":"1701705230172429830","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"4562d1b35c51807943c1f0014df73b92","spanId":"3d2488f8de2781b2","parentSpanId":"","name":"get_dataclay","kind":1,"startTimeUnixNano":"1701705230164003729","endTimeUnixNano":"1701705230164298344","status":{}},{"traceId":"50610c8deb1d3c9f5c4b40e2032746ed","spanId":"da8b811748309a86","parentSpanId":"","name":"register_backend","kind":1,"startTimeUnixNano":"1701705230171532110","endTimeUnixNano":"1701705230172596891","status":{}}]}]}]} -{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"backend"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.42b0"},"spans":[{"traceId":"98c6cce51ed1b2a59367188fbe94a740","spanId":"23cb3724e2018dea","parentSpanId":"d2fee9c6e4a6c00c","name":"DEL","kind":3,"startTimeUnixNano":"1701705239428413237","endTimeUnixNano":"1701705239429024094","attributes":[{"key":"db.statement","value":{"stringValue":"DEL ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"98c6cce51ed1b2a59367188fbe94a740","spanId":"d2fee9c6e4a6c00c","parentSpanId":"53693e050b609991","name":"delete_backend","kind":1,"startTimeUnixNano":"1701705239428192337","endTimeUnixNano":"1701705239429087929","status":{}}]},{"scope":{"name":"dataclay.backend.api"},"spans":[{"traceId":"98c6cce51ed1b2a59367188fbe94a740","spanId":"53693e050b609991","parentSpanId":"","name":"stop","kind":1,"startTimeUnixNano":"1701705239428022353","endTimeUnixNano":"1701705239429391516","status":{}}]}]}]} diff --git a/examples/tracing/json-post-mortem/README.md b/examples/tracing/json-post-mortem/README.md deleted file mode 100644 index 81d94e2..0000000 --- a/examples/tracing/json-post-mortem/README.md +++ /dev/null @@ -1,12 +0,0 @@ - - - - -```bash -docker run \ --v ./config/otel-collector.yaml:/etc/otel-collector.yaml \ -otel/opentelemetry-collector-contrib \ -"--config=/etc/otel-collector.yaml" - - -``` diff --git a/examples/tracing/json-post-mortem/traces/.gitkeep b/examples/tracing/json-post-mortem/traces/.gitkeep deleted file mode 100644 index e69de29..0000000 diff --git a/examples/tracing/json-post-mortem/traces/otel-traces-old.json b/examples/tracing/json-post-mortem/traces/otel-traces-old.json deleted file mode 100755 index 059411b..0000000 --- a/examples/tracing/json-post-mortem/traces/otel-traces-old.json +++ /dev/null @@ -1,2 +0,0 @@ -{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"metadata"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.42b0"},"spans":[{"traceId":"0d7d32349646e7e446c1e6b3aeb84397","spanId":"3e1b90a1100b469b","parentSpanId":"","name":"INFO","kind":3,"startTimeUnixNano":"1701702107608868158","endTimeUnixNano":"1701702107611191198","attributes":[{"key":"db.statement","value":{"stringValue":"INFO"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"5e0c5a0287b80387612487dcac7a1a14","spanId":"95651c442f05f747","parentSpanId":"","name":"PING","kind":3,"startTimeUnixNano":"1701702107611608967","endTimeUnixNano":"1701702107612357331","attributes":[{"key":"db.statement","value":{"stringValue":"PING"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"d446414ba5fbda51be3e76d33e66224f","spanId":"0297192305bf3f9d","parentSpanId":"2a3c79f09785574b","name":"SET","kind":3,"startTimeUnixNano":"1701702107612701909","endTimeUnixNano":"1701702107612884835","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"b83161b3413927f9b89b5c2780f1798a","spanId":"9cad8967c1988f4b","parentSpanId":"492ac84b84ee99f6","name":"SET","kind":3,"startTimeUnixNano":"1701702107833636472","endTimeUnixNano":"1701702107833891851","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"b83161b3413927f9b89b5c2780f1798a","spanId":"1a5e3b8fe3f2b383","parentSpanId":"492ac84b84ee99f6","name":"SET","kind":3,"startTimeUnixNano":"1701702107834050588","endTimeUnixNano":"1701702107834219790","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"d446414ba5fbda51be3e76d33e66224f","spanId":"2a3c79f09785574b","parentSpanId":"","name":"new_dataclay","kind":1,"startTimeUnixNano":"1701702107612537079","endTimeUnixNano":"1701702107612973402","status":{}},{"traceId":"b83161b3413927f9b89b5c2780f1798a","spanId":"492ac84b84ee99f6","parentSpanId":"","name":"new_superuser","kind":1,"startTimeUnixNano":"1701702107613057486","endTimeUnixNano":"1701702107834316885","status":{}}]}]},{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"backend"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.42b0"},"spans":[{"traceId":"625cec5e4d34b378e7cf77fab23fd623","spanId":"c308029557e80b85","parentSpanId":"","name":"INFO","kind":3,"startTimeUnixNano":"1701702107608885081","endTimeUnixNano":"1701702107611454519","attributes":[{"key":"db.statement","value":{"stringValue":"INFO"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"37021309713cf27d1467fdb65da03d0f","spanId":"b1f2c20efc9b9e4f","parentSpanId":"","name":"PING","kind":3,"startTimeUnixNano":"1701702107612378719","endTimeUnixNano":"1701702107613112022","attributes":[{"key":"db.statement","value":{"stringValue":"PING"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"936fff1f2b02f5657702e7ed61803b56","spanId":"09d53858affa37b4","parentSpanId":"2d845f404cccc668","name":"GET","kind":3,"startTimeUnixNano":"1701702107613508289","endTimeUnixNano":"1701702107613682411","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}},{"traceId":"4c5070e5801a6b9d970685a6a8656d17","spanId":"2a400af3e516d80d","parentSpanId":"c51efb7eeadd78f7","name":"SET","kind":3,"startTimeUnixNano":"1701702107622116955","endTimeUnixNano":"1701702107622347811","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"936fff1f2b02f5657702e7ed61803b56","spanId":"2d845f404cccc668","parentSpanId":"","name":"get_dataclay","kind":1,"startTimeUnixNano":"1701702107613426049","endTimeUnixNano":"1701702107613763886","status":{}},{"traceId":"4c5070e5801a6b9d970685a6a8656d17","spanId":"c51efb7eeadd78f7","parentSpanId":"","name":"register_backend","kind":1,"startTimeUnixNano":"1701702107621963058","endTimeUnixNano":"1701702107622428696","status":{}}]}]}]} -{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"backend"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.42b0"},"spans":[{"traceId":"4882fce88c122fe0703713f4a0bde3f7","spanId":"52a08026f99d4c95","parentSpanId":"b69f9222075cbbca","name":"DEL","kind":3,"startTimeUnixNano":"1701702140275583280","endTimeUnixNano":"1701702140276046581","attributes":[{"key":"db.statement","value":{"stringValue":"DEL ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"4882fce88c122fe0703713f4a0bde3f7","spanId":"b69f9222075cbbca","parentSpanId":"8879d66b5e83274b","name":"delete_backend","kind":1,"startTimeUnixNano":"1701702140275378586","endTimeUnixNano":"1701702140276098129","status":{}}]},{"scope":{"name":"dataclay.backend.api"},"spans":[{"traceId":"4882fce88c122fe0703713f4a0bde3f7","spanId":"8879d66b5e83274b","parentSpanId":"","name":"stop","kind":1,"startTimeUnixNano":"1701702140275192259","endTimeUnixNano":"1701702140276317820","status":{}}]}]}]} diff --git a/examples/tracing/json-post-mortem/traces/otel-traces.json b/examples/tracing/json-post-mortem/traces/otel-traces.json deleted file mode 100755 index d82fc7e..0000000 --- a/examples/tracing/json-post-mortem/traces/otel-traces.json +++ /dev/null @@ -1,2 +0,0 @@ -{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"metadata"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.42b0"},"spans":[{"traceId":"d6715bdef218fdec436b1489bacbdcf7","spanId":"a867ad93a8f964e5","parentSpanId":"","name":"INFO","kind":3,"startTimeUnixNano":"1701705230134918507","endTimeUnixNano":"1701705230137576707","attributes":[{"key":"db.statement","value":{"stringValue":"INFO"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"528e9fae31980eede35b6bd93fb50d5f","spanId":"fc99d298c2d5f6ff","parentSpanId":"","name":"PING","kind":3,"startTimeUnixNano":"1701705230138161136","endTimeUnixNano":"1701705230138958869","attributes":[{"key":"db.statement","value":{"stringValue":"PING"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"fa61440cec3f5c5ff52343bee7e4bb36","spanId":"3dbd44b356a477df","parentSpanId":"73cefd78ac0eda25","name":"SET","kind":3,"startTimeUnixNano":"1701705230139336268","endTimeUnixNano":"1701705230139593014","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"f633e3c92c4c21e34d99735d74092b4d","spanId":"e02c6edccfd2de13","parentSpanId":"79ba9fa5f87432bc","name":"SET","kind":3,"startTimeUnixNano":"1701705230361428916","endTimeUnixNano":"1701705230361688669","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}},{"traceId":"f633e3c92c4c21e34d99735d74092b4d","spanId":"5e859d81cba3506b","parentSpanId":"79ba9fa5f87432bc","name":"SET","kind":3,"startTimeUnixNano":"1701705230361793358","endTimeUnixNano":"1701705230361937425","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"fa61440cec3f5c5ff52343bee7e4bb36","spanId":"73cefd78ac0eda25","parentSpanId":"","name":"new_dataclay","kind":1,"startTimeUnixNano":"1701705230139181747","endTimeUnixNano":"1701705230139713062","status":{}},{"traceId":"f633e3c92c4c21e34d99735d74092b4d","spanId":"79ba9fa5f87432bc","parentSpanId":"","name":"new_superuser","kind":1,"startTimeUnixNano":"1701705230139818981","endTimeUnixNano":"1701705230362038240","status":{}}]}]},{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"backend"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.42b0"},"spans":[{"traceId":"ef3002c053e40efa28c670b37d99455d","spanId":"467dabe5b38eff20","parentSpanId":"","name":"INFO","kind":3,"startTimeUnixNano":"1701705230159321426","endTimeUnixNano":"1701705230161034869","attributes":[{"key":"db.statement","value":{"stringValue":"INFO"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"71c9585dd43b6600ed9ee1a70f954f19","spanId":"c03ef11edd8923c8","parentSpanId":"","name":"PING","kind":3,"startTimeUnixNano":"1701705230162612930","endTimeUnixNano":"1701705230163856352","attributes":[{"key":"db.statement","value":{"stringValue":"PING"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"1"}}],"status":{}},{"traceId":"4562d1b35c51807943c1f0014df73b92","spanId":"e2522f18da7e4efa","parentSpanId":"3d2488f8de2781b2","name":"GET","kind":3,"startTimeUnixNano":"1701705230164088832","endTimeUnixNano":"1701705230164234165","attributes":[{"key":"db.statement","value":{"stringValue":"GET ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}},{"traceId":"50610c8deb1d3c9f5c4b40e2032746ed","spanId":"1170a8073b8a01bb","parentSpanId":"da8b811748309a86","name":"SET","kind":3,"startTimeUnixNano":"1701705230171940023","endTimeUnixNano":"1701705230172429830","attributes":[{"key":"db.statement","value":{"stringValue":"SET ? ? ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"4"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"4562d1b35c51807943c1f0014df73b92","spanId":"3d2488f8de2781b2","parentSpanId":"","name":"get_dataclay","kind":1,"startTimeUnixNano":"1701705230164003729","endTimeUnixNano":"1701705230164298344","status":{}},{"traceId":"50610c8deb1d3c9f5c4b40e2032746ed","spanId":"da8b811748309a86","parentSpanId":"","name":"register_backend","kind":1,"startTimeUnixNano":"1701705230171532110","endTimeUnixNano":"1701705230172596891","status":{}}]}]}]} -{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"backend"}}]},"scopeSpans":[{"scope":{"name":"opentelemetry.instrumentation.redis","version":"0.42b0"},"spans":[{"traceId":"98c6cce51ed1b2a59367188fbe94a740","spanId":"23cb3724e2018dea","parentSpanId":"d2fee9c6e4a6c00c","name":"DEL","kind":3,"startTimeUnixNano":"1701705239428413237","endTimeUnixNano":"1701705239429024094","attributes":[{"key":"db.statement","value":{"stringValue":"DEL ?"}},{"key":"db.system","value":{"stringValue":"redis"}},{"key":"db.redis.database_index","value":{"intValue":"0"}},{"key":"net.peer.name","value":{"stringValue":"redis"}},{"key":"net.peer.port","value":{"intValue":"6379"}},{"key":"net.transport","value":{"stringValue":"ip_tcp"}},{"key":"db.redis.args_length","value":{"intValue":"2"}}],"status":{}}]},{"scope":{"name":"dataclay.metadata.api"},"spans":[{"traceId":"98c6cce51ed1b2a59367188fbe94a740","spanId":"d2fee9c6e4a6c00c","parentSpanId":"53693e050b609991","name":"delete_backend","kind":1,"startTimeUnixNano":"1701705239428192337","endTimeUnixNano":"1701705239429087929","status":{}}]},{"scope":{"name":"dataclay.backend.api"},"spans":[{"traceId":"98c6cce51ed1b2a59367188fbe94a740","spanId":"53693e050b609991","parentSpanId":"","name":"stop","kind":1,"startTimeUnixNano":"1701705239428022353","endTimeUnixNano":"1701705239429391516","status":{}}]}]}]} From 927df0dc4c4f53cf658475fe676b4ea9350e0486 Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Thu, 24 Oct 2024 16:20:13 +0800 Subject: [PATCH 15/24] Add tracing real-time example to telemetry --- examples/grafana-tempo/.gitignore | 1 - examples/grafana-tempo/client.py | 18 --- examples/grafana-tempo/dashboard.json | 144 ------------------ examples/telemetry/real-time/.env | 1 + examples/telemetry/real-time/client.py | 19 +++ .../config/grafana-datasources.yaml | 2 + .../real-time}/config/prometheus.yaml | 0 .../real-time}/config/tempo.yaml | 25 ++- .../real-time}/docker-compose.yaml | 26 ++-- .../real-time/otel-collector-config.yaml} | 3 + 10 files changed, 63 insertions(+), 176 deletions(-) delete mode 100644 examples/grafana-tempo/.gitignore delete mode 100644 examples/grafana-tempo/client.py delete mode 100644 examples/grafana-tempo/dashboard.json create mode 100644 examples/telemetry/real-time/.env create mode 100644 examples/telemetry/real-time/client.py rename examples/{grafana-tempo => telemetry/real-time}/config/grafana-datasources.yaml (92%) rename examples/{grafana-tempo => telemetry/real-time}/config/prometheus.yaml (100%) rename examples/{grafana-tempo => telemetry/real-time}/config/tempo.yaml (66%) rename examples/{grafana-tempo => telemetry/real-time}/docker-compose.yaml (78%) rename examples/{grafana-tempo/otel-collector.yaml => telemetry/real-time/otel-collector-config.yaml} (85%) diff --git a/examples/grafana-tempo/.gitignore b/examples/grafana-tempo/.gitignore deleted file mode 100644 index 5229dfe..0000000 --- a/examples/grafana-tempo/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/data/tempo/ diff --git a/examples/grafana-tempo/client.py b/examples/grafana-tempo/client.py deleted file mode 100644 index 8ae1955..0000000 --- a/examples/grafana-tempo/client.py +++ /dev/null @@ -1,18 +0,0 @@ -from dataclay import Client -from dataclay.config import settings -from dataclay.contrib.modeltest.family import Family, Person - -settings.tracing = True - -client = Client(host="127.0.0.1", username="admin", password="admin", dataset="admin") -client.start() - -family = Family() -family.make_persistent("Smith") - -person = Person("Marc", 24) -family.add(person) - -assert family is Family.get_by_alias("Smith") - -Family.delete_alias("Smith") diff --git a/examples/grafana-tempo/dashboard.json b/examples/grafana-tempo/dashboard.json deleted file mode 100644 index 5ed97f3..0000000 --- a/examples/grafana-tempo/dashboard.json +++ /dev/null @@ -1,144 +0,0 @@ -{ - "annotations": { - "list": [ - { - "builtIn": 1, - "datasource": { - "type": "grafana", - "uid": "-- Grafana --" - }, - "enable": true, - "hide": true, - "iconColor": "rgba(0, 211, 255, 1)", - "name": "Annotations & Alerts", - "target": { - "limit": 100, - "matchAny": false, - "tags": [], - "type": "dashboard" - }, - "type": "dashboard" - } - ] - }, - "editable": true, - "fiscalYearStartMonth": 0, - "graphTooltip": 0, - "id": 1, - "links": [], - "liveNow": false, - "panels": [ - { - "datasource": { - "type": "tempo", - "uid": "tempo" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "thresholds" - }, - "custom": { - "align": "auto", - "cellOptions": { - "type": "auto" - }, - "inspect": false - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - }, - { - "color": "red", - "value": 80 - } - ] - } - }, - "overrides": [] - }, - "gridPos": { - "h": 9, - "w": 24, - "x": 0, - "y": 0 - }, - "id": 2, - "options": { - "footer": { - "countRows": false, - "fields": "", - "reducer": [ - "sum" - ], - "show": false - }, - "showHeader": true - }, - "pluginVersion": "9.4.3", - "targets": [ - { - "datasource": { - "type": "tempo", - "uid": "tempo" - }, - "limit": 20, - "queryType": "nativeSearch", - "refId": "A" - } - ], - "title": "Panel Title", - "type": "table" - }, - { - "datasource": { - "type": "tempo", - "uid": "tempo" - }, - "gridPos": { - "h": 10, - "w": 24, - "x": 0, - "y": 9 - }, - "id": 3, - "pluginVersion": "9.4.3", - "targets": [ - { - "datasource": { - "type": "tempo", - "uid": "tempo" - }, - "limit": 20, - "queryType": "serviceMap", - "refId": "A" - } - ], - "title": "Panel Title", - "type": "nodeGraph" - } - ], - "refresh": "5s", - "revision": 1, - "schemaVersion": 38, - "style": "dark", - "tags": [], - "templating": { - "list": [] - }, - "time": { - "from": "now-6h", - "to": "now" - }, - "timepicker": {}, - "timezone": "", - "title": "dataClay", - "uid": "vLqPcbU4z", - "version": 5, - "weekStart": "" -} \ No newline at end of file diff --git a/examples/telemetry/real-time/.env b/examples/telemetry/real-time/.env new file mode 100644 index 0000000..32b1873 --- /dev/null +++ b/examples/telemetry/real-time/.env @@ -0,0 +1 @@ +DATACLAY_TRACING=true \ No newline at end of file diff --git a/examples/telemetry/real-time/client.py b/examples/telemetry/real-time/client.py new file mode 100644 index 0000000..37a81ff --- /dev/null +++ b/examples/telemetry/real-time/client.py @@ -0,0 +1,19 @@ +from dataclay import Client +from dataclay.config import settings +from dataclay.contrib.modeltest.family import Family, Person + +settings.tracing = True + +client = Client(host="127.0.0.1") +client.start() + +# family = Family() +# family.make_persistent("Smith") + +person = Person("Marc", 24) +person.make_persistent() +# family.add(person) + +# assert family is Family.get_by_alias("Smith") + +# Family.delete_alias("Smith") diff --git a/examples/grafana-tempo/config/grafana-datasources.yaml b/examples/telemetry/real-time/config/grafana-datasources.yaml similarity index 92% rename from examples/grafana-tempo/config/grafana-datasources.yaml rename to examples/telemetry/real-time/config/grafana-datasources.yaml index 84c370d..2329110 100644 --- a/examples/grafana-tempo/config/grafana-datasources.yaml +++ b/examples/telemetry/real-time/config/grafana-datasources.yaml @@ -28,3 +28,5 @@ datasources: httpMethod: GET serviceMap: datasourceUid: prometheus + streamingEnabled: + search: true diff --git a/examples/grafana-tempo/config/prometheus.yaml b/examples/telemetry/real-time/config/prometheus.yaml similarity index 100% rename from examples/grafana-tempo/config/prometheus.yaml rename to examples/telemetry/real-time/config/prometheus.yaml diff --git a/examples/grafana-tempo/config/tempo.yaml b/examples/telemetry/real-time/config/tempo.yaml similarity index 66% rename from examples/grafana-tempo/config/tempo.yaml rename to examples/telemetry/real-time/config/tempo.yaml index 49744ba..a47388f 100644 --- a/examples/grafana-tempo/config/tempo.yaml +++ b/examples/telemetry/real-time/config/tempo.yaml @@ -1,5 +1,17 @@ +stream_over_http_enabled: true server: http_listen_port: 3200 + log_level: info + +# query_frontend: +# search: +# duration_slo: 5s +# throughput_bytes_slo: 1.073741824e+09 +# metadata_slo: +# duration_slo: 5s +# throughput_bytes_slo: 1.073741824e+09 +# trace_by_id: +# duration_slo: 5s distributor: receivers: # this configuration will listen on all ports and protocols that tempo is capable of. @@ -29,18 +41,23 @@ metrics_generator: source: tempo cluster: docker-compose storage: - path: /tmp/tempo/generator/wal + path: /var/tempo/generator/wal remote_write: - url: http://prometheus:9090/api/v1/write send_exemplars: true + traces_storage: + path: /var/tempo/generator/traces storage: trace: backend: local # backend configuration to use wal: - path: /tmp/tempo/wal # where to store the the wal locally + path: /var/tempo/wal # where to store the wal locally local: - path: /tmp/tempo/blocks + path: /var/tempo/blocks overrides: - metrics_generator_processors: [service-graphs, span-metrics] # enables metrics generator + defaults: + metrics_generator: + processors: [service-graphs, span-metrics, local-blocks] # enables metrics generator + generate_native_histograms: both diff --git a/examples/grafana-tempo/docker-compose.yaml b/examples/telemetry/real-time/docker-compose.yaml similarity index 78% rename from examples/grafana-tempo/docker-compose.yaml rename to examples/telemetry/real-time/docker-compose.yaml index 13c7ad7..e34be26 100644 --- a/examples/grafana-tempo/docker-compose.yaml +++ b/examples/telemetry/real-time/docker-compose.yaml @@ -8,7 +8,7 @@ services: metadata-service: image: "ghcr.io/bsc-dom/dataclay:dev" build: - context: ../../ + context: ../../../ dockerfile: Dockerfile.dev depends_on: - redis @@ -26,7 +26,7 @@ services: - DATACLAY_TRACING_HOST=otel-collector command: python -m dataclay.metadata volumes: - - ../../:/app:ro + - ../../../:/app:ro backend: image: "ghcr.io/bsc-dom/dataclay:dev" @@ -46,15 +46,22 @@ services: - DATACLAY_TRACING_HOST=otel-collector command: python -m dataclay.backend volumes: - - ../../:/app:ro + - ../../../:/app:ro otel-collector: - image: otel/opentelemetry-collector:0.61.0 - command: [ "--config=/etc/otel-collector.yaml" ] - ports: - - 4317:4317 # otlp grpc + image: otel/opentelemetry-collector-contrib:latest volumes: - - ./otel-collector.yaml:/etc/otel-collector.yaml + - ./otel-collector-config.yaml:/etc/otelcol-contrib/config.yaml + ports: + - 1888:1888 # pprof extension + - 8888:8888 # Prometheus metrics exposed by the Collector + - 8889:8889 # Prometheus exporter metrics + - 13133:13133 # health_check extension + - 4317:4317 # OTLP gRPC receiver + - 4318:4318 # OTLP http receiver + - 55679:55679 # zpages extension + + tempo: image: grafana/tempo:latest @@ -75,13 +82,14 @@ services: - --config.file=/etc/prometheus.yaml - --web.enable-remote-write-receiver - --enable-feature=exemplar-storage + - --enable-feature=native-histograms volumes: - ./config/prometheus.yaml:/etc/prometheus.yaml ports: - "9090:9090" grafana: - image: grafana/grafana:9.4.3 + image: grafana/grafana:latest volumes: - ./config/grafana-datasources.yaml:/etc/grafana/provisioning/datasources/datasources.yaml # - ./data/grafana:/var/lib/grafana diff --git a/examples/grafana-tempo/otel-collector.yaml b/examples/telemetry/real-time/otel-collector-config.yaml similarity index 85% rename from examples/grafana-tempo/otel-collector.yaml rename to examples/telemetry/real-time/otel-collector-config.yaml index 558bc5d..c78400c 100644 --- a/examples/grafana-tempo/otel-collector.yaml +++ b/examples/telemetry/real-time/otel-collector-config.yaml @@ -2,11 +2,14 @@ receivers: otlp: protocols: grpc: + endpoint: "0.0.0.0:4317" + exporters: otlp: endpoint: tempo:4317 tls: insecure: true + service: pipelines: traces: From aa03df9b77d3c6038734c52e2fa69472c7d7c97f Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Thu, 24 Oct 2024 17:44:21 +0800 Subject: [PATCH 16/24] Refactor telemetry.py to use async gRPC instrumentation --- src/dataclay/utils/telemetry.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/dataclay/utils/telemetry.py b/src/dataclay/utils/telemetry.py index 168047b..4f042ab 100644 --- a/src/dataclay/utils/telemetry.py +++ b/src/dataclay/utils/telemetry.py @@ -69,14 +69,14 @@ def set_tracing(service_name, host, port, exporter="otlp"): trace.get_tracer_provider().add_span_processor(processor) if service_name == "client": - from opentelemetry.instrumentation.grpc import GrpcInstrumentorClient + from opentelemetry.instrumentation.grpc import GrpcAioInstrumentorClient - GrpcInstrumentorClient().instrument() + GrpcAioInstrumentorClient().instrument() else: - from opentelemetry.instrumentation.grpc import GrpcInstrumentorServer + from opentelemetry.instrumentation.grpc import GrpcAioInstrumentorServer from opentelemetry.instrumentation.redis import RedisInstrumentor - GrpcInstrumentorServer().instrument() + GrpcAioInstrumentorServer().instrument() RedisInstrumentor().instrument() logger.info("Tracer %s set for service %s", exporter, service_name) From ad45fe8c790cdb094a3420c46126cbb1948c283a Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Wed, 30 Oct 2024 21:59:08 +0800 Subject: [PATCH 17/24] Update MN5 deployment for job tracing --- examples/telemetry/offline/README.md | 12 ++++++++++++ examples/telemetry/offline/json-exporter/.env | 4 ++-- examples/telemetry/offline/json-exporter/job.sh | 4 ++-- orchestration/mn5/bin/dataclay_job_v1 | 1 + orchestration/mn5/bin/deploy_dataclay | 3 +++ orchestration/mn5/config/external_vars.yml | 14 +++++++++----- orchestration/mn5/config/otel-json-exporter.yaml | 4 ++++ src/dataclay/runtime.py | 2 +- src/dataclay/utils/backend_clients.py | 3 +++ 9 files changed, 37 insertions(+), 10 deletions(-) diff --git a/examples/telemetry/offline/README.md b/examples/telemetry/offline/README.md index d0be9ba..0b35e6a 100644 --- a/examples/telemetry/offline/README.md +++ b/examples/telemetry/offline/README.md @@ -60,6 +60,18 @@ otel/opentelemetry-collector-contrib \ "--config=/etc/otel-collector.yaml" ``` +## Copy traces from MareNostrum 5 + +You can copy traces from MareNostrum 5 to your local machine in order to visualize them in Grafana. + +Use `.dataclay/otel-traces.json` for the accumulated traces of several jobs, or `.dataclay//otel-traces.json` for the traces of a specific job. + +```bash +scp transfer1.bsc.es:~/.dataclay/otel-traces.json ./traces/otel-traces.json +``` + +Then, you can visualize the traces in Grafana, as explained above. + ## Troubleshooting If you get a `permission denied` error for the `/traces` folder, then you need to give the permission to the `traces` folder. diff --git a/examples/telemetry/offline/json-exporter/.env b/examples/telemetry/offline/json-exporter/.env index 5f66dd1..24739f3 100644 --- a/examples/telemetry/offline/json-exporter/.env +++ b/examples/telemetry/offline/json-exporter/.env @@ -1,2 +1,2 @@ -DATACLAY_TRACING=true -# DATACLAY_TRACING_HOST=$DATACLAY_METADATA_HOST +DATACLAY_LOGLEVEL=debug +DATACLAY_TRACING=true \ No newline at end of file diff --git a/examples/telemetry/offline/json-exporter/job.sh b/examples/telemetry/offline/json-exporter/job.sh index 30dc590..ed1a6e9 100644 --- a/examples/telemetry/offline/json-exporter/job.sh +++ b/examples/telemetry/offline/json-exporter/job.sh @@ -5,11 +5,11 @@ #SBATCH --nodes=2 #SBATCH --time=00:05:00 #SBATCH --exclusive -#SBATCH --qos=debug +#SBATCH --qos=gp_debug ############################# # Load dataClay -module load DATACLAY/edge +module load hdf5 python/3.12 dataclay/edge # Deploy dataClay and run client dataclay_job_v1 client.py diff --git a/orchestration/mn5/bin/dataclay_job_v1 b/orchestration/mn5/bin/dataclay_job_v1 index 27cf982..174a3f1 100755 --- a/orchestration/mn5/bin/dataclay_job_v1 +++ b/orchestration/mn5/bin/dataclay_job_v1 @@ -18,6 +18,7 @@ deploy_dataclay \ # Run client export DC_HOST=${hostnames[0]} # Used by client.py and ctl.stop_dataclay +export DATACLAY_TRACING_HOST=${hostnames[0]} # Used by client.py python3 -u $1 # Stop dataClay diff --git a/orchestration/mn5/bin/deploy_dataclay b/orchestration/mn5/bin/deploy_dataclay index eeed038..ec9f985 100755 --- a/orchestration/mn5/bin/deploy_dataclay +++ b/orchestration/mn5/bin/deploy_dataclay @@ -124,12 +124,15 @@ if __name__ == "__main__": # Add the dynamic environment variables if metadata_servers: env_vars["DATACLAY_METADATA_HOST"] = metadata_servers[0] + env_vars["DATACLAY_TRACING_HOST"] = metadata_servers[0] elif env_vars.get("DATACLAY_METADATA_HOST") is None: logger.warning("DATACLAY_METADATA_HOST not set.") + if redis_servers: env_vars["DATACLAY_KV_HOST"] = redis_servers[0] elif env_vars.get("DATACLAY_KV_HOST") is None: logger.warning("DATACLAY_KV_HOST not set.") + env_vars["PYTHONPATH"] = f"{os.environ.get('PYTHONPATH')}:{os.getcwd()}" # Create folers diff --git a/orchestration/mn5/config/external_vars.yml b/orchestration/mn5/config/external_vars.yml index 51fcda7..023425d 100644 --- a/orchestration/mn5/config/external_vars.yml +++ b/orchestration/mn5/config/external_vars.yml @@ -1,7 +1,6 @@ -# defaults are necessary here since not defined +# defaults are necessary here since not defined # environment variables will be replaced by empty strings, causing errors global_env: - # system properties PATH: "{{ lookup('env', 'PATH') }}" PYTHONPATH: "{{ lookup('env', 'PYTHONPATH') }}" @@ -11,11 +10,16 @@ global_env: # dataclay properties DATACLAY_LOGLEVEL: "{{ lookup('env', 'DATACLAY_LOGLEVEL') | default('WARNING', true) }}" - DATACLAY_TRACING: "{{ lookup('env', 'DATACLAY_TRACING') | default('false', true) }}" - DATACLAY_TRACING_HOST: "{{ lookup('env', 'DATACLAY_TRACING_HOST') }}" DATACLAY_JOB_PATH: "{{ job_path }}" # Used in otel-json-exporter.yaml + # TODO: Check if DATACLAY_IO_BOUND_MULTIPLIER is only used in backends. Is so.. move it. DATACLAY_IO_BOUND_MULTIPLIER: "{{ lookup('env', 'DATACLAY_IO_BOUND_MULTIPLIER') | default(2, true) }}" + # Tracing properties + DATACLAY_TRACING: "{{ lookup('env', 'DATACLAY_TRACING') | default('false', true) }}" + DATACLAY_TRACING_HOST: "{{ lookup('env', 'DATACLAY_TRACING_HOST') }}" + DATACLAY_TRACING_PORT: "{{ lookup('env', 'DATACLAY_TRACING_PORT') | default(4317, true) }}" + DATACLAY_TRACING_EXPORTER: "{{ lookup('env', 'DATACLAY_TRACING_EXPORTER') | default('otlp', true) }}" + backend_properties: DATACLAY_BACKEND_PORT: "{{ lookup('env', 'DATACLAY_BACKEND_PORT') | default(6867, true) }}" DATACLAY_KV_HOST: "{{ lookup('env', 'DATACLAY_KV_HOST') }}" @@ -49,4 +53,4 @@ redis_port: "{{ lookup('env', 'DATACLAY_KV_PORT') | default(6379, true) }}" cluster_enabled: "{{ lookup('env', 'DATACLAY_KV_CLUSTER_ENABLED') | default('yes', true) }}" # cluster_config_file: "{{ lookup('env', 'DATACLAY_KV_CLUSTER_CONFIG_FILE') | default('nodes.conf', true) }}" cluster_node_timeout: "{{ lookup('env', 'DATACLAY_KV_CLUSTER_NODE_TIMEOUT') | default(5000, true) }}" -cluster_replicas: "{{ lookup('env', 'DATACLAY_KV_CLUSTER_REPLICAS') | default(0, true) }}" \ No newline at end of file +cluster_replicas: "{{ lookup('env', 'DATACLAY_KV_CLUSTER_REPLICAS') | default(0, true) }}" diff --git a/orchestration/mn5/config/otel-json-exporter.yaml b/orchestration/mn5/config/otel-json-exporter.yaml index 989b3c0..9268da1 100644 --- a/orchestration/mn5/config/otel-json-exporter.yaml +++ b/orchestration/mn5/config/otel-json-exporter.yaml @@ -2,13 +2,17 @@ receivers: otlp: protocols: grpc: + endpoint: "0.0.0.0:4317" + exporters: file: path: $DATACLAY_JOB_PATH/otel-traces.json file/2: path: $DATACLAY_JOB_PATH/../otel-traces.json + processors: batch: + service: pipelines: traces: diff --git a/src/dataclay/runtime.py b/src/dataclay/runtime.py index 0277b3f..42c0d54 100644 --- a/src/dataclay/runtime.py +++ b/src/dataclay/runtime.py @@ -472,7 +472,7 @@ async def call_remote_method( ) raise response - logger.info( + logger.debug( "(%s) Remote method '%s' executed successfully", instance._dc_meta.id, method_name, diff --git a/src/dataclay/utils/backend_clients.py b/src/dataclay/utils/backend_clients.py index d4acfd6..a87d16e 100644 --- a/src/dataclay/utils/backend_clients.py +++ b/src/dataclay/utils/backend_clients.py @@ -11,10 +11,12 @@ from dataclay.event_loop import get_dc_event_loop from dataclay.metadata.api import MetadataAPI from dataclay.metadata.kvdata import Backend +from dataclay.utils.telemetry import trace if TYPE_CHECKING: from dataclay.metadata.client import MetadataClient +tracer = trace.get_tracer(__name__) logger = logging.getLogger(__name__) @@ -56,6 +58,7 @@ async def _update_loop(self): logger.info("Update loop has been cancelled.") raise + @tracer.start_as_current_span("update") async def update(self, force: bool = True): """Update the backend clients. From 5f724060bccc6e13ef4d8a40eb627d124f6e94dd Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Thu, 31 Oct 2024 00:33:27 +0800 Subject: [PATCH 18/24] Update telemetry documentation --- docs/telemetry.rst | 105 ++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 104 insertions(+), 1 deletion(-) diff --git a/docs/telemetry.rst b/docs/telemetry.rst index 24263f2..f7976dc 100644 --- a/docs/telemetry.rst +++ b/docs/telemetry.rst @@ -2,8 +2,21 @@ Telemetry ========= dataClay is instrumented with `OpenTelemetry `_ to allow observability of -distributed traces, metrics and logs. +distributed traces, metrics, and logs. You can configure tracing to export telemetry data either in real-time or for post-mortem analysis. Visualizations can be performed in Grafana. +Configuration +------------- + +To activate tracing in dataClay, the following environment variables need to be set: + +- **`DATACLAY_TRACING`**: Set to `true` to enable tracing. +- **`DATACLAY_TRACING_EXPORTER`**: Export traces to the OpenTelemetry Collector (`otlp`) or print traces to the console (`console`). The default is `otlp`. +- **`DATACLAY_TRACING_HOST`**: Host of the OpenTelemetry Collector (default: `localhost`). +- **`DATACLAY_TRACING_PORT`**: Port of the OpenTelemetry Collector (default: `4317`). +- **`DATACLAY_SERVICE_NAME`**: The service name, which identifies dataClay components in trace data. + +Metrics +------- .. list-table:: :header-rows: 1 @@ -26,3 +39,93 @@ distributed traces, metrics and logs. * - dataclay_inmemory_hits_total - Number of inmemory hits - backend, client + +Offline Telemetry Example +------------------------- + +This example demonstrates exporting OpenTelemetry traces to a JSON file for post-mortem analysis in Grafana. + +1. **Activate tracing** by setting environment variables as described above. +2. **Generate traces**: + + - Navigate to the `json-exporter` folder in the `offline telemetry example JSON exporter `_. + - Start dataClay and OpenTelemetry Collector services: + + .. code-block:: bash + + docker compose up + + - Run the dataClay client: + + .. code-block:: bash + + python3 client.py + + - Traces are exported to the `traces` folder. You can visualize the JSON traces in Grafana. + +3. **Visualize in Grafana**: + + - Navigate to the `json-post-mortem` folder in the `offline telemetry example post-mortem `_. + - Start the OpenTelemetry Collector, Tempo, and Grafana services: + + .. code-block:: bash + + docker compose up + + - Open Grafana at (default username/password: `admin`/`admin`). + - In the `Explore` section, select `Tempo` as the data source and use the `Trace ID` field to query traces. + +4. **Alternative Trace Export**: + + - Run the OpenTelemetry Collector manually: + + .. code-block:: bash + + docker run \ + -v ./config/otel-collector.yaml:/etc/otel-collector.yaml \ + otel/opentelemetry-collector-contrib \ + "--config=/etc/otel-collector.yaml" + +5. **Copy Traces from MareNostrum 5**: + + - To analyze traces from MareNostrum 5, copy them locally: + + .. code-block:: bash + + scp transfer1.bsc.es:~/.dataclay/otel-traces.json ./traces/otel-traces.json + +6. **Troubleshooting**: + + - If permission issues arise for the `/traces` folder, adjust permissions: + + .. code-block:: bash + + sudo chmod -R 777 traces + +Real-time Telemetry Example +--------------------------- + +This example demonstrates running a real-time telemetry pipeline. + +1. **Activate tracing** by setting environment variables as described above. +2. **Start services and generate traces**: + + - Go to the `real-time telemetry example folder `_. + - Start dataClay and OpenTelemetry Collector services: + + .. code-block:: bash + + docker compose up + + - Run the dataClay client: + + .. code-block:: bash + + python3 client.py + + - The traces are streamed in real-time to the OpenTelemetry Collector. + +3. **Visualize in Grafana**: + + - Access Grafana at (default username/password: `admin`/`admin`). + - In the `Explore` section, select `Tempo` as the data source to query traces. You can also view interactions in the `Service Graph` section. From 2d494d6bcd79d19f3f73a1a4a2cdc92aa0d4401f Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Mon, 11 Nov 2024 20:50:26 +0800 Subject: [PATCH 19/24] Fix dependency conflict for protobuf in py3.13 --- pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pyproject.toml b/pyproject.toml index 3cfe3aa..e8cac8f 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -68,7 +68,7 @@ telemetry = [ "opentelemetry-exporter-otlp-proto-grpc", "opentelemetry-instrumentation-grpc", "opentelemetry-instrumentation-redis", - "protobuf<5", # Remove when protobuf 5 is supported by opentelemetry + "protobuf<5;python_version<\"3.13\"", # Remove when protobuf 5 is supported by opentelemetry ] [project.scripts] From e45d56b2bb55e475c2e64158260473824205b606 Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Wed, 13 Nov 2024 20:04:40 +0800 Subject: [PATCH 20/24] Remove conditional py3.13 dependencies --- pyproject.toml | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/pyproject.toml b/pyproject.toml index e8cac8f..dfcc6c8 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -28,12 +28,10 @@ requires-python = ">=3.9" dependencies = [ "aiorwlock>=1.4.0", "bcrypt>=4.1.1", - "grpcio>=1.59.2;python_version<\"3.13\"", - "grpcio>=1.66.2;python_version>=\"3.13\"", - "grpcio-health-checking>=1.59.2;python_version<\"3.13\"", - "grpcio-health-checking>=1.66.2;python_version>=\"3.13\"", + "grpcio>=1.67.1", + "grpcio-health-checking>=1.67.1", "hiredis>=3.0.0", - "opentelemetry-api>=1.27.0", + "opentelemetry-api>=1.28.1", "protobuf>=4.25.0", "psutil>=6.1.0", "pydantic-settings>=2.6.0", @@ -68,7 +66,6 @@ telemetry = [ "opentelemetry-exporter-otlp-proto-grpc", "opentelemetry-instrumentation-grpc", "opentelemetry-instrumentation-redis", - "protobuf<5;python_version<\"3.13\"", # Remove when protobuf 5 is supported by opentelemetry ] [project.scripts] From 8c093cef620fdb2de544a12852ffd19b2c9cea52 Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Wed, 13 Nov 2024 20:24:42 +0800 Subject: [PATCH 21/24] Move prometheus examples inside telemetry folder --- examples/telemetry/offline/traces/.gitignore | 1 + examples/{ => telemetry}/prometheus-pushgateway/README.md | 0 examples/{ => telemetry}/prometheus-pushgateway/client.py | 0 .../prometheus-pushgateway/config/prometheus.yaml | 0 .../prometheus-pushgateway/docker-compose.yaml | 6 +++--- examples/telemetry/real-time/.gitignore | 2 ++ 6 files changed, 6 insertions(+), 3 deletions(-) create mode 100644 examples/telemetry/offline/traces/.gitignore rename examples/{ => telemetry}/prometheus-pushgateway/README.md (100%) rename examples/{ => telemetry}/prometheus-pushgateway/client.py (100%) rename examples/{ => telemetry}/prometheus-pushgateway/config/prometheus.yaml (100%) rename examples/{ => telemetry}/prometheus-pushgateway/docker-compose.yaml (95%) create mode 100644 examples/telemetry/real-time/.gitignore diff --git a/examples/telemetry/offline/traces/.gitignore b/examples/telemetry/offline/traces/.gitignore new file mode 100644 index 0000000..320a129 --- /dev/null +++ b/examples/telemetry/offline/traces/.gitignore @@ -0,0 +1 @@ +otel-traces.json \ No newline at end of file diff --git a/examples/prometheus-pushgateway/README.md b/examples/telemetry/prometheus-pushgateway/README.md similarity index 100% rename from examples/prometheus-pushgateway/README.md rename to examples/telemetry/prometheus-pushgateway/README.md diff --git a/examples/prometheus-pushgateway/client.py b/examples/telemetry/prometheus-pushgateway/client.py similarity index 100% rename from examples/prometheus-pushgateway/client.py rename to examples/telemetry/prometheus-pushgateway/client.py diff --git a/examples/prometheus-pushgateway/config/prometheus.yaml b/examples/telemetry/prometheus-pushgateway/config/prometheus.yaml similarity index 100% rename from examples/prometheus-pushgateway/config/prometheus.yaml rename to examples/telemetry/prometheus-pushgateway/config/prometheus.yaml diff --git a/examples/prometheus-pushgateway/docker-compose.yaml b/examples/telemetry/prometheus-pushgateway/docker-compose.yaml similarity index 95% rename from examples/prometheus-pushgateway/docker-compose.yaml rename to examples/telemetry/prometheus-pushgateway/docker-compose.yaml index 1c836bc..47bccce 100644 --- a/examples/prometheus-pushgateway/docker-compose.yaml +++ b/examples/telemetry/prometheus-pushgateway/docker-compose.yaml @@ -8,7 +8,7 @@ services: metadata-service: image: "ghcr.io/bsc-dom/dataclay:dev" build: - context: ../../ + context: ../../../ dockerfile: Dockerfile.dev depends_on: - redis @@ -28,7 +28,7 @@ services: - DATACLAY_METRICS_PORT=8000 command: python -m dataclay.metadata volumes: - - ../../:/app:ro + - ../../../:/app:ro backend: image: "ghcr.io/bsc-dom/dataclay:dev" @@ -47,7 +47,7 @@ services: - DATACLAY_METRICS_PORT=8000 command: python -m dataclay.backend volumes: - - ../../:/app:ro + - ../../../:/app:ro prometheus: image: prom/prometheus:latest diff --git a/examples/telemetry/real-time/.gitignore b/examples/telemetry/real-time/.gitignore new file mode 100644 index 0000000..884f9a8 --- /dev/null +++ b/examples/telemetry/real-time/.gitignore @@ -0,0 +1,2 @@ +/data/tempo/ +dashboard.json \ No newline at end of file From 0f12c61c99c2c6b4c7f6e250b3b3115b2d4c9f95 Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Wed, 13 Nov 2024 20:25:52 +0800 Subject: [PATCH 22/24] Add README file to telemetry/real-time --- examples/telemetry/real-time/README.md | 35 ++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100644 examples/telemetry/real-time/README.md diff --git a/examples/telemetry/real-time/README.md b/examples/telemetry/real-time/README.md new file mode 100644 index 0000000..bfd1cf5 --- /dev/null +++ b/examples/telemetry/real-time/README.md @@ -0,0 +1,35 @@ +# Real-time Telemetry Example + +This example demonstrates how to run a real-time telemetry pipeline with dataClay and OpenTelemetry, and visualize the traces in Grafana. + +To activate traces in dataClay, you need to set the following environment variables: + +- `DATACLAY_TRACING`: Set to `true` to enable tracing. +- `DATACLAY_TRACING_EXPORTER`: Set to `otlp` to export traces to the OpenTelemetry Collector, or `console` to print traces to the console (default is `otlp`). +- `DATACLAY_TRACING_HOST`: The host of the OpenTelemetry Collector (default is `localhost`). +- `DATACLAY_TRACING_PORT`: The port of the OpenTelemetry Collector (default is 4317). +- `DATACLAY_SERVICE_NAME`: The service name. + +## Running the Real-time Telemetry Pipeline + +Go into the `real-time` folder: + +```bash +cd real-time +``` + +Start the dataClay and OpenTelemetry Collector services: + +```bash +docker compose up +``` + +Run the dataClay client to generate some traces: + +```bash +python3 client.py +``` + +The traces are exported to the OpenTelemetry Collector. You can view the traces in the Grafana UI at . The default username and password are both `admin`. + +Navigate to the `Explore` section and select the `Tempo` data source. You can query the traces using the `Trace ID` field. Also, you can see the services and their interactions in the `Service Graph` section. From efca2c13d79f660c048ed9d08a6227ffdb84fc6c Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Thu, 14 Nov 2024 01:06:46 +0800 Subject: [PATCH 23/24] Refactor telemetry documentation --- docs/index.rst | 14 +- docs/telemetry.rst | 131 ------------------ docs/telemetry/configuration.rst | 41 ++++++ docs/{ => telemetry}/hpc-tracing.rst | 10 +- docs/telemetry/offline.rst | 62 +++++++++ docs/telemetry/prometheus.rst | 29 ++++ docs/telemetry/real-time.rst | 28 ++++ .../README.md | 2 +- .../client.py | 0 .../config/prometheus.yaml | 0 .../docker-compose.yaml | 0 11 files changed, 176 insertions(+), 141 deletions(-) delete mode 100644 docs/telemetry.rst create mode 100644 docs/telemetry/configuration.rst rename docs/{ => telemetry}/hpc-tracing.rst (94%) create mode 100644 docs/telemetry/offline.rst create mode 100644 docs/telemetry/prometheus.rst create mode 100644 docs/telemetry/real-time.rst rename examples/telemetry/{prometheus-pushgateway => prometheus}/README.md (96%) rename examples/telemetry/{prometheus-pushgateway => prometheus}/client.py (100%) rename examples/telemetry/{prometheus-pushgateway => prometheus}/config/prometheus.yaml (100%) rename examples/telemetry/{prometheus-pushgateway => prometheus}/docker-compose.yaml (100%) diff --git a/docs/index.rst b/docs/index.rst index 7a407c2..066fd26 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -10,8 +10,6 @@ dataClay main-concepts alien-objects advanced-usage - telemetry - hpc-tracing examples/index .. toctree:: @@ -25,12 +23,22 @@ dataClay deployment/hpc-manual-deployment deployment/compile-redis +.. toctree:: + :hidden: + :caption: Telemetry + + telemetry/configuration + telemetry/offline + telemetry/real-time + telemetry/prometheus + telemetry/hpc-tracing + .. toctree:: :hidden: :caption: Release Notes - releasenotes/3-x releasenotes/4-x + releasenotes/3-x .. toctree:: :hidden: diff --git a/docs/telemetry.rst b/docs/telemetry.rst deleted file mode 100644 index f7976dc..0000000 --- a/docs/telemetry.rst +++ /dev/null @@ -1,131 +0,0 @@ -Telemetry -========= - -dataClay is instrumented with `OpenTelemetry `_ to allow observability of -distributed traces, metrics, and logs. You can configure tracing to export telemetry data either in real-time or for post-mortem analysis. Visualizations can be performed in Grafana. - -Configuration -------------- - -To activate tracing in dataClay, the following environment variables need to be set: - -- **`DATACLAY_TRACING`**: Set to `true` to enable tracing. -- **`DATACLAY_TRACING_EXPORTER`**: Export traces to the OpenTelemetry Collector (`otlp`) or print traces to the console (`console`). The default is `otlp`. -- **`DATACLAY_TRACING_HOST`**: Host of the OpenTelemetry Collector (default: `localhost`). -- **`DATACLAY_TRACING_PORT`**: Port of the OpenTelemetry Collector (default: `4317`). -- **`DATACLAY_SERVICE_NAME`**: The service name, which identifies dataClay components in trace data. - -Metrics -------- - -.. list-table:: - :header-rows: 1 - - * - Metric - - Description - - Service - * - dataclay_inmemory_objects - - Number of objects in memory - - backend, client - * - dataclay_loaded_objects - - Number of loaded objects - - backend - * - dataclay_stored_objects - - Number of stored objects - - backend - * - dataclay_inmemory_misses_total - - Number of inmemory misses - - backend, client - * - dataclay_inmemory_hits_total - - Number of inmemory hits - - backend, client - -Offline Telemetry Example -------------------------- - -This example demonstrates exporting OpenTelemetry traces to a JSON file for post-mortem analysis in Grafana. - -1. **Activate tracing** by setting environment variables as described above. -2. **Generate traces**: - - - Navigate to the `json-exporter` folder in the `offline telemetry example JSON exporter `_. - - Start dataClay and OpenTelemetry Collector services: - - .. code-block:: bash - - docker compose up - - - Run the dataClay client: - - .. code-block:: bash - - python3 client.py - - - Traces are exported to the `traces` folder. You can visualize the JSON traces in Grafana. - -3. **Visualize in Grafana**: - - - Navigate to the `json-post-mortem` folder in the `offline telemetry example post-mortem `_. - - Start the OpenTelemetry Collector, Tempo, and Grafana services: - - .. code-block:: bash - - docker compose up - - - Open Grafana at (default username/password: `admin`/`admin`). - - In the `Explore` section, select `Tempo` as the data source and use the `Trace ID` field to query traces. - -4. **Alternative Trace Export**: - - - Run the OpenTelemetry Collector manually: - - .. code-block:: bash - - docker run \ - -v ./config/otel-collector.yaml:/etc/otel-collector.yaml \ - otel/opentelemetry-collector-contrib \ - "--config=/etc/otel-collector.yaml" - -5. **Copy Traces from MareNostrum 5**: - - - To analyze traces from MareNostrum 5, copy them locally: - - .. code-block:: bash - - scp transfer1.bsc.es:~/.dataclay/otel-traces.json ./traces/otel-traces.json - -6. **Troubleshooting**: - - - If permission issues arise for the `/traces` folder, adjust permissions: - - .. code-block:: bash - - sudo chmod -R 777 traces - -Real-time Telemetry Example ---------------------------- - -This example demonstrates running a real-time telemetry pipeline. - -1. **Activate tracing** by setting environment variables as described above. -2. **Start services and generate traces**: - - - Go to the `real-time telemetry example folder `_. - - Start dataClay and OpenTelemetry Collector services: - - .. code-block:: bash - - docker compose up - - - Run the dataClay client: - - .. code-block:: bash - - python3 client.py - - - The traces are streamed in real-time to the OpenTelemetry Collector. - -3. **Visualize in Grafana**: - - - Access Grafana at (default username/password: `admin`/`admin`). - - In the `Explore` section, select `Tempo` as the data source to query traces. You can also view interactions in the `Service Graph` section. diff --git a/docs/telemetry/configuration.rst b/docs/telemetry/configuration.rst new file mode 100644 index 0000000..5fe39e4 --- /dev/null +++ b/docs/telemetry/configuration.rst @@ -0,0 +1,41 @@ +Telemetry Configuration +======================= + +dataClay is instrumented with `OpenTelemetry `_ to allow observability of +distributed traces, metrics, and logs. You can configure tracing to export telemetry data either in real-time or for post-mortem analysis. Visualizations can be performed in Grafana. + +Configuration +------------- + +To activate tracing in dataClay, the following environment variables need to be set: + +- **`DATACLAY_TRACING`**: Set to `true` to enable tracing. +- **`DATACLAY_TRACING_EXPORTER`**: Export traces to the OpenTelemetry Collector (`otlp`) or print traces to the console (`console`). The default is `otlp`. +- **`DATACLAY_TRACING_HOST`**: Host of the OpenTelemetry Collector (default: `localhost`). +- **`DATACLAY_TRACING_PORT`**: Port of the OpenTelemetry Collector (default: `4317`). +- **`DATACLAY_SERVICE_NAME`**: The service name, which identifies dataClay components in trace data. + +Metrics +------- + +.. list-table:: + :header-rows: 1 + + * - Metric + - Description + - Service + * - dataclay_inmemory_objects + - Number of objects in memory + - backend, client + * - dataclay_loaded_objects + - Number of loaded objects + - backend + * - dataclay_stored_objects + - Number of stored objects + - backend + * - dataclay_inmemory_misses_total + - Number of inmemory misses + - backend, client + * - dataclay_inmemory_hits_total + - Number of inmemory hits + - backend, client diff --git a/docs/hpc-tracing.rst b/docs/telemetry/hpc-tracing.rst similarity index 94% rename from docs/hpc-tracing.rst rename to docs/telemetry/hpc-tracing.rst index 1a26c22..cfacc1d 100644 --- a/docs/hpc-tracing.rst +++ b/docs/telemetry/hpc-tracing.rst @@ -1,12 +1,9 @@ -=========== HPC Tracing =========== -How to generate paraver traces in MN5 -===================================== +How to generate paraver traces in MN5 using COMPSs +-------------------------------------------------- -Using COMPSs ------------- In order to get the traces we will create a script. - First we have to import the COMPSs and DataClay modules in order to be able to use them, as well as defining which python version we will be using: @@ -57,7 +54,8 @@ In order to generate the paraver files, we will call another COMPSs script, "com If we run this script in the same directory where we found the traces ($HOME/.COMPSs/[SLURM_JOB_ID]/trace/), the paraver files will appear. How to inspect the traces in Paraver -==================================== +------------------------------------ + To be able to see these files we will have to open them using the following commands: .. code-block:: bash diff --git a/docs/telemetry/offline.rst b/docs/telemetry/offline.rst new file mode 100644 index 0000000..dc5cb1b --- /dev/null +++ b/docs/telemetry/offline.rst @@ -0,0 +1,62 @@ + +Offline Telemetry Example +========================= + +This example demonstrates exporting OpenTelemetry traces to a JSON file for post-mortem analysis in Grafana. + +1. **Activate tracing** by setting environment variables as described in the `telemetry configuration `_. +2. **Generate traces**: + + - Navigate to the `json-exporter` folder in the `offline telemetry example JSON exporter `_. + - Start dataClay and OpenTelemetry Collector services: + + .. code-block:: bash + + docker compose up + + - Run the dataClay client: + + .. code-block:: bash + + python3 client.py + + - Traces are exported to the `traces` folder. You can visualize the JSON traces in Grafana. + +3. **Visualize in Grafana**: + + - Navigate to the `json-post-mortem` folder in the `offline telemetry example post-mortem `_. + - Start the OpenTelemetry Collector, Tempo, and Grafana services: + + .. code-block:: bash + + docker compose up + + - Open Grafana at (default username/password: `admin`/`admin`). + - In the `Explore` section, select `Tempo` as the data source and use the `Trace ID` field to query traces. + +4. **Alternative Trace Export**: + + - Run the OpenTelemetry Collector manually: + + .. code-block:: bash + + docker run \ + -v ./config/otel-collector.yaml:/etc/otel-collector.yaml \ + otel/opentelemetry-collector-contrib \ + "--config=/etc/otel-collector.yaml" + +5. **Copy Traces from MareNostrum 5**: + + - To analyze traces from MareNostrum 5, copy them locally: + + .. code-block:: bash + + scp transfer1.bsc.es:~/.dataclay/otel-traces.json ./traces/otel-traces.json + +6. **Troubleshooting**: + + - If permission issues arise for the `/traces` folder, adjust permissions: + + .. code-block:: bash + + sudo chmod -R 777 traces diff --git a/docs/telemetry/prometheus.rst b/docs/telemetry/prometheus.rst new file mode 100644 index 0000000..b946e93 --- /dev/null +++ b/docs/telemetry/prometheus.rst @@ -0,0 +1,29 @@ +Prometheus Example +================== + +Deploy dataclay with Prometheus and Pushgateway: + +.. note:: + This example is available in `GitHub `_. + +.. code-block:: bash + + docker compose up -d + +The ``metadata-service`` and ``backends`` will post their metrics to the ``8000`` port. +Prometheus is configured to scrape this port to pull the metrics. + +The ``client.py`` can also push metrics using the ``pushgateway``: + +.. code-block:: bash + + export DATACLAY_METRICS=true + export DATACLAY_METRICS_EXPORTER=pushgateway + export DATACLAY_METRICS_HOST=localhost # the default + export DATACLAY_METRICS_PORT=9091 + python3 client.py + +Go to ``localhost:9090/graph`` to explore the metrics with Prometheus. + +.. note:: + When using ``pushgateway``, a new Python thread will run to push the metrics every 10 seconds. \ No newline at end of file diff --git a/docs/telemetry/real-time.rst b/docs/telemetry/real-time.rst new file mode 100644 index 0000000..6923391 --- /dev/null +++ b/docs/telemetry/real-time.rst @@ -0,0 +1,28 @@ + +Real-time Telemetry Example +=========================== + +This example demonstrates running a real-time telemetry pipeline. + +1. **Activate tracing** by setting environment variables as described in the `telemetry configuration `_. +2. **Start services and generate traces**: + + - Go to the `real-time telemetry example folder `_. + - Start dataClay and OpenTelemetry Collector services: + + .. code-block:: bash + + docker compose up + + - Run the dataClay client: + + .. code-block:: bash + + python3 client.py + + - The traces are streamed in real-time to the OpenTelemetry Collector. + +3. **Visualize in Grafana**: + + - Access Grafana at (default username/password: `admin`/`admin`). + - In the `Explore` section, select `Tempo` as the data source to query traces. You can also view interactions in the `Service Graph` section. diff --git a/examples/telemetry/prometheus-pushgateway/README.md b/examples/telemetry/prometheus/README.md similarity index 96% rename from examples/telemetry/prometheus-pushgateway/README.md rename to examples/telemetry/prometheus/README.md index b6f3ab2..dd155b7 100644 --- a/examples/telemetry/prometheus-pushgateway/README.md +++ b/examples/telemetry/prometheus/README.md @@ -1,4 +1,4 @@ -# Prometheus - pushgateway +# Prometheus Deploy dataclay with Prometheus and pushgateway: diff --git a/examples/telemetry/prometheus-pushgateway/client.py b/examples/telemetry/prometheus/client.py similarity index 100% rename from examples/telemetry/prometheus-pushgateway/client.py rename to examples/telemetry/prometheus/client.py diff --git a/examples/telemetry/prometheus-pushgateway/config/prometheus.yaml b/examples/telemetry/prometheus/config/prometheus.yaml similarity index 100% rename from examples/telemetry/prometheus-pushgateway/config/prometheus.yaml rename to examples/telemetry/prometheus/config/prometheus.yaml diff --git a/examples/telemetry/prometheus-pushgateway/docker-compose.yaml b/examples/telemetry/prometheus/docker-compose.yaml similarity index 100% rename from examples/telemetry/prometheus-pushgateway/docker-compose.yaml rename to examples/telemetry/prometheus/docker-compose.yaml From d7a0539ab98e16627cba771b636592fbcf62009d Mon Sep 17 00:00:00 2001 From: Marc Monfort Date: Mon, 18 Nov 2024 20:47:17 +0800 Subject: [PATCH 24/24] Add Prometheus Pushgateway support and update documentation --- docs/telemetry/prometheus.rst | 61 +++++++++++++++++-- .../telemetry/prometheus-pushgateway/.env | 5 ++ .../prometheus-pushgateway/README.md | 35 +++++++++++ .../client.py | 8 +-- .../config/prometheus.yaml | 12 ++++ .../docker-compose.yaml | 60 ++++++++++++++++++ examples/telemetry/prometheus/README.md | 28 +++++---- .../prometheus/config/prometheus.yaml | 4 -- .../telemetry/prometheus/docker-compose.yaml | 30 +++------ pyproject.toml | 1 + src/dataclay/config.py | 2 +- src/dataclay/utils/metrics.py | 6 ++ 12 files changed, 201 insertions(+), 51 deletions(-) create mode 100644 examples/telemetry/prometheus-pushgateway/.env create mode 100644 examples/telemetry/prometheus-pushgateway/README.md rename examples/telemetry/{prometheus => prometheus-pushgateway}/client.py (56%) create mode 100644 examples/telemetry/prometheus-pushgateway/config/prometheus.yaml create mode 100644 examples/telemetry/prometheus-pushgateway/docker-compose.yaml diff --git a/docs/telemetry/prometheus.rst b/docs/telemetry/prometheus.rst index b946e93..42a3091 100644 --- a/docs/telemetry/prometheus.rst +++ b/docs/telemetry/prometheus.rst @@ -1,10 +1,40 @@ -Prometheus Example -================== +Prometheus +========== -Deploy dataclay with Prometheus and Pushgateway: + +Metrics +------- + +.. list-table:: + :header-rows: 1 + + * - Metric + - Description + - Service + * - dataclay_inmemory_objects + - Number of objects in memory + - backend, client + * - dataclay_loaded_objects + - Number of loaded objects + - backend + * - dataclay_stored_objects + - Number of stored objects + - backend + * - dataclay_inmemory_misses_total + - Number of inmemory misses + - backend, client + * - dataclay_inmemory_hits_total + - Number of inmemory hits + - backend, client + + +Deploy dataClay with Prometheus +------------------------------- + +Run dataClay with Prometheus: .. note:: - This example is available in `GitHub `_. + This example is available in `GitHub `__. .. code-block:: bash @@ -13,6 +43,24 @@ Deploy dataclay with Prometheus and Pushgateway: The ``metadata-service`` and ``backends`` will post their metrics to the ``8000`` port. Prometheus is configured to scrape this port to pull the metrics. +Access Prometheus at `http://localhost:9090 `_. You can query the metrics defined above. + + +Deploy dataClay with Prometheus Pushgateway +------------------------------------------- + +Run dataClay with Prometheus Pushgateway: + +.. note:: + This example is available in `GitHub `__. + +.. code-block:: bash + + docker compose up -d + + +The ``metadata-service`` and ``backends`` will push their metrics to the ``pushgateway`` at the ``9091`` port. + The ``client.py`` can also push metrics using the ``pushgateway``: .. code-block:: bash @@ -23,7 +71,8 @@ The ``client.py`` can also push metrics using the ``pushgateway``: export DATACLAY_METRICS_PORT=9091 python3 client.py -Go to ``localhost:9090/graph`` to explore the metrics with Prometheus. + +Access the Pushgateway at `http://localhost:9091 `_ and Prometheus at `http://localhost:9090 `_. .. note:: - When using ``pushgateway``, a new Python thread will run to push the metrics every 10 seconds. \ No newline at end of file + When using ``pushgateway``, a new Python thread will run to push the metrics every 10 seconds (default). \ No newline at end of file diff --git a/examples/telemetry/prometheus-pushgateway/.env b/examples/telemetry/prometheus-pushgateway/.env new file mode 100644 index 0000000..1c16c90 --- /dev/null +++ b/examples/telemetry/prometheus-pushgateway/.env @@ -0,0 +1,5 @@ +DATACLAY_LOGLEVEL=debug +DATACLAY_METRICS=true +DATACLAY_METRICS_EXPORTER=pushgateway +DATACLAY_METRICS_HOST=localhost # the default +DATACLAY_METRICS_PORT=9091 \ No newline at end of file diff --git a/examples/telemetry/prometheus-pushgateway/README.md b/examples/telemetry/prometheus-pushgateway/README.md new file mode 100644 index 0000000..0ca8111 --- /dev/null +++ b/examples/telemetry/prometheus-pushgateway/README.md @@ -0,0 +1,35 @@ +# Prometheus Pushgateway + +## Metrics + +| Metric | Description | Service | +|------------------------------|---------------------------|------------------| +| `dataclay_inmemory_objects` | Number of objects in memory | backend, client | +| `dataclay_loaded_objects` | Number of loaded objects | backend | +| `dataclay_stored_objects` | Number of stored objects | backend | +| `dataclay_inmemory_misses_total` | Number of inmemory misses | backend, client | +| `dataclay_inmemory_hits_total` | Number of inmemory hits | backend, client | + +## Deploy + +Run dataClay with Prometheus and Pushgateway: + +```bash +docker compose up -d +``` + +The `metadata-service` and `backends` will push their metrics to the `pushgateway` at `9091` port. + +The `client.py` can also push metris using the `pushgateway`: + +```bash +export DATACLAY_METRICS=true +export DATACLAY_METRICS_EXPORTER=pushgateway +export DATACLAY_METRICS_HOST=localhost # the default +export DATACLAY_METRICS_PORT=9091 +python3 client.py +``` + +Access the Pushgateway at [http://localhost:9091](http://localhost:9091) and Prometheus at [http://localhost:9090](http://localhost:9090). + +Note: When using `pushgateway`, a new Python thread will run to push the metrics every 10 seconds (default). diff --git a/examples/telemetry/prometheus/client.py b/examples/telemetry/prometheus-pushgateway/client.py similarity index 56% rename from examples/telemetry/prometheus/client.py rename to examples/telemetry/prometheus-pushgateway/client.py index 552365c..b0adf7b 100644 --- a/examples/telemetry/prometheus/client.py +++ b/examples/telemetry/prometheus-pushgateway/client.py @@ -1,10 +1,7 @@ from dataclay import Client -from dataclay.conf import settings -from dataclay.contrib.modeltest.family import Dog, Family, Person +from dataclay.contrib.modeltest.family import Family, Person -settings.metrics = True - -client = Client(host="127.0.0.1", username="testuser", password="s3cret", dataset="testdata") +client = Client() client.start() family = Family() @@ -21,3 +18,4 @@ Family.delete_alias("Smith") assert person is person_2 +client.stop() diff --git a/examples/telemetry/prometheus-pushgateway/config/prometheus.yaml b/examples/telemetry/prometheus-pushgateway/config/prometheus.yaml new file mode 100644 index 0000000..422fea4 --- /dev/null +++ b/examples/telemetry/prometheus-pushgateway/config/prometheus.yaml @@ -0,0 +1,12 @@ +global: + scrape_interval: 15s + evaluation_interval: 15s + +scrape_configs: + - job_name: "prometheus" + static_configs: + - targets: ["localhost:9090"] + - job_name: "pushgateway" + honor_labels: true + static_configs: + - targets: ["pushgateway:9091"] diff --git a/examples/telemetry/prometheus-pushgateway/docker-compose.yaml b/examples/telemetry/prometheus-pushgateway/docker-compose.yaml new file mode 100644 index 0000000..f0a7861 --- /dev/null +++ b/examples/telemetry/prometheus-pushgateway/docker-compose.yaml @@ -0,0 +1,60 @@ +services: + + redis: + image: redis:latest + ports: + - 6379:6379 + + metadata-service: + image: "ghcr.io/bsc-dom/dataclay:dev" + build: + context: ../../../ + dockerfile: Dockerfile.dev + depends_on: + - redis + ports: + - 16587:16587 + environment: + - DATACLAY_KV_HOST=redis + - DATACLAY_METRICS=true + - DATACLAY_METRICS_EXPORTER=pushgateway + - DATACLAY_METRICS_HOST=pushgateway + - DATACLAY_METRICS_PORT=9091 + - DATACLAY_LOGLEVEL=DEBUG + command: python -m dataclay.metadata + volumes: + - ../../../:/app:ro + + backend: + image: "ghcr.io/bsc-dom/dataclay:dev" + depends_on: + - redis + ports: + - 6867:6867 + environment: + - DATACLAY_KV_HOST=redis + - DATACLAY_METRICS=true + - DATACLAY_METRICS_EXPORTER=pushgateway + - DATACLAY_METRICS_HOST=pushgateway + - DATACLAY_METRICS_PORT=9091 + - DATACLAY_LOGLEVEL=DEBUG + command: python -m dataclay.backend + volumes: + - ../../../:/app:ro + + prometheus: + image: prom/prometheus:latest + ports: + - "9090:9090" + volumes: + - ./config/prometheus.yaml:/etc/prometheus.yaml + command: + - --config.file=/etc/prometheus.yaml + - --web.enable-remote-write-receiver #todo: check if this is needed + - --enable-feature=exemplar-storage #todo: check if this is needed + + pushgateway: + image: prom/pushgateway:latest + ports: + - "9091:9091" + diff --git a/examples/telemetry/prometheus/README.md b/examples/telemetry/prometheus/README.md index dd155b7..ea98679 100644 --- a/examples/telemetry/prometheus/README.md +++ b/examples/telemetry/prometheus/README.md @@ -1,6 +1,18 @@ # Prometheus -Deploy dataclay with Prometheus and pushgateway: +## Metrics + +| Metric | Description | Service | +|------------------------------|---------------------------|------------------| +| `dataclay_inmemory_objects` | Number of objects in memory | backend, client | +| `dataclay_loaded_objects` | Number of loaded objects | backend | +| `dataclay_stored_objects` | Number of stored objects | backend | +| `dataclay_inmemory_misses_total` | Number of inmemory misses | backend, client | +| `dataclay_inmemory_hits_total` | Number of inmemory hits | backend, client | + +## Deploy + +Run dataClay with Prometheus: ```bash docker compose up -d @@ -9,16 +21,6 @@ docker compose up -d The `metadata-service` and `backends` will post their metrics to `8000` port. Prometheus is configured to scrape this port to pull the metrics. -The `client.py` can also push metris using the `pushgateway`: - -```bash -export DATACLAY_METRICS=true -export DATACLAY_METRICS_EXPORTER=pushgateway -export DATACLAY_METRICS_HOST=localhost # the default -export DATACLAY_METRICS_PORT=9091 -python3 client.py -``` - -Go to `localhost:9090/graph` to explore the metrics wiht `Prometheus`. +Access Prometheus at [http://localhost:9090](http://localhost:9090). You can query the metrics defined above. -Note: When using `pushgateway` a new python thread will run to push the metrics every 10 seconds. + \ No newline at end of file diff --git a/examples/telemetry/prometheus/config/prometheus.yaml b/examples/telemetry/prometheus/config/prometheus.yaml index 9b584b7..af04dba 100644 --- a/examples/telemetry/prometheus/config/prometheus.yaml +++ b/examples/telemetry/prometheus/config/prometheus.yaml @@ -12,7 +12,3 @@ scrape_configs: - job_name: "metadata-service" static_configs: - targets: ["metadata-service:8000"] - - job_name: "pushgateway" - honor_labels: true - static_configs: - - targets: ["pushgateway:9091"] diff --git a/examples/telemetry/prometheus/docker-compose.yaml b/examples/telemetry/prometheus/docker-compose.yaml index 47bccce..66bbbde 100644 --- a/examples/telemetry/prometheus/docker-compose.yaml +++ b/examples/telemetry/prometheus/docker-compose.yaml @@ -16,16 +16,10 @@ services: - 16587:16587 environment: - DATACLAY_KV_HOST=redis - - DATACLAY_KV_PORT=6379 - - DATACLAY_PASSWORD=s3cret - - DATACLAY_USERNAME=testuser - - DATACLAY_DATASET=testdata - - DATACLAY_METADATA_PORT=16587 - - DATACLAY_LOGLEVEL=DEBUG - DATACLAY_METRICS=true - DATACLAY_METRICS_EXPORTER=http - # - DATACLAY_METRICS_HOST=pushgateway - DATACLAY_METRICS_PORT=8000 + - DATACLAY_LOGLEVEL=DEBUG command: python -m dataclay.metadata volumes: - ../../../:/app:ro @@ -38,30 +32,22 @@ services: - 6867:6867 environment: - DATACLAY_KV_HOST=redis - - DATACLAY_KV_PORT=6379 - - DATACLAY_BACKEND_PORT=6867 - - DATACLAY_LOGLEVEL=DEBUG - DATACLAY_METRICS=true - DATACLAY_METRICS_EXPORTER=http - # - DATACLAY_METRICS_HOST=pushgateway - DATACLAY_METRICS_PORT=8000 + - DATACLAY_LOGLEVEL=DEBUG command: python -m dataclay.backend volumes: - ../../../:/app:ro prometheus: image: prom/prometheus:latest - command: - - --config.file=/etc/prometheus.yaml - - --web.enable-remote-write-receiver - - --enable-feature=exemplar-storage - volumes: - - ./config/prometheus.yaml:/etc/prometheus.yaml ports: - "9090:9090" - - pushgateway: - image: prom/pushgateway:latest - ports: - - "9091:9091" + volumes: + - ./config/prometheus.yaml:/etc/prometheus.yaml + command: + - --config.file=/etc/prometheus.yaml + - --web.enable-remote-write-receiver #todo: check if this is needed + - --enable-feature=exemplar-storage #todo: check if this is needed diff --git a/pyproject.toml b/pyproject.toml index dfcc6c8..5aca882 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -66,6 +66,7 @@ telemetry = [ "opentelemetry-exporter-otlp-proto-grpc", "opentelemetry-instrumentation-grpc", "opentelemetry-instrumentation-redis", + "prometheus-client", ] [project.scripts] diff --git a/src/dataclay/config.py b/src/dataclay/config.py index e3e4b4d..295951e 100644 --- a/src/dataclay/config.py +++ b/src/dataclay/config.py @@ -189,7 +189,7 @@ class Settings(BaseSettings): # Metrics metrics: bool = False - metrics_exporter: Literal["http", "prometheus", "none"] = "http" + metrics_exporter: Literal["http", "pushgateway", "none"] = "http" metrics_host: str = "localhost" metrics_port: int = 8000 # 9091 for pushgateway metrics_push_interval: int = 10 diff --git a/src/dataclay/utils/metrics.py b/src/dataclay/utils/metrics.py index e70629d..236327f 100644 --- a/src/dataclay/utils/metrics.py +++ b/src/dataclay/utils/metrics.py @@ -4,10 +4,14 @@ If metrics are not enabled, this class will not be imported and the code will not be executed. """ +import logging import os import threading import time +logger = logging.getLogger(__name__) + + from prometheus_client import ( CollectorRegistry, Counter, @@ -20,6 +24,7 @@ def pushgateway_thread(host, port, registry): while True: + logger.debug("Pushing metrics to gateway in %s:%d", host, port) try: push_to_gateway(f"{host}:{port}", job="dataclay", registry=registry) except Exception as e: @@ -28,6 +33,7 @@ def pushgateway_thread(host, port, registry): def set_metrics(host, port, exporter): + logger.info("Setting metrics exporter %s to %s:%d", exporter, host, port) if exporter == "http": start_http_server(port) elif exporter == "pushgateway":