Skip to content

Commit

Permalink
[AIRFLOW-6790] Add basic Tableau Integration (#7410)
Browse files Browse the repository at this point in the history
  • Loading branch information
feluelle authored Feb 22, 2020
1 parent d8801f4 commit a9ad0a9
Show file tree
Hide file tree
Showing 17 changed files with 684 additions and 8 deletions.
2 changes: 2 additions & 0 deletions airflow/models/connection.py
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@
"presto": ("airflow.providers.presto.hooks.presto.PrestoHook", "presto_conn_id"),
"redis": ("airflow.providers.redis.hooks.redis.RedisHook", "redis_conn_id"),
"sqlite": ("airflow.providers.sqlite.hooks.sqlite.SqliteHook", "sqlite_conn_id"),
"tableau": ("airflow.providers.salesforce.hooks.tableau.TableauHook", "tableau_conn_id"),
"vertica": ("airflow.providers.vertica.hooks.vertica.VerticaHook", "vertica_conn_id"),
"wasb": ("airflow.providers.microsoft.azure.hooks.wasb.WasbHook", "wasb_conn_id"),
}
Expand Down Expand Up @@ -159,6 +160,7 @@ class Connection(Base, LoggingMixin):
('grpc', 'GRPC Connection'),
('yandexcloud', 'Yandex Cloud'),
('livy', 'Apache Livy'),
('tableau', 'Tableau'),
]

