[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-03-02 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r386495451
 
 

 ##
 File path: airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py
 ##
 @@ -0,0 +1,88 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Optional
+
+import yaml
+from kubernetes import client
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.cncf.kubernetes.hooks.kubernetes import KubernetesHook
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+Creates sparkApplication object in kubernetes cluster:
+
+   .. seealso::
+For more detail about Spark Application Object have a look at the 
reference:
+
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
+
+:param application_file: filepath to kubernetes custom_resource_definition 
of sparkApplication
+:type application_file:  str
+:param namespace: kubernetes namespace to put sparkApplication
+:type namespace: str
+:param kubernetes_conn_id: the connection to Kubernetes cluster
+:type conn_id: str
+"""
+
+template_fields = ['application_file', 'namespace']
+template_ext = ('yaml', 'yml', 'json')
+ui_color = '#f4a460'
+
+@apply_defaults
+def __init__(self,
+ application_file: str,
+ namespace: Optional[str] = None,
+ conn_id: str = 'kubernetes_default',
+ *args, **kwargs) -> None:
+super().__init__(*args, **kwargs)
+self.application_file = application_file
+self.namespace = namespace
+self.conn_id = conn_id
+
+def execute(self, context):
+self.log.info("Creating sparkApplication")
+hook = KubernetesHook(conn_id=self.conn_id)
+api_client = hook.get_conn()
+api = client.CustomObjectsApi(api_client)
+application_dict = self._load_application_to_dict()
+if self.namespace is None:
+namespace = hook.get_namespace()
+else:
+namespace = self.namespace
+try:
+response = api.create_namespaced_custom_object(
 
 Review comment:
   This logic should be in the hook so that it can be used again by other 
custom operators. Then we can add more methods that will start operators and 
wait for its completion or other depending on the situation.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-03-02 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r386491368
 
 

 ##
 File path: 
airflow/providers/cncf/kubernetes/example_dags/example_spark_kubernetes_operator_spark_pi.yaml
 ##
 @@ -0,0 +1,57 @@
+#
 
 Review comment:
   Can you add this file to MANIFEST.in? 
   https://github.com/apache/airflow/blob/master/MANIFEST.in


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-02-19 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r381607848
 
 

 ##
 File path: tests/test_project_structure.py
 ##
 @@ -36,6 +36,9 @@
 'tests/providers/apache/pig/operators/test_pig.py',
 'tests/providers/apache/spark/hooks/test_spark_jdbc_script.py',
 'tests/providers/cncf/kubernetes/operators/test_kubernetes_pod.py',
+
'tests/providers/cncf/kubernetes/operators/test_spark_kubernetes_operator.py',
+'tests/providers/cncf/kubernetes/hooks/test_kubernetes_hook.py',
+'tests/providers/cncf/kubernetes/sensors/test_spark_kubernetes_sensor.py',
 
 Review comment:
   Why is this not possible? I think that shouldn't be a problem.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371041517
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,72 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+Creates sparkApplication object in kubernetes cluster:
+
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
+
+:param sparkapplication_object: kubernetes custom_resource_definition of 
sparkApplication
+:type sparkapplication_object:  dict
+:param namespace: kubernetes namespace to put sparkApplication
+:type namespace: str
+:param conn_id: the connection to Kubernetes cluster
+:type conn_id: str
+"""
+
+template_fields = ['sparkapplication_object', 'namespace']
+template_ext = ()
+ui_color = '#f4a460'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_object: dict,
+ namespace: str = 'default',
+ conn_id: str = 'kubernetes_default',
+ *args, **kwargs) -> None:
+super().__init__(*args, **kwargs)
+self.sparkapplication_object = sparkapplication_object
+self.namespace = namespace
+self.conn_id = conn_id
+
+def execute(self, context):
+self.log.info("creating sparkApplication")
+hook = Kuberneteshook(conn_id=self.conn_id)
+api_client = hook.get_conn()
+api = client.CustomObjectsApi(api_client)
+try:
+response = api.create_namespaced_custom_object(
+group="sparkoperator.k8s.io",
+version="v1beta2",
+namespace=self.namespace,
+plural="sparkapplications",
+body=self.sparkapplication_object
+)
+self.log.info(response)
 
 Review comment:
   This is especially important when we have log collection engines that index 
