diff --git a/docs/python_api/datasets/biosample_index.md b/docs/python_api/datasets/biosample_index.md new file mode 100644 index 000000000..d3e4ee2c8 --- /dev/null +++ b/docs/python_api/datasets/biosample_index.md @@ -0,0 +1,9 @@ +--- +title: Biosample index +--- + +::: gentropy.dataset.biosample_index.BiosampleIndex + +## Schema + +--8<-- "assets/schemas/biosample_index.md" diff --git a/docs/python_api/datasources/_datasources.md b/docs/python_api/datasources/_datasources.md index e6e081b21..43b212e50 100644 --- a/docs/python_api/datasources/_datasources.md +++ b/docs/python_api/datasources/_datasources.md @@ -26,7 +26,7 @@ This section contains information about the data source harmonisation tools avai 2. GWAS catalog's [harmonisation pipeline](https://www.ebi.ac.uk/gwas/docs/methods/summary-statistics#_harmonised_summary_statistics_data) 3. Ensembl's [Variant Effect Predictor](https://www.ensembl.org/info/docs/tools/vep/index.html) -## Linkage desiquilibrium +## Linkage disequilibrium 1. [GnomAD](gnomad/_gnomad.md) v2.1.1 LD matrixes (7 ancestries) @@ -37,3 +37,8 @@ This section contains information about the data source harmonisation tools avai ## Gene annotation 1. [Open Targets Platform Target Dataset](open_targets/target.md) (derived from Ensembl) + +## Biological samples + +1. [Uberon](biosample_ontologies/_uberon.md) +2. [Cell Ontology](biosample_ontologies/_cell_ontology.md) diff --git a/docs/python_api/datasources/biosample_ontologies/_cell_ontology.md b/docs/python_api/datasources/biosample_ontologies/_cell_ontology.md new file mode 100644 index 000000000..5798e032b --- /dev/null +++ b/docs/python_api/datasources/biosample_ontologies/_cell_ontology.md @@ -0,0 +1,5 @@ +--- +title: Cell Ontology +--- + +The [Cell Ontology](http://www.obofoundry.org/ontology/cl.html) is a structured controlled vocabulary for cell types. It is used to annotate cell types in single-cell RNA-seq data and other omics data. diff --git a/docs/python_api/datasources/biosample_ontologies/_uberon.md b/docs/python_api/datasources/biosample_ontologies/_uberon.md new file mode 100644 index 000000000..4bb47305a --- /dev/null +++ b/docs/python_api/datasources/biosample_ontologies/_uberon.md @@ -0,0 +1,5 @@ +--- +title: Uberon +--- + +The [Uberon](http://uberon.github.io/) ontology is a multi-species anatomy ontology that integrates cross-species ontologies into a single ontology. diff --git a/docs/python_api/steps/biosample_index_step.md b/docs/python_api/steps/biosample_index_step.md new file mode 100644 index 000000000..d8f7abbb4 --- /dev/null +++ b/docs/python_api/steps/biosample_index_step.md @@ -0,0 +1,5 @@ +--- +title: biosample_index +--- + +::: gentropy.biosample_index.BiosampleIndexStep diff --git a/poetry.lock b/poetry.lock index 226311a8b..296f07145 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 1.8.3 and should not be changed by hand. +# This file is automatically @generated by Poetry 1.8.0 and should not be changed by hand. [[package]] name = "aiodns" @@ -3952,6 +3952,7 @@ files = [ {file = "PyYAML-6.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:bf07ee2fef7014951eeb99f56f39c9bb4af143d8aa3c21b1677805985307da34"}, {file = "PyYAML-6.0.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:855fb52b0dc35af121542a76b9a84f8d1cd886ea97c84703eaa6d88e37a2ad28"}, {file = "PyYAML-6.0.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:40df9b996c2b73138957fe23a16a4f0ba614f4c0efce1e9406a184b6d07fa3a9"}, + {file = "PyYAML-6.0.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a08c6f0fe150303c1c6b71ebcd7213c2858041a7e01975da3a99aed1e7a378ef"}, {file = "PyYAML-6.0.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6c22bec3fbe2524cde73d7ada88f6566758a8f7227bfbf93a408a9d86bcc12a0"}, {file = "PyYAML-6.0.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:8d4e9c88387b0f5c7d5f281e55304de64cf7f9c0021a3525bd3b1c542da3b0e4"}, {file = "PyYAML-6.0.1-cp312-cp312-win32.whl", hash = "sha256:d483d2cdf104e7c9fa60c544d92981f12ad66a457afae824d146093b8c294c54"}, diff --git a/src/gentropy/assets/schemas/biosample_index.json b/src/gentropy/assets/schemas/biosample_index.json new file mode 100644 index 000000000..1d68762ac --- /dev/null +++ b/src/gentropy/assets/schemas/biosample_index.json @@ -0,0 +1,83 @@ +{ + "type": "struct", + "fields": [ + { + "name": "biosampleId", + "type": "string", + "nullable": false, + "metadata": {} + }, + { + "name": "biosampleName", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "description", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "xrefs", + "type": { + "type": "array", + "elementType": "string", + "containsNull": true + }, + "nullable": true, + "metadata": {} + }, + { + "name": "synonyms", + "type": { + "type": "array", + "elementType": "string", + "containsNull": true + }, + "nullable": true, + "metadata": {} + }, + { + "name": "parents", + "type": { + "type": "array", + "elementType": "string", + "containsNull": true + }, + "nullable": true, + "metadata": {} + }, + { + "name": "ancestors", + "type": { + "type": "array", + "elementType": "string", + "containsNull": true + }, + "nullable": true, + "metadata": {} + }, + { + "name": "descendants", + "type": { + "type": "array", + "elementType": "string", + "containsNull": true + }, + "nullable": true, + "metadata": {} + }, + { + "name": "children", + "type": { + "type": "array", + "elementType": "string", + "containsNull": true + }, + "nullable": true, + "metadata": {} + } + ] +} diff --git a/src/gentropy/biosample_index.py b/src/gentropy/biosample_index.py new file mode 100644 index 000000000..e85c2e135 --- /dev/null +++ b/src/gentropy/biosample_index.py @@ -0,0 +1,34 @@ +"""Step to generate biosample index dataset.""" +from __future__ import annotations + +from gentropy.common.session import Session +from gentropy.datasource.biosample_ontologies.utils import extract_ontology_from_json + + +class BiosampleIndexStep: + """Biosample index step. + + This step generates a Biosample index dataset from the various ontology sources. Currently Cell Ontology and Uberon are supported. + """ + + def __init__( + self, + session: Session, + cell_ontology_input_path: str, + uberon_input_path: str, + biosample_index_path: str, + ) -> None: + """Run Biosample index generation step. + + Args: + session (Session): Session object. + cell_ontology_input_path (str): Input cell ontology dataset path. + uberon_input_path (str): Input uberon dataset path. + biosample_index_path (str): Output gene index dataset path. + """ + cell_ontology_index = extract_ontology_from_json(cell_ontology_input_path, session.spark) + uberon_index = extract_ontology_from_json(uberon_input_path, session.spark) + + biosample_index = cell_ontology_index.merge_indices([uberon_index]) + + biosample_index.df.write.mode(session.write_mode).parquet(biosample_index_path) diff --git a/src/gentropy/config.py b/src/gentropy/config.py index 4d1174d6b..a1d0cdfc6 100644 --- a/src/gentropy/config.py +++ b/src/gentropy/config.py @@ -51,6 +51,16 @@ class GeneIndexConfig(StepConfig): _target_: str = "gentropy.gene_index.GeneIndexStep" +@dataclass +class BiosampleIndexConfig(StepConfig): + """Biosample index step configuration.""" + + cell_ontology_input_path: str = MISSING + uberon_input_path: str = MISSING + biosample_index_path: str = MISSING + _target_: str = "gentropy.biosample_index.BiosampleIndexStep" + + @dataclass class GWASCatalogStudyCurationConfig(StepConfig): """GWAS Catalog study curation step configuration.""" @@ -505,6 +515,7 @@ class StudyValidationStepConfig(StepConfig): study_index_path: list[str] = MISSING target_index_path: str = MISSING disease_index_path: str = MISSING + biosample_index_path: str = MISSING valid_study_index_path: str = MISSING invalid_study_index_path: str = MISSING invalid_qc_reasons: list[str] = MISSING @@ -545,6 +556,7 @@ def register_config() -> None: cs.store(group="step", name="colocalisation", node=ColocalisationConfig) cs.store(group="step", name="eqtl_catalogue", node=EqtlCatalogueConfig) cs.store(group="step", name="gene_index", node=GeneIndexConfig) + cs.store(group="step", name="biosample_index", node=BiosampleIndexConfig) cs.store( group="step", name="gwas_catalog_study_curation", diff --git a/src/gentropy/dataset/biosample_index.py b/src/gentropy/dataset/biosample_index.py new file mode 100644 index 000000000..35c65a491 --- /dev/null +++ b/src/gentropy/dataset/biosample_index.py @@ -0,0 +1,73 @@ +"""Biosample index dataset.""" + +from __future__ import annotations + +from dataclasses import dataclass +from functools import reduce +from typing import TYPE_CHECKING + +import pyspark.sql.functions as f +from pyspark.sql import DataFrame +from pyspark.sql.types import ArrayType, StringType + +from gentropy.common.schemas import parse_spark_schema +from gentropy.dataset.dataset import Dataset + +if TYPE_CHECKING: + from pyspark.sql.types import StructType + + +@dataclass +class BiosampleIndex(Dataset): + """Biosample index dataset. + + A Biosample index dataset captures the metadata of the biosamples (e.g. tissues, cell types, cell lines, etc) such as alternate names and relationships with other biosamples. + """ + + @classmethod + def get_schema(cls: type[BiosampleIndex]) -> StructType: + """Provide the schema for the BiosampleIndex dataset. + + Returns: + StructType: The schema of the BiosampleIndex dataset. + """ + return parse_spark_schema("biosample_index.json") + + @classmethod + def merge_indices( + cls: type[BiosampleIndex], + biosample_indices : list[BiosampleIndex] + ) -> BiosampleIndex: + """Merge a list of biosample indices into a single biosample index. + + Where there are conflicts, in single values - the first value is taken. In list values, the union of all values is taken. + + Args: + biosample_indices (list[BiosampleIndex]): Biosample indices to merge. + + Returns: + BiosampleIndex: Merged biosample index. + """ + # Extract the DataFrames from the BiosampleIndex objects + biosample_dfs = [biosample_index.df for biosample_index in biosample_indices] + [cls.df] + + # Merge the DataFrames + merged_df = reduce(DataFrame.unionAll, biosample_dfs) + + # Determine aggregation functions for each column + # Currently this will take the first value for single values and merge lists for list values + agg_funcs = [] + for field in merged_df.schema.fields: + if field.name != "biosampleId": # Skip the grouping column + if field.dataType == ArrayType(StringType()): + agg_funcs.append(f.array_distinct(f.flatten(f.col(field.name))).alias(field.name)) + else: + agg_funcs.append(f.first(f.col(field.name), ignorenulls=True).alias(field.name)) + + # Perform aggregation + aggregated_df = merged_df.groupBy("biosampleId").agg(*agg_funcs) + + return BiosampleIndex( + _df=aggregated_df, + _schema=BiosampleIndex.get_schema() + ) diff --git a/src/gentropy/dataset/study_index.py b/src/gentropy/dataset/study_index.py index ac637f137..768a83dad 100644 --- a/src/gentropy/dataset/study_index.py +++ b/src/gentropy/dataset/study_index.py @@ -19,6 +19,7 @@ from pyspark.sql import Column, DataFrame from pyspark.sql.types import StructType + from gentropy.dataset.biosample_index import BiosampleIndex from gentropy.dataset.gene_index import GeneIndex @@ -29,6 +30,7 @@ class StudyQualityCheck(Enum): UNRESOLVED_TARGET (str): Target/gene identifier could not match to reference - Labelling failing target. UNRESOLVED_DISEASE (str): Disease identifier could not match to referece or retired identifier - labelling failing disease UNKNOWN_STUDY_TYPE (str): Indicating the provided type of study is not supported. + UNKNOWN_BIOSAMPLE (str): Flagging if a biosample identifier is not found in the reference. DUPLICATED_STUDY (str): Flagging if a study identifier is not unique. NO_GENE_PROVIDED (str): Flagging QTL studies if the measured """ @@ -36,6 +38,7 @@ class StudyQualityCheck(Enum): UNRESOLVED_TARGET = "Target/gene identifier could not match to reference." UNRESOLVED_DISEASE = "No valid disease identifier found." UNKNOWN_STUDY_TYPE = "This type of study is not supported." + UNKNOWN_BIOSAMPLE = "Biosample identifier was not found in the reference." DUPLICATED_STUDY = "The identifier of this study is not unique." NO_GENE_PROVIDED = "QTL study doesn't have gene assigned." @@ -408,3 +411,36 @@ def validate_target(self: StudyIndex, target_index: GeneIndex) -> StudyIndex: ) return StudyIndex(_df=validated_df, _schema=StudyIndex.get_schema()) + + def validate_biosample(self: StudyIndex, biosample_index: BiosampleIndex) -> StudyIndex: + """Validating biosample identifiers in the study index against the provided biosample index. + + Args: + biosample_index (BiosampleIndex): Biosample index containing a reference of biosample identifiers e.g. cell types, tissues, cell lines, etc. + + Returns: + StudyIndex: with flagged studies if biosampleIndex could not be validated. + """ + biosample_set = biosample_index.df.select("biosampleId", f.lit(True).alias("isIdFound")) + + validated_df = ( + self.df.join(biosample_set, self.df.biosampleFromSourceId == biosample_set.biosampleId, how="left") + .withColumn( + "isIdFound", + f.when( + f.col("isIdFound").isNull(), + f.lit(False), + ).otherwise(f.lit(True)), + ) + .withColumn( + "qualityControls", + StudyIndex.update_quality_flag( + f.col("qualityControls"), + ~f.col("isIdFound"), + StudyQualityCheck.UNKNOWN_BIOSAMPLE, + ), + ) + .drop("isIdFound").drop("biosampleId") + ) + + return StudyIndex(_df=validated_df, _schema=StudyIndex.get_schema()) diff --git a/src/gentropy/datasource/biosample_ontologies/__init__.py b/src/gentropy/datasource/biosample_ontologies/__init__.py new file mode 100644 index 000000000..d3fa6b416 --- /dev/null +++ b/src/gentropy/datasource/biosample_ontologies/__init__.py @@ -0,0 +1,3 @@ +"""Biosample index data source.""" + +from __future__ import annotations diff --git a/src/gentropy/datasource/biosample_ontologies/utils.py b/src/gentropy/datasource/biosample_ontologies/utils.py new file mode 100644 index 000000000..e02c82cb7 --- /dev/null +++ b/src/gentropy/datasource/biosample_ontologies/utils.py @@ -0,0 +1,130 @@ +"""Utility functions for Biosample ontology processing.""" +from pyspark.sql import DataFrame, SparkSession +from pyspark.sql import functions as f +from pyspark.sql.types import ArrayType, StringType +from pyspark.sql.window import Window + +from gentropy.dataset.biosample_index import BiosampleIndex + + +def extract_ontology_from_json( + ontology_json : str, + spark : SparkSession +) -> BiosampleIndex: + """Extracts the ontology information from a JSON file. Currently only supports Uberon and Cell Ontology. + + Args: + ontology_json (str): Path to the JSON file containing the ontology information. + spark (SparkSession): Spark session. + + Returns: + BiosampleIndex: Parsed and annotated biosample index table. + """ + + def json_graph_traversal( + df : DataFrame, + node_col : str, + link_col: str, + traversal_type: str + ) -> DataFrame: + """Traverse a graph represented in a DataFrame to find all ancestors or descendants. + + Args: + df (DataFrame): DataFrame containing the graph data. + node_col (str): Column name for the node. + link_col (str): Column name for the link. + traversal_type (str): Type of traversal - "ancestors" or "descendants". + + Returns: + DataFrame: DataFrame with the result column added. + """ + # Collect graph data as a map + graph_map = df.select(node_col, link_col).rdd.collectAsMap() + broadcasted_graph = spark.sparkContext.broadcast(graph_map) + + def get_relationships( + node : str + ) -> list[str]: + """Get all relationships for a given node. + + Args: + node (str): Node ID. + + Returns: + list[str]: List of relationships. + """ + relationships = set() + stack = [node] + while stack: + current = stack.pop() + if current in broadcasted_graph.value: + current_links = broadcasted_graph.value[current] + stack.extend(current_links) + relationships.update(current_links) + return list(relationships) + + # Choose column name based on traversal type + result_col = "ancestors" if traversal_type == "ancestors" else "descendants" + + # Register the UDF based on traversal type + relationship_udf = f.udf(get_relationships, ArrayType(StringType())) + + # Apply the UDF to create the result column + return df.withColumn(result_col, relationship_udf(f.col(node_col))) + + # Load the JSON file + df = spark.read.json(ontology_json, multiLine=True) + + # Exploding the 'graphs' array to make individual records easier to access + df_graphs = df.select(f.explode_outer("graphs").alias("graph")) + + # Exploding the 'nodes' array within each graph + df_nodes = df_graphs.select( + f.col("graph.id").alias("graph_id"), + f.explode_outer("graph.nodes").alias("node")) + + # Exploding the 'edges' array within each graph for relationship data + df_edges = df_graphs.select( + f.col("graph.id").alias("graph_id"), + f.explode_outer("graph.edges").alias("edge") + ).select( + f.col("edge.sub").alias("subject"), + f.col("edge.pred").alias("predicate"), + f.col("edge.obj").alias("object") + ) + df_edges = df_edges.withColumn("subject", f.regexp_replace(f.col("subject"), "http://purl.obolibrary.org/obo/", "")) + df_edges = df_edges.withColumn("object", f.regexp_replace(f.col("object"), "http://purl.obolibrary.org/obo/", "")) + + # Extract the relevant information from the nodes + transformed_df = df_nodes.select( + f.regexp_replace(f.col("node.id"), "http://purl.obolibrary.org/obo/", "").alias("biosampleId"), + f.col("node.lbl").alias("biosampleName"), + f.col("node.meta.definition.val").alias("description"), + f.collect_set(f.col("node.meta.xrefs.val")).over(Window.partitionBy("node.id")).getItem(0).alias("xrefs"), + f.collect_set(f.col("node.meta.synonyms.val")).over(Window.partitionBy("node.id")).getItem(0).alias("synonyms")) + + + # Extract the relationships from the edges + # Prepare relationship-specific DataFrames + df_parents = df_edges.filter(f.col("predicate") == "is_a").select("subject", "object").withColumnRenamed("object", "parent") + df_children = df_edges.filter(f.col("predicate") == "is_a").select("object", "subject").withColumnRenamed("subject", "child") + + # Aggregate relationships back to nodes + df_parents_grouped = df_parents.groupBy("subject").agg(f.array_distinct(f.collect_list("parent")).alias("parents")) + df_children_grouped = df_children.groupBy("object").agg(f.array_distinct(f.collect_list("child")).alias("children")) + + # Get all ancestors + df_with_ancestors = json_graph_traversal(df_parents_grouped, "subject", "parents", "ancestors") + # Get all descendants + df_with_descendants = json_graph_traversal(df_children_grouped, "object", "children", "descendants") + + # Join the ancestor and descendant DataFrames + df_with_relationships = df_with_ancestors.join(df_with_descendants, df_with_ancestors.subject == df_with_descendants.object, "full_outer").withColumn("biosampleId", f.coalesce(df_with_ancestors.subject, df_with_descendants.object)).drop("subject", "object") + + # Join the original DataFrame with the relationship DataFrame + final_df = transformed_df.join(df_with_relationships, ["biosampleId"], "left") + + return BiosampleIndex( + _df=final_df, + _schema=BiosampleIndex.get_schema() + ) diff --git a/src/gentropy/study_validation.py b/src/gentropy/study_validation.py index 5bfb83fe0..6f905f89b 100644 --- a/src/gentropy/study_validation.py +++ b/src/gentropy/study_validation.py @@ -5,6 +5,7 @@ from pyspark.sql import functions as f from gentropy.common.session import Session +from gentropy.dataset.biosample_index import BiosampleIndex from gentropy.dataset.gene_index import GeneIndex from gentropy.dataset.study_index import StudyIndex @@ -22,6 +23,7 @@ def __init__( study_index_path: list[str], target_index_path: str, disease_index_path: str, + biosample_index_path: str, valid_study_index_path: str, invalid_study_index_path: str, invalid_qc_reasons: list[str] = [], @@ -33,12 +35,14 @@ def __init__( study_index_path (list[str]): Path to study index file. target_index_path (str): Path to target index file. disease_index_path (str): Path to disease index file. + biosample_index_path (str): Path to biosample index file. valid_study_index_path (str): Path to write the valid records. invalid_study_index_path (str): Path to write the output file. invalid_qc_reasons (list[str]): List of invalid quality check reason names from `StudyQualityCheck` (e.g. ['DUPLICATED_STUDY']). """ # Reading datasets: target_index = GeneIndex.from_parquet(session, target_index_path) + biosample_index = BiosampleIndex.from_parquet(session, biosample_index_path) # Reading disease index and pre-process. # This logic does not belong anywhere, but gentorpy has no disease dataset yet. disease_index = ( @@ -63,6 +67,7 @@ def __init__( .validate_study_type() # Flagging non-supported study types. .validate_target(target_index) # Flagging QTL studies with invalid targets .validate_disease(disease_index) # Flagging invalid EFOs + .validate_biosample(biosample_index) # Flagging studies with invalid biosamples ).persist() # we will need this for 2 types of outputs study_index_with_qc.valid_rows( diff --git a/tests/gentropy/conftest.py b/tests/gentropy/conftest.py index 629f3a505..35fcc6e24 100644 --- a/tests/gentropy/conftest.py +++ b/tests/gentropy/conftest.py @@ -13,6 +13,7 @@ from gentropy.common.Liftover import LiftOverSpark from gentropy.common.session import Session +from gentropy.dataset.biosample_index import BiosampleIndex from gentropy.dataset.colocalisation import Colocalisation from gentropy.dataset.gene_index import GeneIndex from gentropy.dataset.intervals import Intervals @@ -559,6 +560,35 @@ def mock_gene_index(spark: SparkSession) -> GeneIndex: return GeneIndex(_df=data_spec.build(), _schema=gi_schema) +@pytest.fixture() +def mock_biosample_index(spark: SparkSession) -> BiosampleIndex: + """Mock biosample index dataset.""" + bi_schema = BiosampleIndex.get_schema() + + # Makes arrays of varying length with random integers between 1 and 100 + array_expression = "transform(sequence(1, 1 + floor(rand() * 9)), x -> cast((rand() * 100) as int))" + + data_spec = ( + dg.DataGenerator( + spark, + rows=400, + partitions=4, + randomSeedMethod="hash_fieldname", + ) + .withSchema(bi_schema) + .withColumnSpec("biosampleName", percentNulls=0.1) + .withColumnSpec("description", percentNulls=0.1) + .withColumnSpec("xrefs", expr=array_expression, percentNulls=0.1) + .withColumnSpec("synonyms", expr=array_expression, percentNulls=0.1) + .withColumnSpec("parents", expr=array_expression, percentNulls=0.1) + .withColumnSpec("ancestors", expr=array_expression, percentNulls=0.1) + .withColumnSpec("descendants", expr=array_expression, percentNulls=0.1) + .withColumnSpec("children", expr=array_expression, percentNulls=0.1) + ) + + return BiosampleIndex(_df=data_spec.build(), _schema=bi_schema) + + @pytest.fixture() def liftover_chain_37_to_38(spark: SparkSession) -> LiftOverSpark: """Sample liftover chain file.""" diff --git a/tests/gentropy/data_samples/cell_ontology_sample.json b/tests/gentropy/data_samples/cell_ontology_sample.json new file mode 100644 index 000000000..5e73bfdee --- /dev/null +++ b/tests/gentropy/data_samples/cell_ontology_sample.json @@ -0,0 +1,351 @@ +{ + "graphs": [ + { + "id": "http://purl.obolibrary.org/obo/cl.json", + "meta": { + "basicPropertyValues": [ + { + "pred": "http://purl.obolibrary.org/obo/IAO_0000700", + "val": "http://purl.obolibrary.org/obo/CL_0000000" + }, + { + "pred": "http://purl.org/dc/elements/1.1/description", + "val": "An ontology of cell types." + }, + { + "pred": "http://purl.org/dc/elements/1.1/title", + "val": "Cell Ontology" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0001-5208-3432" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0001-9114-8737" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0001-9990-8331" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-2244-7917" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-6601-2165" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-7073-9172" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-8688-6599" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-9900-7880" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0003-1980-3228" + }, + { + "pred": "http://purl.org/dc/terms/license", + "val": "http://creativecommons.org/licenses/by/4.0/" + }, + { + "pred": "http://www.w3.org/2000/01/rdf-schema#comment", + "val": "See PMID:15693950, PMID:12799354, PMID:20123131, PMID:21208450; Contact Alexander Diehl, addiehl@buffalo.edu, university at buffalo." + }, + { + "pred": "http://www.w3.org/2002/07/owl#versionInfo", + "val": "2024-08-16" + } + ], + "version": "http://purl.obolibrary.org/obo/cl/releases/2024-08-16/cl.json" + }, + "nodes": [ + { + "id": "http://purl.obolibrary.org/obo/CL_0000653", + "lbl": "podocyte", + "type": "CLASS", + "meta": { + "definition": { + "val": "A specialized kidney epithelial cell, contained within a glomerulus, that contains \"feet\" that interdigitate with the \"feet\" of other podocytes.", + "xrefs": ["GOC:tfm", "https://doi.org/10.1101/2021.10.10.463829"] + }, + "subsets": [ + "http://purl.obolibrary.org/obo/cl#cellxgene_subset", + "http://purl.obolibrary.org/obo/uberon/core#human_reference_atlas" + ], + "synonyms": [ + { + "pred": "hasBroadSynonym", + "val": "epithelial cell of visceral layer of glomerular capsule", + "xrefs": ["FMA:70967"] + }, + { + "pred": "hasExactSynonym", + "val": "glomerular podocyte", + "xrefs": ["FMA:70967"] + }, + { + "pred": "hasExactSynonym", + "val": "glomerular visceral epithelial cell" + }, + { + "pred": "hasExactSynonym", + "val": "kidney podocyte" + }, + { + "pred": "hasExactSynonym", + "val": "renal podocyte" + } + ], + "xrefs": [ + { + "val": "BTO:0002295" + }, + { + "val": "FMA:70967" + }, + { + "val": "ZFA:0009285" + } + ], + "basicPropertyValues": [ + { + "pred": "http://purl.obolibrary.org/obo/RO_0002175", + "val": "http://purl.obolibrary.org/obo/NCBITaxon_9606" + }, + { + "pred": "http://www.w3.org/2000/01/rdf-schema#seeAlso", + "val": "https://github.com/obophenotype/cell-ontology/issues/1460" + } + ] + } + }, + { + "id": "http://purl.obolibrary.org/obo/CL_0000654", + "lbl": "primary oocyte", + "type": "CLASS", + "meta": { + "definition": { + "val": "A primary oocyte is an oocyte that has not completed female meosis I.", + "xrefs": ["GOC:tfm", "ISBN:0721662544"] + }, + "subsets": [ + "http://purl.obolibrary.org/obo/uberon/core#human_reference_atlas" + ], + "synonyms": [ + { + "pred": "hasRelatedSynonym", + "val": "primary oogonium" + } + ], + "xrefs": [ + { + "val": "BTO:0000512" + }, + { + "val": "FMA:18645" + } + ], + "basicPropertyValues": [ + { + "pred": "http://purl.obolibrary.org/obo/RO_0002175", + "val": "http://purl.obolibrary.org/obo/NCBITaxon_9606" + } + ] + } + }, + { + "id": "http://purl.obolibrary.org/obo/CL_0000655", + "lbl": "secondary oocyte", + "type": "CLASS", + "meta": { + "definition": { + "val": "A secondary oocyte is an oocyte that has not completed meiosis II.", + "xrefs": ["GOC:tfm", "ISBN:0721662544"] + }, + "synonyms": [ + { + "pred": "hasRelatedSynonym", + "val": "primary oogonium" + } + ], + "xrefs": [ + { + "val": "BTO:0003094" + }, + { + "val": "FMA:18646" + } + ] + } + }, + { + "id": "http://purl.obolibrary.org/obo/CL_0000656", + "lbl": "primary spermatocyte", + "type": "CLASS", + "meta": { + "definition": { + "val": "A diploid cell that has derived from a spermatogonium and can subsequently begin meiosis and divide into two haploid secondary spermatocytes.", + "xrefs": ["GOC:tfm", "ISBN:0721662544"] + }, + "xrefs": [ + { + "val": "BTO:0001115" + }, + { + "val": "CALOHA:TS-2194" + }, + { + "val": "FMA:72292" + } + ] + } + }, + { + "id": "http://purl.obolibrary.org/obo/CL_0000657", + "lbl": "secondary spermatocyte", + "type": "CLASS", + "meta": { + "definition": { + "val": "One of the two haploid cells into which a primary spermatocyte divides, and which in turn gives origin to spermatids.", + "xrefs": ["GOC:tfm", "ISBN:0721662544"] + }, + "xrefs": [ + { + "val": "BTO:0000709" + }, + { + "val": "CALOHA:TS-2195" + }, + { + "val": "FBbt:00004941" + }, + { + "val": "FMA:72293" + } + ] + } + }, + { + "id": "http://purl.obolibrary.org/obo/CL_0000658", + "lbl": "cuticle secreting cell", + "type": "CLASS", + "meta": { + "definition": { + "val": "An epithelial cell that secretes cuticle.", + "xrefs": ["GOC:tfm"] + } + } + }, + { + "id": "http://purl.obolibrary.org/obo/CL_0000659", + "lbl": "eggshell secreting cell", + "type": "CLASS", + "meta": { + "definition": { + "val": "An extracellular matrix secreting cell that secretes eggshell.", + "xrefs": ["GOC:tfm"] + } + } + }, + { + "id": "http://purl.obolibrary.org/obo/CL_1000451", + "lbl": "obsolete epithelial cell of visceral layer of glomerular capsule", + "type": "CLASS", + "meta": { + "basicPropertyValues": [ + { + "pred": "http://purl.obolibrary.org/obo/IAO_0100001", + "val": "http://purl.obolibrary.org/obo/CL_0000653" + } + ], + "deprecated": true + } + } + ], + "edges": [ + { + "sub": "http://purl.obolibrary.org/obo/UBERON_0005751", + "pred": "http://purl.obolibrary.org/obo/BFO_0000051", + "obj": "http://purl.obolibrary.org/obo/CL_0000653" + }, + { + "sub": "http://purl.obolibrary.org/obo/GO_1903210", + "pred": "http://purl.obolibrary.org/obo/BFO_0000066", + "obj": "http://purl.obolibrary.org/obo/CL_0000653" + }, + { + "sub": "http://purl.obolibrary.org/obo/GO_0090521", + "pred": "http://purl.obolibrary.org/obo/RO_0002565", + "obj": "http://purl.obolibrary.org/obo/CL_0000653" + }, + { + "sub": "http://purl.obolibrary.org/obo/GO_0072015", + "pred": "http://purl.obolibrary.org/obo/RO_0002296", + "obj": "http://purl.obolibrary.org/obo/CL_0000653" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_4030008", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_0000653" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0002525", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_0000653" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0002523", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_0000653" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0000653", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_0002522" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0000653", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_1000450" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0000653", + "pred": "http://purl.obolibrary.org/obo/BFO_0000050", + "obj": "http://purl.obolibrary.org/obo/UBERON_0005751" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0000655", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_0000023", + "meta": { + "basicPropertyValues": [ + { + "pred": "http://www.geneontology.org/formats/oboInOwl#is_inferred", + "val": "true" + } + ] + } + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0000655", + "pred": "http://purl.obolibrary.org/obo/CL_4030044", + "obj": "http://purl.obolibrary.org/obo/GO_0007147" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0000655", + "pred": "http://purl.obolibrary.org/obo/RO_0002202", + "obj": "http://purl.obolibrary.org/obo/CL_0000654" + } + ] + } + ] +} diff --git a/tests/gentropy/data_samples/uberon_sample.json b/tests/gentropy/data_samples/uberon_sample.json new file mode 100644 index 000000000..7dedfa23c --- /dev/null +++ b/tests/gentropy/data_samples/uberon_sample.json @@ -0,0 +1,889 @@ +{ + "graphs": [ + { + "id": "http://purl.obolibrary.org/obo/uberon.json", + "meta": { + "basicPropertyValues": [ + { + "pred": "http://purl.obolibrary.org/obo/IAO_0000700", + "val": "http://purl.obolibrary.org/obo/UBERON_0000104" + }, + { + "pred": "http://purl.obolibrary.org/obo/IAO_0000700", + "val": "http://purl.obolibrary.org/obo/UBERON_0001062" + }, + { + "pred": "http://purl.org/dc/elements/1.1/creator", + "val": "https://orcid.org/0000-0001-5839-6798" + }, + { + "pred": "http://purl.org/dc/elements/1.1/creator", + "val": "https://orcid.org/0000-0001-7972-3866" + }, + { + "pred": "http://purl.org/dc/elements/1.1/creator", + "val": "https://orcid.org/0000-0001-9114-8737" + }, + { + "pred": "http://purl.org/dc/elements/1.1/creator", + "val": "https://orcid.org/0000-0002-1810-9886" + }, + { + "pred": "http://purl.org/dc/elements/1.1/creator", + "val": "https://orcid.org/0000-0002-6601-2165" + }, + { + "pred": "http://purl.org/dc/elements/1.1/creator", + "val": "https://orcid.org/0000-0002-7356-1779" + }, + { + "pred": "http://purl.org/dc/elements/1.1/creator", + "val": "https://orcid.org/0000-0002-9611-1279" + }, + { + "pred": "http://purl.org/dc/elements/1.1/creator", + "val": "https://orcid.org/0000-0003-3162-7490" + }, + { + "pred": "http://purl.org/dc/elements/1.1/creator", + "val": "https://orcid.org/0000-0003-3308-6245" + }, + { + "pred": "http://purl.org/dc/elements/1.1/description", + "val": "Uberon is an integrated cross-species anatomy ontology representing a variety of entities classified according to traditional anatomical criteria such as structure, function and developmental lineage. The ontology includes comprehensive relationships to taxon-specific anatomical ontologies, allowing integration of functional, phenotype and expression data." + }, + { + "pred": "http://purl.org/dc/elements/1.1/publisher", + "val": "http://uberon.org" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://dbpedia.org" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://palaeos.com" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://www.brain-map.org" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://braininfo.rprc.washington.edu/" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://en.wikipedia.org/wiki/" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://ontology.neuinfo.org/NIF/BiomaterialEntities/NIF-GrossAnatomy.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://pons.incf.org/wiki/Common_Upper_Mammalian_Brain_Ontology_%28Cumbo%29" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/aao.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/aba.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/aeo.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/bila.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/bto.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/caro.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/cl.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/ehdaa2.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/emapa.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/fbbt.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/fma.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/go.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/hp.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/ma.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/mp.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/tao.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/vhog.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/vsao.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/wbbt.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/xao.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://purl.obolibrary.org/obo/zfa.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://uri.neuinfo.org/nif/nifstd" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://www.e-lico.eu/public/kupo/kupo.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://www.ebi.ac.uk/efo/efo.owl" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:0030229073 Invertebrate Zoology, Barnes" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:0073040584 Vertebrates, Kardong" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:0123813611 Comparative Anatomy and Histology: A Mouse and Human Atlas, Treuting and Dintzis" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:0226313379 Fins into Limbs: Evolution, Development, and Transformation, Hall" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:0443065837 Human embryology, Larsen" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:0471888893 Comparative Vertebrate Neuroanatomy: Evolution and Adaptation by Butler and Hodos" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:0683400088 Stedman's Medical Dictionary" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:1588900649 Color Atlas and Textbook of Human Anatomy: Nervous system and sensory organs By Werner Kahle, Michael Frotscher" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:1588903958 Principles and practice of pediatric neurosurgery By A. Leland Albright, P. David Adelson, Ian F. Pollack" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:1607950324 Craniofacial Embryogenetics & Development, 2nd edition, Sperber" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:978-0-12-369548-2 Principles of Developmental Genetics, Sally A Moody" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:9780120749034 The laboratory rat" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:9780397517251 Surgical anatomy of the hand and upper extremity. By James R. Doyle and Michael J. Botte" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:9780674021839 The Tree of Life - Guillaume Lecointre, Herve Le Guyader" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "ISBN:9780878932504 Developmental Biology" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "MESH" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "PMID:11433360 Placental development: lessons from mouse mutants" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "PMID:16417468 Forgotten and novel aspects in pancreas development, Pieler and Chen" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "aggregates AAO from 13:04:2012" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "aggregates TAO from 09:08:2012" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "aggregates VSAO from 16:07:2012" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://wiki.phenotypercn.org/wg/phenotypercn/index.php?title=Neural_Crest_Workshop" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "http://wiki.phenotypercn.org/wiki/August_2012_Notes" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "https://docs.google.com/document/d/16JZOuH9sh_a8uIXA4cqg0Q1H6MV5yCj3-rhuKsZoV_U/edit" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "https://docs.google.com/document/d/1MnUgispgGfNQoezYzWzzGTnkAnI0gzRnJIwdip6MMtw/edit" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "https://docs.google.com/document/d/1cPWBqrl_Qy7XHEWFqtR_PgQX61yRkgGuLaiDpnEXxkE/edit" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "https://docs.google.com/document/d/1r9kNPpFYGdu0SpJDLyFAVQczBlG0wAZCBMd18gG3Ot8/edit#" + }, + { + "pred": "http://purl.org/dc/elements/1.1/source", + "val": "https://docs.google.com/spreadsheet/ccc?key=0Aj8NJdyb-leqdDM0R3hTVTRHRExDVjRCSkZEbDc5N1E#gid=0" + }, + { + "pred": "http://purl.org/dc/elements/1.1/title", + "val": "Uber-anatomy ontology" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://github.com/orgs/pato-ontology/teams/pato-community" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0001-5889-4463" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0001-7433-0086" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0001-7476-6306" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0001-7920-5321" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0001-7958-3701" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0001-8682-8754" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0001-9107-0714" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0001-9990-8331" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-0819-0473" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-0956-8634" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-1112-5832" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-1572-1316" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-1604-3078" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-1615-2899" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-2061-091X" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-2244-7917" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-3437-3329" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-3467-2636" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-3734-1859" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-5111-7263" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-6490-7723" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-7073-9172" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-8406-3871" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-8455-3213" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-8688-6599" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-9415-5104" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-9818-3030" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0002-9900-7880" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0003-1980-3228" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0003-2105-2283" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0003-2338-2550" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0003-3691-0324" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://orcid.org/0000-0003-4423-4370" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://www.wikidata.org/wiki/Q11695472" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://www.wikidata.org/wiki/Q23809253" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://www.wikidata.org/wiki/Q4964264" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://www.wikidata.org/wiki/Q54985720" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://www.wikidata.org/wiki/Q6983890" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://www.wikidata.org/wiki/Q7650732" + }, + { + "pred": "http://purl.org/dc/terms/contributor", + "val": "https://www.wikidata.org/wiki/Q85793053" + }, + { + "pred": "http://purl.org/dc/terms/isReferencedBy", + "val": "http://genomebiology.com/2012/13/1/R5" + }, + { + "pred": "http://purl.org/dc/terms/isReferencedBy", + "val": "http://www.ncbi.nlm.nih.gov/pubmed/22293552" + }, + { + "pred": "http://purl.org/dc/terms/license", + "val": "http://creativecommons.org/licenses/by/3.0/" + }, + { + "pred": "http://usefulinc.com/ns/doap#GitRepository", + "val": "https://github.com/cmungall/uberon/" + }, + { + "pred": "http://usefulinc.com/ns/doap#SVNRepository", + "val": "https://obo.svn.sourceforge.net/svnroot/obo/uberon/" + }, + { + "pred": "http://usefulinc.com/ns/doap#bug-database", + "val": "https://github.com/obophenotype/uberon/issues/" + }, + { + "pred": "http://usefulinc.com/ns/doap#mailing-list", + "val": "https://lists.sourceforge.net/lists/listinfo/obo-anatomy" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#default-namespace", + "val": "uberon" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#hasOBOFormatVersion", + "val": "1.2" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-equivalent", + "val": "AEO" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-equivalent", + "val": "BILA" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-equivalent", + "val": "BSPO" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-equivalent", + "val": "CARO" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-equivalent", + "val": "GO" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-equivalent", + "val": "OG" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-equivalent", + "val": "VSAO" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-has-subclass", + "val": "EHDAA" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-has-subclass", + "val": "EV" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-has-subclass", + "val": "NCIT" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-has-subclass", + "val": "OGES" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-has-subclass", + "val": "SCTID" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-is_a", + "val": "BFO" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-is_a", + "val": "VHOG" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "AAO part_of NCBITaxon:8292" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "DHBA part_of NCBITaxon:9606" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "EHDAA2 part_of NCBITaxon:9606" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "EMAPA part_of NCBITaxon:10090" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "FBdv part_of NCBITaxon:7227" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "FMA part_of NCBITaxon:9606" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "HAO part_of NCBITaxon:7399" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "HBA part_of NCBITaxon:9606" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "HsapDv part_of NCBITaxon:9606" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "KUPO part_of NCBITaxon:9606" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "MA part_of NCBITaxon:10090" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "MFO part_of NCBITaxon:8089" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "MmusDv part_of NCBITaxon:10090" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "OlatDv part_of NCBITaxon:8089" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "PBA part_of NCBITaxon:9443" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "SPD part_of NCBITaxon:6893" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "TADS part_of NCBITaxon:6939" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "TAO part_of NCBITaxon:32443" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "TGMA part_of NCBITaxon:44484" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "WBbt part_of NCBITaxon:6237" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "WBls part_of NCBITaxon:6237" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "XAO part_of NCBITaxon:8353" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "ZFA part_of NCBITaxon:7954" + }, + { + "pred": "http://www.geneontology.org/formats/oboInOwl#treat-xrefs-as-reverse-genus-differentia", + "val": "ZFS part_of NCBITaxon:7954" + }, + { + "pred": "http://www.w3.org/2000/01/rdf-schema#comment", + "val": "Aurelie Comte, Bill Bug, Catherine Leroy, Duncan Davidson and Trish Whetzel are also contributors. However their ORCIDs were not found." + }, + { + "pred": "http://www.w3.org/2002/07/owl#versionInfo", + "val": "2024-09-03" + }, + { + "pred": "http://xmlns.com/foaf/0.1/homepage", + "val": "http://uberon.org" + } + ], + "version": "http://purl.obolibrary.org/obo/uberon/releases/2024-09-03/uberon.json" + }, + "nodes": [ + { + "id": "http://purl.obolibrary.org/obo/CL_1001593", + "lbl": "parathyroid glandular cell", + "type": "CLASS", + "meta": { + "definition": { + "val": "Glandular cell of parathyroid epithelium. Example: Parathyroid chief cell and parathyroid oxyphil cells.", + "xrefs": ["HPA:HPA", "NPX:PDR"] + }, + "synonyms": [ + { + "pred": "hasRelatedSynonym", + "val": "parathyroid gland glandular cell", + "xrefs": ["CALOHA:TS-1279"] + }, + { + "pred": "hasRelatedSynonym", + "val": "parathyroid gland glandular cells", + "xrefs": ["CALOHA:TS-1279"] + } + ], + "xrefs": [ + { + "val": "CALOHA:TS-1279" + } + ] + } + }, + { + "id": "http://purl.obolibrary.org/obo/CL_1001595", + "lbl": "rectum glandular cell", + "type": "CLASS", + "meta": { + "definition": { + "val": "Glandular cell of rectal epithelium. Example: Goblet cell; enterocytes or absorptive cells; enteroendocrine and M cells.", + "xrefs": ["NPX:PDR"] + }, + "synonyms": [ + { + "pred": "hasRelatedSynonym", + "val": "rectal glandular cell", + "xrefs": ["CALOHA:TS-1281"] + }, + { + "pred": "hasRelatedSynonym", + "val": "rectum glandular cells", + "xrefs": ["CALOHA:TS-1281"] + } + ], + "xrefs": [ + { + "val": "CALOHA:TS-1281" + } + ] + } + }, + { + "id": "http://purl.obolibrary.org/obo/CL_1001596", + "lbl": "salivary gland glandular cell", + "type": "CLASS", + "meta": { + "definition": { + "val": "Glandular cell of salivary gland. Example: Serous cells, mucous cells, cuboidal epithelial cells of the intercalated ducts, simple cuboidal epithelium of the striated ducts, epithelial cells of excretory ducts.", + "xrefs": ["HPA:HPA", "NPX:PDR"] + }, + "synonyms": [ + { + "pred": "hasRelatedSynonym", + "val": "salivary gland glandular cells", + "xrefs": ["CALOHA:TS-1282"] + } + ], + "xrefs": [ + { + "val": "CALOHA:TS-1282" + } + ] + } + }, + { + "id": "http://purl.obolibrary.org/obo/CL_0000653", + "lbl": "podocyte", + "type": "CLASS", + "meta": { + "definition": { + "val": "A specialized kidney epithelial cell, contained within a glomerulus, that contains \"feet\" that interdigitate with the \"feet\" of other podocytes.", + "xrefs": ["GOC:tfm", "https://doi.org/10.1101/2021.10.10.463829"] + }, + "subsets": [ + "http://purl.obolibrary.org/obo/cl#cellxgene_subset", + "http://purl.obolibrary.org/obo/uberon/core#human_reference_atlas" + ], + "synonyms": [ + { + "pred": "hasBroadSynonym", + "val": "epithelial cell of visceral layer of glomerular capsule", + "xrefs": ["FMA:70967"] + }, + { + "pred": "hasExactSynonym", + "val": "glomerular podocyte", + "xrefs": ["FMA:70967"] + }, + { + "pred": "hasExactSynonym", + "val": "glomerular visceral epithelial cell" + }, + { + "pred": "hasExactSynonym", + "val": "kidney podocyte" + }, + { + "pred": "hasExactSynonym", + "val": "renal podocyte" + } + ], + "xrefs": [ + { + "val": "BTO:0002295" + }, + { + "val": "FMA:70967" + } + ], + "basicPropertyValues": [ + { + "pred": "http://purl.obolibrary.org/obo/RO_0002175", + "val": "http://purl.obolibrary.org/obo/NCBITaxon_9606" + }, + { + "pred": "http://www.w3.org/2000/01/rdf-schema#seeAlso", + "val": "https://github.com/obophenotype/cell-ontology/issues/1460" + } + ] + } + } + ], + "edges": [ + { + "sub": "http://purl.obolibrary.org/obo/CL_1001596", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_0000150" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_1001596", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_0000152" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_1001596", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_0002251" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_1001596", + "pred": "http://purl.obolibrary.org/obo/BFO_0000050", + "obj": "http://purl.obolibrary.org/obo/UBERON_0001044" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_1001596", + "pred": "http://purl.obolibrary.org/obo/BFO_0000050", + "obj": "http://purl.obolibrary.org/obo/UBERON_0004809" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0002623", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_0000622", + "meta": { + "basicPropertyValues": [ + { + "pred": "http://www.geneontology.org/formats/oboInOwl#is_inferred", + "val": "true" + } + ] + } + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0002623", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_1001596" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0002623", + "pred": "http://purl.obolibrary.org/obo/BFO_0000050", + "obj": "http://purl.obolibrary.org/obo/UBERON_0001044" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0002623", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_0000622", + "meta": { + "basicPropertyValues": [ + { + "pred": "http://www.geneontology.org/formats/oboInOwl#is_inferred", + "val": "true" + } + ] + } + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0002623", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_1001596" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0002623", + "pred": "http://purl.obolibrary.org/obo/BFO_0000050", + "obj": "http://purl.obolibrary.org/obo/UBERON_0001044" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0000653", + "pred": "is_a", + "obj": "http://purl.obolibrary.org/obo/CL_1000450" + }, + { + "sub": "http://purl.obolibrary.org/obo/CL_0000653", + "pred": "http://purl.obolibrary.org/obo/BFO_0000050", + "obj": "http://purl.obolibrary.org/obo/UBERON_0005751" + } + ] + } + ] +} diff --git a/tests/gentropy/dataset/test_biosample_index.py b/tests/gentropy/dataset/test_biosample_index.py new file mode 100644 index 000000000..c647710d1 --- /dev/null +++ b/tests/gentropy/dataset/test_biosample_index.py @@ -0,0 +1,8 @@ +"""Tests on Biosample index.""" + +from gentropy.dataset.biosample_index import BiosampleIndex + + +def test_biosample_index_creation(mock_biosample_index: BiosampleIndex) -> None: + """Test biosample index creation with mock biosample index.""" + assert isinstance(mock_biosample_index, BiosampleIndex) diff --git a/tests/gentropy/datasource/biosample_ontologies/test_biosample_ontology.py b/tests/gentropy/datasource/biosample_ontologies/test_biosample_ontology.py new file mode 100644 index 000000000..b88623b0d --- /dev/null +++ b/tests/gentropy/datasource/biosample_ontologies/test_biosample_ontology.py @@ -0,0 +1,50 @@ +"""Tests for biosample index dataset.""" + +from __future__ import annotations + +from typing import TYPE_CHECKING + +from gentropy.dataset.biosample_index import BiosampleIndex +from gentropy.datasource.biosample_ontologies.utils import extract_ontology_from_json + +if TYPE_CHECKING: + from pyspark.sql import SparkSession + + +class TestOntologyParger: + """Testing ontology parser.""" + + SAMPLE_CELL_ONTOLOGY_PATH = "tests/gentropy/data_samples/cell_ontology_sample.json" + SAMPLE_UBERON_PATH = "tests/gentropy/data_samples/uberon_sample.json" + + def test_cell_ontology_parser( + self: TestOntologyParger, spark: SparkSession + ) -> None: + """Test cell ontology parser.""" + cell_ontology = extract_ontology_from_json( + self.SAMPLE_CELL_ONTOLOGY_PATH, spark + ) + assert isinstance( + cell_ontology, BiosampleIndex + ), "Cell ontology subset is not parsed correctly to BiosampleIndex." + + def test_uberon_parser(self: TestOntologyParger, spark: SparkSession) -> None: + """Test uberon parser.""" + uberon = extract_ontology_from_json(self.SAMPLE_UBERON_PATH, spark) + assert isinstance( + uberon, BiosampleIndex + ), "Uberon subset is not parsed correctly to BiosampleIndex." + + def test_merge_biosample_indices( + self: TestOntologyParger, spark: SparkSession + ) -> None: + """Test merging of biosample indices.""" + cell_ontology = extract_ontology_from_json( + self.SAMPLE_CELL_ONTOLOGY_PATH, spark + ) + uberon = extract_ontology_from_json(self.SAMPLE_UBERON_PATH, spark) + + merged = cell_ontology.merge_indices([uberon]) + assert isinstance( + merged, BiosampleIndex + ), "Merging of biosample indices is not correct."