Skip to content

Commit

Permalink
K8s docs ignore tag and show executor (#7053)
Browse files Browse the repository at this point in the history
  • Loading branch information
johannkm committed Mar 14, 2022
1 parent 396da6c commit 8ad1195
Show file tree
Hide file tree
Showing 4 changed files with 39 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ The values for each of these keys is a dictionary with the YAML configuration fo

If your instance is using the <PyObject module="dagster_k8s" object="K8sRunLauncher" /> or <PyObject module="dagster_celery_k8s" object="CeleryK8sRunLauncher" />, you can use the `dagster-k8s/config` tag on a Dagster job. For example:

```python file=/deploying/kubernetes/k8s_config_tag.py startafter=start_k8s_config_job endbefore=end_k8s_config_job
```python file=/deploying/kubernetes/k8s_config_tag_job.py startafter=start_k8s_config endbefore=end_k8s_config
@job(
tags={
"dagster-k8s/config": {
Expand Down Expand Up @@ -68,7 +68,7 @@ def my_job():

If your Dagster job is configured with an executor that runs each op in its own pod, like the <PyObject module="dagster_k8s" object="k8s_job_executor" /> or <PyObject module="dagster_celery_k8s" object="celery_k8s_job_executor" />, you can also use the `dagster-k8s/config` tag on a Dagster op to control the Kubernetes configuration for that specific op. For example:

```python file=/deploying/kubernetes/k8s_config_tag.py startafter=start_k8s_config_op endbefore=end_k8s_config_op
```python file=/deploying/kubernetes/k8s_config_tag_op.py startafter=start_k8s_config endbefore=end_k8s_config
@op(
tags={
"dagster-k8s/config": {
Expand All @@ -82,8 +82,14 @@ If your Dagster job is configured with an executor that runs each op in its own
)
def my_op(context):
context.log.info("running")

@job(executor_def=k8s_job_executor)
def my_job():
my_op()
```

Non-k8s run launchers and executors will ignore the `dagster-k8s/config` tag.

## Configuring an External Database

In a real deployment, users will likely want to set up an external PostgreSQL database and configure the `postgresql` section of `values.yaml`.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,23 +2,11 @@


# fmt: off
# start_k8s_config_op
@op(
tags={
"dagster-k8s/config": {
"container_config": {
"resources": {
"requests": {"cpu": "200m", "memory": "32Mi"},
}
},
}
}
)
@op
def my_op(context):
context.log.info("running")
# end_k8s_config_op

# start_k8s_config_job
# start_k8s_config
@job(
tags={
"dagster-k8s/config": {
Expand Down Expand Up @@ -59,5 +47,5 @@ def my_op(context):
)
def my_job():
my_op()
# end_k8s_config_job
# end_k8s_config
# fmt: on
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
from dagster_k8s import k8s_job_executor

from dagster import job, op


# fmt: off
# start_k8s_config
@op(
tags={
"dagster-k8s/config": {
"container_config": {
"resources": {
"requests": {"cpu": "200m", "memory": "32Mi"},
}
},
}
}
)
def my_op(context):
context.log.info("running")

@job(executor_def=k8s_job_executor)
def my_job():
my_op()
# end_k8s_config
# fmt: on
Original file line number Diff line number Diff line change
@@ -1,7 +1,8 @@
from dagster_k8s import DagsterK8sJobConfig, construct_dagster_k8s_job
from dagster_k8s.job import get_user_defined_k8s_config

from docs_snippets.deploying.kubernetes.k8s_config_tag import my_job, my_op
from docs_snippets.deploying.kubernetes.k8s_config_tag_job import my_job
from docs_snippets.deploying.kubernetes.k8s_config_tag_op import my_op


def test_k8s_tag_job():
Expand Down

1 comment on commit 8ad1195

@vercel
Copy link

@vercel vercel bot commented on 8ad1195 Mar 14, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please sign in to comment.