You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/08/17 12:03:47 UTC

[GitHub] [airflow] FloChehab opened a new issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

FloChehab opened a new issue #10362:
URL: https://github.com/apache/airflow/issues/10362


   **Apache Airflow version**: 1.10.12rc1
   
   **Kubernetes version (if you are using kubernetes)** (use `kubectl version`): v1.16.11-gke.5
   
   **Environment**:
   
   I am using the official docker image with the helm chart in this repo (with really minor tweaks). Config is pretty close to default.
   Docker image: `apache/airflow:v1-10-test-python3.8` (exact: `apache/airflow@sha256:53ac5c6f7df40c886cd1918f40d7802fd57e90f60cfaacc9c66f1f0f56890af4`)
   
   - **Cloud provider or hardware configuration**: /
   - **OS** (e.g. from /etc/os-release): /
   - **Kernel** (e.g. `uname -a`): /
   - **Install tools**: /
   - **Others**: /
   
   **What happened**:
   
   Using the KubernetesPodOperator with one or more secrets with mount type `volume` leads to incorrect pod configuration with duplicated volumes and `mountPath` ; and as a result the pod is not schedulable. 
   
   `HTTP response body: {"kind":"Status","apiVersion":"v1","metadata":{},"status":"Failure","message":"Pod \"airflow-test-pod-88c703e254004b26b008f9a7f63e1349\" is invalid: spec.containers[0].volumeMounts[1].mountPath: Invalid value: \"/var/location\": must be unique","reason":"Invalid","details":{"name":"airflow-test-pod-88c703e254004b26b008f9a7f63e1349","kind":"Pod","causes":[{"reason":"FieldValueInvalid","message":"Invalid value: \"/var/location\": must be unique","field":"spec.containers[0].volumeMounts[1].mountPath"}]},"code":422}`
   
   (full log is available below)
   
   **What you expected to happen**:
   
   To produce the correct pod configuration.
   
   **How to reproduce it**:
   
   Use this simple dag:
   
   ```python
   from airflow.contrib.operators.kubernetes_pod_operator import KubernetesPodOperator
   from airflow.kubernetes.secret import Secret
   from airflow.models import DAG
   from airflow.utils.dates import days_ago
   
   
   default_args = {
       'owner': 'Airflow',
       'start_date': days_ago(2)
   }
   
   with DAG(
       dag_id='bug_secret',
       default_args=default_args,
       schedule_interval=None
   ) as dag:
       k = KubernetesPodOperator(
           namespace='airflow',
           image="ubuntu:16.04",
           cmds=["bash", "-cx"],
           arguments=["echo", "10"],
           secrets=[Secret(
               deploy_type="volume",
               deploy_target="/var/location",
               secret="my-secret",
               key="content.json",
           )],
           name="airflow-test-pod",
           task_id="task",
           get_logs=True,
           is_delete_operator_pod=True,
       )
   ```
   
   
   **Anything else we need to know**:
   
   On the one hand, if I had to guess, I'd say that the issues comes from this PR https://github.com/apache/airflow/pull/10084 . On the other hand, this feature seems to be unittested, so I don't really understand what is going on.
   
   <details><summary>Task execution logs</summary>
   ```txt
   [2020-08-17 11:52:57,525] {taskinstance.py:901} INFO - Executing <Task(KubernetesPodOperator): task> on 2020-08-17T11:50:46.676804+00:00
   [2020-08-17 11:52:57,530] {standard_task_runner.py:54} INFO - Started process 3401 to run task
   [2020-08-17 11:52:57,578] {standard_task_runner.py:77} INFO - Running: ['airflow', 'run', 'bug_secret', 'task', '2020-08-17T11:50:46.676804+00:00', '--job_id', '3', '--pool', 'default_pool', '--raw', '-sd', 'DAGS_FOLDER/bug_secret.py', '--cfg_path', '/tmp/tmp8motpeb_']
   [2020-08-17 11:52:57,579] {standard_task_runner.py:78} INFO - Job 3: Subtask task
   [2020-08-17 11:52:57,639] {logging_mixin.py:112} INFO - Running %s on host %s <TaskInstance: bug_secret.task 2020-08-17T11:50:46.676804+00:00 [running]> airflow-scheduler-0.airflow-scheduler.dev-airflow-helm.svc.cluster.local
   [2020-08-17 11:52:57,789] {logging_mixin.py:112} WARNING - /home/airflow/.local/lib/python3.8/site-packages/airflow/kubernetes/pod_launcher.py:309: DeprecationWarning: Using `airflow.contrib.kubernetes.pod.Pod` is deprecated. Please use `k8s.V1Pod`.
     dummy_pod = Pod(
   [2020-08-17 11:52:57,789] {logging_mixin.py:112} WARNING - /home/airflow/.local/lib/python3.8/site-packages/airflow/kubernetes/pod_launcher.py:77: DeprecationWarning: Using `airflow.contrib.kubernetes.pod.Pod` is deprecated. Please use `k8s.V1Pod` instead.
     pod = self._mutate_pod_backcompat(pod)
   [2020-08-17 11:52:57,845] {pod_launcher.py:88} ERROR - Exception when attempting to create Namespaced Pod: {
     "apiVersion": "v1",
     "kind": "Pod",
     "metadata": {
       "labels": {
         "airflow_version": "1.10.12",
         "kubernetes_pod_operator": "True",
         "dag_id": "bug_secret",
         "task_id": "task",
         "execution_date": "2020-08-17T115046.6768040000-0d7e92426",
         "try_number": "2"
       },
       "name": "airflow-test-pod-151ddf3a60fc4cc4981d2390c4e4b4ce",
       "namespace": "airflow"
     },
     "spec": {
       "affinity": {},
       "containers": [
         {
           "args": [
             "echo",
             "10"
           ],
           "command": [
             "bash",
             "-cx"
           ],
           "env": [],
           "image": "ubuntu:16.04",
           "imagePullPolicy": "IfNotPresent",
           "name": "base",
           "resources": {
             "limits": {},
             "requests": {}
           },
           "volumeMounts": [
             {
               "mountPath": "/var/location",
               "name": "secretvold175d8b5-89b6-4ed4-957b-ff997a029463",
               "readOnly": true
             },
             {
               "mountPath": "/var/location",
               "name": "secretvole148d425-b284-410b-87b0-b0049fed1de5",
               "readOnly": true
             }
           ]
         }
       ],
       "hostNetwork": false,
       "imagePullSecrets": [],
       "initContainers": [],
       "restartPolicy": "Never",
       "securityContext": {},
       "serviceAccountName": "default",
       "tolerations": [],
       "volumes": [
         {
           "name": "secretvold175d8b5-89b6-4ed4-957b-ff997a029463",
           "secret": {
             "secretName": "my-secret"
           }
         },
         {
           "name": "secretvole148d425-b284-410b-87b0-b0049fed1de5",
           "secret": {
             "secretName": "secretvold175d8b5-89b6-4ed4-957b-ff997a029463"
           }
         }
       ]
     }
   }
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/kubernetes/pod_launcher.py", line 84, in run_pod_async
       resp = self._client.create_namespaced_pod(body=sanitized_pod,
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/api/core_v1_api.py", line 6174, in create_namespaced_pod
       (data) = self.create_namespaced_pod_with_http_info(namespace, body, **kwargs)  # noqa: E501
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/api/core_v1_api.py", line 6251, in create_namespaced_pod_with_http_info
       return self.api_client.call_api(
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/api_client.py", line 340, in call_api
       return self.__call_api(resource_path, method,
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/api_client.py", line 172, in __call_api
       response_data = self.request(
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/api_client.py", line 382, in request
       return self.rest_client.POST(url,
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/rest.py", line 272, in POST
       return self.request("POST", url,
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/rest.py", line 231, in request
       raise ApiException(http_resp=r)
   kubernetes.client.rest.ApiException: (422)
   Reason: Unprocessable Entity
   HTTP response headers: HTTPHeaderDict({'Audit-Id': 'fb7acb18-2ead-4c92-ac27-953133e57f83', 'Cache-Control': 'no-cache, private', 'Content-Type': 'application/json', 'Date': 'Mon, 17 Aug 2020 11:52:57 GMT', 'Content-Length': '518'})
   HTTP response body: {"kind":"Status","apiVersion":"v1","metadata":{},"status":"Failure","message":"Pod \"airflow-test-pod-151ddf3a60fc4cc4981d2390c4e4b4ce\" is invalid: spec.containers[0].volumeMounts[1].mountPath: Invalid value: \"/var/location\": must be unique","reason":"Invalid","details":{"name":"airflow-test-pod-151ddf3a60fc4cc4981d2390c4e4b4ce","kind":"Pod","causes":[{"reason":"FieldValueInvalid","message":"Invalid value: \"/var/location\": must be unique","field":"spec.containers[0].volumeMounts[1].mountPath"}]},"code":422}
   
   
   [2020-08-17 11:52:57,873] {taskinstance.py:1150} ERROR - (422)
   Reason: Unprocessable Entity
   HTTP response headers: HTTPHeaderDict({'Audit-Id': 'fb7acb18-2ead-4c92-ac27-953133e57f83', 'Cache-Control': 'no-cache, private', 'Content-Type': 'application/json', 'Date': 'Mon, 17 Aug 2020 11:52:57 GMT', 'Content-Length': '518'})
   HTTP response body: {"kind":"Status","apiVersion":"v1","metadata":{},"status":"Failure","message":"Pod \"airflow-test-pod-151ddf3a60fc4cc4981d2390c4e4b4ce\" is invalid: spec.containers[0].volumeMounts[1].mountPath: Invalid value: \"/var/location\": must be unique","reason":"Invalid","details":{"name":"airflow-test-pod-151ddf3a60fc4cc4981d2390c4e4b4ce","kind":"Pod","causes":[{"reason":"FieldValueInvalid","message":"Invalid value: \"/var/location\": must be unique","field":"spec.containers[0].volumeMounts[1].mountPath"}]},"code":422}
   
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/models/taskinstance.py", line 984, in _run_raw_task
       result = task_copy.execute(context=context)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/contrib/operators/kubernetes_pod_operator.py", line 284, in execute
       final_state, _, result = self.create_new_pod_for_operator(labels, launcher)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/contrib/operators/kubernetes_pod_operator.py", line 400, in create_new_pod_for_operator
       launcher.start_pod(
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/kubernetes/pod_launcher.py", line 135, in start_pod
       resp = self.run_pod_async(pod)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/kubernetes/pod_launcher.py", line 90, in run_pod_async
       raise e
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/kubernetes/pod_launcher.py", line 84, in run_pod_async
       resp = self._client.create_namespaced_pod(body=sanitized_pod,
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/api/core_v1_api.py", line 6174, in create_namespaced_pod
       (data) = self.create_namespaced_pod_with_http_info(namespace, body, **kwargs)  # noqa: E501
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/api/core_v1_api.py", line 6251, in create_namespaced_pod_with_http_info
       return self.api_client.call_api(
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/api_client.py", line 340, in call_api
       return self.__call_api(resource_path, method,
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/api_client.py", line 172, in __call_api
       response_data = self.request(
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/api_client.py", line 382, in request
       return self.rest_client.POST(url,
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/rest.py", line 272, in POST
       return self.request("POST", url,
     File "/home/airflow/.local/lib/python3.8/site-packages/kubernetes/client/rest.py", line 231, in request
       raise ApiException(http_resp=r)
   kubernetes.client.rest.ApiException: (422)
   Reason: Unprocessable Entity
   HTTP response headers: HTTPHeaderDict({'Audit-Id': 'fb7acb18-2ead-4c92-ac27-953133e57f83', 'Cache-Control': 'no-cache, private', 'Content-Type': 'application/json', 'Date': 'Mon, 17 Aug 2020 11:52:57 GMT', 'Content-Length': '518'})
   HTTP response body: {"kind":"Status","apiVersion":"v1","metadata":{},"status":"Failure","message":"Pod \"airflow-test-pod-151ddf3a60fc4cc4981d2390c4e4b4ce\" is invalid: spec.containers[0].volumeMounts[1].mountPath: Invalid value: \"/var/location\": must be unique","reason":"Invalid","details":{"name":"airflow-test-pod-151ddf3a60fc4cc4981d2390c4e4b4ce","kind":"Pod","causes":[{"reason":"FieldValueInvalid","message":"Invalid value: \"/var/location\": must be unique","field":"spec.containers[0].volumeMounts[1].mountPath"}]},"code":422}
   ```
   </details>
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] boring-cyborg[bot] commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-674840843


   Thanks for opening your first issue here! Be sure to follow the issue template!
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil edited a comment on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
kaxil edited a comment on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-675132778


   Unfortunately, the PR hasn't passed the tests, so we will have to wait for tomorrow to cut rc2. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FloChehab edited a comment on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