def __init__(
Expand Down
16 changes: 16 additions & 0 deletions airflow/providers/salesforce/example_dags/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# 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.
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
#
# 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.
"""
This is an example dag that performs two refresh operations on a Tableau Workbook aka Extract. The first one
waits until it succeeds. The second does not wait since this is an asynchronous operation and we don't know
when the operation actually finishes. That's why we have another task that checks only that.
"""
from datetime import timedelta

from airflow import DAG
from airflow.providers.salesforce.operators.tableau_refresh_workbook import TableauRefreshWorkbookOperator
from airflow.providers.salesforce.sensors.tableau_job_status import TableauJobStatusSensor
from airflow.utils.dates import days_ago

DEFAULT_ARGS = {
'owner': 'airflow',
'depends_on_past': False,
'start_date': days_ago(2),
'email': ['airflow@example.com'],
'email_on_failure': False,
'email_on_retry': False
}

with DAG(
dag_id='example_tableau_refresh_workbook',
default_args=DEFAULT_ARGS,
dagrun_timeout=timedelta(hours=2),
schedule_interval=None,
tags=['example'],
) as dag:
# Refreshes a workbook and waits until it succeeds.
task_refresh_workbook_blocking = TableauRefreshWorkbookOperator(
site_id='my_site',
workbook_name='MyWorkbook',
blocking=True,
task_id='refresh_tableau_workbook_blocking',
)
# Refreshes a workbook and does not wait until it succeeds.
task_refresh_workbook_non_blocking = TableauRefreshWorkbookOperator(
site_id='my_site',
workbook_name='MyWorkbook',
blocking=False,
task_id='refresh_tableau_workbook_non_blocking',
)
# The following task queries the status of the workbook refresh job until it succeeds.
task_check_job_status = TableauJobStatusSensor(
site_id='my_site',
job_id="{{ ti.xcom_pull(task_ids='refresh_tableau_workbook_non_blocking') }}",
task_id='check_tableau_job_status',
)
task_refresh_workbook_non_blocking >> task_check_job_status
110 changes: 110 additions & 0 deletions airflow/providers/salesforce/hooks/tableau.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
# 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 enum import Enum
from typing import Optional

from tableauserverclient import Pager, PersonalAccessTokenAuth, Server, TableauAuth
from tableauserverclient.server import Auth

from airflow.hooks.base_hook import BaseHook


class TableauJobFinishCode(Enum):
"""
The finish code indicates the status of the job.
.. seealso:: https://help.tableau.com/current/api/rest_api/en-us/REST/rest_api_ref.htm#query_job
"""
PENDING = -1
SUCCESS = 0
ERROR = 1
CANCELED = 2


class TableauHook(BaseHook):
"""
Connects to the Tableau Server Instance and allows to communicate with it.
.. seealso:: https://tableau.github.io/server-client-python/docs/
:param site_id: The id of the site where the workbook belongs to.
It will connect to the default site if you don't provide an id.
:type site_id: Optional[str]
:param tableau_conn_id: The Tableau Connection id containing the credentials
to authenticate to the Tableau Server.
:type tableau_conn_id: str
"""

def __init__(self, site_id: Optional[str] = None, tableau_conn_id: str = 'tableau_default'):
self.tableau_conn_id = tableau_conn_id
self.conn = self.get_connection(self.tableau_conn_id)
self.site_id = site_id or self.conn.extra_dejson.get('site_id', '')
self.server = Server(self.conn.host, use_server_version=True)
self.tableau_conn = None

def __enter__(self):
if not self.tableau_conn:
self.tableau_conn = self.get_conn()
return self

def __exit__(self, exc_type, exc_val, exc_tb):
self.server.auth.sign_out()

def get_conn(self) -> Auth.contextmgr:
"""
Signs in to the Tableau Server and automatically signs out if used as ContextManager.
:return: an authorized Tableau Server Context Manager object.
:rtype: tableauserverclient.server.Auth.contextmgr
"""
if self.conn.login and self.conn.password:
return self._auth_via_password()
if 'token_name' in self.conn.extra_dejson and 'personal_access_token' in self.conn.extra_dejson:
return self._auth_via_token()
raise NotImplementedError('No Authentication method found for given Credentials!')

def _auth_via_password(self) -> Auth.contextmgr:
tableau_auth = TableauAuth(
username=self.conn.login,
password=self.conn.password,
site_id=self.site_id
)
return self.server.auth.sign_in(tableau_auth)

def _auth_via_token(self) -> Auth.contextmgr:
tableau_auth = PersonalAccessTokenAuth(
token_name=self.conn.extra_dejson['token_name'],
personal_access_token=self.conn.extra_dejson['personal_access_token'],
site_id=self.site_id
)
return self.server.auth.sign_in_with_personal_access_token(tableau_auth)

def get_all(self, resource_name: str) -> Pager:
"""
Get all items of the given resource.
.. seealso:: https://tableau.github.io/server-client-python/docs/page-through-results
:param resource_name: The name of the resource to paginate.
For example: jobs or workbooks
:type resource_name: str
:return: all items by returning a Pager.
:rtype: tableauserverclient.Pager
"""
resource = getattr(self.server, resource_name)
return Pager(resource.get)
16 changes: 16 additions & 0 deletions airflow/providers/salesforce/operators/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# 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.
94 changes: 94 additions & 0 deletions airflow/providers/salesforce/operators/tableau_refresh_workbook.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,94 @@
# 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 tableauserverclient import WorkbookItem

from airflow import AirflowException
from airflow.models import BaseOperator
from airflow.providers.salesforce.hooks.tableau import TableauHook
from airflow.utils.decorators import apply_defaults


class TableauRefreshWorkbookOperator(BaseOperator):
"""
Refreshes a Tableau Workbook/Extract
.. seealso:: https://tableau.github.io/server-client-python/docs/api-ref#workbooks
:param workbook_name: The name of the workbook to refresh.
:type workbook_name: str
:param site_id: The id of the site where the workbook belongs to.
:type site_id: Optional[str]
:param blocking: By default the extract refresh will be blocking means it will wait until it has finished.
:type blocking: bool
:param tableau_conn_id: The Tableau Connection id containing the credentials
to authenticate to the Tableau Server.
:type tableau_conn_id: str
"""

@apply_defaults
def __init__(self,
workbook_name: str,
site_id: Optional[str] = None,
blocking: bool = True,
tableau_conn_id: str = 'tableau_default',
*args,
**kwargs):
super().__init__(*args, **kwargs)
self.workbook_name = workbook_name
self.site_id = site_id
self.blocking = blocking
self.tableau_conn_id = tableau_conn_id

def execute(self, context: dict) -> str:
"""
Executes the Tableau Extract Refresh and pushes the job id to xcom.
:param context: The task context during execution.
:type context: dict
:return: the id of the job that executes the extract refresh
:rtype: str
"""
with TableauHook(self.site_id, self.tableau_conn_id) as tableau_hook:
workbook = self._get_workbook_by_name(tableau_hook)

job_id = self._refresh_workbook(tableau_hook, workbook.id)
if self.blocking:
from airflow.providers.salesforce.sensors.tableau_job_status import TableauJobStatusSensor
TableauJobStatusSensor(
job_id=job_id,
site_id=self.site_id,
tableau_conn_id=self.tableau_conn_id,
task_id='wait_until_succeeded',
dag=None
).execute(context={})
self.log.info('Workbook %s has been successfully refreshed.', self.workbook_name)
return job_id

def _get_workbook_by_name(self, tableau_hook: TableauHook) -> WorkbookItem:
for workbook in tableau_hook.get_all(resource_name='workbooks'):
if workbook.name == self.workbook_name:
self.log.info('Found matching workbook with id %s', workbook.id)
return workbook

raise AirflowException(f'Workbook {self.workbook_name} not found!')

def _refresh_workbook(self, tableau_hook: TableauHook, workbook_id: str) -> str:
job = tableau_hook.server.workbooks.refresh(workbook_id)
self.log.info('Refreshing Workbook %s...', self.workbook_name)
return job.id
16 changes: 16 additions & 0 deletions airflow/providers/salesforce/sensors/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# 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.
Loading

0 comments on commit a9ad0a9

Please sign in to comment.