Skip to content

Commit

Permalink
Chart: Adds support for custom command and args (#16153)
Browse files Browse the repository at this point in the history
Some images may not want to use the same command/args as the community
image, so expose them as parameters.
  • Loading branch information
jedcunningham committed May 29, 2021
1 parent 595374a commit e7d3fa2
Show file tree
Hide file tree
Showing 10 changed files with 492 additions and 35 deletions.
9 changes: 5 additions & 4 deletions chart/templates/flower/flower-deployment.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -77,10 +77,11 @@ spec:
- name: flower
image: {{ template "flower_image" . }}
imagePullPolicy: {{ .Values.images.flower.pullPolicy }}
{{- if semverCompare ">=2.0.0" .Values.airflowVersion }}
args: ["bash", "-c", "airflow celery flower"]
{{- else }}
args: ["bash", "-c", "airflow flower"]
{{- if .Values.flower.command }}
command: {{ tpl (toYaml .Values.flower.command) . | nindent 12 }}
{{- end }}
{{- if .Values.flower.args }}
args: {{ tpl (toYaml .Values.flower.args) . | nindent 12 }}
{{- end }}
resources:
{{ toYaml .Values.flower.resources | indent 12 }}
Expand Down
19 changes: 15 additions & 4 deletions chart/templates/scheduler/scheduler-deployment.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,12 @@ spec:
- name: scheduler
image: {{ template "airflow_image" . }}
imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
args: ["bash", "-c", "exec airflow scheduler"]
{{- if .Values.scheduler.command }}
command: {{ tpl (toYaml .Values.scheduler.command) . | nindent 12 }}
{{- end }}
{{- if .Values.scheduler.args }}
args: {{ tpl (toYaml .Values.scheduler.args) . | nindent 12 }}
{{- end }}
envFrom:
{{- include "custom_airflow_environment_from" . | default "\n []" | indent 10 }}
env:
Expand Down Expand Up @@ -188,14 +193,20 @@ spec:
{{- if .Values.dags.gitSync.enabled }}
{{- include "git_sync_container" . | indent 8 }}
{{- end }}
# Always start the garbage collector sidecar.
- name: scheduler-gc
{{- if .Values.scheduler.logGroomerSidecar.enabled }}
- name: scheduler-log-groomer
image: {{ template "airflow_image" . }}
imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
args: ["bash", "/clean-logs"]
{{- if .Values.scheduler.logGroomerSidecar.command }}
command: {{ tpl (toYaml .Values.scheduler.logGroomerSidecar.command) . | nindent 12 }}
{{- end }}
{{- if .Values.scheduler.logGroomerSidecar.args }}
args: {{ tpl (toYaml .Values.scheduler.logGroomerSidecar.args) . | nindent 12 }}
{{- end }}
volumeMounts:
- name: logs
mountPath: {{ template "airflow_logs" . }}
{{- end }}
{{- if .Values.scheduler.extraContainers }}
{{- toYaml .Values.scheduler.extraContainers | nindent 8 }}
{{- end }}
Expand Down
7 changes: 6 additions & 1 deletion chart/templates/webserver/webserver-deployment.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,12 @@ spec:
- name: webserver
image: {{ template "airflow_image" . }}
imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
args: ["bash", "-c", "exec airflow webserver"]
{{- if .Values.webserver.command }}
command: {{ tpl (toYaml .Values.webserver.command) . | nindent 12 }}
{{- end }}
{{- if .Values.webserver.args }}
args: {{ tpl (toYaml .Values.webserver.args) . | nindent 12 }}
{{- end }}
resources:
{{ toYaml .Values.webserver.resources | indent 12 }}
volumeMounts:
Expand Down
18 changes: 12 additions & 6 deletions chart/templates/workers/worker-deployment.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -130,10 +130,11 @@ spec:
- name: worker
image: {{ template "airflow_image" . }}
imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
{{- if semverCompare ">=2.0.0" .Values.airflowVersion }}
args: [ "bash", "-c", "airflow celery worker" ]
{{- else }}
args: [ "bash", "-c", "airflow worker" ]
{{- if .Values.workers.command }}
command: {{ tpl (toYaml .Values.workers.command) . | nindent 12 }}
{{- end }}
{{- if .Values.workers.args }}
args: {{ tpl (toYaml .Values.workers.args) . | nindent 12 }}
{{- end }}
resources:
{{ toYaml .Values.workers.resources | indent 12 }}
Expand Down Expand Up @@ -183,10 +184,15 @@ spec:
{{- include "git_sync_container" . | indent 8 }}
{{- end }}
{{- if $persistence }}
- name: worker-gc
- name: worker-log-groomer
image: {{ template "airflow_image" . }}
imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
args: ["bash", "/clean-logs"]
{{- if .Values.workers.logGroomerSidecar.command }}
command: {{ tpl (toYaml .Values.workers.logGroomerSidecar.command) . | nindent 12 }}
{{- end }}
{{- if .Values.workers.logGroomerSidecar.args }}
args: {{ tpl (toYaml .Values.workers.logGroomerSidecar.args) . | nindent 12 }}
{{- end }}
volumeMounts:
- name: logs
mountPath: {{ template "airflow_logs" . }}
Expand Down
49 changes: 32 additions & 17 deletions chart/tests/test_flower.py
Original file line number Diff line number Diff line change
Expand Up @@ -47,22 +47,10 @@ def test_create_flower(self, executor, flower_enabled, created):
@pytest.mark.parametrize(
"airflow_version, expected_arg",
[
(
"2.0.2",
"airflow celery flower",
),
(
"1.10.14",
"airflow flower",
),
(
"1.9.0",
"airflow flower",
),
(
"2.1.0",
"airflow celery flower",
),
("2.0.2", "airflow celery flower"),
("1.10.14", "airflow flower"),
("1.9.0", "airflow flower"),
("2.1.0", "airflow celery flower"),
],
)
def test_args_with_airflow_version(self, airflow_version, expected_arg):
Expand All @@ -78,9 +66,36 @@ def test_args_with_airflow_version(self, airflow_version, expected_arg):
assert jmespath.search("spec.template.spec.containers[0].args", docs[0]) == [
"bash",
"-c",
expected_arg,
f"exec \\\n{expected_arg}",
]

@pytest.mark.parametrize(
"command, args",
[
(None, None),
(None, ["custom", "args"]),
(["custom", "command"], None),
(["custom", "command"], ["custom", "args"]),
],
)
def test_command_and_args_overrides(self, command, args):
docs = render_chart(
values={"flower": {"command": command, "args": args}},
show_only=["templates/flower/flower-deployment.yaml"],
)

assert command == jmespath.search("spec.template.spec.containers[0].command", docs[0])
assert args == jmespath.search("spec.template.spec.containers[0].args", docs[0])

def test_command_and_args_overrides_are_templated(self):
docs = render_chart(
values={"flower": {"command": ["{{ .Release.Name }}"], "args": ["{{ .Release.Service }}"]}},
show_only=["templates/flower/flower-deployment.yaml"],
)

assert ["RELEASE-NAME"] == jmespath.search("spec.template.spec.containers[0].command", docs[0])
assert ["Helm"] == jmespath.search("spec.template.spec.containers[0].args", docs[0])

def test_should_create_flower_deployment_with_authorization(self):
docs = render_chart(
values={
Expand Down
80 changes: 80 additions & 0 deletions chart/tests/test_scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -284,3 +284,83 @@ def test_scheduler_strategy(self, executor, persistence, strategy, expected_stra
)

assert expected_strategy == jmespath.search("spec.strategy", docs[0])

def test_default_command_and_args(self):
docs = render_chart(show_only=["templates/scheduler/scheduler-deployment.yaml"])

assert jmespath.search("spec.template.spec.containers[0].command", docs[0]) is None
assert ["bash", "-c", "exec airflow scheduler"] == jmespath.search(
"spec.template.spec.containers[0].args", docs[0]
)

@parameterized.expand(
[
(None, None),
(None, ["custom", "args"]),
(["custom", "command"], None),
(["custom", "command"], ["custom", "args"]),
]
)
def test_command_and_args_overrides(self, command, args):
docs = render_chart(
values={"scheduler": {"command": command, "args": args}},
show_only=["templates/scheduler/scheduler-deployment.yaml"],
)

assert command == jmespath.search("spec.template.spec.containers[0].command", docs[0])
assert args == jmespath.search("spec.template.spec.containers[0].args", docs[0])

def test_command_and_args_overrides_are_templated(self):
docs = render_chart(
values={"scheduler": {"command": ["{{ .Release.Name }}"], "args": ["{{ .Release.Service }}"]}},
show_only=["templates/scheduler/scheduler-deployment.yaml"],
)

assert ["RELEASE-NAME"] == jmespath.search("spec.template.spec.containers[0].command", docs[0])
assert ["Helm"] == jmespath.search("spec.template.spec.containers[0].args", docs[0])

def test_log_groomer_collector_can_be_disabled(self):
docs = render_chart(
values={"scheduler": {"logGroomerSidecar": {"enabled": False}}},
show_only=["templates/scheduler/scheduler-deployment.yaml"],
)
assert 1 == len(jmespath.search("spec.template.spec.containers", docs[0]))

def test_log_groomer_collector_default_command_and_args(self):
docs = render_chart(show_only=["templates/scheduler/scheduler-deployment.yaml"])

assert jmespath.search("spec.template.spec.containers[1].command", docs[0]) is None
assert ["bash", "/clean-logs"] == jmespath.search("spec.template.spec.containers[1].args", docs[0])

@parameterized.expand(
[
(None, None),
(None, ["custom", "args"]),
(["custom", "command"], None),
(["custom", "command"], ["custom", "args"]),
]
)
def test_log_groomer_command_and_args_overrides(self, command, args):
docs = render_chart(
values={"scheduler": {"logGroomerSidecar": {"command": command, "args": args}}},
show_only=["templates/scheduler/scheduler-deployment.yaml"],
)

assert command == jmespath.search("spec.template.spec.containers[1].command", docs[0])
assert args == jmespath.search("spec.template.spec.containers[1].args", docs[0])

def test_log_groomer_command_and_args_overrides_are_templated(self):
docs = render_chart(
values={
"scheduler": {
"logGroomerSidecar": {
"command": ["{{ .Release.Name }}"],
"args": ["{{ .Release.Service }}"],
}
}
},
show_only=["templates/scheduler/scheduler-deployment.yaml"],
)

assert ["RELEASE-NAME"] == jmespath.search("spec.template.spec.containers[1].command", docs[0])
assert ["Helm"] == jmespath.search("spec.template.spec.containers[1].args", docs[0])
34 changes: 34 additions & 0 deletions chart/tests/test_webserver.py
Original file line number Diff line number Diff line change
Expand Up @@ -292,6 +292,40 @@ def test_airflow_local_settings(self):
"readOnly": True,
} in jmespath.search("spec.template.spec.containers[0].volumeMounts", docs[0])

def test_default_command_and_args(self):
docs = render_chart(show_only=["templates/webserver/webserver-deployment.yaml"])

assert jmespath.search("spec.template.spec.containers[0].command", docs[0]) is None
assert ["bash", "-c", "exec airflow webserver"] == jmespath.search(
"spec.template.spec.containers[0].args", docs[0]
)

@parameterized.expand(
[
(None, None),
(None, ["custom", "args"]),
(["custom", "command"], None),
(["custom", "command"], ["custom", "args"]),
]
)
def test_command_and_args_overrides(self, command, args):
docs = render_chart(
values={"webserver": {"command": command, "args": args}},
show_only=["templates/webserver/webserver-deployment.yaml"],
)

assert command == jmespath.search("spec.template.spec.containers[0].command", docs[0])
assert args == jmespath.search("spec.template.spec.containers[0].args", docs[0])

def test_command_and_args_overrides_are_templated(self):
docs = render_chart(
values={"webserver": {"command": ["{{ .Release.Name }}"], "args": ["{{ .Release.Service }}"]}},
show_only=["templates/webserver/webserver-deployment.yaml"],
)

assert ["RELEASE-NAME"] == jmespath.search("spec.template.spec.containers[0].command", docs[0])
assert ["Helm"] == jmespath.search("spec.template.spec.containers[0].args", docs[0])


class WebserverServiceTest(unittest.TestCase):
def test_default_service(self):
Expand Down
88 changes: 88 additions & 0 deletions chart/tests/test_worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -279,3 +279,91 @@ def test_airflow_local_settings_kerberos_sidecar(self):
"subPath": "airflow_local_settings.py",
"readOnly": True,
} in jmespath.search("spec.template.spec.containers[2].volumeMounts", docs[0])

@parameterized.expand(
[
("1.9.0", "airflow worker"),
("1.10.14", "airflow worker"),
("2.0.2", "airflow celery worker"),
("2.1.0", "airflow celery worker"),
],
)
def test_default_command_and_args_airflow_version(self, airflow_version, expected_arg):
docs = render_chart(
values={
"airflowVersion": airflow_version,
},
show_only=["templates/workers/worker-deployment.yaml"],
)

assert jmespath.search("spec.template.spec.containers[0].command", docs[0]) is None
assert [
"bash",
"-c",
f"exec \\\n{expected_arg}",
] == jmespath.search("spec.template.spec.containers[0].args", docs[0])

@parameterized.expand(
[
(None, None),
(None, ["custom", "args"]),
(["custom", "command"], None),
(["custom", "command"], ["custom", "args"]),
]
)
def test_command_and_args_overrides(self, command, args):
docs = render_chart(
values={"workers": {"command": command, "args": args}},
show_only=["templates/workers/worker-deployment.yaml"],
)

assert command == jmespath.search("spec.template.spec.containers[0].command", docs[0])
assert args == jmespath.search("spec.template.spec.containers[0].args", docs[0])

def test_command_and_args_overrides_are_templated(self):
docs = render_chart(
values={"workers": {"command": ["{{ .Release.Name }}"], "args": ["{{ .Release.Service }}"]}},
show_only=["templates/workers/worker-deployment.yaml"],
)

assert ["RELEASE-NAME"] == jmespath.search("spec.template.spec.containers[0].command", docs[0])
assert ["Helm"] == jmespath.search("spec.template.spec.containers[0].args", docs[0])

def test_log_groomer_default_command_and_args(self):
docs = render_chart(show_only=["templates/workers/worker-deployment.yaml"])

assert jmespath.search("spec.template.spec.containers[1].command", docs[0]) is None
assert ["bash", "/clean-logs"] == jmespath.search("spec.template.spec.containers[1].args", docs[0])

@parameterized.expand(
[
(None, None),
(None, ["custom", "args"]),
(["custom", "command"], None),
(["custom", "command"], ["custom", "args"]),
]
)
def test_log_groomer_command_and_args_overrides(self, command, args):
docs = render_chart(
values={"workers": {"logGroomerSidecar": {"command": command, "args": args}}},
show_only=["templates/workers/worker-deployment.yaml"],
)

assert command == jmespath.search("spec.template.spec.containers[1].command", docs[0])
assert args == jmespath.search("spec.template.spec.containers[1].args", docs[0])

def test_log_groomer_command_and_args_overrides_are_templated(self):
docs = render_chart(
values={
"workers": {
"logGroomerSidecar": {
"command": ["{{ .Release.Name }}"],
"args": ["{{ .Release.Service }}"],
}
}
},
show_only=["templates/workers/worker-deployment.yaml"],
)

assert ["RELEASE-NAME"] == jmespath.search("spec.template.spec.containers[1].command", docs[0])
assert ["Helm"] == jmespath.search("spec.template.spec.containers[1].args", docs[0])

0 comments on commit e7d3fa2

Please sign in to comment.