FloChehab edited a comment on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-674904511


   I am trying to pin point exactly where this mutation happens in my case, I'll come back once I have more info to share.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FloChehab commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
FloChehab commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-675561848


   > @FloChehab Thank you for testing our RC! :D
   
   You are welcome ! 
   
   I was wondering if there is any mailing lists with RC announcements ? So that I can keep inform and test things "on time". Or maybe, I should join slack ? 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-675009686


   Awesome, we will create 1.10.12rc2 in an hour


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-674890085


   hmm there are duplicate "MountPaths", do you specify this VolumeMounts anywhere else like in pod_mutation_hook ?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dimberman commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
dimberman commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-674985991


   @FloChehab thank you for bringing this to our attention. I've pushed a fix here https://github.com/apache/airflow/commit/1ad6ec43439cb1781176e567d3c7fab813404f1f


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FloChehab commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
FloChehab commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-674894783


   Hi @kaxil, I am running things with default everything basically (but with the chart from the master branch if this can have impact).
   I've tried to reproduce the bug following the testing on kubernetes guide but I can't do it. It seems to work.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FloChehab commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
FloChehab commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-674926967


   I've removed all calls to `settings.pod_mutation_hook` and I still get the error. 
   Also I am getting a bunch of ```DeprecationWarning: Using `airflow.contrib.kubernetes.pod.Pod` is deprecated. Please use `k8s.V1Pod`.```: I am not sure if this is suppose to happen since I am not doing anything fancy in the dag above.
   
   Finally, the mutation happens either on line 97 https://github.com/apache/airflow/blob/v1-10-test/airflow/kubernetes/pod_launcher.py#L97 or line 103. I am not really familiar with debugging internal airflow code and my iteration setup is not ideal. Can you confirm the bug ?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FloChehab commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