messages, but we don't necessarily want to index data. The number of messages 
is limited and it is much more effective.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371041416
 
 

 ##
 File path: airflow/contrib/sensors/spark_kubernetes_sensor.py
 ##
 @@ -0,0 +1,80 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Dict
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesSensor(BaseSensorOperator):
+"""
+checks sparkApplication object in kubernetes cluster:
+
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
+:param sparkapplication_name: sparkapplication resource name
+:type str
+:param namespace: the kubernetes namespace where the sparkApplication 
reside in
+:type str
+:param conn_id: the connection to Kubernetes cluster
+:type str
+"""
+
+template_fields = ('sparkapplication_name', 'namespace')
+INTERMEDIATE_STATES = ('SUBMITTED', 'RUNNING',)
+FAILURE_STATES = ('FAILED', 'SUBMISSION_FAILED', 'UNKNOWN')
+SUCCESS_STATES = 'COMPLETED'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_name: str,
+ namespace: str = 'default',
+ conn_id: str = 'kubernetes_default',
+ *args, **kwargs):
+super().__init__(*args, **kwargs)
+self.sparkapplication_name = sparkapplication_name
+self.namespace = namespace
+self.conn_id = conn_id
+
+def poke(self, context: Dict):
+self.log.info("Poking: %s", self.sparkapplication_name)
+api_client = Kuberneteshook(conn_id=self.conn_id).get_conn()
+custom_resource_definition_api = client.CustomObjectsApi(api_client)
+try:
+response = 
custom_resource_definition_api.get_namespaced_custom_object(
+group="sparkoperator.k8s.io",
+version="v1beta2",
+namespace=self.namespace,
+plural="sparkapplications",
+name=self.sparkapplication_name
+)
+sparkapplication_state = 
response['status']['applicationState']['state']
+if sparkapplication_state in self.FAILURE_STATES:
+raise AirflowException("spark application failed with state: 
%s" % sparkapplication_state)
+if sparkapplication_state in self.INTERMEDIATE_STATES:
+self.log.info("spark application is still in state: %s", 
sparkapplication_state)
 
 Review comment:
   ```suggestion
   self.log.info("Spark application is still in state: %s", 
sparkapplication_state)
   ```


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371041407
 
 

 ##
 File path: airflow/contrib/sensors/spark_kubernetes_sensor.py
 ##
 @@ -0,0 +1,80 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Dict
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesSensor(BaseSensorOperator):
+"""
+checks sparkApplication object in kubernetes cluster:
+
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
+:param sparkapplication_name: sparkapplication resource name
+:type str
+:param namespace: the kubernetes namespace where the sparkApplication 
reside in
+:type str
+:param conn_id: the connection to Kubernetes cluster
+:type str
+"""
+
+template_fields = ('sparkapplication_name', 'namespace')
+INTERMEDIATE_STATES = ('SUBMITTED', 'RUNNING',)
+FAILURE_STATES = ('FAILED', 'SUBMISSION_FAILED', 'UNKNOWN')
+SUCCESS_STATES = 'COMPLETED'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_name: str,
+ namespace: str = 'default',
+ conn_id: str = 'kubernetes_default',
+ *args, **kwargs):
+super().__init__(*args, **kwargs)
+self.sparkapplication_name = sparkapplication_name
+self.namespace = namespace
+self.conn_id = conn_id
+
+def poke(self, context: Dict):
+self.log.info("Poking: %s", self.sparkapplication_name)
+api_client = Kuberneteshook(conn_id=self.conn_id).get_conn()
+custom_resource_definition_api = client.CustomObjectsApi(api_client)
+try:
+response = 
custom_resource_definition_api.get_namespaced_custom_object(
+group="sparkoperator.k8s.io",
+version="v1beta2",
+namespace=self.namespace,
+plural="sparkapplications",
+name=self.sparkapplication_name
+)
+sparkapplication_state = 
response['status']['applicationState']['state']
+if sparkapplication_state in self.FAILURE_STATES:
+raise AirflowException("spark application failed with state: 
%s" % sparkapplication_state)
+if sparkapplication_state in self.INTERMEDIATE_STATES:
+self.log.info("spark application is still in state: %s", 
sparkapplication_state)
+return False
+if sparkapplication_state in self.SUCCESS_STATES:
+self.log.info("spark application ended successfully")
+return True
+raise AirflowException("unknown spark application state: %s" % 
sparkapplication_state)
 
 Review comment:
   ```suggestion
   raise AirflowException("Unknown spark application state: %s" % 
sparkapplication_state)
   ```


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371041376
 
 

 ##
 File path: airflow/contrib/sensors/spark_kubernetes_sensor.py
 ##
 @@ -0,0 +1,80 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Dict
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesSensor(BaseSensorOperator):
+"""
+checks sparkApplication object in kubernetes cluster:
+
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
 
 Review comment:
   ```suggestion
   .. seealso::
   For more detail about Spark Application Object have a look at the 
reference:
   
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
   ```


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371041394
 
 

 ##
 File path: airflow/contrib/sensors/spark_kubernetes_sensor.py
 ##
 @@ -0,0 +1,80 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Dict
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesSensor(BaseSensorOperator):
+"""
+checks sparkApplication object in kubernetes cluster:
+
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
+:param sparkapplication_name: sparkapplication resource name
+:type str
 
 Review comment:
   ```suggestion
   :type sparkapplication_name: str
   ```
   This is needed for the documentation to be built successfully.
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371041345
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,72 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+Creates sparkApplication object in kubernetes cluster:
+
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
+
+:param sparkapplication_object: kubernetes custom_resource_definition of 
sparkApplication
+:type sparkapplication_object:  dict
+:param namespace: kubernetes namespace to put sparkApplication
+:type namespace: str
+:param conn_id: the connection to Kubernetes cluster
+:type conn_id: str
+"""
+
+template_fields = ['sparkapplication_object', 'namespace']
+template_ext = ()
+ui_color = '#f4a460'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_object: dict,
+ namespace: str = 'default',
+ conn_id: str = 'kubernetes_default',
+ *args, **kwargs) -> None:
+super().__init__(*args, **kwargs)
+self.sparkapplication_object = sparkapplication_object
+self.namespace = namespace
+self.conn_id = conn_id
+
+def execute(self, context):
+self.log.info("creating sparkApplication")
+hook = Kuberneteshook(conn_id=self.conn_id)
+api_client = hook.get_conn()
+api = client.CustomObjectsApi(api_client)
+try:
+response = api.create_namespaced_custom_object(
+group="sparkoperator.k8s.io",
+version="v1beta2",
+namespace=self.namespace,
+plural="sparkapplications",
+body=self.sparkapplication_object
+)
+self.log.info(response)
 
 Review comment:
   ```suggestion
   self.log.debug("Response: %s", response)
   ```
   This is too much information for this login level. Data should also be 
provided as arguments, not as messages. The first argument specifies the 
message. The remaining arguments specify the arguments of the message.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371041185
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,72 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+Creates sparkApplication object in kubernetes cluster:
+
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
 
 Review comment:
   ```suggestion
   .. seealso::
   For more detail about Spark Application Object have a look at the 
reference:
   
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
   ```


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371041215
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,72 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+Creates sparkApplication object in kubernetes cluster:
+
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
+
+:param sparkapplication_object: kubernetes custom_resource_definition of 
sparkApplication
+:type sparkapplication_object:  dict
+:param namespace: kubernetes namespace to put sparkApplication
+:type namespace: str
+:param conn_id: the connection to Kubernetes cluster
+:type conn_id: str
+"""
+
+template_fields = ['sparkapplication_object', 'namespace']
+template_ext = ()
+ui_color = '#f4a460'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_object: dict,
+ namespace: str = 'default',
+ conn_id: str = 'kubernetes_default',
+ *args, **kwargs) -> None:
+super().__init__(*args, **kwargs)
+self.sparkapplication_object = sparkapplication_object
+self.namespace = namespace
+self.conn_id = conn_id
+
+def execute(self, context):
+self.log.info("creating sparkApplication")
 
 Review comment:
   ```suggestion
   self.log.info("Creating sparkApplication")
   ```


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371041185
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,72 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+Creates sparkApplication object in kubernetes cluster:
+
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
 
 Review comment:
   ```suggestion
   .. seealso::
   For more detail about  Spark Application Object have a look at the 
reference:
   
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
   ```


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371041066
 
 

 ##
 File path: airflow/contrib/hooks/kubernetes_hook.py
 ##
 @@ -0,0 +1,53 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import tempfile
