From 9e4ac28ec9684122a49fd365a0afa0dee4fc9199 Mon Sep 17 00:00:00 2001 From: Shivdeep Singh Date: Mon, 30 Sep 2024 13:42:48 +0530 Subject: [PATCH] Add license select filter Signed-off-by: Shivdeep Singh --- .make.versions | 3 + README.md | 1 + kfp/README.md | 1 + transforms/code/license_select/Makefile | 62 +++++ transforms/code/license_select/README.md | 11 + .../code/license_select/kfp_ray/Makefile | 51 +++++ .../kfp_ray/license_select_wf.py | 210 +++++++++++++++++ .../code/license_select/python/.dockerignore | 1 + .../code/license_select/python/Dockerfile | 35 +++ .../code/license_select/python/Makefile | 56 +++++ .../code/license_select/python/README.md | 99 ++++++++ .../code/license_select/python/pyproject.toml | 45 ++++ .../python/src/license_select_local.py | 43 ++++ .../python/src/license_select_local_python.py | 54 +++++ .../python/src/license_select_transform.py | 176 ++++++++++++++ .../src/license_select_transform_python.py | 27 +++ .../license_select/python/src/transformer.py | 139 ++++++++++++ .../python/test-data/expected/metadata.json | 214 ++++++++++++++++++ .../test-data/expected/sample_1.parquet | Bin 0 -> 3586 bytes .../test-data/expected/sample_2.parquet | Bin 0 -> 3925 bytes .../python/test-data/input/sample_1.parquet | Bin 0 -> 3326 bytes .../python/test-data/input/sample_2.parquet | Bin 0 -> 3665 bytes .../test-data/sample_approved_licenses.json | 173 ++++++++++++++ .../python/test/test_license_select.py | 88 +++++++ .../python/test/test_license_select_python.py | 41 ++++ .../code/license_select/ray/.dockerignore | 1 + transforms/code/license_select/ray/Dockerfile | 37 +++ transforms/code/license_select/ray/Makefile | 65 ++++++ transforms/code/license_select/ray/README.md | 50 ++++ .../code/license_select/ray/pyproject.toml | 46 ++++ .../ray/src/license_select_local_ray.py | 61 +++++ .../ray/src/license_select_s3_ray.py | 56 +++++ .../ray/src/license_select_transform_ray.py | 30 +++ .../ray/test-data/expected/metadata.json | 214 ++++++++++++++++++ .../ray/test-data/expected/sample_1.parquet | Bin 0 -> 3586 bytes .../ray/test-data/expected/sample_2.parquet | Bin 0 -> 3925 bytes .../ray/test-data/input/sample_1.parquet | Bin 0 -> 3326 bytes .../ray/test-data/input/sample_2.parquet | Bin 0 -> 3665 bytes .../test-data/sample_approved_licenses.json | 173 ++++++++++++++ .../ray/test/test_license_select_ray.py | 40 ++++ 40 files changed, 2303 insertions(+) create mode 100644 transforms/code/license_select/Makefile create mode 100644 transforms/code/license_select/README.md create mode 100644 transforms/code/license_select/kfp_ray/Makefile create mode 100644 transforms/code/license_select/kfp_ray/license_select_wf.py create mode 100644 transforms/code/license_select/python/.dockerignore create mode 100644 transforms/code/license_select/python/Dockerfile create mode 100644 transforms/code/license_select/python/Makefile create mode 100644 transforms/code/license_select/python/README.md create mode 100644 transforms/code/license_select/python/pyproject.toml create mode 100644 transforms/code/license_select/python/src/license_select_local.py create mode 100644 transforms/code/license_select/python/src/license_select_local_python.py create mode 100644 transforms/code/license_select/python/src/license_select_transform.py create mode 100644 transforms/code/license_select/python/src/license_select_transform_python.py create mode 100644 transforms/code/license_select/python/src/transformer.py create mode 100644 transforms/code/license_select/python/test-data/expected/metadata.json create mode 100644 transforms/code/license_select/python/test-data/expected/sample_1.parquet create mode 100644 transforms/code/license_select/python/test-data/expected/sample_2.parquet create mode 100644 transforms/code/license_select/python/test-data/input/sample_1.parquet create mode 100644 transforms/code/license_select/python/test-data/input/sample_2.parquet create mode 100644 transforms/code/license_select/python/test-data/sample_approved_licenses.json create mode 100644 transforms/code/license_select/python/test/test_license_select.py create mode 100644 transforms/code/license_select/python/test/test_license_select_python.py create mode 100644 transforms/code/license_select/ray/.dockerignore create mode 100644 transforms/code/license_select/ray/Dockerfile create mode 100644 transforms/code/license_select/ray/Makefile create mode 100644 transforms/code/license_select/ray/README.md create mode 100644 transforms/code/license_select/ray/pyproject.toml create mode 100644 transforms/code/license_select/ray/src/license_select_local_ray.py create mode 100644 transforms/code/license_select/ray/src/license_select_s3_ray.py create mode 100644 transforms/code/license_select/ray/src/license_select_transform_ray.py create mode 100644 transforms/code/license_select/ray/test-data/expected/metadata.json create mode 100644 transforms/code/license_select/ray/test-data/expected/sample_1.parquet create mode 100644 transforms/code/license_select/ray/test-data/expected/sample_2.parquet create mode 100644 transforms/code/license_select/ray/test-data/input/sample_1.parquet create mode 100644 transforms/code/license_select/ray/test-data/input/sample_2.parquet create mode 100644 transforms/code/license_select/ray/test-data/sample_approved_licenses.json create mode 100644 transforms/code/license_select/ray/test/test_license_select_ray.py diff --git a/.make.versions b/.make.versions index f0dc3428c..e612fd21d 100644 --- a/.make.versions +++ b/.make.versions @@ -99,6 +99,9 @@ TEXT_ENCODER_RAY_VERSION=$(DPK_VERSION) HEADER_CLEANSER_PYTHON_VERSION=$(DPK_VERSION) HEADER_CLEANSER_RAY_VERSION=$(DPK_VERSION) +LICENSE_SELECT_PYTHON_VERSION=$(DPK_VERSION) +LICENSE_SELECT_RAY_VERSION=$(DPK_VERSION) + PII_REDACTOR_PYTHON_VERSION=$(DPK_VERSION) HTML2PARQUET_PYTHON_VERSION=$(DPK_VERSION) diff --git a/README.md b/README.md index b5da26f29..d4cf200e4 100644 --- a/README.md +++ b/README.md @@ -152,6 +152,7 @@ The matrix below shows the the combination of modules and supported runtimes. Al | [Malware annotation](transforms/code/malware/python/README.md) | :white_check_mark: |:white_check_mark:| |:white_check_mark: | | [Header cleanser](transforms/code/header_cleanser/python/README.md) | :white_check_mark: |:white_check_mark:| |:white_check_mark: | | [Semantic file ordering](transforms/code/repo_level_ordering/ray/README.md) | |:white_check_mark:| | | +| [License Select Annotation](transforms/code/license_select/python/README.md) | :white_check_mark: |:white_check_mark:| |:white_check_mark: | Contributors are welcome to add new modules to expand to other data modalities as well as add runtime support for existing modules! diff --git a/kfp/README.md b/kfp/README.md index 2468e2429..2757b7cf9 100644 --- a/kfp/README.md +++ b/kfp/README.md @@ -9,6 +9,7 @@ | code/code2parquet | [code2parquet_wf.py](../transforms/code/code2parquet/kfp_ray/code2parquet_wf.py) | | code/code_quality | [code_quality_wf.py](../transforms/code/code_quality/kfp_ray/code_quality_wf.py) | | code/proglang_select | [proglang_select_wf.py](../transforms/code/proglang_select/kfp_ray/proglang_select_wf.py) | +| code/license_select | [license_select_wf.py](../transforms/code/license_select/kfp_ray/license_select_wf.py) | universal/doc_id | [doc_id_wf.py](../transforms/universal/doc_id/kfp_ray/doc_id_wf.py) | | universal/ededup | [ededup_wf.py](../transforms/universal/ededup/kfp_ray/ededup_wf.py) | | universal/fdedup | [fdedup_wf.py](../transforms/universal/fdedup/kfp_ray/fdedup_wf.py) | diff --git a/transforms/code/license_select/Makefile b/transforms/code/license_select/Makefile new file mode 100644 index 000000000..b19f5c963 --- /dev/null +++ b/transforms/code/license_select/Makefile @@ -0,0 +1,62 @@ +REPOROOT=../../.. +# Use make help, to see the available rules +include $(REPOROOT)/.make.defaults + +setup:: + @# Help: Recursively make $@ all subdirs + $(MAKE) RULE=$@ .recurse + +clean:: + @# Help: Recursively make $@ all subdirs + $(MAKE) RULE=$@ .recurse + +build:: + @# Help: Recursively make $@ in subdirs + $(MAKE) RULE=$@ .recurse +venv:: + @# Help: Recursively make $@ in subdirs + $(MAKE) RULE=$@ .recurse + +image:: + @# Help: Recursively make $@ in all subdirs + @$(MAKE) RULE=$@ .recurse + +set-versions: + @# Help: Recursively $@ in all subdirs + @$(MAKE) RULE=$@ .recurse + +publish:: + @# Help: Recursively make $@ in all subdirs + @$(MAKE) RULE=$@ .recurse + +test-image:: + @# Help: Recursively make $@ in all subdirs + @$(MAKE) RULE=$@ .recurse + +test:: + @# Help: Recursively make $@ in all subdirs + @$(MAKE) RULE=$@ .recurse + +test-src:: + @# Help: Recursively make $@ in all subdirs + $(MAKE) RULE=$@ .recurse + +load-image:: + @# Help: Recursively make $@ in all subdirs + $(MAKE) RULE=$@ .recurse + +.PHONY: workflow-venv +workflow-venv: + $(MAKE) -C kfp_ray workflow-venv + +.PHONY: workflow-test +workflow-test: + $(MAKE) -C kfp_ray workflow-test + +.PHONY: workflow-upload +workflow-upload: + $(MAKE) -C kfp_ray workflow-upload + +.PHONY: workflow-build +workflow-build: + $(MAKE) -C kfp_ray workflow-build diff --git a/transforms/code/license_select/README.md b/transforms/code/license_select/README.md new file mode 100644 index 000000000..1e0136628 --- /dev/null +++ b/transforms/code/license_select/README.md @@ -0,0 +1,11 @@ +# License Select + +The License Select transform checks if the `license` of input data is in approved/denied list. It is implemented as per the set of [transform project conventions](../../README.md#transform-project-conventions) the following runtimes are available: + +* [python](python/README.md) - provides the base python-based transformation +implementation. +* [ray](ray/README.md) - enables the running of the base python transformation +in a Ray runtime +* [kfp](kfp_ray/README.md) - enables running the ray docker image +in a kubernetes cluster using a generated `yaml` file. + diff --git a/transforms/code/license_select/kfp_ray/Makefile b/transforms/code/license_select/kfp_ray/Makefile new file mode 100644 index 000000000..9f21f3d58 --- /dev/null +++ b/transforms/code/license_select/kfp_ray/Makefile @@ -0,0 +1,51 @@ +REPOROOT=${CURDIR}/../../../../ +WORKFLOW_VENV_ACTIVATE=${REPOROOT}/transforms/venv/bin/activate +include $(REPOROOT)/transforms/.make.workflows + +SRC_DIR=${CURDIR}/../ray/ + +PYTHON_WF := $(shell find ./ -name '*_wf.py') +YAML_WF := $(patsubst %.py, %.yaml, ${PYTHON_WF}) + +workflow-venv: .check_python_version ${WORKFLOW_VENV_ACTIVATE} + +.PHONY: clean +clean: + @# Help: Clean up the virtual environment. + rm -rf ${REPOROOT}/transforms/venv + +venv:: + +build:: + +setup:: + +test:: + +test-src:: + +test-image:: + +publish:: + +image:: + +load-image:: + +docker-load-image:: + +docker-save-image:: + +.PHONY: workflow-build +workflow-build: workflow-venv + $(MAKE) $(YAML_WF) + +.PHONY: workflow-test +workflow-test: workflow-build + $(MAKE) .workflows.test-pipeline TRANSFORM_SRC=${SRC_DIR} PIPELINE_FILE=license_select_wf.yaml + +.PHONY: workflow-upload +workflow-upload: workflow-build + @for file in $(YAML_WF); do \ + $(MAKE) .workflows.upload-pipeline PIPELINE_FILE=$$file; \ + done diff --git a/transforms/code/license_select/kfp_ray/license_select_wf.py b/transforms/code/license_select/kfp_ray/license_select_wf.py new file mode 100644 index 000000000..1938decfe --- /dev/null +++ b/transforms/code/license_select/kfp_ray/license_select_wf.py @@ -0,0 +1,210 @@ +# (C) Copyright IBM Corp. 2024. +# Licensed under the Apache License, Version 2.0 (the “License”); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an “AS IS” BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import os + +import kfp.compiler as compiler +import kfp.components as comp +import kfp.dsl as dsl +from workflow_support.compile_utils import ONE_HOUR_SEC, ONE_WEEK_SEC, ComponentUtils + + +# the name of the job script +EXEC_SCRIPT_NAME: str = "license_select_transform_ray.py" + +task_image = "quay.io/dataprep1/data-prep-kit/license_select-ray:latest" + + +# components +base_kfp_image = "quay.io/dataprep1/data-prep-kit/kfp-data-processing:latest" + +# path to kfp component specifications files +component_spec_path = "../../../../kfp/kfp_ray_components/" + +# compute execution parameters. Here different transforms might need different implementations. As +# a result, instead of creating a component we are creating it in place here. +def compute_exec_params_func( + worker_options: dict, + actor_options: dict, + data_s3_config: str, + data_max_files: int, + data_num_samples: int, + runtime_pipeline_id: str, + runtime_job_id: str, + runtime_code_location: str, + lc_license_column_name: str, + lc_licenses_file: str, +) -> dict: + from runtime_utils import KFPUtils + + return { + "data_s3_config": data_s3_config, + "data_max_files": data_max_files, + "data_num_samples": data_num_samples, + "runtime_num_workers": KFPUtils.default_compute_execution_params(str(worker_options), str(actor_options)), + "runtime_worker_options": str(actor_options), + "runtime_pipeline_id": runtime_pipeline_id, + "runtime_job_id": runtime_job_id, + "runtime_code_location": str(runtime_code_location), + "lc_license_column_name": lc_license_column_name, + "lc_licenses_file": lc_licenses_file, + } + + +# KFPv1 and KFP2 uses different methods to create a component from a function. KFPv1 uses the +# `create_component_from_func` function, but it is deprecated by KFPv2 and so has a different import path. +# KFPv2 recommends using the `@dsl.component` decorator, which doesn't exist in KFPv1. Therefore, here we use +# this if/else statement and explicitly call the decorator. +if os.getenv("KFPv2", "0") == "1": + # In KFPv2 dsl.RUN_ID_PLACEHOLDER is deprecated and cannot be used since SDK 2.5.0. On another hand we cannot create + # a unique string in a component (at runtime) and pass it to the `clean_up_task` of `ExitHandler`, due to + # https://github.com/kubeflow/pipelines/issues/10187. Therefore, meantime we use a unique string created at + # compilation time. + import uuid + + compute_exec_params_op = dsl.component_decorator.component( + func=compute_exec_params_func, base_image=base_kfp_image + ) + print( + "WARNING: the ray cluster name can be non-unique at runtime, please do not execute simultaneous Runs of the " + + "same version of the same pipeline !!!" + ) + run_id = uuid.uuid4().hex +else: + compute_exec_params_op = comp.create_component_from_func(func=compute_exec_params_func, base_image=base_kfp_image) + run_id = dsl.RUN_ID_PLACEHOLDER + +# create Ray cluster +create_ray_op = comp.load_component_from_file(component_spec_path + "createRayClusterComponent.yaml") +# execute job +execute_ray_jobs_op = comp.load_component_from_file(component_spec_path + "executeRayJobComponent.yaml") +# clean up Ray +cleanup_ray_op = comp.load_component_from_file(component_spec_path + "deleteRayClusterComponent.yaml") +# Task name is part of the pipeline name, the ray cluster name and the job name in DMF. +TASK_NAME: str = "license_select" +PREFIX: str = "license_select" + + +@dsl.pipeline( + name=TASK_NAME + "-ray-pipeline", + description="Pipeline for license select task", +) +def license_select( + ray_name: str = "license_select-kfp-ray", # name of Ray cluster + # Add image_pull_secret and image_pull_policy to ray workers if needed + ray_head_options: dict = {"cpu": 1, "memory": 4, "image": task_image}, + ray_worker_options: dict = { + "replicas": 2, + "max_replicas": 2, + "min_replicas": 2, + "cpu": 2, + "memory": 4, + "image": task_image, + }, + server_url: str = "http://kuberay-apiserver-service.kuberay.svc.cluster.local:8888", + # data access + data_s3_config: str = "{'input_folder': 'test/license_select/input/', 'output_folder': 'test/license_select/output/'}", + data_s3_access_secret: str = "s3-secret", + data_max_files: int = -1, + data_num_samples: int = -1, + # orchestrator + runtime_actor_options: dict = {"num_cpus": 0.8}, + runtime_pipeline_id: str = "runtime_pipeline_id", + runtime_code_location: str = "{'github': 'github', 'commit_hash': '12345', 'path': 'path'}", + # license select parameters + lc_license_column_name: str = "license", + lc_licenses_file: str = "test/license_select/sample_approved_licenses.json", + # additional parameters + additional_params: str = '{"wait_interval": 2, "wait_cluster_ready_tmout": 400, "wait_cluster_up_tmout": 300, "wait_job_ready_tmout": 400, "wait_print_tmout": 30, "http_retries": 5}', +): + """ + Pipeline to execute License Select transform + :param ray_name: name of the Ray cluster + :param ray_head_options: head node options, containing the following: + cpu - number of cpus + memory - memory + image - image to use + image_pull_secret - image pull secret + :param ray_worker_options: worker node options (we here are using only 1 worker pool), containing the following: + replicas - number of replicas to create + max_replicas - max number of replicas + min_replicas - min number of replicas + cpu - number of cpus + memory - memory + image - image to use + image_pull_secret - image pull secret + :param server_url - server url + :param additional_params: additional (support) parameters, containing the following: + wait_interval - wait interval for API server, sec + wait_cluster_ready_tmout - time to wait for cluster ready, sec + wait_cluster_up_tmout - time to wait for cluster up, sec + wait_job_ready_tmout - time to wait for job ready, sec + wait_print_tmout - time between prints, sec + http_retries - httpt retries for API server calls + :param data_s3_config - s3 configuration + :param data_s3_access_secret - s3 access secret + :param data_max_files - max files to process + :param data_num_samples - num samples to process + :param runtime_actor_options - actor options + :param runtime_pipeline_id - pipeline id + :param lc_license_column_name - Name of the column holds the license to process + :param lc_licenses_file - path to license list json file + :return: None + """ + # create clean_up task + clean_up_task = cleanup_ray_op(ray_name=ray_name, run_id=run_id, server_url=server_url) + ComponentUtils.add_settings_to_component(clean_up_task, 60) + # pipeline definition + with dsl.ExitHandler(clean_up_task): + # compute execution params + compute_exec_params = compute_exec_params_op( + worker_options=ray_worker_options, + actor_options=runtime_actor_options, + data_s3_config=data_s3_config, + data_max_files=data_max_files, + data_num_samples=data_num_samples, + runtime_pipeline_id=runtime_pipeline_id, + runtime_job_id=run_id, + runtime_code_location=runtime_code_location, + lc_license_column_name=lc_license_column_name, + lc_licenses_file=lc_licenses_file, + ) + ComponentUtils.add_settings_to_component(compute_exec_params, ONE_HOUR_SEC * 2) + # start Ray cluster + ray_cluster = create_ray_op( + ray_name=ray_name, + run_id=run_id, + ray_head_options=ray_head_options, + ray_worker_options=ray_worker_options, + server_url=server_url, + additional_params=additional_params, + ) + ComponentUtils.add_settings_to_component(ray_cluster, ONE_HOUR_SEC * 2) + ray_cluster.after(compute_exec_params) + # Execute job + execute_job = execute_ray_jobs_op( + ray_name=ray_name, + run_id=run_id, + additional_params=additional_params, + # note that the parameters below are specific for this transform + exec_params=compute_exec_params.output, + exec_script_name=EXEC_SCRIPT_NAME, + server_url=server_url, + ) + ComponentUtils.add_settings_to_component(execute_job, ONE_WEEK_SEC) + ComponentUtils.set_s3_env_vars_to_component(execute_job, data_s3_access_secret) + execute_job.after(ray_cluster) + + +if __name__ == "__main__": + # Compiling the pipeline + compiler.Compiler().compile(license_select, __file__.replace(".py", ".yaml")) diff --git a/transforms/code/license_select/python/.dockerignore b/transforms/code/license_select/python/.dockerignore new file mode 100644 index 000000000..f7275bbbd --- /dev/null +++ b/transforms/code/license_select/python/.dockerignore @@ -0,0 +1 @@ +venv/ diff --git a/transforms/code/license_select/python/Dockerfile b/transforms/code/license_select/python/Dockerfile new file mode 100644 index 000000000..6831306c3 --- /dev/null +++ b/transforms/code/license_select/python/Dockerfile @@ -0,0 +1,35 @@ +FROM docker.io/python:3.10.14-slim-bullseye + +RUN pip install --upgrade --no-cache-dir pip +# install pytest +RUN pip install --no-cache-dir pytest + +# Create a user and use it to run the transform +RUN useradd -ms /bin/bash dpk +USER dpk +WORKDIR /home/dpk + +# Copy and install data processing libraries +# These are expected to be placed in the docker context before this is run (see the make image). +COPY --chown=dpk:root data-processing-lib-python/ data-processing-lib-python/ +RUN cd data-processing-lib-python && pip install --no-cache-dir -e . + +# END OF STEPS destined for a data-prep-kit base image + +COPY --chown=dpk:root src/ src/ +COPY --chown=dpk:root pyproject.toml pyproject.toml +RUN pip install --no-cache-dir -e . + +# copy source data +COPY ./src/license_select_transform_python.py . +COPY ./src/license_select_local.py local/ + +# copy test +COPY test/ test/ +COPY test-data/ test-data/ + +# Put these at the end since they seem to upset the docker cache. +ARG BUILD_DATE +ARG GIT_COMMIT +LABEL build-date=$BUILD_DATE +LABEL git-commit=$GIT_COMMIT diff --git a/transforms/code/license_select/python/Makefile b/transforms/code/license_select/python/Makefile new file mode 100644 index 000000000..63a8cf008 --- /dev/null +++ b/transforms/code/license_select/python/Makefile @@ -0,0 +1,56 @@ + +# Define the root of the local git clone for the common rules to be able +# know where they are running from. +REPOROOT=../../../.. +include $(REPOROOT)/transforms/.make.transforms + +TRANSFORM_NAME=license_select +# $(REPOROOT)/.make.versions file contains the versions +DOCKER_IMAGE_VERSION=${LICENSE_SELECT_PYTHON_VERSION} + +# Use default rule inherited from makefile.common +clean:: .transforms.clean + +# Use default rule inherited from makefile.common +test:: .transforms.python-test + +# Use default rule inherited from makefile.common +image:: .transforms.python-image + +# Use default rule inherited from makefile.common +venv:: .transforms.python-venv + +test-src:: .transforms.test-src + +test-image:: .transforms.python-test-image + +run-cli-sample: + $(MAKE) RUN_FILE=$(TRANSFORM_NAME)_transform_python.py \ + RUN_ARGS="--data_local_config \"{ 'input_folder' : '../test-data/input', 'output_folder' : '../output'}\" \ + --lc_license_column_name license \ + --lc_licenses_file ../test-data/sample_approved_licenses.json" \ + .transforms.run-src-file + +run-local-sample: .transforms.run-local-sample + +build:: build-dist image + +publish:: publish-dist publish-image + +publish-image:: .transforms.publish-image-python + +setup:: .transforms.setup + +# distribution versions is the same as image version. +set-versions: + $(MAKE) TRANSFORM_PYTHON_VERSION=${LICENSE_SELECT_PYTHON_VERSION} TOML_VERSION=$(LICENSE_SELECT_PYTHON_VERSION) .transforms.set-versions + +build-dist:: .defaults.build-dist + +publish-dist:: .defaults.publish-dist + +setup:: .transforms.setup +run-local-sample: .transforms.run-local-sample +run-local-python-sample: .transforms.run-local-python-sample + +load-image:: .transforms.load-image diff --git a/transforms/code/license_select/python/README.md b/transforms/code/license_select/python/README.md new file mode 100644 index 000000000..6c88c7e63 --- /dev/null +++ b/transforms/code/license_select/python/README.md @@ -0,0 +1,99 @@ +# License Select + +Please see the set of +[transform project conventions](../../../README.md#transform-project-conventions) +for details on general project conventions, transform configuration, +testing and IDE set up. + +## Summary + +The License Select transform checks if the `license` of input data is in approved/denied list. It is implemented as per the set of [transform project conventions](../../README.md#transform-project-conventions) the following runtimes are available: +This filter scans the license column of an input dataset and appends the `license_status` column to the dataset. + +The type of the license column can be either string or list of strings. For strings, the license name is checked against the list of approved licenses. For list of strings, each license name in the list is checked against the list of approved licenses, and all must be approved. + +If the license is approved, the license_status column contains True; otherwise False. + +## Configuration and command line Options + +The set of dictionary keys holding license_select configuration for values are as follows: + +The transform can be configured with the following key/value pairs from the configuration dictionary. + +```python +# Sample params dictionary passed to the transform + +{ +"license_select_params" : { + "license_column_name": "license", + "deny_licenses": False, + "licenses": [ 'MIT', 'Apache'], + "allow_no_license": False, + } +} +``` + +**license_column_name** - The name of the column with licenses. + +**deny_licenses** - A boolean value, True for denied licesnes, False for approved licenses. + +**licenses** - A list of licenses used as approve/deny list. + +**allow_no_license** - A boolean value, used to retain the values with no license in the column `license_column_name` + + +## Running + +### Launcher Command Line Options + +The following command line arguments are available in addition to +the options provided by the [python launcher](../../../../data-processing-lib/doc/python-launcher-options.md). + + `--lc_license_column_name` - set the name of the column holds license to process + + `--lc_allow_no_license` - allow entries with no associated license (default: false) + + `--lc_licenses_file` - S3 or local path to allowed/denied licenses JSON file + + `--lc_deny_licenses` - allow all licences except those in licenses_file (default: false) + +- The optional `lc_license_column_name` parameter is used to specify the column name in the input dataset that contains the license information. The default column name is license. + +- The optional `lc_allow_no_license` option allows any records without a license to be accepted by the filter. If this option is not set, records without a license are rejected. + +- The required `lc_licenses_file` options allows a list of licenses to be specified. An S3 or local file path should be supplied (including bucket name, for example: bucket-name/path/to/licenses.json) with the file contents being a JSON list of strings. For example: + + >[ + 'Apache-2.0', + 'MIT' + ] + +- The optional `lc_deny_licenses` flag is used when `lc_licenses_file` specifies the licenses that will be rejected, with all other licenses being accepted. These parameters do not affect handling of records with no license information, which is dictated by the allow_no_license option. + + +### Running the samples + +To run the samples, use the following make targets + +`run-cli-sample` + +`run-local-python-sample` + +These targets will activate the virtual environment and set up any configuration needed. Use the -n option of make to see the detail of what is done to run the sample. + +For example, +``` +make run-cli-sample + +``` +... +Then + +ls output +To see results of the transform. + +### Transforming data using the transform image + +To use the transform image to transform your data, please refer to the +[running images quickstart](../../../../doc/quick-start/run-transform-image.md), +substituting the name of this transform image and runtime as appropriate. diff --git a/transforms/code/license_select/python/pyproject.toml b/transforms/code/license_select/python/pyproject.toml new file mode 100644 index 000000000..1058b0440 --- /dev/null +++ b/transforms/code/license_select/python/pyproject.toml @@ -0,0 +1,45 @@ +[project] +name = "dpk_license_select_transform_python" +version = "0.2.1.dev0" +requires-python = ">=3.10" +description = "License Select Python Transform" +license = {text = "Apache-2.0"} +readme = {file = "README.md", content-type = "text/markdown"} +authors = [ + { name = "Shivdeep Singh", email = "shivdeep.singh@ibm.com" }, + { name = "Mark Lewis", email = "mark_lewis@uk.ibm.com" }, +] +dependencies = [ + "data-prep-toolkit==0.2.1.dev0", +] + +[build-system] +requires = ["setuptools>=68.0.0", "wheel", "setuptools_scm[toml]>=7.1.0"] +build-backend = "setuptools.build_meta" + +[project.optional-dependencies] +dev = [ + "twine", + "pytest>=7.3.2", + "pytest-dotenv>=0.5.2", + "pytest-env>=1.0.0", + "pre-commit>=3.3.2", + "pytest-cov>=4.1.0", + "pytest-mock>=3.10.0", + "moto==5.0.5", + "markupsafe==2.0.1", +] + +[options] +package_dir = ["src","test"] + +[options.packages.find] +where = ["src/"] + +[tool.pytest.ini_options] +# Currently we use low coverage since we have to run tests separately (see makefile) +#addopts = "--cov --cov-report term-missing --cov-fail-under 25" +markers = ["unit: unit tests", "integration: integration tests"] + +[tool.coverage.run] +include = ["src/*"] diff --git a/transforms/code/license_select/python/src/license_select_local.py b/transforms/code/license_select/python/src/license_select_local.py new file mode 100644 index 000000000..a16cbb27a --- /dev/null +++ b/transforms/code/license_select/python/src/license_select_local.py @@ -0,0 +1,43 @@ +# (C) Copyright IBM Corp. 2024. +# Licensed under the Apache License, Version 2.0 (the “License”); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an “AS IS” BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import json +import os + +from data_processing.data_access import DataAccessLocal +from license_select_transform import LicenseSelectTransform + + +input_folder = os.path.abspath(os.path.join(os.path.dirname(__file__), "../test-data/input")) + +licenses_file = os.path.abspath(os.path.join(os.path.dirname(__file__), "../test-data/sample_approved_licenses.json")) + +with open(licenses_file) as f: + licenses = json.loads(f.read()) + +if __name__ == "__main__": + license_select_params = { + "license_select_params": { + "license_column_name": "license", + "allow_no_license": False, + "licenses": licenses, + "deny": False, + } + } + transform = LicenseSelectTransform(license_select_params) + + data_access = DataAccessLocal() + table, _ = data_access.get_table(os.path.join(input_folder, "sample_1.parquet")) + print(f"input table: {table}") + # Transform the table + table_list, metadata = transform.transform(table) + print(f"\noutput table: {table_list}") diff --git a/transforms/code/license_select/python/src/license_select_local_python.py b/transforms/code/license_select/python/src/license_select_local_python.py new file mode 100644 index 000000000..2306e00ac --- /dev/null +++ b/transforms/code/license_select/python/src/license_select_local_python.py @@ -0,0 +1,54 @@ +# (C) Copyright IBM Corp. 2024. +# Licensed under the Apache License, Version 2.0 (the “License”); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an “AS IS” BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import os +import sys +from pathlib import Path + +from data_processing.runtime.pure_python import PythonTransformLauncher +from data_processing.utils import ParamsUtils +from license_select_transform import LicenseSelectTransformConfiguration + + +input_folder = os.path.abspath(os.path.join(os.path.dirname(__file__), "../test-data/input")) +output_folder = os.path.abspath(os.path.join(os.path.dirname(__file__), "../output")) +approved_license_file = os.path.abspath( + os.path.join(os.path.dirname(__file__), "../test-data/sample_approved_licenses.json") +) + +local_conf = { + "input_folder": input_folder, + "output_folder": output_folder, +} + +# create launcher +launcher = PythonTransformLauncher(runtime_config=LicenseSelectTransformConfiguration()) + + +code_location = {"github": "github", "commit_hash": "12345", "path": "path"} +params = { + # Data access. Only required parameters are specified + "data_local_config": ParamsUtils.convert_to_ast(local_conf), + # runtime + "runtime_pipeline_id": "pipeline_id", + "runtime_job_id": "job_id", + "runtime_code_location": ParamsUtils.convert_to_ast(code_location), + # license select configuration + "lc_license_column_name": "license", + "lc_licenses_file": approved_license_file, +} + +if __name__ == "__main__": + Path(output_folder).mkdir(parents=True, exist_ok=True) + sys.argv = ParamsUtils.dict_to_req(d=params) + # launch + launcher.launch() diff --git a/transforms/code/license_select/python/src/license_select_transform.py b/transforms/code/license_select/python/src/license_select_transform.py new file mode 100644 index 000000000..dc4ad2b72 --- /dev/null +++ b/transforms/code/license_select/python/src/license_select_transform.py @@ -0,0 +1,176 @@ +# Licensed under the Apache License, Version 2.0 (the “License”); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an “AS IS” BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +################################################################################ + + +import json +from argparse import ArgumentParser, Namespace + +import pyarrow as pa +from data_processing.data_access import DataAccess, DataAccessFactory +from data_processing.transform import AbstractTableTransform, TransformConfiguration +from data_processing.utils import ( + CLIArgumentProvider, + TransformUtils, + get_logger, + str2bool, +) +from transformer import AllowLicenseStatusTransformer, DenyLicenseStatusTransformer + + +logger = get_logger(__name__) + +LICENSE_SELECT_PARAMS = "license_select_params" + +shortname = "lc" +CLI_PREFIX = f"{shortname}_" + +LICENSE_COLUMN_NAME_KEY = "license_column_name" +LICENSE_COLUMN_NAME_CLI_KEY = f"{CLI_PREFIX}{LICENSE_COLUMN_NAME_KEY}" + +DENY_LICENSES_KEY = "deny_licenses" +DENY_LICENSES_CLI_KEY = f"{CLI_PREFIX}{DENY_LICENSES_KEY}" + +LICENSES_FILE_KEY = "licenses_file" +LICENSES_FILE_CLI_KEY = f"{CLI_PREFIX}{LICENSES_FILE_KEY}" + +ALLOW_NO_LICENSE_KEY = "allow_no_license" +ALLOW_NO_LICENSE_CLI_KEY = f"{CLI_PREFIX}{ALLOW_NO_LICENSE_KEY}" + +LICENSE_COLUMN_DEFAULT = "license" +LICENSES_KEY = "licenses" + + +def _get_supported_licenses(license_file: str, data_access: DataAccess) -> list[str]: + logger.info(f"Getting supported licenses from file {license_file}") + licenses_list = None + try: + licenses_list_json, _ = data_access.get_file(license_file) + licenses_list = json.loads(licenses_list_json.decode("utf-8")) + logger.info(f"Read a list of {len(licenses_list)} licenses.") + except Exception as e: + logger.error(f"Failed to read file: {license_file} due to {e}") + return licenses_list + + +class LicenseSelectTransform(AbstractTableTransform): + """It can be used to select the rows/records of data with licenses + matching those in the approved/deny list. It adds a new column: `license_status` + to indicate the selected/denied licenses. + + config: dictionary of configuration data + license_select_params: A dictionary with the following keys. + license_column_name - The name of the column with license, default: 'licence'. + allow_no_license - Allows to select rows with no license. default: False + licenses - A list of licenses + deny_licenses - if selected, the the licenses list is used as deny list, default: False + Example: + config = { + "license_select_params": { + "license_column_name": "license", + "allow_no_license": False, + "licenses": ["MIT", "Apache 2.0"], + "deny_licenses": False + } + } + """ + + def __init__(self, config: dict): + super().__init__(config) + + try: + self.license_select = config.get(LICENSE_SELECT_PARAMS) + self.license_column = self.license_select.get(LICENSE_COLUMN_NAME_KEY, LICENSE_COLUMN_DEFAULT) + allow_no_license = self.license_select.get(ALLOW_NO_LICENSE_KEY, False) + licenses = self.license_select.get(LICENSES_KEY, None) + if not licenses or not isinstance(licenses, list): + raise ValueError("license list not found.") + deny = self.license_select.get(DENY_LICENSES_KEY, False) + logger.debug(f"LICENSE_SELECT_PARAMS: {self.license_select}") + except Exception as e: + raise ValueError(f"Invalid Argument: cannot create LicenseSelectTransform object: {e}.") + + if not deny: + self.transformer = AllowLicenseStatusTransformer( + license_column=self.license_column, + allow_no_license=allow_no_license, + licenses=licenses, + ) + else: + self.transformer = DenyLicenseStatusTransformer( + license_column=self.license_column, + allow_no_license=allow_no_license, + licenses=licenses, + ) + + def transform(self, table: pa.Table, file_name: str = None) -> tuple[list[pa.Table], dict]: + """ + Transforms input tables by adding a boolean `license_status` column + indicating whether the license is approved/denied. + """ + TransformUtils.validate_columns(table=table, required=[self.license_column]) + new_table = self.transformer.transform(table) + return [new_table], {} + + +class LicenseSelectTransformConfiguration(TransformConfiguration): + def __init__(self): + super().__init__(name="license_select", transform_class=LicenseSelectTransform) + + def add_input_params(self, parser: ArgumentParser) -> None: + parser.add_argument( + f"--{LICENSE_COLUMN_NAME_CLI_KEY}", + required=False, + type=str, + default=LICENSE_COLUMN_DEFAULT, + help="Name of the column holds the data to process", + ) + parser.add_argument( + f"--{ALLOW_NO_LICENSE_CLI_KEY}", + required=False, + type=lambda x: bool(str2bool(x)), + default=False, + help="allow entries with no associated license (default: false)", + ) + parser.add_argument( + f"--{LICENSES_FILE_CLI_KEY}", + required=True, + type=str, + help="S3 or local path to allowed/denied licenses JSON file", + ) + parser.add_argument( + f"--{DENY_LICENSES_CLI_KEY}", + type=lambda x: bool(str2bool(x)), + required=False, + default=False, + help="allow all licences except those in licenses_file (default: false)", + ) + # Create the DataAccessFactor to use CLI args + self.daf = DataAccessFactory(CLI_PREFIX, False) + # Add the DataAccessFactory parameters to the transform's configuration parameters. + self.daf.add_input_params(parser) + + def apply_input_params(self, args: Namespace) -> bool: + captured = CLIArgumentProvider.capture_parameters(args, CLI_PREFIX, False) + data_access = self.daf.create_data_access() + deny = captured.get(DENY_LICENSES_KEY, False) + license_list_file = captured.get(LICENSES_FILE_KEY) + # Read licenses from allow-list or deny-list + licenses = _get_supported_licenses(license_list_file, data_access) + self.params = { + LICENSE_SELECT_PARAMS: { + LICENSE_COLUMN_NAME_KEY: captured.get(LICENSE_COLUMN_NAME_KEY), + ALLOW_NO_LICENSE_KEY: captured.get(ALLOW_NO_LICENSE_KEY), + LICENSES_KEY: licenses, + DENY_LICENSES_KEY: deny, + } + } + return True diff --git a/transforms/code/license_select/python/src/license_select_transform_python.py b/transforms/code/license_select/python/src/license_select_transform_python.py new file mode 100644 index 000000000..3ceaf7f32 --- /dev/null +++ b/transforms/code/license_select/python/src/license_select_transform_python.py @@ -0,0 +1,27 @@ +# Licensed under the Apache License, Version 2.0 (the “License”); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an “AS IS” BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from data_processing.runtime.pure_python import PythonTransformLauncher + +################################################################################ +from data_processing.runtime.pure_python.runtime_configuration import ( + PythonTransformRuntimeConfiguration, +) +from license_select_transform import LicenseSelectTransformConfiguration + + +class LicenseSelectPythonTransformConfiguration(PythonTransformRuntimeConfiguration): + def __init__(self): + super().__init__(transform_config=LicenseSelectTransformConfiguration()) + + +if __name__ == "__main__": + launcher = PythonTransformLauncher(LicenseSelectPythonTransformConfiguration()) + launcher.launch() diff --git a/transforms/code/license_select/python/src/transformer.py b/transforms/code/license_select/python/src/transformer.py new file mode 100644 index 000000000..edddd0c0c --- /dev/null +++ b/transforms/code/license_select/python/src/transformer.py @@ -0,0 +1,139 @@ +from collections.abc import Callable, Iterable +from typing import TypeVar + +import pyarrow +import pyarrow.compute as pc +from pyarrow import ListScalar, Table + + +T = TypeVar("T") +R = TypeVar("R") + +Mapping = Callable[[T], R] + + +def exact_matcher(matches: Iterable[T]) -> Mapping[T, bool]: + """ + Returns a Mapping that generates a boolean indicating whether an input exactly matches any of a set of values. + """ + unique_matches = set(matches) + return lambda name: name in unique_matches + + +class AllowLicenseStatusTransformer: + """ + Transforms input tables by adding a boolean `license_status` column indicating whether the license is approved. + """ + + def __init__( + self, + license_column: str = None, + allow_no_license: bool = False, + licenses: list[str] = None, + ): + self._license_column = license_column or "license" + + if not licenses: + raise TypeError("No approved licenses found.") + licenses = [_lower_case(license) for license in licenses] + if allow_no_license: + licenses.append(None) + + self._string_transformer = LicenseStringTransformer(licenses) + self._list_transformer = LicenseListTransformer(licenses) + + def transform(self, data: Table) -> Table: + license_type = data.schema.field(self._license_column).type + + data = _rename_column(data, self._license_column, "license") + + if pyarrow.types.is_string(license_type) or pyarrow.types.is_null(license_type): + return self._string_transformer.transform(data) + if license_type.equals(pyarrow.list_(pyarrow.string())): + return self._list_transformer.transform(data) + + raise TypeError(f"Invalid {self._license_column} column type: {license_type}") + + +def _lower_case(value: str | None) -> str | None: + if value == None: + return None + return str.casefold(value) + + +class DenyLicenseStatusTransformer: + def __init__( + self, + license_column: str = None, + allow_no_license: bool = False, + licenses: list[str] = None, + ): + self._transformer = AllowLicenseStatusTransformer( + license_column=license_column, + allow_no_license=(not allow_no_license), + licenses=licenses, + ) + + def transform(self, data: Table) -> Table: + result = self._transformer.transform(data) + license_status = pc.invert(result.column("license_status")) + return _update_column(result, "license_status", license_status) + + +class LicenseStringTransformer: + """ + Transforms input tables with a string type license column + """ + + def __init__(self, allowed_licenses: Iterable[str]): + self._allowed_licenses = pyarrow.array(allowed_licenses) + + def transform(self, data: Table) -> Table: + license_type = data.schema.field("license").type + + if pyarrow.types.is_null(license_type): + orig_licenses = data.column("license") + licenses = pc.cast(orig_licenses, pyarrow.string()) + else: + licenses = pc.utf8_lower(data.column("license")) + + license_status = pc.is_in(licenses, value_set=self._allowed_licenses, skip_nulls=False) + return data.append_column("license_status", license_status) + + +class LicenseListTransformer: + """ + Transforms input tables with a list type license column + """ + + def __init__(self, allowed_licenses: Iterable[str]): + self._matcher = exact_matcher(allowed_licenses) + + def transform(self, data: Table) -> Table: + statuses = map(self._license_status, data.column("license")) + status_column = pyarrow.array(statuses, type=pyarrow.bool_()) + return data.append_column("license_status", status_column) + + def _license_status(self, licenses: ListScalar) -> bool: + licenses = licenses.as_py() + if len(licenses) == 0: + licenses.append(None) + + for license in map(_lower_case, licenses): + if not self._matcher(license): + return False + + return len(licenses) > 0 + + +def _rename_column(data: Table, from_column: str, to_column: str) -> Table: + if from_column == to_column: + return data + + new_names = [to_column if name == from_column else name for name in data.column_names] + return data.rename_columns(new_names) + + +def _update_column(data: Table, column_name: str, column: pyarrow.Array) -> Table: + column_index = data.column_names.index(column_name) + return data.set_column(column_index, column_name, column) diff --git a/transforms/code/license_select/python/test-data/expected/metadata.json b/transforms/code/license_select/python/test-data/expected/metadata.json new file mode 100644 index 000000000..105043eab --- /dev/null +++ b/transforms/code/license_select/python/test-data/expected/metadata.json @@ -0,0 +1,214 @@ +{ + "pipeline": "pipeline_id", + "job details": { + "job category": "preprocessing", + "job name": "license_select", + "job type": "pure python", + "job id": "job_id", + "start_time": "2024-06-27 15:53:31", + "end_time": "2024-06-27 15:53:31", + "status": "success" + }, + "code": null, + "job_input_params": { + "license_select_params": { + "license_column_name": "license", + "allow_no_license": false, + "licenses": [ + "MIT", + "Apache-2.0", + "BSD-3-Clause", + "Unlicense", + "CC0-1.0", + "BSD-2-Clause", + "CC-BY-4.0", + "CC-BY-3.0", + "0BSD", + "WTFPL", + "MIT-0", + "ISC", + "ADSL", + "BSL-1.0", + "Zlib", + "FTL", + "MS-PL", + "BSD-2-Clause-FreeBSD", + "FSFAP", + "BSD-Source-Code", + "Apache-1.1", + "BSD-4-Clause", + "Ruby", + "MulanPSL-1.0", + "BSD-1-Clause", + "X11", + "Condor-1.1", + "PostgreSQL", + "CECILL-B", + "Intel", + "Vim", + "Naumen", + "OML", + "BSD-3-Clause-Clear", + "AML", + "PHP-3.01", + "OpenSSL", + "PSF-2.0", + "Xnet", + "Linux-OpenIB", + "BSD-3-Clause-LBNL", + "UPL-1.0", + "BlueOak-1.0.0", + "Info-ZIP", + "BSD-4-Clause-UC", + "bzip2-1.0.6", + "W3C", + "W3C-20150513", + "DOC", + "ICU", + "CC-BY-2.0", + "curl", + "MTLL", + "OLDAP-2.2.1", + "ECL-2.0", + "Adobe-Glyph", + "BSD-2-Clause-Patent", + "IJG", + "PHP-3.0", + "ZPL-2.1", + "MIT-advertising", + "NCSA", + "Fair", + "BSD-3-Clause-Attribution", + "OLDAP-2.3", + "NLPL", + "BSD-3-Clause-Open-MPI", + "Python-2.0", + "NASA-1.3", + "TCL", + "BSD-3-Clause-No-Nuclear-Warranty", + "ImageMagick", + "Net-SNMP", + "OLDAP-2.5", + "MIT-feh", + "OLDAP-2.4", + "MITNFA", + "libpng-2.0", + "EFL-2.0", + "OLDAP-2.7", + "IBM-pibs", + "libtiff", + "OLDAP-2.8", + "Adobe-2006", + "BSD-2-Clause-NetBSD", + "zlib-acknowledgement", + "OLDAP-2.6", + "BSD-3-Clause-No-Nuclear-License-2014", + "OLDAP-1.4", + "Libpng", + "MIT-CMU", + "JasPer-2.0", + "Zend-2.0", + "TCP-wrappers", + "XFree86-1.1", + "FSFUL", + "OLDAP-1.3", + "SGI-B-2.0", + "NetCDF", + "Zed", + "ZPL-2.0", + "Apache-1.0", + "CC-BY-1.0", + "OLDAP-2.1", + "OLDAP-1.2", + "OLDAP-2.0", + "NTP", + "AMPAS", + "Barr", + "mpich2", + "ANTLR-PD", + "Xerox", + "Spencer-94", + "AMDPLPA", + "BSD-3-Clause-No-Nuclear-License", + "HPND", + "ECL-1.0", + "MirOS", + "Qhull", + "ZPL-1.1", + "TU-Berlin-2.0", + "Spencer-86", + "SMLNJ", + "xinetd", + "OLDAP-2.2.2", + "MIT-enna", + "Font-exception-2.0", + "FSFULLR", + "TU-Berlin-1.0", + "xpp", + "NRL", + "W3C-19980720", + "EFL-1.0", + "eGenix", + "Unicode-DFS-2016", + "SWL", + "Spencer-99", + "Plexus", + "VSL-1.0", + "Leptonica", + "Unicode-DFS-2015", + "Mup", + "Giftware", + "OLDAP-2.2", + "APAFML", + "NBPL-1.0", + "OLDAP-1.1", + "Entessa", + "Multics", + "Newsletr", + "psutils", + "bzip2-1.0.5", + "Afmparse", + "diffmark", + "BSD-2-Clause-Views", + "DSDP", + "MIT-Modern-Variant", + "ANTLR-PD-fallback", + "Bahyph", + "BSD-3-Clause-Modification", + "BSD-4-Clause-Shortened", + "HTMLTIDY", + "MIT-open-group", + "MulanPSL-2.0", + "OLDAP-2.0.1", + "Saxpath", + "Borceux", + "Crossword", + "CrystalStacker", + "Rdisc", + "Wsuipa" + ], + "deny": false + }, + "checkpointing": false, + "max_files": -1, + "random_samples": -1, + "files_to_use": [".parquet"] + }, + "job_output_stats": { + "source_files": 2, + "source_size": 6991, + "result_files": 2, + "result_size": 7511, + "processing_time": 0.032202959060668945, + "source_doc_count": 4, + "result_doc_count": 4 + }, + "source": { + "name": "../test-data/input", + "type": "path" + }, + "target": { + "name": "../output", + "type": "path" + } +} diff --git a/transforms/code/license_select/python/test-data/expected/sample_1.parquet b/transforms/code/license_select/python/test-data/expected/sample_1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..6ef75f6fb15d106369160eb67c58963bb4f6f9d3 GIT binary patch literal 3586 zcmds4&2Jh<6dy1+cH&r8)J4!*)n=<8M{-=i#tms&bszzj0CA$EPNhsv3KD=6(Hs@6DTMdk)*t<*=rtTN`)TbGFlk?a2n0 z*9@c0itVnU-oAaCyZh-#0%ygLY?}?P2S4z|zi?$kS6JJT9C-EGy1HD79XR0MV4psd zx((Y`-#8fBj%IGPU1uw(tr*q@7c6YfgdH)6xy6Jwf#e#q$Skh!q7BJIp$sSiLhB7M zkMbNI;XeaNQz4L-a~XwG|F{ZJgk{zb&w$Uj;E(yX8fT* z=+9v2_h9t3Kf3ha&nZIcj~G9K#~Qp4g8loPc6eS%ApHz6@B* zmgm*=nerHt)o`VTM$7ZU7y8;4{dGC|Vwu!u>VGAbb%vc#S>0IxjND$4yh{Ax6wxD8C#9VgPEwr65O`lAf^iWG(RI56%LH9&WEmelgN+yOmqb_Q*$I$awIT6=QIn<;Wr5JJvz+ZyU&eg$jq|f%4CO7R(%vsGl#`I>@6_6K-w@0LB-qj?h-K9ri=yF%;4#`P$hEliN z?h^t%t!wLTi%8?it=LwAgYs}=^ciJh3qITkg;#K;@#O|kF~*FPp-sm3FW@jm>J zuP-s)!RZFhV%)%)29>4LbNW6JKA!5!`x$Q!Pa1{ld5rrQ{NPQ3WNh++J;sF?*rz|? zO~b9{gq>=l+>`TyBU>f?iJq3VVWBID$y2`7Ztzy2DVwUn>-`3ACY!Q&L~LAD9N5JS z)|Kr_Q?d3zhWq1vPs#AR{3G7xGlerXC-h02QZ=p;eJzo)RWa^X6CQTGI>mHrV%#9M zvL)DZhTzpZb#ZW7OSdtf%vEO1BwkC<9$s?#k6Xg#^ueF zE}240%}~8luU486BUU>AzIj`(i^-OpsB~mgq45Nvk}DoI^pi{6l_A(?9jM=g7tPCs zTwOI^(n+3m(=g;*tZ?Pok!eIb6JQ{X-?~wkJ%)b8C)B1ELF@}=?D}eNQh6v6@(Es=3G0emObl zH=ws>apAlKS4ns=uc(;~n1*bXy?XpxyacEjoR4A(z&AA{wFI7kT^QD?g(TVYRyqmm zp}eio{8UpIDhZ<}>*Q?tLHS^(Uu^L@$q{TP^-I*Ng)>P^xim(LcT4)tkhfyw7Usx9 z8TTt`s;cBJleS99(uoz6WvgxT?sWxeqm RA%6uV`i zMe2-#srZhi8jfbf=z^kY%_WL&TAKU`y?Fi_47CJ{YB-KbJ)|t6jN~pq&{;4;@-g=-h59!aGKtq4@k##Y69A+Kwxqr`q%UV zIU7k4!MR9yM)E1O$%)WZ=&PB~pVQIbC!;g}DS9G!R}ka`0nHpd2m;12a;bkd?r!0^ zeDemhMWH3^NtS9lV>Z-{3u$;F-vsqx-rwFD;$0Jrx(g@GcZcSxo$8 zGh^}5WVDa(&Z9|irK?U59tgHfLm2~Z5){HgUwjGKo4E^Q)0!$NhV27;6$pJBi2iar z`usKqHezB7DuVooocGDZM=Tr&gabr~hS2(#{O`Jc4fp727Gp_W^j#@cV6-_=Io%3NGI=ws5HB_l7`v_d!><$D?@mP@@j7(P? zK^7bV9G}sO0qUwzhXH-CpIct1&xgQ_q5y>61#Dw0BUnZxS3-pOI>(oNFG{+9eBbps zEcw7b{TevN<%Qtt9=)977on!*r9x9hOR%~)6J}SitY#YnfHx!?-j48WVlAGCGc?dc z_q-1YlX3Xc=R|lGmCH}~2YzV`|8V0z{-o=V5kkB0k3XLyw2CryltZYFvUP7RlNph> z@#N+}UfRw2V`x1qo}5LfjbI%-v5~BeA23F!$bf$4Q&!j9TAo|2Bug!^z&WC^tv*pR zqS7r|0-rizkDGPYC^kf0)>yS&XZ2J=qXNy_vuV&Wx&PgRX#&13c z*mI|qm2yhTV@ybu`IHIvPilLfZ@`#nD_ojUDjCl%tEW{#=Z@v9CwFRtBU^~Tt4+hp_ZrWAgxLPS4$px+5(@BWBhB5T{PKcFEYB)wQS$tOGON^h7geK>6 zV58g^@K#`K9Ltbj8~hkDpEG*A=mO8#wOo3@H>-DWjuTbBmuDjbW5{b_h&zLG1-{g( z0oOP_$dMxzv|;S6s%~o4m0e%Xb4-;lH{|^0V80ygwd=6AhW$dmI9E#gHpg&ggHKH~ zN1?j?UdBj z&ekzfag3nb=Py~S6i)>{?RqwjS$A7q?Xm`gPhlQ6Qg0ta7tS+6CT{ZHAzsB@@4OaH zp-8O5Xfy9A;nUvJ_j*t-Ky$!VWu?Txo=8Yg2BaNWx;9v!^VUOg8C@@ptVhLTT$K)R zg9Vu~R!3OL$AddwT|r31_*b|_EqUNgcCfz=!o>^fP8DU^Xd4T_Z@<3v?Rg-)_`H=^ zA66vO{d45ag1o=q~c6G8NsiXgFoU0{3_UiAJl&X D^h#pA literal 0 HcmV?d00001 diff --git a/transforms/code/license_select/python/test-data/input/sample_1.parquet b/transforms/code/license_select/python/test-data/input/sample_1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..51fd1b49b07f56495fda663f8fbf6926471d15c8 GIT binary patch literal 3326 zcmds4&2Ji45MMAjcH>r2)J4!*)n=<8M{*p%#to@kbs+|pVq#-^7lZ!*ba_YJ2p@;qno%a^-l5lELsp{H(A2ajj_nVpbM#(vS0YJp(~8*NghmnOIMfoGKU_V?=p{{ zNKV&{bk+`suBVwBmhWxEwG{)cvGLk^$nA)x=$mw69Z24w7wN^-ZE8&lPzVD;0MmL6 zETBAthxvy9Nhlcd)mTQ!!avReBw?9}!WfH07NY527SO<&zEt2rswBUS*4i?mG`z50Hd;B$I~8vG>6Lq zh?5Io@(P(BR<6iggR83{H{kMBB=I7WetA9p&2`M>>ij%1sd}DDF1kujGbMVFqE~K5 zuPT3=s;>fSAe_V-7EW--vP3Mkn%eo_-lFu|sDJux)GhJHwL~nvl=%JH`xKR0PcGUL zQYAO}UHp>Ok*ZysD3;-y$Ymcek1yGCkim*|;L2+3I>RE#)IflI!+;(1Jja7=hO8|c zmN~N2@G?7k0-M0!330BC-OhSB)ipysnUw{!PQ0Ow;aDH(ilqh&eA6CI6_$LaDIR8K zYM!J@o&@HXtcJiG8ZbV4d{n-BpB;tB%(4K6LA%0UX1L5~-iQ&-*JvGxjiY2IZ{l^# zu;wje22Xts|rEch@R)T0SD(`#!gBuo?y>$wG7x7KjBTo?^J}XR<6;PtAZz^J^hhh zl(k{ak;MEte`a-gRO`v6YVi7?%bWS0EFNPUUllLhdWqAO&1O$Q2O#4H<9%N#@!R}E z-sMZRXKF)0Lf;=GlcgIg&B z_wuvWW;3r;Fegq|#k>vv&pU?%8#rbMno!7Sts?Qun+07mg)_B8^j@dc?17I=`w;k6 zUA-gb&*WUwmQ98D6NF}^e$>@ZU*WC{!M^N4{wBOgTt39=tJOUn$JsFrL#{MWWwY!7 zpKZtm_|9hlBLnAgZU$}0al6$xRjbBeY?GBu19O<CopNECZ{EDXHO{uhah*{S_E6VsD0fKkC9%=( zK(29oP$N&N8e#7JwrLyf&4WNM3YoUp?5UOA>3ump9CYDsg>j)?g0JKPpL25N0;VCO zMv#wxi zRg&}^{mU%<<7^oQ)NP7-9r$vWq9{{R&ZQ<|3`xU`W|(cuVc3S@uuQ|n3KRI~_eZf+ z=7IrH{J_$%qvIG`M7rKyW5l+ls~@wggO}i_BUwz#aZKhRW1(lRW}z5M=IrXujqqk% zv5@4TBV=`u6$?DwbhI{RVqm!j=5_%63T?b=X$~qPNsXlvi8bbe&9$+Eu(N*LHxYZE zVI9}B8|PzQIZF(j&7y4p8^ykg!Z?n<){}eqDaw;f#hJt1pI*NjMj9wevjXPx`E3_A( zElC~eNU=22x#mDEUTVWLg*KQ^^8WhP1n;_pn=afmkM~!B(0768SGS@+EfM0cy15n~ zOGiieK72F_p$yFlLdU_8X`ySt&4NKV=&P?Fd$oLt?53_M2-`le7lF_>f#}a4L_hz4 z0Go1g9jZtos`2V})?EYJP{=>KV9!BD zO1cI20I2 zr)55@Q)l=}aj~YfRUd(CnEkP$EuSI9A;=8mNUG#W5cq;c7|b=?gb90coL}E&2NPhX zQ2;{k{bds`Q&^@n?}Ui(H6ZWG;V#+HMSC;mq~<-x^h?M!Aukp09?{D=`HpB?UMh4= zwj`@hGGX>5%W8KC0QkVL;p2ixlUwm*Ji!7z>@fO-G8Km(c0h%fxJu=Tc;7E#;nxfI z$g|aa%yHZy{E_EVj@#t2O)k%IO)l5;WZCSLzDtC=WBvYN&Ywf;dGYi-$8|Ya2R(6; zvq=YYjw>a=KKlu8=x)6rY*thI9l0ntGTzgkXjvKcOO_<2&-t@av2j(gXP0xFrfgJN3O)oG*B#C~N{-*+AM-Y!D?L{WLYMg2t0q;Vucb1!Dkj}( z%8y;Ij$^ttF{u+<84I?YBk}5HUF@A#Q`f{TSDv1pnvtBQG{! zOm>lwNuX-h^UE6y5CTX zdUt4(lntF=81bEytJ(DA9_3W&SzX*u`1wfcY9S9kDy=bZ#e_}vG7;B?I9L_)c*u(( ziGp3vXU2T-oTAH@&=Z>CSU1e_4N z)UQ=bX)@<=HVxyxyrFpeQw^c7r1Xxgk=*jVeQmQ>KI1g9N3b37m#SAw&m}SAdOps0 zcTd~w^Eg3nVSzNt@EpUC)BsK)jW;6qo8znls1M{iAD(zO+gmg)6>y7*uFxCt$Qi^w za}3QkD0cIn6VJ%^2SRvAJ;+|33(xc8U08Gv>`#au>Q9l>4|op!qCtj3f27)Y^~dX> z@%&w$Z*LiiH?frC9#Ub(l&ohia@G~oT=$A7Gc1yfUp^xb;3LlerT$o;r~~+C@^8br BG>HHJ literal 0 HcmV?d00001 diff --git a/transforms/code/license_select/python/test-data/sample_approved_licenses.json b/transforms/code/license_select/python/test-data/sample_approved_licenses.json new file mode 100644 index 000000000..6b4a37698 --- /dev/null +++ b/transforms/code/license_select/python/test-data/sample_approved_licenses.json @@ -0,0 +1,173 @@ +[ + "MIT", + "Apache-2.0", + "BSD-3-Clause", + "Unlicense", + "CC0-1.0", + "BSD-2-Clause", + "CC-BY-4.0", + "CC-BY-3.0", + "0BSD", + "WTFPL", + "MIT-0", + "ISC", + "ADSL", + "BSL-1.0", + "Zlib", + "FTL", + "MS-PL", + "BSD-2-Clause-FreeBSD", + "FSFAP", + "BSD-Source-Code", + "Apache-1.1", + "BSD-4-Clause", + "Ruby", + "MulanPSL-1.0", + "BSD-1-Clause", + "X11", + "Condor-1.1", + "PostgreSQL", + "CECILL-B", + "Intel", + "Vim", + "Naumen", + "OML", + "BSD-3-Clause-Clear", + "AML", + "PHP-3.01", + "OpenSSL", + "PSF-2.0", + "Xnet", + "Linux-OpenIB", + "BSD-3-Clause-LBNL", + "UPL-1.0", + "BlueOak-1.0.0", + "Info-ZIP", + "BSD-4-Clause-UC", + "bzip2-1.0.6", + "W3C", + "W3C-20150513", + "DOC", + "ICU", + "CC-BY-2.0", + "curl", + "MTLL", + "OLDAP-2.2.1", + "ECL-2.0", + "Adobe-Glyph", + "BSD-2-Clause-Patent", + "IJG", + "PHP-3.0", + "ZPL-2.1", + "MIT-advertising", + "NCSA", + "Fair", + "BSD-3-Clause-Attribution", + "OLDAP-2.3", + "NLPL", + "BSD-3-Clause-Open-MPI", + "Python-2.0", + "NASA-1.3", + "TCL", + "BSD-3-Clause-No-Nuclear-Warranty", + "ImageMagick", + "Net-SNMP", + "OLDAP-2.5", + "MIT-feh", + "OLDAP-2.4", + "MITNFA", + "libpng-2.0", + "EFL-2.0", + "OLDAP-2.7", + "IBM-pibs", + "libtiff", + "OLDAP-2.8", + "Adobe-2006", + "BSD-2-Clause-NetBSD", + "zlib-acknowledgement", + "OLDAP-2.6", + "BSD-3-Clause-No-Nuclear-License-2014", + "OLDAP-1.4", + "Libpng", + "MIT-CMU", + "JasPer-2.0", + "Zend-2.0", + "TCP-wrappers", + "XFree86-1.1", + "FSFUL", + "OLDAP-1.3", + "SGI-B-2.0", + "NetCDF", + "Zed", + "ZPL-2.0", + "Apache-1.0", + "CC-BY-1.0", + "OLDAP-2.1", + "OLDAP-1.2", + "OLDAP-2.0", + "NTP", + "AMPAS", + "Barr", + "mpich2", + "ANTLR-PD", + "Xerox", + "Spencer-94", + "AMDPLPA", + "BSD-3-Clause-No-Nuclear-License", + "HPND", + "ECL-1.0", + "MirOS", + "Qhull", + "ZPL-1.1", + "TU-Berlin-2.0", + "Spencer-86", + "SMLNJ", + "xinetd", + "OLDAP-2.2.2", + "MIT-enna", + "Font-exception-2.0", + "FSFULLR", + "TU-Berlin-1.0", + "xpp", + "NRL", + "W3C-19980720", + "EFL-1.0", + "eGenix", + "Unicode-DFS-2016", + "SWL", + "Spencer-99", + "Plexus", + "VSL-1.0", + "Leptonica", + "Unicode-DFS-2015", + "Mup", + "Giftware", + "OLDAP-2.2", + "APAFML", + "NBPL-1.0", + "OLDAP-1.1", + "Entessa", + "Multics", + "Newsletr", + "psutils", + "bzip2-1.0.5", + "Afmparse", + "diffmark", + "BSD-2-Clause-Views", + "DSDP", + "MIT-Modern-Variant", + "ANTLR-PD-fallback", + "Bahyph", + "BSD-3-Clause-Modification", + "BSD-4-Clause-Shortened", + "HTMLTIDY", + "MIT-open-group", + "MulanPSL-2.0", + "OLDAP-2.0.1", + "Saxpath", + "Borceux", + "Crossword", + "CrystalStacker", + "Rdisc", + "Wsuipa" +] diff --git a/transforms/code/license_select/python/test/test_license_select.py b/transforms/code/license_select/python/test/test_license_select.py new file mode 100644 index 000000000..fc0b6a9ba --- /dev/null +++ b/transforms/code/license_select/python/test/test_license_select.py @@ -0,0 +1,88 @@ +# (C) Copyright IBM Corp. 2024. +# Licensed under the Apache License, Version 2.0 (the “License”); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an “AS IS” BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import os + +import pyarrow as pa +from data_processing.test_support.transform import AbstractTableTransformTest +from data_processing.transform import get_transform_config +from license_select_transform import ( + LICENSE_COLUMN_NAME_CLI_KEY, + LICENSE_SELECT_PARAMS, + LICENSES_FILE_CLI_KEY, + LicenseSelectTransform, + LicenseSelectTransformConfiguration, +) + + +class TestLicenseSelectTransform(AbstractTableTransformTest): + """ + Extends the super-class to define the test data for the tests defined there. + The name of this class MUST begin with the word Test so that pytest recognizes it as a test class. + """ + + table = pa.table( + { + "document_id": ["ID_1", "ID_2"], + "license": ["Apache-2.0", "BAD_LICENSE"], + } + ) + expected_table = pa.table( + { + "document_id": ["ID_1", "ID_2"], + "license": ["Apache-2.0", "BAD_LICENSE"], + "license_status": [True, False], + } + ) + expected_metadata_list = [{}, {}] + + def get_test_transform_fixtures(self) -> list[tuple]: + test_src_dir = os.path.abspath(os.path.dirname(__file__)) + approved_license_file = os.path.abspath( + os.path.join(test_src_dir, "../test-data/sample_approved_licenses.json") + ) + cli = [ + # When running outside the Ray orchestrator and its DataAccess/Factory, there is + # no Runtime class to load the domains and the Transform must do it itself using + # the lang_select_local_config for this test. + f"--{LICENSE_COLUMN_NAME_CLI_KEY}", + "license", + f"--{LICENSES_FILE_CLI_KEY}", + approved_license_file, + ] + + # Use the ProgLangMatchTransformConfiguration to compute the config parameters + lstc = LicenseSelectTransformConfiguration() + config = get_transform_config(lstc, cli) + + fixtures = [ + ( + LicenseSelectTransform( + { + LICENSE_SELECT_PARAMS: { + "license_column_name": "license", + "allow_no_license": False, + "licenses": ["Apache-2.0"], + "deny": False, + } + } + ), + [self.table], + [self.expected_table], + self.expected_metadata_list, + ), + ] + return fixtures + + +if __name__ == "__main__": + t = TestLicenseSelectTransform() diff --git a/transforms/code/license_select/python/test/test_license_select_python.py b/transforms/code/license_select/python/test/test_license_select_python.py new file mode 100644 index 000000000..329ba2862 --- /dev/null +++ b/transforms/code/license_select/python/test/test_license_select_python.py @@ -0,0 +1,41 @@ +# (C) Copyright IBM Corp. 2024. +# Licensed under the Apache License, Version 2.0 (the “License”); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an “AS IS” BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import os + +from data_processing.runtime.pure_python import PythonTransformLauncher +from data_processing.test_support.launch.transform_test import ( + AbstractTransformLauncherTest, +) +from license_select_transform import LICENSE_COLUMN_NAME_CLI_KEY, LICENSES_FILE_CLI_KEY +from license_select_transform_python import LicenseSelectPythonTransformConfiguration + + +class TestPythonLicenseSelect(AbstractTransformLauncherTest): + """ + Extends the super-class to define the test data for the tests defined there. + The name of this class MUST begin with the word Test so that pytest recognizes it as a test class. + """ + + def get_test_transform_fixtures(self) -> list[tuple]: + basedir = "../test-data" + basedir = os.path.abspath(os.path.join(os.path.dirname(__file__), basedir)) + fixtures = [] + launcher = PythonTransformLauncher(LicenseSelectPythonTransformConfiguration()) + license_file_path = os.path.join(basedir, "sample_approved_licenses.json") + print(license_file_path) + config = { + LICENSE_COLUMN_NAME_CLI_KEY: "license", + LICENSES_FILE_CLI_KEY: license_file_path, + } + fixtures.append((launcher, config, basedir + "/input", basedir + "/expected")) + return fixtures diff --git a/transforms/code/license_select/ray/.dockerignore b/transforms/code/license_select/ray/.dockerignore new file mode 100644 index 000000000..f7275bbbd --- /dev/null +++ b/transforms/code/license_select/ray/.dockerignore @@ -0,0 +1 @@ +venv/ diff --git a/transforms/code/license_select/ray/Dockerfile b/transforms/code/license_select/ray/Dockerfile new file mode 100644 index 000000000..39e4674bb --- /dev/null +++ b/transforms/code/license_select/ray/Dockerfile @@ -0,0 +1,37 @@ +ARG BASE_IMAGE=docker.io/rayproject/ray:2.24.0-py310 + +FROM ${BASE_IMAGE} + +RUN pip install --upgrade --no-cache-dir pip + +# install pytest +RUN pip install --no-cache-dir pytest + +# Copy in the data processing framework source/project and install it +# This is expected to be placed in the docker context before this is run (see the make image). +COPY --chown=ray:users data-processing-lib-python/ data-processing-lib-python/ +RUN cd data-processing-lib-python && pip install --no-cache-dir -e . +COPY --chown=ray:users data-processing-lib-ray/ data-processing-lib-ray/ +RUN cd data-processing-lib-ray && pip install --no-cache-dir -e . +COPY --chown=ray:users python-transform/ python-transform/ +RUN cd python-transform && pip install --no-cache-dir -e . + +COPY --chown=ray:users src/ src/ +COPY --chown=ray:users pyproject.toml pyproject.toml +COPY --chown=ray:users README.md README.md +RUN pip install --no-cache-dir -e . + +# copy source data +COPY src/license_select_transform_ray.py . +COPY src/license_select_local_ray.py local/ + +# copy test +COPY test/ test/ +COPY test-data/ test-data/ + +# Put these at the end since they seem to upset the docker cache. +ARG BUILD_DATE +ARG GIT_COMMIT +LABEL build-date=$BUILD_DATE +LABEL git-commit=$GIT_COMMIT + diff --git a/transforms/code/license_select/ray/Makefile b/transforms/code/license_select/ray/Makefile new file mode 100644 index 000000000..9f15ba4ce --- /dev/null +++ b/transforms/code/license_select/ray/Makefile @@ -0,0 +1,65 @@ + +# Define the root of the local git clone for the common rules to be able +# know where they are running from. +REPOROOT=../../../.. +# Include a library of common .transform.* targets which most +# transforms should be able to reuse. However, feel free +# to override/redefine the rules below. + +include $(REPOROOT)/transforms/.make.transforms + +BASE_IMAGE=$(RAY_BASE_IMAGE) +TRANSFORM_NAME=license_select +# $(REPOROOT)/.make.versions file contains the versions +DOCKER_IMAGE_VERSION=${LICENSE_SELECT_RAY_VERSION} + +# Use default rule inherited from makefile.common +clean:: .transforms.clean + +# Use default rule inherited from makefile.common +test:: .transforms.ray-test + +# Use default rule inherited from makefile.common +image:: .transforms.ray-image + +# Use default rule inherited from makefile.common +venv:: .transforms.ray-venv + +image:: .transforms.ray-image + +test-src:: .transforms.test-src + +setup:: .transforms.setup + +test-image:: .transforms.ray-test-image + +build:: build-dist image + +publish:: publish-dist publish-image + +publish-image:: .transforms.publish-image-ray + +setup:: .transforms.setup + +# distribution versions is the same as image version. +set-versions: + $(MAKE) TRANSFORM_PYTHON_VERSION=${LICENSE_SELECT_PYTHON_VERSION} TOML_VERSION=$(LICENSE_SELECT_RAY_VERSION) .transforms.set-versions + +build-dist:: .defaults.build-dist + +publish-dist:: .defaults.publish-dist + +run-cli-ray-sample:. + $(MAKE) RUN_FILE=$(TRANSFORM_NAME)_transform_ray.py \ + RUN_ARGS="--run_locally True --data_local_config \"{ 'input_folder' : '../../python/test-data/input', 'output_folder' : '../output'}\" \ + --lc_license_column_name license \ + --lc_licenses_file ../test-data/sample_approved_licenses.json" \ + .transforms.run-src-file + +run-local-sample: .transforms.run-local-ray-sample + +run-s3-sample: .transforms.run-s3-ray-sample + +minio-start: .minio-start + +load-image:: .transforms.load-image diff --git a/transforms/code/license_select/ray/README.md b/transforms/code/license_select/ray/README.md new file mode 100644 index 000000000..8a8e6a67c --- /dev/null +++ b/transforms/code/license_select/ray/README.md @@ -0,0 +1,50 @@ +# License Select + +Please see the set of +[transform project conventions](../../../README.md) +for details on general project conventions, transform configuration, +testing and IDE set up. + +## Summary + +This project wraps the [license select transform](../python/README.md) with a Ray runtime. + +## Running + +### Launcher Command Line Options + +In addition to those available to the transform as defined in [here](../python/README.md), +the set of +[ray launcher](../../../../data-processing-lib/doc/ray-launcher-options.md) are available. + +### Running the samples + +To run the samples, use the following `make` targets + +* `run-cli-ray-sample` +* `run-local-ray-sample` +* `run-s3-ray-sample` + * Requires prior invocation of `make minio-start` to load data into local minio for S3 access. + +These targets will activate the virtual environment and set up any configuration needed. +Use the `-n` option of `make` to see the detail of what is done to run the sample. + +For example, + +```shell +make run-cli-ray-sample +... +``` + +Then + +```shell +ls output +``` +To see results of the transform. + +### Transforming data using the transform image + +To use the transform image to transform your data, please refer to the +[running images quickstart](../../../../doc/quick-start/run-transform-image.md), +substituting the name of this transform image and runtime as appropriate. diff --git a/transforms/code/license_select/ray/pyproject.toml b/transforms/code/license_select/ray/pyproject.toml new file mode 100644 index 000000000..89b4b9ea5 --- /dev/null +++ b/transforms/code/license_select/ray/pyproject.toml @@ -0,0 +1,46 @@ +[project] +name = "dpk_license_select_transform_ray" +version = "0.2.1.dev0" +requires-python = ">=3.10" +description = "License Select Transform" +license = {text = "Apache-2.0"} +readme = {file = "README.md", content-type = "text/markdown"} +authors = [ + { name = "Shivdeep Singh", email = "shivdeep.singh@ibm.com" }, + { name = "Mark Lewis", email = "mark_lewis@uk.ibm.com" }, +] +dependencies = [ + "dpk-license-select-transform-python==0.2.1.dev0", + "data-prep-toolkit-ray==0.2.1.dev0", +] + +[build-system] +requires = ["setuptools>=68.0.0", "wheel", "setuptools_scm[toml]>=7.1.0"] +build-backend = "setuptools.build_meta" + +[project.optional-dependencies] +dev = [ + "twine", + "pytest>=7.3.2", + "pytest-dotenv>=0.5.2", + "pytest-env>=1.0.0", + "pre-commit>=3.3.2", + "pytest-cov>=4.1.0", + "pytest-mock>=3.10.0", + "moto==5.0.5", + "markupsafe==2.0.1", +] + +[options] +package_dir = ["src","test"] + +[options.packages.find] +where = ["src/"] + +[tool.pytest.ini_options] +# Currently we use low coverage since we have to run tests separately (see makefile) +#addopts = "--cov --cov-report term-missing --cov-fail-under 25" +markers = ["unit: unit tests", "integration: integration tests"] + +[tool.coverage.run] +include = ["src/*"] diff --git a/transforms/code/license_select/ray/src/license_select_local_ray.py b/transforms/code/license_select/ray/src/license_select_local_ray.py new file mode 100644 index 000000000..10fdced1d --- /dev/null +++ b/transforms/code/license_select/ray/src/license_select_local_ray.py @@ -0,0 +1,61 @@ +# (C) Copyright IBM Corp. 2024. +# Licensed under the Apache License, Version 2.0 (the “License”); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an “AS IS” BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import os +import sys +from pathlib import Path + +from data_processing.utils import ParamsUtils +from data_processing_ray.runtime.ray import RayTransformLauncher +from license_select_transform_ray import LicenseSelectRayTransformConfiguration + + +input_folder = os.path.abspath(os.path.join(os.path.dirname(__file__), "../test-data/input")) +output_folder = os.path.abspath(os.path.join(os.path.dirname(__file__), "../output")) +approved_licenses_file = os.path.abspath( + os.path.join(os.path.dirname(__file__), "../test-data/sample_approved_licenses.json") +) + + +local_conf = { + "input_folder": input_folder, + "output_folder": output_folder, +} + +# create launcher +launcher = RayTransformLauncher(LicenseSelectRayTransformConfiguration()) + + +worker_options = {"num_cpus": 0.8} +code_location = {"github": "github", "commit_hash": "12345", "path": "path"} +params = { + # where to run + "run_locally": True, + # Data access. Only required parameters are specified + "data_local_config": ParamsUtils.convert_to_ast(local_conf), + # orchestrator + "runtime_worker_options": ParamsUtils.convert_to_ast(worker_options), + "runtime_num_workers": 3, + "runtime_pipeline_id": "pipeline_id", + "runtime_job_id": "job_id", + "runtime_creation_delay": 0, + "runtime_code_location": ParamsUtils.convert_to_ast(code_location), + # license select configuration + "lc_license_column_name": "license", + "lc_licenses_file": approved_licenses_file, +} + +if __name__ == "__main__": + Path(output_folder).mkdir(parents=True, exist_ok=True) + sys.argv = ParamsUtils.dict_to_req(d=params) + # launch + launcher.launch() diff --git a/transforms/code/license_select/ray/src/license_select_s3_ray.py b/transforms/code/license_select/ray/src/license_select_s3_ray.py new file mode 100644 index 000000000..6d26df93c --- /dev/null +++ b/transforms/code/license_select/ray/src/license_select_s3_ray.py @@ -0,0 +1,56 @@ +# (C) Copyright IBM Corp. 2024. +# Licensed under the Apache License, Version 2.0 (the “License”); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an “AS IS” BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +############################################################################### + +import sys + +from data_processing.utils import ParamsUtils +from data_processing_ray.runtime.ray import RayTransformLauncher +from license_select_transform_ray import LicenseSelectRayTransformConfiguration + + +s3_cred = { + "access_key": "localminioaccesskey", + "secret_key": "localminiosecretkey", + "url": "http://localhost:9000", +} +s3_conf = { + "input_folder": "test-data/input", + "output_folder": "test-data/output", +} + +# create launcher +launcher = RayTransformLauncher(LicenseSelectRayTransformConfiguration()) + + +worker_options = {"num_cpus": 0.8} +code_location = {"github": "github", "commit_hash": "12345", "path": "path"} +params = { + # where to run + "run_locally": True, + # Data access. Only required parameters are specified + "data_s3_cred": ParamsUtils.convert_to_ast(s3_cred), + "data_s3_config": ParamsUtils.convert_to_ast(s3_conf), + # orchestrator + "runtime_worker_options": ParamsUtils.convert_to_ast(worker_options), + "runtime_num_workers": 3, + "runtime_pipeline_id": "pipeline_id", + "runtime_job_id": "job_id", + "runtime_creation_delay": 0, + "runtime_code_location": ParamsUtils.convert_to_ast(code_location), + "lc_license_column_name": "license", + "lc_licenses_file": "test-data/sample_approved_licenses.json", +} + +if __name__ == "__main__": + sys.argv = ParamsUtils.dict_to_req(d=params) + # launch + launcher.launch() diff --git a/transforms/code/license_select/ray/src/license_select_transform_ray.py b/transforms/code/license_select/ray/src/license_select_transform_ray.py new file mode 100644 index 000000000..de384997a --- /dev/null +++ b/transforms/code/license_select/ray/src/license_select_transform_ray.py @@ -0,0 +1,30 @@ +# Licensed under the Apache License, Version 2.0 (the “License”); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an “AS IS” BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +################################################################################ +from data_processing_ray.runtime.ray import RayTransformLauncher +from data_processing_ray.runtime.ray.runtime_configuration import ( + RayTransformRuntimeConfiguration, +) +from license_select_transform import LicenseSelectTransformConfiguration + + +class LicenseSelectRayTransformConfiguration(RayTransformRuntimeConfiguration): + def __init__(self): + super().__init__(transform_config=LicenseSelectTransformConfiguration()) + + +def main(): + launcher = RayTransformLauncher(LicenseSelectRayTransformConfiguration()) + launcher.launch() + + +if __name__ == "__main__": + main() diff --git a/transforms/code/license_select/ray/test-data/expected/metadata.json b/transforms/code/license_select/ray/test-data/expected/metadata.json new file mode 100644 index 000000000..105043eab --- /dev/null +++ b/transforms/code/license_select/ray/test-data/expected/metadata.json @@ -0,0 +1,214 @@ +{ + "pipeline": "pipeline_id", + "job details": { + "job category": "preprocessing", + "job name": "license_select", + "job type": "pure python", + "job id": "job_id", + "start_time": "2024-06-27 15:53:31", + "end_time": "2024-06-27 15:53:31", + "status": "success" + }, + "code": null, + "job_input_params": { + "license_select_params": { + "license_column_name": "license", + "allow_no_license": false, + "licenses": [ + "MIT", + "Apache-2.0", + "BSD-3-Clause", + "Unlicense", + "CC0-1.0", + "BSD-2-Clause", + "CC-BY-4.0", + "CC-BY-3.0", + "0BSD", + "WTFPL", + "MIT-0", + "ISC", + "ADSL", + "BSL-1.0", + "Zlib", + "FTL", + "MS-PL", + "BSD-2-Clause-FreeBSD", + "FSFAP", + "BSD-Source-Code", + "Apache-1.1", + "BSD-4-Clause", + "Ruby", + "MulanPSL-1.0", + "BSD-1-Clause", + "X11", + "Condor-1.1", + "PostgreSQL", + "CECILL-B", + "Intel", + "Vim", + "Naumen", + "OML", + "BSD-3-Clause-Clear", + "AML", + "PHP-3.01", + "OpenSSL", + "PSF-2.0", + "Xnet", + "Linux-OpenIB", + "BSD-3-Clause-LBNL", + "UPL-1.0", + "BlueOak-1.0.0", + "Info-ZIP", + "BSD-4-Clause-UC", + "bzip2-1.0.6", + "W3C", + "W3C-20150513", + "DOC", + "ICU", + "CC-BY-2.0", + "curl", + "MTLL", + "OLDAP-2.2.1", + "ECL-2.0", + "Adobe-Glyph", + "BSD-2-Clause-Patent", + "IJG", + "PHP-3.0", + "ZPL-2.1", + "MIT-advertising", + "NCSA", + "Fair", + "BSD-3-Clause-Attribution", + "OLDAP-2.3", + "NLPL", + "BSD-3-Clause-Open-MPI", + "Python-2.0", + "NASA-1.3", + "TCL", + "BSD-3-Clause-No-Nuclear-Warranty", + "ImageMagick", + "Net-SNMP", + "OLDAP-2.5", + "MIT-feh", + "OLDAP-2.4", + "MITNFA", + "libpng-2.0", + "EFL-2.0", + "OLDAP-2.7", + "IBM-pibs", + "libtiff", + "OLDAP-2.8", + "Adobe-2006", + "BSD-2-Clause-NetBSD", + "zlib-acknowledgement", + "OLDAP-2.6", + "BSD-3-Clause-No-Nuclear-License-2014", + "OLDAP-1.4", + "Libpng", + "MIT-CMU", + "JasPer-2.0", + "Zend-2.0", + "TCP-wrappers", + "XFree86-1.1", + "FSFUL", + "OLDAP-1.3", + "SGI-B-2.0", + "NetCDF", + "Zed", + "ZPL-2.0", + "Apache-1.0", + "CC-BY-1.0", + "OLDAP-2.1", + "OLDAP-1.2", + "OLDAP-2.0", + "NTP", + "AMPAS", + "Barr", + "mpich2", + "ANTLR-PD", + "Xerox", + "Spencer-94", + "AMDPLPA", + "BSD-3-Clause-No-Nuclear-License", + "HPND", + "ECL-1.0", + "MirOS", + "Qhull", + "ZPL-1.1", + "TU-Berlin-2.0", + "Spencer-86", + "SMLNJ", + "xinetd", + "OLDAP-2.2.2", + "MIT-enna", + "Font-exception-2.0", + "FSFULLR", + "TU-Berlin-1.0", + "xpp", + "NRL", + "W3C-19980720", + "EFL-1.0", + "eGenix", + "Unicode-DFS-2016", + "SWL", + "Spencer-99", + "Plexus", + "VSL-1.0", + "Leptonica", + "Unicode-DFS-2015", + "Mup", + "Giftware", + "OLDAP-2.2", + "APAFML", + "NBPL-1.0", + "OLDAP-1.1", + "Entessa", + "Multics", + "Newsletr", + "psutils", + "bzip2-1.0.5", + "Afmparse", + "diffmark", + "BSD-2-Clause-Views", + "DSDP", + "MIT-Modern-Variant", + "ANTLR-PD-fallback", + "Bahyph", + "BSD-3-Clause-Modification", + "BSD-4-Clause-Shortened", + "HTMLTIDY", + "MIT-open-group", + "MulanPSL-2.0", + "OLDAP-2.0.1", + "Saxpath", + "Borceux", + "Crossword", + "CrystalStacker", + "Rdisc", + "Wsuipa" + ], + "deny": false + }, + "checkpointing": false, + "max_files": -1, + "random_samples": -1, + "files_to_use": [".parquet"] + }, + "job_output_stats": { + "source_files": 2, + "source_size": 6991, + "result_files": 2, + "result_size": 7511, + "processing_time": 0.032202959060668945, + "source_doc_count": 4, + "result_doc_count": 4 + }, + "source": { + "name": "../test-data/input", + "type": "path" + }, + "target": { + "name": "../output", + "type": "path" + } +} diff --git a/transforms/code/license_select/ray/test-data/expected/sample_1.parquet b/transforms/code/license_select/ray/test-data/expected/sample_1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..6ef75f6fb15d106369160eb67c58963bb4f6f9d3 GIT binary patch literal 3586 zcmds4&2Jh<6dy1+cH&r8)J4!*)n=<8M{-=i#tms&bszzj0CA$EPNhsv3KD=6(Hs@6DTMdk)*t<*=rtTN`)TbGFlk?a2n0 z*9@c0itVnU-oAaCyZh-#0%ygLY?}?P2S4z|zi?$kS6JJT9C-EGy1HD79XR0MV4psd zx((Y`-#8fBj%IGPU1uw(tr*q@7c6YfgdH)6xy6Jwf#e#q$Skh!q7BJIp$sSiLhB7M zkMbNI;XeaNQz4L-a~XwG|F{ZJgk{zb&w$Uj;E(yX8fT* z=+9v2_h9t3Kf3ha&nZIcj~G9K#~Qp4g8loPc6eS%ApHz6@B* zmgm*=nerHt)o`VTM$7ZU7y8;4{dGC|Vwu!u>VGAbb%vc#S>0IxjND$4yh{Ax6wxD8C#9VgPEwr65O`lAf^iWG(RI56%LH9&WEmelgN+yOmqb_Q*$I$awIT6=QIn<;Wr5JJvz+ZyU&eg$jq|f%4CO7R(%vsGl#`I>@6_6K-w@0LB-qj?h-K9ri=yF%;4#`P$hEliN z?h^t%t!wLTi%8?it=LwAgYs}=^ciJh3qITkg;#K;@#O|kF~*FPp-sm3FW@jm>J zuP-s)!RZFhV%)%)29>4LbNW6JKA!5!`x$Q!Pa1{ld5rrQ{NPQ3WNh++J;sF?*rz|? zO~b9{gq>=l+>`TyBU>f?iJq3VVWBID$y2`7Ztzy2DVwUn>-`3ACY!Q&L~LAD9N5JS z)|Kr_Q?d3zhWq1vPs#AR{3G7xGlerXC-h02QZ=p;eJzo)RWa^X6CQTGI>mHrV%#9M zvL)DZhTzpZb#ZW7OSdtf%vEO1BwkC<9$s?#k6Xg#^ueF zE}240%}~8luU486BUU>AzIj`(i^-OpsB~mgq45Nvk}DoI^pi{6l_A(?9jM=g7tPCs zTwOI^(n+3m(=g;*tZ?Pok!eIb6JQ{X-?~wkJ%)b8C)B1ELF@}=?D}eNQh6v6@(Es=3G0emObl zH=ws>apAlKS4ns=uc(;~n1*bXy?XpxyacEjoR4A(z&AA{wFI7kT^QD?g(TVYRyqmm zp}eio{8UpIDhZ<}>*Q?tLHS^(Uu^L@$q{TP^-I*Ng)>P^xim(LcT4)tkhfyw7Usx9 z8TTt`s;cBJleS99(uoz6WvgxT?sWxeqm RA%6uV`i zMe2-#srZhi8jfbf=z^kY%_WL&TAKU`y?Fi_47CJ{YB-KbJ)|t6jN~pq&{;4;@-g=-h59!aGKtq4@k##Y69A+Kwxqr`q%UV zIU7k4!MR9yM)E1O$%)WZ=&PB~pVQIbC!;g}DS9G!R}ka`0nHpd2m;12a;bkd?r!0^ zeDemhMWH3^NtS9lV>Z-{3u$;F-vsqx-rwFD;$0Jrx(g@GcZcSxo$8 zGh^}5WVDa(&Z9|irK?U59tgHfLm2~Z5){HgUwjGKo4E^Q)0!$NhV27;6$pJBi2iar z`usKqHezB7DuVooocGDZM=Tr&gabr~hS2(#{O`Jc4fp727Gp_W^j#@cV6-_=Io%3NGI=ws5HB_l7`v_d!><$D?@mP@@j7(P? zK^7bV9G}sO0qUwzhXH-CpIct1&xgQ_q5y>61#Dw0BUnZxS3-pOI>(oNFG{+9eBbps zEcw7b{TevN<%Qtt9=)977on!*r9x9hOR%~)6J}SitY#YnfHx!?-j48WVlAGCGc?dc z_q-1YlX3Xc=R|lGmCH}~2YzV`|8V0z{-o=V5kkB0k3XLyw2CryltZYFvUP7RlNph> z@#N+}UfRw2V`x1qo}5LfjbI%-v5~BeA23F!$bf$4Q&!j9TAo|2Bug!^z&WC^tv*pR zqS7r|0-rizkDGPYC^kf0)>yS&XZ2J=qXNy_vuV&Wx&PgRX#&13c z*mI|qm2yhTV@ybu`IHIvPilLfZ@`#nD_ojUDjCl%tEW{#=Z@v9CwFRtBU^~Tt4+hp_ZrWAgxLPS4$px+5(@BWBhB5T{PKcFEYB)wQS$tOGON^h7geK>6 zV58g^@K#`K9Ltbj8~hkDpEG*A=mO8#wOo3@H>-DWjuTbBmuDjbW5{b_h&zLG1-{g( z0oOP_$dMxzv|;S6s%~o4m0e%Xb4-;lH{|^0V80ygwd=6AhW$dmI9E#gHpg&ggHKH~ zN1?j?UdBj z&ekzfag3nb=Py~S6i)>{?RqwjS$A7q?Xm`gPhlQ6Qg0ta7tS+6CT{ZHAzsB@@4OaH zp-8O5Xfy9A;nUvJ_j*t-Ky$!VWu?Txo=8Yg2BaNWx;9v!^VUOg8C@@ptVhLTT$K)R zg9Vu~R!3OL$AddwT|r31_*b|_EqUNgcCfz=!o>^fP8DU^Xd4T_Z@<3v?Rg-)_`H=^ zA66vO{d45ag1o=q~c6G8NsiXgFoU0{3_UiAJl&X D^h#pA literal 0 HcmV?d00001 diff --git a/transforms/code/license_select/ray/test-data/input/sample_1.parquet b/transforms/code/license_select/ray/test-data/input/sample_1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..51fd1b49b07f56495fda663f8fbf6926471d15c8 GIT binary patch literal 3326 zcmds4&2Ji45MMAjcH>r2)J4!*)n=<8M{*p%#to@kbs+|pVq#-^7lZ!*ba_YJ2p@;qno%a^-l5lELsp{H(A2ajj_nVpbM#(vS0YJp(~8*NghmnOIMfoGKU_V?=p{{ zNKV&{bk+`suBVwBmhWxEwG{)cvGLk^$nA)x=$mw69Z24w7wN^-ZE8&lPzVD;0MmL6 zETBAthxvy9Nhlcd)mTQ!!avReBw?9}!WfH07NY527SO<&zEt2rswBUS*4i?mG`z50Hd;B$I~8vG>6Lq zh?5Io@(P(BR<6iggR83{H{kMBB=I7WetA9p&2`M>>ij%1sd}DDF1kujGbMVFqE~K5 zuPT3=s;>fSAe_V-7EW--vP3Mkn%eo_-lFu|sDJux)GhJHwL~nvl=%JH`xKR0PcGUL zQYAO}UHp>Ok*ZysD3;-y$Ymcek1yGCkim*|;L2+3I>RE#)IflI!+;(1Jja7=hO8|c zmN~N2@G?7k0-M0!330BC-OhSB)ipysnUw{!PQ0Ow;aDH(ilqh&eA6CI6_$LaDIR8K zYM!J@o&@HXtcJiG8ZbV4d{n-BpB;tB%(4K6LA%0UX1L5~-iQ&-*JvGxjiY2IZ{l^# zu;wje22Xts|rEch@R)T0SD(`#!gBuo?y>$wG7x7KjBTo?^J}XR<6;PtAZz^J^hhh zl(k{ak;MEte`a-gRO`v6YVi7?%bWS0EFNPUUllLhdWqAO&1O$Q2O#4H<9%N#@!R}E z-sMZRXKF)0Lf;=GlcgIg&B z_wuvWW;3r;Fegq|#k>vv&pU?%8#rbMno!7Sts?Qun+07mg)_B8^j@dc?17I=`w;k6 zUA-gb&*WUwmQ98D6NF}^e$>@ZU*WC{!M^N4{wBOgTt39=tJOUn$JsFrL#{MWWwY!7 zpKZtm_|9hlBLnAgZU$}0al6$xRjbBeY?GBu19O<CopNECZ{EDXHO{uhah*{S_E6VsD0fKkC9%=( zK(29oP$N&N8e#7JwrLyf&4WNM3YoUp?5UOA>3ump9CYDsg>j)?g0JKPpL25N0;VCO zMv#wxi zRg&}^{mU%<<7^oQ)NP7-9r$vWq9{{R&ZQ<|3`xU`W|(cuVc3S@uuQ|n3KRI~_eZf+ z=7IrH{J_$%qvIG`M7rKyW5l+ls~@wggO}i_BUwz#aZKhRW1(lRW}z5M=IrXujqqk% zv5@4TBV=`u6$?DwbhI{RVqm!j=5_%63T?b=X$~qPNsXlvi8bbe&9$+Eu(N*LHxYZE zVI9}B8|PzQIZF(j&7y4p8^ykg!Z?n<){}eqDaw;f#hJt1pI*NjMj9wevjXPx`E3_A( zElC~eNU=22x#mDEUTVWLg*KQ^^8WhP1n;_pn=afmkM~!B(0768SGS@+EfM0cy15n~ zOGiieK72F_p$yFlLdU_8X`ySt&4NKV=&P?Fd$oLt?53_M2-`le7lF_>f#}a4L_hz4 z0Go1g9jZtos`2V})?EYJP{=>KV9!BD zO1cI20I2 zr)55@Q)l=}aj~YfRUd(CnEkP$EuSI9A;=8mNUG#W5cq;c7|b=?gb90coL}E&2NPhX zQ2;{k{bds`Q&^@n?}Ui(H6ZWG;V#+HMSC;mq~<-x^h?M!Aukp09?{D=`HpB?UMh4= zwj`@hGGX>5%W8KC0QkVL;p2ixlUwm*Ji!7z>@fO-G8Km(c0h%fxJu=Tc;7E#;nxfI z$g|aa%yHZy{E_EVj@#t2O)k%IO)l5;WZCSLzDtC=WBvYN&Ywf;dGYi-$8|Ya2R(6; zvq=YYjw>a=KKlu8=x)6rY*thI9l0ntGTzgkXjvKcOO_<2&-t@av2j(gXP0xFrfgJN3O)oG*B#C~N{-*+AM-Y!D?L{WLYMg2t0q;Vucb1!Dkj}( z%8y;Ij$^ttF{u+<84I?YBk}5HUF@A#Q`f{TSDv1pnvtBQG{! zOm>lwNuX-h^UE6y5CTX zdUt4(lntF=81bEytJ(DA9_3W&SzX*u`1wfcY9S9kDy=bZ#e_}vG7;B?I9L_)c*u(( ziGp3vXU2T-oTAH@&=Z>CSU1e_4N z)UQ=bX)@<=HVxyxyrFpeQw^c7r1Xxgk=*jVeQmQ>KI1g9N3b37m#SAw&m}SAdOps0 zcTd~w^Eg3nVSzNt@EpUC)BsK)jW;6qo8znls1M{iAD(zO+gmg)6>y7*uFxCt$Qi^w za}3QkD0cIn6VJ%^2SRvAJ;+|33(xc8U08Gv>`#au>Q9l>4|op!qCtj3f27)Y^~dX> z@%&w$Z*LiiH?frC9#Ub(l&ohia@G~oT=$A7Gc1yfUp^xb;3LlerT$o;r~~+C@^8br BG>HHJ literal 0 HcmV?d00001 diff --git a/transforms/code/license_select/ray/test-data/sample_approved_licenses.json b/transforms/code/license_select/ray/test-data/sample_approved_licenses.json new file mode 100644 index 000000000..6b4a37698 --- /dev/null +++ b/transforms/code/license_select/ray/test-data/sample_approved_licenses.json @@ -0,0 +1,173 @@ +[ + "MIT", + "Apache-2.0", + "BSD-3-Clause", + "Unlicense", + "CC0-1.0", + "BSD-2-Clause", + "CC-BY-4.0", + "CC-BY-3.0", + "0BSD", + "WTFPL", + "MIT-0", + "ISC", + "ADSL", + "BSL-1.0", + "Zlib", + "FTL", + "MS-PL", + "BSD-2-Clause-FreeBSD", + "FSFAP", + "BSD-Source-Code", + "Apache-1.1", + "BSD-4-Clause", + "Ruby", + "MulanPSL-1.0", + "BSD-1-Clause", + "X11", + "Condor-1.1", + "PostgreSQL", + "CECILL-B", + "Intel", + "Vim", + "Naumen", + "OML", + "BSD-3-Clause-Clear", + "AML", + "PHP-3.01", + "OpenSSL", + "PSF-2.0", + "Xnet", + "Linux-OpenIB", + "BSD-3-Clause-LBNL", + "UPL-1.0", + "BlueOak-1.0.0", + "Info-ZIP", + "BSD-4-Clause-UC", + "bzip2-1.0.6", + "W3C", + "W3C-20150513", + "DOC", + "ICU", + "CC-BY-2.0", + "curl", + "MTLL", + "OLDAP-2.2.1", + "ECL-2.0", + "Adobe-Glyph", + "BSD-2-Clause-Patent", + "IJG", + "PHP-3.0", + "ZPL-2.1", + "MIT-advertising", + "NCSA", + "Fair", + "BSD-3-Clause-Attribution", + "OLDAP-2.3", + "NLPL", + "BSD-3-Clause-Open-MPI", + "Python-2.0", + "NASA-1.3", + "TCL", + "BSD-3-Clause-No-Nuclear-Warranty", + "ImageMagick", + "Net-SNMP", + "OLDAP-2.5", + "MIT-feh", + "OLDAP-2.4", + "MITNFA", + "libpng-2.0", + "EFL-2.0", + "OLDAP-2.7", + "IBM-pibs", + "libtiff", + "OLDAP-2.8", + "Adobe-2006", + "BSD-2-Clause-NetBSD", + "zlib-acknowledgement", + "OLDAP-2.6", + "BSD-3-Clause-No-Nuclear-License-2014", + "OLDAP-1.4", + "Libpng", + "MIT-CMU", + "JasPer-2.0", + "Zend-2.0", + "TCP-wrappers", + "XFree86-1.1", + "FSFUL", + "OLDAP-1.3", + "SGI-B-2.0", + "NetCDF", + "Zed", + "ZPL-2.0", + "Apache-1.0", + "CC-BY-1.0", + "OLDAP-2.1", + "OLDAP-1.2", + "OLDAP-2.0", + "NTP", + "AMPAS", + "Barr", + "mpich2", + "ANTLR-PD", + "Xerox", + "Spencer-94", + "AMDPLPA", + "BSD-3-Clause-No-Nuclear-License", + "HPND", + "ECL-1.0", + "MirOS", + "Qhull", + "ZPL-1.1", + "TU-Berlin-2.0", + "Spencer-86", + "SMLNJ", + "xinetd", + "OLDAP-2.2.2", + "MIT-enna", + "Font-exception-2.0", + "FSFULLR", + "TU-Berlin-1.0", + "xpp", + "NRL", + "W3C-19980720", + "EFL-1.0", + "eGenix", + "Unicode-DFS-2016", + "SWL", + "Spencer-99", + "Plexus", + "VSL-1.0", + "Leptonica", + "Unicode-DFS-2015", + "Mup", + "Giftware", + "OLDAP-2.2", + "APAFML", + "NBPL-1.0", + "OLDAP-1.1", + "Entessa", + "Multics", + "Newsletr", + "psutils", + "bzip2-1.0.5", + "Afmparse", + "diffmark", + "BSD-2-Clause-Views", + "DSDP", + "MIT-Modern-Variant", + "ANTLR-PD-fallback", + "Bahyph", + "BSD-3-Clause-Modification", + "BSD-4-Clause-Shortened", + "HTMLTIDY", + "MIT-open-group", + "MulanPSL-2.0", + "OLDAP-2.0.1", + "Saxpath", + "Borceux", + "Crossword", + "CrystalStacker", + "Rdisc", + "Wsuipa" +] diff --git a/transforms/code/license_select/ray/test/test_license_select_ray.py b/transforms/code/license_select/ray/test/test_license_select_ray.py new file mode 100644 index 000000000..2d3190e69 --- /dev/null +++ b/transforms/code/license_select/ray/test/test_license_select_ray.py @@ -0,0 +1,40 @@ +# (C) Copyright IBM Corp. 2024. +# Licensed under the Apache License, Version 2.0 (the “License”); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an “AS IS” BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import os + +from data_processing.test_support.launch.transform_test import ( + AbstractTransformLauncherTest, +) +from data_processing_ray.runtime.ray import RayTransformLauncher +from license_select_transform import LICENSE_COLUMN_NAME_CLI_KEY, LICENSES_FILE_CLI_KEY +from license_select_transform_ray import LicenseSelectRayTransformConfiguration + + +class TestPythonLicenseSelect(AbstractTransformLauncherTest): + """ + Extends the super-class to define the test data for the tests defined there. + The name of this class MUST begin with the word Test so that pytest recognizes it as a test class. + """ + + def get_test_transform_fixtures(self) -> list[tuple]: + basedir = "../test-data" + basedir = os.path.abspath(os.path.join(os.path.dirname(__file__), basedir)) + fixtures = [] + launcher = RayTransformLauncher(LicenseSelectRayTransformConfiguration()) + config = { + "run_locally": True, + LICENSE_COLUMN_NAME_CLI_KEY: "license", + LICENSES_FILE_CLI_KEY: os.path.join(basedir, "sample_approved_licenses.json"), + } + fixtures.append((launcher, config, basedir + "/input", basedir + "/expected")) + return fixtures