# -*- 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 __future__ import print_function, unicode_literals
import collections
import contextlib
import os
import random
import re
import subprocess
import time
import socket
from collections import OrderedDict
from tempfile import NamedTemporaryFile
import six
import unicodecsv as csv
from past.builtins import basestring
from past.builtins import unicode
from six.moves import zip
import airflow.security.utils as utils
from airflow.configuration import conf
from airflow.exceptions import AirflowException
from airflow.hooks.base_hook import BaseHook
from airflow.utils.file import TemporaryDirectory
from airflow.utils.helpers import as_flattened_list
from airflow.utils.operator_helpers import AIRFLOW_VAR_NAME_FORMAT_MAPPING
[docs]HIVE_QUEUE_PRIORITIES = ['VERY_HIGH', 'HIGH', 'NORMAL', 'LOW', 'VERY_LOW']
[docs]def get_context_from_env_var():
"""
Extract context from env variable, e.g. dag_id, task_id and execution_date,
so that they can be used inside BashOperator and PythonOperator.
:return: The context of interest.
"""
return {format_map['default']: os.environ.get(format_map['env_var_format'], '')
for format_map in AIRFLOW_VAR_NAME_FORMAT_MAPPING.values()}
[docs]class HiveCliHook(BaseHook):
"""Simple wrapper around the hive CLI.
It also supports the ``beeline``
a lighter CLI that runs JDBC and is replacing the heavier
traditional CLI. To enable ``beeline``, set the use_beeline param in the
extra field of your connection as in ``{ "use_beeline": true }``
Note that you can also set default hive CLI parameters using the
``hive_cli_params`` to be used in your connection as in
``{"hive_cli_params": "-hiveconf mapred.job.tracker=some.jobtracker:444"}``
Parameters passed here can be overridden by run_cli's hive_conf param
The extra connection parameter ``auth`` gets passed as in the ``jdbc``
connection string as is.
:param mapred_queue: queue used by the Hadoop Scheduler (Capacity or Fair)
:type mapred_queue: str
:param mapred_queue_priority: priority within the job queue.
Possible settings include: VERY_HIGH, HIGH, NORMAL, LOW, VERY_LOW
:type mapred_queue_priority: str
:param mapred_job_name: This name will appear in the jobtracker.
This can make monitoring easier.
:type mapred_job_name: str
"""
def __init__(
self,
hive_cli_conn_id="hive_cli_default",
run_as=None,
mapred_queue=None,
mapred_queue_priority=None,
mapred_job_name=None):
conn = self.get_connection(hive_cli_conn_id)
self.hive_cli_params = conn.extra_dejson.get('hive_cli_params', '')
self.use_beeline = conn.extra_dejson.get('use_beeline', False)
self.auth = conn.extra_dejson.get('auth', 'noSasl')
self.conn = conn
self.run_as = run_as
if mapred_queue_priority:
mapred_queue_priority = mapred_queue_priority.upper()
if mapred_queue_priority not in HIVE_QUEUE_PRIORITIES:
raise AirflowException(
"Invalid Mapred Queue Priority. Valid values are: "
"{}".format(', '.join(HIVE_QUEUE_PRIORITIES)))
self.mapred_queue = mapred_queue or conf.get('hive',
'default_hive_mapred_queue')
self.mapred_queue_priority = mapred_queue_priority
self.mapred_job_name = mapred_job_name
[docs] def _get_proxy_user(self):
"""
This function set the proper proxy_user value in case the user overwtire the default.
"""
conn = self.conn
proxy_user_value = conn.extra_dejson.get('proxy_user', "")
if proxy_user_value == "login" and conn.login:
return "hive.server2.proxy.user={0}".format(conn.login)
if proxy_user_value == "owner" and self.run_as:
return "hive.server2.proxy.user={0}".format(self.run_as)
if proxy_user_value != "": # There is a custom proxy user
return "hive.server2.proxy.user={0}".format(proxy_user_value)
return proxy_user_value # The default proxy user (undefined)
[docs] def _prepare_cli_cmd(self):
"""
This function creates the command list from available information
"""
conn = self.conn
hive_bin = 'hive'
cmd_extra = []
if self.use_beeline:
hive_bin = 'beeline'
jdbc_url = "jdbc:hive2://{host}:{port}/{schema}".format(
host=conn.host, port=conn.port, schema=conn.schema)
if conf.get('core', 'security') == 'kerberos':
template = conn.extra_dejson.get(
'principal', "hive/_HOST@EXAMPLE.COM")
if "_HOST" in template:
template = utils.replace_hostname_pattern(
utils.get_components(template))
proxy_user = self._get_proxy_user()
jdbc_url += ";principal={template};{proxy_user}".format(
template=template, proxy_user=proxy_user)
elif self.auth:
jdbc_url += ";auth=" + self.auth
jdbc_url = '"{}"'.format(jdbc_url)
cmd_extra += ['-u', jdbc_url]
if conn.login:
cmd_extra += ['-n', conn.login]
if conn.password:
cmd_extra += ['-p', conn.password]
hive_params_list = self.hive_cli_params.split()
return [hive_bin] + cmd_extra + hive_params_list
@staticmethod
[docs] def _prepare_hiveconf(d):
"""
This function prepares a list of hiveconf params
from a dictionary of key value pairs.
:param d:
:type d: dict
>>> hh = HiveCliHook()
>>> hive_conf = {"hive.exec.dynamic.partition": "true",
... "hive.exec.dynamic.partition.mode": "nonstrict"}
>>> hh._prepare_hiveconf(hive_conf)
["-hiveconf", "hive.exec.dynamic.partition=true",\
"-hiveconf", "hive.exec.dynamic.partition.mode=nonstrict"]
"""
if not d:
return []
return as_flattened_list(
zip(["-hiveconf"] * len(d),
["{}={}".format(k, v) for k, v in collections.OrderedDict(sorted(d.items())).items()])
)
[docs] def run_cli(self, hql, schema=None, verbose=True, hive_conf=None):
"""
Run an hql statement using the hive cli. If hive_conf is specified
it should be a dict and the entries will be set as key/value pairs
in HiveConf
:param hive_conf: if specified these key value pairs will be passed
to hive as ``-hiveconf "key"="value"``. Note that they will be
passed after the ``hive_cli_params`` and thus will override
whatever values are specified in the database.
:type hive_conf: dict
>>> hh = HiveCliHook()
>>> result = hh.run_cli("USE airflow;")
>>> ("OK" in result)
True
"""
conn = self.conn
schema = schema or conn.schema
if schema:
hql = "USE {schema};\n{hql}".format(schema=schema, hql=hql)
with TemporaryDirectory(prefix='airflow_hiveop_') as tmp_dir:
with NamedTemporaryFile(dir=tmp_dir) as f:
hql = hql + '\n'
f.write(hql.encode('UTF-8'))
f.flush()
hive_cmd = self._prepare_cli_cmd()
env_context = get_context_from_env_var()
# Only extend the hive_conf if it is defined.
if hive_conf:
env_context.update(hive_conf)
hive_conf_params = self._prepare_hiveconf(env_context)
if self.mapred_queue:
hive_conf_params.extend(
['-hiveconf',
'mapreduce.job.queuename={}'
.format(self.mapred_queue),
'-hiveconf',
'mapred.job.queue.name={}'
.format(self.mapred_queue),
'-hiveconf',
'tez.queue.name={}'
.format(self.mapred_queue)
])
if self.mapred_queue_priority:
hive_conf_params.extend(
['-hiveconf',
'mapreduce.job.priority={}'
.format(self.mapred_queue_priority)])
if self.mapred_job_name:
hive_conf_params.extend(
['-hiveconf',
'mapred.job.name={}'
.format(self.mapred_job_name)])
hive_cmd.extend(hive_conf_params)
hive_cmd.extend(['-f', f.name])
if verbose:
self.log.info("%s", " ".join(hive_cmd))
sp = subprocess.Popen(
hive_cmd,
stdout=subprocess.PIPE,
stderr=subprocess.STDOUT,
cwd=tmp_dir,
close_fds=True)
self.sp = sp
stdout = ''
while True:
line = sp.stdout.readline()
if not line:
break
stdout += line.decode('UTF-8')
if verbose:
self.log.info(line.decode('UTF-8').strip())
sp.wait()
if sp.returncode:
raise AirflowException(stdout)
return stdout
[docs] def test_hql(self, hql):
"""
Test an hql statement using the hive cli and EXPLAIN
"""
create, insert, other = [], [], []
for query in hql.split(';'): # naive
query_original = query
query = query.lower().strip()
if query.startswith('create table'):
create.append(query_original)
elif query.startswith(('set ',
'add jar ',
'create temporary function')):
other.append(query_original)
elif query.startswith('insert'):
insert.append(query_original)
other = ';'.join(other)
for query_set in [create, insert]:
for query in query_set:
query_preview = ' '.join(query.split())[:50]
self.log.info("Testing HQL [%s (...)]", query_preview)
if query_set == insert:
query = other + '; explain ' + query
else:
query = 'explain ' + query
try:
self.run_cli(query, verbose=False)
except AirflowException as e:
message = e.args[0].split('\n')[-2]
self.log.info(message)
error_loc = re.search(r'(\d+):(\d+)', message)
if error_loc and error_loc.group(1).isdigit():
lst = int(error_loc.group(1))
begin = max(lst - 2, 0)
end = min(lst + 3, len(query.split('\n')))
context = '\n'.join(query.split('\n')[begin:end])
self.log.info("Context :\n %s", context)
else:
self.log.info("SUCCESS")
[docs] def load_df(
self,
df,
table,
field_dict=None,
delimiter=',',
encoding='utf8',
pandas_kwargs=None, **kwargs):
"""
Loads a pandas DataFrame into hive.
Hive data types will be inferred if not passed but column names will
not be sanitized.
:param df: DataFrame to load into a Hive table
:type df: pandas.DataFrame
:param table: target Hive table, use dot notation to target a
specific database
:type table: str
:param field_dict: mapping from column name to hive data type.
Note that it must be OrderedDict so as to keep columns' order.
:type field_dict: collections.OrderedDict
:param delimiter: field delimiter in the file
:type delimiter: str
:param encoding: str encoding to use when writing DataFrame to file
:type encoding: str
:param pandas_kwargs: passed to DataFrame.to_csv
:type pandas_kwargs: dict
:param kwargs: passed to self.load_file
"""
def _infer_field_types_from_df(df):
DTYPE_KIND_HIVE_TYPE = {
'b': 'BOOLEAN', # boolean
'i': 'BIGINT', # signed integer
'u': 'BIGINT', # unsigned integer
'f': 'DOUBLE', # floating-point
'c': 'STRING', # complex floating-point
'M': 'TIMESTAMP', # datetime
'O': 'STRING', # object
'S': 'STRING', # (byte-)string
'U': 'STRING', # Unicode
'V': 'STRING' # void
}
d = OrderedDict()
for col, dtype in df.dtypes.iteritems():
d[col] = DTYPE_KIND_HIVE_TYPE[dtype.kind]
return d
if pandas_kwargs is None:
pandas_kwargs = {}
with TemporaryDirectory(prefix='airflow_hiveop_') as tmp_dir:
with NamedTemporaryFile(dir=tmp_dir, mode="w") as f:
if field_dict is None:
field_dict = _infer_field_types_from_df(df)
df.to_csv(path_or_buf=f,
sep=(delimiter.encode(encoding)
if six.PY2 and isinstance(delimiter, unicode)
else delimiter),
header=False,
index=False,
encoding=encoding,
date_format="%Y-%m-%d %H:%M:%S",
**pandas_kwargs)
f.flush()
return self.load_file(filepath=f.name,
table=table,
delimiter=delimiter,
field_dict=field_dict,
**kwargs)
[docs] def load_file(
self,
filepath,
table,
delimiter=",",
field_dict=None,
create=True,
overwrite=True,
partition=None,
recreate=False,
tblproperties=None):
"""
Loads a local file into Hive
Note that the table generated in Hive uses ``STORED AS textfile``
which isn't the most efficient serialization format. If a
large amount of data is loaded and/or if the tables gets
queried considerably, you may want to use this operator only to
stage the data into a temporary table before loading it into its
final destination using a ``HiveOperator``.
:param filepath: local filepath of the file to load
:type filepath: str
:param table: target Hive table, use dot notation to target a
specific database
:type table: str
:param delimiter: field delimiter in the file
:type delimiter: str
:param field_dict: A dictionary of the fields name in the file
as keys and their Hive types as values.
Note that it must be OrderedDict so as to keep columns' order.
:type field_dict: collections.OrderedDict
:param create: whether to create the table if it doesn't exist
:type create: bool
:param overwrite: whether to overwrite the data in table or partition
:type overwrite: bool
:param partition: target partition as a dict of partition columns
and values
:type partition: dict
:param recreate: whether to drop and recreate the table at every
execution
:type recreate: bool
:param tblproperties: TBLPROPERTIES of the hive table being created
:type tblproperties: dict
"""
hql = ''
if recreate:
hql += "DROP TABLE IF EXISTS {table};\n".format(table=table)
if create or recreate:
if field_dict is None:
raise ValueError("Must provide a field dict when creating a table")
fields = ",\n ".join(
['`{k}` {v}'.format(k=k.strip('`'), v=v) for k, v in field_dict.items()])
hql += "CREATE TABLE IF NOT EXISTS {table} (\n{fields})\n".format(
table=table, fields=fields)
if partition:
pfields = ",\n ".join(
[p + " STRING" for p in partition])
hql += "PARTITIONED BY ({pfields})\n".format(pfields=pfields)
hql += "ROW FORMAT DELIMITED\n"
hql += "FIELDS TERMINATED BY '{delimiter}'\n".format(delimiter=delimiter)
hql += "STORED AS textfile\n"
if tblproperties is not None:
tprops = ", ".join(
["'{0}'='{1}'".format(k, v) for k, v in tblproperties.items()])
hql += "TBLPROPERTIES({tprops})\n".format(tprops=tprops)
hql += ";"
self.log.info(hql)
self.run_cli(hql)
hql = "LOAD DATA LOCAL INPATH '{filepath}' ".format(filepath=filepath)
if overwrite:
hql += "OVERWRITE "
hql += "INTO TABLE {table} ".format(table=table)
if partition:
pvals = ", ".join(
["{0}='{1}'".format(k, v) for k, v in partition.items()])
hql += "PARTITION ({pvals})".format(pvals=pvals)
# As a workaround for HIVE-10541, add a newline character
# at the end of hql (AIRFLOW-2412).
hql += ';\n'
self.log.info(hql)
self.run_cli(hql)
[docs] def kill(self):
if hasattr(self, 'sp'):
if self.sp.poll() is None:
print("Killing the Hive job")
self.sp.terminate()
time.sleep(60)
self.sp.kill()
[docs]class HiveServer2Hook(BaseHook):
"""
Wrapper around the pyhive library
Notes:
* the default authMechanism is PLAIN, to override it you
can specify it in the ``extra`` of your connection in the UI
* the default for run_set_variable_statements is true, if you
are using impala you may need to set it to false in the
``extra`` of your connection in the UI
"""
def __init__(self, hiveserver2_conn_id='hiveserver2_default'):
self.hiveserver2_conn_id = hiveserver2_conn_id
[docs] def get_conn(self, schema=None):
"""
Returns a Hive connection object.
"""
db = self.get_connection(self.hiveserver2_conn_id)
auth_mechanism = db.extra_dejson.get('authMechanism', 'NONE')
if auth_mechanism == 'NONE' and db.login is None:
# we need to give a username
username = 'airflow'
kerberos_service_name = None
if conf.get('core', 'security') == 'kerberos':
auth_mechanism = db.extra_dejson.get('authMechanism', 'KERBEROS')
kerberos_service_name = db.extra_dejson.get('kerberos_service_name', 'hive')
# pyhive uses GSSAPI instead of KERBEROS as a auth_mechanism identifier
if auth_mechanism == 'GSSAPI':
self.log.warning(
"Detected deprecated 'GSSAPI' for authMechanism "
"for %s. Please use 'KERBEROS' instead",
self.hiveserver2_conn_id
)
auth_mechanism = 'KERBEROS'
from pyhive.hive import connect
return connect(
host=db.host,
port=db.port,
auth=auth_mechanism,
kerberos_service_name=kerberos_service_name,
username=db.login or username,
password=db.password,
database=schema or db.schema or 'default')
[docs] def _get_results(self, hql, schema='default', fetch_size=None, hive_conf=None):
from pyhive.exc import ProgrammingError
if isinstance(hql, basestring):
hql = [hql]
previous_description = None
with contextlib.closing(self.get_conn(schema)) as conn, \
contextlib.closing(conn.cursor()) as cur:
cur.arraysize = fetch_size or 1000
# not all query services (e.g. impala AIRFLOW-4434) support the set command
db = self.get_connection(self.hiveserver2_conn_id)
if db.extra_dejson.get('run_set_variable_statements', True):
env_context = get_context_from_env_var()
if hive_conf:
env_context.update(hive_conf)
for k, v in env_context.items():
cur.execute("set {}={}".format(k, v))
for statement in hql:
cur.execute(statement)
# we only get results of statements that returns
lowered_statement = statement.lower().strip()
if (lowered_statement.startswith('select') or
lowered_statement.startswith('with') or
(lowered_statement.startswith('set') and
'=' not in lowered_statement)):
description = [c for c in cur.description]
if previous_description and previous_description != description:
message = '''The statements are producing different descriptions:
Current: {}
Previous: {}'''.format(repr(description),
repr(previous_description))
raise ValueError(message)
elif not previous_description:
previous_description = description
yield description
try:
# DB API 2 raises when no results are returned
# we're silencing here as some statements in the list
# may be `SET` or DDL
for row in cur:
yield row
except ProgrammingError:
self.log.debug("get_results returned no records")
[docs] def get_results(self, hql, schema='default', fetch_size=None, hive_conf=None):
"""
Get results of the provided hql in target schema.
:param hql: hql to be executed.
:type hql: str or list
:param schema: target schema, default to 'default'.
:type schema: str
:param fetch_size: max size of result to fetch.
:type fetch_size: int
:param hive_conf: hive_conf to execute alone with the hql.
:type hive_conf: dict
:return: results of hql execution, dict with data (list of results) and header
:rtype: dict
"""
results_iter = self._get_results(hql, schema,
fetch_size=fetch_size, hive_conf=hive_conf)
header = next(results_iter)
results = {
'data': list(results_iter),
'header': header
}
return results
[docs] def to_csv(
self,
hql,
csv_filepath,
schema='default',
delimiter=',',
lineterminator='\r\n',
output_header=True,
fetch_size=1000,
hive_conf=None):
"""
Execute hql in target schema and write results to a csv file.
:param hql: hql to be executed.
:type hql: str or list
:param csv_filepath: filepath of csv to write results into.
:type csv_filepath: str
:param schema: target schema, default to 'default'.
:type schema: str
:param delimiter: delimiter of the csv file, default to ','.
:type delimiter: str
:param lineterminator: lineterminator of the csv file.
:type lineterminator: str
:param output_header: header of the csv file, default to True.
:type output_header: bool
:param fetch_size: number of result rows to write into the csv file, default to 1000.
:type fetch_size: int
:param hive_conf: hive_conf to execute alone with the hql.
:type hive_conf: dict
"""
results_iter = self._get_results(hql, schema,
fetch_size=fetch_size, hive_conf=hive_conf)
header = next(results_iter)
message = None
i = 0
with open(csv_filepath, 'wb') as f:
writer = csv.writer(f,
delimiter=delimiter,
lineterminator=lineterminator,
encoding='utf-8')
try:
if output_header:
self.log.debug('Cursor description is %s', header)
writer.writerow([c[0] for c in header])
for i, row in enumerate(results_iter, 1):
writer.writerow(row)
if i % fetch_size == 0:
self.log.info("Written %s rows so far.", i)
except ValueError as exception:
message = str(exception)
if message:
# need to clean up the file first
os.remove(csv_filepath)
raise ValueError(message)
self.log.info("Done. Loaded a total of %s rows.", i)
[docs] def get_records(self, hql, schema='default'):
"""
Get a set of records from a Hive query.
:param hql: hql to be executed.
:type hql: str or list
:param schema: target schema, default to 'default'.
:type schema: str
:param hive_conf: hive_conf to execute alone with the hql.
:type hive_conf: dict
:return: result of hive execution
:rtype: list
>>> hh = HiveServer2Hook()
>>> sql = "SELECT * FROM airflow.static_babynames LIMIT 100"
>>> len(hh.get_records(sql))
100
"""
return self.get_results(hql, schema=schema)['data']
[docs] def get_pandas_df(self, hql, schema='default'):
"""
Get a pandas dataframe from a Hive query
:param hql: hql to be executed.
:type hql: str or list
:param schema: target schema, default to 'default'.
:type schema: str
:return: result of hql execution
:rtype: DataFrame
>>> hh = HiveServer2Hook()
>>> sql = "SELECT * FROM airflow.static_babynames LIMIT 100"
>>> df = hh.get_pandas_df(sql)
>>> len(df.index)
100
:return: pandas.DateFrame
"""
import pandas as pd
res = self.get_results(hql, schema=schema)
df = pd.DataFrame(res['data'])
df.columns = [c[0] for c in res['header']]
return df