diff --git a/swh/indexer/cli.py b/swh/indexer/cli.py index 02dedbe..2c909b9 100644 --- a/swh/indexer/cli.py +++ b/swh/indexer/cli.py @@ -1,388 +1,397 @@ # Copyright (C) 2019-2022 The Software Heritage developers # See the AUTHORS file at the top-level directory of this distribution # License: GNU General Public License version 3, or any later version # See top-level LICENSE file for more information from typing import Callable, Dict, Iterator, List, Optional # WARNING: do not import unnecessary things here to keep cli startup time under # control import click from swh.core.cli import CONTEXT_SETTINGS, AliasedGroup from swh.core.cli import swh as swh_cli_group @swh_cli_group.group( name="indexer", context_settings=CONTEXT_SETTINGS, cls=AliasedGroup ) @click.option( "--config-file", "-C", default=None, type=click.Path( exists=True, dir_okay=False, ), help="Configuration file.", ) @click.pass_context def indexer_cli_group(ctx, config_file): """Software Heritage Indexer tools. The Indexer is used to mine the content of the archive and extract derived information from archive source code artifacts. """ from swh.core import config ctx.ensure_object(dict) conf = config.read(config_file) ctx.obj["config"] = conf def _get_api(getter, config, config_key, url): if url: config[config_key] = {"cls": "remote", "url": url} elif config_key not in config: raise click.ClickException("Missing configuration for {}".format(config_key)) return getter(**config[config_key]) @indexer_cli_group.group("mapping") def mapping(): """Manage Software Heritage Indexer mappings.""" pass @mapping.command("list") def mapping_list(): """Prints the list of known mappings.""" from swh.indexer import metadata_dictionary mapping_names = [mapping.name for mapping in metadata_dictionary.MAPPINGS.values()] mapping_names.sort() for mapping_name in mapping_names: click.echo(mapping_name) @mapping.command("list-terms") @click.option( "--exclude-mapping", multiple=True, help="Exclude the given mapping from the output" ) @click.option( "--concise", is_flag=True, default=False, help="Don't print the list of mappings supporting each term.", ) def mapping_list_terms(concise, exclude_mapping): """Prints the list of known CodeMeta terms, and which mappings support them.""" from swh.indexer import metadata_dictionary properties = metadata_dictionary.list_terms() for (property_name, supported_mappings) in sorted(properties.items()): supported_mappings = {m.name for m in supported_mappings} supported_mappings -= set(exclude_mapping) if supported_mappings: if concise: click.echo(property_name) else: click.echo("{}:".format(property_name)) click.echo("\t" + ", ".join(sorted(supported_mappings))) @mapping.command("translate") @click.argument("mapping-name") @click.argument("file", type=click.File("rb")) def mapping_translate(mapping_name, file): """Translates file from mapping-name to codemeta format.""" import json from swh.indexer import metadata_dictionary mapping_cls = [ cls for cls in metadata_dictionary.MAPPINGS.values() if cls.name == mapping_name ] if not mapping_cls: raise click.ClickException("Unknown mapping {}".format(mapping_name)) assert len(mapping_cls) == 1 mapping_cls = mapping_cls[0] mapping = mapping_cls() codemeta_doc = mapping.translate(file.read()) click.echo(json.dumps(codemeta_doc, indent=4)) @indexer_cli_group.group("schedule") @click.option("--scheduler-url", "-s", default=None, help="URL of the scheduler API") @click.option( "--indexer-storage-url", "-i", default=None, help="URL of the indexer storage API" ) @click.option( "--storage-url", "-g", default=None, help="URL of the (graph) storage API" ) @click.option( "--dry-run/--no-dry-run", is_flag=True, default=False, help="List only what would be scheduled.", ) @click.pass_context def schedule(ctx, scheduler_url, storage_url, indexer_storage_url, dry_run): """Manipulate Software Heritage Indexer tasks. Via SWH Scheduler's API.""" from swh.indexer.storage import get_indexer_storage from swh.scheduler import get_scheduler from swh.storage import get_storage ctx.obj["indexer_storage"] = _get_api( get_indexer_storage, ctx.obj["config"], "indexer_storage", indexer_storage_url ) ctx.obj["storage"] = _get_api( get_storage, ctx.obj["config"], "storage", storage_url ) ctx.obj["scheduler"] = _get_api( get_scheduler, ctx.obj["config"], "scheduler", scheduler_url ) if dry_run: ctx.obj["scheduler"] = None def list_origins_by_producer(idx_storage, mappings, tool_ids) -> Iterator[str]: next_page_token = "" limit = 10000 while next_page_token is not None: result = idx_storage.origin_intrinsic_metadata_search_by_producer( page_token=next_page_token, limit=limit, ids_only=True, mappings=mappings or None, tool_ids=tool_ids or None, ) next_page_token = result.next_page_token yield from result.results @schedule.command("reindex_origin_metadata") @click.option( "--batch-size", "-b", "origin_batch_size", default=10, show_default=True, type=int, help="Number of origins per task", ) @click.option( "--tool-id", "-t", "tool_ids", type=int, multiple=True, help="Restrict search of old metadata to this/these tool ids.", ) @click.option( "--mapping", "-m", "mappings", multiple=True, help="Mapping(s) that should be re-scheduled (eg. 'npm', 'gemspec', 'maven')", ) @click.option( "--task-type", default="index-origin-metadata", show_default=True, help="Name of the task type to schedule.", ) @click.pass_context def schedule_origin_metadata_reindex( ctx, origin_batch_size, tool_ids, mappings, task_type ): """Schedules indexing tasks for origins that were already indexed.""" from swh.scheduler.cli_utils import schedule_origin_batches idx_storage = ctx.obj["indexer_storage"] scheduler = ctx.obj["scheduler"] origins = list_origins_by_producer(idx_storage, mappings, tool_ids) kwargs = {"retries_left": 1} schedule_origin_batches(scheduler, task_type, origins, origin_batch_size, kwargs) @indexer_cli_group.command("journal-client") @click.argument( "indexer", type=click.Choice( [ "origin-intrinsic-metadata", "extrinsic-metadata", "content-mimetype", + "content-fossology-license", "*", ] ), required=False # TODO: remove required=False after we stop using it ) @click.option("--scheduler-url", "-s", default=None, help="URL of the scheduler API") @click.option( "--origin-metadata-task-type", default="index-origin-metadata", help="Name of the task running the origin metadata indexer.", ) @click.option( "--broker", "brokers", type=str, multiple=True, help="Kafka broker to connect to." ) @click.option( "--prefix", type=str, default=None, help="Prefix of Kafka topic names to read from." ) @click.option("--group-id", type=str, help="Consumer/group id for reading from Kafka.") @click.option( "--stop-after-objects", "-m", default=None, type=int, help="Maximum number of objects to replay. Default is to run forever.", ) @click.pass_context def journal_client( ctx, indexer: Optional[str], scheduler_url: str, origin_metadata_task_type: str, brokers: List[str], prefix: str, group_id: str, stop_after_objects: Optional[int], ): """ Listens for new objects from the SWH Journal, and either: * runs the indexer with the name passed as argument, if any * schedules tasks to run relevant indexers (currently, only origin-intrinsic-metadata) on these new objects otherwise. Passing '*' as indexer name runs all indexers. """ import functools import warnings from swh.indexer.indexer import BaseIndexer, ObjectsDict from swh.indexer.journal_client import process_journal_objects from swh.journal.client import get_journal_client from swh.scheduler import get_scheduler cfg = ctx.obj["config"] journal_cfg = cfg.get("journal", {}) scheduler = _get_api(get_scheduler, cfg, "scheduler", scheduler_url) brokers = brokers or journal_cfg.get("brokers") if not brokers: raise ValueError("The brokers configuration is mandatory.") prefix = prefix or journal_cfg.get("prefix") group_id = group_id or journal_cfg.get("group_id") origin_metadata_task_type = origin_metadata_task_type or journal_cfg.get( "origin_metadata_task_type" ) stop_after_objects = stop_after_objects or journal_cfg.get("stop_after_objects") object_types = set() worker_fns: List[Callable[[ObjectsDict], Dict]] = [] if indexer is None: warnings.warn( "'swh indexer journal-client' with no argument creates scheduler tasks " "to index, rather than index directly.", DeprecationWarning, ) object_types.add("origin_visit_status") worker_fns.append( functools.partial( process_journal_objects, scheduler=scheduler, task_names={ "origin_metadata": origin_metadata_task_type, }, ) ) idx: Optional[BaseIndexer] = None if indexer in ("origin-intrinsic-metadata", "*"): from swh.indexer.metadata import OriginMetadataIndexer object_types.add("origin_visit_status") idx = OriginMetadataIndexer() idx.catch_exceptions = False # don't commit offsets if indexation failed worker_fns.append(idx.process_journal_objects) if indexer in ("extrinsic-metadata", "*"): from swh.indexer.metadata import ExtrinsicMetadataIndexer object_types.add("raw_extrinsic_metadata") idx = ExtrinsicMetadataIndexer() idx.catch_exceptions = False # don't commit offsets if indexation failed worker_fns.append(idx.process_journal_objects) if indexer in ("content-mimetype", "*"): from swh.indexer.mimetype import MimetypeIndexer object_types.add("content") idx = MimetypeIndexer() idx.catch_exceptions = False # don't commit offsets if indexation failed worker_fns.append(idx.process_journal_objects) + if indexer in ("content-fossology-license", "*"): + from swh.indexer.fossology_license import FossologyLicenseIndexer + + object_types.add("content") + idx = FossologyLicenseIndexer() + idx.catch_exceptions = False # don't commit offsets if indexation failed + worker_fns.append(idx.process_journal_objects) + if not worker_fns: raise click.ClickException(f"Unknown indexer: {indexer}") client = get_journal_client( cls="kafka", brokers=brokers, prefix=prefix, group_id=group_id, object_types=list(object_types), stop_after_objects=stop_after_objects, ) def worker_fn(objects: ObjectsDict): for fn in worker_fns: fn(objects) try: client.process(worker_fn) except KeyboardInterrupt: ctx.exit(0) else: print("Done.") finally: client.close() @indexer_cli_group.command("rpc-serve") @click.argument("config-path", required=True) @click.option("--host", default="0.0.0.0", help="Host to run the server") @click.option("--port", default=5007, type=click.INT, help="Binding port of the server") @click.option( "--debug/--nodebug", default=True, help="Indicates if the server should run in debug mode", ) def rpc_server(config_path, host, port, debug): """Starts a Software Heritage Indexer RPC HTTP server.""" from swh.indexer.storage.api.server import app, load_and_check_config api_cfg = load_and_check_config(config_path, type="any") app.config.update(api_cfg) app.run(host, port=int(port), debug=bool(debug)) def main(): return indexer_cli_group(auto_envvar_prefix="SWH_INDEXER") if __name__ == "__main__": main() diff --git a/swh/indexer/tests/test_cli.py b/swh/indexer/tests/test_cli.py index fb78e88..1504374 100644 --- a/swh/indexer/tests/test_cli.py +++ b/swh/indexer/tests/test_cli.py @@ -1,819 +1,908 @@ # Copyright (C) 2019-2022 The Software Heritage developers # See the AUTHORS file at the top-level directory of this distribution # License: GNU General Public License version 3, or any later version # See top-level LICENSE file for more information import datetime from functools import reduce import re from typing import Any, Dict, List from unittest.mock import patch import attr from click.testing import CliRunner from confluent_kafka import Consumer import pytest +from swh.indexer import fossology_license from swh.indexer.cli import indexer_cli_group from swh.indexer.storage.interface import IndexerStorageInterface from swh.indexer.storage.model import ( + ContentLicenseRow, ContentMimetypeRow, DirectoryIntrinsicMetadataRow, OriginExtrinsicMetadataRow, OriginIntrinsicMetadataRow, ) from swh.journal.writer import get_journal_writer from swh.model.hashutil import hash_to_bytes from swh.model.model import Content, Origin, OriginVisitStatus from .test_metadata import REMD -from .utils import DIRECTORY2, RAW_CONTENTS, REVISION +from .utils import ( + DIRECTORY2, + RAW_CONTENT_IDS, + RAW_CONTENTS, + REVISION, + SHA1_TO_LICENSES, + mock_compute_license, +) def fill_idx_storage(idx_storage: IndexerStorageInterface, nb_rows: int) -> List[int]: tools: List[Dict[str, Any]] = [ { "tool_name": "tool %d" % i, "tool_version": "0.0.1", "tool_configuration": {}, } for i in range(2) ] tools = idx_storage.indexer_configuration_add(tools) origin_metadata = [ OriginIntrinsicMetadataRow( id="file://dev/%04d" % origin_id, from_directory=hash_to_bytes("abcd{:0>36}".format(origin_id)), indexer_configuration_id=tools[origin_id % 2]["id"], metadata={"name": "origin %d" % origin_id}, mappings=["mapping%d" % (origin_id % 10)], ) for origin_id in range(nb_rows) ] directory_metadata = [ DirectoryIntrinsicMetadataRow( id=hash_to_bytes("abcd{:0>36}".format(origin_id)), indexer_configuration_id=tools[origin_id % 2]["id"], metadata={"name": "origin %d" % origin_id}, mappings=["mapping%d" % (origin_id % 10)], ) for origin_id in range(nb_rows) ] idx_storage.directory_intrinsic_metadata_add(directory_metadata) idx_storage.origin_intrinsic_metadata_add(origin_metadata) return [tool["id"] for tool in tools] def _origins_in_task_args(tasks): """Returns the set of origins contained in the arguments of the provided tasks (assumed to be of type index-origin-metadata).""" return reduce( set.union, (set(task["arguments"]["args"][0]) for task in tasks), set() ) def _assert_tasks_for_origins(tasks, origins): expected_kwargs = {} assert {task["type"] for task in tasks} == {"index-origin-metadata"} assert all(len(task["arguments"]["args"]) == 1 for task in tasks) for task in tasks: assert task["arguments"]["kwargs"] == expected_kwargs, task assert _origins_in_task_args(tasks) == set(["file://dev/%04d" % i for i in origins]) @pytest.fixture def cli_runner(): return CliRunner() def test_cli_mapping_list(cli_runner, swh_config): result = cli_runner.invoke( indexer_cli_group, ["-C", swh_config, "mapping", "list"], catch_exceptions=False, ) expected_output = "\n".join( [ "cff", "codemeta", "composer", "gemspec", "github", "maven", "npm", "pkg-info", "pubspec", "", ] # must be sorted for test to pass ) assert result.exit_code == 0, result.output assert result.output == expected_output def test_cli_mapping_list_terms(cli_runner, swh_config): result = cli_runner.invoke( indexer_cli_group, ["-C", swh_config, "mapping", "list-terms"], catch_exceptions=False, ) assert result.exit_code == 0, result.output assert re.search(r"http://schema.org/url:\n.*npm", result.output) assert re.search(r"http://schema.org/url:\n.*codemeta", result.output) assert re.search( r"https://codemeta.github.io/terms/developmentStatus:\n\tcodemeta", result.output, ) def test_cli_mapping_list_terms_exclude(cli_runner, swh_config): result = cli_runner.invoke( indexer_cli_group, ["-C", swh_config, "mapping", "list-terms", "--exclude-mapping", "codemeta"], catch_exceptions=False, ) assert result.exit_code == 0, result.output assert re.search(r"http://schema.org/url:\n.*npm", result.output) assert not re.search(r"http://schema.org/url:\n.*codemeta", result.output) assert not re.search( r"https://codemeta.github.io/terms/developmentStatus:\n\tcodemeta", result.output, ) @patch("swh.scheduler.cli.utils.TASK_BATCH_SIZE", 3) @patch("swh.scheduler.cli_utils.TASK_BATCH_SIZE", 3) def test_cli_origin_metadata_reindex_empty_db( cli_runner, swh_config, indexer_scheduler, idx_storage, storage ): result = cli_runner.invoke( indexer_cli_group, [ "-C", swh_config, "schedule", "reindex_origin_metadata", ], catch_exceptions=False, ) expected_output = "Nothing to do (no origin metadata matched the criteria).\n" assert result.exit_code == 0, result.output assert result.output == expected_output tasks = indexer_scheduler.search_tasks() assert len(tasks) == 0 @patch("swh.scheduler.cli.utils.TASK_BATCH_SIZE", 3) @patch("swh.scheduler.cli_utils.TASK_BATCH_SIZE", 3) def test_cli_origin_metadata_reindex_divisor( cli_runner, swh_config, indexer_scheduler, idx_storage, storage ): """Tests the re-indexing when origin_batch_size*task_batch_size is a divisor of nb_origins.""" fill_idx_storage(idx_storage, 90) result = cli_runner.invoke( indexer_cli_group, [ "-C", swh_config, "schedule", "reindex_origin_metadata", ], catch_exceptions=False, ) # Check the output expected_output = ( "Scheduled 3 tasks (30 origins).\n" "Scheduled 6 tasks (60 origins).\n" "Scheduled 9 tasks (90 origins).\n" "Done.\n" ) assert result.exit_code == 0, result.output assert result.output == expected_output # Check scheduled tasks tasks = indexer_scheduler.search_tasks() assert len(tasks) == 9 _assert_tasks_for_origins(tasks, range(90)) @patch("swh.scheduler.cli.utils.TASK_BATCH_SIZE", 3) @patch("swh.scheduler.cli_utils.TASK_BATCH_SIZE", 3) def test_cli_origin_metadata_reindex_dry_run( cli_runner, swh_config, indexer_scheduler, idx_storage, storage ): """Tests the re-indexing when origin_batch_size*task_batch_size is a divisor of nb_origins.""" fill_idx_storage(idx_storage, 90) result = cli_runner.invoke( indexer_cli_group, [ "-C", swh_config, "schedule", "--dry-run", "reindex_origin_metadata", ], catch_exceptions=False, ) # Check the output expected_output = ( "Scheduled 3 tasks (30 origins).\n" "Scheduled 6 tasks (60 origins).\n" "Scheduled 9 tasks (90 origins).\n" "Done.\n" ) assert result.exit_code == 0, result.output assert result.output == expected_output # Check scheduled tasks tasks = indexer_scheduler.search_tasks() assert len(tasks) == 0 @patch("swh.scheduler.cli.utils.TASK_BATCH_SIZE", 3) @patch("swh.scheduler.cli_utils.TASK_BATCH_SIZE", 3) def test_cli_origin_metadata_reindex_nondivisor( cli_runner, swh_config, indexer_scheduler, idx_storage, storage ): """Tests the re-indexing when neither origin_batch_size or task_batch_size is a divisor of nb_origins.""" fill_idx_storage(idx_storage, 70) result = cli_runner.invoke( indexer_cli_group, [ "-C", swh_config, "schedule", "reindex_origin_metadata", "--batch-size", "20", ], catch_exceptions=False, ) # Check the output expected_output = ( "Scheduled 3 tasks (60 origins).\n" "Scheduled 4 tasks (70 origins).\n" "Done.\n" ) assert result.exit_code == 0, result.output assert result.output == expected_output # Check scheduled tasks tasks = indexer_scheduler.search_tasks() assert len(tasks) == 4 _assert_tasks_for_origins(tasks, range(70)) @patch("swh.scheduler.cli.utils.TASK_BATCH_SIZE", 3) @patch("swh.scheduler.cli_utils.TASK_BATCH_SIZE", 3) def test_cli_origin_metadata_reindex_filter_one_mapping( cli_runner, swh_config, indexer_scheduler, idx_storage, storage ): """Tests the re-indexing when origin_batch_size*task_batch_size is a divisor of nb_origins.""" fill_idx_storage(idx_storage, 110) result = cli_runner.invoke( indexer_cli_group, [ "-C", swh_config, "schedule", "reindex_origin_metadata", "--mapping", "mapping1", ], catch_exceptions=False, ) # Check the output expected_output = "Scheduled 2 tasks (11 origins).\nDone.\n" assert result.exit_code == 0, result.output assert result.output == expected_output # Check scheduled tasks tasks = indexer_scheduler.search_tasks() assert len(tasks) == 2 _assert_tasks_for_origins(tasks, [1, 11, 21, 31, 41, 51, 61, 71, 81, 91, 101]) @patch("swh.scheduler.cli.utils.TASK_BATCH_SIZE", 3) @patch("swh.scheduler.cli_utils.TASK_BATCH_SIZE", 3) def test_cli_origin_metadata_reindex_filter_two_mappings( cli_runner, swh_config, indexer_scheduler, idx_storage, storage ): """Tests the re-indexing when origin_batch_size*task_batch_size is a divisor of nb_origins.""" fill_idx_storage(idx_storage, 110) result = cli_runner.invoke( indexer_cli_group, [ "--config-file", swh_config, "schedule", "reindex_origin_metadata", "--mapping", "mapping1", "--mapping", "mapping2", ], catch_exceptions=False, ) # Check the output expected_output = "Scheduled 3 tasks (22 origins).\nDone.\n" assert result.exit_code == 0, result.output assert result.output == expected_output # Check scheduled tasks tasks = indexer_scheduler.search_tasks() assert len(tasks) == 3 _assert_tasks_for_origins( tasks, [ 1, 11, 21, 31, 41, 51, 61, 71, 81, 91, 101, 2, 12, 22, 32, 42, 52, 62, 72, 82, 92, 102, ], ) @patch("swh.scheduler.cli.utils.TASK_BATCH_SIZE", 3) @patch("swh.scheduler.cli_utils.TASK_BATCH_SIZE", 3) def test_cli_origin_metadata_reindex_filter_one_tool( cli_runner, swh_config, indexer_scheduler, idx_storage, storage ): """Tests the re-indexing when origin_batch_size*task_batch_size is a divisor of nb_origins.""" tool_ids = fill_idx_storage(idx_storage, 110) result = cli_runner.invoke( indexer_cli_group, [ "-C", swh_config, "schedule", "reindex_origin_metadata", "--tool-id", str(tool_ids[0]), ], catch_exceptions=False, ) # Check the output expected_output = ( "Scheduled 3 tasks (30 origins).\n" "Scheduled 6 tasks (55 origins).\n" "Done.\n" ) assert result.exit_code == 0, result.output assert result.output == expected_output # Check scheduled tasks tasks = indexer_scheduler.search_tasks() assert len(tasks) == 6 _assert_tasks_for_origins(tasks, [x * 2 for x in range(55)]) def now(): return datetime.datetime.now(tz=datetime.timezone.utc) def test_cli_journal_client_schedule( cli_runner, swh_config, indexer_scheduler, kafka_prefix: str, kafka_server, consumer: Consumer, ): """Test the 'swh indexer journal-client' cli tool.""" journal_writer = get_journal_writer( "kafka", brokers=[kafka_server], prefix=kafka_prefix, client_id="test producer", value_sanitizer=lambda object_type, value: value, flush_timeout=3, # fail early if something is going wrong ) visit_statuses = [ OriginVisitStatus( origin="file:///dev/zero", visit=1, date=now(), status="full", snapshot=None, ), OriginVisitStatus( origin="file:///dev/foobar", visit=2, date=now(), status="full", snapshot=None, ), OriginVisitStatus( origin="file:///tmp/spamegg", visit=3, date=now(), status="full", snapshot=None, ), OriginVisitStatus( origin="file:///dev/0002", visit=6, date=now(), status="full", snapshot=None, ), OriginVisitStatus( # will be filtered out due to its 'partial' status origin="file:///dev/0000", visit=4, date=now(), status="partial", snapshot=None, ), OriginVisitStatus( # will be filtered out due to its 'ongoing' status origin="file:///dev/0001", visit=5, date=now(), status="ongoing", snapshot=None, ), ] journal_writer.write_additions("origin_visit_status", visit_statuses) visit_statuses_full = [vs for vs in visit_statuses if vs.status == "full"] result = cli_runner.invoke( indexer_cli_group, [ "-C", swh_config, "journal-client", "--broker", kafka_server, "--prefix", kafka_prefix, "--group-id", "test-consumer", "--stop-after-objects", len(visit_statuses), "--origin-metadata-task-type", "index-origin-metadata", ], catch_exceptions=False, ) # Check the output expected_output = "Done.\n" assert result.exit_code == 0, result.output assert result.output == expected_output # Check scheduled tasks tasks = indexer_scheduler.search_tasks(task_type="index-origin-metadata") # This can be split into multiple tasks but no more than the origin-visit-statuses # written in the journal assert len(tasks) <= len(visit_statuses_full) actual_origins = [] for task in tasks: actual_task = dict(task) assert actual_task["type"] == "index-origin-metadata" scheduled_origins = actual_task["arguments"]["args"][0] actual_origins.extend(scheduled_origins) assert set(actual_origins) == {vs.origin for vs in visit_statuses_full} def test_cli_journal_client_without_brokers( cli_runner, swh_config, kafka_prefix: str, kafka_server, consumer: Consumer ): """Without brokers configuration, the cli fails.""" with pytest.raises(ValueError, match="brokers"): cli_runner.invoke( indexer_cli_group, [ "-C", swh_config, "journal-client", ], catch_exceptions=False, ) @pytest.mark.parametrize("indexer_name", ["origin-intrinsic-metadata", "*"]) def test_cli_journal_client_index__origin_intrinsic_metadata( cli_runner, swh_config, kafka_prefix: str, kafka_server, consumer: Consumer, idx_storage, storage, mocker, swh_indexer_config, indexer_name: str, ): """Test the 'swh indexer journal-client' cli tool.""" journal_writer = get_journal_writer( "kafka", brokers=[kafka_server], prefix=kafka_prefix, client_id="test producer", value_sanitizer=lambda object_type, value: value, flush_timeout=3, # fail early if something is going wrong ) visit_statuses = [ OriginVisitStatus( origin="file:///dev/zero", visit=1, date=now(), status="full", snapshot=None, ), OriginVisitStatus( origin="file:///dev/foobar", visit=2, date=now(), status="full", snapshot=None, ), OriginVisitStatus( origin="file:///tmp/spamegg", visit=3, date=now(), status="full", snapshot=None, ), OriginVisitStatus( origin="file:///dev/0002", visit=6, date=now(), status="full", snapshot=None, ), OriginVisitStatus( # will be filtered out due to its 'partial' status origin="file:///dev/0000", visit=4, date=now(), status="partial", snapshot=None, ), OriginVisitStatus( # will be filtered out due to its 'ongoing' status origin="file:///dev/0001", visit=5, date=now(), status="ongoing", snapshot=None, ), ] journal_writer.write_additions("origin_visit_status", visit_statuses) visit_statuses_full = [vs for vs in visit_statuses if vs.status == "full"] storage.revision_add([REVISION]) mocker.patch( "swh.indexer.metadata.get_head_swhid", return_value=REVISION.swhid(), ) mocker.patch( "swh.indexer.metadata.DirectoryMetadataIndexer.index", return_value=[ DirectoryIntrinsicMetadataRow( id=DIRECTORY2.id, indexer_configuration_id=1, mappings=["cff"], metadata={"foo": "bar"}, ) ], ) result = cli_runner.invoke( indexer_cli_group, [ "-C", swh_config, "journal-client", indexer_name, "--broker", kafka_server, "--prefix", kafka_prefix, "--group-id", "test-consumer", "--stop-after-objects", len(visit_statuses), ], catch_exceptions=False, ) # Check the output expected_output = "Done.\n" assert result.exit_code == 0, result.output assert result.output == expected_output results = idx_storage.origin_intrinsic_metadata_get( [status.origin for status in visit_statuses] ) expected_results = [ OriginIntrinsicMetadataRow( id=status.origin, from_directory=DIRECTORY2.id, tool={"id": 1, **swh_indexer_config["tools"]}, mappings=["cff"], metadata={"foo": "bar"}, ) for status in sorted(visit_statuses_full, key=lambda r: r.origin) ] assert sorted(results, key=lambda r: r.id) == expected_results @pytest.mark.parametrize("indexer_name", ["extrinsic-metadata", "*"]) def test_cli_journal_client_index__origin_extrinsic_metadata( cli_runner, swh_config, kafka_prefix: str, kafka_server, consumer: Consumer, idx_storage, storage, mocker, swh_indexer_config, indexer_name: str, ): """Test the 'swh indexer journal-client' cli tool.""" journal_writer = get_journal_writer( "kafka", brokers=[kafka_server], prefix=kafka_prefix, client_id="test producer", value_sanitizer=lambda object_type, value: value, flush_timeout=3, # fail early if something is going wrong ) origin = Origin("http://example.org/repo.git") storage.origin_add([origin]) raw_extrinsic_metadata = attr.evolve(REMD, target=origin.swhid()) raw_extrinsic_metadata = attr.evolve( raw_extrinsic_metadata, id=raw_extrinsic_metadata.compute_hash() ) journal_writer.write_additions("raw_extrinsic_metadata", [raw_extrinsic_metadata]) result = cli_runner.invoke( indexer_cli_group, [ "-C", swh_config, "journal-client", indexer_name, "--broker", kafka_server, "--prefix", kafka_prefix, "--group-id", "test-consumer", "--stop-after-objects", 1, ], catch_exceptions=False, ) # Check the output expected_output = "Done.\n" assert result.exit_code == 0, result.output assert result.output == expected_output results = idx_storage.origin_extrinsic_metadata_get([origin.url]) expected_results = [ OriginExtrinsicMetadataRow( id=origin.url, from_remd_id=raw_extrinsic_metadata.id, tool={"id": 1, **swh_indexer_config["tools"]}, mappings=["github"], metadata={ "@context": "https://doi.org/10.5063/schema/codemeta-2.0", "type": "https://forgefed.org/ns#Repository", "name": "test software", }, ) ] assert sorted(results, key=lambda r: r.id) == expected_results def test_cli_journal_client_index__content_mimetype( cli_runner, swh_config, kafka_prefix: str, kafka_server, consumer: Consumer, idx_storage, obj_storage, storage, mocker, swh_indexer_config, ): """Test the 'swh indexer journal-client' cli tool.""" journal_writer = get_journal_writer( "kafka", brokers=[kafka_server], prefix=kafka_prefix, client_id="test producer", value_sanitizer=lambda object_type, value: value, flush_timeout=3, # fail early if something is going wrong ) contents = [] expected_results = [] content_ids = [] for content_id, (raw_content, mimetypes, encoding) in RAW_CONTENTS.items(): content = Content.from_data(raw_content) assert content_id == content.sha1 contents.append(content) content_ids.append(content_id) # Older libmagic versions (e.g. buster: 1:5.35-4+deb10u2, bullseye: 1:5.39-3) # returns different results. This allows to deal with such a case when executing # tests on different environments machines (e.g. ci tox, ci debian, dev machine, # ...) all_mimetypes = mimetypes if isinstance(mimetypes, tuple) else [mimetypes] expected_results.extend( [ ContentMimetypeRow( id=content.sha1, tool={"id": 1, **swh_indexer_config["tools"]}, mimetype=mimetype, encoding=encoding, ) for mimetype in all_mimetypes ] ) assert len(contents) == len(RAW_CONTENTS) - storage.content_add(contents) journal_writer.write_additions("content", contents) result = cli_runner.invoke( indexer_cli_group, [ "-C", swh_config, "journal-client", "content-mimetype", "--broker", kafka_server, "--prefix", kafka_prefix, "--group-id", "test-consumer", "--stop-after-objects", len(contents), ], catch_exceptions=False, ) # Check the output expected_output = "Done.\n" assert result.exit_code == 0, result.output assert result.output == expected_output results = idx_storage.content_mimetype_get(content_ids) - assert len(results) > 0 + assert len(results) == len(contents) + for result in results: + assert result in expected_results + + +def test_cli_journal_client_index__fossology_license( + cli_runner, + swh_config, + kafka_prefix: str, + kafka_server, + consumer: Consumer, + idx_storage, + obj_storage, + storage, + mocker, + swh_indexer_config, +): + """Test the 'swh indexer journal-client' cli tool.""" + + # Patch + fossology_license.compute_license = mock_compute_license + + journal_writer = get_journal_writer( + "kafka", + brokers=[kafka_server], + prefix=kafka_prefix, + client_id="test producer", + value_sanitizer=lambda object_type, value: value, + flush_timeout=3, # fail early if something is going wrong + ) + + tool = {"id": 1, **swh_indexer_config["tools"]} + + id0, id1, id2 = RAW_CONTENT_IDS + + contents = [] + content_ids = [] + expected_results = [] + for content_id, (raw_content, _, _) in RAW_CONTENTS.items(): + content = Content.from_data(raw_content) + assert content_id == content.sha1 + + contents.append(content) + content_ids.append(content_id) + + expected_results.extend( + [ + ContentLicenseRow(id=content_id, tool=tool, license=license) + for license in SHA1_TO_LICENSES[content_id] + ] + ) + + assert len(contents) == len(RAW_CONTENTS) + + journal_writer.write_additions("content", contents) + + result = cli_runner.invoke( + indexer_cli_group, + [ + "-C", + swh_config, + "journal-client", + "content-fossology-license", + "--broker", + kafka_server, + "--prefix", + kafka_prefix, + "--group-id", + "test-consumer", + "--stop-after-objects", + len(contents), + ], + catch_exceptions=False, + ) + + # Check the output + expected_output = "Done.\n" + assert result.exit_code == 0, result.output + assert result.output == expected_output + + results = idx_storage.content_fossology_license_get(content_ids) + assert len(results) == len(expected_results) for result in results: assert result in expected_results diff --git a/swh/indexer/tests/test_fossology_license.py b/swh/indexer/tests/test_fossology_license.py index a38167b..079256d 100644 --- a/swh/indexer/tests/test_fossology_license.py +++ b/swh/indexer/tests/test_fossology_license.py @@ -1,159 +1,150 @@ # Copyright (C) 2017-2022 The Software Heritage developers # See the AUTHORS file at the top-level directory of this distribution # License: GNU General Public License version 3, or any later version # See top-level LICENSE file for more information from typing import Any, Dict import unittest from unittest.mock import patch import pytest from swh.indexer import fossology_license from swh.indexer.fossology_license import ( FossologyLicenseIndexer, FossologyLicensePartitionIndexer, compute_license, ) from swh.indexer.storage.model import ContentLicenseRow from swh.indexer.tests.utils import ( BASE_TEST_CONFIG, RAW_CONTENT_IDS, SHA1_TO_LICENSES, CommonContentIndexerPartitionTest, CommonContentIndexerTest, fill_obj_storage, fill_storage, filter_dict, + mock_compute_license, ) -from swh.model.hashutil import hash_to_bytes class BasicTest(unittest.TestCase): @patch("swh.indexer.fossology_license.subprocess") def test_compute_license(self, mock_subprocess): """Computing licenses from a raw content should return results""" for path, intermediary_result, output in [ (b"some/path", None, []), (b"some/path/2", [], []), (b"other/path", " contains license(s) GPL,AGPL", ["GPL", "AGPL"]), ]: mock_subprocess.check_output.return_value = intermediary_result actual_result = compute_license(path) self.assertEqual( actual_result, { "licenses": output, "path": path, }, ) -def mock_compute_license(path): - """path is the content identifier""" - if isinstance(id, bytes): - path = path.decode("utf-8") - # path is something like /tmp/tmpXXX/ so we keep only the sha1 part - id_ = path.split("/")[-1] - return {"licenses": SHA1_TO_LICENSES.get(hash_to_bytes(id_), [])} - - CONFIG = { **BASE_TEST_CONFIG, "workdir": "/tmp", "tools": { "name": "nomos", "version": "3.1.0rc2-31-ga2cbb8c", "configuration": { "command_line": "nomossa ", }, }, } # type: Dict[str, Any] RANGE_CONFIG = dict(list(CONFIG.items()) + [("write_batch_size", 100)]) class TestFossologyLicenseIndexer(CommonContentIndexerTest, unittest.TestCase): """Language indexer test scenarios: - Known sha1s in the input list have their data indexed - Unknown sha1 in the input list are not indexed """ def get_indexer_results(self, ids): yield from self.idx_storage.content_fossology_license_get(ids) def setUp(self): super().setUp() # replace actual license computation with a mock self.orig_compute_license = fossology_license.compute_license fossology_license.compute_license = mock_compute_license self.indexer = FossologyLicenseIndexer(CONFIG) self.indexer.catch_exceptions = False self.idx_storage = self.indexer.idx_storage fill_storage(self.indexer.storage) fill_obj_storage(self.indexer.objstorage) self.id0, self.id1, self.id2 = RAW_CONTENT_IDS tool = {k.replace("tool_", ""): v for (k, v) in self.indexer.tool.items()} # then self.expected_results = [ *[ ContentLicenseRow(id=self.id0, tool=tool, license=license) for license in SHA1_TO_LICENSES[self.id0] ], *[ ContentLicenseRow(id=self.id1, tool=tool, license=license) for license in SHA1_TO_LICENSES[self.id1] ], *[], # self.id2 ] def tearDown(self): super().tearDown() fossology_license.compute_license = self.orig_compute_license class TestFossologyLicensePartitionIndexer( CommonContentIndexerPartitionTest, unittest.TestCase ): """Range Fossology License Indexer tests. - new data within range are indexed - no data outside a range are indexed - with filtering existing indexed data prior to compute new index - without filtering existing indexed data prior to compute new index """ def setUp(self): super().setUp() # replace actual license computation with a mock self.orig_compute_license = fossology_license.compute_license fossology_license.compute_license = mock_compute_license self.indexer = FossologyLicensePartitionIndexer(config=RANGE_CONFIG) self.indexer.catch_exceptions = False fill_storage(self.indexer.storage) fill_obj_storage(self.indexer.objstorage) def tearDown(self): super().tearDown() fossology_license.compute_license = self.orig_compute_license def test_fossology_w_no_tool(): with pytest.raises(ValueError): FossologyLicenseIndexer(config=filter_dict(CONFIG, "tools")) def test_fossology_range_w_no_tool(): with pytest.raises(ValueError): FossologyLicensePartitionIndexer(config=filter_dict(RANGE_CONFIG, "tools")) diff --git a/swh/indexer/tests/utils.py b/swh/indexer/tests/utils.py index 1e80dc2..c5586b3 100644 --- a/swh/indexer/tests/utils.py +++ b/swh/indexer/tests/utils.py @@ -1,793 +1,802 @@ # Copyright (C) 2017-2022 The Software Heritage developers # See the AUTHORS file at the top-level directory of this distribution # License: GNU General Public License version 3, or any later version # See top-level LICENSE file for more information import abc import datetime import functools from typing import Any, Dict, List, Tuple import unittest from hypothesis import strategies from swh.core.api.classes import stream_results from swh.indexer.storage import INDEXER_CFG_KEY from swh.model.hashutil import hash_to_bytes from swh.model.model import ( Content, Directory, DirectoryEntry, ObjectType, Origin, OriginVisit, OriginVisitStatus, Person, Release, Revision, RevisionType, Snapshot, SnapshotBranch, TargetType, TimestampWithTimezone, ) from swh.storage.utils import now BASE_TEST_CONFIG: Dict[str, Dict[str, Any]] = { "storage": {"cls": "memory"}, "objstorage": {"cls": "memory"}, INDEXER_CFG_KEY: {"cls": "memory"}, } ORIGIN_VISITS = [ {"type": "git", "origin": "https://github.com/SoftwareHeritage/swh-storage"}, {"type": "ftp", "origin": "rsync://ftp.gnu.org/gnu/3dldf"}, { "type": "deposit", "origin": "https://forge.softwareheritage.org/source/jesuisgpl/", }, { "type": "pypi", "origin": "https://old-pypi.example.org/project/limnoria/", }, # with rev head {"type": "pypi", "origin": "https://pypi.org/project/limnoria/"}, # with rel head {"type": "svn", "origin": "http://0-512-md.googlecode.com/svn/"}, {"type": "git", "origin": "https://github.com/librariesio/yarn-parser"}, {"type": "git", "origin": "https://github.com/librariesio/yarn-parser.git"}, {"type": "git", "origin": "https://npm.example.org/yarn-parser"}, ] ORIGINS = [Origin(url=visit["origin"]) for visit in ORIGIN_VISITS] OBJ_STORAGE_RAW_CONTENT: Dict[str, bytes] = { "text:some": b"this is some text", "text:another": b"another text", "text:yet": b"yet another text", "python:code": b""" import unittest import logging from swh.indexer.mimetype import MimetypeIndexer from swh.indexer.tests.test_utils import MockObjStorage class MockStorage(): def content_mimetype_add(self, mimetypes): self.state = mimetypes def indexer_configuration_add(self, tools): return [{ 'id': 10, }] """, "c:struct": b""" #ifndef __AVL__ #define __AVL__ typedef struct _avl_tree avl_tree; typedef struct _data_t { int content; } data_t; """, "lisp:assertion": b""" (should 'pygments (recognize 'lisp 'easily)) """, "json:test-metadata-package.json": b""" { "name": "test_metadata", "version": "0.0.1", "description": "Simple package.json test for indexer", "repository": { "type": "git", "url": "https://github.com/moranegg/metadata_test" } } """, "json:npm-package.json": b""" { "version": "5.0.3", "name": "npm", "description": "a package manager for JavaScript", "keywords": [ "install", "modules", "package manager", "package.json" ], "preferGlobal": true, "config": { "publishtest": false }, "homepage": "https://docs.npmjs.com/", "author": "Isaac Z. Schlueter (http://blog.izs.me)", "repository": { "type": "git", "url": "https://github.com/npm/npm" }, "bugs": { "url": "https://github.com/npm/npm/issues" }, "dependencies": { "JSONStream": "~1.3.1", "abbrev": "~1.1.0", "ansi-regex": "~2.1.1", "ansicolors": "~0.3.2", "ansistyles": "~0.1.3" }, "devDependencies": { "tacks": "~1.2.6", "tap": "~10.3.2" }, "license": "Artistic-2.0" } """, "text:carriage-return": b""" """, "text:empty": b"", # was 626364 / b'bcd' "text:unimportant": b"unimportant content for bcd", # was 636465 / b'cde' now yarn-parser package.json "json:yarn-parser-package.json": b""" { "name": "yarn-parser", "version": "1.0.0", "description": "Tiny web service for parsing yarn.lock files", "main": "index.js", "scripts": { "start": "node index.js", "test": "mocha" }, "engines": { "node": "9.8.0" }, "repository": { "type": "git", "url": "git+https://github.com/librariesio/yarn-parser.git" }, "keywords": [ "yarn", "parse", "lock", "dependencies" ], "author": "Andrew Nesbitt", "license": "AGPL-3.0", "bugs": { "url": "https://github.com/librariesio/yarn-parser/issues" }, "homepage": "https://github.com/librariesio/yarn-parser#readme", "dependencies": { "@yarnpkg/lockfile": "^1.0.0", "body-parser": "^1.15.2", "express": "^4.14.0" }, "devDependencies": { "chai": "^4.1.2", "mocha": "^5.2.0", "request": "^2.87.0", "test": "^0.6.0" } } """, } MAPPING_DESCRIPTION_CONTENT_SHA1GIT: Dict[str, bytes] = {} MAPPING_DESCRIPTION_CONTENT_SHA1: Dict[str, bytes] = {} OBJ_STORAGE_DATA: Dict[bytes, bytes] = {} for key_description, data in OBJ_STORAGE_RAW_CONTENT.items(): content = Content.from_data(data) MAPPING_DESCRIPTION_CONTENT_SHA1GIT[key_description] = content.sha1_git MAPPING_DESCRIPTION_CONTENT_SHA1[key_description] = content.sha1 OBJ_STORAGE_DATA[content.sha1] = data RAW_CONTENT_METADATA = [ ( "du français".encode(), "text/plain", "utf-8", ), ( b"def __init__(self):", ("text/x-python", "text/x-script.python"), "us-ascii", ), ( b"\xff\xfe\x00\x00\x00\x00\xff\xfe\xff\xff", "application/octet-stream", "", ), ] RAW_CONTENTS: Dict[bytes, Tuple] = {} RAW_CONTENT_IDS: List[bytes] = [] for index, raw_content_d in enumerate(RAW_CONTENT_METADATA): raw_content = raw_content_d[0] content = Content.from_data(raw_content) RAW_CONTENTS[content.sha1] = raw_content_d RAW_CONTENT_IDS.append(content.sha1) # and write it to objstorage data so it's flushed in the objstorage OBJ_STORAGE_DATA[content.sha1] = raw_content SHA1_TO_LICENSES: Dict[bytes, List[str]] = { RAW_CONTENT_IDS[0]: ["GPL"], RAW_CONTENT_IDS[1]: ["AGPL"], RAW_CONTENT_IDS[2]: [], } SHA1_TO_CTAGS: Dict[bytes, List[Dict[str, Any]]] = { RAW_CONTENT_IDS[0]: [ { "name": "foo", "kind": "str", "line": 10, "lang": "bar", } ], RAW_CONTENT_IDS[1]: [ { "name": "symbol", "kind": "float", "line": 99, "lang": "python", } ], RAW_CONTENT_IDS[2]: [ { "name": "let", "kind": "int", "line": 100, "lang": "haskell", } ], } DIRECTORY = Directory( entries=( DirectoryEntry( name=b"index.js", type="file", target=MAPPING_DESCRIPTION_CONTENT_SHA1GIT["text:some"], perms=0o100644, ), DirectoryEntry( name=b"package.json", type="file", target=MAPPING_DESCRIPTION_CONTENT_SHA1GIT[ "json:test-metadata-package.json" ], perms=0o100644, ), DirectoryEntry( name=b".github", type="dir", target=Directory(entries=()).id, perms=0o040000, ), ), ) DIRECTORY2 = Directory( entries=( DirectoryEntry( name=b"package.json", type="file", target=MAPPING_DESCRIPTION_CONTENT_SHA1GIT["json:yarn-parser-package.json"], perms=0o100644, ), ), ) _utc_plus_2 = datetime.timezone(datetime.timedelta(minutes=120)) REVISION = Revision( message=b"Improve search functionality", author=Person( name=b"Andrew Nesbitt", fullname=b"Andrew Nesbitt ", email=b"andrewnez@gmail.com", ), committer=Person( name=b"Andrew Nesbitt", fullname=b"Andrew Nesbitt ", email=b"andrewnez@gmail.com", ), committer_date=TimestampWithTimezone.from_datetime( datetime.datetime(2013, 10, 4, 12, 50, 49, tzinfo=_utc_plus_2) ), type=RevisionType.GIT, synthetic=False, date=TimestampWithTimezone.from_datetime( datetime.datetime(2017, 2, 20, 16, 14, 16, tzinfo=_utc_plus_2) ), directory=DIRECTORY2.id, parents=(), ) REVISIONS = [REVISION] RELEASE = Release( name=b"v0.0.0", message=None, author=Person( name=b"Andrew Nesbitt", fullname=b"Andrew Nesbitt ", email=b"andrewnez@gmail.com", ), synthetic=False, date=TimestampWithTimezone.from_datetime( datetime.datetime(2017, 2, 20, 16, 14, 16, tzinfo=_utc_plus_2) ), target_type=ObjectType.DIRECTORY, target=DIRECTORY2.id, ) RELEASES = [RELEASE] SNAPSHOTS = [ # https://github.com/SoftwareHeritage/swh-storage Snapshot( branches={ b"refs/heads/add-revision-origin-cache": SnapshotBranch( target=b'L[\xce\x1c\x88\x8eF\t\xf1"\x19\x1e\xfb\xc0s\xe7/\xe9l\x1e', target_type=TargetType.REVISION, ), b"refs/head/master": SnapshotBranch( target=b"8K\x12\x00d\x03\xcc\xe4]bS\xe3\x8f{\xd7}\xac\xefrm", target_type=TargetType.REVISION, ), b"HEAD": SnapshotBranch( target=b"refs/head/master", target_type=TargetType.ALIAS ), b"refs/tags/v0.0.103": SnapshotBranch( target=b'\xb6"Im{\xfdLb\xb0\x94N\xea\x96m\x13x\x88+\x0f\xdd', target_type=TargetType.RELEASE, ), }, ), # rsync://ftp.gnu.org/gnu/3dldf Snapshot( branches={ b"3DLDF-1.1.4.tar.gz": SnapshotBranch( target=b'dJ\xfb\x1c\x91\xf4\x82B%]6\xa2\x90|\xd3\xfc"G\x99\x11', target_type=TargetType.REVISION, ), b"3DLDF-2.0.2.tar.gz": SnapshotBranch( target=b"\xb6\x0e\xe7\x9e9\xac\xaa\x19\x9e=\xd1\xc5\x00\\\xc6\xfc\xe0\xa6\xb4V", # noqa target_type=TargetType.REVISION, ), b"3DLDF-2.0.3-examples.tar.gz": SnapshotBranch( target=b"!H\x19\xc0\xee\x82-\x12F1\xbd\x97\xfe\xadZ\x80\x80\xc1\x83\xff", # noqa target_type=TargetType.REVISION, ), b"3DLDF-2.0.3.tar.gz": SnapshotBranch( target=b"\x8e\xa9\x8e/\xea}\x9feF\xf4\x9f\xfd\xee\xcc\x1a\xb4`\x8c\x8by", # noqa target_type=TargetType.REVISION, ), b"3DLDF-2.0.tar.gz": SnapshotBranch( target=b"F6*\xff(?\x19a\xef\xb6\xc2\x1fv$S\xe3G\xd3\xd1m", target_type=TargetType.REVISION, ), }, ), # https://forge.softwareheritage.org/source/jesuisgpl/", Snapshot( branches={ b"master": SnapshotBranch( target=b"\xe7n\xa4\x9c\x9f\xfb\xb7\xf76\x11\x08{\xa6\xe9\x99\xb1\x9e]q\xeb", # noqa target_type=TargetType.REVISION, ) }, ), # https://old-pypi.example.org/project/limnoria/ Snapshot( branches={ b"HEAD": SnapshotBranch( target=b"releases/2018.09.09", target_type=TargetType.ALIAS ), b"releases/2018.09.01": SnapshotBranch( target=b"<\xee1(\xe8\x8d_\xc1\xc9\xa6rT\xf1\x1d\xbb\xdfF\xfdw\xcf", target_type=TargetType.REVISION, ), b"releases/2018.09.09": SnapshotBranch( target=b"\x83\xb9\xb6\xc7\x05\xb1%\xd0\xfem\xd8kA\x10\x9d\xc5\xfa2\xf8t", # noqa target_type=TargetType.REVISION, ), }, ), # https://pypi.org/project/limnoria/ Snapshot( branches={ b"HEAD": SnapshotBranch( target=b"releases/2018.09.09", target_type=TargetType.ALIAS ), b"releases/2018.09.01": SnapshotBranch( target=b"<\xee1(\xe8\x8d_\xc1\xc9\xa6rT\xf1\x1d\xbb\xdfF\xfdw\xcf", target_type=TargetType.RELEASE, ), b"releases/2018.09.09": SnapshotBranch( target=b"\x83\xb9\xb6\xc7\x05\xb1%\xd0\xfem\xd8kA\x10\x9d\xc5\xfa2\xf8t", # noqa target_type=TargetType.RELEASE, ), }, ), # http://0-512-md.googlecode.com/svn/ Snapshot( branches={ b"master": SnapshotBranch( target=b"\xe4?r\xe1,\x88\xab\xec\xe7\x9a\x87\xb8\xc9\xad#.\x1bw=\x18", target_type=TargetType.REVISION, ) }, ), # https://github.com/librariesio/yarn-parser Snapshot( branches={ b"HEAD": SnapshotBranch( target=REVISION.id, target_type=TargetType.REVISION, ) }, ), # https://github.com/librariesio/yarn-parser.git Snapshot( branches={ b"HEAD": SnapshotBranch( target=REVISION.id, target_type=TargetType.REVISION, ) }, ), # https://npm.example.org/yarn-parser Snapshot( branches={ b"HEAD": SnapshotBranch( target=RELEASE.id, target_type=TargetType.RELEASE, ) }, ), ] assert len(SNAPSHOTS) == len(ORIGIN_VISITS) YARN_PARSER_METADATA = { "@context": "https://doi.org/10.5063/schema/codemeta-2.0", "url": "https://github.com/librariesio/yarn-parser#readme", "codeRepository": "git+git+https://github.com/librariesio/yarn-parser.git", "author": [{"type": "Person", "name": "Andrew Nesbitt"}], "license": "https://spdx.org/licenses/AGPL-3.0", "version": "1.0.0", "description": "Tiny web service for parsing yarn.lock files", "issueTracker": "https://github.com/librariesio/yarn-parser/issues", "name": "yarn-parser", "keywords": ["yarn", "parse", "lock", "dependencies"], "type": "SoftwareSourceCode", } json_dict_keys = strategies.one_of( strategies.characters(), strategies.just("type"), strategies.just("url"), strategies.just("name"), strategies.just("email"), strategies.just("@id"), strategies.just("@context"), strategies.just("repository"), strategies.just("license"), strategies.just("repositories"), strategies.just("licenses"), ) """Hypothesis strategy that generates strings, with an emphasis on those that are often used as dictionary keys in metadata files.""" generic_json_document = strategies.recursive( strategies.none() | strategies.booleans() | strategies.floats() | strategies.characters(), lambda children: ( strategies.lists(children, min_size=1) | strategies.dictionaries(json_dict_keys, children, min_size=1) ), ) """Hypothesis strategy that generates possible values for values of JSON metadata files.""" def json_document_strategy(keys=None): """Generates an hypothesis strategy that generates metadata files for a JSON-based format that uses the given keys.""" if keys is None: keys = strategies.characters() else: keys = strategies.one_of(map(strategies.just, keys)) return strategies.dictionaries(keys, generic_json_document, min_size=1) def _tree_to_xml(root, xmlns, data): def encode(s): "Skips unpaired surrogates generated by json_document_strategy" return s.encode("utf8", "replace") def to_xml(data, indent=b" "): if data is None: return b"" elif isinstance(data, (bool, str, int, float)): return indent + encode(str(data)) elif isinstance(data, list): return b"\n".join(to_xml(v, indent=indent) for v in data) elif isinstance(data, dict): lines = [] for (key, value) in data.items(): lines.append(indent + encode("<{}>".format(key))) lines.append(to_xml(value, indent=indent + b" ")) lines.append(indent + encode("".format(key))) return b"\n".join(lines) else: raise TypeError(data) return b"\n".join( [ '<{} xmlns="{}">'.format(root, xmlns).encode(), to_xml(data), "".format(root).encode(), ] ) class TreeToXmlTest(unittest.TestCase): def test_leaves(self): self.assertEqual( _tree_to_xml("root", "http://example.com", None), b'\n\n', ) self.assertEqual( _tree_to_xml("root", "http://example.com", True), b'\n True\n', ) self.assertEqual( _tree_to_xml("root", "http://example.com", "abc"), b'\n abc\n', ) self.assertEqual( _tree_to_xml("root", "http://example.com", 42), b'\n 42\n', ) self.assertEqual( _tree_to_xml("root", "http://example.com", 3.14), b'\n 3.14\n', ) def test_dict(self): self.assertIn( _tree_to_xml("root", "http://example.com", {"foo": "bar", "baz": "qux"}), [ b'\n' b" \n bar\n \n" b" \n qux\n \n" b"", b'\n' b" \n qux\n \n" b" \n bar\n \n" b"", ], ) def test_list(self): self.assertEqual( _tree_to_xml( "root", "http://example.com", [ {"foo": "bar"}, {"foo": "baz"}, ], ), b'\n' b" \n bar\n \n" b" \n baz\n \n" b"", ) def xml_document_strategy(keys, root, xmlns): """Generates an hypothesis strategy that generates metadata files for an XML format that uses the given keys.""" return strategies.builds( functools.partial(_tree_to_xml, root, xmlns), json_document_strategy(keys) ) def filter_dict(d, keys): "return a copy of the dict with keys deleted" if not isinstance(keys, (list, tuple)): keys = (keys,) return dict((k, v) for (k, v) in d.items() if k not in keys) def fill_obj_storage(obj_storage): """Add some content in an object storage.""" for obj_id, content in OBJ_STORAGE_DATA.items(): obj_storage.add(content, obj_id) def fill_storage(storage): """Fill in storage with consistent test dataset.""" storage.content_add([Content.from_data(data) for data in OBJ_STORAGE_DATA.values()]) storage.directory_add([DIRECTORY, DIRECTORY2]) storage.revision_add(REVISIONS) storage.release_add(RELEASES) storage.snapshot_add(SNAPSHOTS) storage.origin_add(ORIGINS) for visit, snapshot in zip(ORIGIN_VISITS, SNAPSHOTS): assert snapshot.id is not None visit = storage.origin_visit_add( [OriginVisit(origin=visit["origin"], date=now(), type=visit["type"])] )[0] visit_status = OriginVisitStatus( origin=visit.origin, visit=visit.visit, date=now(), status="full", snapshot=snapshot.id, ) storage.origin_visit_status_add([visit_status]) class CommonContentIndexerTest(metaclass=abc.ABCMeta): def get_indexer_results(self, ids): """Override this for indexers that don't have a mock storage.""" return self.indexer.idx_storage.state def assert_results_ok(self, sha1s, expected_results=None): sha1s = [hash_to_bytes(sha1) for sha1 in sha1s] actual_results = list(self.get_indexer_results(sha1s)) if expected_results is None: expected_results = self.expected_results # expected results may contain slightly duplicated results assert 0 < len(actual_results) <= len(expected_results) for result in actual_results: assert result in expected_results def test_index(self): """Known sha1 have their data indexed""" sha1s = [self.id0, self.id1, self.id2] # when self.indexer.run(sha1s) self.assert_results_ok(sha1s) # 2nd pass self.indexer.run(sha1s) self.assert_results_ok(sha1s) def test_index_one_unknown_sha1(self): """Unknown sha1s are not indexed""" sha1s = [ self.id1, "799a5ef812c53907562fe379d4b3851e69c7cb15", # unknown "800a5ef812c53907562fe379d4b3851e69c7cb15", # unknown ] # unknown # when self.indexer.run(sha1s) # then expected_results = [res for res in self.expected_results if res.id in sha1s] self.assert_results_ok(sha1s, expected_results) class CommonContentIndexerPartitionTest: """Allows to factorize tests on range indexer.""" def setUp(self): self.contents = sorted(OBJ_STORAGE_DATA) def assert_results_ok(self, partition_id, nb_partitions, actual_results): expected_ids = [ c.sha1 for c in stream_results( self.indexer.storage.content_get_partition, partition_id=partition_id, nb_partitions=nb_partitions, ) ] actual_results = list(actual_results) for indexed_data in actual_results: _id = indexed_data.id assert _id in expected_ids _tool_id = indexed_data.indexer_configuration_id assert _tool_id == self.indexer.tool["id"] def test__index_contents(self): """Indexing contents without existing data results in indexed data""" partition_id = 0 nb_partitions = 4 actual_results = list( self.indexer._index_contents(partition_id, nb_partitions, indexed={}) ) self.assert_results_ok(partition_id, nb_partitions, actual_results) def test__index_contents_with_indexed_data(self): """Indexing contents with existing data results in less indexed data""" partition_id = 3 nb_partitions = 4 # first pass actual_results = list( self.indexer._index_contents(partition_id, nb_partitions, indexed={}), ) self.assert_results_ok(partition_id, nb_partitions, actual_results) indexed_ids = {res.id for res in actual_results} actual_results = list( self.indexer._index_contents( partition_id, nb_partitions, indexed=indexed_ids ) ) # already indexed, so nothing new assert actual_results == [] def test_generate_content_get(self): """Optimal indexing should result in indexed data""" partition_id = 0 nb_partitions = 1 actual_results = self.indexer.run( partition_id, nb_partitions, skip_existing=False ) assert actual_results["status"] == "eventful", actual_results def test_generate_content_get_no_result(self): """No result indexed returns False""" actual_results = self.indexer.run(1, 2**512, incremental=False) assert actual_results == {"status": "uneventful"} + + +def mock_compute_license(path): + """path is the content identifier""" + if isinstance(id, bytes): + path = path.decode("utf-8") + # path is something like /tmp/tmpXXX/ so we keep only the sha1 part + id_ = path.split("/")[-1] + return {"licenses": SHA1_TO_LICENSES.get(hash_to_bytes(id_), [])}