+
+from kubernetes import client, config
+
+from airflow.hooks.base_hook import BaseHook
+
+
+class Kuberneteshook(BaseHook):
+"""
+creates kubernetes API connection.
 
 Review comment:
   ```suggestion
   Creates Kubernetes API connection.
   ```


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371041053
 
 

 ##
 File path: airflow/contrib/hooks/kubernetes_hook.py
 ##
 @@ -0,0 +1,53 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import tempfile
+
+from kubernetes import client, config
+
+from airflow.hooks.base_hook import BaseHook
+
+
+class Kuberneteshook(BaseHook):
+"""
+creates kubernetes API connection.
+:param conn_id: the connection to Kubernetes cluster
 
 Review comment:
   ```suggestion
   
   :param conn_id: the connection to Kubernetes cluster
   ```
   This is needed for the documentation to be built successfully.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371040906
 
 

 ##
 File path: airflow/contrib/hooks/kubernetes_hook.py
 ##
 @@ -0,0 +1,53 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import tempfile
+
+from kubernetes import client, config
+
+from airflow.hooks.base_hook import BaseHook
+
+
+class Kuberneteshook(BaseHook):
+"""
+creates kubernetes API connection.
+:param conn_id: the connection to Kubernetes cluster
+"""
+
+def __init__(
+self,
+conn_id="kubernetes_default"
+):
+self.connection = self.get_connection(conn_id)
+
+def get_conn(self):
+"""
+Returns kubernetes api session for use with requests
+"""
+if self.connection.extra is None or self.connection.extra == '':
+self.log.info("loading kube_config from: default file")
+else:
+extra_obj = self.connection.extra_dejson
+if "in_cluster" in extra_obj.keys():
+self.log.info("loading kube_config from: in_cluster 
configuration")
+config.load_incluster_config()
+else:
+with tempfile.NamedTemporaryFile() as temp_config:
+self.log.info("loading kube_config from: connection extra 
json")
+temp_config.write(self.connection.extra.encode())
 
 Review comment:
   The configuration must be saved under the dictionary key. You can't pass the 
entire dictionary.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371040841
 
 

 ##
 File path: airflow/contrib/hooks/kubernetes_hook.py
 ##
 @@ -0,0 +1,53 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import tempfile
+
+from kubernetes import client, config
+
+from airflow.hooks.base_hook import BaseHook
+
+
+class Kuberneteshook(BaseHook):
+"""
+creates kubernetes API connection.
+:param conn_id: the connection to Kubernetes cluster
+"""
+
+def __init__(
+self,
+conn_id="kubernetes_default"
+):
+self.connection = self.get_connection(conn_id)
+
+def get_conn(self):
+"""
+Returns kubernetes api session for use with requests
+"""
+if self.connection.extra is None or self.connection.extra == '':
+self.log.info("loading kube_config from: default file")
+else:
+extra_obj = self.connection.extra_dejson
+if "in_cluster" in extra_obj.keys():
 
 Review comment:
   All keys should have a crazy prefix, otherwise it will not be compatible 
with our Web UI. You will not be able to add the edit box in the connection 
edit form. 
   For example:
   GCP:
   https://airflow.readthedocs.io/en/latest/howto/connection/gcp.html - 
`extra__google_cloud_platform__`
   GRPC:
   `extra__grpc__`
   
https://github.com/apache/airflow/blob/master/airflow/contrib/hooks/grpc_hook.py#L123-L134
   You can copy the get_field method if you need them.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371040492
 
 

 ##
 File path: airflow/contrib/example_dags/example_spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,79 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from datetime import timedelta
+
+import yaml
+
+# [START import_module]
+# The DAG object; we'll need this to instantiate a DAG
+from airflow import DAG
+# Operators; we need this to operate!
+from airflow.contrib.operators.spark_kubernetes_operator import 
SparkKubernetesOperator
+from airflow.contrib.sensors.spark_kubernetes_sensor import 
SparkKubernetesSensor
+from airflow.utils.dates import days_ago
+
+# [END import_module]
+
+# [START default_args]
+# These args will get passed on to each operator
+# You can override them on a per-task basis during operator initialization
+default_args = {
+'owner': 'airflow',
+'depends_on_past': False,
+'start_date': days_ago(1),
+'email': ['airf...@example.com'],
+'email_on_failure': False,
+'email_on_retry': False,
+'max_active_runs': 1
+}
+# [END default_args]
+
+# [START instantiate_dag]
+
+# yaml definition of sparkApplication for example use:
+# 
https://raw.githubusercontent.com/GoogleCloudPlatform/spark-on-k8s-operator/master/examples/spark-pi.yaml
+spark_application_path = ""
+with open(spark_application_path) as spark_application_yaml:
 
 Review comment:
   Is there a reason why you didn't use template_ext here?
   
https://github.com/apache/airflow/blob/master/airflow/models/baseoperator.py#L251-L252


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371031062
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,77 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Optional
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+creates sparkapplication object in kubernetes cluster
+
+:param sparkapplication_object: kubernetes custom_resource_definition of 
sparkApplication
+:param namespace: kubernetes namespace to put sparkApplication
+:param kube_config: kubernetes kube_config path
+:param in_cluster: if airflow runs inside kubernetes pod take 
configuration from inside the cluster.
+"""
+
+template_fields = ['sparkapplication_object', 'namespace', 'kube_config']
+template_ext = ()
+ui_color = '#f4a460'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_object: dict,
+ namespace: str = 'default',
+ kube_config: Optional[str] = None,
+ in_cluster: bool = False,
+ *args, **kwargs) -> None:
+super().__init__(*args, **kwargs)
+self.sparkapplication_object = sparkapplication_object
+self.namespace = namespace
+self.kube_config = kube_config
+self.in_cluster = in_cluster
+if kwargs.get('xcom_push') is not None:
+raise AirflowException("'xcom_push' was deprecated, use 
'BaseOperator.do_xcom_push' instead")
+
+def execute(self, context):
+self.log.info("creating sparkApplication")
+hook = Kuberneteshook(
+kube_config=self.kube_config,
+in_cluster=self.in_cluster
+)
+api_client = hook.get_conn()
+api = client.CustomObjectsApi(api_client)
 
 Review comment:
   I think that it is worth giving the option to configure namespace both via 
the operator and via the connection. Look here: 
https://github.com/apache/airflow/blob/43dcc13/airflow/gcp/hooks/base.py#L332-L358
   
   Namespace is not something that is created by the individual Airflow user. 
Namespace should be defined at the organization-level. The organization 
determines which servers it wants to connect to. The operator should specify 
only operational parameters. Sometimes using a different namespace can also be 
an operational activity if DAG performs organization-level activities. But the 
most common cases are the namespace allocation by a specific cluster 
administrator. 


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-26 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r371031062
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,77 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Optional
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+creates sparkapplication object in kubernetes cluster
+
+:param sparkapplication_object: kubernetes custom_resource_definition of 
sparkApplication
+:param namespace: kubernetes namespace to put sparkApplication
+:param kube_config: kubernetes kube_config path
+:param in_cluster: if airflow runs inside kubernetes pod take 
configuration from inside the cluster.
+"""
+
+template_fields = ['sparkapplication_object', 'namespace', 'kube_config']
+template_ext = ()
+ui_color = '#f4a460'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_object: dict,
+ namespace: str = 'default',
+ kube_config: Optional[str] = None,
+ in_cluster: bool = False,
+ *args, **kwargs) -> None:
+super().__init__(*args, **kwargs)
+self.sparkapplication_object = sparkapplication_object
+self.namespace = namespace
+self.kube_config = kube_config
+self.in_cluster = in_cluster
+if kwargs.get('xcom_push') is not None:
+raise AirflowException("'xcom_push' was deprecated, use 
'BaseOperator.do_xcom_push' instead")
+
+def execute(self, context):
+self.log.info("creating sparkApplication")
+hook = Kuberneteshook(
+kube_config=self.kube_config,
+in_cluster=self.in_cluster
+)
+api_client = hook.get_conn()
+api = client.CustomObjectsApi(api_client)
 
 Review comment:
   I think that it is worth giving the option to configure namespace both via 
the operator and via the connection. Look here: 
https://github.com/apache/airflow/blob/43dcc13/airflow/gcp/hooks/base.py#L332-L358
   
   Namespace is not something that is created by the individual Airflow user. 
Namespace should be defined at the organization-level. The organization 
determines which servers it wants to connect to. The operator should specify 
only operational parameters. Sometimes using a different namespace can also be 
an operational activity if DAG performs organization-level activities.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-25 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r370954667
 
 

 ##
 File path: docs/operators-and-hooks-ref.rst
 ##
 @@ -1049,8 +1049,11 @@ These integrations allow you to perform various 
operations using various softwar
* - `Kubernetes `__
  - :doc:`How to use `
  -
+ - :mod:`airflow.contrib.hooks.kubernetes_hook`
 
 Review comment:
   This is a difficult question, because it requires many components to make it 
work. After a long thought, I think it can stay here, but I will ask other 
people. It can be considered as part of the Kubernetes extension, so the 
operator and hook can be in one place.
   CC: @kaxil @nuclearpinguin @potiuk 


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-25 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r370953103
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,75 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Optional
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+creates sparkapplication object in kubernetes cluster
+
+:param sparkapplication_object: kubernetes custom_resource_definition of 
sparkApplication
+:param namespace: kubernetes namespace to put sparkApplication
+:param kube_config: kubernetes kube_config path
+:param in_cluster: if airflow runs inside kubernetes pod take 
configuration from inside the cluster.
 
 Review comment:
   I think `kube_config` and `in_cluster` should be stored in connection and 
not be passed by the user. These values configured by the administrator should 
be constant. 


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-25 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r370952981
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,75 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Optional
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+creates sparkapplication object in kubernetes cluster
 
 Review comment:
   ```suggestion
   Creates Spark Application Object in Kubernetes cluster:
   ```
   Can you also add link to: 
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/api-docs.md#sparkapplication
 ?


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-25 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r370952030
 
 

 ##
 File path: airflow/contrib/hooks/kubernetes_hook.py
 ##
 @@ -0,0 +1,50 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Optional
+
+from kubernetes import client, config
+
+from airflow.hooks.base_hook import BaseHook
+
+
+class Kuberneteshook(BaseHook):
+"""
+creates kubernetes API connection.
+:param kube_config: kubeconfig - kubernetes connection string
+:param in_cluster: if airflow runs inside kubernetes pod take 
configuration from inside the cluster.
+"""
+
+def __init__(
+self,
+kube_config: Optional[str] = None,
+in_cluster: bool = False
+):
+self.kube_config = kube_config
+self.in_cluster = in_cluster
+
+def get_conn(self):
 
 Review comment:
   Can you add typehint here also? 


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-25 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r370951969
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,77 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Optional
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+creates sparkapplication object in kubernetes cluster
+
+:param sparkapplication_object: kubernetes custom_resource_definition of 
sparkApplication
+:param namespace: kubernetes namespace to put sparkApplication
+:param kube_config: kubernetes kube_config path
+:param in_cluster: if airflow runs inside kubernetes pod take 
configuration from inside the cluster.
+"""
+
+template_fields = ['sparkapplication_object', 'namespace', 'kube_config']
+template_ext = ()
+ui_color = '#f4a460'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_object: dict,
+ namespace: str = 'default',
+ kube_config: Optional[str] = None,
+ in_cluster: bool = False,
+ *args, **kwargs) -> None:
+super().__init__(*args, **kwargs)
+self.sparkapplication_object = sparkapplication_object
+self.namespace = namespace
+self.kube_config = kube_config
+self.in_cluster = in_cluster
+if kwargs.get('xcom_push') is not None:
+raise AirflowException("'xcom_push' was deprecated, use 
'BaseOperator.do_xcom_push' instead")
+
+def execute(self, context):
+self.log.info("creating sparkApplication")
+hook = Kuberneteshook(
+kube_config=self.kube_config,
+in_cluster=self.in_cluster
+)
+api_client = hook.get_conn()
+api = client.CustomObjectsApi(api_client)
 
 Review comment:
   Hook should allow reuse of code. This hook ensures the best integration of 
Airflow with an external library.  It is perfectly fine if you need to create 
more methods, but you will not need to create all the methods that are 
available in the client, because not all methods make sense in the case of 
Airflow e.g. no one will perform cluster administrative operations using 
Airflow. If you create a method, you will be able to provide additional unique 
improvements for Airflow, e.g. wait for the correct completion of the 
operation, or provide a default value for some parameters (e.g. project ID or 
namespace). A possible improvement that I can now imagine is storing the 
namespace value in the connection configuration.  A similar improvement exists 
in the integration of Google Cloud Platform - 
https://github.com/apache/airflow/blob/43dcc13/airflow/gcp/hooks/base.py#L332-L358


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-25 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r370950333
 
 

 ##
 File path: airflow/contrib/example_dags/example_spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,128 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from datetime import timedelta
+
+import yaml
+
+# [START import_module]
+# The DAG object; we'll need this to instantiate a DAG
+from airflow import DAG
+# Operators; we need this to operate!
+from airflow.contrib.operators.spark_kubernetes_operator import 
SparkKubernetesOperator
+from airflow.contrib.sensors.spark_kubernetes_sensor import 
SparkKubernetesSensor
+from airflow.utils.dates import days_ago
+
+# [END import_module]
+
+# [START default_args]
+# These args will get passed on to each operator
+# You can override them on a per-task basis during operator initialization
+default_args = {
+'owner': 'airflow',
+'depends_on_past': False,
+'start_date': days_ago(1),
+'email': ['airf...@example.com'],
+'email_on_failure': False,
+'email_on_retry': False,
+'max_active_runs': 1
+}
+# [END default_args]
+
+# [START instantiate_dag]
+spark_application_yaml = """
+#
+# Copyright 2017 Google LLC
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# https://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+apiVersion: "sparkoperator.k8s.io/v1beta2"
+kind: SparkApplication
+metadata:
+  name: spark-pi
+  namespace: default
+spec:
+  type: Scala
+  mode: cluster
+  image: "gcr.io/spark-operator/spark:v2.4.4"
+  imagePullPolicy: Always
+  mainClass: org.apache.spark.examples.SparkPi
+  mainApplicationFile: 
"local:///opt/spark/examples/jars/spark-examples_2.11-2.4.4.jar"
+  sparkVersion: "2.4.4"
+  restartPolicy:
+type: Never
+  volumes:
+- name: "test-volume"
+  hostPath:
+path: "/tmp"
+type: Directory
+  driver:
+cores: 1
+coreLimit: "1200m"
+memory: "512m"
+labels:
+  version: 2.4.4
+serviceAccount: default
+volumeMounts:
+  - name: "test-volume"
+mountPath: "/tmp"
+  executor:
+cores: 1
+instances: 1
+memory: "512m"
+labels:
+  version: 2.4.4
+volumeMounts:
+  - name: "test-volume"
+mountPath: "/tmp"
+"""
 
 Review comment:
   Mixing code from different technologies is very problematic.I remember there 
is a word that describes this problem, but I don't remember the exact name. A 
similar problem occurs with SQL and other languages. SQL can be used in other 
code, but when it is very complex people started using ORM to connect the two 
worlds - SQL and Objects. 
   
   You can create a file in the directory next to the file. For example. If 
your Python file is in the ``airflow/contrib/example_dags`` directory, you can 
keep additional files in the ``airflow/contrib/example_dags`` directory


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-15 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r366816324
 
 

 ##
 File path: airflow/contrib/example_dags/example_spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,128 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from datetime import timedelta
+
+import yaml
+
+# [START import_module]
+# The DAG object; we'll need this to instantiate a DAG
+from airflow import DAG
+# Operators; we need this to operate!
+from airflow.contrib.operators.spark_kubernetes_operator import 
SparkKubernetesOperator
+from airflow.contrib.sensors.spark_kubernetes_sensor import 
SparkKubernetesSensor
+from airflow.utils.dates import days_ago
+
+# [END import_module]
+
+# [START default_args]
+# These args will get passed on to each operator
+# You can override them on a per-task basis during operator initialization
+default_args = {
+'owner': 'airflow',
+'depends_on_past': False,
+'start_date': days_ago(1),
+'email': ['airf...@example.com'],
+'email_on_failure': False,
+'email_on_retry': False,
+'max_active_runs': 1
+}
+# [END default_args]
+
+# [START instantiate_dag]
+spark_application_yaml = """
+#
+# Copyright 2017 Google LLC
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# https://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+apiVersion: "sparkoperator.k8s.io/v1beta2"
+kind: SparkApplication
+metadata:
+  name: spark-pi
+  namespace: default
+spec:
+  type: Scala
+  mode: cluster
+  image: "gcr.io/spark-operator/spark:v2.4.4"
+  imagePullPolicy: Always
+  mainClass: org.apache.spark.examples.SparkPi
+  mainApplicationFile: 
"local:///opt/spark/examples/jars/spark-examples_2.11-2.4.4.jar"
+  sparkVersion: "2.4.4"
+  restartPolicy:
+type: Never
+  volumes:
+- name: "test-volume"
+  hostPath:
+path: "/tmp"
+type: Directory
+  driver:
+cores: 1
+coreLimit: "1200m"
+memory: "512m"
+labels:
+  version: 2.4.4
+serviceAccount: default
+volumeMounts:
+  - name: "test-volume"
+mountPath: "/tmp"
+  executor:
+cores: 1
+instances: 1
+memory: "512m"
+labels:
+  version: 2.4.4
+volumeMounts:
+  - name: "test-volume"
+mountPath: "/tmp"
+"""
 
 Review comment:
   The API of KubernetesPodOperator is being modernized.  
   https://github.com/apache/airflow/pull/6375
   https://github.com/apache/airflow/pull/6230
   You should create this file in the same directory as the python file - 
`example_dag`


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-15 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r366816834
 
 

 ##
 File path: airflow/contrib/sensors/spark_kubernetes_sensor.py
 ##
 @@ -0,0 +1,82 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Dict, Optional
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesSensor(BaseSensorOperator):
+"""
+checks sparkapplication state on kubernetes
+:param sparkapplication_name: sparkapplication resource name
+:param namespace: the kubernetes namespace where the sparkApplication 
reside in
+:param kube_config: kubeconfig file location
+:param in_cluster: if airflow runs inside kubernetes pod take 
configuration from inside the cluster
+"""
+
+template_fields = ('sparkapplication_name', 'namespace', 'kube_config')
+INTERMEDIATE_STATES = ('SUBMITTED', 'RUNNING',)
+FAILURE_STATES = ('FAILED', 'SUBMISSION_FAILED', 'UNKNOWN')
+SUCCESS_STATES = 'COMPLETED'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_name: str,
+ namespace: str = 'default',
+ kube_config: Optional[str] = None,
+ in_cluster: bool = False,
+ *args, **kwargs):
+super().__init__(*args, **kwargs)
+self.sparkapplication_name = sparkapplication_name
+self.namespace = namespace
+self.kube_config = kube_config
+self.in_cluster = in_cluster
+self.api_client = Kuberneteshook(
 
 Review comment:
   Maybe it's worth moving it to the pre_execute method?  The constructor is 
called by every worker repeatedly when each task is started. It really should 
be low cost.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-15 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r366816324
 
 

 ##
 File path: airflow/contrib/example_dags/example_spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,128 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from datetime import timedelta
+
+import yaml
+
+# [START import_module]
+# The DAG object; we'll need this to instantiate a DAG
+from airflow import DAG
+# Operators; we need this to operate!
+from airflow.contrib.operators.spark_kubernetes_operator import 
SparkKubernetesOperator
+from airflow.contrib.sensors.spark_kubernetes_sensor import 
SparkKubernetesSensor
+from airflow.utils.dates import days_ago
+
+# [END import_module]
+
+# [START default_args]
+# These args will get passed on to each operator
+# You can override them on a per-task basis during operator initialization
+default_args = {
+'owner': 'airflow',
+'depends_on_past': False,
+'start_date': days_ago(1),
+'email': ['airf...@example.com'],
+'email_on_failure': False,
+'email_on_retry': False,
+'max_active_runs': 1
+}
+# [END default_args]
+
+# [START instantiate_dag]
+spark_application_yaml = """
+#
+# Copyright 2017 Google LLC
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# https://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+apiVersion: "sparkoperator.k8s.io/v1beta2"
+kind: SparkApplication
+metadata:
+  name: spark-pi
+  namespace: default
+spec:
+  type: Scala
+  mode: cluster
+  image: "gcr.io/spark-operator/spark:v2.4.4"
+  imagePullPolicy: Always
+  mainClass: org.apache.spark.examples.SparkPi
+  mainApplicationFile: 
"local:///opt/spark/examples/jars/spark-examples_2.11-2.4.4.jar"
+  sparkVersion: "2.4.4"
+  restartPolicy:
+type: Never
+  volumes:
+- name: "test-volume"
+  hostPath:
+path: "/tmp"
+type: Directory
+  driver:
+cores: 1
+coreLimit: "1200m"
+memory: "512m"
+labels:
+  version: 2.4.4
+serviceAccount: default
+volumeMounts:
+  - name: "test-volume"
+mountPath: "/tmp"
+  executor:
+cores: 1
+instances: 1
+memory: "512m"
+labels:
+  version: 2.4.4
+volumeMounts:
+  - name: "test-volume"
+mountPath: "/tmp"
+"""
 
 Review comment:
   The API of KubernetesPodOperator is being modernized.  
   https://github.com/apache/airflow/pull/6375
   https://github.com/apache/airflow/pull/6230
   You should create this file in the same directory as the python file - 
`example_dag`


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-14 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r366484195
 
 

 ##
 File path: docs/operators-and-hooks-ref.rst
 ##
 @@ -1049,8 +1049,11 @@ These integrations allow you to perform various 
operations using various softwar
* - `Kubernetes `__
  - :doc:`How to use `
  -
+ - :mod:`airflow.contrib.hooks.kubernetes_hook`
 
 Review comment:
   This is probably not part of Kubernetes, but uses it, so it should be in a 
separate section. A similar situation is with Hadoop and Hive. Hive uses 
Hadoop, but Hive is not Hadoop.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-14 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r366482784
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,77 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Optional
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+creates sparkapplication object in kubernetes cluster
+
+:param sparkapplication_object: kubernetes custom_resource_definition of 
sparkApplication
+:param namespace: kubernetes namespace to put sparkApplication
+:param kube_config: kubernetes kube_config path
+:param in_cluster: if airflow runs inside kubernetes pod take 
configuration from inside the cluster.
+"""
+
+template_fields = ['sparkapplication_object', 'namespace', 'kube_config']
+template_ext = ()
+ui_color = '#f4a460'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_object: dict,
+ namespace: str = 'default',
+ kube_config: Optional[str] = None,
+ in_cluster: bool = False,
+ *args, **kwargs) -> None:
+super().__init__(*args, **kwargs)
+self.sparkapplication_object = sparkapplication_object
+self.namespace = namespace
+self.kube_config = kube_config
+self.in_cluster = in_cluster
+if kwargs.get('xcom_push') is not None:
+raise AirflowException("'xcom_push' was deprecated, use 
'BaseOperator.do_xcom_push' instead")
+
+def execute(self, context):
+self.log.info("creating sparkApplication")
+hook = Kuberneteshook(
+kube_config=self.kube_config,
+in_cluster=self.in_cluster
+)
+api_client = hook.get_conn()
+api = client.CustomObjectsApi(api_client)
 
 Review comment:
   It is not recommended to call methods from external libraries in the 
operator. This is the role of 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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-14 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r366478778
 
 

 ##
 File path: airflow/contrib/sensors/spark_kubernetes_sensor.py
 ##
 @@ -0,0 +1,82 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Dict, Optional
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesSensor(BaseSensorOperator):
+"""
+checks sparkapplication state on kubernetes
+:param sparkapplication_name: sparkapplication resource name
+:param namespace: the kubernetes namespace where the sparkApplication 
reside in
+:param kube_config: kubeconfig file location
+:param in_cluster: if airflow runs inside kubernetes pod take 
configuration from inside the cluster
+"""
+
+template_fields = ('sparkapplication_name', 'namespace', 'kube_config')
+INTERMEDIATE_STATES = ('SUBMITTED', 'RUNNING',)
+FAILURE_STATES = ('FAILED', 'SUBMISSION_FAILED', 'UNKNOWN')
+SUCCESS_STATES = 'COMPLETED'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_name: str,
+ namespace: str = 'default',
+ kube_config: Optional[str] = None,
+ in_cluster: bool = False,
+ *args, **kwargs):
+super().__init__(*args, **kwargs)
+self.sparkapplication_name = sparkapplication_name
+self.namespace = namespace
+self.kube_config = kube_config
+self.in_cluster = in_cluster
+self.api_client = Kuberneteshook(
 
 Review comment:
   Hook should not be initialized in the constructor but in the execute method. 
The constructor should be as light as possible, because this causes unnecessary 
load. DAG is loaded many times in various components, and the hook is needed 
only when executing the execute method.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-14 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r366477385
 
 

 ##
 File path: airflow/contrib/operators/spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,77 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Optional
+
+from kubernetes import client
+
+from airflow.contrib.hooks.kubernetes_hook import Kuberneteshook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class SparkKubernetesOperator(BaseOperator):
+"""
+creates sparkapplication object in kubernetes cluster
+
+:param sparkapplication_object: kubernetes custom_resource_definition of 
sparkApplication
+:param namespace: kubernetes namespace to put sparkApplication
+:param kube_config: kubernetes kube_config path
+:param in_cluster: if airflow runs inside kubernetes pod take 
configuration from inside the cluster.
+"""
+
+template_fields = ['sparkapplication_object', 'namespace', 'kube_config']
+template_ext = ()
+ui_color = '#f4a460'
+
+@apply_defaults
+def __init__(self,
+ sparkapplication_object: dict,
+ namespace: str = 'default',
+ kube_config: Optional[str] = None,
+ in_cluster: bool = False,
+ *args, **kwargs) -> None:
+super().__init__(*args, **kwargs)
+self.sparkapplication_object = sparkapplication_object
+self.namespace = namespace
+self.kube_config = kube_config
+self.in_cluster = in_cluster
+if kwargs.get('xcom_push') is not None:
 
 Review comment:
   ```suggestion
   ```
   It seems to me that it is not needed. Am I right? 


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add spark-on-k8s operator/hook/sensor

2020-01-14 Thread GitBox
mik-laj commented on a change in pull request #7163: [AIRFLOW-6542] add 
spark-on-k8s operator/hook/sensor
URL: https://github.com/apache/airflow/pull/7163#discussion_r366476870
 
 

 ##
 File path: airflow/contrib/example_dags/example_spark_kubernetes_operator.py
 ##
 @@ -0,0 +1,128 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from datetime import timedelta
+
+import yaml
+
+# [START import_module]
+# The DAG object; we'll need this to instantiate a DAG
+from airflow import DAG
+# Operators; we need this to operate!
+from airflow.contrib.operators.spark_kubernetes_operator import 
SparkKubernetesOperator
+from airflow.contrib.sensors.spark_kubernetes_sensor import 
SparkKubernetesSensor
+from airflow.utils.dates import days_ago
+
+# [END import_module]
+
+# [START default_args]
+# These args will get passed on to each operator
+# You can override them on a per-task basis during operator initialization
+default_args = {
+'owner': 'airflow',
+'depends_on_past': False,
+'start_date': days_ago(1),
+'email': ['airf...@example.com'],
+'email_on_failure': False,
+'email_on_retry': False,
+'max_active_runs': 1
+}
+# [END default_args]
+
+# [START instantiate_dag]
+spark_application_yaml = """
+#
+# Copyright 2017 Google LLC
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# https://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+apiVersion: "sparkoperator.k8s.io/v1beta2"
+kind: SparkApplication
+metadata:
+  name: spark-pi
+  namespace: default
+spec:
+  type: Scala
+  mode: cluster
+  image: "gcr.io/spark-operator/spark:v2.4.4"
+  imagePullPolicy: Always
+  mainClass: org.apache.spark.examples.SparkPi
+  mainApplicationFile: 
"local:///opt/spark/examples/jars/spark-examples_2.11-2.4.4.jar"
+  sparkVersion: "2.4.4"
+  restartPolicy:
+type: Never
+  volumes:
+- name: "test-volume"
+  hostPath:
+path: "/tmp"
+type: Directory
+  driver:
+cores: 1
+coreLimit: "1200m"
+memory: "512m"
+labels:
+  version: 2.4.4
+serviceAccount: default
+volumeMounts:
+  - name: "test-volume"
+mountPath: "/tmp"
+  executor:
+cores: 1
+instances: 1
+memory: "512m"
+labels:
+  version: 2.4.4
+volumeMounts:
+  - name: "test-volume"
+mountPath: "/tmp"
+"""
 
 Review comment:
   Can you store content inthe file and `template_ext`, please? DAG should be 
simple. 
   
https://airflow.readthedocs.io/en/latest/_api/airflow/models/baseoperator/index.html#airflow.models.baseoperator.BaseOperator.template_ext


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:
us...@infra.apache.org


With regards,
Apache Git Services