FloChehab commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-674931078


   I am using the chart as is, with the following "custom" values:
   
   ```yaml
   ---
   ###################
   # Sub charts config
   postgresql:
     persistence:
       enabled: true
   
   redis:
     persistence:
       enabled: false
   
   ########################
   # Airflow chart config
   images:
     airflow:
       repository: image
       tag: some tag
       pullPolicy: Always
   
   workers:
     persistence:
       enabled: true
       size: 8Gi
   
   # doesn't play nicely with rbac
   webserver:
     defaultUser:
       enabled: false
   
   env:
     - name: "AIRFLOW__WEBSERVER__RBAC"
       value: "False"
     - name: "AIRFLOW__API__AUTH_BACKEND"
       value: "airflow.api.auth.backend.default"
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FloChehab commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
FloChehab commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-675571475


   Done, thanks for the info !


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FloChehab commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
FloChehab commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-675008416


   The fix has solved this particular issue @dimberman ! Thanks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-674928911


   The following is just warning not an actual error so should be fine.
   ```
   DeprecationWarning: Using `airflow.contrib.kubernetes.pod.Pod` is deprecated. Please use `k8s.V1Pod`. 
   ```
   
   Do you use the chart as it is? i.e do you alter Mutation settings or anything else?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil edited a comment on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
kaxil edited a comment on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-675132778


   Unfortunately, the PR hasn't passed the tests, so we will have to wait for tomorrow to cur rc2. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dimberman commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
dimberman commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-675558588


   @FloChehab Thank you for testing our RC! :D


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FloChehab commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
FloChehab commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-674987715


   Thanks @dimberman , I'll build an image with this fix and I'll test it asap.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FloChehab commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
FloChehab commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-674904511


   I am trying to pin point exactly where this mutation happen in my case, I'll come back once I have more info to share.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dimberman closed issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
dimberman closed issue #10362:
URL: https://github.com/apache/airflow/issues/10362


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] FloChehab commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
FloChehab commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-675557523


   Thanks a lot !


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-675132778


   Unfortunately, the PR hasn't passed the tests, so we will have to wait for tomorrow. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-675565185


   Yes for sure, Check http://airflow.apache.org/community/
   
   Send an email to dev-subscribe@airflow.apache.org to subscribe to Dev mailing list where this is announced


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-674932227






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dimberman commented on issue #10362: Airflow 1.10.12rc1, KubernetesPodOperator broken with secret & deploy_type "volume"

Posted by GitBox <gi...@apache.org>.
dimberman commented on issue #10362:
URL: https://github.com/apache/airflow/issues/10362#issuecomment-675553704


   Issue should be fixed for RC2 with this PR ^


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org