diff --git a/docs/mint.json b/docs/mint.json index 7f9ebcda2e55..88b312eeb9bc 100644 --- a/docs/mint.json +++ b/docs/mint.json @@ -1261,6 +1261,10 @@ "destination": "/3.0/api-ref/python", "source": "/latest/api-ref/prefect/:slug*" }, + { + "destination": "/3.0/api-ref/python", + "source": "/api-ref/prefect/:slug*" + }, { "source": "/3.0rc/:slug*", "destination": "/3.0/:slug*" diff --git a/src/integrations/prefect-azure/README.md b/src/integrations/prefect-azure/README.md index 6fd1627ba8e4..ef16e78eeb2c 100644 --- a/src/integrations/prefect-azure/README.md +++ b/src/integrations/prefect-azure/README.md @@ -20,16 +20,19 @@ pip install prefect-azure ``` To use Blob Storage: + ```bash pip install "prefect-azure[blob_storage]" ``` To use Cosmos DB: + ```bash pip install "prefect-azure[cosmos_db]" ``` To use ML Datastore: + ```bash pip install "prefect-azure[ml_datastore]" ``` @@ -61,6 +64,7 @@ example_blob_storage_download_flow() ``` Use `with_options` to customize options on any existing task or flow: + ```python custom_blob_storage_download_flow = example_blob_storage_download_flow.with_options( name="My custom task name", @@ -121,25 +125,29 @@ container_instance_job.save("aci-dev") ``` Then, create the deployment either on the UI or through the CLI: + ```bash prefect deployment build a_flow_module.py:log_hello_flow --name aci-dev -ib container-instance-job/aci-dev ``` -Visit [Prefect Deployments](https://docs.prefect.io/tutorials/deployments/) for more information about deployments. +Visit [Prefect Deployments](https://docs.prefect.io/latest/deploy/) for more information about deployments. ## Azure Container Instance Worker -The Azure Container Instance worker is an excellent way to run -your workflows on Azure. + +The Azure Container Instance worker is an excellent way to run +your workflows on Azure. To get started, create an Azure Container Instances typed work pool: + ``` prefect work-pool create -t azure-container-instance my-aci-work-pool ``` Then, run a worker that pulls jobs from the work pool: + ``` prefect worker start -n my-aci-worker -p my-aci-work-pool ``` -The worker should automatically read the work pool's type and start an +The worker should automatically read the work pool's type and start an Azure Container Instance worker. diff --git a/src/integrations/prefect-azure/prefect_azure/deployments/steps.py b/src/integrations/prefect-azure/prefect_azure/deployments/steps.py index eb1f73540c53..c1aad2bda199 100644 --- a/src/integrations/prefect-azure/prefect_azure/deployments/steps.py +++ b/src/integrations/prefect-azure/prefect_azure/deployments/steps.py @@ -31,7 +31,7 @@ !!! note Azure Storage account needs to have Hierarchical Namespace disabled. -For more information about using deployment steps, check out out the Prefect [docs](https://docs.prefect.io/latest/concepts/projects/#the-prefect-yaml-file). +For more information about using deployment steps, check out out the Prefect [docs](https://docs.prefect.io/latest/deploy/). """ # noqa from pathlib import Path, PurePosixPath diff --git a/src/integrations/prefect-bitbucket/README.md b/src/integrations/prefect-bitbucket/README.md index dba239f4ccb7..8d6b9c37b7c3 100644 --- a/src/integrations/prefect-bitbucket/README.md +++ b/src/integrations/prefect-bitbucket/README.md @@ -7,142 +7,4 @@

- -## Welcome! - -Prefect integrations for working with Bitbucket repositories. - -## Getting Started - -### Python setup - -Requires an installation of Python 3.9+. - -We recommend using a Python virtual environment manager such as pipenv, conda or virtualenv. - -These tasks are designed to work with Prefect 2.0. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -### Installation - -Install `prefect-bitbucket` with `pip`: - -```bash -pip install prefect-bitbucket -``` - -Then, register to [view the block](https://docs.prefect.io/ui/blocks/) on Prefect Cloud: - -```bash -prefect block register -m prefect_bitbucket -``` - -Note, to use the `load` method on Blocks, you must already have a block document [saved through code](https://docs.prefect.io/concepts/blocks/#saving-blocks) or [saved through the UI](https://docs.prefect.io/ui/blocks/). - -### Write and run a flow -#### Load a pre-existing BitBucketCredentials block - -```python -from prefect import flow -from prefect_bitbucket.credentials import BitBucketCredentials - -@flow -def use_stored_bitbucket_creds_flow(): - bitbucket_credentials_block = BitBucketCredentials.load("BLOCK_NAME") - - return bitbucket_credentials_block - -use_stored_bitbucket_creds_flow() -``` - -#### Create a new BitBucketCredentials block in a flow - -```python -from prefect import flow -from prefect_bitbucket.credentials import BitBucketCredentials - -@flow -def create_new_bitbucket_creds_flow(): - bitbucket_credentials_block = BitBucketCredentials( - token="my-token", - username="my-username" - ) - -create_new_bitbucket_creds_flow() -``` - -#### Create a BitBucketRepository block for a public repo -```python -from prefect_bitbucket import BitBucketRepository - -public_repo = "https://bitbucket.org/my-workspace/my-repository.git" - -# Creates a public BitBucket repository BitBucketRepository block -public_bitbucket_block = BitBucketRepository( - repository=public_repo -) - -# Saves the BitBucketRepository block to your Prefect workspace (in the Blocks tab) -public_bitbucket_block.save("my-bitbucket-block") -``` - -#### Create a BitBucketRepository block for a public repo at a specific branch or tag -```python -from prefect_bitbucket import BitBucketRepository - -public_repo = "https://bitbucket.org/my-workspace/my-repository.git" - -# Creates a public BitBucket repository BitBucketRepository block -branch_bitbucket_block = BitBucketRepository( - reference="my-branch-or-tag", # e.g "master" - repository=public_repo -) - -# Saves the BitBucketRepository block to your Prefect workspace (in the Blocks tab) -branch_bitbucket_block.save("my-bitbucket-branch-block") -``` -#### Create a new BitBucketCredentials block and a BitBucketRepository block for a private repo -```python -from prefect_bitbucket import BitBucketCredentials, BitBucketRepository - -# For a private repo, we need credentials to access it -bitbucket_credentials_block = BitBucketCredentials( - token="my-token", - username="my-username" # optional -) - -# Saves the BitBucketCredentials block to your Prefect workspace (in the Blocks tab) -bitbucket_credentials_block.save(name="my-bitbucket-credentials-block") - - -# Creates a private BitBucket repository BitBucketRepository block -private_repo = "https://bitbucket.org/my-workspace/my-repository.git" -private_bitbucket_block = BitBucketRepository( - repository=private_repo, - bitbucket_credentials=bitbucket_credentials_block -) - -# Saves the BitBucketRepository block to your Prefect workspace (in the Blocks tab) -private_bitbucket_block.save(name="my-private-bitbucket-block") -``` - -#### Use a preexisting BitBucketCredentials block to create a BitBucketRepository block for a private repo -```python -from prefect_bitbucket import BitBucketCredentials, BitBucketRepository - -# Loads a preexisting BitBucketCredentials block -BitBucketCredentials.load("my-bitbucket-credentials-block") - -# Creates a private BitBucket repository BitBucketRepository block -private_repo = "https://bitbucket.org/my-workspace/my-repository.git" -private_bitbucket_block = BitBucketRepository( - repository=private_repo, - bitbucket_credentials=bitbucket_credentials_block -) - -# Saves the BitBucketRepository block to your Prefect workspace (in the Blocks tab) -private_bitbucket_block.save(name="my-private-bitbucket-block") -``` - -!!! info "Differences between Bitbucket Server and Bitbucket Cloud" - - For Bitbucket Cloud, only set the `token` to authenticate. For Bitbucket Server, set both the `token` and the `username`. +See the docs at [https://docs.prefect.io/integrations/prefect-bitbucket](https://docs.prefect.io) for more information. diff --git a/src/integrations/prefect-dask/README.md b/src/integrations/prefect-dask/README.md index 8ccf9fdaf98b..ea6db4a5c87d 100644 --- a/src/integrations/prefect-dask/README.md +++ b/src/integrations/prefect-dask/README.md @@ -7,253 +7,4 @@

-The `prefect-dask` collection makes it easy to include distributed processing for your flows. Check out the examples below to get started! - -## Getting Started - -### Integrate with Prefect flows - -Perhaps you're already working with Prefect flows. Say your flow downloads many images to train your machine learning model. Unfortunately, it takes a long time to download your flows because your code is running sequentially. - -After installing `prefect-dask` you can parallelize your flow in three simple steps: - -1. Add the import: `from prefect_dask import DaskTaskRunner` -2. Specify the task runner in the flow decorator: `@flow(task_runner=DaskTaskRunner)` -3. Submit tasks to the flow's task runner: `a_task.submit(*args, **kwargs)` - -The parallelized code runs in about 1/3 of the time in our test! And that's without distributing the workload over multiple machines. -Here's the before and after! - -=== "Before" - ```python hl_lines="1" - # Completed in 15.2 seconds - - from typing import List - from pathlib import Path - - import httpx - from prefect import flow, task - - URL_FORMAT = ( - "https://www.cpc.ncep.noaa.gov/products/NMME/archive/" - "{year:04d}{month:02d}0800/current/images/nino34.rescaling.ENSMEAN.png" - ) - - @task - def download_image(year: int, month: int, directory: Path) -> Path: - # download image from URL - url = URL_FORMAT.format(year=year, month=month) - resp = httpx.get(url) - - # save content to directory/YYYYMM.png - file_path = (directory / url.split("/")[-1]).with_stem(f"{year:04d}{month:02d}") - file_path.write_bytes(resp.content) - return file_path - - @flow - def download_nino_34_plumes_from_year(year: int) -> List[Path]: - # create a directory to hold images - directory = Path("data") - directory.mkdir(exist_ok=True) - - # download all images - file_paths = [] - for month in range(1, 12 + 1): - file_path = download_image(year, month, directory) - file_paths.append(file_path) - return file_paths - - if __name__ == "__main__": - download_nino_34_plumes_from_year(2022) - ``` - -=== "After" - - ```python hl_lines="1 8 26 35" - # Completed in 5.7 seconds - - from typing import List - from pathlib import Path - - import httpx - from prefect import flow, task - from prefect_dask import DaskTaskRunner - - URL_FORMAT = ( - "https://www.cpc.ncep.noaa.gov/products/NMME/archive/" - "{year:04d}{month:02d}0800/current/images/nino34.rescaling.ENSMEAN.png" - ) - - @task - def download_image(year: int, month: int, directory: Path) -> Path: - # download image from URL - url = URL_FORMAT.format(year=year, month=month) - resp = httpx.get(url) - - # save content to directory/YYYYMM.png - file_path = (directory / url.split("/")[-1]).with_stem(f"{year:04d}{month:02d}") - file_path.write_bytes(resp.content) - return file_path - - @flow(task_runner=DaskTaskRunner(cluster_kwargs={"processes": False})) - def download_nino_34_plumes_from_year(year: int) -> List[Path]: - # create a directory to hold images - directory = Path("data") - directory.mkdir(exist_ok=True) - - # download all images - file_paths = [] - for month in range(1, 12 + 1): - file_path = download_image.submit(year, month, directory) - file_paths.append(file_path) - return file_paths - - if __name__ == "__main__": - download_nino_34_plumes_from_year(2022) - ``` - -The original flow completes in 15.2 seconds. - -However, with just a few minor tweaks, we were able to reduce the runtime by nearly **three** folds, down to just **5.7** seconds! - -### Integrate with Dask client/cluster and collections - -Suppose you have an existing Dask client/cluster and collection, like a `dask.dataframe.DataFrame`, and you want to add observability. - -With `prefect-dask`, there's no major overhaul necessary because Prefect was designed with incremental adoption in mind! It's as easy as: - -1. Adding the imports -2. Sprinkling a few `task` and `flow` decorators -3. Using `get_dask_client` context manager on collections to distribute work across workers -4. Specifying the task runner and client's address in the flow decorator -5. Submitting the tasks to the flow's task runner - -=== "Before" - - ```python - import dask.dataframe - import dask.distributed - - - - client = dask.distributed.Client() - - - def read_data(start: str, end: str) -> dask.dataframe.DataFrame: - df = dask.datasets.timeseries(start, end, partition_freq="4w") - return df - - - def process_data(df: dask.dataframe.DataFrame) -> dask.dataframe.DataFrame: - - df_yearly_avg = df.groupby(df.index.year).mean() - return df_yearly_avg.compute() - - - def dask_pipeline(): - df = read_data("1988", "2022") - df_yearly_average = process_data(df) - return df_yearly_average - - dask_pipeline() - ``` - - -=== "After" - - ```python hl_lines="3 4 8 13 15 19 21 22" - import dask.dataframe - import dask.distributed - from prefect import flow, task - from prefect_dask import DaskTaskRunner, get_dask_client - - client = dask.distributed.Client() - - @task - def read_data(start: str, end: str) -> dask.dataframe.DataFrame: - df = dask.datasets.timeseries(start, end, partition_freq="4w") - return df - - @task - def process_data(df: dask.dataframe.DataFrame) -> dask.dataframe.DataFrame: - with get_dask_client(): - df_yearly_avg = df.groupby(df.index.year).mean() - return df_yearly_avg.compute() - - @flow(task_runner=DaskTaskRunner(address=client.scheduler.address)) - def dask_pipeline(): - df = read_data.submit("1988", "2022") - df_yearly_average = process_data.submit(df) - return df_yearly_average - - dask_pipeline() - ``` - -Now, you can conveniently see when each task completed, both in the terminal and the UI! - -```bash -14:10:09.845 | INFO | prefect.engine - Created flow run 'chocolate-pony' for flow 'dask-flow' -14:10:09.847 | INFO | prefect.task_runner.dask - Connecting to an existing Dask cluster at tcp://127.0.0.1:59255 -14:10:09.857 | INFO | distributed.scheduler - Receive client connection: Client-8c1e0f24-9133-11ed-800e-86f2469c4e7a -14:10:09.859 | INFO | distributed.core - Starting established connection to tcp://127.0.0.1:59516 -14:10:09.862 | INFO | prefect.task_runner.dask - The Dask dashboard is available at http://127.0.0.1:8787/status -14:10:11.344 | INFO | Flow run 'chocolate-pony' - Created task run 'read_data-5bc97744-0' for task 'read_data' -14:10:11.626 | INFO | Flow run 'chocolate-pony' - Submitted task run 'read_data-5bc97744-0' for execution. -14:10:11.795 | INFO | Flow run 'chocolate-pony' - Created task run 'process_data-090555ba-0' for task 'process_data' -14:10:11.798 | INFO | Flow run 'chocolate-pony' - Submitted task run 'process_data-090555ba-0' for execution. -14:10:13.279 | INFO | Task run 'read_data-5bc97744-0' - Finished in state Completed() -14:11:43.539 | INFO | Task run 'process_data-090555ba-0' - Finished in state Completed() -14:11:43.883 | INFO | Flow run 'chocolate-pony' - Finished in state Completed('All states completed.') -``` - -## Resources - -For additional examples, check out the [Usage Guide](usage_guide)! - -### Installation - -Get started by installing `prefect-dask`! - -=== "pip" - - ```bash - pip install -U prefect-dask - ``` - -=== "conda" - - ```bash - conda install -c conda-forge prefect-dask - ``` - -Requires an installation of Python 3.7+. - -We recommend using a Python virtual environment manager such as pipenv, conda, or virtualenv. - -These tasks are designed to work with Prefect 2. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -### Feedback - -If you encounter any bugs while using `prefect-dask`, feel free to open an issue in the [prefect](https://github.com/PrefectHQ/prefect) repository. - -If you have any questions or issues while using `prefect-dask`, you can find help in either the [Prefect Discourse forum](https://discourse.prefect.io/) or the [Prefect Slack community](https://prefect.io/slack). - -### Contributing - -If you'd like to help contribute to fix an issue or add a feature to `prefect-dask`, please [propose changes through a pull request from a fork of the repository](https://docs.github.com/en/pull-requests/collaborating-with-pull-requests/proposing-changes-to-your-work-with-pull-requests/creating-a-pull-request-from-a-fork). - -Here are the steps: - -1. [Fork the repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#forking-a-repository) -2. [Clone the forked repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#cloning-your-forked-repository) -3. Install the repository and its dependencies: -``` -pip install -e ".[dev]" -``` -4. Make desired changes -5. Add tests -6. Install `pre-commit` to perform quality checks prior to commit: -``` -pre-commit install -``` -7. `git commit`, `git push`, and create a pull request +See the docs at [https://docs.prefect.io/integrations/prefect-dask](https://docs.prefect.io/integrations/prefect-dask) for more information. diff --git a/src/integrations/prefect-databricks/README.md b/src/integrations/prefect-databricks/README.md index dae3faf5a827..419365325487 100644 --- a/src/integrations/prefect-databricks/README.md +++ b/src/integrations/prefect-databricks/README.md @@ -1,7 +1,5 @@ # prefect-databricks -Visit the full docs [here](https://PrefectHQ.github.io/prefect-databricks) to see additional examples and the API reference. -

PyPI @@ -10,163 +8,4 @@ Visit the full docs [here](https://PrefectHQ.github.io/prefect-databricks) to se

-## Welcome! - -Prefect integrations for interacting with Databricks - -The tasks within this collection were created by a code generator using the service's OpenAPI spec. - -The service's REST API documentation can be found [here](https://docs.databricks.com/dev-tools/api/latest/index.html). - -## Getting Started - -### Python setup - -Requires an installation of Python 3.7+. - -We recommend using a Python virtual environment manager such as pipenv, conda or virtualenv. - -These tasks are designed to work with Prefect 2. For more information about how to use Prefect, please refer to the [Prefect documentation](https://orion-docs.prefect.io/). - -### Installation - -Install `prefect-databricks` with `pip`: - -```bash -pip install prefect-databricks -``` - -A list of available blocks in `prefect-databricks` and their setup instructions can be found [here](https://PrefectHQ.github.io/prefect-databricks/#blocks-catalog). - -### Lists jobs on the Databricks instance - -```python -from prefect import flow -from prefect_databricks import DatabricksCredentials -from prefect_databricks.jobs import jobs_list - - -@flow -def example_execute_endpoint_flow(): - databricks_credentials = DatabricksCredentials.load("my-block") - jobs = jobs_list( - databricks_credentials, - limit=5 - ) - return jobs - -example_execute_endpoint_flow() -``` - -### Use `with_options` to customize options on any existing task or flow - -```python -custom_example_execute_endpoint_flow = example_execute_endpoint_flow.with_options( - name="My custom flow name", - retries=2, - retry_delay_seconds=10, -) -``` - -### Launch a new cluster and run a Databricks notebook - -Notebook named `example.ipynb` on Databricks which accepts a name parameter: - -```python -name = dbutils.widgets.get("name") -message = f"Don't worry {name}, I got your request! Welcome to prefect-databricks!" -print(message) -``` - -Prefect flow that launches a new cluster to run `example.ipynb`: - -```python -from prefect import flow -from prefect_databricks import DatabricksCredentials -from prefect_databricks.jobs import jobs_runs_submit -from prefect_databricks.models.jobs import ( - AutoScale, - AwsAttributes, - JobTaskSettings, - NotebookTask, - NewCluster, -) - - -@flow -def jobs_runs_submit_flow(notebook_path, **base_parameters): - databricks_credentials = DatabricksCredentials.load("my-block") - - # specify new cluster settings - aws_attributes = AwsAttributes( - availability="SPOT", - zone_id="us-west-2a", - ebs_volume_type="GENERAL_PURPOSE_SSD", - ebs_volume_count=3, - ebs_volume_size=100, - ) - auto_scale = AutoScale(min_workers=1, max_workers=2) - new_cluster = NewCluster( - aws_attributes=aws_attributes, - autoscale=auto_scale, - node_type_id="m4.large", - spark_version="10.4.x-scala2.12", - spark_conf={"spark.speculation": True}, - ) - - # specify notebook to use and parameters to pass - notebook_task = NotebookTask( - notebook_path=notebook_path, - base_parameters=base_parameters, - ) - - # compile job task settings - job_task_settings = JobTaskSettings( - new_cluster=new_cluster, - notebook_task=notebook_task, - task_key="prefect-task" - ) - - run = jobs_runs_submit( - databricks_credentials=databricks_credentials, - run_name="prefect-job", - tasks=[job_task_settings] - ) - - return run - - -jobs_runs_submit_flow("/Users/username@gmail.com/example.ipynb", name="Marvin") -``` - -Note, instead of using the built-in models, you may also input valid JSON. For example, `AutoScale(min_workers=1, max_workers=2)` is equivalent to `{"min_workers": 1, "max_workers": 2}`. - -For more tips on how to use tasks and flows in a Collection, check out [Using Collections](https://orion-docs.prefect.io/collections/usage/)! - -## Resources - -If you encounter any bugs while using `prefect-databricks`, feel free to open an issue in the [prefect-databricks](https://github.com/PrefectHQ/prefect-databricks) repository. - -If you have any questions or issues while using `prefect-databricks`, you can find help in either the [Prefect Discourse forum](https://discourse.prefect.io/) or the [Prefect Slack community](https://prefect.io/slack). - -Feel free to star or watch [`prefect-databricks`](https://github.com/PrefectHQ/prefect-databricks) for updates too! - -## Contributing - -If you'd like to help contribute to fix an issue or add a feature to `prefect-databricks`, please [propose changes through a pull request from a fork of the repository](https://docs.github.com/en/pull-requests/collaborating-with-pull-requests/proposing-changes-to-your-work-with-pull-requests/creating-a-pull-request-from-a-fork). - -Here are the steps: -1. [Fork the repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#forking-a-repository) -2. [Clone the forked repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#cloning-your-forked-repository) -3. Install the repository and its dependencies: -``` -pip install -e ".[dev]" -``` -4. Make desired changes -5. Add tests -6. Insert an entry to [CHANGELOG.md](https://github.com/PrefectHQ/prefect-databricks/blob/main/CHANGELOG.md) -7. Install `pre-commit` to perform quality checks prior to commit: -``` -pre-commit install -``` -8. `git commit`, `git push`, and create a pull request +See the docs at [https://docs.prefect.io/integrations/prefect-databricks](https://docs.prefect.io/integrations/prefect-databricks) for more information. diff --git a/src/integrations/prefect-dbt/README.md b/src/integrations/prefect-dbt/README.md index ffff6ffb7836..190e285c03d6 100644 --- a/src/integrations/prefect-dbt/README.md +++ b/src/integrations/prefect-dbt/README.md @@ -7,296 +7,4 @@

-With prefect-dbt, you can easily trigger and monitor dbt Cloud jobs, execute dbt Core CLI commands, and incorporate other services, like Snowflake, into your dbt runs! - -Check out the examples below to get started! - -## Getting Started - -Be sure to install [prefect-dbt](#installation) and [save a block](#saving-credentials-to-block) to run the examples below! - -### Integrate dbt Cloud jobs with Prefect flows - -If you have an existing dbt Cloud job, take advantage of the flow, `run_dbt_cloud_job`. - -This flow triggers the job and waits until the job run is finished. - -If certain nodes fail, `run_dbt_cloud_job` efficiently retries the specific, unsuccessful nodes. - -```python -from prefect import flow - -from prefect_dbt.cloud import DbtCloudJob -from prefect_dbt.cloud.jobs import run_dbt_cloud_job - -@flow -def run_dbt_job_flow(): - result = run_dbt_cloud_job( - dbt_cloud_job=DbtCloudJob.load("my-block-name"), - targeted_retries=5, - ) - return result - -run_dbt_job_flow() -``` - -### Integrate dbt Core CLI commands with Prefect flows - -`prefect-dbt` also supports execution of dbt Core CLI commands. - -To get started, if you don't have a `DbtCoreOperation` block already saved, -set the commands that you want to run; it can include a mix of dbt and non-dbt commands. - -Then, optionally specify the `project_dir`. - -If `profiles_dir` is unset, it will try to use the `DBT_PROFILES_DIR` environment variable. -If that's also not set, it will use the default directory `$HOME/.dbt/`. - -#### Using an existing profile - -If you already have an existing dbt profile, specify the `profiles_dir` where `profiles.yml` is located. - -```python -from prefect import flow -from prefect_dbt.cli.commands import DbtCoreOperation - -@flow -def trigger_dbt_flow() -> str: - result = DbtCoreOperation( - commands=["pwd", "dbt debug", "dbt run"], - project_dir="PROJECT-DIRECTORY-PLACEHOLDER", - profiles_dir="PROFILES-DIRECTORY-PLACEHOLDER" - ).run() - return result - -trigger_dbt_flow() -``` - -#### Writing a new profile - -To setup a new profile, first [save and load a DbtCliProfile block](#saving-credentials-to-block) and use it in `DbtCoreOperation`. - -Then, specify `profiles_dir` where `profiles.yml` will be written. - -```python -from prefect import flow -from prefect_dbt.cli import DbtCliProfile, DbtCoreOperation - -@flow -def trigger_dbt_flow(): - dbt_cli_profile = DbtCliProfile.load("DBT-CORE-OPERATION-BLOCK-NAME-PLACEHOLDER") - with DbtCoreOperation( - commands=["dbt debug", "dbt run"], - project_dir="PROJECT-DIRECTORY-PLACEHOLDER", - profiles_dir="PROFILES-DIRECTORY-PLACEHOLDER", - dbt_cli_profile=dbt_cli_profile, - ) as dbt_operation: - dbt_process = dbt_operation.trigger() - # do other things before waiting for completion - dbt_process.wait_for_completion() - result = dbt_process.fetch_result() - return result - -trigger_dbt_flow() -``` - -## Resources - -If you need help getting started with or using dbt, please consult the [dbt documentation](https://docs.getdbt.com/docs/building-a-dbt-project/documentation). - -### Installation - -To use `prefect-dbt` with dbt Cloud: - -```bash -pip install prefect-dbt -``` - -To use dbt Core (CLI): - -```bash -pip install "prefect-dbt[cli]" -``` - -To use dbt Core with Snowflake profiles: - -```bash -pip install "prefect-dbt[snowflake]" -``` - -To use dbt Core with BigQuery profiles: - -```bash -pip install "prefect-dbt[bigquery]" -``` - -To use dbt Core with Postgres profiles: - -```bash -pip install "prefect-dbt[postgres]" -``` - -!!! warning "Some dbt Core profiles require additional installation" - - According to dbt's [Databricks setup page](https://docs.getdbt.com/reference/warehouse-setups/databricks-setup), users must first install the adapter: - - ```bash - pip install dbt-databricks - ``` - - Check out the [desired profile setup page](https://docs.getdbt.com/reference/profiles.yml) on the sidebar for others. - -Requires an installation of Python 3.9+. - -We recommend using a Python virtual environment manager such as pipenv, conda or virtualenv. - -These tasks are designed to work with Prefect 2. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -### Saving credentials to block - -Note, to use the `load` method on Blocks, you must already have a block document [saved through code](https://docs.prefect.io/concepts/blocks/#saving-blocks) or [saved through the UI](https://docs.prefect.io/ui/blocks/). - -!!! info "Registering blocks" - - Register blocks in this module to - [view and edit them](https://docs.prefect.io/ui/blocks/) - on Prefect Cloud: - - ```bash - prefect block register -m prefect_dbt - ``` - -#### dbt Cloud - -To create a dbt Cloud credentials block: - -1. Head over to your [dbt Cloud profile](https://cloud.getdbt.com/settings/profile). -2. Login to your dbt Cloud account. -3. Scroll down to "API" or click "API Access" on the sidebar. -4. Copy the API Key. -5. Click Projects on the sidebar. -6. Copy the account ID from the URL: `https://cloud.getdbt.com/settings/accounts/`. -7. Create a short script, replacing the placeholders. - -```python -from prefect_dbt.cloud import DbtCloudCredentials - -DbtCloudCredentials( - api_key="API-KEY-PLACEHOLDER", - account_id="ACCOUNT-ID-PLACEHOLDER" -).save("CREDENTIALS-BLOCK-NAME-PLACEHOLDER") -``` - -Then, to create a dbt Cloud job block: - -1. Head over to your [dbt home page](https://cloud.getdbt.com/). -2. On the top nav bar, click on Deploy -> Jobs. -3. Select a job. -4. Copy the job ID from the URL: `https://cloud.getdbt.com/deploy//projects//jobs/` -5. Create a short script, replacing the placeholders. - -```python -from prefect_dbt.cloud import DbtCloudCredentials, DbtCloudJob - -dbt_cloud_credentials = DbtCloudCredentials.load("CREDENTIALS-BLOCK-NAME-PLACEHOLDER") -dbt_cloud_job = DbtCloudJob( - dbt_cloud_credentials=dbt_cloud_credentials, - job_id="JOB-ID-PLACEHOLDER" -).save("JOB-BLOCK-NAME-PLACEHOLDER") -``` - -Congrats! You can now easily load the saved block, which holds your credentials: - -```python -from prefect_dbt.cloud import DbtCloudJob - -DbtCloudJob.load("JOB-BLOCK-NAME-PLACEHOLDER") -``` - -#### dbt Core CLI - -!!! info "Available `TargetConfigs` blocks" - - The following may vary slightly depending on the service you want to incorporate. - - Visit the [API Reference](cli/configs/base) to see other built-in `TargetConfigs` blocks. - - If the desired service profile is not available, check out the - [Examples Catalog](examples_catalog/#clicredentials-module) to see how you can - build one from the generic `TargetConfigs` class. - -To create dbt Core target config and profile blocks for BigQuery: - -1. Save and load a [`GcpCredentials` block](https://prefecthq.github.io/prefect-gcp/#saving-credentials-to-a-block). -2. Determine the schema / dataset you want to use in BigQuery. -3. Create a short script, replacing the placeholders. - -```python -from prefect_gcp.credentials import GcpCredentials -from prefect_dbt.cli import BigQueryTargetConfigs, DbtCliProfile - -credentials = GcpCredentials.load("CREDENTIALS-BLOCK-NAME-PLACEHOLDER") -target_configs = BigQueryTargetConfigs( - schema="SCHEMA-NAME-PLACEHOLDER", # also known as dataset - credentials=credentials, -) -target_configs.save("TARGET-CONFIGS-BLOCK-NAME-PLACEHOLDER") - -dbt_cli_profile = DbtCliProfile( - name="PROFILE-NAME-PLACEHOLDER", - target="TARGET-NAME-placeholder", - target_configs=target_configs, -) -dbt_cli_profile.save("DBT-CLI-PROFILE-BLOCK-NAME-PLACEHOLDER") -``` - -Then, to create a dbt Core operation block: - -1. Determine the dbt commands you want to run. -2. Create a short script, replacing the placeholders. - -```python -from prefect_dbt.cli import DbtCliProfile, DbtCoreOperation - -dbt_cli_profile = DbtCliProfile.load("DBT-CLI-PROFILE-BLOCK-NAME-PLACEHOLDER") -dbt_core_operation = DbtCoreOperation( - commands=["DBT-CLI-COMMANDS-PLACEHOLDER"], - dbt_cli_profile=dbt_cli_profile, - overwrite_profiles=True, -) -dbt_core_operation.save("DBT-CORE-OPERATION-BLOCK-NAME-PLACEHOLDER") -``` - -Congrats! You can now easily load the saved block, which holds your credentials: - -```python -from prefect_dbt.cloud import DbtCoreOperation - -DbtCoreOperation.load("DBT-CORE-OPERATION-BLOCK-NAME-PLACEHOLDER") -``` - -### Feedback - -If you encounter any bugs while using `prefect-dbt`, feel free to open an issue in the [prefect](https://github.com/PrefectHQ/prefect) repository. - -If you have any questions or issues while using `prefect-dbt`, you can find help in the [Prefect Slack community](https://prefect.io/slack). - -### Contributing - -If you'd like to help contribute to fix an issue or add a feature to `prefect-dbt`, please [propose changes through a pull request from a fork of the repository](https://docs.github.com/en/pull-requests/collaborating-with-pull-requests/proposing-changes-to-your-work-with-pull-requests/creating-a-pull-request-from-a-fork). - -Here are the steps: - -1. [Fork the repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#forking-a-repository) -2. [Clone the forked repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#cloning-your-forked-repository) -3. Install the repository and its dependencies: -``` -pip install -e ".[dev]" -``` -4. Make desired changes -5. Add tests -6. Install `pre-commit` to perform quality checks prior to commit: -``` -pre-commit install -``` -7. `git commit`, `git push`, and create a pull request +See the docs at [https://docs.prefect.io/integrations/prefect-dbt](https://docs.prefect.io/integrations/prefect-dbt) for more information. diff --git a/src/integrations/prefect-docker/README.md b/src/integrations/prefect-docker/README.md index 6227af051815..afc4ce0b860a 100644 --- a/src/integrations/prefect-docker/README.md +++ b/src/integrations/prefect-docker/README.md @@ -7,108 +7,4 @@

-## Welcome! - -Prefect integrations for working with Docker. - -Note! The `DockerRegistryCredentials` in `prefect-docker` is a unique block, separate from the `DockerRegistry` in `prefect` core. While `DockerRegistry` implements a few functionality from both `DockerHost` and `DockerRegistryCredentials` for convenience, it does not allow much configuration to interact with a Docker host. - -Do not use `DockerRegistry` with this collection. Instead, use `DockerHost` and `DockerRegistryCredentials`. - -## Getting Started - -### Python setup - -Requires an installation of Python 3.9+. - -We recommend using a Python virtual environment manager such as pipenv, conda, or virtualenv. - -These tasks are designed to work with Prefect 2. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -### Installation - -Install `prefect-docker` with `pip`: - -```bash -pip install prefect-docker -``` - -Then, register to [view the block](https://docs.prefect.io/concepts/blocks/) on Prefect Cloud: - -```bash -prefect block register -m prefect_docker -``` - -Note, to use the `load` method on Blocks, you must already have a block document [saved through code](https://docs.prefect.io/concepts/blocks/#saving-blocks) or saved through the UI. - -### Pull image, and create, start, log, stop, and remove Docker container - -```python -from prefect import flow -from prefect.logging import get_run_logger -from prefect_docker.images import pull_docker_image -from prefect_docker.containers import ( - create_docker_container, - start_docker_container, - get_docker_container_logs, - stop_docker_container, - remove_docker_container, -) - - -@flow -def docker_flow(): - logger = get_run_logger() - pull_docker_image("prefecthq/prefect", "latest") - container = create_docker_container( - image="prefecthq/prefect", command="echo 'hello world!' && sleep 60" - ) - start_docker_container(container_id=container.id) - logs = get_docker_container_logs(container_id=container.id) - logger.info(logs) - stop_docker_container(container_id=container.id) - remove_docker_container(container_id=container.id) - return container -``` - -### Use a custom Docker Host to create a Docker container -```python -from prefect import flow -from prefect_docker import DockerHost -from prefect_docker.containers import create_docker_container - -@flow -def create_docker_container_flow(): - docker_host = DockerHost( - base_url="tcp://127.0.0.1:1234", - max_pool_size=4 - ) - container = create_docker_container( - docker_host=docker_host, - image="prefecthq/prefect", - command="echo 'hello world!'" - ) - -create_docker_container_flow() -``` - -## Resources - -If you encounter any bugs while using `prefect-docker`, feel free to open an issue in the [prefect](https://github.com/PrefectHQ/prefect) repository. - -If you have any questions or issues while using `prefect-docker`, you can find help in the [Prefect Slack community](https://prefect.io/slack). - -## Development - -If you'd like to install a version of `prefect-docker` for development, clone the repository and perform an editable install with `pip`: - -```bash -git clone https://github.com/PrefectHQ/prefect-docker.git - -cd prefect-docker/ - -pip install -e ".[dev]" - -# Install linting pre-commit hooks -pre-commit install -``` +See the docs at [https://docs.prefect.io/integrations/prefect-docker](https://docs.prefect.io/integrations/prefect-docker) for more information. diff --git a/src/integrations/prefect-docker/prefect_docker/worker.py b/src/integrations/prefect-docker/prefect_docker/worker.py index 726a266c175d..e529b123375d 100644 --- a/src/integrations/prefect-docker/prefect_docker/worker.py +++ b/src/integrations/prefect-docker/prefect_docker/worker.py @@ -11,7 +11,7 @@ to poll for flow runs. For more information about work pools and workers, -checkout out the [Prefect docs](https://docs.prefect.io/concepts/work-pools/). +checkout out the [Prefect docs](https://docs.prefect.io/latest/deploy/infrastructure-concepts). """ import enum diff --git a/src/integrations/prefect-email/README.md b/src/integrations/prefect-email/README.md index 907459934961..2d47b473bc9f 100644 --- a/src/integrations/prefect-email/README.md +++ b/src/integrations/prefect-email/README.md @@ -7,139 +7,4 @@

-Visit the full docs [here](https://PrefectHQ.github.io/prefect-email) to see additional examples and the API reference. - -`prefect-email` is a collection of prebuilt Prefect integrations that can be used to interact with email services. - -## Getting Started - -### Integrate with Prefect flows - -`prefect-email` makes sending emails effortless, giving you peace of mind that your emails are being sent as expected. - -First, install [prefect-email](#installation) and [save your email credentials to a block](#saving-credentials-to-block) to run the examples below! - -```python -from prefect import flow -from prefect_email import EmailServerCredentials, email_send_message - -@flow -def example_email_send_message_flow(email_addresses): - email_server_credentials = EmailServerCredentials.load("BLOCK-NAME-PLACEHOLDER") - for email_address in email_addresses: - subject = email_send_message.with_options(name=f"email {email_address}").submit( - email_server_credentials=email_server_credentials, - subject="Example Flow Notification using Gmail", - msg="This proves email_send_message works!", - email_to=email_address, - ) - -example_email_send_message_flow(["EMAIL-ADDRESS-PLACEHOLDER"]) -``` - -Outputs: - -```bash -16:58:27.646 | INFO | prefect.engine - Created flow run 'busy-bat' for flow 'example-email-send-message-flow' -16:58:29.225 | INFO | Flow run 'busy-bat' - Created task run 'email someone@gmail.com-0' for task 'email someone@gmail.com' -16:58:29.229 | INFO | Flow run 'busy-bat' - Submitted task run 'email someone@gmail.com-0' for execution. -16:58:31.523 | INFO | Task run 'email someone@gmail.com-0' - Finished in state Completed() -16:58:31.713 | INFO | Flow run 'busy-bat' - Finished in state Completed('All states completed.') -``` - -Please note, many email services, like Gmail, require an [App Password](https://support.google.com/accounts/answer/185833) to successfully send emails. If you encounter an error similar to `smtplib.SMTPAuthenticationError: (535, b'5.7.8 Username and Password not accepted...`, it's likely you are not using an App Password. - -### Capture exceptions and notify by email - -Perhaps you want an email notification with the details of the exception when your flow run fails. - -`prefect-email` can be wrapped in an `except` statement to do just that! - -```python -from prefect import flow -from prefect.context import get_run_context -from prefect_email import EmailServerCredentials, email_send_message - -def notify_exc_by_email(exc): - context = get_run_context() - flow_run_name = context.flow_run.name - email_server_credentials = EmailServerCredentials.load("email-server-credentials") - email_send_message( - email_server_credentials=email_server_credentials, - subject=f"Flow run {flow_run_name!r} failed", - msg=f"Flow run {flow_run_name!r} failed due to {exc}.", - email_to=email_server_credentials.username, - ) - -@flow -def example_flow(): - try: - 1 / 0 - except Exception as exc: - notify_exc_by_email(exc) - raise - -example_flow() -``` - -## Resources - -For more tips on how to use tasks and flows in a Collection, check out [Using Collections](https://docs.prefect.io/collections/usage/)! - -### Installation - -Install `prefect-email` with `pip`: - -```bash -pip install prefect-email -``` - -Then, register to [view the block](https://docs.prefect.io/ui/blocks/) on Prefect Cloud: - -```bash -prefect block register -m prefect_email -``` - -Note, to use the `load` method on Blocks, you must already have a block document [saved through code](https://docs.prefect.io/concepts/blocks/#saving-blocks) or [saved through the UI](https://docs.prefect.io/ui/blocks/). - -Requires an installation of Python 3.9+. - -We recommend using a Python virtual environment manager such as pipenv, conda or virtualenv. - -These tasks are designed to work with Prefect 2. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -### Saving credentials to block - -Note, to use the `load` method on Blocks, you must already have a block document [saved through code](https://docs.prefect.io/concepts/blocks/#saving-blocks) or [saved through the UI](https://docs.prefect.io/ui/blocks/). - -Below is a walkthrough on saving block documents through code. - -Create a short script, replacing the placeholders. - -```python -from prefect_email import EmailServerCredentials - -credentials = EmailServerCredentials( - username="EMAIL-ADDRESS-PLACEHOLDER", - password="PASSWORD-PLACEHOLDER", # must be an app password -) -credentials.save("BLOCK-NAME-PLACEHOLDER") -``` - -Congrats! You can now easily load the saved block, which holds your credentials: - -```python -from prefect_email import EmailServerCredentials - -EmailServerCredentials.load("BLOCK_NAME_PLACEHOLDER") -``` - -!!! info "Registering blocks" - - Register blocks in this module to - [view and edit them](https://docs.prefect.io/ui/blocks/) - on Prefect Cloud: - - ```bash - prefect block register -m prefect_email - ``` +See the docs at [https://docs.prefect.io/integrations/prefect-email](https://docs.prefect.io/integrations/prefect-email) for more information. diff --git a/src/integrations/prefect-gcp/prefect_gcp/utilities.py b/src/integrations/prefect-gcp/prefect_gcp/utilities.py index 49a6e295f630..e43b0ccfb455 100644 --- a/src/integrations/prefect-gcp/prefect_gcp/utilities.py +++ b/src/integrations/prefect-gcp/prefect_gcp/utilities.py @@ -39,7 +39,7 @@ def slugify_name(name: str, max_length: int = 30) -> Optional[str]: This module is deprecated as of March 2024 and will not be available after September 2024. It has been replaced by the Cloud Run and Cloud Run V2 workers, which offer enhanced functionality and better performance. -For upgrade instructions, see https://docs.prefect.io/latest/guides/upgrade-guide-agents-to-workers/. +For upgrade instructions, see https://docs.prefect.io/latest/resources/upgrade-agents-to-workers. Integrations with Google Cloud Run Job. diff --git a/src/integrations/prefect-gcp/prefect_gcp/workers/cloud_run.py b/src/integrations/prefect-gcp/prefect_gcp/workers/cloud_run.py index 1a993525ff43..bc20017ee06e 100644 --- a/src/integrations/prefect-gcp/prefect_gcp/workers/cloud_run.py +++ b/src/integrations/prefect-gcp/prefect_gcp/workers/cloud_run.py @@ -16,7 +16,7 @@ ## Configuration Read more about configuring work pools -[here](https://docs.prefect.io/latest/concepts/work-pools/#work-pool-overview). +[here](https://docs.prefect.io/3.0/deploy/infrastructure-concepts/work-pools). ## Advanced Configuration !!! example "Using a custom Cloud Run job template" diff --git a/src/integrations/prefect-gcp/prefect_gcp/workers/vertex.py b/src/integrations/prefect-gcp/prefect_gcp/workers/vertex.py index 98dac526e5fa..add6ec12798a 100644 --- a/src/integrations/prefect-gcp/prefect_gcp/workers/vertex.py +++ b/src/integrations/prefect-gcp/prefect_gcp/workers/vertex.py @@ -16,7 +16,7 @@ ## Configuration Read more about configuring work pools -[here](https://docs.prefect.io/latest/concepts/work-pools/#work-pool-overview). +[here](https://docs.prefect.io/3.0/deploy/infrastructure-concepts/work-pools). """ import asyncio diff --git a/src/integrations/prefect-github/README.md b/src/integrations/prefect-github/README.md index d4b13857bd2e..2ee33be89c6f 100644 --- a/src/integrations/prefect-github/README.md +++ b/src/integrations/prefect-github/README.md @@ -7,85 +7,4 @@

-## Welcome! - -Prefect integrations interacting with GitHub. - -The tasks within this collection were created by a code generator using the GitHub GraphQL schema. - -## Getting Started - -### Python setup - -Requires an installation of Python 3.9 or newer. - -We recommend using a Python virtual environment manager such as pipenv, conda or virtualenv. - -These tasks are designed to work with Prefect 2. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -### Installation - -Install `prefect-github` with `pip`: - -```bash -pip install prefect-github -``` - -Then, register to [view the block](https://docs.prefect.io/ui/blocks/) on Prefect Cloud: - -```bash -prefect block register -m prefect_github -``` - -Note, to use the `load` method on Blocks, you must already have a block document [saved through code](https://orion-docs.prefect.io/concepts/blocks/#saving-blocks) or saved through the UI. - -### Write and run a flow - -```python -from prefect import flow -from prefect_github import GitHubCredentials -from prefect_github.repository import query_repository -from prefect_github.mutations import add_star_starrable - - -@flow() -def github_add_star_flow(): - github_credentials = GitHubCredentials.load("github-token") - repository_id = query_repository( - "PrefectHQ", - "Prefect", - github_credentials=github_credentials, - return_fields="id" - )["id"] - starrable = add_star_starrable( - repository_id, - github_credentials - ) - return starrable - - -github_add_star_flow() -``` - -## Resources - -If you encounter any bugs while using `prefect-github`, feel free to open an issue in the [prefect-github](https://github.com/PrefectHQ/prefect-github) repository. - -If you have any questions or issues while using `prefect-github`, you can find help in the [Prefect Slack community](https://prefect.io/slack). - -Feel free to ⭐️ or watch [`prefect-github`](https://github.com/PrefectHQ/prefect-github) for updates too! - -## Development - -If you'd like to install a version of `prefect-github` for development, clone the repository and perform an editable install with `pip`: - -```bash -git clone https://github.com/PrefectHQ/prefect-github.git - -cd prefect-github/ - -pip install -e ".[dev]" - -# Install linting pre-commit hooks -pre-commit install -``` +See the docs at [https://docs.prefect.io/integrations/prefect-github](https://docs.prefect.io/integrations/prefect-github) for more information. diff --git a/src/integrations/prefect-gitlab/README.md b/src/integrations/prefect-gitlab/README.md index 0ace5870af1e..8d0f3b1df7a5 100644 --- a/src/integrations/prefect-gitlab/README.md +++ b/src/integrations/prefect-gitlab/README.md @@ -7,78 +7,4 @@

-## Welcome! - -`prefect-gitlab` is a Prefect collection for working with GitLab repositories. - -## Getting Started - -### Python setup - -Requires an installation of Python 3.9 or higher. - -We recommend using a Python virtual environment manager such as pipenv, conda, or virtualenv. - -This integration is designed to work with Prefect 2.3.0 or higher. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -### Installation - -Install `prefect-gitlab` with `pip`: - -```bash -pip install prefect-gitlab -``` - -Then, register the [block types](https://docs.prefect.io/concepts/blocks/)) in this integration to view the storage block type on Prefect Cloud: - -```bash -prefect block register -m prefect_gitlab -``` - -Note, to use the `load` method on a block, you must already have a block document [saved](https://docs.prefect.io/concepts/blocks/). - -## Creating a GitLab storage block - -### In Python - -```python -from prefect_gitlab import GitLabRepository - -# public GitLab repository -public_gitlab_block = GitLabRepository( - name="my-gitlab-block", - repository="https://gitlab.com/testing/my-repository.git" -) - -public_gitlab_block.save() - - -# specific branch or tag of a GitLab repository -branch_gitlab_block = GitLabRepository( - name="my-gitlab-block", - reference="branch-or-tag-name", - repository="https://gitlab.com/testing/my-repository.git" -) - -branch_gitlab_block.save() - - -# Get all history of a specific branch or tag of a GitLab repository -branch_gitlab_block = GitLabRepository( - name="my-gitlab-block", - reference="branch-or-tag-name", - git_depth=None, - repository="https://gitlab.com/testing/my-repository.git" -) - -branch_gitlab_block.save() - -# private GitLab repository -private_gitlab_block = GitLabRepository( - name="my-private-gitlab-block", - repository="https://gitlab.com/testing/my-repository.git", - access_token="YOUR_GITLAB_PERSONAL_ACCESS_TOKEN" -) - -private_gitlab_block.save() -``` +See the docs at [https://docs.prefect.io/integrations/prefect-gitlab](https://docs.prefect.io/integrations/prefect-gitlab) for more information. diff --git a/src/integrations/prefect-kubernetes/README.md b/src/integrations/prefect-kubernetes/README.md index d62c14110dca..5c0fd0d53bd0 100644 --- a/src/integrations/prefect-kubernetes/README.md +++ b/src/integrations/prefect-kubernetes/README.md @@ -7,149 +7,4 @@

- -## Welcome! - -`prefect-kubernetes` is a collection of Prefect tasks, flows, and blocks enabling orchestration, observation and management of Kubernetes resources. - -Jump to [examples](#example-usage). - -## Resources - -For more tips on how to use tasks and flows in a Collection, check out [Using Collections](https://docs.prefect.io/collections/usage/)! - -### Installation -Install `prefect-kubernetes` with `pip`: -```bash - pip install prefect-kubernetes - ``` - -Requires an installation of Python 3.9+. - -We recommend using a Python virtual environment manager such as pipenv, conda or virtualenv. - -These tasks are designed to work with Prefect 2. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -Then, to register [blocks](https://docs.prefect.io/ui/blocks/) on Prefect Cloud: - -```bash -prefect block register -m prefect_kubernetes -``` - -Note, to use the `load` method on Blocks, you must already have a block document [saved through code](https://docs.prefect.io/concepts/blocks/#saving-blocks) or saved through the UI. - - -### Example Usage - -#### Use `with_options` to customize options on any existing task or flow - -```python -from prefect_kubernetes.flows import run_namespaced_job - -customized_run_namespaced_job = run_namespaced_job.with_options( - name="My flow running a Kubernetes Job", - retries=2, - retry_delay_seconds=10, -) # this is now a new flow object that can be called -``` - -For more tips on how to use tasks and flows in a Collection, check out [Using Collections](https://docs.prefect.io/collections/usage/)! - - -#### Specify and run a Kubernetes Job from a yaml file - -```python -from prefect_kubernetes.credentials import KubernetesCredentials -from prefect_kubernetes.flows import run_namespaced_job # this is a flow -from prefect_kubernetes.jobs import KubernetesJob - -k8s_creds = KubernetesCredentials.load("k8s-creds") - -job = KubernetesJob.from_yaml_file( # or create in the UI with a dict manifest - credentials=k8s_creds, - manifest_path="path/to/job.yaml", -) - -job.save("my-k8s-job", overwrite=True) - -if __name__ == "__main__": - # run the flow - run_namespaced_job(job) -``` - -#### Generate a resource-specific client from `KubernetesClusterConfig` - -```python -# with minikube / docker desktop & a valid ~/.kube/config this should ~just work~ -from prefect_kubernetes.credentials import KubernetesCredentials, KubernetesClusterConfig - -k8s_config = KubernetesClusterConfig.from_file('~/.kube/config') - -k8s_credentials = KubernetesCredentials(cluster_config=k8s_config) - -with k8s_credentials.get_client("core") as v1_core_client: - for namespace in v1_core_client.list_namespace().items: - print(namespace.metadata.name) -``` - - -#### List jobs in a specific namespace - -```python -from prefect import flow -from prefect_kubernetes.credentials import KubernetesCredentials -from prefect_kubernetes.jobs import list_namespaced_job - -@flow -def kubernetes_orchestrator(): - v1_job_list = list_namespaced_job( - kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), - namespace="my-namespace", - ) -``` - -#### Patch an existing deployment - -```python -from kubernetes_asyncio.client.models import V1Deployment - -from prefect import flow -from prefect_kubernetes.credentials import KubernetesCredentials -from prefect_kubernetes.deployments import patch_namespaced_deployment - -@flow -def kubernetes_orchestrator(): - v1_deployment = patch_namespaced_deployment( - kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), - deployment_name="my-deployment", - deployment_updates=yaml.safe_load(...), - namespace="my-namespace" - ) - print(v1_deployment) -``` - -## Feedback - -If you encounter any bugs while using `prefect-kubernetes`, feel free to open an issue in the [prefect](https://github.com/PrefectHQ/prefect) repository. - -If you have any questions or issues while using `prefect-kubernetes`, you can find help in either the [Prefect Discourse forum](https://discourse.prefect.io/) or the [Prefect Slack community](https://prefect.io/slack). - -## Contributing - -If you'd like to help contribute to fix an issue or add a feature to `prefect-kubernetes`, please [propose changes through a pull request from a fork of the repository](https://docs.github.com/en/pull-requests/collaborating-with-pull-requests/proposing-changes-to-your-work-with-pull-requests/creating-a-pull-request-from-a-fork). - -Here are the steps: - -1. [Fork the repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#forking-a-repository) -2. [Clone the forked repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#cloning-your-forked-repository) -3. Install the repository and its dependencies: -``` - pip install -e ".[dev]" -``` -4. Make desired changes -5. Add tests -6. Install `pre-commit` to perform quality checks prior to commit: -``` - pre-commit install - ``` -8. `git commit`, `git push`, and create a pull request +See the docs at [https://docs.prefect.io/integrations/prefect-kubernetes](https://docs.prefect.io/integrations/prefect-kubernetes) for more information. diff --git a/src/integrations/prefect-ray/README.md b/src/integrations/prefect-ray/README.md index 3d0e7c7385ad..467c20fc9130 100644 --- a/src/integrations/prefect-ray/README.md +++ b/src/integrations/prefect-ray/README.md @@ -9,195 +9,4 @@

-## Welcome! -Visit the full docs [here](https://PrefectHQ.github.io/prefect-ray) to see additional examples and the API reference. - -`prefect-ray` contains Prefect integrations with the [Ray](https://www.ray.io/) execution framework, a flexible distributed computing framework for Python. - -Provides a `RayTaskRunner` that enables Prefect flows to run tasks execute tasks in parallel using Ray. - -## Getting Started - -### Python setup - -Requires an installation of Python 3.9 or newer. - -We recommend using a Python virtual environment manager such as pipenv, conda, or virtualenv. - -These tasks are designed to work with Prefect 2.0+. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -### Installation - -Install `prefect-ray` with `pip`: - -```bash -pip install prefect-ray -``` - -Users running Apple Silicon (such as M1 macs) should check out the Ray docs [here](https://docs.ray.io/en/master/ray-overview/installation.html#m1-mac-apple-silicon-support) for more details. - -## Running tasks on Ray - -The `RayTaskRunner` is a [Prefect task runner](https://docs.prefect.io/concepts/task-runners/) that submits tasks to [Ray](https://www.ray.io/) for parallel execution. - -By default, a temporary Ray instance is created for the duration of the flow run. - -For example, this flow counts to 3 in parallel. - -```python -import time - -from prefect import flow, task -from prefect_ray import RayTaskRunner - -@task -def shout(number): - time.sleep(0.5) - print(f"#{number}") - -@flow(task_runner=RayTaskRunner) -def count_to(highest_number): - for number in range(highest_number): - shout.submit(number) - -if __name__ == "__main__": - count_to(10) - -# outputs -#3 -#7 -#2 -#6 -#4 -#0 -#1 -#5 -#8 -#9 -``` - -If you already have a Ray instance running, you can provide the connection URL via an `address` argument. - -To configure your flow to use the `RayTaskRunner`: - -1. Make sure the `prefect-ray` collection is installed as described earlier: `pip install prefect-ray`. -2. In your flow code, import `RayTaskRunner` from `prefect_ray.task_runners`. -3. Assign it as the task runner when the flow is defined using the `task_runner=RayTaskRunner` argument. - -For example, this flow uses the `RayTaskRunner` with a local, temporary Ray instance created by Prefect at flow run time. - -```python -from prefect import flow -from prefect_ray.task_runners import RayTaskRunner - -@flow(task_runner=RayTaskRunner()) -def my_flow(): - ... -``` - -This flow uses the `RayTaskRunner` configured to access an existing Ray instance at `ray://192.0.2.255:8786`. - -```python -from prefect import flow -from prefect_ray.task_runners import RayTaskRunner - -@flow(task_runner=RayTaskRunner(address="ray://192.0.2.255:8786")) -def my_flow(): - ... -``` - -`RayTaskRunner` accepts the following optional parameters: - -| Parameter | Description | -| ----------- | ----------------------------------------------------------------------------------------------------------------------------------- | -| address | Address of a currently running Ray instance, starting with the [ray://](https://docs.ray.io/en/master/cluster/ray-client.html) URI. | -| init_kwargs | Additional kwargs to use when calling `ray.init`. | - -Note that Ray Client uses the [ray://](https://docs.ray.io/en/master/cluster/ray-client.html) URI to indicate the address of a Ray instance. If you don't provide the `address` of a Ray instance, Prefect creates a temporary instance automatically. - -!!! warning "Ray environment limitations" - Ray support for non-x86/64 architectures such as ARM/M1 processors with installation from `pip` alone and will be skipped during installation of Prefect. It is possible to manually install the blocking component with `conda`. See the [Ray documentation](https://docs.ray.io/en/latest/ray-overview/installation.html#m1-mac-apple-silicon-support) for instructions. - - See the [Ray installation documentation](https://docs.ray.io/en/latest/ray-overview/installation.html) for further compatibility information. - -## Running tasks on a Ray remote cluster - -When using the `RayTaskRunner` with a remote Ray cluster, you may run into issues that are not seen when using a local Ray instance. To resolve these issues, we recommend taking the following steps when working with a remote Ray cluster: - -1. By default, Prefect will not persist any data to the filesystem of the remote ray worker. However, if you want to take advantage of Prefect's caching ability, you will need to configure a remote result storage to persist results across task runs. - -We recommend using the [Prefect UI to configure a storage block](https://docs.prefect.io/concepts/blocks/) to use for remote results storage. - -Here's an example of a flow that uses caching and remote result storage: -```python -from typing import List - -from prefect import flow, task -from prefect.logging import get_run_logger -from prefect.filesystems import S3 -from prefect.tasks import task_input_hash -from prefect_ray.task_runners import RayTaskRunner - - -# The result of this task will be cached in the configured result storage -@task(cache_key_fn=task_input_hash) -def say_hello(name: str) -> None: - logger = get_run_logger() - # This log statement will print only on the first run. Subsequent runs will be cached. - logger.info(f"hello {name}!") - return name - - -@flow( - task_runner=RayTaskRunner( - address="ray://:10001", - ), - # Using an S3 block that has already been created via the Prefect UI - result_storage="s3/my-result-storage", -) -def greetings(names: List[str]) -> None: - for name in names: - say_hello.submit(name) - - -if __name__ == "__main__": - greetings(["arthur", "trillian", "ford", "marvin"]) -``` - -2. If you get an error stating that the module 'prefect' cannot be found, ensure `prefect` is installed on the remote cluster, with: -```bash -pip install prefect -``` - -3. If you get an error with a message similar to "File system created with scheme 's3' could not be created", ensure the required Python modules are installed on **both local and remote machines**. For example, if using S3 for the remote storage: -```bash -pip install s3fs -``` - -4. If you are seeing timeout or other connection errors, double check the address provided to the `RayTaskRunner`. The address should look similar to: `address='ray://:10001'`: -```bash -RayTaskRunner(address="ray://1.23.199.255:10001") -``` - -## Specifying remote options - -The `remote_options` context can be used to control the task’s remote options. - -For example, we can set the number of CPUs and GPUs to use for the `process` task: - -```python -from prefect import flow, task -from prefect_ray.task_runners import RayTaskRunner -from prefect_ray.context import remote_options - -@task -def process(x): - return x + 1 - - -@flow(task_runner=RayTaskRunner()) -def my_flow(): - # equivalent to setting @ray.remote(num_cpus=4, num_gpus=2) - with remote_options(num_cpus=4, num_gpus=2): - process.submit(42) -``` +See the docs at [https://docs.prefect.io/integrations/prefect-ray](https://docs.prefect.io/integrations/prefect-ray) for more information. diff --git a/src/integrations/prefect-ray/prefect_ray/task_runners.py b/src/integrations/prefect-ray/prefect_ray/task_runners.py index 42bf842c4244..c35e5a2ae1ae 100644 --- a/src/integrations/prefect-ray/prefect_ray/task_runners.py +++ b/src/integrations/prefect-ray/prefect_ray/task_runners.py @@ -1,6 +1,6 @@ """ Interface and implementations of the Ray Task Runner. -[Task Runners](https://docs.prefect.io/api-ref/prefect/task-runners/) +[Task Runners](https://docs.prefect.io/latest/develop/task-runners/) in Prefect are responsible for managing the execution of Prefect task runs. Generally speaking, users are not expected to interact with task runners outside of configuring and initializing them for a flow. diff --git a/src/integrations/prefect-redis/README.md b/src/integrations/prefect-redis/README.md index 363a4b3a3ed5..c1f0e20c01db 100644 --- a/src/integrations/prefect-redis/README.md +++ b/src/integrations/prefect-redis/README.md @@ -7,68 +7,4 @@

-## Welcome! - -Prefect integrations for working with Redis - -## Getting Started - -### Python setup - -Requires an installation of Python 3.9+. - -We recommend using a Python virtual environment manager such as pipenv, conda or virtualenv. - -These tasks are designed to work with Prefect 2.0. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -### Installation - -Install `prefect-redis` with `pip`: - -```bash -pip install prefect-redis -``` - -Then, register to view the block on Prefect Cloud: - -```bash -prefect block register -m prefect_redis.credentials -``` - -Note, to use the `load` method on Blocks, you must already have a block document [saved through code](https://docs.prefect.io/concepts/blocks/#saving-blocks) or [saved through the UI](https://docs.prefect.io/ui/blocks/). - -### Write and run a flow - -```python -from prefect import flow -from prefect_redis import ( - RedisCredentials, - redis_set, - redis_get, -) - - -@flow -def example_flow(): - - # Load credentials-block - credentials = RedisCredentials.load("my-redis-store") - - # Set a redis-key - Supports any object that is not a live connection - redis_set(credentials, "mykey", {"foo": "bar"}) - - # Get a redis key - val = redis_get(credentials, "mykey") - - print(val) - -example_flow() -``` - -## Resources - -If you encounter any bugs while using `prefect-redis`, feel free to open an issue in the [prefect-redis](https://github.com/C4IROcean/prefect-redis) repository. - -If you have any questions or issues while using `prefect-redis`, you can find help in either the [Prefect Discourse forum](https://discourse.prefect.io/) or the [Prefect Slack community](https://prefect.io/slack). - - +For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). diff --git a/src/integrations/prefect-shell/README.md b/src/integrations/prefect-shell/README.md index bcf3b055db8a..c758320bc8b9 100644 --- a/src/integrations/prefect-shell/README.md +++ b/src/integrations/prefect-shell/README.md @@ -1,4 +1,4 @@ -# Integrating shell commands into your dataflow with `prefect-shell` +# Integrate shell commands into your workflows with `prefect-shell`

@@ -7,157 +7,4 @@

-Visit the full docs [here](https://PrefectHQ.github.io/prefect-shell) to see additional examples and the API reference. - -The prefect-shell collection makes it easy to execute shell commands in your Prefect flows. Check out the examples below to get started! - -## Getting Started - -### Integrate with Prefect flows - -With prefect-shell, you can bring your trusty shell commands (and/or scripts) straight into the Prefect flow party, complete with awesome Prefect logging. - -No more separate logs, just seamless integration. Let's get the shell-abration started! - -```python -from prefect import flow -from datetime import datetime -from prefect_shell import ShellOperation - -@flow -def download_data(): - today = datetime.today().strftime("%Y%m%d") - - # for short running operations, you can use the `run` method - # which automatically manages the context - ShellOperation( - commands=[ - "mkdir -p data", - "mkdir -p data/${today}" - ], - env={"today": today} - ).run() - - # for long running operations, you can use a context manager - with ShellOperation( - commands=[ - "curl -O https://masie_web.apps.nsidc.org/pub/DATASETS/NOAA/G02135/north/daily/data/N_seaice_extent_daily_v3.0.csv", - ], - working_dir=f"data/{today}", - ) as download_csv_operation: - - # trigger runs the process in the background - download_csv_process = download_csv_operation.trigger() - - # then do other things here in the meantime, like download another file - ... - - # when you're ready, wait for the process to finish - download_csv_process.wait_for_completion() - - # if you'd like to get the output lines, you can use the `fetch_result` method - output_lines = download_csv_process.fetch_result() - -download_data() -``` - -Outputs: -```bash -14:48:16.550 | INFO | prefect.engine - Created flow run 'tentacled-chachalaca' for flow 'download-data' -14:48:17.977 | INFO | Flow run 'tentacled-chachalaca' - PID 19360 triggered with 2 commands running inside the '.' directory. -14:48:17.987 | INFO | Flow run 'tentacled-chachalaca' - PID 19360 completed with return code 0. -14:48:17.994 | INFO | Flow run 'tentacled-chachalaca' - PID 19363 triggered with 1 commands running inside the PosixPath('data/20230201') directory. -14:48:18.009 | INFO | Flow run 'tentacled-chachalaca' - PID 19363 stream output: - % Total % Received % Xferd Average Speed Time Time Time Current - Dl -14:48:18.010 | INFO | Flow run 'tentacled-chachalaca' - PID 19363 stream output: -oad Upload Total Spent Left Speed - 0 0 0 0 0 0 0 0 --:--:-- --:--:-- --:--:-- 0 -14:48:18.840 | INFO | Flow run 'tentacled-chachalaca' - PID 19363 stream output: - 11 1630k 11 192k 0 0 229k 0 0:00:07 --:--:-- 0:00:07 231k -14:48:19.839 | INFO | Flow run 'tentacled-chachalaca' - PID 19363 stream output: - 83 1630k 83 1368k 0 0 745k 0 0:00:02 0:00:01 0:00:01 747k -14:48:19.993 | INFO | Flow run 'tentacled-chachalaca' - PID 19363 stream output: -100 1630k 100 1630k 0 0 819k 0 0 -14:48:19.994 | INFO | Flow run 'tentacled-chachalaca' - PID 19363 stream output: -:00:01 0:00:01 --:--:-- 821k -14:48:19.996 | INFO | Flow run 'tentacled-chachalaca' - PID 19363 completed with return code 0. -14:48:19.998 | INFO | Flow run 'tentacled-chachalaca' - Successfully closed all open processes. -14:48:20.203 | INFO | Flow run 'tentacled-chachalaca' - Finished in state Completed() -``` - -!!! info "Utilize Previously Saved Blocks" - - You can save commands within a `ShellOperation` block, then reuse them across multiple flows, or even plain Python scripts. - - Save the block with desired commands: - - ```python - from prefect_shell import ShellOperation - - ping_op = ShellOperation(commands=["ping -t 1 prefect.io"]) - ping_op.save("block-name") - ``` - - Load the saved block: - - ```python - from prefect_shell import ShellOperation - - ping_op = ShellOperation.load("block-name") - ``` - - To [view and edit the blocks](https://orion-docs.prefect.io/ui/blocks/) on Prefect UI: - - ```bash - prefect block register -m prefect_shell - ``` - -## Resources - -For more tips on how to use tasks and flows in a Collection, check out [Using Collections](https://orion-docs.prefect.io/collections/usage/)! - -### Installation - -Install `prefect-shell` with `pip`: - -```bash -pip install -U prefect-shell -``` - -A list of available blocks in `prefect-shell` and their setup instructions can be found [here](https://PrefectHQ.github.io/prefect-shell/blocks_catalog). - -Requires an installation of Python 3.7+. - -We recommend using a Python virtual environment manager such as pipenv, conda or virtualenv. - -These tasks are designed to work with Prefect 2. For more information about how to use Prefect, please refer to the [Prefect documentation](https://orion-docs.prefect.io/). - -### Feedback - -If you encounter any bugs while using `prefect-shell`, feel free to open an issue in the [prefect-shell](https://github.com/PrefectHQ/prefect-shell) repository. - -If you have any questions or issues while using `prefect-shell`, you can find help in either the [Prefect Discourse forum](https://discourse.prefect.io/) or the [Prefect Slack community](https://prefect.io/slack). - -Feel free to star or watch [`prefect-shell`](https://github.com/PrefectHQ/prefect-shell) for updates too! - -### Contributing - -If you'd like to help contribute to fix an issue or add a feature to `prefect-shell`, please [propose changes through a pull request from a fork of the repository](https://docs.github.com/en/pull-requests/collaborating-with-pull-requests/proposing-changes-to-your-work-with-pull-requests/creating-a-pull-request-from-a-fork). - -Here are the steps: - -1. [Fork the repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#forking-a-repository) -2. [Clone the forked repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#cloning-your-forked-repository) -3. Install the repository and its dependencies: -``` -pip install -e ".[dev]" -``` -4. Make desired changes -5. Add tests -6. Insert an entry to [CHANGELOG.md](https://github.com/PrefectHQ/prefect-shell/blob/main/CHANGELOG.md) -7. Install `pre-commit` to perform quality checks prior to commit: -``` -pre-commit install -``` -8. `git commit`, `git push`, and create a pull request +See the docs at [https://docs.prefect.io/integrations/prefect-shell](https://docs.prefect.io/integrations/prefect-shell) for more information. diff --git a/src/integrations/prefect-snowflake/README.md b/src/integrations/prefect-snowflake/README.md index 9842a36c5693..e9743dd43d5d 100644 --- a/src/integrations/prefect-snowflake/README.md +++ b/src/integrations/prefect-snowflake/README.md @@ -7,253 +7,4 @@

-## Welcome! - -The prefect-snowflake collection makes it easy to connect to a Snowflake database in your Prefect flows. Check out the examples below to get started! - -## Getting Started - -### Integrate with Prefect flows - -Prefect works with Snowflake by providing dataflow automation for faster, more efficient data pipeline creation, execution, and monitoring. - -This results in reduced errors, increased confidence in your data, and ultimately, faster insights. - -To set up a table, use the `execute` and `execute_many` methods. Then, use the `fetch_many` method to retrieve data in a stream until there's no more data. - -By using the `SnowflakeConnector` as a context manager, you can make sure that the Snowflake connection and cursors are closed properly after you're done with them. - -Be sure to install [prefect-snowflake](#installation) and [save to block](#saving-credentials-to-block) to run the examples below! - -=== "Sync" - -```python -from prefect import flow, task -from prefect_snowflake import SnowflakeConnector - - -@task -def setup_table(block_name: str) -> None: - with SnowflakeConnector.load(block_name) as connector: - connector.execute( - "CREATE TABLE IF NOT EXISTS customers (name varchar, address varchar);" - ) - connector.execute_many( - "INSERT INTO customers (name, address) VALUES (%(name)s, %(address)s);", - seq_of_parameters=[ - {"name": "Ford", "address": "Highway 42"}, - {"name": "Unknown", "address": "Space"}, - {"name": "Me", "address": "Myway 88"}, - ], - ) - -@task -def fetch_data(block_name: str) -> list: - all_rows = [] - with SnowflakeConnector.load(block_name) as connector: - while True: - # Repeated fetch* calls using the same operation will - # skip re-executing and instead return the next set of results - new_rows = connector.fetch_many("SELECT * FROM customers", size=2) - if len(new_rows) == 0: - break - all_rows.append(new_rows) - return all_rows - -@flow -def snowflake_flow(block_name: str) -> list: - setup_table(block_name) - all_rows = fetch_data(block_name) - return all_rows - -snowflake_flow() -``` - -=== "Async" - -```python -from prefect import flow, task -from prefect_snowflake import SnowflakeConnector -import asyncio - -@task -async def setup_table(block_name: str) -> None: - with await SnowflakeConnector.load(block_name) as connector: - await connector.execute( - "CREATE TABLE IF NOT EXISTS customers (name varchar, address varchar);" - ) - await connector.execute_many( - "INSERT INTO customers (name, address) VALUES (%(name)s, %(address)s);", - seq_of_parameters=[ - {"name": "Ford", "address": "Highway 42"}, - {"name": "Unknown", "address": "Space"}, - {"name": "Me", "address": "Myway 88"}, - ], - ) - -@task -async def fetch_data(block_name: str) -> list: - all_rows = [] - with await SnowflakeConnector.load(block_name) as connector: - while True: - # Repeated fetch* calls using the same operation will - # skip re-executing and instead return the next set of results - new_rows = await connector.fetch_many("SELECT * FROM customers", size=2) - if len(new_rows) == 0: - break - all_rows.append(new_rows) - return all_rows - -@flow -async def snowflake_flow(block_name: str) -> list: - await setup_table(block_name) - all_rows = await fetch_data(block_name) - return all_rows - -asyncio.run(snowflake_flow("example")) -``` - -### Access underlying Snowflake connection - -If the native methods of the block don't meet your requirements, don't worry. - -You have the option to access the underlying Snowflake connection and utilize its built-in methods as well. - -```python -import pandas as pd -from prefect import flow -from prefect_snowflake.database import SnowflakeConnector -from snowflake.connector.pandas_tools import write_pandas - -@flow -def snowflake_write_pandas_flow(): - connector = SnowflakeConnector.load("my-block") - with connector.get_connection() as connection: - table_name = "TABLE_NAME" - ddl = "NAME STRING, NUMBER INT" - statement = f'CREATE TABLE IF NOT EXISTS {table_name} ({ddl})' - with connection.cursor() as cursor: - cursor.execute(statement) - - # case sensitivity matters here! - df = pd.DataFrame([('Marvin', 42), ('Ford', 88)], columns=['NAME', 'NUMBER']) - success, num_chunks, num_rows, _ = write_pandas( - conn=connection, - df=df, - table_name=table_name, - database=snowflake_connector.database, - schema=snowflake_connector.schema_ # note the "_" suffix - ) -``` - -## Resources - -For more tips on how to use tasks and flows in an integration, check out [Using Collections](https://docs.prefect.io/integrations/usage/)! - -### Installation - -Install `prefect-snowflake` with `pip`: - -```bash -pip install prefect-snowflake -``` - -A list of available blocks in `prefect-snowflake` and their setup instructions can be found [here](https://PrefectHQ.github.io/prefect-snowflake/blocks_catalog). - -Requires an installation of Python 3.9+. - -We recommend using a Python virtual environment manager such as pipenv, conda or virtualenv. - -These tasks are designed to work with Prefect 2. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -### Saving credentials to block - -Note, to use the `load` method on Blocks, you must already have a block document [saved through code](https://docs.prefect.io/concepts/blocks/#saving-blocks) or saved through the UI. - -Below is a walkthrough on saving a `SnowflakeCredentials` block through code. - -1. Head over to https://app.snowflake.com/. -2. Login to your Snowflake account, e.g. nh12345.us-east-2.aws, with your username and password. -3. Use those credentials to fill replace the placeholders below. - -```python -from prefect_snowflake import SnowflakeCredentials - -credentials = SnowflakeCredentials( - account="ACCOUNT-PLACEHOLDER", # resembles nh12345.us-east-2.aws - user="USER-PLACEHOLDER", - password="PASSWORD-PLACEHOLDER" -) -credentials.save("CREDENTIALS-BLOCK-NAME-PLACEHOLDER") -``` - -Then, to create a `SnowflakeConnector` block: - -1. After logging in, click on any worksheet. -2. On the left side, select a database and schema. -3. On the top right, select a warehouse. -3. Create a short script, replacing the placeholders below. - -```python -from prefect_snowflake import SnowflakeCredentials, SnowflakeConnector - -credentials = SnowflakeCredentials.load("CREDENTIALS-BLOCK-NAME-PLACEHOLDER") - -connector = SnowflakeConnector( - credentials=credentials, - database="DATABASE-PLACEHOLDER", - schema="SCHEMA-PLACEHOLDER", - warehouse="COMPUTE_WH", -) -connector.save("CONNECTOR-BLOCK-NAME-PLACEHOLDER") -``` - -Congrats! You can now easily load the saved block, which holds your credentials and connection info: - -```python -from prefect_snowflake import SnowflakeCredentials, SnowflakeConnector - -SnowflakeCredentials.load("CREDENTIALS-BLOCK-NAME-PLACEHOLDER") -SnowflakeConnector.load("CONNECTOR-BLOCK-NAME-PLACEHOLDER") -``` - -!!! info "Registering blocks" - -Register blocks in this module to -[view and edit them](https://docs.prefect.io/ui/blocks/) -on Prefect Cloud: - -```bash -prefect block register -m prefect_snowflake -``` - -A list of available blocks in `prefect-snowflake` and their setup instructions can be found [here](https://PrefectHQ.github.io/prefect-snowflake/blocks_catalog). - -### Feedback - -If you encounter any bugs while using `prefect-snowflake`, feel free to open an issue in the [prefect-snowflake](https://github.com/PrefectHQ/prefect-snowflake) repository. - -If you have any questions or issues while using `prefect-snowflake`, you can find help in either the [Prefect Discourse forum](https://discourse.prefect.io/) or the [Prefect Slack community](https://prefect.io/slack). - -Feel free to star or watch [`prefect-snowflake`](https://github.com/PrefectHQ/prefect-snowflake) for updates too! - -### Contributing - -If you'd like to help contribute to fix an issue or add a feature to `prefect-snowflake`, please [propose changes through a pull request from a fork of the repository](https://docs.github.com/en/pull-requests/collaborating-with-pull-requests/proposing-changes-to-your-work-with-pull-requests/creating-a-pull-request-from-a-fork). - -Here are the steps: - -1. [Fork the repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#forking-a-repository) -2. [Clone the forked repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#cloning-your-forked-repository) -3. Install the repository and its dependencies: -``` -pip install -e ".[dev]" -``` -4. Make desired changes -5. Add tests -6. Insert an entry to [CHANGELOG.md](https://github.com/PrefectHQ/prefect-snowflake/blob/main/CHANGELOG.md) -7. Install `pre-commit` to perform quality checks prior to commit: -``` -pre-commit install -``` -8. `git commit`, `git push`, and create a pull request +See the docs at [https://docs.prefect.io/integrations/prefect-snowflake](https://docs.prefect.io/integrations/prefect-snowflake) for more information. diff --git a/src/integrations/prefect-sqlalchemy/README.md b/src/integrations/prefect-sqlalchemy/README.md index 0932e13a889e..3c1179182acb 100644 --- a/src/integrations/prefect-sqlalchemy/README.md +++ b/src/integrations/prefect-sqlalchemy/README.md @@ -7,232 +7,4 @@

-Visit the full docs [here](https://PrefectHQ.github.io/prefect-sqlalchemy) to see additional examples and the API reference. - -The prefect-sqlalchemy collection makes it easy to connect to a database in your Prefect flows. Check out the examples below to get started! - -## Getting started - -### Integrate with Prefect flows - -Prefect and SQLAlchemy are a data powerhouse duo. With Prefect, your workflows are orchestratable and observable, and with SQLAlchemy, your databases are a snap to handle! Get ready to experience the ultimate data "flow-chemistry"! - -To set up a table, use the `execute` and `execute_many` methods. Then, use the `fetch_many` method to retrieve data in a stream until there's no more data. - -By using the `SqlAlchemyConnector` as a context manager, you can make sure that the SQLAlchemy engine and any connected resources are closed properly after you're done with them. - -Be sure to install [prefect-sqlalchemy](#installation) and [save your credentials to a Prefect block](#saving-credentials-to-block) to run the examples below! - -!!! note "Async support" - - `SqlAlchemyConnector` also supports async workflows! Just be sure to save, load, and use an async driver as in the example below. - - ```python - from prefect_sqlalchemy import SqlAlchemyConnector, ConnectionComponents, AsyncDriver - - connector = SqlAlchemyConnector( - connection_info=ConnectionComponents( - driver=AsyncDriver.SQLITE_AIOSQLITE, - database="DATABASE-PLACEHOLDER.db" - ) - ) - - connector.save("BLOCK_NAME-PLACEHOLDER") - ``` - -=== "Sync" - - ```python - from prefect import flow, task - from prefect_sqlalchemy import SqlAlchemyConnector - - - @task - def setup_table(block_name: str) -> None: - with SqlAlchemyConnector.load(block_name) as connector: - connector.execute( - "CREATE TABLE IF NOT EXISTS customers (name varchar, address varchar);" - ) - connector.execute( - "INSERT INTO customers (name, address) VALUES (:name, :address);", - parameters={"name": "Marvin", "address": "Highway 42"}, - ) - connector.execute_many( - "INSERT INTO customers (name, address) VALUES (:name, :address);", - seq_of_parameters=[ - {"name": "Ford", "address": "Highway 42"}, - {"name": "Unknown", "address": "Highway 42"}, - ], - ) - - @task - def fetch_data(block_name: str) -> list: - all_rows = [] - with SqlAlchemyConnector.load(block_name) as connector: - while True: - # Repeated fetch* calls using the same operation will - # skip re-executing and instead return the next set of results - new_rows = connector.fetch_many("SELECT * FROM customers", size=2) - if len(new_rows) == 0: - break - all_rows.append(new_rows) - return all_rows - - @flow - def sqlalchemy_flow(block_name: str) -> list: - setup_table(block_name) - all_rows = fetch_data(block_name) - return all_rows - - - sqlalchemy_flow("BLOCK-NAME-PLACEHOLDER") - ``` - -=== "Async" - - ```python - from prefect import flow, task - from prefect_sqlalchemy import SqlAlchemyConnector - import asyncio - - @task - async def setup_table(block_name: str) -> None: - async with await SqlAlchemyConnector.load(block_name) as connector: - await connector.execute( - "CREATE TABLE IF NOT EXISTS customers (name varchar, address varchar);" - ) - await connector.execute( - "INSERT INTO customers (name, address) VALUES (:name, :address);", - parameters={"name": "Marvin", "address": "Highway 42"}, - ) - await connector.execute_many( - "INSERT INTO customers (name, address) VALUES (:name, :address);", - seq_of_parameters=[ - {"name": "Ford", "address": "Highway 42"}, - {"name": "Unknown", "address": "Highway 42"}, - ], - ) - - @task - async def fetch_data(block_name: str) -> list: - all_rows = [] - async with await SqlAlchemyConnector.load(block_name) as connector: - while True: - # Repeated fetch* calls using the same operation will - # skip re-executing and instead return the next set of results - new_rows = await connector.fetch_many("SELECT * FROM customers", size=2) - if len(new_rows) == 0: - break - all_rows.append(new_rows) - return all_rows - - @flow - async def sqlalchemy_flow(block_name: str) -> list: - await setup_table(block_name) - all_rows = await fetch_data(block_name) - return all_rows - - - asyncio.run(sqlalchemy_flow("BLOCK-NAME-PLACEHOLDER")) - ``` - -## Resources - -For more tips on how to use tasks and flows provided in a Prefect integration library, check out the [Prefect docs on using integrations](https://docs.prefect.io/integrations/usage/). - -### Installation - -Install `prefect-sqlalchemy` with `pip`: - -```bash -pip install prefect-sqlalchemy -``` - -Requires an installation of Python 3.9 or higher. - -We recommend using a Python virtual environment manager such as pipenv, conda, or virtualenv. - -The tasks in this library are designed to work with Prefect 2. For more information about how to use Prefect, please refer to the [Prefect documentation](https://docs.prefect.io/). - -### Saving credentials to a block - -To use the `load` method on Blocks, you must have a block document [saved through code](https://docs.prefect.io/concepts/blocks/#saving-blocks) or saved through the UI. - -Below is a walkthrough on saving block documents through code; simply create a short script, replacing the placeholders. - -```python -from prefect_sqlalchemy import SqlAlchemyConnector, ConnectionComponents, SyncDriver - -connector = SqlAlchemyConnector( - connection_info=ConnectionComponents( - driver=SyncDriver.POSTGRESQL_PSYCOPG2, - username="USERNAME-PLACEHOLDER", - password="PASSWORD-PLACEHOLDER", - host="localhost", - port=5432, - database="DATABASE-PLACEHOLDER", - ) -) - -connector.save("BLOCK_NAME-PLACEHOLDER") -``` - -Congrats! You can now easily load the saved block, which holds your credentials: - -```python -from prefect_sqlalchemy import SqlAlchemyConnector - -SqlAlchemyConnector.load("BLOCK_NAME-PLACEHOLDER") -``` - -The required keywords depend upon the desired driver. For example, SQLite requires only the `driver` and `database` arguments: - -```python -from prefect_sqlalchemy import SqlAlchemyConnector, ConnectionComponents, SyncDriver - -connector = SqlAlchemyConnector( - connection_info=ConnectionComponents( - driver=SyncDriver.SQLITE_PYSQLITE, - database="DATABASE-PLACEHOLDER.db" - ) -) - -connector.save("BLOCK_NAME-PLACEHOLDER") -``` - -!!! info "Registering blocks" - - Register blocks in this module to - [view and edit them](https://orion-docs.prefect.io/ui/blocks/) - on Prefect Cloud: - - ```bash - prefect block register -m prefect_sqlalchemy - ``` - -### Feedback - -If you encounter any bugs while using `prefect-sqlalchemy`, please open an issue in the [prefect](https://github.com/PrefectHQ/prefect) repository. - -If you have any questions or issues while using `prefect-sqlalchemy`, you can find help in the [Prefect Community Slack ](https://prefect.io/slack). - - -### Contributing - -If you'd like to help contribute to fix an issue or add a feature to `prefect-sqlalchemy`, please [propose changes through a pull request from a fork of the repository](https://docs.github.com/en/pull-requests/collaborating-with-pull-requests/proposing-changes-to-your-work-with-pull-requests/creating-a-pull-request-from-a-fork). - -Here are the steps: - -1. [Fork the repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#forking-a-repository) -2. [Clone the forked repository](https://docs.github.com/en/get-started/quickstart/fork-a-repo#cloning-your-forked-repository) -3. Install the repository and its dependencies: -``` -pip install -e ".[dev]" -``` -4. Make desired changes -5. Add tests -6. Install `pre-commit` to perform quality checks prior to commit: -``` -pre-commit install -``` -7. `git commit`, `git push`, and create a pull request +See the docs at [https://docs.prefect.io/integrations/prefect-sqlalchemy](https://docs.prefect.io/integrations/prefect-alchemy) for more information. diff --git a/src/prefect/_internal/compatibility/migration.py b/src/prefect/_internal/compatibility/migration.py index 3dfc82abd111..f160990d77dc 100644 --- a/src/prefect/_internal/compatibility/migration.py +++ b/src/prefect/_internal/compatibility/migration.py @@ -60,7 +60,7 @@ "prefect.client:get_client": "prefect.client.orchestration:get_client", } -upgrade_guide_msg = "Refer to the upgrade guide for more information: https://docs.prefect.io/latest/guides/upgrade-guide-agents-to-workers/." +upgrade_guide_msg = "Refer to the upgrade guide for more information: https://docs.prefect.io/latest/resources/upgrade-agents-to-workers." REMOVED_IN_V3 = { "prefect.client.schemas.objects:MinimalDeploymentSchedule": "Use `prefect.client.schemas.actions.DeploymentScheduleCreate` instead.", diff --git a/src/prefect/artifacts.py b/src/prefect/artifacts.py index 8dc6666aad87..785effbe9221 100644 --- a/src/prefect/artifacts.py +++ b/src/prefect/artifacts.py @@ -31,7 +31,7 @@ class Artifact(ArtifactRequest): """ An artifact is a piece of data that is created by a flow or task run. - https://docs.prefect.io/latest/concepts/artifacts/ + https://docs.prefect.io/latest/develop/artifacts Arguments: type: A string identifying the type of artifact. diff --git a/src/prefect/blocks/notifications.py b/src/prefect/blocks/notifications.py index debd2fab7fb3..8c565e499d35 100644 --- a/src/prefect/blocks/notifications.py +++ b/src/prefect/blocks/notifications.py @@ -73,7 +73,7 @@ class AppriseNotificationBlock(AbstractAppriseNotificationBlock, ABC): A base class for sending notifications using Apprise, through webhook URLs. """ - _documentation_url = "https://docs.prefect.io/ui/notifications/" + _documentation_url = "https://docs.prefect.io/latest/automate/events/automations-triggers#sending-notifications-with-automations" url: SecretStr = Field( default=..., title="Webhook URL", @@ -100,7 +100,7 @@ class SlackWebhook(AppriseNotificationBlock): _block_type_name = "Slack Webhook" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/c1965ecbf8704ee1ea20d77786de9a41ce1087d1-500x500.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/notifications/#prefect.blocks.notifications.SlackWebhook" + _documentation_url = "https://docs.prefect.io/latest/automate/events/automations-triggers#sending-notifications-with-automations" url: SecretStr = Field( default=..., @@ -126,7 +126,7 @@ class MicrosoftTeamsWebhook(AppriseNotificationBlock): _block_type_name = "Microsoft Teams Webhook" _block_type_slug = "ms-teams-webhook" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/817efe008a57f0a24f3587414714b563e5e23658-250x250.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/notifications/#prefect.blocks.notifications.MicrosoftTeamsWebhook" + _documentation_url = "https://docs.prefect.io/latest/automate/events/automations-triggers#sending-notifications-with-automations" url: SecretStr = Field( default=..., @@ -181,7 +181,7 @@ class PagerDutyWebHook(AbstractAppriseNotificationBlock): _block_type_name = "Pager Duty Webhook" _block_type_slug = "pager-duty-webhook" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/8dbf37d17089c1ce531708eac2e510801f7b3aee-250x250.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/notifications/#prefect.blocks.notifications.PagerDutyWebHook" + _documentation_url = "https://docs.prefect.io/latest/automate/events/automations-triggers#sending-notifications-with-automations" # The default cannot be prefect_default because NotifyPagerDuty's # PAGERDUTY_SEVERITY_MAP only has these notify types defined as keys @@ -291,7 +291,7 @@ class TwilioSMS(AbstractAppriseNotificationBlock): _block_type_name = "Twilio SMS" _block_type_slug = "twilio-sms" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/8bd8777999f82112c09b9c8d57083ac75a4a0d65-250x250.png" # noqa - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/notifications/#prefect.blocks.notifications.TwilioSMS" + _documentation_url = "https://docs.prefect.io/latest/automate/events/automations-triggers#sending-notifications-with-automations" account_sid: str = Field( default=..., @@ -360,7 +360,7 @@ class OpsgenieWebhook(AbstractAppriseNotificationBlock): _block_type_name = "Opsgenie Webhook" _block_type_slug = "opsgenie-webhook" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/d8b5bc6244ae6cd83b62ec42f10d96e14d6e9113-280x280.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/notifications/#prefect.blocks.notifications.OpsgenieWebhook" + _documentation_url = "https://docs.prefect.io/latest/automate/events/automations-triggers#sending-notifications-with-automations" apikey: SecretStr = Field( default=..., @@ -478,7 +478,7 @@ class MattermostWebhook(AbstractAppriseNotificationBlock): _block_type_name = "Mattermost Webhook" _block_type_slug = "mattermost-webhook" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/1350a147130bf82cbc799a5f868d2c0116207736-250x250.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/notifications/#prefect.blocks.notifications.MattermostWebhook" + _documentation_url = "https://docs.prefect.io/latest/automate/events/automations-triggers#sending-notifications-with-automations" hostname: str = Field( default=..., @@ -559,7 +559,7 @@ class DiscordWebhook(AbstractAppriseNotificationBlock): _block_type_name = "Discord Webhook" _block_type_slug = "discord-webhook" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/9e94976c80ef925b66d24e5d14f0d47baa6b8f88-250x250.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/notifications/#prefect.blocks.notifications.DiscordWebhook" + _documentation_url = "https://docs.prefect.io/latest/automate/events/automations-triggers#sending-notifications-with-automations" webhook_id: SecretStr = Field( default=..., @@ -658,7 +658,7 @@ class CustomWebhookNotificationBlock(NotificationBlock): _block_type_name = "Custom Webhook" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/c7247cb359eb6cf276734d4b1fbf00fb8930e89e-250x250.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/notifications/#prefect.blocks.notifications.CustomWebhookNotificationBlock" + _documentation_url = "https://docs.prefect.io/latest/automate/events/automations-triggers#sending-notifications-with-automations" name: str = Field(title="Name", description="Name of the webhook.") @@ -789,7 +789,7 @@ class SendgridEmail(AbstractAppriseNotificationBlock): _block_type_name = "Sendgrid Email" _block_type_slug = "sendgrid-email" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/82bc6ed16ca42a2252a5512c72233a253b8a58eb-250x250.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/notifications/#prefect.blocks.notifications.SendgridEmail" + _documentation_url = "https://docs.prefect.io/latest/automate/events/automations-triggers#sending-notifications-with-automations" api_key: SecretStr = Field( default=..., diff --git a/src/prefect/blocks/system.py b/src/prefect/blocks/system.py index 43430af39e73..135e7f21e0f1 100644 --- a/src/prefect/blocks/system.py +++ b/src/prefect/blocks/system.py @@ -44,7 +44,7 @@ class JSON(Block): """ _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/4fcef2294b6eeb423b1332d1ece5156bf296ff96-48x48.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/system/#prefect.blocks.system.JSON" + _documentation_url = "https://docs.prefect.io/latest/develop/blocks" value: Any = Field(default=..., description="A JSON-compatible value.") @@ -71,7 +71,7 @@ class String(Block): """ _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/c262ea2c80a2c043564e8763f3370c3db5a6b3e6-48x48.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/system/#prefect.blocks.system.String" + _documentation_url = "https://docs.prefect.io/latest/develop/blocks" value: str = Field(default=..., description="A string value.") @@ -99,7 +99,7 @@ class DateTime(Block): _block_type_name = "Date Time" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/8b3da9a6621e92108b8e6a75b82e15374e170ff7-48x48.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/system/#prefect.blocks.system.DateTime" + _documentation_url = "https://docs.prefect.io/latest/develop/blocks" value: PydanticDateTime = Field( default=..., @@ -129,7 +129,7 @@ class Secret(Block, Generic[T]): """ _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/c6f20e556dd16effda9df16551feecfb5822092b-48x48.png" - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/system/#prefect.blocks.system.Secret" + _documentation_url = "https://docs.prefect.io/latest/develop/blocks" value: Union[SecretStr, PydanticSecret[T]] = Field( default=..., diff --git a/src/prefect/blocks/webhook.py b/src/prefect/blocks/webhook.py index bee82305bbbc..4fa4933e3054 100644 --- a/src/prefect/blocks/webhook.py +++ b/src/prefect/blocks/webhook.py @@ -19,7 +19,9 @@ class Webhook(Block): _block_type_name = "Webhook" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/c7247cb359eb6cf276734d4b1fbf00fb8930e89e-250x250.png" # type: ignore - _documentation_url = "https://docs.prefect.io/api-ref/prefect/blocks/webhook/#prefect.blocks.webhook.Webhook" + _documentation_url = ( + "https://docs.prefect.io/latest/automate/events/webhook-triggers" + ) method: Literal["GET", "POST", "PUT", "PATCH", "DELETE"] = Field( default="POST", description="The webhook request method. Defaults to `POST`." diff --git a/src/prefect/cli/cloud/webhook.py b/src/prefect/cli/cloud/webhook.py index 9eee6d4ce772..2af798b9cfb4 100644 --- a/src/prefect/cli/cloud/webhook.py +++ b/src/prefect/cli/cloud/webhook.py @@ -86,7 +86,7 @@ async def create( ' should define (at minimum) the following attributes: \n{ "event":' ' "your.event.name", "resource": { "prefect.resource.id":' ' "your.resource.id" } }' - " \nhttps://docs.prefect.io/latest/cloud/webhooks/#webhook-templates" + " \nhttps://docs.prefect.io/latest/automate/events/webhook-triggers#webhook-templates" ) confirm_logged_in() diff --git a/src/prefect/cli/worker.py b/src/prefect/cli/worker.py index 440389fbe9d3..523787548eed 100644 --- a/src/prefect/cli/worker.py +++ b/src/prefect/cli/worker.py @@ -224,7 +224,7 @@ async def _retrieve_worker_type_from_pool(work_pool_name: Optional[str] = None) if work_pool.is_push_pool or work_pool.is_managed_pool: exit_with_error( "Workers are not required for push work pools. " - "See https://docs.prefect.io/latest/guides/deployment/push-work-pools/ " + "See https://docs.prefect.io/latest/deploy/infrastructure-examples/serverless " "for more details." ) diff --git a/src/prefect/deployments/runner.py b/src/prefect/deployments/runner.py index bd1331294bec..f49586f406ba 100644 --- a/src/prefect/deployments/runner.py +++ b/src/prefect/deployments/runner.py @@ -508,7 +508,7 @@ def from_flow( no_file_location_error = ( "Flows defined interactively cannot be deployed. Check out the" " quickstart guide for help getting started:" - " https://docs.prefect.io/latest/getting-started/quickstart" + " https://docs.prefect.io/latest/get-started/quickstart" ) ## first see if an entrypoint can be determined flow_file = getattr(flow, "__globals__", {}).get("__file__") @@ -851,14 +851,14 @@ def local_flow(): " or specify a remote storage location for the flow with `.from_source`." " If you are attempting to deploy a flow to a local process work pool," " consider using `flow.serve` instead. See the documentation for more" - " information: https://docs.prefect.io/latest/concepts/flows/#serving-a-flow" + " information: https://docs.prefect.io/latest/deploy/run-flows-in-local-processes" ) elif work_pool.type == "process" and not ignore_warnings: console.print( "Looks like you're deploying to a process work pool. If you're creating a" " deployment for local development, calling `.serve` on your flow is a great" " way to get started. See the documentation for more information:" - " https://docs.prefect.io/latest/concepts/flows/#serving-a-flow. " + " https://docs.prefect.io/latest/deploy/run-flows-in-local-processes " " Set `ignore_warnings=True` to suppress this message.", style="yellow", ) diff --git a/src/prefect/filesystems.py b/src/prefect/filesystems.py index 22a25a6b51a2..4e99b7ad3df0 100644 --- a/src/prefect/filesystems.py +++ b/src/prefect/filesystems.py @@ -84,7 +84,7 @@ class LocalFileSystem(WritableFileSystem, WritableDeploymentStorage): _block_type_name = "Local File System" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/ad39089fa66d273b943394a68f003f7a19aa850e-48x48.png" _documentation_url = ( - "https://docs.prefect.io/concepts/filesystems/#local-filesystem" + "https://docs.prefect.io/latest/develop/results#specifying-a-default-filesystem" ) basepath: Optional[str] = Field( @@ -260,7 +260,7 @@ class RemoteFileSystem(WritableFileSystem, WritableDeploymentStorage): _block_type_name = "Remote File System" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/e86b41bc0f9c99ba9489abeee83433b43d5c9365-48x48.png" _documentation_url = ( - "https://docs.prefect.io/concepts/filesystems/#remote-file-system" + "https://docs.prefect.io/latest/develop/results#specifying-a-default-filesystem" ) basepath: str = Field( @@ -433,7 +433,9 @@ class SMB(WritableFileSystem, WritableDeploymentStorage): _block_type_name = "SMB" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/3f624663f7beb97d011d011bffd51ecf6c499efc-195x195.png" - _documentation_url = "https://docs.prefect.io/concepts/filesystems/#smb" + _documentation_url = ( + "https://docs.prefect.io/latest/develop/results#specifying-a-default-filesystem" + ) share_path: str = Field( default=..., diff --git a/src/prefect/workers/process.py b/src/prefect/workers/process.py index 89fb199ce182..a180b39bd822 100644 --- a/src/prefect/workers/process.py +++ b/src/prefect/workers/process.py @@ -144,9 +144,7 @@ class ProcessWorker(BaseWorker): " when first getting started." ) _display_name = "Process" - _documentation_url = ( - "https://docs.prefect.io/latest/api-ref/prefect/workers/process/" - ) + _documentation_url = "https://docs.prefect.io/latest/get-started/quickstart" _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/356e6766a91baf20e1d08bbe16e8b5aaef4d8643-48x48.png" async def start( diff --git a/tests/cli/test_worker.py b/tests/cli/test_worker.py index 4164b401c912..080636d581b6 100644 --- a/tests/cli/test_worker.py +++ b/tests/cli/test_worker.py @@ -493,7 +493,7 @@ async def test_worker_does_not_run_with_push_pool(push_work_pool): ), ( "Workers are not required for push work pools. " - "See https://docs.prefect.io/latest/guides/deployment/push-work-pools/ " + "See https://docs.prefect.io/latest/deploy/infrastructure-examples/serverless " "for more details." ), ],