diff --git a/python/lsst/daf/butler/_formatter.py b/python/lsst/daf/butler/_formatter.py index 7250187423..c94f5d552a 100644 --- a/python/lsst/daf/butler/_formatter.py +++ b/python/lsst/daf/butler/_formatter.py @@ -33,6 +33,7 @@ "FormatterV1inV2", "FormatterFactory", "FormatterParameter", + "FormatterNotImplementedError", "FileIntegrityError", ) @@ -72,6 +73,12 @@ class FileIntegrityError(RuntimeError): """ +class FormatterNotImplementedError(NotImplementedError): + """Formatter does not implement the specific read or write method + that is being requested. + """ + + class FormatterV2(metaclass=ABCMeta): """Interface for reading and writing datasets using URIs. @@ -94,12 +101,26 @@ class FormatterV2(metaclass=ABCMeta): **kwargs Additional arguments that will be ignored but allow for `Formatter` V1 parameters to be given. - """ - allow_remote_file_read: bool = False - """If `True` default implementation will call user-provided - `read_from_cached_uri` method first before falling back to attempting - a local file read.""" + Notes + ----- + In most cases a Formatter author will not want to override the default + `read` method but will instead want to implement one or all of + `read_from_bytes`, `read_from_uri`, or `read_from_local_file`. The method + `read_from_uri` will always be attempted first and could be more efficient + (since it allows the possibility for a subset of the data file to be + accessed when parameters or components are specified) but it will not + update the local cache. If the entire contents of the remote file are + being accessed consider raising `FormatterNotImplementedError` and falling + back to `read_from_local_file` since this will store the file in the + cache to allow a more efficient subsequent read. + + Similarly for writes, the recommendation is not to override the `write` + method but instead to implement `write_local_file` and, optionally, + `to_bytes`. For local URIs the system will always call `write_local_file` + to ensure atomic writes are implemented. For remote URIs with local caching + disabled, `to_bytes` will be called first. + """ unsupported_parameters: ClassVar[Set[str] | None] = frozenset() """Set of read parameters not understood by this `Formatter`. An empty set @@ -256,7 +277,10 @@ def _is_disassembled(self) -> bool: return self.file_descriptor.component is not None def _check_resource_size(self, uri: ResourcePath, recorded_size: int, resource_size: int) -> None: - """Compare the recorded size with the resource size.""" + """Compare the recorded size with the resource size. + + The given URI will not be accessed. + """ if recorded_size >= 0 and resource_size != recorded_size: raise FileIntegrityError( "Integrity failure in Datastore. " @@ -281,6 +305,17 @@ def _get_cache_ref(self) -> DatasetRef: cache_ref = self.dataset_ref.makeComponentRef(self.file_descriptor.component) return cache_ref + def _ensure_cache( + self, cache_manager: AbstractDatastoreCacheManager | None = None + ) -> AbstractDatastoreCacheManager: + """Return the cache if given else return a null cache.""" + if cache_manager is None: + # Circular import avoidance. + from .datastore.cache_manager import DatastoreDisabledCacheManager + + cache_manager = DatastoreDisabledCacheManager(None, None) + return cache_manager + def read( self, component: str | None = None, @@ -336,27 +371,33 @@ def read( * A derived component. The file to read is a component but not the specified component. The caching needs the component from which it's derived. + + Raises + ------ + FormatterNotImplementedError + Raised if no implementations were found that could read this + resource. """ - if self.allow_remote_file_read: - # In case someone set the flag incorrectly, if this is not - # implemented fall back to the other methods. - with contextlib.suppress(NotImplementedError): - return self.read_from_possibly_cached_uri( - component, expected_size, cache_manager=cache_manager - ) + # First see if the formatter can support direct remote read from + # a URI. + with contextlib.suppress(FormatterNotImplementedError): + return self.read_directly_from_possibly_cached_uri( + component, expected_size, cache_manager=cache_manager + ) # Should we check the size with file_uri.size() if expected size is -1? # Do we give a choice here at all and allow a formatter to say one - # or the other? Should the cache be checked (if a put is followed - # by a get it might still be in the cache). + # or the other? if expected_size > 0 and expected_size < self.nbytes_read_threshold: # If this method is not implemented, fall back to using a file. # V1 formatters in emulation won't set the read threshold. - with contextlib.suppress(NotImplementedError): + with contextlib.suppress(FormatterNotImplementedError): return self.read_from_possibly_cached_bytes( component, expected_size, cache_manager=cache_manager ) + # Finally, try to read the local file. + # If this is not implemented we let the exception through. return self.read_from_possibly_cached_local_file( component, expected_size, cache_manager=cache_manager ) @@ -396,11 +437,7 @@ def read_from_possibly_cached_bytes( method will not cache a remote dataset and assumes that any size checks have been performed before calling this method. """ - if cache_manager is None: - # Circular import avoidance. - from .datastore.cache_manager import DatastoreDisabledCacheManager - - cache_manager = DatastoreDisabledCacheManager(None, None) + cache_manager = self._ensure_cache(cache_manager) uri = self.file_descriptor.location.uri cache_ref = self._get_cache_ref() @@ -431,7 +468,7 @@ def read_from_possibly_cached_bytes( return self.read_from_bytes(serialized_dataset, component=component) - def read_from_possibly_cached_uri( + def read_directly_from_possibly_cached_uri( self, component: str | None = None, expected_size: int = -1, @@ -461,16 +498,14 @@ def read_from_possibly_cached_uri( Notes ----- - Calls `read_from_file` but will first check the datastore cache - in case the file is present and reads that in preference. This - method will not cache a remote dataset and will only do a size - check for local files. - """ - if cache_manager is None: - # Circular import avoidance. - from .datastore.cache_manager import DatastoreDisabledCacheManager + This method will first check the datastore cache + in case the file is present locally. This method will not cache a + remote dataset and will only do a size check for local files to avoid + unnecessary round trips to a remote server. - cache_manager = DatastoreDisabledCacheManager(None, None) + The URI will be read by calling `read_from_uri`. + """ + cache_manager = self._ensure_cache(cache_manager) uri = self.file_descriptor.location.uri cache_ref = self._get_cache_ref() @@ -507,7 +542,7 @@ def read_from_possibly_cached_uri( self.name(), ), ): - result = self.read_direct_file(desired_uri, component=component) + result = self.read_from_uri(desired_uri, component=component) return result @@ -543,14 +578,9 @@ def read_from_possibly_cached_local_file( Notes ----- The file will be downloaded and cached if it is a remote resource. - The file contents will be read using `read_local_file`. + The file contents will be read using `read_from_local_file`. """ - if cache_manager is None: - # Circular import avoidance. - from .datastore.cache_manager import DatastoreDisabledCacheManager - - cache_manager = DatastoreDisabledCacheManager(None, None) - + cache_manager = self._ensure_cache(cache_manager) uri = self.file_descriptor.location.uri # Need to have something we can look up in the cache. @@ -610,7 +640,7 @@ def read_from_possibly_cached_local_file( self.name(), ), ): - result = self.read_local_file(local_uri, component=component) + result = self.read_from_local_file(local_uri, component=component) # File was read successfully so can move to cache if can_be_cached: @@ -618,7 +648,7 @@ def read_from_possibly_cached_local_file( return result - def read_direct_file(self, uri: ResourcePath, component: str | None = None) -> Any: + def read_from_uri(self, uri: ResourcePath, component: str | None = None) -> Any: """Read a dataset from a URI that can be local or remote. Parameters @@ -634,6 +664,12 @@ def read_direct_file(self, uri: ResourcePath, component: str | None = None) -> A in_memory_dataset : `typing.Any` The Python object read from the resource. + Raises + ------ + FormatterNotImplementedError + Raised if there is no support for direct reads from a, possibly, + remote URI. + Notes ----- This method is only called if the class property @@ -642,11 +678,19 @@ def read_direct_file(self, uri: ResourcePath, component: str | None = None) -> A It is possible that a cached local file will be given to this method even if it was originally a remote URI. This can happen if the write resulted in the file being added to the local cache. + + If the full file is being read this file will not be added to the + local cache. Consider raising `FormatterNotImplementedError` in + this situation, for example if there are no parameters or component + specified, and allowing the system to fall back to calling + `read_from_local_file` (which will populate the cache if configured + to do so). """ - raise NotImplementedError("This formatter does not know how to read a file.") + raise FormatterNotImplementedError("This formatter does not know how to read a file.") - def read_local_file(self, local_uri: ResourcePath, component: str | None = None) -> Any: - """Read a dataset from a local file system. + def read_from_local_file(self, local_uri: ResourcePath, component: str | None = None) -> Any: + """Read a dataset from a URI guaranteed to refer to the local file + system. Parameters ---------- @@ -661,12 +705,18 @@ def read_local_file(self, local_uri: ResourcePath, component: str | None = None) in_memory_dataset : `typing.Any` The Python object read from the resource. + Raises + ------ + FormatterNotImplementedError + Raised if there is no implementation written to read data + from a local file. + Notes ----- - This method will only be called if `read_cached_file` was not called - or if `read_from_bytes` was called successfully. + This method will only be called if neither `read_from_uri` nor + `read_from_bytes` were called successfully. """ - raise NotImplementedError("This formatter does not know how to read a local file.") + raise FormatterNotImplementedError("This formatter does not know how to read a local file.") def read_from_bytes(self, serialized_bytes: bytes, component: str | None = None) -> Any: """Read a dataset from a byte stream. @@ -683,15 +733,23 @@ def read_from_bytes(self, serialized_bytes: bytes, component: str | None = None) in_memory_dataset : `typing.Any` The Python object read from the resource. + Raises + ------ + FormatterNotImplementedError + Raised if there is no implementation for constructing a Python + object from the serialized bytes. + Notes ----- - This method will only be called if `read_cached_file` was not called. + This method will only be called if `read_from_uri` was not called. Additionally, in the default implementation of the `read` method, this method will only be called if the expected size of the resource is known and it is less than the class property ``nbytes_read_threshold``. """ - raise NotImplementedError("This formatter does not know how to convert bytes to a Python type.") + raise FormatterNotImplementedError( + "This formatter does not know how to convert bytes to a Python type." + ) def write( self, @@ -740,20 +798,18 @@ def write_direct( Notes ----- - This method will call `to_bytes` to serialize the in memory dataset + This method will call `to_bytes` to serialize the in-memory dataset and then will call the `~lsst.resources.ResourcePath.write` method directly. If the dataset should be cached or is local the file will not be written and the method will return `False`. This is because local URIs should be written to a temporary file name and then renamed to allow - atomic writes. That path is handled by `write_locally_then_move`. + atomic writes. That path is handled by `write_locally_then_move` (via + `write_local_file`) and is preferred over this method being subclassed + and the atomic write re-implemented. """ - if cache_manager is None: - # Circular import avoidance. - from .datastore.cache_manager import DatastoreDisabledCacheManager - - cache_manager = DatastoreDisabledCacheManager(None, None) + cache_manager = self._ensure_cache(cache_manager) # For remote URIs some datasets can be serialized directly # to bytes and sent to the remote datastore without writing a @@ -765,7 +821,7 @@ def write_direct( # Remote URI that is not cached so can write directly. try: serialized_dataset = self.to_bytes(in_memory_dataset) - except NotImplementedError: + except FormatterNotImplementedError: # Fallback to the file writing option. pass except Exception as e: @@ -799,11 +855,7 @@ def write_locally_then_move( A cache manager to use to allow a formatter to cache the written file. """ - if cache_manager is None: - # Circular import avoidance. - from .datastore.cache_manager import DatastoreDisabledCacheManager - - cache_manager = DatastoreDisabledCacheManager(None, None) + cache_manager = self._ensure_cache(cache_manager) # Always write to a temporary even if # using a local file system -- that gives us atomic writes. @@ -868,7 +920,7 @@ def to_bytes(self, in_memory_dataset: Any) -> bytes: serialized_dataset : `bytes` Bytes representing the serialized dataset. """ - raise NotImplementedError( + raise FormatterNotImplementedError( f"This formatter can not convert {get_full_type_name(in_memory_dataset)} directly to bytes." ) @@ -884,12 +936,6 @@ def make_updated_location(self, location: Location) -> Location: ------- updated : `Location` A new `Location` with a new file extension applied. - - Raises - ------ - NotImplementedError - Raised if there is no ``extension`` attribute associated with - this formatter. """ location = location.clone() # If the extension is "" the extension will be removed. @@ -1180,7 +1226,7 @@ def read(self, component: str | None = None) -> Any: inMemoryDataset : `object` The requested Dataset. """ - raise NotImplementedError("Type does not support reading") + raise FormatterNotImplementedError("Type does not support reading") @abstractmethod def write(self, inMemoryDataset: Any) -> None: @@ -1191,7 +1237,7 @@ def write(self, inMemoryDataset: Any) -> None: inMemoryDataset : `object` The Dataset to store. """ - raise NotImplementedError("Type does not support writing") + raise FormatterNotImplementedError("Type does not support writing") @classmethod def can_read_bytes(cls) -> bool: @@ -1207,7 +1253,7 @@ def can_read_bytes(cls) -> bool: try: # We know the arguments are incompatible cls.fromBytes(cls, b"") # type: ignore - except NotImplementedError: + except FormatterNotImplementedError: return False except Exception: # There will be problems with the bytes we are supplying so ignore @@ -1232,7 +1278,7 @@ def fromBytes(self, serializedDataset: bytes, component: str | None = None) -> o The requested data as a Python object. The type of object is controlled by the specific formatter. """ - raise NotImplementedError("Type does not support reading from bytes.") + raise FormatterNotImplementedError("Type does not support reading from bytes.") def toBytes(self, inMemoryDataset: Any) -> bytes: """Serialize the Dataset to bytes based on formatter. @@ -1247,7 +1293,7 @@ def toBytes(self, inMemoryDataset: Any) -> bytes: serializedDataset : `bytes` Bytes representing the serialized dataset. """ - raise NotImplementedError("Type does not support writing to bytes.") + raise FormatterNotImplementedError("Type does not support writing to bytes.") @contextlib.contextmanager def _updateLocation(self, location: Location | None) -> Iterator[Location]: @@ -1852,23 +1898,43 @@ def validate_write_recipes( # type: ignore # for a dynamic shim. Luckily the shim is only used as an instance. return self._formatter.validate_write_recipes(recipes) - def read_local_file(self, local_uri: ResourcePath, component: str | None = None) -> Any: + def read_from_local_file(self, local_uri: ResourcePath, component: str | None = None) -> Any: # Need to temporarily override the location since the V1 formatter # will not know anything about this local file. location = Location(*local_uri.split()) with self._formatter._updateLocation(location): - result = self._formatter.read(component=component) + try: + result = self._formatter.read(component=component) + except NotImplementedError as e: + # V1 raises NotImplementedError but V2 is expecting something + # slightly different. + raise FormatterNotImplementedError(str(e)) from e return result def read_from_bytes(self, serialized_bytes: bytes, component: str | None = None) -> Any: - return self._formatter.fromBytes(serialized_bytes, component=component) + try: + return self._formatter.fromBytes(serialized_bytes, component=component) + except NotImplementedError as e: + # V1 raises NotImplementedError but V2 is expecting something + # slightly different. + raise FormatterNotImplementedError(str(e)) from e def to_bytes(self, in_memory_dataset: Any) -> bytes: - return self._formatter.toBytes(in_memory_dataset) + try: + return self._formatter.toBytes(in_memory_dataset) + except NotImplementedError as e: + # V1 raises NotImplementedError but V2 is expecting something + # slightly different. + raise FormatterNotImplementedError(str(e)) from e def write_local_file(self, in_memory_dataset: Any, uri: ResourcePath) -> None: with self._formatter._updateLocation(Location(None, uri)): - self._formatter.write(in_memory_dataset) + try: + self._formatter.write(in_memory_dataset) + except NotImplementedError as e: + # V1 raises NotImplementedError but V2 is expecting something + # slightly different. + raise FormatterNotImplementedError(str(e)) from e # Type to use when allowing a Formatter or its class name diff --git a/python/lsst/daf/butler/formatters/astropyTable.py b/python/lsst/daf/butler/formatters/astropyTable.py index 784046e6c1..260ddf9023 100644 --- a/python/lsst/daf/butler/formatters/astropyTable.py +++ b/python/lsst/daf/butler/formatters/astropyTable.py @@ -54,7 +54,7 @@ def get_write_extension(self) -> str: # Other supported formats can be added here raise RuntimeError(f"Requested file format '{format}' is not supported for Table") - def read_local_file(self, local_uri: ResourcePath, component: str | None = None) -> Any: + def read_from_local_file(self, local_uri: ResourcePath, component: str | None = None) -> Any: pytype = self.file_descriptor.storageClass.pytype if not issubclass(pytype, astropy.table.Table): raise TypeError(f"Python type {pytype} does not seem to be a astropy Table type") diff --git a/python/lsst/daf/butler/formatters/json.py b/python/lsst/daf/butler/formatters/json.py index f9d0f386ca..19acd4a353 100644 --- a/python/lsst/daf/butler/formatters/json.py +++ b/python/lsst/daf/butler/formatters/json.py @@ -43,11 +43,10 @@ class JsonFormatter(TypelessFormatter): """Read and write JSON files.""" - allow_remote_file_read = True default_extension = ".json" unsupported_parameters = None - def read_cached_file(self, uri: ResourcePath, component: str | None = None) -> Any: + def read_from_uri(self, uri: ResourcePath, component: str | None = None) -> Any: # json.load() reads the entire file content into memory # and is no different from json.loads(uri.read()). It does not attempt # to support incremental reading to minimize memory usage. diff --git a/python/lsst/daf/butler/formatters/logs.py b/python/lsst/daf/butler/formatters/logs.py index 60b1c52ed3..f38153f238 100644 --- a/python/lsst/daf/butler/formatters/logs.py +++ b/python/lsst/daf/butler/formatters/logs.py @@ -43,12 +43,14 @@ class ButlerLogRecordsFormatter(FormatterV2): model. In the future this may be changed to be able to read `ButlerLogRecord` one at time from the file and return a subset of records given some filtering parameters. - """ - # Log files can be large and ResourcePath.open() does not support - # readline() or __iter__ in all cases and ButlerLogRecords.from_stream - # does not use `.read()` for chunking. Therefore must use local file. - allow_remote_file_read = False + Notes + ----- + Log files can be large and ResourcePath.open() does not support + ``readline()`` or ``__iter__`` in all cases and + ``ButlerLogRecords.from_stream`` does not use `.read()` for chunking. + Therefore must use local file. + """ default_extension = ".json" supported_extensions = frozenset({".log"}) @@ -60,7 +62,7 @@ def _get_read_pytype(self) -> type[ButlerLogRecords]: raise RuntimeError(f"Python type {pytype} does not seem to be a ButlerLogRecords type") return pytype - def read_local_file(self, uri: ResourcePath, component: str | None = None) -> Any: + def read_from_local_file(self, uri: ResourcePath, component: str | None = None) -> Any: # ResourcePath open() cannot do a per-line read. return self._get_read_pytype().from_file(uri.ospath) diff --git a/python/lsst/daf/butler/formatters/packages.py b/python/lsst/daf/butler/formatters/packages.py index d680c31b7b..562bb57c1d 100644 --- a/python/lsst/daf/butler/formatters/packages.py +++ b/python/lsst/daf/butler/formatters/packages.py @@ -45,7 +45,6 @@ class PackagesFormatterV2(FormatterV2): options are ``yaml``, ``json``, and ``pickle``. """ - allow_remote_file_read = True supported_write_parameters = frozenset({"format"}) supported_extensions = frozenset({".yaml", ".pickle", ".pkl", ".json"}) @@ -57,7 +56,7 @@ def get_write_extension(self) -> str: raise RuntimeError(f"Requested file format '{format}' is not supported for Packages") return ext - def read_cached_file(self, uri: ResourcePath, component: str | None = None) -> Any: + def read_from_uri(self, uri: ResourcePath, component: str | None = None) -> Any: # Read the full file using the class associated with the # storage class it was originally written with. # Read the bytes directly from resource. These are not going to be diff --git a/python/lsst/daf/butler/formatters/pickle.py b/python/lsst/daf/butler/formatters/pickle.py index 7c0d88e43a..438b3bc1db 100644 --- a/python/lsst/daf/butler/formatters/pickle.py +++ b/python/lsst/daf/butler/formatters/pickle.py @@ -45,11 +45,10 @@ class PickleFormatter(TypelessFormatter): files. """ - allow_remote_file_read = True default_extension = ".pickle" unsupported_parameters = None - def read_cached_file(self, uri: ResourcePath, component: str | None = None) -> Any: + def read_from_uri(self, uri: ResourcePath, component: str | None = None) -> Any: # Read the pickle file directly from the resource into memory. try: data = pickle.loads(uri.read()) diff --git a/python/lsst/daf/butler/formatters/yaml.py b/python/lsst/daf/butler/formatters/yaml.py index 8fd6c60d87..a4a772826f 100644 --- a/python/lsst/daf/butler/formatters/yaml.py +++ b/python/lsst/daf/butler/formatters/yaml.py @@ -43,12 +43,11 @@ class YamlFormatter(TypelessFormatter): """Read and write YAML files.""" - allow_remote_file_read = True default_extension = ".yaml" unsupported_parameters = None supported_write_parameters = frozenset({"unsafe_dump"}) - def read_cached_file(self, uri: ResourcePath, component: str | None = None) -> Any: + def read_from_uri(self, uri: ResourcePath, component: str | None = None) -> Any: # Can not use ResourcePath.open() data = yaml.safe_load(uri.read()) return data @@ -86,7 +85,7 @@ def to_bytes(self, in_memory_dataset: Any) -> bytes: """ converted = False if hasattr(in_memory_dataset, "model_dump") and hasattr(in_memory_dataset, "model_dump_json"): - # Pydantic v2-like model if both model_dump() and model_json() + # Pydantic v2-like model if both model_dump() and model_dump_json() # exist. with contextlib.suppress(Exception): in_memory_dataset = in_memory_dataset.model_dump() @@ -105,10 +104,12 @@ def to_bytes(self, in_memory_dataset: Any) -> bytes: in_memory_dataset = in_memory_dataset._asdict() unsafe_dump = self.write_parameters.get("unsafe_dump", False) + # Now that Python always uses an order dict, do not sort keys + # on write so that order can be preserved on read. if unsafe_dump: - serialized = yaml.dump(in_memory_dataset) + serialized = yaml.dump(in_memory_dataset, sort_keys=False) else: - serialized = yaml.safe_dump(in_memory_dataset) + serialized = yaml.safe_dump(in_memory_dataset, sort_keys=False) return serialized.encode() diff --git a/python/lsst/daf/butler/tests/_datasetsHelper.py b/python/lsst/daf/butler/tests/_datasetsHelper.py index 1d9ae9a2bb..f4076cce7f 100644 --- a/python/lsst/daf/butler/tests/_datasetsHelper.py +++ b/python/lsst/daf/butler/tests/_datasetsHelper.py @@ -39,7 +39,14 @@ from collections.abc import Iterable, Mapping from typing import TYPE_CHECKING, Any -from lsst.daf.butler import DataCoordinate, DatasetRef, DatasetType, DimensionGroup, StorageClass +from lsst.daf.butler import ( + DataCoordinate, + DatasetRef, + DatasetType, + DimensionGroup, + FormatterNotImplementedError, + StorageClass, +) from lsst.daf.butler.datastore import Datastore from lsst.daf.butler.formatters.yaml import YamlFormatter from lsst.resources import ResourcePath @@ -184,11 +191,11 @@ def makeDatastore(self, sub: str | None = None) -> Datastore: class BadWriteFormatter(YamlFormatter): """A formatter that never works but does leave a file behind.""" - def read_cached_file(self, uri: ResourcePath, component: str | None = None) -> Any: - raise NotImplementedError("This formatter can not read anything") + def read_from_uri(self, uri: ResourcePath, component: str | None = None) -> Any: + raise FormatterNotImplementedError("This formatter can not read anything") def to_bytes(self, in_memory_dataset: Any) -> bytes: - raise NotImplementedError("This formatter can not serialize to bytes.") + raise FormatterNotImplementedError("This formatter can not serialize to bytes.") def write_direct( self, @@ -199,22 +206,10 @@ def write_direct( uri.write(b"") raise RuntimeError("Did not succeed in writing file.") - def _readFile(self, path: str, pytype: type[Any] | None = None) -> Any: - raise NotImplementedError("This formatter can not read anything") - - def _writeFile(self, inMemoryDataset: Any) -> None: - """Write an empty file and then raise an exception.""" - with open(self.fileDescriptor.location.path, "wb"): - pass - raise RuntimeError("Did not succeed in writing file") - class BadNoWriteFormatter(BadWriteFormatter): """A formatter that always fails without writing anything.""" - def _writeFile(self, inMemoryDataset: Any) -> None: - raise RuntimeError("Did not writing anything at all") - def write_direct( self, in_memory_dataset: Any, @@ -227,7 +222,7 @@ def write_direct( class MultiDetectorFormatter(YamlFormatter): """A formatter that requires a detector to be specified in the dataID.""" - def read_cached_file(self, uri: ResourcePath, component: str | None = None) -> Any: + def read_from_uri(self, uri: ResourcePath, component: str | None = None) -> Any: if self.data_id is None: raise RuntimeError("This formatter requires a dataId") if "detector" not in self.data_id: @@ -235,23 +230,8 @@ def read_cached_file(self, uri: ResourcePath, component: str | None = None) -> A key = f"detector{self.data_id['detector']}" - data = super().read_cached_file(uri, component) + data = super().read_from_uri(uri, component) if key not in data: raise RuntimeError(f"Could not find '{key}' in data file.") return data[key] - - def _writeFile(self, inMemoryDataset: Any) -> None: - raise NotImplementedError("Can not write") - - def _fromBytes(self, serializedDataset: bytes, pytype: type[Any] | None = None) -> Any: - data = super()._fromBytes(serializedDataset) - if self.dataId is None: - raise RuntimeError("This formatter requires a dataId") - if "detector" not in self.dataId: - raise RuntimeError("This formatter requires detector to be present in dataId") - key = f"detector{self.dataId['detector']}" - assert pytype is not None - if key in data: - return pytype(data[key]) - raise RuntimeError(f"Could not find '{key}' in data file")